All of lore.kernel.org
 help / color / mirror / Atom feed
* [RFC 0/5] virtio support for container
@ 2015-11-05 18:31 Jianfeng Tan
  2015-11-05 18:31 ` [RFC 1/5] virtio/container: add handler for ioport rd/wr Jianfeng Tan
                   ` (14 more replies)
  0 siblings, 15 replies; 196+ messages in thread
From: Jianfeng Tan @ 2015-11-05 18:31 UTC (permalink / raw)
  To: dev
  Cc: nakajima.yoshihiro, zhbzg, mst, gaoxiaoqiu, oscar.zhangbo,
	ann.zhuangyanying, zhoujingbin, guohongzhen

This patchset only acts as a PoC to request the community for comments.
 
This patchset is to provide high performance networking interface
(virtio) for container-based DPDK applications. The way of starting
DPDK applications in containers with ownership of NIC devices
exclusively is beyond the scope. The basic idea here is to present
a new virtual device (named eth_cvio), which can be discovered
and initialized in container-based DPDK applications rte_eal_init().
To minimize the change, we reuse already-existing virtio frontend
driver code (driver/net/virtio/).
 
Compared to QEMU/VM case, virtio device framework (translates I/O
port r/w operations into unix socket/cuse protocol, which is originally
provided in QEMU),  is integrated in virtio frontend driver. Aka, this
new converged driver actually plays the role of original frontend
driver and the role of QEMU device framework.
 
The biggest difference here lies in how to calculate relative address
for backend. The principle of virtio is that: based on one or multiple
shared memory segments, vhost maintains a reference system with
the base addresses and length of these segments so that an address
from VM comes (usually GPA, Guest Physical Address), vhost can
translate it into self-recognizable address (aka VVA, Vhost Virtual
Address). To decrease the overhead of address translation, we should
maintain as few segments as better. In the context of virtual machines,
GPA is always locally continuous. So it's a good choice. In container's
case, CVA (Container Virtual Address) can be used. This means that:
a. when set_base_addr, CVA address is used; b. when preparing RX's
descriptors, CVA address is used; c. when transmitting packets, CVA is
filled in TX's descriptors; d. in TX and CQ's header, CVA is used.
 
How to share memory? In VM's case, qemu always shares all physical
layout to backend. But it's not feasible for a container, as a process,
to share all virtual memory regions to backend. So only specified
virtual memory regions (type is shared) are sent to backend. It leads
to a limitation that only addresses in these areas can be used to
transmit or receive packets. For now, the shared memory is created
in /dev/shm using shm_open() in the memory initialization process.
 
How to use?
 
a. Apply the patch of virtio for container. We need two copies of
patched code (referred as dpdk-app/ and dpdk-vhost/)
 
b. To compile container apps:
$: cd dpdk-app
$: vim config/common_linuxapp (uncomment "CONFIG_RTE_VIRTIO_VDEV=y")
$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
 
c. To build a docker image using Dockerfile below.
$: cat ./Dockerfile
FROM ubuntu:latest
WORKDIR /usr/src/dpdk
COPY . /usr/src/dpdk
CMD ["/usr/src/dpdk/examples/l2fwd/build/l2fwd", "-c", "0xc", "-n", "4", "--no-huge", "--no-pci", "--vdev=eth_cvio0,queue_num=256,rx=1,tx=1,cq=0,path=/var/run/usvhost", "--", "-p", "0x1"]
$: docker build -t dpdk-app-l2fwd .
 
d. To compile vhost:
$: cd dpdk-vhost
$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
 
e. Start vhost-switch
$: ./examples/vhost/build/vhost-switch -c 3 -n 4 --socket-mem 1024,1024 -- -p 0x1 --stats 1
 
f. Start docker
$: docker run -i -t -v <path to vhost unix socket>:/var/run/usvhost dpdk-app-l2fwd

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>

Jianfeng Tan (5):
  virtio/container: add handler for ioport rd/wr
  virtio/container: add a new virtual device named eth_cvio
  virtio/container: unify desc->addr assignment
  virtio/container: adjust memory initialization process
  vhost/container: change mode of vhost listening socket

 config/common_linuxapp                       |   5 +
 drivers/net/virtio/Makefile                  |   4 +
 drivers/net/virtio/vhost-user.c              | 433 +++++++++++++++++++++++++++
 drivers/net/virtio/vhost-user.h              | 137 +++++++++
 drivers/net/virtio/virtio_ethdev.c           | 319 +++++++++++++++-----
 drivers/net/virtio/virtio_ethdev.h           |  16 +
 drivers/net/virtio/virtio_pci.h              |  32 +-
 drivers/net/virtio/virtio_rxtx.c             |   9 +-
 drivers/net/virtio/virtio_rxtx_simple.c      |   9 +-
 drivers/net/virtio/virtqueue.h               |   9 +-
 lib/librte_eal/common/include/rte_memory.h   |   5 +
 lib/librte_eal/linuxapp/eal/eal_memory.c     |  58 +++-
 lib/librte_mempool/rte_mempool.c             |  16 +-
 lib/librte_vhost/vhost_user/vhost-net-user.c |   5 +
 14 files changed, 967 insertions(+), 90 deletions(-)
 create mode 100644 drivers/net/virtio/vhost-user.c
 create mode 100644 drivers/net/virtio/vhost-user.h

-- 
2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [RFC 1/5] virtio/container: add handler for ioport rd/wr
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
@ 2015-11-05 18:31 ` Jianfeng Tan
  2015-11-05 18:31 ` [RFC 2/5] virtio/container: add a new virtual device named eth_cvio Jianfeng Tan
                   ` (13 subsequent siblings)
  14 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2015-11-05 18:31 UTC (permalink / raw)
  To: dev
  Cc: nakajima.yoshihiro, zhbzg, mst, gaoxiaoqiu, oscar.zhangbo,
	ann.zhuangyanying, zhoujingbin, guohongzhen

Add handler to turn ioport rd/wr into vhost user unix socket msgs.
Add field, like kickfd, callfd in struct virtio_hw.
Add CONFIG_RTE_VIRTIO_VDEV to control virtio vdev, disabled by default.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 config/common_linuxapp          |   5 +
 drivers/net/virtio/Makefile     |   4 +
 drivers/net/virtio/vhost-user.c | 433 ++++++++++++++++++++++++++++++++++++++++
 drivers/net/virtio/vhost-user.h | 137 +++++++++++++
 drivers/net/virtio/virtio_pci.h |  32 ++-
 5 files changed, 610 insertions(+), 1 deletion(-)
 create mode 100644 drivers/net/virtio/vhost-user.c
 create mode 100644 drivers/net/virtio/vhost-user.h

diff --git a/config/common_linuxapp b/config/common_linuxapp
index c1d4bbd..99dd348 100644
--- a/config/common_linuxapp
+++ b/config/common_linuxapp
@@ -477,3 +477,8 @@ CONFIG_RTE_APP_TEST=y
 CONFIG_RTE_TEST_PMD=y
 CONFIG_RTE_TEST_PMD_RECORD_CORE_CYCLES=n
 CONFIG_RTE_TEST_PMD_RECORD_BURST_STATS=n
+
+#
+# Enable virtio support for container
+#
+#CONFIG_RTE_VIRTIO_VDEV=y
diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index 43835ba..dddf125 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -52,6 +52,10 @@ SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_ethdev.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx_simple.c
 
+ifeq ($(CONFIG_RTE_VIRTIO_VDEV),y)
+	SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += vhost-user.c
+endif
+
 # this lib depends upon:
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_eal lib/librte_ether
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_mempool lib/librte_mbuf
diff --git a/drivers/net/virtio/vhost-user.c b/drivers/net/virtio/vhost-user.c
new file mode 100644
index 0000000..d0960ce
--- /dev/null
+++ b/drivers/net/virtio/vhost-user.c
@@ -0,0 +1,433 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2015 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+#include <stdint.h>
+#include <sys/types.h>
+#include <sys/socket.h>
+#include <sys/un.h>
+#include <stdio.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <string.h>
+#include <errno.h>
+#include <assert.h>
+#include <sys/mman.h>
+#include <unistd.h>
+#include <sys/eventfd.h>
+
+#include <rte_mbuf.h>
+
+#include "virtio_pci.h"
+#include "virtio_logs.h"
+#include "virtio_ethdev.h"
+#include "virtqueue.h"
+#include "vhost-user.h"
+
+static int
+vhost_user_read(int fd, void *buf, int len, int *fds, int fd_num)
+{
+	struct msghdr msgh;
+	struct iovec iov;
+	int r;
+
+	size_t fd_size = fd_num * sizeof(int);
+	char control[CMSG_SPACE(fd_size)];
+	struct cmsghdr *cmsg;
+
+	memset(&msgh, 0, sizeof(msgh));
+	memset(control, 0, sizeof(control));
+
+	iov.iov_base = (uint8_t *)buf;
+	iov.iov_len = len;
+
+	msgh.msg_iov = &iov;
+	msgh.msg_iovlen = 1;
+
+	msgh.msg_control = control;
+	msgh.msg_controllen = sizeof(control);
+
+	cmsg = CMSG_FIRSTHDR(&msgh);
+
+	cmsg->cmsg_len = CMSG_LEN(fd_size);
+	cmsg->cmsg_level = SOL_SOCKET;
+	cmsg->cmsg_type = SCM_RIGHTS;
+	memcpy(CMSG_DATA(cmsg), fds, fd_size);
+
+	do {
+		r = sendmsg(fd, &msgh, 0);
+	} while (r < 0 && errno == EINTR);
+
+	return r;
+}
+
+static int
+vhost_user_write(int fd, VhostUserMsg *msg)
+{
+	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
+	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
+	
+
+	ret = recv(fd, (void *)msg, sz_hdr, 0);
+	if (ret < sz_hdr) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
+				ret, sz_hdr);
+		goto fail;
+	}
+
+	/* validate msg flags */
+	if (msg->flags != (valid_flags)) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg: flags 0x%x instead of 0x%x.",
+				msg->flags, valid_flags);
+		goto fail;
+	}
+
+	sz_payload = msg->size;
+	if (sz_payload) {
+		ret = recv(fd, (void *)((uint8_t*)msg + sz_hdr), sz_payload, 0);
+		if (ret < sz_payload) {
+			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
+				   ret, msg->size);
+			goto fail;
+		}
+	}
+
+	return 0;
+
+fail:
+	return -1;
+}
+
+static VhostUserMsg m __attribute__ ((unused));
+static int 
+vhost_user_sendmsg(struct virtio_hw *hw, VhostUserRequest req, void *arg)
+{
+	VhostUserMsg msg;
+	VhostUserMemoryRegion *mr;
+	struct vhost_vring_file *file = 0;
+	int need_reply = 0;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+	ssize_t fd_num = 0;
+	int len;
+
+	msg.request = req;
+	msg.flags = VHOST_USER_VERSION;
+	msg.size = 0;
+
+	switch (req) {
+	case VHOST_USER_GET_FEATURES:
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_FEATURES:
+	case VHOST_USER_SET_LOG_BASE:
+		msg.payload.u64 = *((__u64 *)arg);
+		msg.size = sizeof(m.payload.u64);
+		break;
+
+	case VHOST_USER_SET_MEM_TABLE:
+		{
+		int fd;
+		void *addr;
+		uint64_t size;
+
+		rte_memseg_info_get(0, &fd, &size, &addr);
+
+		mr = &msg.payload.memory.regions[0];
+		mr->userspace_addr = (uint64_t)addr;
+		mr->memory_size = size;
+		/* to keep continuity, use virtual address here */
+		mr->guest_phys_addr = (uint64_t)addr;
+		mr->mmap_offset = 0;
+		fds[fd_num++] = fd;
+		msg.payload.memory.nregions = 1;
+
+		msg.size = sizeof(m.payload.memory.nregions);
+		msg.size += sizeof(m.payload.memory.padding);
+		msg.size += fd_num * sizeof(VhostUserMemoryRegion);
+
+		break;
+		}
+	case VHOST_USER_SET_LOG_FD:
+		fds[fd_num++] = *((int *)arg);
+		break;
+
+	case VHOST_USER_SET_VRING_NUM:
+	case VHOST_USER_SET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(struct vhost_vring_state));
+		msg.size = sizeof(m.payload.state);
+		break;
+
+	case VHOST_USER_GET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(struct vhost_vring_state));
+		msg.size = sizeof(m.payload.state);
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_VRING_ADDR:
+		memcpy(&msg.payload.addr, arg, sizeof(struct vhost_vring_addr));
+		msg.size = sizeof(m.payload.addr);
+		break;
+
+	case VHOST_USER_SET_VRING_KICK:
+	case VHOST_USER_SET_VRING_CALL:
+	case VHOST_USER_SET_VRING_ERR:
+		file = arg;
+		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
+		msg.size = sizeof(m.payload.u64);
+		if (file->fd > 0)
+			fds[fd_num++] = file->fd;
+		else
+			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
+		break;
+	default:
+		PMD_DRV_LOG(ERR, "vhost-user trying to send unhandled ioctl");
+		return -1;
+	}
+
+	len = VHOST_USER_HDR_SIZE + msg.size;
+	if (vhost_user_read(hw->sockfd, &msg, len, fds, fd_num) < 0)
+		return 0;
+
+	if (need_reply) {
+		if (vhost_user_write(hw->sockfd, &msg) < 0)
+			return -1;
+
+		if (req != msg.request) {
+			PMD_DRV_LOG(ERR, "Received unexpected msg type."
+					" Expected %d received %d",
+					req, msg.request);
+			return -1;
+		}
+
+		switch (req) {
+		case VHOST_USER_GET_FEATURES:
+			if (msg.size != sizeof(m.payload.u64)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			*((__u64 *)arg) = msg.payload.u64;
+			break;
+		case VHOST_USER_GET_VRING_BASE:
+			if (msg.size != sizeof(m.payload.state)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			memcpy(arg, &msg.payload.state, sizeof(struct vhost_vring_state));
+			break;
+		default:
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+	}
+
+	return 0;
+}
+
+static void
+kick_one_vq(struct virtio_hw *hw, struct virtqueue *vq, unsigned queue_sel)
+{
+	struct vhost_vring_file file;
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL comes
+	 * firstly because vhost depends on this msg to allocate virtqueue
+	 * pair.
+	 */
+	file.index = queue_sel;
+	file.fd = hw->callfd;
+	vhost_user_sendmsg(hw, VHOST_USER_SET_VRING_CALL, &file);
+
+	struct vhost_vring_state state;
+	state.index = queue_sel;
+	state.num = vq->vq_ring.num;
+	vhost_user_sendmsg(hw, VHOST_USER_SET_VRING_NUM, &state);
+
+	state.num = 0; /* no reservation */
+	vhost_user_sendmsg(hw, VHOST_USER_SET_VRING_BASE, &state);
+
+	struct vhost_vring_addr addr = {
+		.index = queue_sel,
+		.desc_user_addr = (uint64_t)vq->vq_ring.desc,
+		.avail_user_addr = (uint64_t)vq->vq_ring.avail,
+		.used_user_addr = (uint64_t)vq->vq_ring.used,
+		.log_guest_addr = 0,
+		.flags = 0, /* disable log */
+	};
+	vhost_user_sendmsg(hw, VHOST_USER_SET_VRING_ADDR, &addr);
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_KICK comes
+	 * lastly because vhost depends on this msg to judge if
+	 * virtio_is_ready().
+	 */
+	file.fd = hw->kickfd;
+	vhost_user_sendmsg(hw, VHOST_USER_SET_VRING_KICK, &file);
+}
+
+static void kick_all_vq(struct virtio_hw *hw)
+{
+	unsigned i, queue_sel;
+	struct rte_eth_dev_data *data = hw->data;
+
+	vhost_user_sendmsg(hw, VHOST_USER_SET_MEM_TABLE, NULL);
+
+	for (i = 0; i < data->nb_rx_queues; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
+		kick_one_vq(hw, data->rx_queues[i], queue_sel);
+	}
+	for (i = 0; i < data->nb_tx_queues; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
+		kick_one_vq(hw, data->tx_queues[i], queue_sel);
+	}
+}
+
+void
+virtio_ioport_write(struct virtio_hw *hw, uint64_t addr, uint32_t val)
+{
+	uint64_t guest_features;
+
+	switch (addr) {
+	case VIRTIO_PCI_GUEST_FEATURES:
+		hw->guest_features = val;
+		guest_features = val;
+		vhost_user_sendmsg(hw, VHOST_USER_SET_FEATURES, &guest_features);
+		break;
+	case VIRTIO_PCI_QUEUE_PFN:
+		/* do nothing */
+		break;
+	case VIRTIO_PCI_QUEUE_SEL:
+		hw->queue_sel = val;
+		break;
+	case VIRTIO_PCI_STATUS:
+		if (val & VIRTIO_CONFIG_S_DRIVER_OK)
+			kick_all_vq(hw);
+		hw->status = val & 0xFF;
+		break;
+	default:
+		PMD_DRV_LOG(ERR, "unexpected address %"PRIu64" value 0x%x\n",
+				addr, val);
+		break;
+	}
+}
+
+uint32_t
+virtio_ioport_read(struct virtio_hw *hw, uint64_t addr)
+{
+	uint32_t ret = 0xFFFFFFFF;
+	uint64_t host_features;
+
+	PMD_DRV_LOG(INFO, "addr: %"PRIu64"\n", addr);
+
+	switch (addr) {
+	case VIRTIO_PCI_HOST_FEATURES:
+		vhost_user_sendmsg(hw, VHOST_USER_GET_FEATURES, &host_features);
+		ret = host_features;
+		break;
+	case VIRTIO_PCI_GUEST_FEATURES:
+		ret = hw->guest_features;
+		break;
+	case VIRTIO_PCI_QUEUE_PFN:
+		PMD_DRV_LOG(ERR, "VIRTIO_PCI_QUEUE_PFN (r) not supported\n");
+		break;
+	case VIRTIO_PCI_QUEUE_NUM:
+		ret = hw->queue_num;
+		break;
+	case VIRTIO_PCI_QUEUE_SEL:
+		ret = hw->queue_sel;
+		break;
+	case VIRTIO_PCI_STATUS:
+		ret = hw->status;
+		break;
+	default:
+		PMD_DRV_LOG(ERR, "%"PRIu64" (r) not supported\n", addr);
+		break;
+	}
+
+	return ret;
+}
+
+int
+virtio_vdev_init(struct rte_eth_dev_data *data, const char *path,
+		int nb_rx, int nb_tx, int nb_cq __attribute__ ((unused)),
+		int queue_num)
+{
+	int flag;
+	int sockfd, callfd, kickfd;
+	struct sockaddr_un un;
+	struct virtio_hw *hw = data->dev_private;
+
+	hw->data = data;
+	hw->path = strdup(path);
+	hw->max_rx_queues = nb_rx;
+	hw->max_tx_queues = nb_tx;
+	hw->queue_num = queue_num;
+
+	/* TODO: cq */
+
+	sockfd = socket(AF_UNIX, SOCK_STREAM, 0);
+
+	if (sockfd < 0) {
+		PMD_DRV_LOG(ERR, "socket error, %s\n", strerror(errno));
+		exit(-1);
+	}
+	flag = fcntl(sockfd, F_GETFD);
+	fcntl(sockfd, F_SETFD, flag | FD_CLOEXEC);
+
+	memset(&un, 0, sizeof(un));
+	un.sun_family = AF_UNIX;
+	snprintf(un.sun_path, sizeof(un.sun_path), "%s", hw->path);
+	if (connect(sockfd, (struct sockaddr *)&un, sizeof(un)) < 0) {
+		PMD_DRV_LOG(ERR, "connect error, %s\n", strerror(errno));
+		exit(-1);
+	}
+	hw->sockfd = sockfd;
+
+	/* or use invalid flag to disable it, but vhost-dpdk uses this to judge
+	 * if dev is alive. so finally we need two real event_fds.
+	 */
+	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (callfd < 0) {
+		PMD_DRV_LOG(ERR, "callfd error, %s\n", strerror(errno));
+		exit(-1);
+	}
+	hw->callfd = callfd;
+
+	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (kickfd < 0) {
+		PMD_DRV_LOG(ERR, "callfd error, %s\n", strerror(errno));
+		exit(-1);
+	}
+	hw->kickfd = kickfd;
+
+	/* VHOST_USER_SET_OWNER */
+	vhost_user_sendmsg(hw, VHOST_USER_SET_OWNER, NULL);
+
+	return 0;
+}
diff --git a/drivers/net/virtio/vhost-user.h b/drivers/net/virtio/vhost-user.h
new file mode 100644
index 0000000..148e2b7
--- /dev/null
+++ b/drivers/net/virtio/vhost-user.h
@@ -0,0 +1,137 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2014 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VHOST_NET_USER_H
+#define _VHOST_NET_USER_H
+
+#include <stdint.h>
+#include <linux/types.h>
+
+#define VHOST_MEMORY_MAX_NREGIONS 8
+
+struct vhost_vring_state {
+	unsigned int index;
+	unsigned int num;
+};
+
+struct vhost_vring_file {
+	unsigned int index;
+	int fd;
+};
+
+struct vhost_vring_addr {
+    unsigned int index;
+    /* Option flags. */
+    unsigned int flags;
+    /* Flag values: */
+    /* Whether log address is valid. If set enables logging. */
+#define VHOST_VRING_F_LOG 0
+
+    /* Start of array of descriptors (virtually contiguous) */
+    __u64 desc_user_addr;
+    /* Used structure address. Must be 32 bit aligned */
+    __u64 used_user_addr;
+    /* Available structure address. Must be 16 bit aligned */
+    __u64 avail_user_addr;
+    /* Logging support. */
+    /* Log writes to used structure, at offset calculated from specified
+     * address. Address must be 32 bit aligned. */
+    __u64 log_guest_addr;
+};
+
+#define VIRTIO_CONFIG_S_DRIVER_OK   4
+
+/* refer to hw/virtio/vhost-user.c */
+
+typedef enum VhostUserRequest {
+	VHOST_USER_NONE = 0,
+	VHOST_USER_GET_FEATURES = 1,
+	VHOST_USER_SET_FEATURES = 2,
+	VHOST_USER_SET_OWNER = 3,
+	VHOST_USER_RESET_OWNER = 4,
+	VHOST_USER_SET_MEM_TABLE = 5,
+	VHOST_USER_SET_LOG_BASE = 6,
+	VHOST_USER_SET_LOG_FD = 7,
+	VHOST_USER_SET_VRING_NUM = 8,
+	VHOST_USER_SET_VRING_ADDR = 9,
+	VHOST_USER_SET_VRING_BASE = 10,
+	VHOST_USER_GET_VRING_BASE = 11,
+	VHOST_USER_SET_VRING_KICK = 12,
+	VHOST_USER_SET_VRING_CALL = 13,
+	VHOST_USER_SET_VRING_ERR = 14,
+	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
+	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
+	VHOST_USER_GET_QUEUE_NUM = 17,
+	VHOST_USER_SET_VRING_ENABLE = 18,
+	VHOST_USER_MAX
+} VhostUserRequest;
+
+typedef struct VhostUserMemoryRegion {
+	uint64_t guest_phys_addr;
+	uint64_t memory_size;
+	uint64_t userspace_addr;
+	uint64_t mmap_offset;
+} VhostUserMemoryRegion;
+
+typedef struct VhostUserMemory {
+	uint32_t nregions;
+	uint32_t padding;
+	VhostUserMemoryRegion regions[VHOST_MEMORY_MAX_NREGIONS];
+} VhostUserMemory;
+
+typedef struct VhostUserMsg {
+	VhostUserRequest request;
+
+#define VHOST_USER_VERSION_MASK     0x3
+#define VHOST_USER_REPLY_MASK       (0x1 << 2)
+	uint32_t flags;
+	uint32_t size; /* the following payload size */
+	union {
+#define VHOST_USER_VRING_IDX_MASK   0xff
+#define VHOST_USER_VRING_NOFD_MASK  (0x1<<8)
+		uint64_t u64;
+		struct vhost_vring_state state;
+		struct vhost_vring_addr addr;
+		VhostUserMemory memory;
+	} payload;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+} __attribute((packed)) VhostUserMsg;
+
+#define VHOST_USER_HDR_SIZE offsetof(VhostUserMsg, payload.u64)
+#define VHOST_USER_PAYLOAD_SIZE (sizeof(VhostUserMsg) - VHOST_USER_HDR_SIZE)
+
+/* The version of the protocol we support */
+#define VHOST_USER_VERSION    0x1
+
+/*****************************************************************************/
+#endif
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index 47f722a..e33579f 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -147,7 +147,6 @@ struct virtqueue;
  * rest are per-device feature bits.
  */
 #define VIRTIO_TRANSPORT_F_START 28
-#define VIRTIO_TRANSPORT_F_END   32
 
 /* The Guest publishes the used index for which it expects an interrupt
  * at the end of the avail ring. Host should ignore the avail->flags field. */
@@ -174,6 +173,16 @@ struct virtio_hw {
 	uint8_t	    use_msix;
 	uint8_t     started;
 	uint8_t     mac_addr[ETHER_ADDR_LEN];
+#ifdef RTE_VIRTIO_VDEV
+	uint32_t	queue_num;
+	struct rte_eth_dev_data *data;
+	char		*path;
+	int			sockfd;
+	int			callfd;
+	int			kickfd;
+	uint32_t	queue_sel;
+	uint8_t		status;
+#endif
 };
 
 /*
@@ -226,6 +235,25 @@ outl_p(unsigned int data, unsigned int port)
 }
 #endif
 
+#ifdef RTE_VIRTIO_VDEV
+uint32_t virtio_ioport_read(struct virtio_hw *, uint64_t);
+void virtio_ioport_write(struct virtio_hw *, uint64_t, uint32_t);
+
+#define VIRTIO_READ_REG_1(hw, reg) \
+	virtio_ioport_read(hw, reg)
+#define VIRTIO_WRITE_REG_1(hw, reg, value) \
+	virtio_ioport_write(hw, reg, value)
+#define VIRTIO_READ_REG_2(hw, reg) \
+	virtio_ioport_read(hw, reg)
+#define VIRTIO_WRITE_REG_2(hw, reg, value) \
+	virtio_ioport_write(hw, reg, value)
+#define VIRTIO_READ_REG_4(hw, reg) \
+	virtio_ioport_read(hw, reg)
+#define VIRTIO_WRITE_REG_4(hw, reg, value) \
+	virtio_ioport_write(hw, reg, value)
+
+#else /* RTE_VIRTIO_VDEV */
+
 #define VIRTIO_PCI_REG_ADDR(hw, reg) \
 	(unsigned short)((hw)->io_base + (reg))
 
@@ -244,6 +272,8 @@ outl_p(unsigned int data, unsigned int port)
 #define VIRTIO_WRITE_REG_4(hw, reg, value) \
 	outl_p((unsigned int)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg))))
 
+#endif /* RTE_VIRTIO_VDEV */
+
 static inline int
 vtpci_with_feature(struct virtio_hw *hw, uint32_t bit)
 {
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [RFC 2/5] virtio/container: add a new virtual device named eth_cvio
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
  2015-11-05 18:31 ` [RFC 1/5] virtio/container: add handler for ioport rd/wr Jianfeng Tan
@ 2015-11-05 18:31 ` Jianfeng Tan
  2015-11-05 18:31 ` [RFC 3/5] virtio/container: unify desc->addr assignment Jianfeng Tan
                   ` (12 subsequent siblings)
  14 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2015-11-05 18:31 UTC (permalink / raw)
  To: dev
  Cc: nakajima.yoshihiro, zhbzg, mst, gaoxiaoqiu, oscar.zhangbo,
	ann.zhuangyanying, zhoujingbin, guohongzhen

Add a new virtual device named eth_cvio, it can be used just like
eth_ring, eth_null, etc. Configured paramters include number of rx,
tx, cq, path of vhost unix socket, and queue size. The major
difference with virtio for vm is that here we use virtual address
instead of physical address for vhost to calculate relative address.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 drivers/net/virtio/virtio_ethdev.c | 319 +++++++++++++++++++++++++++++--------
 drivers/net/virtio/virtio_ethdev.h |  16 ++
 drivers/net/virtio/virtqueue.h     |   9 +-
 3 files changed, 275 insertions(+), 69 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 6001108..b5e2126 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -56,6 +56,7 @@
 #include <rte_memory.h>
 #include <rte_eal.h>
 #include <rte_dev.h>
+#include <rte_kvargs.h>
 
 #include "virtio_ethdev.h"
 #include "virtio_pci.h"
@@ -63,7 +64,6 @@
 #include "virtqueue.h"
 #include "virtio_rxtx.h"
 
-
 static int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
 static int eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev);
 static int  virtio_dev_configure(struct rte_eth_dev *dev);
@@ -164,8 +164,7 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	if ((vq->vq_free_cnt < ((uint32_t)pkt_num + 2)) || (pkt_num < 1))
 		return -1;
 
-	memcpy(vq->virtio_net_hdr_mz->addr, ctrl,
-		sizeof(struct virtio_pmd_ctrl));
+	memcpy(vq->virtio_net_hdr_vaddr, ctrl, sizeof(struct virtio_pmd_ctrl));
 
 	/*
 	 * Format is enforced in qemu code:
@@ -174,14 +173,14 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	 * One RX packet for ACK.
 	 */
 	vq->vq_ring.desc[head].flags = VRING_DESC_F_NEXT;
-	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mz->phys_addr;
+	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mem;
 	vq->vq_ring.desc[head].len = sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_free_cnt--;
 	i = vq->vq_ring.desc[head].next;
 
 	for (k = 0; k < pkt_num; k++) {
 		vq->vq_ring.desc[i].flags = VRING_DESC_F_NEXT;
-		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr)
 			+ sizeof(ctrl->status) + sizeof(uint8_t)*sum;
 		vq->vq_ring.desc[i].len = dlen[k];
@@ -191,7 +190,7 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	}
 
 	vq->vq_ring.desc[i].flags = VRING_DESC_F_WRITE;
-	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_ring.desc[i].len = sizeof(ctrl->status);
 	vq->vq_free_cnt--;
@@ -236,7 +235,7 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	PMD_INIT_LOG(DEBUG, "vq->vq_free_cnt=%d\nvq->vq_desc_head_idx=%d",
 			vq->vq_free_cnt, vq->vq_desc_head_idx);
 
-	memcpy(&result, vq->virtio_net_hdr_mz->addr,
+	memcpy(&result, vq->virtio_net_hdr_vaddr,
 			sizeof(struct virtio_pmd_ctrl));
 
 	return result.status;
@@ -374,66 +373,79 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		}
 	}
 
-	/*
-	 * Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
-	 * and only accepts 32 bit page frame number.
-	 * Check if the allocated physical memory exceeds 16TB.
-	 */
-	if ((mz->phys_addr + vq->vq_ring_size - 1) >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
-		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
-		rte_free(vq);
-		return -ENOMEM;
-	}
-
 	memset(mz->addr, 0, sizeof(mz->len));
 	vq->mz = mz;
-	vq->vq_ring_mem = mz->phys_addr;
 	vq->vq_ring_virt_mem = mz->addr;
-	PMD_INIT_LOG(DEBUG, "vq->vq_ring_mem:      0x%"PRIx64, (uint64_t)mz->phys_addr);
-	PMD_INIT_LOG(DEBUG, "vq->vq_ring_virt_mem: 0x%"PRIx64, (uint64_t)(uintptr_t)mz->addr);
+
+	if (dev->dev_type == RTE_ETH_DEV_PCI) {
+		vq->vq_ring_mem = mz->phys_addr;
+
+		/* Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
+		 * and only accepts 32 bit page frame number.
+		 * Check if the allocated physical memory exceeds 16TB.
+		 */
+		uint64_t last_physaddr = vq->vq_ring_mem + vq->vq_ring_size - 1;
+		if (last_physaddr >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
+			PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
+			rte_free(vq);
+			return -ENOMEM;
+		}
+	}
+#ifdef RTE_VIRTIO_VDEV
+	else { /* RTE_ETH_DEV_VIRTUAL */
+		/* Use virtual addr to fill!!! */
+		vq->vq_ring_mem = (phys_addr_t)mz->addr;
+
+		/* TODO: check last_physaddr */
+	}
+#endif
+
+	PMD_INIT_LOG(DEBUG, "vq->vq_ring_mem:      0x%"PRIx64,
+			(uint64_t)vq->vq_ring_mem);
+	PMD_INIT_LOG(DEBUG, "vq->vq_ring_virt_mem: 0x%"PRIx64,
+			(uint64_t)(uintptr_t)vq->vq_ring_virt_mem);
 	vq->virtio_net_hdr_mz  = NULL;
 	vq->virtio_net_hdr_mem = 0;
 
+	uint64_t hdr_size = 0;
 	if (queue_type == VTNET_TQ) {
 		/*
 		 * For each xmit packet, allocate a virtio_net_hdr
 		 */
 		snprintf(vq_name, sizeof(vq_name), "port%d_tvq%d_hdrzone",
-			dev->data->port_id, queue_idx);
-		vq->virtio_net_hdr_mz = rte_memzone_reserve_aligned(vq_name,
-			vq_size * hw->vtnet_hdr_size,
-			socket_id, 0, RTE_CACHE_LINE_SIZE);
-		if (vq->virtio_net_hdr_mz == NULL) {
-			if (rte_errno == EEXIST)
-				vq->virtio_net_hdr_mz =
-					rte_memzone_lookup(vq_name);
-			if (vq->virtio_net_hdr_mz == NULL) {
-				rte_free(vq);
-				return -ENOMEM;
-			}
-		}
-		vq->virtio_net_hdr_mem =
-			vq->virtio_net_hdr_mz->phys_addr;
-		memset(vq->virtio_net_hdr_mz->addr, 0,
-			vq_size * hw->vtnet_hdr_size);
+				dev->data->port_id, queue_idx);
+		hdr_size = vq_size * hw->vtnet_hdr_size;
 	} else if (queue_type == VTNET_CQ) {
-		/* Allocate a page for control vq command, data and status */
 		snprintf(vq_name, sizeof(vq_name), "port%d_cvq_hdrzone",
-			dev->data->port_id);
-		vq->virtio_net_hdr_mz = rte_memzone_reserve_aligned(vq_name,
-			PAGE_SIZE, socket_id, 0, RTE_CACHE_LINE_SIZE);
-		if (vq->virtio_net_hdr_mz == NULL) {
+				dev->data->port_id);
+		/* Allocate a page for control vq command, data and status */
+		hdr_size = PAGE_SIZE;
+	}
+
+	if (hdr_size) { /* queue_type is VTNET_TQ or VTNET_CQ */
+		mz = rte_memzone_reserve_aligned(vq_name,
+				hdr_size, socket_id, 0, RTE_CACHE_LINE_SIZE);
+		if (mz == NULL) {
 			if (rte_errno == EEXIST)
-				vq->virtio_net_hdr_mz =
-					rte_memzone_lookup(vq_name);
-			if (vq->virtio_net_hdr_mz == NULL) {
+				mz = rte_memzone_lookup(vq_name);
+			if (mz == NULL) {
 				rte_free(vq);
 				return -ENOMEM;
 			}
 		}
-		vq->virtio_net_hdr_mem =
-			vq->virtio_net_hdr_mz->phys_addr;
-		memset(vq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
+		vq->virtio_net_hdr_mz = mz;
+		vq->virtio_net_hdr_vaddr = mz->addr;
+		memset(vq->virtio_net_hdr_vaddr, 0, hdr_size);
+
+		if (dev->dev_type == RTE_ETH_DEV_PCI) {
+			vq->virtio_net_hdr_mem = mz->phys_addr;
+		}
+#ifdef RTE_VIRTIO_VDEV
+		else {
+			/* Use vaddr!!! */
+			vq->virtio_net_hdr_mem = (phys_addr_t)mz->addr;
+		}
+#endif
 	}
 
 	/*
@@ -491,8 +503,10 @@ virtio_dev_close(struct rte_eth_dev *dev)
 	PMD_INIT_LOG(DEBUG, "virtio_dev_close");
 
 	/* reset the NIC */
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		vtpci_irq_config(hw, VIRTIO_MSI_NO_VECTOR);
+	if (dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			vtpci_irq_config(hw, VIRTIO_MSI_NO_VECTOR);
+	}
 	vtpci_reset(hw);
 	hw->started = 0;
 	virtio_dev_free_mbufs(dev);
@@ -1288,11 +1302,18 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 	}
 
 	pci_dev = eth_dev->pci_dev;
-	if (virtio_resource_init(pci_dev) < 0)
-		return -1;
-
-	hw->use_msix = virtio_has_msix(&pci_dev->addr);
-	hw->io_base = (uint32_t)(uintptr_t)pci_dev->mem_resource[0].addr;
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (virtio_resource_init(pci_dev) < 0)
+			return -1;
+		hw->use_msix = virtio_has_msix(&pci_dev->addr);
+		hw->io_base = (uint32_t)(uintptr_t)pci_dev->mem_resource[0].addr;
+	}
+#ifdef RTE_VIRTIO_VDEV
+	else {
+		hw->use_msix = 0;
+		hw->io_base = 0;
+	}
+#endif
 
 	/* Reset the device although not necessary at startup */
 	vtpci_reset(hw);
@@ -1305,8 +1326,10 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 	virtio_negotiate_features(hw);
 
 	/* If host does not support status then disable LSC */
-	if (!vtpci_with_feature(hw, VIRTIO_NET_F_STATUS))
-		pci_dev->driver->drv_flags &= ~RTE_PCI_DRV_INTR_LSC;
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (!vtpci_with_feature(hw, VIRTIO_NET_F_STATUS))
+			pci_dev->driver->drv_flags &= ~RTE_PCI_DRV_INTR_LSC;
+	}
 
 	rx_func_get(eth_dev);
 
@@ -1385,12 +1408,12 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 	PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
 			eth_dev->data->port_id, pci_dev->id.vendor_id,
 			pci_dev->id.device_id);
-
-	/* Setup interrupt callback  */
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		rte_intr_callback_register(&pci_dev->intr_handle,
-				   virtio_interrupt_handler, eth_dev);
-
+		/* Setup interrupt callback  */
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			rte_intr_callback_register(&pci_dev->intr_handle,
+					virtio_interrupt_handler, eth_dev);
+	}
 	virtio_dev_cq_start(eth_dev);
 
 	return 0;
@@ -1423,10 +1446,12 @@ eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev)
 	eth_dev->data->mac_addrs = NULL;
 
 	/* reset interrupt callback  */
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		rte_intr_callback_unregister(&pci_dev->intr_handle,
-						virtio_interrupt_handler,
-						eth_dev);
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			rte_intr_callback_unregister(&pci_dev->intr_handle,
+					virtio_interrupt_handler,
+					eth_dev);
+	}
 
 	PMD_INIT_LOG(DEBUG, "dev_uninit completed");
 
@@ -1481,6 +1506,11 @@ virtio_dev_configure(struct rte_eth_dev *dev)
 		return (-EINVAL);
 	}
 
+#ifdef RTE_VIRTIO_VDEV
+	if (dev->dev_type == RTE_ETH_DEV_VIRTUAL)
+		return 0;
+#endif
+
 	hw->vlan_strip = rxmode->hw_vlan_strip;
 
 	if (rxmode->hw_vlan_filter
@@ -1688,3 +1718,156 @@ static struct rte_driver rte_virtio_driver = {
 };
 
 PMD_REGISTER_DRIVER(rte_virtio_driver);
+
+#ifdef RTE_VIRTIO_VDEV
+
+#define ETH_CVIO_ARG_RX_NUM		"rx"
+#define ETH_CVIO_ARG_TX_NUM		"tx"
+#define ETH_CVIO_ARG_CQ_NUM		"cq"
+#define ETH_CVIO_ARG_SK_PATH	"path"
+#define ETH_CVIO_ARG_QUEUE_SIZE	"queue_num"
+/*TODO: specify mac addr */
+static const char *valid_args[] = {
+	ETH_CVIO_ARG_RX_NUM,
+	ETH_CVIO_ARG_TX_NUM,
+	ETH_CVIO_ARG_CQ_NUM,
+	ETH_CVIO_ARG_SK_PATH,
+	ETH_CVIO_ARG_QUEUE_SIZE,
+	NULL
+};
+
+static int
+get_string_arg(const char *key __rte_unused,
+		const char *value, void *extra_args)
+{
+	if ((value == NULL) || (extra_args == NULL))
+		return -EINVAL;
+
+	strcpy(extra_args, value);
+
+	return 0;
+}
+
+static int
+get_integer_arg(const char *key __rte_unused,
+		const char *value, void *extra_args)
+{
+	uint64_t *p_u64 = extra_args;
+
+	if ((value == NULL) || (extra_args == NULL))
+		return -EINVAL;
+
+	*p_u64 = (uint64_t)strtoull(value, NULL, 0);
+
+	return 0;
+}
+
+static struct rte_eth_dev *
+cvio_eth_dev_alloc(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct rte_eth_dev_data *data;
+	struct rte_pci_device *pci_dev;
+	struct virtio_hw *hw;
+
+	eth_dev = rte_eth_dev_allocate(name, RTE_ETH_DEV_VIRTUAL);
+	if (eth_dev == NULL)
+		rte_panic("cannot alloc rte_eth_dev\n");
+
+	data = eth_dev->data;
+
+	pci_dev = rte_zmalloc(NULL, sizeof(*pci_dev), 0);
+	if (!pci_dev)
+		rte_panic("cannot alloc pci_dev\n");
+	hw = rte_zmalloc(NULL, sizeof(*hw), 0);
+	if (!hw)
+		rte_panic("malloc virtio_hw failed\n");
+
+	data->dev_private = hw;
+	pci_dev->numa_node = SOCKET_ID_ANY;
+	/* TODO: should remove pci_dev after Bernard Iremonger's patch applied */
+	eth_dev->pci_dev = pci_dev;
+	/* will be used in virtio_dev_info_get() */
+	eth_dev->driver = &rte_virtio_pmd;
+	/* TAILQ_INIT(&(eth_dev->link_intr_cbs)); */
+	return eth_dev;
+}
+
+/*
+ * Dev initialization routine.
+ * Invoked once for each virtio vdev at EAL init time,
+ * See rte_eal_dev_init().
+ * Returns 0 on success.
+ */
+static int
+rte_cvio_pmd_devinit(const char *name, const char *params)
+{
+	struct rte_kvargs *kvlist = NULL;
+	struct rte_eth_dev *eth_dev = NULL;
+	uint64_t nb_rx = 1, nb_tx = 1, nb_cq = 0, queue_num = 256;
+	char sock_path[256];
+
+	if (params == NULL || params[0] == '\0') {
+		rte_panic("param is null\n");
+	}
+
+	kvlist = rte_kvargs_parse(params, valid_args);
+	if (!kvlist)
+		rte_panic("error when parsing param\n");
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_SK_PATH) == 1) {
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_SK_PATH,
+				&get_string_arg, sock_path);
+	} else {
+		rte_panic("no arg: %s\n", ETH_CVIO_ARG_SK_PATH);
+	}
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_QUEUE_SIZE) == 1) {
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_QUEUE_SIZE,
+			&get_integer_arg, &queue_num);
+	}
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_RX_NUM) == 1) {
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_RX_NUM,
+			&get_integer_arg, &nb_rx);
+	}
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_TX_NUM) == 1) {
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_TX_NUM,
+			&get_integer_arg, &nb_tx);
+	}
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_CQ_NUM) == 1) {
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_CQ_NUM,
+			&get_integer_arg, &nb_cq);
+	}
+
+	eth_dev = cvio_eth_dev_alloc(name);
+
+	virtio_vdev_init(eth_dev->data, sock_path,
+			nb_rx, nb_tx, nb_cq, queue_num);
+
+	/* originally, this will be called in rte_eal_pci_probe() */
+	eth_virtio_dev_init(eth_dev);
+
+	return 0;
+}
+
+static int
+rte_cvio_pmd_devuninit(const char *name)
+{
+	/* TODO: if it's last one, memory init, free memory */
+	rte_panic("%s", name);
+	return 0;
+}
+
+static struct rte_driver rte_cvio_driver = {
+	.name   = "eth_cvio",
+	.type   = PMD_VDEV,
+	.init   = rte_cvio_pmd_devinit,
+	.uninit = rte_cvio_pmd_devuninit,
+};
+
+PMD_REGISTER_DRIVER(rte_cvio_driver);
+
+#endif
diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
index ae2d47d..25613ac 100644
--- a/drivers/net/virtio/virtio_ethdev.h
+++ b/drivers/net/virtio/virtio_ethdev.h
@@ -56,6 +56,17 @@
 #define VIRTIO_MAX_RX_PKTLEN  9728
 
 /* Features desired/implemented by this driver. */
+#ifdef RTE_VIRTIO_VDEV
+/* use random mac addr for now */
+/* control queue not available for now */
+#define VIRTIO_PMD_GUEST_FEATURES		\
+	(1u << VIRTIO_NET_F_STATUS	  |	\
+	 1u << VIRTIO_NET_F_MQ		  |	\
+	 1u << VIRTIO_NET_F_CTRL_MAC_ADDR |	\
+	 1u << VIRTIO_NET_F_CTRL_RX	  |	\
+	 1u << VIRTIO_NET_F_CTRL_VLAN	  |	\
+	 1u << VIRTIO_NET_F_MRG_RXBUF)
+#else
 #define VIRTIO_PMD_GUEST_FEATURES		\
 	(1u << VIRTIO_NET_F_MAC		  |	\
 	 1u << VIRTIO_NET_F_STATUS	  |	\
@@ -65,6 +76,7 @@
 	 1u << VIRTIO_NET_F_CTRL_RX	  |	\
 	 1u << VIRTIO_NET_F_CTRL_VLAN	  |	\
 	 1u << VIRTIO_NET_F_MRG_RXBUF)
+#endif
 
 /*
  * CQ function prototype
@@ -122,5 +134,9 @@ uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 #define VTNET_LRO_FEATURES (VIRTIO_NET_F_GUEST_TSO4 | \
 			    VIRTIO_NET_F_GUEST_TSO6 | VIRTIO_NET_F_GUEST_ECN)
 
+#ifdef RTE_VIRTIO_VDEV
+int virtio_vdev_init(struct rte_eth_dev_data *data, const char *path,
+		int nb_rx, int nb_tx, int nb_cq, int queue_num);
+#endif
 
 #endif /* _VIRTIO_ETHDEV_H_ */
diff --git a/drivers/net/virtio/virtqueue.h b/drivers/net/virtio/virtqueue.h
index 689c321..7eb4187 100644
--- a/drivers/net/virtio/virtqueue.h
+++ b/drivers/net/virtio/virtqueue.h
@@ -68,8 +68,13 @@ struct rte_mbuf;
 
 #define VIRTQUEUE_MAX_NAME_SZ 32
 
+#ifdef RTE_VIRTIO_VDEV
+#define RTE_MBUF_DATA_DMA_ADDR(mb) \
+	((uint64_t)(mb)->buf_addr + (mb)->data_off)
+#else
 #define RTE_MBUF_DATA_DMA_ADDR(mb) \
 	(uint64_t) ((mb)->buf_physaddr + (mb)->data_off)
+#endif /* RTE_VIRTIO_VDEV */
 
 #define VTNET_SQ_RQ_QUEUE_IDX 0
 #define VTNET_SQ_TQ_QUEUE_IDX 1
@@ -169,7 +174,8 @@ struct virtqueue {
 
 	void        *vq_ring_virt_mem;    /**< linear address of vring*/
 	unsigned int vq_ring_size;
-	phys_addr_t vq_ring_mem;          /**< physical address of vring */
+	phys_addr_t vq_ring_mem;          /**< physical address of vring for non-vdev,
+										virtual addr of vring for vdev*/
 
 	struct vring vq_ring;    /**< vring keeping desc, used and avail */
 	uint16_t    vq_free_cnt; /**< num of desc available */
@@ -190,6 +196,7 @@ struct virtqueue {
 	uint16_t vq_avail_idx;
 	uint64_t mbuf_initializer; /**< value to init mbufs. */
 	phys_addr_t virtio_net_hdr_mem; /**< hdr for each xmit packet */
+	void        *virtio_net_hdr_vaddr;    /**< linear address of vring*/
 
 	struct rte_mbuf **sw_ring; /**< RX software ring. */
 	/* dummy mbuf, for wraparound when processing RX ring. */
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [RFC 3/5] virtio/container: unify desc->addr assignment
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
  2015-11-05 18:31 ` [RFC 1/5] virtio/container: add handler for ioport rd/wr Jianfeng Tan
  2015-11-05 18:31 ` [RFC 2/5] virtio/container: add a new virtual device named eth_cvio Jianfeng Tan
@ 2015-11-05 18:31 ` Jianfeng Tan
  2015-11-05 18:31 ` [RFC 4/5] virtio/container: adjust memory initialization process Jianfeng Tan
                   ` (11 subsequent siblings)
  14 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2015-11-05 18:31 UTC (permalink / raw)
  To: dev
  Cc: nakajima.yoshihiro, zhbzg, mst, gaoxiaoqiu, oscar.zhangbo,
	ann.zhuangyanying, zhoujingbin, guohongzhen

Unify desc->addr assignment using RTE_MBUF_DATA_DMA_ADDR. virtio
for vm uses physical address, while virtio for container uses
virtual address.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 drivers/net/virtio/virtio_rxtx.c        | 9 ++++-----
 drivers/net/virtio/virtio_rxtx_simple.c | 9 ++++-----
 2 files changed, 8 insertions(+), 10 deletions(-)

diff --git a/drivers/net/virtio/virtio_rxtx.c b/drivers/net/virtio/virtio_rxtx.c
index 5770fa2..1cfb2b9 100644
--- a/drivers/net/virtio/virtio_rxtx.c
+++ b/drivers/net/virtio/virtio_rxtx.c
@@ -191,8 +191,7 @@ virtqueue_enqueue_recv_refill(struct virtqueue *vq, struct rte_mbuf *cookie)
 
 	start_dp = vq->vq_ring.desc;
 	start_dp[idx].addr =
-		(uint64_t)(cookie->buf_physaddr + RTE_PKTMBUF_HEADROOM
-		- hw->vtnet_hdr_size);
+		RTE_MBUF_DATA_DMA_ADDR(cookie) - hw->vtnet_hdr_size;
 	start_dp[idx].len =
 		cookie->buf_len - RTE_PKTMBUF_HEADROOM + hw->vtnet_hdr_size;
 	start_dp[idx].flags =  VRING_DESC_F_WRITE;
@@ -343,7 +342,7 @@ virtio_dev_vring_start(struct virtqueue *vq, int queue_type)
 		VIRTIO_WRITE_REG_2(vq->hw, VIRTIO_PCI_QUEUE_SEL,
 			vq->vq_queue_index);
 		VIRTIO_WRITE_REG_4(vq->hw, VIRTIO_PCI_QUEUE_PFN,
-			vq->mz->phys_addr >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
+			vq->vq_ring_mem >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
 	} else if (queue_type == VTNET_TQ) {
 		if (use_simple_rxtx) {
 			int mid_idx  = vq->vq_nentries >> 1;
@@ -366,12 +365,12 @@ virtio_dev_vring_start(struct virtqueue *vq, int queue_type)
 		VIRTIO_WRITE_REG_2(vq->hw, VIRTIO_PCI_QUEUE_SEL,
 			vq->vq_queue_index);
 		VIRTIO_WRITE_REG_4(vq->hw, VIRTIO_PCI_QUEUE_PFN,
-			vq->mz->phys_addr >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
+			vq->vq_ring_mem >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
 	} else {
 		VIRTIO_WRITE_REG_2(vq->hw, VIRTIO_PCI_QUEUE_SEL,
 			vq->vq_queue_index);
 		VIRTIO_WRITE_REG_4(vq->hw, VIRTIO_PCI_QUEUE_PFN,
-			vq->mz->phys_addr >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
+			vq->vq_ring_mem >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
 	}
 }
 
diff --git a/drivers/net/virtio/virtio_rxtx_simple.c b/drivers/net/virtio/virtio_rxtx_simple.c
index ff3c11a..d1bb4c4 100644
--- a/drivers/net/virtio/virtio_rxtx_simple.c
+++ b/drivers/net/virtio/virtio_rxtx_simple.c
@@ -80,8 +80,8 @@ virtqueue_enqueue_recv_refill_simple(struct virtqueue *vq,
 	vq->sw_ring[desc_idx] = cookie;
 
 	start_dp = vq->vq_ring.desc;
-	start_dp[desc_idx].addr = (uint64_t)((uintptr_t)cookie->buf_physaddr +
-		RTE_PKTMBUF_HEADROOM - sizeof(struct virtio_net_hdr));
+	start_dp[desc_idx].addr = RTE_MBUF_DATA_DMA_ADDR(cookie)
+		- sizeof(struct virtio_net_hdr);
 	start_dp[desc_idx].len = cookie->buf_len -
 		RTE_PKTMBUF_HEADROOM + sizeof(struct virtio_net_hdr);
 
@@ -118,9 +118,8 @@ virtio_rxq_rearm_vec(struct virtqueue *rxvq)
 		p = (uintptr_t)&sw_ring[i]->rearm_data;
 		*(uint64_t *)p = rxvq->mbuf_initializer;
 
-		start_dp[i].addr =
-			(uint64_t)((uintptr_t)sw_ring[i]->buf_physaddr +
-			RTE_PKTMBUF_HEADROOM - sizeof(struct virtio_net_hdr));
+		start_dp[i].addr = RTE_MBUF_DATA_DMA_ADDR(sw_ring[i])
+			- sizeof(struct virtio_net_hdr);
 		start_dp[i].len = sw_ring[i]->buf_len -
 			RTE_PKTMBUF_HEADROOM + sizeof(struct virtio_net_hdr);
 	}
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [RFC 4/5] virtio/container: adjust memory initialization process
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
                   ` (2 preceding siblings ...)
  2015-11-05 18:31 ` [RFC 3/5] virtio/container: unify desc->addr assignment Jianfeng Tan
@ 2015-11-05 18:31 ` Jianfeng Tan
  2015-11-06 16:21   ` Ananyev, Konstantin
  2015-11-05 18:31 ` [RFC 5/5] vhost/container: change mode of vhost listening socket Jianfeng Tan
                   ` (10 subsequent siblings)
  14 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2015-11-05 18:31 UTC (permalink / raw)
  To: dev
  Cc: nakajima.yoshihiro, zhbzg, mst, gaoxiaoqiu, oscar.zhangbo,
	ann.zhuangyanying, zhoujingbin, guohongzhen

When using virtio for container, we should specify --no-huge so
that in memory initialization, shm_open() is used to alloc memory
from tmpfs filesystem /dev/shm/.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 lib/librte_eal/common/include/rte_memory.h |  5 +++
 lib/librte_eal/linuxapp/eal/eal_memory.c   | 58 ++++++++++++++++++++++++++++--
 lib/librte_mempool/rte_mempool.c           | 16 ++++-----
 3 files changed, 69 insertions(+), 10 deletions(-)

diff --git a/lib/librte_eal/common/include/rte_memory.h b/lib/librte_eal/common/include/rte_memory.h
index 1bed415..9c1effc 100644
--- a/lib/librte_eal/common/include/rte_memory.h
+++ b/lib/librte_eal/common/include/rte_memory.h
@@ -100,6 +100,7 @@ struct rte_memseg {
 	int32_t socket_id;          /**< NUMA socket ID. */
 	uint32_t nchannel;          /**< Number of channels. */
 	uint32_t nrank;             /**< Number of ranks. */
+	int fd;                     /**< fd used for share this memory */
 #ifdef RTE_LIBRTE_XEN_DOM0
 	 /**< store segment MFNs */
 	uint64_t mfn[DOM0_NUM_MEMBLOCK];
@@ -128,6 +129,10 @@ int rte_mem_lock_page(const void *virt);
  */
 phys_addr_t rte_mem_virt2phy(const void *virt);
 
+
+int
+rte_memseg_info_get(int index, int *pfd, uint64_t *psize, void **paddr);
+
 /**
  * Get the layout of the available physical memory.
  *
diff --git a/lib/librte_eal/linuxapp/eal/eal_memory.c b/lib/librte_eal/linuxapp/eal/eal_memory.c
index ac2745e..9abbfc6 100644
--- a/lib/librte_eal/linuxapp/eal/eal_memory.c
+++ b/lib/librte_eal/linuxapp/eal/eal_memory.c
@@ -80,6 +80,9 @@
 #include <errno.h>
 #include <sys/ioctl.h>
 #include <sys/time.h>
+#include <mntent.h>
+#include <sys/mman.h>
+#include <sys/file.h>
 
 #include <rte_log.h>
 #include <rte_memory.h>
@@ -143,6 +146,18 @@ rte_mem_lock_page(const void *virt)
 	return mlock((void*)aligned, page_size);
 }
 
+int
+rte_memseg_info_get(int index, int *pfd, uint64_t *psize, void **paddr)
+{
+	struct rte_mem_config *mcfg;
+	mcfg = rte_eal_get_configuration()->mem_config;
+
+	*pfd = mcfg->memseg[index].fd;
+	*psize = (uint64_t)mcfg->memseg[index].len;
+	*paddr = (void *)(uint64_t)mcfg->memseg[index].addr;
+	return 0;
+}
+
 /*
  * Get physical address of any mapped virtual address in the current process.
  */
@@ -1044,6 +1059,42 @@ calc_num_pages_per_socket(uint64_t * memory,
 	return total_num_pages;
 }
 
+static void *
+rte_eal_shm_create(int *pfd)
+{
+	int ret, fd;
+	char filepath[256];
+	void *vaddr;
+	uint64_t size = internal_config.memory;
+
+	sprintf(filepath, "/%s_cvio", internal_config.hugefile_prefix);
+
+	fd = shm_open(filepath, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR);
+	if (fd < 0) {
+		rte_panic("shm_open %s failed: %s\n", filepath, strerror(errno));
+	}
+	ret = flock(fd, LOCK_EX);
+	if (ret < 0) {
+		close(fd);
+		rte_panic("flock %s failed: %s\n", filepath, strerror(errno));
+	}
+
+	ret = ftruncate(fd, size);
+	if (ret < 0) {
+		rte_panic("ftruncate failed: %s\n", strerror(errno));
+	}
+	/* flag: MAP_HUGETLB */
+	vaddr = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0);
+	if (vaddr == MAP_FAILED) {
+		rte_panic("mmap failed: %s\n", strerror(errno));
+	}
+	memset(vaddr, 0, size);
+	*pfd = fd;
+
+	return vaddr;
+}
+
+
 /*
  * Prepare physical memory mapping: fill configuration structure with
  * these infos, return 0 on success.
@@ -1072,7 +1123,9 @@ rte_eal_hugepage_init(void)
 	int new_pages_count[MAX_HUGEPAGE_SIZES];
 #endif
 
+#ifndef RTE_VIRTIO_VDEV
 	test_proc_pagemap_readable();
+#endif
 
 	memset(used_hp, 0, sizeof(used_hp));
 
@@ -1081,8 +1134,8 @@ rte_eal_hugepage_init(void)
 
 	/* hugetlbfs can be disabled */
 	if (internal_config.no_hugetlbfs) {
-		addr = mmap(NULL, internal_config.memory, PROT_READ | PROT_WRITE,
-				MAP_PRIVATE | MAP_ANONYMOUS, 0, 0);
+		int fd;
+		addr = rte_eal_shm_create(&fd);
 		if (addr == MAP_FAILED) {
 			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n", __func__,
 					strerror(errno));
@@ -1093,6 +1146,7 @@ rte_eal_hugepage_init(void)
 		mcfg->memseg[0].hugepage_sz = RTE_PGSIZE_4K;
 		mcfg->memseg[0].len = internal_config.memory;
 		mcfg->memseg[0].socket_id = 0;
+		mcfg->memseg[0].fd = fd;
 		return 0;
 	}
 
diff --git a/lib/librte_mempool/rte_mempool.c b/lib/librte_mempool/rte_mempool.c
index e57cbbd..8f8852b 100644
--- a/lib/librte_mempool/rte_mempool.c
+++ b/lib/librte_mempool/rte_mempool.c
@@ -453,13 +453,6 @@ rte_mempool_xmem_create(const char *name, unsigned n, unsigned elt_size,
 		rte_errno = EINVAL;
 		return NULL;
 	}
-
-	/* check that we have both VA and PA */
-	if (vaddr != NULL && paddr == NULL) {
-		rte_errno = EINVAL;
-		return NULL;
-	}
-
 	/* Check that pg_num and pg_shift parameters are valid. */
 	if (pg_num < RTE_DIM(mp->elt_pa) || pg_shift > MEMPOOL_PG_SHIFT_MAX) {
 		rte_errno = EINVAL;
@@ -596,8 +589,15 @@ rte_mempool_xmem_create(const char *name, unsigned n, unsigned elt_size,
 
 	/* mempool elements in a separate chunk of memory. */
 	} else {
+		/* when VA is specified, PA should be specified? */
+		if (rte_eal_has_hugepages()) {
+			if (paddr == NULL) {
+				rte_errno = EINVAL;
+				return NULL;
+			}
+			memcpy(mp->elt_pa, paddr, sizeof (mp->elt_pa[0]) * pg_num);
+		}
 		mp->elt_va_start = (uintptr_t)vaddr;
-		memcpy(mp->elt_pa, paddr, sizeof (mp->elt_pa[0]) * pg_num);
 	}
 
 	mp->elt_va_end = mp->elt_va_start;
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [RFC 5/5] vhost/container: change mode of vhost listening socket
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
                   ` (3 preceding siblings ...)
  2015-11-05 18:31 ` [RFC 4/5] virtio/container: adjust memory initialization process Jianfeng Tan
@ 2015-11-05 18:31 ` Jianfeng Tan
  2015-11-09  3:54   ` Yuanhan Liu
  2015-11-24  3:53 ` [RFC 0/5] virtio support for container Zhuangyanying
                   ` (9 subsequent siblings)
  14 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2015-11-05 18:31 UTC (permalink / raw)
  To: dev
  Cc: nakajima.yoshihiro, zhbzg, mst, gaoxiaoqiu, oscar.zhangbo,
	ann.zhuangyanying, zhoujingbin, guohongzhen

Change vhost listening socket mode so that users in groups and
others can connect to vhost listening socket.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 lib/librte_vhost/vhost_user/vhost-net-user.c | 5 +++++
 1 file changed, 5 insertions(+)

diff --git a/lib/librte_vhost/vhost_user/vhost-net-user.c b/lib/librte_vhost/vhost_user/vhost-net-user.c
index 2dc0547..7b24f7c 100644
--- a/lib/librte_vhost/vhost_user/vhost-net-user.c
+++ b/lib/librte_vhost/vhost_user/vhost-net-user.c
@@ -42,6 +42,7 @@
 #include <sys/un.h>
 #include <errno.h>
 #include <pthread.h>
+#include <sys/stat.h>
 
 #include <rte_log.h>
 #include <rte_virtio_net.h>
@@ -137,6 +138,10 @@ uds_socket(const char *path)
 	if (ret == -1)
 		goto err;
 
+	ret = chmod(un.sun_path, 0666);
+	if (ret == 0)
+		RTE_LOG(INFO, VHOST_CONFIG, "chmod 0666, ok\n");
+
 	return sockfd;
 
 err:
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* Re: [RFC 4/5] virtio/container: adjust memory initialization process
  2015-11-05 18:31 ` [RFC 4/5] virtio/container: adjust memory initialization process Jianfeng Tan
@ 2015-11-06 16:21   ` Ananyev, Konstantin
  2015-11-08 11:18     ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Ananyev, Konstantin @ 2015-11-06 16:21 UTC (permalink / raw)
  To: Tan, Jianfeng, dev
  Cc: nakajima.yoshihiro, zhbzg, mst, gaoxiaoqiu, oscar.zhangbo,
	ann.zhuangyanying, zhoujingbin, guohongzhen

Hi,

> -----Original Message-----
> From: dev [mailto:dev-bounces@dpdk.org] On Behalf Of Jianfeng Tan
> Sent: Thursday, November 05, 2015 6:31 PM
> To: dev@dpdk.org
> Cc: nakajima.yoshihiro@lab.ntt.co.jp; zhbzg@huawei.com; mst@redhat.com; gaoxiaoqiu@huawei.com; oscar.zhangbo@huawei.com;
> ann.zhuangyanying@huawei.com; zhoujingbin@huawei.com; guohongzhen@huawei.com
> Subject: [dpdk-dev] [RFC 4/5] virtio/container: adjust memory initialization process
> 
> When using virtio for container, we should specify --no-huge so
> that in memory initialization, shm_open() is used to alloc memory
> from tmpfs filesystem /dev/shm/.
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ---
>  lib/librte_eal/common/include/rte_memory.h |  5 +++
>  lib/librte_eal/linuxapp/eal/eal_memory.c   | 58 ++++++++++++++++++++++++++++--
>  lib/librte_mempool/rte_mempool.c           | 16 ++++-----
>  3 files changed, 69 insertions(+), 10 deletions(-)
> 
> diff --git a/lib/librte_eal/common/include/rte_memory.h b/lib/librte_eal/common/include/rte_memory.h
> index 1bed415..9c1effc 100644
> --- a/lib/librte_eal/common/include/rte_memory.h
> +++ b/lib/librte_eal/common/include/rte_memory.h
> @@ -100,6 +100,7 @@ struct rte_memseg {
>  	int32_t socket_id;          /**< NUMA socket ID. */
>  	uint32_t nchannel;          /**< Number of channels. */
>  	uint32_t nrank;             /**< Number of ranks. */
> +	int fd;                     /**< fd used for share this memory */
>  #ifdef RTE_LIBRTE_XEN_DOM0
>  	 /**< store segment MFNs */
>  	uint64_t mfn[DOM0_NUM_MEMBLOCK];
> @@ -128,6 +129,10 @@ int rte_mem_lock_page(const void *virt);
>   */
>  phys_addr_t rte_mem_virt2phy(const void *virt);
> 
> +
> +int
> +rte_memseg_info_get(int index, int *pfd, uint64_t *psize, void **paddr);
> +
>  /**
>   * Get the layout of the available physical memory.
>   *
> diff --git a/lib/librte_eal/linuxapp/eal/eal_memory.c b/lib/librte_eal/linuxapp/eal/eal_memory.c
> index ac2745e..9abbfc6 100644
> --- a/lib/librte_eal/linuxapp/eal/eal_memory.c
> +++ b/lib/librte_eal/linuxapp/eal/eal_memory.c
> @@ -80,6 +80,9 @@
>  #include <errno.h>
>  #include <sys/ioctl.h>
>  #include <sys/time.h>
> +#include <mntent.h>
> +#include <sys/mman.h>
> +#include <sys/file.h>
> 
>  #include <rte_log.h>
>  #include <rte_memory.h>
> @@ -143,6 +146,18 @@ rte_mem_lock_page(const void *virt)
>  	return mlock((void*)aligned, page_size);
>  }
> 
> +int
> +rte_memseg_info_get(int index, int *pfd, uint64_t *psize, void **paddr)
> +{
> +	struct rte_mem_config *mcfg;
> +	mcfg = rte_eal_get_configuration()->mem_config;
> +
> +	*pfd = mcfg->memseg[index].fd;
> +	*psize = (uint64_t)mcfg->memseg[index].len;
> +	*paddr = (void *)(uint64_t)mcfg->memseg[index].addr;
> +	return 0;
> +}

Wonder who will use that function?
Can't see any references to that function in that patch or next. 

> +
>  /*
>   * Get physical address of any mapped virtual address in the current process.
>   */
> @@ -1044,6 +1059,42 @@ calc_num_pages_per_socket(uint64_t * memory,
>  	return total_num_pages;
>  }
> 
> +static void *
> +rte_eal_shm_create(int *pfd)
> +{
> +	int ret, fd;
> +	char filepath[256];
> +	void *vaddr;
> +	uint64_t size = internal_config.memory;
> +
> +	sprintf(filepath, "/%s_cvio", internal_config.hugefile_prefix);
> +
> +	fd = shm_open(filepath, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR);
> +	if (fd < 0) {
> +		rte_panic("shm_open %s failed: %s\n", filepath, strerror(errno));
> +	}
> +	ret = flock(fd, LOCK_EX);
> +	if (ret < 0) {
> +		close(fd);
> +		rte_panic("flock %s failed: %s\n", filepath, strerror(errno));
> +	}
> +
> +	ret = ftruncate(fd, size);
> +	if (ret < 0) {
> +		rte_panic("ftruncate failed: %s\n", strerror(errno));
> +	}
> +	/* flag: MAP_HUGETLB */

Any explanation what that comment means here?
Do you plan to use MAP_HUGETLb in the call below or ...?

> +	vaddr = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0);
> +	if (vaddr == MAP_FAILED) {
> +		rte_panic("mmap failed: %s\n", strerror(errno));
> +	}
> +	memset(vaddr, 0, size);
> +	*pfd = fd;
> +
> +	return vaddr;
> +}
> +
> +
>  /*
>   * Prepare physical memory mapping: fill configuration structure with
>   * these infos, return 0 on success.
> @@ -1072,7 +1123,9 @@ rte_eal_hugepage_init(void)
>  	int new_pages_count[MAX_HUGEPAGE_SIZES];
>  #endif
> 
> +#ifndef RTE_VIRTIO_VDEV
>  	test_proc_pagemap_readable();
> +#endif
> 
>  	memset(used_hp, 0, sizeof(used_hp));
> 
> @@ -1081,8 +1134,8 @@ rte_eal_hugepage_init(void)
> 
>  	/* hugetlbfs can be disabled */
>  	if (internal_config.no_hugetlbfs) {
> -		addr = mmap(NULL, internal_config.memory, PROT_READ | PROT_WRITE,
> -				MAP_PRIVATE | MAP_ANONYMOUS, 0, 0);
> +		int fd;
> +		addr = rte_eal_shm_create(&fd);

Why do you remove ability to map(dev/zero) here?
Probably not everyone plan to use --no-hugepages only inside containers. 


>  		if (addr == MAP_FAILED) {
>  			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n", __func__,
>  					strerror(errno));
> @@ -1093,6 +1146,7 @@ rte_eal_hugepage_init(void)
>  		mcfg->memseg[0].hugepage_sz = RTE_PGSIZE_4K;
>  		mcfg->memseg[0].len = internal_config.memory;
>  		mcfg->memseg[0].socket_id = 0;
> +		mcfg->memseg[0].fd = fd;
>  		return 0;
>  	}
> 
> diff --git a/lib/librte_mempool/rte_mempool.c b/lib/librte_mempool/rte_mempool.c
> index e57cbbd..8f8852b 100644
> --- a/lib/librte_mempool/rte_mempool.c
> +++ b/lib/librte_mempool/rte_mempool.c
> @@ -453,13 +453,6 @@ rte_mempool_xmem_create(const char *name, unsigned n, unsigned elt_size,
>  		rte_errno = EINVAL;
>  		return NULL;
>  	}
> -
> -	/* check that we have both VA and PA */
> -	if (vaddr != NULL && paddr == NULL) {
> -		rte_errno = EINVAL;
> -		return NULL;
> -	}
> -
>  	/* Check that pg_num and pg_shift parameters are valid. */
>  	if (pg_num < RTE_DIM(mp->elt_pa) || pg_shift > MEMPOOL_PG_SHIFT_MAX) {
>  		rte_errno = EINVAL;
> @@ -596,8 +589,15 @@ rte_mempool_xmem_create(const char *name, unsigned n, unsigned elt_size,
> 
>  	/* mempool elements in a separate chunk of memory. */
>  	} else {
> +		/* when VA is specified, PA should be specified? */
> +		if (rte_eal_has_hugepages()) {
> +			if (paddr == NULL) {
> +				rte_errno = EINVAL;
> +				return NULL;
> +			}
> +			memcpy(mp->elt_pa, paddr, sizeof (mp->elt_pa[0]) * pg_num);
> +		}
>  		mp->elt_va_start = (uintptr_t)vaddr;
> -		memcpy(mp->elt_pa, paddr, sizeof (mp->elt_pa[0]) * pg_num);

Could you explain the reason for that change?
Specially why mempool over external memory now only allowed for hugepages config?
Konstantin

>  	}
> 
>  	mp->elt_va_end = mp->elt_va_start;
> --
> 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [RFC 4/5] virtio/container: adjust memory initialization process
  2015-11-06 16:21   ` Ananyev, Konstantin
@ 2015-11-08 11:18     ` Tan, Jianfeng
  2015-11-09 13:32       ` Ananyev, Konstantin
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2015-11-08 11:18 UTC (permalink / raw)
  To: Ananyev, Konstantin, dev
  Cc: nakajima.yoshihiro, zhbzg, mst, gaoxiaoqiu, oscar.zhangbo,
	ann.zhuangyanying, zhoujingbin, guohongzhen



> -----Original Message-----
> From: Ananyev, Konstantin
> Sent: Saturday, November 7, 2015 12:22 AM
> To: Tan, Jianfeng; dev@dpdk.org
> Cc: nakajima.yoshihiro@lab.ntt.co.jp; zhbzg@huawei.com; mst@redhat.com;
> gaoxiaoqiu@huawei.com; oscar.zhangbo@huawei.com;
> ann.zhuangyanying@huawei.com; zhoujingbin@huawei.com;
> guohongzhen@huawei.com
> Subject: RE: [dpdk-dev] [RFC 4/5] virtio/container: adjust memory
> initialization process
> 
> Hi,
> 
> > -----Original Message-----
> > From: dev [mailto:dev-bounces@dpdk.org] On Behalf Of Jianfeng Tan
> > Sent: Thursday, November 05, 2015 6:31 PM
> > To: dev@dpdk.org
> > Cc: nakajima.yoshihiro@lab.ntt.co.jp; zhbzg@huawei.com;
> > mst@redhat.com; gaoxiaoqiu@huawei.com; oscar.zhangbo@huawei.com;
> > ann.zhuangyanying@huawei.com; zhoujingbin@huawei.com;
> > guohongzhen@huawei.com
> > Subject: [dpdk-dev] [RFC 4/5] virtio/container: adjust memory
> > initialization process
> >
> > When using virtio for container, we should specify --no-huge so that
> > in memory initialization, shm_open() is used to alloc memory from
> > tmpfs filesystem /dev/shm/.
> >
> > Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> > Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> > ---
......
> > +int
> > +rte_memseg_info_get(int index, int *pfd, uint64_t *psize, void
> > +**paddr) {
> > +	struct rte_mem_config *mcfg;
> > +	mcfg = rte_eal_get_configuration()->mem_config;
> > +
> > +	*pfd = mcfg->memseg[index].fd;
> > +	*psize = (uint64_t)mcfg->memseg[index].len;
> > +	*paddr = (void *)(uint64_t)mcfg->memseg[index].addr;
> > +	return 0;
> > +}
> 
> Wonder who will use that function?
> Can't see any references to that function in that patch or next.

This function is used in 1/5, when virtio front end needs to send VHOST_USER_SET_MEM_TABLE to back end.

> > +
> >  /*
> >   * Get physical address of any mapped virtual address in the current
> process.
> >   */
> > @@ -1044,6 +1059,42 @@ calc_num_pages_per_socket(uint64_t *
> memory,
> >  	return total_num_pages;
> >  }
> >
> > +static void *
> > +rte_eal_shm_create(int *pfd)
> > +{
> > +	int ret, fd;
> > +	char filepath[256];
> > +	void *vaddr;
> > +	uint64_t size = internal_config.memory;
> > +
> > +	sprintf(filepath, "/%s_cvio", internal_config.hugefile_prefix);
> > +
> > +	fd = shm_open(filepath, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR);
> > +	if (fd < 0) {
> > +		rte_panic("shm_open %s failed: %s\n", filepath,
> strerror(errno));
> > +	}
> > +	ret = flock(fd, LOCK_EX);
> > +	if (ret < 0) {
> > +		close(fd);
> > +		rte_panic("flock %s failed: %s\n", filepath, strerror(errno));
> > +	}
> > +
> > +	ret = ftruncate(fd, size);
> > +	if (ret < 0) {
> > +		rte_panic("ftruncate failed: %s\n", strerror(errno));
> > +	}
> > +	/* flag: MAP_HUGETLB */
> 
> Any explanation what that comment means here?
> Do you plan to use MAP_HUGETLb in the call below or ...?

Yes, it's a todo item. Shm_open() just uses a tmpfs mounted at /dev/shm. So I wonder maybe we can use this flag to make
sure  os allocates hugepages here if user would like to use hugepages.

>>
......
> > @@ -1081,8 +1134,8 @@ rte_eal_hugepage_init(void)
> >
> >  	/* hugetlbfs can be disabled */
> >  	if (internal_config.no_hugetlbfs) {
> > -		addr = mmap(NULL, internal_config.memory, PROT_READ |
> PROT_WRITE,
> > -				MAP_PRIVATE | MAP_ANONYMOUS, 0, 0);
> > +		int fd;
> > +		addr = rte_eal_shm_create(&fd);
> 
> Why do you remove ability to map(dev/zero) here?
> Probably not everyone plan to use --no-hugepages only inside containers.

>From my understanding, mmap here is just to allocate some memory, which is initialized to be all zero. I cannot understand what's
the relationship with /dev/zero. rte_eal_shm_create() can do the original function, plus it generates a fd to point to this chunk of
memory. This fd is indispensable in vhost protocol when VHOST_USER_SET_MEM_TABLE using sendmsg().

> 
> 
> >  		if (addr == MAP_FAILED) {
> >  			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n",
> __func__,
> >  					strerror(errno));
> > @@ -1093,6 +1146,7 @@ rte_eal_hugepage_init(void)
> >  		mcfg->memseg[0].hugepage_sz = RTE_PGSIZE_4K;
> >  		mcfg->memseg[0].len = internal_config.memory;
> >  		mcfg->memseg[0].socket_id = 0;
> > +		mcfg->memseg[0].fd = fd;
> >  		return 0;
> >  	}
> >
> > diff --git a/lib/librte_mempool/rte_mempool.c
> > b/lib/librte_mempool/rte_mempool.c
> > index e57cbbd..8f8852b 100644
> > --- a/lib/librte_mempool/rte_mempool.c
> > +++ b/lib/librte_mempool/rte_mempool.c
> > @@ -453,13 +453,6 @@ rte_mempool_xmem_create(const char *name,
> unsigned n, unsigned elt_size,
> >  		rte_errno = EINVAL;
> >  		return NULL;
> >  	}
> > -
> > -	/* check that we have both VA and PA */
> > -	if (vaddr != NULL && paddr == NULL) {
> > -		rte_errno = EINVAL;
> > -		return NULL;
> > -	}
> > -
> >  	/* Check that pg_num and pg_shift parameters are valid. */
> >  	if (pg_num < RTE_DIM(mp->elt_pa) || pg_shift >
> MEMPOOL_PG_SHIFT_MAX) {
> >  		rte_errno = EINVAL;
> > @@ -596,8 +589,15 @@ rte_mempool_xmem_create(const char *name,
> > unsigned n, unsigned elt_size,
> >
> >  	/* mempool elements in a separate chunk of memory. */
> >  	} else {
> > +		/* when VA is specified, PA should be specified? */
> > +		if (rte_eal_has_hugepages()) {
> > +			if (paddr == NULL) {
> > +				rte_errno = EINVAL;
> > +				return NULL;
> > +			}
> > +			memcpy(mp->elt_pa, paddr, sizeof (mp->elt_pa[0])
> * pg_num);
> > +		}
> >  		mp->elt_va_start = (uintptr_t)vaddr;
> > -		memcpy(mp->elt_pa, paddr, sizeof (mp->elt_pa[0]) *
> pg_num);
> 
> Could you explain the reason for that change?
> Specially why mempool over external memory now only allowed for
> hugepages config?
> Konstantin

Oops, you're right! This change was previously for creating a mbuf mempool at a given vaddr and without
giving any paddr[].  And now we don't need to care about neither vaddr nor paddr[] so I should have reverted
change in this file.

> 
> >  	}
> >
> >  	mp->elt_va_end = mp->elt_va_start;
> > --
> > 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [RFC 5/5] vhost/container: change mode of vhost listening socket
  2015-11-05 18:31 ` [RFC 5/5] vhost/container: change mode of vhost listening socket Jianfeng Tan
@ 2015-11-09  3:54   ` Yuanhan Liu
  2015-11-09  5:15     ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2015-11-09  3:54 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: nakajima.yoshihiro, zhbzg, mst, dev, oscar.zhangbo, gaoxiaoqiu,
	ann.zhuangyanying, zhoujingbin, guohongzhen

On Fri, Nov 06, 2015 at 02:31:16AM +0800, Jianfeng Tan wrote:
> Change vhost listening socket mode so that users in groups and
> others can connect to vhost listening socket.
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ---
>  lib/librte_vhost/vhost_user/vhost-net-user.c | 5 +++++
>  1 file changed, 5 insertions(+)
> 
> diff --git a/lib/librte_vhost/vhost_user/vhost-net-user.c b/lib/librte_vhost/vhost_user/vhost-net-user.c
> index 2dc0547..7b24f7c 100644
> --- a/lib/librte_vhost/vhost_user/vhost-net-user.c
> +++ b/lib/librte_vhost/vhost_user/vhost-net-user.c
> @@ -42,6 +42,7 @@
>  #include <sys/un.h>
>  #include <errno.h>
>  #include <pthread.h>
> +#include <sys/stat.h>
>  
>  #include <rte_log.h>
>  #include <rte_virtio_net.h>
> @@ -137,6 +138,10 @@ uds_socket(const char *path)
>  	if (ret == -1)
>  		goto err;
>  
> +	ret = chmod(un.sun_path, 0666);
> +	if (ret == 0)
> +		RTE_LOG(INFO, VHOST_CONFIG, "chmod 0666, ok\n");

That doesn't seem right to me. Doing that kind of change in a libraray
doesn't seem to be a good practice, don't even to say changing it to
"0666" blindly, which allows every body to access it.

	--yliu

> +
>  	return sockfd;
>  
>  err:
> -- 
> 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [RFC 5/5] vhost/container: change mode of vhost listening socket
  2015-11-09  3:54   ` Yuanhan Liu
@ 2015-11-09  5:15     ` Tan, Jianfeng
  2015-11-09  5:40       ` Yuanhan Liu
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2015-11-09  5:15 UTC (permalink / raw)
  To: Yuanhan Liu
  Cc: nakajima.yoshihiro, zhbzg, mst, dev, oscar.zhangbo, gaoxiaoqiu,
	ann.zhuangyanying, zhoujingbin, guohongzhen



> -----Original Message-----
> From: Yuanhan Liu [mailto:yuanhan.liu@linux.intel.com]
> Sent: Monday, November 9, 2015 11:55 AM
> To: Tan, Jianfeng
> Cc: dev@dpdk.org; nakajima.yoshihiro@lab.ntt.co.jp; zhbzg@huawei.com;
> mst@redhat.com; gaoxiaoqiu@huawei.com; oscar.zhangbo@huawei.com;
> ann.zhuangyanying@huawei.com; zhoujingbin@huawei.com;
> guohongzhen@huawei.com
> Subject: Re: [dpdk-dev] [RFC 5/5] vhost/container: change mode of vhost
> listening socket
> 
> On Fri, Nov 06, 2015 at 02:31:16AM +0800, Jianfeng Tan wrote:
> > Change vhost listening socket mode so that users in groups and others
> > can connect to vhost listening socket.
> >
> > Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> > Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> > ---
> >  lib/librte_vhost/vhost_user/vhost-net-user.c | 5 +++++
> >  1 file changed, 5 insertions(+)
> >
> > diff --git a/lib/librte_vhost/vhost_user/vhost-net-user.c
> > b/lib/librte_vhost/vhost_user/vhost-net-user.c
> > index 2dc0547..7b24f7c 100644
> > --- a/lib/librte_vhost/vhost_user/vhost-net-user.c
> > +++ b/lib/librte_vhost/vhost_user/vhost-net-user.c
> > @@ -42,6 +42,7 @@
> >  #include <sys/un.h>
> >  #include <errno.h>
> >  #include <pthread.h>
> > +#include <sys/stat.h>
> >
> >  #include <rte_log.h>
> >  #include <rte_virtio_net.h>
> > @@ -137,6 +138,10 @@ uds_socket(const char *path)
> >  	if (ret == -1)
> >  		goto err;
> >
> > +	ret = chmod(un.sun_path, 0666);
> > +	if (ret == 0)
> > +		RTE_LOG(INFO, VHOST_CONFIG, "chmod 0666, ok\n");
> 
> That doesn't seem right to me. Doing that kind of change in a libraray doesn't
> seem to be a good practice, don't even to say changing it to "0666" blindly,
> which allows every body to access it.
> 
> 	--yliu

Hi Yuanhan,

The original intention for this change is for the use case: use "root" to
start ovs-dpdk (or any other switch application), but use other users to
run some containers. Not with this change, other users cannot connect
to vhost listening socket.

This change is not necessary if using root to start a container. It's indeed
a question worth discussion: whether it's reasonable to allow everybody
to start a virtio device.

Thanks,
Jianfeng

> 
> > +
> >  	return sockfd;
> >
> >  err:
> > --
> > 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [RFC 5/5] vhost/container: change mode of vhost listening socket
  2015-11-09  5:15     ` Tan, Jianfeng
@ 2015-11-09  5:40       ` Yuanhan Liu
  2015-11-09  5:46         ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2015-11-09  5:40 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: nakajima.yoshihiro, zhbzg, mst, dev, oscar.zhangbo, gaoxiaoqiu,
	ann.zhuangyanying, zhoujingbin, guohongzhen

On Mon, Nov 09, 2015 at 05:15:23AM +0000, Tan, Jianfeng wrote:
...
> > >
> > > +	ret = chmod(un.sun_path, 0666);
> > > +	if (ret == 0)
> > > +		RTE_LOG(INFO, VHOST_CONFIG, "chmod 0666, ok\n");
> > 
> > That doesn't seem right to me. Doing that kind of change in a libraray doesn't
> > seem to be a good practice, don't even to say changing it to "0666" blindly,
> > which allows every body to access it.
> > 
> > 	--yliu
> 
> Hi Yuanhan,
> 
> The original intention for this change is for the use case: use "root" to
> start ovs-dpdk (or any other switch application), but use other users to
> run some containers. Not with this change, other users cannot connect
> to vhost listening socket.

I know your concern, do it with some user space utils (like chmod) then,
but not in a libraray.

BTW, "chown", limiting it to a specific user, or "chmod g+rw", limiting
it to a specific group, is more appropriate here.

	--yliu
> 
> This change is not necessary if using root to start a container. It's indeed
> a question worth discussion: whether it's reasonable to allow everybody
> to start a virtio device.
> 
> Thanks,
> Jianfeng
> 
> > 
> > > +
> > >  	return sockfd;
> > >
> > >  err:
> > > --
> > > 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [RFC 5/5] vhost/container: change mode of vhost listening socket
  2015-11-09  5:40       ` Yuanhan Liu
@ 2015-11-09  5:46         ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2015-11-09  5:46 UTC (permalink / raw)
  To: Yuanhan Liu
  Cc: nakajima.yoshihiro, zhbzg, mst, dev, oscar.zhangbo, gaoxiaoqiu,
	ann.zhuangyanying, zhoujingbin, guohongzhen



> -----Original Message-----
> From: Yuanhan Liu [mailto:yuanhan.liu@linux.intel.com]
> Sent: Monday, November 9, 2015 1:41 PM
> To: Tan, Jianfeng
> Cc: dev@dpdk.org; nakajima.yoshihiro@lab.ntt.co.jp; zhbzg@huawei.com;
> mst@redhat.com; gaoxiaoqiu@huawei.com; oscar.zhangbo@huawei.com;
> ann.zhuangyanying@huawei.com; zhoujingbin@huawei.com;
> guohongzhen@huawei.com
> Subject: Re: [dpdk-dev] [RFC 5/5] vhost/container: change mode of vhost
> listening socket
> 
> On Mon, Nov 09, 2015 at 05:15:23AM +0000, Tan, Jianfeng wrote:
> ...
> > > >
> > > > +	ret = chmod(un.sun_path, 0666);
> > > > +	if (ret == 0)
> > > > +		RTE_LOG(INFO, VHOST_CONFIG, "chmod 0666, ok\n");
> > >
> > > That doesn't seem right to me. Doing that kind of change in a
> > > libraray doesn't seem to be a good practice, don't even to say
> > > changing it to "0666" blindly, which allows every body to access it.
> > >
> > > 	--yliu
> >
> > Hi Yuanhan,
> >
> > The original intention for this change is for the use case: use "root"
> > to start ovs-dpdk (or any other switch application), but use other
> > users to run some containers. Not with this change, other users cannot
> > connect to vhost listening socket.
> 
> I know your concern, do it with some user space utils (like chmod) then, but
> not in a libraray.
> 
> BTW, "chown", limiting it to a specific user, or "chmod g+rw", limiting it to a
> specific group, is more appropriate here.
> 
> 	--yliu

Got your point. Consider to revert this change in next version.

Thanks!
Jianfeng

> >
> > This change is not necessary if using root to start a container. It's
> > indeed a question worth discussion: whether it's reasonable to allow
> > everybody to start a virtio device.
> >
> > Thanks,
> > Jianfeng
> >
> > >
> > > > +
> > > >  	return sockfd;
> > > >
> > > >  err:
> > > > --
> > > > 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [RFC 4/5] virtio/container: adjust memory initialization process
  2015-11-08 11:18     ` Tan, Jianfeng
@ 2015-11-09 13:32       ` Ananyev, Konstantin
  2015-11-09 14:13         ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Ananyev, Konstantin @ 2015-11-09 13:32 UTC (permalink / raw)
  To: Tan, Jianfeng, dev
  Cc: nakajima.yoshihiro, zhbzg, mst, gaoxiaoqiu, oscar.zhangbo,
	ann.zhuangyanying, zhoujingbin, guohongzhen



> 
> > -----Original Message-----
> > From: Ananyev, Konstantin
> > Sent: Saturday, November 7, 2015 12:22 AM
> > To: Tan, Jianfeng; dev@dpdk.org
> > Cc: nakajima.yoshihiro@lab.ntt.co.jp; zhbzg@huawei.com; mst@redhat.com;
> > gaoxiaoqiu@huawei.com; oscar.zhangbo@huawei.com;
> > ann.zhuangyanying@huawei.com; zhoujingbin@huawei.com;
> > guohongzhen@huawei.com
> > Subject: RE: [dpdk-dev] [RFC 4/5] virtio/container: adjust memory
> > initialization process
> >
> > Hi,
> >
> > > -----Original Message-----
> > > From: dev [mailto:dev-bounces@dpdk.org] On Behalf Of Jianfeng Tan
> > > Sent: Thursday, November 05, 2015 6:31 PM
> > > To: dev@dpdk.org
> > > Cc: nakajima.yoshihiro@lab.ntt.co.jp; zhbzg@huawei.com;
> > > mst@redhat.com; gaoxiaoqiu@huawei.com; oscar.zhangbo@huawei.com;
> > > ann.zhuangyanying@huawei.com; zhoujingbin@huawei.com;
> > > guohongzhen@huawei.com
> > > Subject: [dpdk-dev] [RFC 4/5] virtio/container: adjust memory
> > > initialization process
> > >
> > > When using virtio for container, we should specify --no-huge so that
> > > in memory initialization, shm_open() is used to alloc memory from
> > > tmpfs filesystem /dev/shm/.
> > >
> > > Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> > > Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> > > ---
> ......
> > > +int
> > > +rte_memseg_info_get(int index, int *pfd, uint64_t *psize, void
> > > +**paddr) {
> > > +	struct rte_mem_config *mcfg;
> > > +	mcfg = rte_eal_get_configuration()->mem_config;
> > > +
> > > +	*pfd = mcfg->memseg[index].fd;
> > > +	*psize = (uint64_t)mcfg->memseg[index].len;
> > > +	*paddr = (void *)(uint64_t)mcfg->memseg[index].addr;
> > > +	return 0;
> > > +}
> >
> > Wonder who will use that function?
> > Can't see any references to that function in that patch or next.
> 
> This function is used in 1/5, when virtio front end needs to send VHOST_USER_SET_MEM_TABLE to back end.

Ok, but hen this function should be defined in the patch *before* it is used, not after.
Another thing: probably better to create a struct for all memseg parameters you want to retrieve,
and pass it to the function, instead of several pointers. 

> 
> > > +
> > >  /*
> > >   * Get physical address of any mapped virtual address in the current
> > process.
> > >   */
> > > @@ -1044,6 +1059,42 @@ calc_num_pages_per_socket(uint64_t *
> > memory,
> > >  	return total_num_pages;
> > >  }
> > >
> > > +static void *
> > > +rte_eal_shm_create(int *pfd)
> > > +{
> > > +	int ret, fd;
> > > +	char filepath[256];
> > > +	void *vaddr;
> > > +	uint64_t size = internal_config.memory;
> > > +
> > > +	sprintf(filepath, "/%s_cvio", internal_config.hugefile_prefix);
> > > +
> > > +	fd = shm_open(filepath, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR);
> > > +	if (fd < 0) {
> > > +		rte_panic("shm_open %s failed: %s\n", filepath,
> > strerror(errno));
> > > +	}
> > > +	ret = flock(fd, LOCK_EX);
> > > +	if (ret < 0) {
> > > +		close(fd);
> > > +		rte_panic("flock %s failed: %s\n", filepath, strerror(errno));
> > > +	}
> > > +
> > > +	ret = ftruncate(fd, size);
> > > +	if (ret < 0) {
> > > +		rte_panic("ftruncate failed: %s\n", strerror(errno));
> > > +	}
> > > +	/* flag: MAP_HUGETLB */
> >
> > Any explanation what that comment means here?
> > Do you plan to use MAP_HUGETLb in the call below or ...?
> 
> Yes, it's a todo item. Shm_open() just uses a tmpfs mounted at /dev/shm. So I wonder maybe we can use this flag to make
> sure  os allocates hugepages here if user would like to use hugepages.
> 
> >>
> ......
> > > @@ -1081,8 +1134,8 @@ rte_eal_hugepage_init(void)
> > >
> > >  	/* hugetlbfs can be disabled */
> > >  	if (internal_config.no_hugetlbfs) {
> > > -		addr = mmap(NULL, internal_config.memory, PROT_READ |
> > PROT_WRITE,
> > > -				MAP_PRIVATE | MAP_ANONYMOUS, 0, 0);
> > > +		int fd;
> > > +		addr = rte_eal_shm_create(&fd);
> >
> > Why do you remove ability to map(dev/zero) here?
> > Probably not everyone plan to use --no-hugepages only inside containers.
> 
> From my understanding, mmap here is just to allocate some memory, which is initialized to be all zero. I cannot understand what's
> the relationship with /dev/zero.

I used it here as a synonym for mmap(, ..., MAP_ANONYMOUS,...).

 rte_eal_shm_create() can do the original function, plus it generates a fd to point to this chunk of
> memory. This fd is indispensable in vhost protocol when VHOST_USER_SET_MEM_TABLE using sendmsg().


My question was:
Right now for --no-hugepages it allocates a chunk of memory that is not backed-up by any file and is private to the process:

addr = mmap(NULL, internal_config.memory, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, 0, 0); 

You changed it to shared memory region allocation:

fd = shm_open(filepath, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR);
addr = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0);

I understand that you need it for your containers stuff - but I suppose you have to add
new functionality without breaking existing one>
There could be other users of --no-hugepages and they probably want existing behaviour.
Konstantin

> 
> >
> >
> > >  		if (addr == MAP_FAILED) {
> > >  			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n",
> > __func__,
> > >  					strerror(errno));
> > > @@ -1093,6 +1146,7 @@ rte_eal_hugepage_init(void)
> > >  		mcfg->memseg[0].hugepage_sz = RTE_PGSIZE_4K;
> > >  		mcfg->memseg[0].len = internal_config.memory;
> > >  		mcfg->memseg[0].socket_id = 0;
> > > +		mcfg->memseg[0].fd = fd;
> > >  		return 0;
> > >  	}
> > >
> > > diff --git a/lib/librte_mempool/rte_mempool.c
> > > b/lib/librte_mempool/rte_mempool.c
> > > index e57cbbd..8f8852b 100644
> > > --- a/lib/librte_mempool/rte_mempool.c
> > > +++ b/lib/librte_mempool/rte_mempool.c
> > > @@ -453,13 +453,6 @@ rte_mempool_xmem_create(const char *name,
> > unsigned n, unsigned elt_size,
> > >  		rte_errno = EINVAL;
> > >  		return NULL;
> > >  	}
> > > -
> > > -	/* check that we have both VA and PA */
> > > -	if (vaddr != NULL && paddr == NULL) {
> > > -		rte_errno = EINVAL;
> > > -		return NULL;
> > > -	}
> > > -
> > >  	/* Check that pg_num and pg_shift parameters are valid. */
> > >  	if (pg_num < RTE_DIM(mp->elt_pa) || pg_shift >
> > MEMPOOL_PG_SHIFT_MAX) {
> > >  		rte_errno = EINVAL;
> > > @@ -596,8 +589,15 @@ rte_mempool_xmem_create(const char *name,
> > > unsigned n, unsigned elt_size,
> > >
> > >  	/* mempool elements in a separate chunk of memory. */
> > >  	} else {
> > > +		/* when VA is specified, PA should be specified? */
> > > +		if (rte_eal_has_hugepages()) {
> > > +			if (paddr == NULL) {
> > > +				rte_errno = EINVAL;
> > > +				return NULL;
> > > +			}
> > > +			memcpy(mp->elt_pa, paddr, sizeof (mp->elt_pa[0])
> > * pg_num);
> > > +		}
> > >  		mp->elt_va_start = (uintptr_t)vaddr;
> > > -		memcpy(mp->elt_pa, paddr, sizeof (mp->elt_pa[0]) *
> > pg_num);
> >
> > Could you explain the reason for that change?
> > Specially why mempool over external memory now only allowed for
> > hugepages config?
> > Konstantin
> 
> Oops, you're right! This change was previously for creating a mbuf mempool at a given vaddr and without
> giving any paddr[].  And now we don't need to care about neither vaddr nor paddr[] so I should have reverted
> change in this file.
> 
> >
> > >  	}
> > >
> > >  	mp->elt_va_end = mp->elt_va_start;
> > > --
> > > 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [RFC 4/5] virtio/container: adjust memory initialization process
  2015-11-09 13:32       ` Ananyev, Konstantin
@ 2015-11-09 14:13         ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2015-11-09 14:13 UTC (permalink / raw)
  To: Ananyev, Konstantin, dev
  Cc: nakajima.yoshihiro, zhbzg, mst, gaoxiaoqiu, oscar.zhangbo,
	ann.zhuangyanying, zhoujingbin, guohongzhen



> ......
> > > > +int
> > > > +rte_memseg_info_get(int index, int *pfd, uint64_t *psize, void
> > > > +**paddr) {
> > > > +	struct rte_mem_config *mcfg;
> > > > +	mcfg = rte_eal_get_configuration()->mem_config;
> > > > +
> > > > +	*pfd = mcfg->memseg[index].fd;
> > > > +	*psize = (uint64_t)mcfg->memseg[index].len;
> > > > +	*paddr = (void *)(uint64_t)mcfg->memseg[index].addr;
> > > > +	return 0;
> > > > +}
> > >
> > > Wonder who will use that function?
> > > Can't see any references to that function in that patch or next.
> >
> > This function is used in 1/5, when virtio front end needs to send
> VHOST_USER_SET_MEM_TABLE to back end.
> 
> Ok, but hen this function should be defined in the patch *before* it is used,
> not after.
> Another thing: probably better to create a struct for all memseg parameters
> you want to retrieve, and pass it to the function, instead of several pointers.

Very good suggestion! I'll fix it in next version.

> > > > +		addr = rte_eal_shm_create(&fd);
> > >
> > > Why do you remove ability to map(dev/zero) here?
> > > Probably not everyone plan to use --no-hugepages only inside containers.
> >
> > From my understanding, mmap here is just to allocate some memory,
> > which is initialized to be all zero. I cannot understand what's the
> relationship with /dev/zero.
> 
> I used it here as a synonym for mmap(, ..., MAP_ANONYMOUS,...).
> 
>  rte_eal_shm_create() can do the original function, plus it generates a fd to
> point to this chunk of
> > memory. This fd is indispensable in vhost protocol when
> VHOST_USER_SET_MEM_TABLE using sendmsg().
> 
> 
> My question was:
> Right now for --no-hugepages it allocates a chunk of memory that is not
> backed-up by any file and is private to the process:
> 
> addr = mmap(NULL, internal_config.memory, PROT_READ | PROT_WRITE,
> MAP_PRIVATE | MAP_ANONYMOUS, 0, 0);
> 
> You changed it to shared memory region allocation:
> 
> fd = shm_open(filepath, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR); addr =
> mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0);
> 
> I understand that you need it for your containers stuff - but I suppose you
> have to add new functionality without breaking existing one> There could be
> other users of --no-hugepages and they probably want existing behaviour.
> Konstantin

Thank you for patient analysis and I agree with you. I should have not break
compatibility with existing applications. I'd like to redesign this in next version.
Maybe a new cmd option is necessary here.

Jianfeng

.....
> > > > --
> > > > 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [RFC 0/5] virtio support for container
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
                   ` (4 preceding siblings ...)
  2015-11-05 18:31 ` [RFC 5/5] vhost/container: change mode of vhost listening socket Jianfeng Tan
@ 2015-11-24  3:53 ` Zhuangyanying
  2015-11-24  6:19   ` Tan, Jianfeng
  2016-01-10 11:42 ` [PATCH 0/4] " Jianfeng Tan
                   ` (8 subsequent siblings)
  14 siblings, 1 reply; 196+ messages in thread
From: Zhuangyanying @ 2015-11-24  3:53 UTC (permalink / raw)
  To: Jianfeng Tan, dev
  Cc: nakajima.yoshihiro, Zhbzg, mst, gaoxiaoqiu, Zhangbo (Oscar),
	Zhoujingbin, Guohongzhen



> -----Original Message-----
> From: Jianfeng Tan [mailto:jianfeng.tan@intel.com]
> Sent: Friday, November 06, 2015 2:31 AM
> To: dev@dpdk.org
> Cc: mst@redhat.com; mukawa@igel.co.jp; nakajima.yoshihiro@lab.ntt.co.jp;
> michael.qiu@intel.com; Guohongzhen; Zhoujingbin; Zhuangyanying; Zhangbo
> (Oscar); gaoxiaoqiu; Zhbzg; huawei.xie@intel.com; Jianfeng Tan
> Subject: [RFC 0/5] virtio support for container
> 
> This patchset only acts as a PoC to request the community for comments.
> 
> This patchset is to provide high performance networking interface
> (virtio) for container-based DPDK applications. The way of starting DPDK
> applications in containers with ownership of NIC devices exclusively is beyond
> the scope. The basic idea here is to present a new virtual device (named
> eth_cvio), which can be discovered and initialized in container-based DPDK
> applications rte_eal_init().
> To minimize the change, we reuse already-existing virtio frontend driver code
> (driver/net/virtio/).
> 
> Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
> operations into unix socket/cuse protocol, which is originally provided in QEMU),
> is integrated in virtio frontend driver. Aka, this new converged driver actually
> plays the role of original frontend driver and the role of QEMU device
> framework.
> 
> The biggest difference here lies in how to calculate relative address for backend.
> The principle of virtio is that: based on one or multiple shared memory
> segments, vhost maintains a reference system with the base addresses and
> length of these segments so that an address from VM comes (usually GPA,
> Guest Physical Address), vhost can translate it into self-recognizable address
> (aka VVA, Vhost Virtual Address). To decrease the overhead of address
> translation, we should maintain as few segments as better. In the context of
> virtual machines, GPA is always locally continuous. So it's a good choice. In
> container's case, CVA (Container Virtual Address) can be used. This means
> that:
> a. when set_base_addr, CVA address is used; b. when preparing RX's
> descriptors, CVA address is used; c. when transmitting packets, CVA is filled in
> TX's descriptors; d. in TX and CQ's header, CVA is used.
> 
> How to share memory? In VM's case, qemu always shares all physical layout to
> backend. But it's not feasible for a container, as a process, to share all virtual
> memory regions to backend. So only specified virtual memory regions (type is
> shared) are sent to backend. It leads to a limitation that only addresses in
> these areas can be used to transmit or receive packets. For now, the shared
> memory is created in /dev/shm using shm_open() in the memory initialization
> process.
> 
> How to use?
> 
> a. Apply the patch of virtio for container. We need two copies of patched code
> (referred as dpdk-app/ and dpdk-vhost/)
> 
> b. To compile container apps:
> $: cd dpdk-app
> $: vim config/common_linuxapp (uncomment "CONFIG_RTE_VIRTIO_VDEV=y")
> $: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> 
> c. To build a docker image using Dockerfile below.
> $: cat ./Dockerfile
> FROM ubuntu:latest
> WORKDIR /usr/src/dpdk
> COPY . /usr/src/dpdk
> CMD ["/usr/src/dpdk/examples/l2fwd/build/l2fwd", "-c", "0xc", "-n", "4",
> "--no-huge", "--no-pci",
> "--vdev=eth_cvio0,queue_num=256,rx=1,tx=1,cq=0,path=/var/run/usvhost",
> "--", "-p", "0x1"]
> $: docker build -t dpdk-app-l2fwd .
> 
> d. To compile vhost:
> $: cd dpdk-vhost
> $: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> 
> e. Start vhost-switch
> $: ./examples/vhost/build/vhost-switch -c 3 -n 4 --socket-mem 1024,1024 -- -p
> 0x1 --stats 1
> 
> f. Start docker
> $: docker run -i -t -v <path to vhost unix socket>:/var/run/usvhost
> dpdk-app-l2fwd
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> 
> Jianfeng Tan (5):
>   virtio/container: add handler for ioport rd/wr
>   virtio/container: add a new virtual device named eth_cvio
>   virtio/container: unify desc->addr assignment
>   virtio/container: adjust memory initialization process
>   vhost/container: change mode of vhost listening socket
> 
>  config/common_linuxapp                       |   5 +
>  drivers/net/virtio/Makefile                  |   4 +
>  drivers/net/virtio/vhost-user.c              | 433
> +++++++++++++++++++++++++++
>  drivers/net/virtio/vhost-user.h              | 137 +++++++++
>  drivers/net/virtio/virtio_ethdev.c           | 319 +++++++++++++++-----
>  drivers/net/virtio/virtio_ethdev.h           |  16 +
>  drivers/net/virtio/virtio_pci.h              |  32 +-
>  drivers/net/virtio/virtio_rxtx.c             |   9 +-
>  drivers/net/virtio/virtio_rxtx_simple.c      |   9 +-
>  drivers/net/virtio/virtqueue.h               |   9 +-
>  lib/librte_eal/common/include/rte_memory.h   |   5 +
>  lib/librte_eal/linuxapp/eal/eal_memory.c     |  58 +++-
>  lib/librte_mempool/rte_mempool.c             |  16 +-
>  lib/librte_vhost/vhost_user/vhost-net-user.c |   5 +
>  14 files changed, 967 insertions(+), 90 deletions(-)  create mode 100644
> drivers/net/virtio/vhost-user.c  create mode 100644
> drivers/net/virtio/vhost-user.h
> 
> --
> 2.1.4

This patch arose a good idea to add an extra abstracted IO layer,  which would make it simple to extend the function to the kernel mode switch(such as OVS). That's great.
But I have one question here: 
    it's the issue on VHOST_USER_SET_MEM_TABLE. you alloc memory from tmpfs filesyste, just one fd, could used rte_memseg_info_get() to 
	directly get the memory topology, However, things change in kernel-space, because mempool should be created on each container's
	hugetlbfs(rather than tmpfs), which is seperated from each other, at last, considering of the ioctl's parameter. 
       My solution is as follows for your reference:
/*
	reg = mem->regions;
	reg->guest_phys_addr = (__u64) ((struct virtqueue *)(dev->data->rx_queues[0]))->mpool->elt_va_start;
	reg->userspace_addr = reg->guest_phys_addr;
	reg->memory_size = ((struct virtqueue *)(dev->data->rx_queues[0]))->mpool->elt_va_end - reg->guest_phys_addr;

	reg = mem->regions + 1;
	reg->guest_phys_addr = (__u64)(((struct virtqueue *)(dev->data->tx_queues[0]))->virtio_net_hdr_mem);
	reg->userspace_addr = reg->guest_phys_addr;
	reg->memory_size = vq_size * internals->vtnet_hdr_size;
*/	  
	   But it's a little ugly, any better idea?

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [RFC 0/5] virtio support for container
  2015-11-24  3:53 ` [RFC 0/5] virtio support for container Zhuangyanying
@ 2015-11-24  6:19   ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2015-11-24  6:19 UTC (permalink / raw)
  To: Zhuangyanying, dev
  Cc: nakajima.yoshihiro, Zhbzg, mst, gaoxiaoqiu, Zhangbo (Oscar),
	Zhoujingbin, Guohongzhen



> -----Original Message-----
> From: Zhuangyanying [mailto:ann.zhuangyanying@huawei.com]
> Sent: Tuesday, November 24, 2015 11:53 AM
> To: Tan, Jianfeng; dev@dpdk.org
> Cc: mst@redhat.com; mukawa@igel.co.jp; nakajima.yoshihiro@lab.ntt.co.jp;
> Qiu, Michael; Guohongzhen; Zhoujingbin; Zhangbo (Oscar); gaoxiaoqiu;
> Zhbzg; Xie, Huawei
> Subject: RE: [RFC 0/5] virtio support for container
> 
> 
> 
> > -----Original Message-----
> > From: Jianfeng Tan [mailto:jianfeng.tan@intel.com]
> > Sent: Friday, November 06, 2015 2:31 AM
> > To: dev@dpdk.org
> > Cc: mst@redhat.com; mukawa@igel.co.jp;
> nakajima.yoshihiro@lab.ntt.co.jp;
> > michael.qiu@intel.com; Guohongzhen; Zhoujingbin; Zhuangyanying;
> Zhangbo
> > (Oscar); gaoxiaoqiu; Zhbzg; huawei.xie@intel.com; Jianfeng Tan
> > Subject: [RFC 0/5] virtio support for container
> >
...
> > 2.1.4
> 
> This patch arose a good idea to add an extra abstracted IO layer,  which
> would make it simple to extend the function to the kernel mode switch(such
> as OVS). That's great.
> But I have one question here:
>     it's the issue on VHOST_USER_SET_MEM_TABLE. you alloc memory from
> tmpfs filesyste, just one fd, could used rte_memseg_info_get() to
> 	directly get the memory topology, However, things change in kernel-
> space, because mempool should be created on each container's
> 	hugetlbfs(rather than tmpfs), which is seperated from each other, at
> last, considering of the ioctl's parameter.
>        My solution is as follows for your reference:
> /*
> 	reg = mem->regions;
> 	reg->guest_phys_addr = (__u64) ((struct virtqueue *)(dev->data-
> >rx_queues[0]))->mpool->elt_va_start;
> 	reg->userspace_addr = reg->guest_phys_addr;
> 	reg->memory_size = ((struct virtqueue *)(dev->data-
> >rx_queues[0]))->mpool->elt_va_end - reg->guest_phys_addr;
> 
> 	reg = mem->regions + 1;
> 	reg->guest_phys_addr = (__u64)(((struct virtqueue *)(dev->data-
> >tx_queues[0]))->virtio_net_hdr_mem);
> 	reg->userspace_addr = reg->guest_phys_addr;
> 	reg->memory_size = vq_size * internals->vtnet_hdr_size;
> */
> 	   But it's a little ugly, any better idea?

Hi Yanying,

Your solution seems ok for me when used with kernel vhost-net, because vhost
kthread just shares the same mm_struct with virtio process. But it will not work
with vhost-user, which realize memory sharing through putting fd in sendmsg().
Worse, it will not work with userspace vhost_cuse (see
lib/librte_vhost/vhost_cuse/), either, because current implementation supposes
VM's physical memory is backed by one huge file. Actually, what we need to do
Is enhancing userspace vhost_cuse, so that it supports cross-file memory region.

With below solutions to support hugetlbfs FYI:

To support hugetlbfs, my previous idea is to use -v option of "docker run"
to map hugetlbfs into its /dev/shm, so that we can create a "huge" shm file
on hugetlbfs. But this seems not accepted by others.

You mentioned the situation that DPDK now creates a file for each hugepage.
Maybe we just need to share all these hugepages with vhost. To minimize the
memory translation effort, we need to require that we use as few pages as
possible. Can you accept this solution?

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH 0/4] virtio support for container
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
                   ` (5 preceding siblings ...)
  2015-11-24  3:53 ` [RFC 0/5] virtio support for container Zhuangyanying
@ 2016-01-10 11:42 ` Jianfeng Tan
  2016-01-10 11:42   ` [PATCH 1/4] mem: add --single-file to create single mem-backed file Jianfeng Tan
                     ` (6 more replies)
  2016-02-05 11:20 ` [PATCH v2 0/5] " Jianfeng Tan
                   ` (7 subsequent siblings)
  14 siblings, 7 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-01-10 11:42 UTC (permalink / raw)
  To: dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

This patchset is to provide high performance networking interface (virtio)
for container-based DPDK applications. The way of starting DPDK apps in
containers with ownership of NIC devices exclusively is beyond the scope.
The basic idea here is to present a new virtual device (named eth_cvio),
which can be discovered and initialized in container-based DPDK apps using
rte_eal_init(). To minimize the change, we reuse already-existing virtio
frontend driver code (driver/net/virtio/).
 
Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
operations into unix socket/cuse protocol, which is originally provided in
QEMU), is integrated in virtio frontend driver. So this converged driver
actually plays the role of original frontend driver and the role of QEMU
device framework.
 
The major difference lies in how to calculate relative address for vhost.
The principle of virtio is that: based on one or multiple shared memory
segments, vhost maintains a reference system with the base addresses and
length for each segment so that an address from VM comes (usually GPA,
Guest Physical Address) can be translated into vhost-recognizable address
(named VVA, Vhost Virtual Address). To decrease the overhead of address
translation, we should maintain as few segments as possible. In VM's case,
GPA is always locally continuous. In container's case, CVA (Container
Virtual Address) can be used. Specifically:
a. when set_base_addr, CVA address is used;
b. when preparing RX's descriptors, CVA address is used;
c. when transmitting packets, CVA is filled in TX's descriptors;
d. in TX and CQ's header, CVA is used.
 
How to share memory? In VM's case, qemu always shares all physical layout
to backend. But it's not feasible for a container, as a process, to share
all virtual memory regions to backend. So only specified virtual memory
regions (with type of shared) are sent to backend. It's a limitation that
only addresses in these areas can be used to transmit or receive packets.

Known issues

a. When used with vhost-net, root privilege is required to create tap
device inside.
b. Control queue and multi-queue are not supported yet.
c. When --single-file option is used, socket_id of the memory may be
wrong. (Use "numactl -N x -m x" to work around this for now)
 
How to use?

a. Apply this patchset.

b. To compile container apps:
$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc

c. To build a docker image using Dockerfile below.
$: cat ./Dockerfile
FROM ubuntu:latest
WORKDIR /usr/src/dpdk
COPY . /usr/src/dpdk
ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
$: docker build -t dpdk-app-l2fwd .

d. Used with vhost-user
$: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
	--socket-mem 1024,1024 -- -p 0x1 --stats 1
$: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=eth_cvio0,path=/var/run/usvhost -- -p 0x1

f. Used with vhost-net
$: modprobe vhost
$: modprobe vhost-net
$: docker run -i -t --privileged \
	-v /dev/vhost-net:/dev/vhost-net \
	-v /dev/net/tun:/dev/net/tun \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=eth_cvio0,path=/dev/vhost-net -- -p 0x1

By the way, it's not necessary to run in a container.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>

Jianfeng Tan (4):
  mem: add --single-file to create single mem-backed file
  mem: add API to obstain memory-backed file info
  virtio/vdev: add ways to interact with vhost
  virtio/vdev: add a new vdev named eth_cvio

 config/common_linuxapp                     |   5 +
 drivers/net/virtio/Makefile                |   4 +
 drivers/net/virtio/vhost.c                 | 734 +++++++++++++++++++++++++++++
 drivers/net/virtio/vhost.h                 | 192 ++++++++
 drivers/net/virtio/virtio_ethdev.c         | 338 ++++++++++---
 drivers/net/virtio/virtio_ethdev.h         |   4 +
 drivers/net/virtio/virtio_pci.h            |  52 +-
 drivers/net/virtio/virtio_rxtx.c           |  11 +-
 drivers/net/virtio/virtio_rxtx_simple.c    |  14 +-
 drivers/net/virtio/virtqueue.h             |  13 +-
 lib/librte_eal/common/eal_common_options.c |  17 +
 lib/librte_eal/common/eal_internal_cfg.h   |   1 +
 lib/librte_eal/common/eal_options.h        |   2 +
 lib/librte_eal/common/include/rte_memory.h |  16 +
 lib/librte_eal/linuxapp/eal/eal_memory.c   |  82 +++-
 15 files changed, 1392 insertions(+), 93 deletions(-)
 create mode 100644 drivers/net/virtio/vhost.c
 create mode 100644 drivers/net/virtio/vhost.h

-- 
2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH 1/4] mem: add --single-file to create single mem-backed file
  2016-01-10 11:42 ` [PATCH 0/4] " Jianfeng Tan
@ 2016-01-10 11:42   ` Jianfeng Tan
  2016-01-21  1:57     ` Xie, Huawei
  2016-01-10 11:43   ` [PATCH 2/4] mem: add API to obstain memory-backed file info Jianfeng Tan
                     ` (5 subsequent siblings)
  6 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-01-10 11:42 UTC (permalink / raw)
  To: dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

Originally, there're two cons in using hugepage: a. needs root
privilege to touch /proc/self/pagemap, which is a premise to
alllocate physically contiguous memseg; b. possibly too many
hugepage file are created, especially used with 2M hugepage.

For virtual devices, they don't care about physical-contiguity
of allocated hugepages at all. Option --single-file is to
provide a way to allocate all hugepages into single mem-backed
file.

Known issue:
a. single-file option relys on kernel to allocate numa-affinitive
memory.
b. possible ABI break, originally, --no-huge uses anonymous memory
instead of file-backed way to create memory.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 lib/librte_eal/common/eal_common_options.c | 17 +++++++++++
 lib/librte_eal/common/eal_internal_cfg.h   |  1 +
 lib/librte_eal/common/eal_options.h        |  2 ++
 lib/librte_eal/linuxapp/eal/eal_memory.c   | 45 ++++++++++++++++++++++++++----
 4 files changed, 60 insertions(+), 5 deletions(-)

diff --git a/lib/librte_eal/common/eal_common_options.c b/lib/librte_eal/common/eal_common_options.c
index 29942ea..65bccbd 100644
--- a/lib/librte_eal/common/eal_common_options.c
+++ b/lib/librte_eal/common/eal_common_options.c
@@ -95,6 +95,7 @@ eal_long_options[] = {
 	{OPT_VFIO_INTR,         1, NULL, OPT_VFIO_INTR_NUM        },
 	{OPT_VMWARE_TSC_MAP,    0, NULL, OPT_VMWARE_TSC_MAP_NUM   },
 	{OPT_XEN_DOM0,          0, NULL, OPT_XEN_DOM0_NUM         },
+	{OPT_SINGLE_FILE,       0, NULL, OPT_SINGLE_FILE_NUM      },
 	{0,                     0, NULL, 0                        }
 };
 
@@ -897,6 +898,10 @@ eal_parse_common_option(int opt, const char *optarg,
 		}
 		break;
 
+	case OPT_SINGLE_FILE_NUM:
+		conf->single_file = 1;
+		break;
+
 	/* don't know what to do, leave this to caller */
 	default:
 		return 1;
@@ -956,6 +961,16 @@ eal_check_common_options(struct internal_config *internal_cfg)
 			"be specified together with --"OPT_NO_HUGE"\n");
 		return -1;
 	}
+	if (internal_cfg->single_file && internal_cfg->force_sockets == 1) {
+		RTE_LOG(ERR, EAL, "Option --"OPT_SINGLE_FILE" cannot "
+			"be specified together with --"OPT_SOCKET_MEM"\n");
+		return -1;
+	}
+	if (internal_cfg->single_file && internal_cfg->hugepage_unlink) {
+		RTE_LOG(ERR, EAL, "Option --"OPT_HUGE_UNLINK" cannot "
+			"be specified together with --"OPT_SINGLE_FILE"\n");
+		return -1;
+	}
 
 	if (internal_cfg->no_hugetlbfs && internal_cfg->hugepage_unlink) {
 		RTE_LOG(ERR, EAL, "Option --"OPT_HUGE_UNLINK" cannot "
@@ -994,6 +1009,8 @@ eal_common_usage(void)
 	       "  -n CHANNELS         Number of memory channels\n"
 	       "  -m MB               Memory to allocate (see also --"OPT_SOCKET_MEM")\n"
 	       "  -r RANKS            Force number of memory ranks (don't detect)\n"
+	       "  --"OPT_SINGLE_FILE" Create just single file for shared memory, and \n"
+	       "                      do not promise physical contiguity of memseg\n"
 	       "  -b, --"OPT_PCI_BLACKLIST" Add a PCI device in black list.\n"
 	       "                      Prevent EAL from using this PCI device. The argument\n"
 	       "                      format is <domain:bus:devid.func>.\n"
diff --git a/lib/librte_eal/common/eal_internal_cfg.h b/lib/librte_eal/common/eal_internal_cfg.h
index 5f1367e..9117ed9 100644
--- a/lib/librte_eal/common/eal_internal_cfg.h
+++ b/lib/librte_eal/common/eal_internal_cfg.h
@@ -61,6 +61,7 @@ struct hugepage_info {
  */
 struct internal_config {
 	volatile size_t memory;           /**< amount of asked memory */
+	volatile unsigned single_file;    /**< mmap all hugepages in single file */
 	volatile unsigned force_nchannel; /**< force number of channels */
 	volatile unsigned force_nrank;    /**< force number of ranks */
 	volatile unsigned no_hugetlbfs;   /**< true to disable hugetlbfs */
diff --git a/lib/librte_eal/common/eal_options.h b/lib/librte_eal/common/eal_options.h
index a881c62..e5da14a 100644
--- a/lib/librte_eal/common/eal_options.h
+++ b/lib/librte_eal/common/eal_options.h
@@ -83,6 +83,8 @@ enum {
 	OPT_VMWARE_TSC_MAP_NUM,
 #define OPT_XEN_DOM0          "xen-dom0"
 	OPT_XEN_DOM0_NUM,
+#define OPT_SINGLE_FILE       "single-file"
+	OPT_SINGLE_FILE_NUM,
 	OPT_LONG_MAX_NUM
 };
 
diff --git a/lib/librte_eal/linuxapp/eal/eal_memory.c b/lib/librte_eal/linuxapp/eal/eal_memory.c
index 846fd31..2bb1163 100644
--- a/lib/librte_eal/linuxapp/eal/eal_memory.c
+++ b/lib/librte_eal/linuxapp/eal/eal_memory.c
@@ -80,6 +80,10 @@
 #include <errno.h>
 #include <sys/ioctl.h>
 #include <sys/time.h>
+#include <mntent.h>
+#include <sys/mman.h>
+#include <sys/file.h>
+#include <sys/vfs.h>
 
 #include <rte_log.h>
 #include <rte_memory.h>
@@ -92,6 +96,9 @@
 #include <rte_common.h>
 #include <rte_string_fns.h>
 
+#define _GNU_SOURCE
+#include <sys/syscall.h>
+
 #include "eal_private.h"
 #include "eal_internal_cfg.h"
 #include "eal_filesystem.h"
@@ -768,6 +775,7 @@ create_shared_memory(const char *filename, const size_t mem_size)
 	}
 	retval = mmap(NULL, mem_size, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0);
 	close(fd);
+
 	return retval;
 }
 
@@ -1110,10 +1118,34 @@ rte_eal_hugepage_init(void)
 	/* get pointer to global configuration */
 	mcfg = rte_eal_get_configuration()->mem_config;
 
-	/* hugetlbfs can be disabled */
-	if (internal_config.no_hugetlbfs) {
+	/* when hugetlbfs is disabled or single-file option is specified */
+	if (internal_config.no_hugetlbfs || internal_config.single_file) {
+		int fd;
+		uint64_t pagesize;
+		unsigned socket_id;
+		char filepath[MAX_HUGEPAGE_PATH];
+
+		syscall(SYS_getcpu, NULL, &socket_id, NULL);
+
+		if (internal_config.no_hugetlbfs) {
+			eal_get_hugefile_path(filepath, sizeof(filepath),
+					"/dev/shm", 0);
+			pagesize = RTE_PGSIZE_4K;
+		} else {
+			struct hugepage_info *hpi = &internal_config.hugepage_info[0];
+			eal_get_hugefile_path(filepath, sizeof(filepath),
+					hpi->hugedir, 0);
+			pagesize = hpi->hugepage_sz;
+		}
+		fd = open(filepath, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR);
+		if (fd < 0) {
+			RTE_LOG(ERR, EAL, "%s: open %s failed: %s\n", __func__,
+					filepath, strerror(errno));
+			return -1;
+		}
+
 		addr = mmap(NULL, internal_config.memory, PROT_READ | PROT_WRITE,
-				MAP_PRIVATE | MAP_ANONYMOUS, 0, 0);
+				MAP_SHARED | MAP_POPULATE, fd, 0);
 		if (addr == MAP_FAILED) {
 			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n", __func__,
 					strerror(errno));
@@ -1121,9 +1153,12 @@ rte_eal_hugepage_init(void)
 		}
 		mcfg->memseg[0].phys_addr = (phys_addr_t)(uintptr_t)addr;
 		mcfg->memseg[0].addr = addr;
-		mcfg->memseg[0].hugepage_sz = RTE_PGSIZE_4K;
+		mcfg->memseg[0].hugepage_sz = pagesize;
 		mcfg->memseg[0].len = internal_config.memory;
-		mcfg->memseg[0].socket_id = 0;
+		mcfg->memseg[0].socket_id = socket_id;
+
+		close(fd);
+
 		return 0;
 	}
 
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-10 11:42 ` [PATCH 0/4] " Jianfeng Tan
  2016-01-10 11:42   ` [PATCH 1/4] mem: add --single-file to create single mem-backed file Jianfeng Tan
@ 2016-01-10 11:43   ` Jianfeng Tan
  2016-01-11 11:43     ` Pavel Fedin
  2016-01-11 20:26     ` Rich Lane
  2016-01-10 11:43   ` [PATCH 3/4] virtio/vdev: add ways to interact with vhost Jianfeng Tan
                     ` (4 subsequent siblings)
  6 siblings, 2 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-01-10 11:43 UTC (permalink / raw)
  To: dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

A new API named rte_eal_get_backfile_info() and a new data
struct back_file is added to obstain information of memory-
backed file info.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 lib/librte_eal/common/include/rte_memory.h | 16 +++++++++++++
 lib/librte_eal/linuxapp/eal/eal_memory.c   | 37 ++++++++++++++++++++++++++++++
 2 files changed, 53 insertions(+)

diff --git a/lib/librte_eal/common/include/rte_memory.h b/lib/librte_eal/common/include/rte_memory.h
index 9c9e40f..75ef8db 100644
--- a/lib/librte_eal/common/include/rte_memory.h
+++ b/lib/librte_eal/common/include/rte_memory.h
@@ -109,6 +109,22 @@ struct rte_memseg {
 } __rte_packed;
 
 /**
+ * This struct is used to store information about memory-backed file that
+ * we mapped in memory initialization.
+ */
+struct back_file {
+	void *addr;         /**< virtual addr */
+	size_t size;        /**< the page size */
+	char filepath[PATH_MAX]; /**< path to backing file on filesystem */
+};
+
+/**
+  * Get the hugepage file information. Caller to free.
+  * Return number of hugepage files used.
+  */
+int rte_eal_get_backfile_info(struct back_file **);
+
+/**
  * Lock page in physical memory and prevent from swapping.
  *
  * @param virt
diff --git a/lib/librte_eal/linuxapp/eal/eal_memory.c b/lib/librte_eal/linuxapp/eal/eal_memory.c
index 2bb1163..6ca1404 100644
--- a/lib/librte_eal/linuxapp/eal/eal_memory.c
+++ b/lib/librte_eal/linuxapp/eal/eal_memory.c
@@ -758,6 +758,9 @@ sort_by_physaddr(struct hugepage_file *hugepg_tbl, struct hugepage_info *hpi)
 	return 0;
 }
 
+static struct hugepage_file *hugepage_files;
+static int num_hugepage_files;
+
 /*
  * Uses mmap to create a shared memory area for storage of data
  * Used in this file to store the hugepage file map on disk
@@ -776,9 +779,29 @@ create_shared_memory(const char *filename, const size_t mem_size)
 	retval = mmap(NULL, mem_size, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0);
 	close(fd);
 
+	hugepage_files = retval;
+	num_hugepage_files = mem_size / (sizeof(struct hugepage_file));
+
 	return retval;
 }
 
+int
+rte_eal_get_backfile_info(struct back_file **p)
+{
+	struct back_file *backfiles;
+	int i, num_backfiles = num_hugepage_files;
+
+	backfiles = malloc(sizeof(struct back_file) * num_backfiles);
+	for (i = 0; i < num_backfiles; ++i) {
+		backfiles[i].addr = hugepage_files[i].final_va;
+		backfiles[i].size = hugepage_files[i].size;
+		strcpy(backfiles[i].filepath, hugepage_files[i].filepath);
+	}
+
+	*p = backfiles;
+	return num_backfiles;
+}
+
 /*
  * this copies *active* hugepages from one hugepage table to another.
  * destination is typically the shared memory.
@@ -1157,6 +1180,20 @@ rte_eal_hugepage_init(void)
 		mcfg->memseg[0].len = internal_config.memory;
 		mcfg->memseg[0].socket_id = socket_id;
 
+		hugepage = create_shared_memory(eal_hugepage_info_path(),
+				sizeof(struct hugepage_file));
+		hugepage->orig_va = addr;
+		hugepage->final_va = addr;
+		hugepage->physaddr = rte_mem_virt2phy(addr);
+		hugepage->size = pagesize;
+		hugepage->socket_id = socket_id;
+		hugepage->file_id = 0;
+		hugepage->memseg_id = 0;
+#ifdef RTE_EAL_SINGLE_FILE_SEGMENTS
+		hugepage->repeated = internal_config.memory / pagesize;
+#endif
+		strncpy(hugepage->filepath, filepath, MAX_HUGEPAGE_PATH);
+
 		close(fd);
 
 		return 0;
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH 3/4] virtio/vdev: add ways to interact with vhost
  2016-01-10 11:42 ` [PATCH 0/4] " Jianfeng Tan
  2016-01-10 11:42   ` [PATCH 1/4] mem: add --single-file to create single mem-backed file Jianfeng Tan
  2016-01-10 11:43   ` [PATCH 2/4] mem: add API to obstain memory-backed file info Jianfeng Tan
@ 2016-01-10 11:43   ` Jianfeng Tan
  2016-01-11 10:42     ` Pavel Fedin
                       ` (2 more replies)
  2016-01-10 11:43   ` [PATCH 4/4] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
                     ` (3 subsequent siblings)
  6 siblings, 3 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-01-10 11:43 UTC (permalink / raw)
  To: dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

Depends on the type of vhost file: vhost-user is used if the given
path points to a unix socket; vhost-net is used if the given path
points to a char device.

NOTE: we now keep CONFIG_RTE_VIRTIO_VDEV undefined by default, need
to be uncommented when in use.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 config/common_linuxapp             |   5 +
 drivers/net/virtio/Makefile        |   4 +
 drivers/net/virtio/vhost.c         | 734 +++++++++++++++++++++++++++++++++++++
 drivers/net/virtio/vhost.h         | 192 ++++++++++
 drivers/net/virtio/virtio_ethdev.h |   5 +-
 drivers/net/virtio/virtio_pci.h    |  52 ++-
 6 files changed, 990 insertions(+), 2 deletions(-)
 create mode 100644 drivers/net/virtio/vhost.c
 create mode 100644 drivers/net/virtio/vhost.h

diff --git a/config/common_linuxapp b/config/common_linuxapp
index 74bc515..f76e162 100644
--- a/config/common_linuxapp
+++ b/config/common_linuxapp
@@ -534,3 +534,8 @@ CONFIG_RTE_APP_TEST=y
 CONFIG_RTE_TEST_PMD=y
 CONFIG_RTE_TEST_PMD_RECORD_CORE_CYCLES=n
 CONFIG_RTE_TEST_PMD_RECORD_BURST_STATS=n
+
+#
+# Enable virtio support for container
+#
+CONFIG_RTE_VIRTIO_VDEV=y
diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index 43835ba..0877023 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -52,6 +52,10 @@ SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_ethdev.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx_simple.c
 
+ifeq ($(CONFIG_RTE_VIRTIO_VDEV),y)
+	SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += vhost.c
+endif
+
 # this lib depends upon:
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_eal lib/librte_ether
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_mempool lib/librte_mbuf
diff --git a/drivers/net/virtio/vhost.c b/drivers/net/virtio/vhost.c
new file mode 100644
index 0000000..e423e02
--- /dev/null
+++ b/drivers/net/virtio/vhost.c
@@ -0,0 +1,734 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2015 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <sys/types.h>
+#include <sys/socket.h>
+#include <sys/un.h>
+#include <stdio.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <string.h>
+#include <errno.h>
+#include <assert.h>
+#include <sys/mman.h>
+#include <unistd.h>
+#include <sys/eventfd.h>
+#include <sys/ioctl.h>
+#include <net/if.h>
+
+#include <rte_mbuf.h>
+#include <rte_memory.h>
+#include <rte_eal_memconfig.h>
+
+#include "virtio_pci.h"
+#include "virtio_logs.h"
+#include "virtio_ethdev.h"
+#include "virtqueue.h"
+#include "vhost.h"
+
+static int
+vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
+{
+	struct msghdr msgh;
+	struct iovec iov;
+	int r;
+
+	size_t fd_size = fd_num * sizeof(int);
+	char control[CMSG_SPACE(fd_size)];
+	struct cmsghdr *cmsg;
+
+	memset(&msgh, 0, sizeof(msgh));
+	memset(control, 0, sizeof(control));
+
+	iov.iov_base = (uint8_t *)buf;
+	iov.iov_len = len;
+
+	msgh.msg_iov = &iov;
+	msgh.msg_iovlen = 1;
+
+	msgh.msg_control = control;
+	msgh.msg_controllen = sizeof(control);
+
+	cmsg = CMSG_FIRSTHDR(&msgh);
+
+	cmsg->cmsg_len = CMSG_LEN(fd_size);
+	cmsg->cmsg_level = SOL_SOCKET;
+	cmsg->cmsg_type = SCM_RIGHTS;
+	memcpy(CMSG_DATA(cmsg), fds, fd_size);
+
+	do {
+		r = sendmsg(fd, &msgh, 0);
+	} while (r < 0 && errno == EINTR);
+
+	return r;
+}
+
+static int
+vhost_user_read(int fd, VhostUserMsg *msg)
+{
+	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
+	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
+
+	ret = recv(fd, (void *)msg, sz_hdr, 0);
+	if (ret < sz_hdr) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
+				ret, sz_hdr);
+		goto fail;
+	}
+
+	/* validate msg flags */
+	if (msg->flags != (valid_flags)) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg: flags 0x%x instead of 0x%x.",
+				msg->flags, valid_flags);
+		goto fail;
+	}
+
+	sz_payload = msg->size;
+	if (sz_payload) {
+		ret = recv(fd, (void *)((uint8_t *)msg + sz_hdr), sz_payload, 0);
+		if (ret < sz_payload) {
+			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
+					ret, msg->size);
+			goto fail;
+		}
+	}
+
+	return 0;
+
+fail:
+	return -1;
+}
+
+static VhostUserMsg m __attribute__ ((unused));
+
+static void
+prepare_vhost_memory_user(VhostUserMsg *msg, int fds[])
+{
+	int i, num;
+	struct back_file *huges;
+	struct vhost_memory_region *mr;
+
+	num = rte_eal_get_backfile_info(&huges);
+
+	if (num > VHOST_MEMORY_MAX_NREGIONS)
+		rte_panic("%d hugepage files exceed the maximum of %d for "
+				"vhost-user\n", num, VHOST_MEMORY_MAX_NREGIONS);
+
+	for (i = 0; i < num; ++i) {
+		mr = &msg->payload.memory.regions[i];
+		mr->guest_phys_addr = (uint64_t)huges[i].addr; /* use vaddr! */
+		mr->userspace_addr = (uint64_t)huges[i].addr;
+		mr->memory_size = huges[i].size;
+		mr->mmap_offset = 0;
+		fds[i] = open(huges[i].filepath, O_RDWR);
+	}
+
+	msg->payload.memory.nregions = num;
+	msg->payload.memory.padding = 0;
+	free(huges);
+}
+
+static int
+vhost_user_sock(struct virtio_hw *hw, unsigned long int req, void *arg)
+{
+	VhostUserMsg msg;
+	struct vhost_vring_file *file = 0;
+	int need_reply = 0;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+	int fd_num = 0;
+	int i, len;
+
+	msg.request = req;
+	msg.flags = VHOST_USER_VERSION;
+	msg.size = 0;
+
+	switch (req) {
+	case VHOST_USER_GET_FEATURES:
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_FEATURES:
+	case VHOST_USER_SET_LOG_BASE:
+		msg.payload.u64 = *((__u64 *)arg);
+		msg.size = sizeof(m.payload.u64);
+		break;
+
+	case VHOST_USER_SET_OWNER:
+	case VHOST_USER_RESET_OWNER:
+		break;
+
+	case VHOST_USER_SET_MEM_TABLE:
+		prepare_vhost_memory_user(&msg, fds);
+		fd_num = msg.payload.memory.nregions;
+		msg.size = sizeof(m.payload.memory.nregions);
+		msg.size += sizeof(m.payload.memory.padding);
+		msg.size += fd_num * sizeof(struct vhost_memory_region);
+		break;
+
+	case VHOST_USER_SET_LOG_FD:
+		fds[fd_num++] = *((int *)arg);
+		break;
+
+	case VHOST_USER_SET_VRING_NUM:
+	case VHOST_USER_SET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(struct vhost_vring_state));
+		msg.size = sizeof(m.payload.state);
+		break;
+
+	case VHOST_USER_GET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(struct vhost_vring_state));
+		msg.size = sizeof(m.payload.state);
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_VRING_ADDR:
+		memcpy(&msg.payload.addr, arg, sizeof(struct vhost_vring_addr));
+		msg.size = sizeof(m.payload.addr);
+		break;
+
+	case VHOST_USER_SET_VRING_KICK:
+	case VHOST_USER_SET_VRING_CALL:
+	case VHOST_USER_SET_VRING_ERR:
+		file = arg;
+		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
+		msg.size = sizeof(m.payload.u64);
+		if (file->fd > 0)
+			fds[fd_num++] = file->fd;
+		else
+			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
+		break;
+
+	default:
+		PMD_DRV_LOG(ERR, "vhost-user trying to send unhandled msg type");
+		return -1;
+	}
+
+	len = VHOST_USER_HDR_SIZE + msg.size;
+	if (vhost_user_write(hw->vhostfd, &msg, len, fds, fd_num) < 0)
+		return 0;
+
+	if (req == VHOST_USER_SET_MEM_TABLE)
+		for (i = 0; i < fd_num; ++i)
+			close(fds[i]);
+
+	if (need_reply) {
+		if (vhost_user_read(hw->vhostfd, &msg) < 0)
+			return -1;
+
+		if (req != msg.request) {
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+
+		switch (req) {
+		case VHOST_USER_GET_FEATURES:
+			if (msg.size != sizeof(m.payload.u64)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			*((__u64 *)arg) = msg.payload.u64;
+			break;
+		case VHOST_USER_GET_VRING_BASE:
+			if (msg.size != sizeof(m.payload.state)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			memcpy(arg, &msg.payload.state, sizeof(struct vhost_vring_state));
+			break;
+		default:
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+	}
+
+	return 0;
+}
+
+static int
+vhost_kernel_ioctl(struct virtio_hw *hw, unsigned long int req, void *arg)
+{
+	return ioctl(hw->vhostfd, req, arg);
+}
+
+enum {
+	VHOST_MSG_SET_OWNER,
+	VHOST_MSG_SET_FEATURES,
+	VHOST_MSG_GET_FEATURES,
+	VHOST_MSG_SET_VRING_CALL,
+	VHOST_MSG_SET_VRING_NUM,
+	VHOST_MSG_SET_VRING_BASE,
+	VHOST_MSG_SET_VRING_ADDR,
+	VHOST_MSG_SET_VRING_KICK,
+	VHOST_MSG_SET_MEM_TABLE,
+	VHOST_MSG_MAX,
+};
+
+static const char *vhost_msg_strings[] = {
+	"VHOST_MSG_SET_OWNER",
+	"VHOST_MSG_SET_FEATURES",
+	"VHOST_MSG_GET_FEATURES",
+	"VHOST_MSG_SET_VRING_CALL",
+	"VHOST_MSG_SET_VRING_NUM",
+	"VHOST_MSG_SET_VRING_BASE",
+	"VHOST_MSG_SET_VRING_ADDR",
+	"VHOST_MSG_SET_VRING_KICK",
+	"VHOST_MSG_SET_MEM_TABLE",
+	NULL,
+};
+
+static unsigned long int vhost_req_map[][2] = {
+	{VHOST_SET_OWNER,		VHOST_USER_SET_OWNER},
+	{VHOST_SET_FEATURES,	VHOST_USER_SET_FEATURES},
+	{VHOST_GET_FEATURES,	VHOST_USER_GET_FEATURES},
+	{VHOST_SET_VRING_CALL,	VHOST_USER_SET_VRING_CALL},
+	{VHOST_SET_VRING_NUM,	VHOST_USER_SET_VRING_NUM},
+	{VHOST_SET_VRING_BASE,	VHOST_USER_SET_VRING_BASE},
+	{VHOST_SET_VRING_ADDR,	VHOST_USER_SET_VRING_ADDR},
+	{VHOST_SET_VRING_KICK,	VHOST_USER_SET_VRING_KICK},
+	{VHOST_SET_MEM_TABLE,	VHOST_USER_SET_MEM_TABLE},
+};
+
+static int
+vhost_call(struct virtio_hw *hw, unsigned long int req_orig, void *arg)
+{
+	int req_new;
+	int ret = 0;
+
+	if (req_orig >= VHOST_MSG_MAX)
+		rte_panic("invalid req: %lu\n", req_orig);
+
+	PMD_DRV_LOG(INFO, "%s\n", vhost_msg_strings[req_orig]);
+	req_new = vhost_req_map[req_orig][hw->type];
+	if (hw->type == VHOST_USER)
+		ret = vhost_user_sock(hw, req_new, arg);
+	else
+		ret = vhost_kernel_ioctl(hw, req_new, arg);
+
+	if (ret < 0)
+		rte_panic("vhost_call %s failed: %s\n",
+				vhost_msg_strings[req_orig], strerror(errno));
+
+	return ret;
+}
+
+static void
+kick_one_vq(struct virtio_hw *hw, struct virtqueue *vq, unsigned queue_sel)
+{
+	int callfd, kickfd;
+	struct vhost_vring_file file;
+
+	/* or use invalid flag to disable it, but vhost-dpdk uses this to judge
+	 * if dev is alive. so finally we need two real event_fds.
+	 */
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL comes
+	 * firstly because vhost depends on this msg to allocate virtqueue
+	 * pair.
+	 */
+	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (callfd < 0)
+		rte_panic("callfd error, %s\n", strerror(errno));
+
+	file.index = queue_sel;
+	file.fd = callfd;
+	vhost_call(hw, VHOST_MSG_SET_VRING_CALL, &file);
+	hw->callfds[queue_sel] = callfd;
+
+	struct vhost_vring_state state;
+	state.index = queue_sel;
+	state.num = vq->vq_ring.num;
+	vhost_call(hw, VHOST_MSG_SET_VRING_NUM, &state);
+
+	state.num = 0; /* no reservation */
+	vhost_call(hw, VHOST_MSG_SET_VRING_BASE, &state);
+
+	struct vhost_vring_addr addr = {
+		.index = queue_sel,
+		.desc_user_addr = (uint64_t)vq->vq_ring.desc,
+		.avail_user_addr = (uint64_t)vq->vq_ring.avail,
+		.used_user_addr = (uint64_t)vq->vq_ring.used,
+		.log_guest_addr = 0,
+		.flags = 0, /* disable log */
+	};
+	vhost_call(hw, VHOST_MSG_SET_VRING_ADDR, &addr);
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_KICK comes
+	 * lastly because vhost depends on this msg to judge if
+	 * virtio_is_ready().
+	 */
+
+	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (kickfd < 0)
+		rte_panic("kickfd error, %s\n", strerror(errno));
+
+	file.fd = kickfd;
+	vhost_call(hw, VHOST_MSG_SET_VRING_KICK, &file);
+	hw->kickfds[queue_sel] = kickfd;
+}
+
+/**
+ * Merge those virtually adjacent memsegs into one region.
+ */
+static void
+prepare_vhost_memory_kernel(struct vhost_memory_kernel **p_vm)
+{
+	unsigned i, j, k = 0;
+	struct rte_memseg *seg;
+	struct vhost_memory_region *mr;
+	struct vhost_memory_kernel *vm;
+
+	vm = malloc(sizeof(struct vhost_memory_kernel)
+			+ RTE_MAX_MEMSEG * sizeof(struct vhost_memory_region));
+
+	for (i = 0; i < RTE_MAX_MEMSEG; ++i) {
+		seg = &rte_eal_get_configuration()->mem_config->memseg[i];
+		if (seg->addr == NULL)
+			break;
+
+		int new_region = 1;
+		for (j = 0; j < k; ++j) {
+			mr = &vm->regions[j];
+
+			if (mr->userspace_addr + mr->memory_size
+					== (uint64_t)seg->addr) {
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+
+			if ((uint64_t)seg->addr + seg->len
+					== mr->userspace_addr) {
+				mr->guest_phys_addr = (uint64_t)seg->addr;
+				mr->userspace_addr = (uint64_t)seg->addr;
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+		}
+
+		if (new_region == 0)
+			continue;
+
+		mr = &vm->regions[k++];
+		mr->guest_phys_addr = (uint64_t)seg->addr; /* use vaddr here! */
+		mr->userspace_addr = (uint64_t)seg->addr;
+		mr->memory_size = seg->len;
+		mr->mmap_offset = 0;
+	}
+
+	vm->nregions = k;
+	vm->padding = 0;
+	*p_vm = vm;
+}
+
+static void kick_all_vq(struct virtio_hw *hw)
+{
+	int ret;
+	unsigned i, queue_sel, nvqs;
+	struct rte_eth_dev_data *data = hw->data;
+
+	if (hw->type == VHOST_KERNEL) {
+		struct vhost_memory_kernel *vm = NULL;
+		prepare_vhost_memory_kernel(&vm);
+		vhost_call(hw, VHOST_MSG_SET_MEM_TABLE, vm);
+		free(vm);
+	} else {
+		/* construct vhost_memory inside prepare_vhost_memory_user() */
+		vhost_call(hw, VHOST_MSG_SET_MEM_TABLE, NULL);
+	}
+
+	for (i = 0; i < data->nb_rx_queues; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
+		kick_one_vq(hw, data->rx_queues[i], queue_sel);
+	}
+	for (i = 0; i < data->nb_tx_queues; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
+		kick_one_vq(hw, data->tx_queues[i], queue_sel);
+	}
+
+	/* after setup all virtqueues, we need to set_features again
+	 * so that these features can be set into each virtqueue in
+	 * vhost side.
+	 */
+	uint64_t features = hw->guest_features;
+	features &= ~(1ull << VIRTIO_NET_F_MAC);
+	vhost_call(hw, VHOST_MSG_SET_FEATURES, &features);
+	if (ioctl(hw->backfd, TUNSETVNETHDRSZ, &hw->vtnet_hdr_size) == -1)
+		rte_panic("TUNSETVNETHDRSZ failed: %s\n", strerror(errno));
+	PMD_DRV_LOG(INFO, "set features:%"PRIx64"\n", features);
+
+	if (hw->type == VHOST_KERNEL) {
+		struct vhost_vring_file file;
+
+		file.fd = hw->backfd;
+		nvqs = data->nb_rx_queues + data->nb_tx_queues;
+		for (file.index = 0; file.index < nvqs; ++file.index) {
+			ret = vhost_kernel_ioctl(hw, VHOST_NET_SET_BACKEND, &file);
+			if (ret < 0)
+				rte_panic("VHOST_NET_SET_BACKEND failed, %s\n",
+						strerror(errno));
+		}
+	}
+
+	/* TODO: VHOST_SET_LOG_BASE */
+}
+
+void
+virtio_ioport_write(struct virtio_hw *hw, uint64_t addr, uint32_t val)
+{
+	uint64_t guest_features;
+
+	switch (addr) {
+	case VIRTIO_PCI_GUEST_FEATURES:
+		guest_features = val;
+		guest_features &= ~(1ull << VIRTIO_NET_F_MAC);
+		vhost_call(hw, VHOST_MSG_SET_FEATURES, &guest_features);
+		break;
+	case VIRTIO_PCI_QUEUE_PFN:
+		/* do nothing */
+		break;
+	case VIRTIO_PCI_QUEUE_SEL:
+		hw->queue_sel = val;
+		break;
+	case VIRTIO_PCI_STATUS:
+		if (val & VIRTIO_CONFIG_S_DRIVER_OK)
+			kick_all_vq(hw);
+		hw->status = val & 0xFF;
+		break;
+	case VIRTIO_PCI_QUEUE_NOTIFY:
+		{
+			int ret;
+			uint64_t buf = 1;
+			ret = write(hw->kickfds[val], &buf, sizeof(uint64_t));
+			if (ret == -1)
+				rte_panic("VIRTIO_PCI_QUEUE_NOTIFY failed: %s\n",
+						strerror(errno));
+			break;
+		}
+	default:
+		PMD_DRV_LOG(ERR, "unexpected address %"PRIu64" value 0x%x\n",
+				addr, val);
+		break;
+	}
+}
+
+uint32_t
+virtio_ioport_read(struct virtio_hw *hw, uint64_t addr)
+{
+	uint32_t ret = 0xFFFFFFFF;
+	uint64_t host_features;
+
+	PMD_DRV_LOG(INFO, "addr: %"PRIu64"\n", addr);
+
+	switch (addr) {
+	case VIRTIO_PCI_HOST_FEATURES:
+		vhost_call(hw, VHOST_MSG_GET_FEATURES, &host_features);
+		PMD_DRV_LOG(INFO, "get_features: %"PRIx64"\n", host_features);
+		if (hw->mac_specified)
+			host_features |= (1ull << VIRTIO_NET_F_MAC);
+		/* disable it until we support CQ */
+		host_features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
+		ret = host_features;
+		break;
+	case VIRTIO_PCI_GUEST_FEATURES:
+		ret = hw->guest_features;
+		break;
+	case VIRTIO_PCI_QUEUE_NUM:
+		ret = hw->queue_num;
+		break;
+	case VIRTIO_PCI_QUEUE_SEL:
+		ret = hw->queue_sel;
+		break;
+	case VIRTIO_PCI_STATUS:
+		ret = hw->status;
+		break;
+	case 20: /* mac addr: 0~3 */
+		if (hw->mac_specified) {
+			uint32_t m0 = hw->mac_addr[0],
+				 m1 = hw->mac_addr[1],
+				 m2 = hw->mac_addr[2],
+				 m3 = hw->mac_addr[3];
+			ret = (m3 << 24) | (m2 << 16) | (m1 << 8) | m0;
+		}
+		break;
+	case 24: /* mac addr: 4~5 */
+		if (hw->mac_specified) {
+			uint32_t m4 = hw->mac_addr[4],
+				 m5 = hw->mac_addr[5];
+			ret = (m5 << 8) | m4;
+		}
+		break;
+	default:
+		PMD_DRV_LOG(ERR, "%"PRIu64" (r) not supported\n", addr);
+		break;
+	}
+
+	return ret;
+}
+
+#define TUN_DEF_SNDBUF	(1ull << 20)
+
+static void
+vhost_kernel_backend_setup(struct virtio_hw *hw)
+{
+	int fd;
+	int len = sizeof(struct virtio_net_hdr);
+	int req_mq = 0;
+	int sndbuf = TUN_DEF_SNDBUF;
+	unsigned int features;
+	struct ifreq ifr;
+
+	/* TODO:
+	 * 1. get and set offload capability, tap_probe_has_ufo, tap_fd_set_offload
+	 * 2. verify we can get and set vnet_hdr_len, tap_probe_vnet_hdr_len
+
+	 * 1. get number of memory regions from vhost module parameter
+	 * max_mem_regions, supported in newer version linux kernel
+	 */
+
+	fd = open(PATH_NET_TUN, O_RDWR);
+	if (fd < 0)
+		rte_panic("open %s error, %s\n", PATH_NET_TUN, strerror(errno));
+
+	memset(&ifr, 0, sizeof(ifr));
+	ifr.ifr_flags = IFF_TAP | IFF_NO_PI;
+
+	if (ioctl(fd, TUNGETFEATURES, &features) == -1)
+		rte_panic("TUNGETFEATURES failed: %s", strerror(errno));
+
+	if (features & IFF_ONE_QUEUE)
+		ifr.ifr_flags |= IFF_ONE_QUEUE;
+
+	if (features & IFF_VNET_HDR)
+		ifr.ifr_flags |= IFF_VNET_HDR;
+	else
+		rte_panic("vnet_hdr requested, but kernel does not support\n");
+
+	if (req_mq) {
+		if (features & IFF_MULTI_QUEUE)
+			ifr.ifr_flags |= IFF_MULTI_QUEUE;
+		else
+			rte_panic("multiqueue requested, but kernel does not support\n");
+	}
+
+	strncpy(ifr.ifr_name, "tap%d", IFNAMSIZ);
+	if (ioctl(fd, TUNSETIFF, (void *) &ifr) == -1)
+		rte_panic("TUNSETIFF failed: %s", strerror(errno));
+	fcntl(fd, F_SETFL, O_NONBLOCK);
+
+	if (ioctl(fd, TUNSETVNETHDRSZ, &len) == -1)
+		rte_panic("TUNSETVNETHDRSZ failed: %s\n", strerror(errno));
+
+	if (ioctl(fd, TUNSETSNDBUF, &sndbuf) == -1)
+		rte_panic("TUNSETSNDBUF failed: %s", strerror(errno));
+
+	hw->backfd = fd;
+
+	hw->vhostfd = open(hw->path, O_RDWR);
+	if (hw->vhostfd == -1)
+		rte_panic("open %s failed: %s\n", hw->path, strerror(errno));
+}
+
+static void
+vhost_user_backend_setup(struct virtio_hw *hw)
+{
+	int fd;
+	int flag;
+	struct sockaddr_un un;
+
+	fd = socket(AF_UNIX, SOCK_STREAM, 0);
+	if (fd < 0)
+		rte_panic("socket error, %s\n", strerror(errno));
+
+	flag = fcntl(fd, F_GETFD);
+	fcntl(fd, F_SETFD, flag | FD_CLOEXEC);
+
+	memset(&un, 0, sizeof(un));
+	un.sun_family = AF_UNIX;
+	snprintf(un.sun_path, sizeof(un.sun_path), "%s", hw->path);
+	if (connect(fd, (struct sockaddr *)&un, sizeof(un)) < 0) {
+		PMD_DRV_LOG(ERR, "connect error, %s\n", strerror(errno));
+		exit(-1);
+	}
+
+	hw->vhostfd = fd;
+}
+
+void
+virtio_vdev_init(struct rte_eth_dev_data *data, const char *path,
+		int nb_rx, int nb_tx, int nb_cq __attribute__ ((unused)),
+		int queue_num, char *mac)
+{
+	int i;
+	int ret;
+	struct stat s;
+	uint32_t tmp[ETHER_ADDR_LEN];
+	struct virtio_hw *hw = data->dev_private;
+
+	hw->data = data;
+	hw->path = strdup(path);
+	hw->max_rx_queues = nb_rx;
+	hw->max_tx_queues = nb_tx;
+	hw->queue_num = queue_num;
+	hw->mac_specified = 0;
+	if (mac) {
+		ret = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0], &tmp[1],
+				&tmp[2], &tmp[3], &tmp[4], &tmp[5]);
+		if (ret == ETHER_ADDR_LEN) {
+			for (i = 0; i < ETHER_ADDR_LEN; ++i)
+				hw->mac_addr[i] = (uint8_t)tmp[i];
+			hw->mac_specified = 1;
+		}
+	}
+
+	/* TODO: cq */
+
+	ret = stat(hw->path, &s);
+	if (ret < 0)
+		rte_panic("stat: %s failed, %s\n", hw->path, strerror(errno));
+
+	switch (s.st_mode & S_IFMT) {
+		case S_IFCHR:
+			hw->type = VHOST_KERNEL;
+			vhost_kernel_backend_setup(hw);
+			break;
+		case S_IFSOCK:
+			hw->type = VHOST_USER;
+			vhost_user_backend_setup(hw);
+			break;
+		default:
+			rte_panic("unknown file type of %s\n", hw->path);
+	}
+	if (vhost_call(hw, VHOST_MSG_SET_OWNER, NULL) == -1)
+		rte_panic("vhost set_owner failed: %s\n", strerror(errno));
+}
diff --git a/drivers/net/virtio/vhost.h b/drivers/net/virtio/vhost.h
new file mode 100644
index 0000000..c7517f6
--- /dev/null
+++ b/drivers/net/virtio/vhost.h
@@ -0,0 +1,192 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2014 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VHOST_NET_USER_H
+#define _VHOST_NET_USER_H
+
+#include <stdint.h>
+#include <linux/types.h>
+#include <linux/ioctl.h>
+
+#define VHOST_MEMORY_MAX_NREGIONS 8
+
+struct vhost_vring_state {
+	unsigned int index;
+	unsigned int num;
+};
+
+struct vhost_vring_file {
+	unsigned int index;
+	int fd;
+};
+
+struct vhost_vring_addr {
+	unsigned int index;
+	/* Option flags. */
+	unsigned int flags;
+	/* Flag values: */
+	/* Whether log address is valid. If set enables logging. */
+#define VHOST_VRING_F_LOG 0
+
+	/* Start of array of descriptors (virtually contiguous) */
+	uint64_t desc_user_addr;
+	/* Used structure address. Must be 32 bit aligned */
+	uint64_t used_user_addr;
+	/* Available structure address. Must be 16 bit aligned */
+	uint64_t avail_user_addr;
+	/* Logging support. */
+	/* Log writes to used structure, at offset calculated from specified
+	 * address. Address must be 32 bit aligned. */
+	uint64_t log_guest_addr;
+};
+
+#define VIRTIO_CONFIG_S_DRIVER_OK   4
+
+typedef enum VhostUserRequest {
+	VHOST_USER_NONE = 0,
+	VHOST_USER_GET_FEATURES = 1,
+	VHOST_USER_SET_FEATURES = 2,
+	VHOST_USER_SET_OWNER = 3,
+	VHOST_USER_RESET_OWNER = 4,
+	VHOST_USER_SET_MEM_TABLE = 5,
+	VHOST_USER_SET_LOG_BASE = 6,
+	VHOST_USER_SET_LOG_FD = 7,
+	VHOST_USER_SET_VRING_NUM = 8,
+	VHOST_USER_SET_VRING_ADDR = 9,
+	VHOST_USER_SET_VRING_BASE = 10,
+	VHOST_USER_GET_VRING_BASE = 11,
+	VHOST_USER_SET_VRING_KICK = 12,
+	VHOST_USER_SET_VRING_CALL = 13,
+	VHOST_USER_SET_VRING_ERR = 14,
+	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
+	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
+	VHOST_USER_GET_QUEUE_NUM = 17,
+	VHOST_USER_SET_VRING_ENABLE = 18,
+	VHOST_USER_MAX
+} VhostUserRequest;
+
+struct vhost_memory_region {
+	uint64_t guest_phys_addr;
+	uint64_t memory_size; /* bytes */
+	uint64_t userspace_addr;
+	uint64_t mmap_offset;
+};
+struct vhost_memory_kernel {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[0];
+};
+
+struct vhost_memory {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[VHOST_MEMORY_MAX_NREGIONS];
+};
+
+typedef struct VhostUserMsg {
+	VhostUserRequest request;
+
+#define VHOST_USER_VERSION_MASK     0x3
+#define VHOST_USER_REPLY_MASK       (0x1 << 2)
+	uint32_t flags;
+	uint32_t size; /* the following payload size */
+	union {
+#define VHOST_USER_VRING_IDX_MASK   0xff
+#define VHOST_USER_VRING_NOFD_MASK  (0x1 << 8)
+		uint64_t u64;
+		struct vhost_vring_state state;
+		struct vhost_vring_addr addr;
+		struct vhost_memory memory;
+	} payload;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+} __attribute((packed)) VhostUserMsg;
+
+#define VHOST_USER_HDR_SIZE offsetof(VhostUserMsg, payload.u64)
+#define VHOST_USER_PAYLOAD_SIZE (sizeof(VhostUserMsg) - VHOST_USER_HDR_SIZE)
+
+/* The version of the protocol we support */
+#define VHOST_USER_VERSION    0x1
+
+/* ioctls */
+
+#define VHOST_VIRTIO 0xAF
+
+#define VHOST_GET_FEATURES	_IOR(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_FEATURES	_IOW(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_OWNER _IO(VHOST_VIRTIO, 0x01)
+#define VHOST_RESET_OWNER _IO(VHOST_VIRTIO, 0x02)
+#define VHOST_SET_MEM_TABLE	_IOW(VHOST_VIRTIO, 0x03, struct vhost_memory_kernel)
+#define VHOST_SET_LOG_BASE _IOW(VHOST_VIRTIO, 0x04, __u64)
+#define VHOST_SET_LOG_FD _IOW(VHOST_VIRTIO, 0x07, int)
+#define VHOST_SET_VRING_NUM _IOW(VHOST_VIRTIO, 0x10, struct vhost_vring_state)
+#define VHOST_SET_VRING_ADDR _IOW(VHOST_VIRTIO, 0x11, struct vhost_vring_addr)
+#define VHOST_SET_VRING_BASE _IOW(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_GET_VRING_BASE _IOWR(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_SET_VRING_KICK _IOW(VHOST_VIRTIO, 0x20, struct vhost_vring_file)
+#define VHOST_SET_VRING_CALL _IOW(VHOST_VIRTIO, 0x21, struct vhost_vring_file)
+#define VHOST_SET_VRING_ERR _IOW(VHOST_VIRTIO, 0x22, struct vhost_vring_file)
+#define VHOST_NET_SET_BACKEND _IOW(VHOST_VIRTIO, 0x30, struct vhost_vring_file)
+
+/*****************************************************************************/
+
+/* Ioctl defines */
+#define TUNSETIFF     _IOW('T', 202, int)
+#define TUNGETFEATURES _IOR('T', 207, unsigned int)
+#define TUNSETOFFLOAD  _IOW('T', 208, unsigned int)
+#define TUNGETIFF      _IOR('T', 210, unsigned int)
+#define TUNSETSNDBUF   _IOW('T', 212, int)
+#define TUNGETVNETHDRSZ _IOR('T', 215, int)
+#define TUNSETVNETHDRSZ _IOW('T', 216, int)
+#define TUNSETQUEUE  _IOW('T', 217, int)
+#define TUNSETVNETLE _IOW('T', 220, int)
+#define TUNSETVNETBE _IOW('T', 222, int)
+
+/* TUNSETIFF ifr flags */
+#define IFF_TAP          0x0002
+#define IFF_NO_PI        0x1000
+#define IFF_ONE_QUEUE    0x2000
+#define IFF_VNET_HDR     0x4000
+#define IFF_MULTI_QUEUE  0x0100
+#define IFF_ATTACH_QUEUE 0x0200
+#define IFF_DETACH_QUEUE 0x0400
+
+/* Features for GSO (TUNSETOFFLOAD). */
+#define TUN_F_CSUM	0x01	/* You can hand me unchecksummed packets. */
+#define TUN_F_TSO4	0x02	/* I can handle TSO for IPv4 packets */
+#define TUN_F_TSO6	0x04	/* I can handle TSO for IPv6 packets */
+#define TUN_F_TSO_ECN	0x08	/* I can handle TSO with ECN bits. */
+#define TUN_F_UFO	0x10	/* I can handle UFO packets */
+
+#define PATH_NET_TUN "/dev/net/tun"
+
+#endif
diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
index ae2d47d..9e1ecb3 100644
--- a/drivers/net/virtio/virtio_ethdev.h
+++ b/drivers/net/virtio/virtio_ethdev.h
@@ -122,5 +122,8 @@ uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 #define VTNET_LRO_FEATURES (VIRTIO_NET_F_GUEST_TSO4 | \
 			    VIRTIO_NET_F_GUEST_TSO6 | VIRTIO_NET_F_GUEST_ECN)
 
-
+#ifdef RTE_VIRTIO_VDEV
+void virtio_vdev_init(struct rte_eth_dev_data *data, const char *path,
+		int nb_rx, int nb_tx, int nb_cq, int queue_num, char *mac);
+#endif
 #endif /* _VIRTIO_ETHDEV_H_ */
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index 47f722a..af05ae2 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -147,7 +147,6 @@ struct virtqueue;
  * rest are per-device feature bits.
  */
 #define VIRTIO_TRANSPORT_F_START 28
-#define VIRTIO_TRANSPORT_F_END   32
 
 /* The Guest publishes the used index for which it expects an interrupt
  * at the end of the avail ring. Host should ignore the avail->flags field. */
@@ -165,6 +164,7 @@ struct virtqueue;
 
 struct virtio_hw {
 	struct virtqueue *cvq;
+#define VIRTIO_VDEV_IO_BASE	0xffffffff
 	uint32_t    io_base;
 	uint32_t    guest_features;
 	uint32_t    max_tx_queues;
@@ -174,6 +174,21 @@ struct virtio_hw {
 	uint8_t	    use_msix;
 	uint8_t     started;
 	uint8_t     mac_addr[ETHER_ADDR_LEN];
+#ifdef RTE_VIRTIO_VDEV
+#define VHOST_KERNEL	0
+#define VHOST_USER		1
+	int			type; /* type of backend */
+	uint32_t	queue_num;
+	char		*path;
+	int			mac_specified;
+	int			vhostfd;
+	int			backfd; /* tap device used in vhost-net */
+	int			callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int			kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	uint32_t	queue_sel;
+	uint8_t		status;
+	struct rte_eth_dev_data *data;
+#endif
 };
 
 /*
@@ -229,6 +244,39 @@ outl_p(unsigned int data, unsigned int port)
 #define VIRTIO_PCI_REG_ADDR(hw, reg) \
 	(unsigned short)((hw)->io_base + (reg))
 
+#ifdef RTE_VIRTIO_VDEV
+uint32_t virtio_ioport_read(struct virtio_hw *, uint64_t);
+void virtio_ioport_write(struct virtio_hw *, uint64_t, uint32_t);
+
+#define VIRTIO_READ_REG_1(hw, reg) \
+	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
+	inb((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
+	:virtio_ioport_read(hw, reg)
+#define VIRTIO_WRITE_REG_1(hw, reg, value) \
+	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
+	outb_p((unsigned char)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
+	:virtio_ioport_write(hw, reg, value)
+
+#define VIRTIO_READ_REG_2(hw, reg) \
+	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
+	inw((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
+	:virtio_ioport_read(hw, reg)
+#define VIRTIO_WRITE_REG_2(hw, reg, value) \
+	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
+	outw_p((unsigned short)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
+	:virtio_ioport_write(hw, reg, value)
+
+#define VIRTIO_READ_REG_4(hw, reg) \
+	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
+	inl((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
+	:virtio_ioport_read(hw, reg)
+#define VIRTIO_WRITE_REG_4(hw, reg, value) \
+	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
+	outl_p((unsigned int)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
+	:virtio_ioport_write(hw, reg, value)
+
+#else /* RTE_VIRTIO_VDEV */
+
 #define VIRTIO_READ_REG_1(hw, reg) \
 	inb((VIRTIO_PCI_REG_ADDR((hw), (reg))))
 #define VIRTIO_WRITE_REG_1(hw, reg, value) \
@@ -244,6 +292,8 @@ outl_p(unsigned int data, unsigned int port)
 #define VIRTIO_WRITE_REG_4(hw, reg, value) \
 	outl_p((unsigned int)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg))))
 
+#endif /* RTE_VIRTIO_VDEV */
+
 static inline int
 vtpci_with_feature(struct virtio_hw *hw, uint32_t bit)
 {
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH 4/4] virtio/vdev: add a new vdev named eth_cvio
  2016-01-10 11:42 ` [PATCH 0/4] " Jianfeng Tan
                     ` (2 preceding siblings ...)
  2016-01-10 11:43   ` [PATCH 3/4] virtio/vdev: add ways to interact with vhost Jianfeng Tan
@ 2016-01-10 11:43   ` Jianfeng Tan
  2016-01-12  7:45     ` Pavel Fedin
  2016-01-27  3:10     ` Qiu, Michael
  2016-01-11 14:21   ` [PATCH 0/4] virtio support for container Pavel Fedin
                     ` (2 subsequent siblings)
  6 siblings, 2 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-01-10 11:43 UTC (permalink / raw)
  To: dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

Add a new virtual device named eth_cvio, it can be used just like
eth_ring, eth_null, etc.

Configured parameters include:
    - rx (optional, 1 by default): number of rx, only allowed to be
				   1 for now.
    - tx (optional, 1 by default): number of tx, only allowed to be
				   1 for now.
    - cq (optional, 0 by default): if ctrl queue is enabled, not
				   supported for now.
    - mac (optional): mac address, random value will be given if not
		      specified.
    - queue_num (optional, 256 by default): size of virtqueue.
    - path (madatory): path of vhost, depends on the file type:
		       vhost-user is used if the given path points to
	               a unix socket; vhost-net is used if the given
		       path points to a char device.

The major difference with original virtio for vm is that, here we
use virtual address instead of physical address for vhost to
calculate relative address.

When enable CONFIG_RTE_VIRTIO_VDEV (enabled by default), the compiled
library can be used in both VM and container environment.

Examples:
a. Use vhost-net as a backend
sudo numactl -N 1 -m 1 ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
    -m 1024 --no-pci --single-file --file-prefix=l2fwd \
    --vdev=eth_cvio0,mac=00:01:02:03:04:05,path=/dev/vhost-net \
    -- -p 0x1

b. Use vhost-user as a backend
numactl -N 1 -m 1 ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 -m 1024 \
    --no-pci --single-file --file-prefix=l2fwd \
    --vdev=eth_cvio0,mac=00:01:02:03:04:05,path=<path_to_vhost_user> \
    -- -p 0x1

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 drivers/net/virtio/virtio_ethdev.c      | 338 +++++++++++++++++++++++++-------
 drivers/net/virtio/virtio_ethdev.h      |   1 +
 drivers/net/virtio/virtio_pci.h         |  24 +--
 drivers/net/virtio/virtio_rxtx.c        |  11 +-
 drivers/net/virtio/virtio_rxtx_simple.c |  14 +-
 drivers/net/virtio/virtqueue.h          |  13 +-
 6 files changed, 302 insertions(+), 99 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index d928339..6e46060 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -56,6 +56,7 @@
 #include <rte_memory.h>
 #include <rte_eal.h>
 #include <rte_dev.h>
+#include <rte_kvargs.h>
 
 #include "virtio_ethdev.h"
 #include "virtio_pci.h"
@@ -174,14 +175,14 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	 * One RX packet for ACK.
 	 */
 	vq->vq_ring.desc[head].flags = VRING_DESC_F_NEXT;
-	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mz->phys_addr;
+	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mem;
 	vq->vq_ring.desc[head].len = sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_free_cnt--;
 	i = vq->vq_ring.desc[head].next;
 
 	for (k = 0; k < pkt_num; k++) {
 		vq->vq_ring.desc[i].flags = VRING_DESC_F_NEXT;
-		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr)
 			+ sizeof(ctrl->status) + sizeof(uint8_t)*sum;
 		vq->vq_ring.desc[i].len = dlen[k];
@@ -191,7 +192,7 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	}
 
 	vq->vq_ring.desc[i].flags = VRING_DESC_F_WRITE;
-	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_ring.desc[i].len = sizeof(ctrl->status);
 	vq->vq_free_cnt--;
@@ -374,68 +375,85 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		}
 	}
 
-	/*
-	 * Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
-	 * and only accepts 32 bit page frame number.
-	 * Check if the allocated physical memory exceeds 16TB.
-	 */
-	if ((mz->phys_addr + vq->vq_ring_size - 1) >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
-		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
-		rte_free(vq);
-		return -ENOMEM;
-	}
-
 	memset(mz->addr, 0, sizeof(mz->len));
 	vq->mz = mz;
-	vq->vq_ring_mem = mz->phys_addr;
 	vq->vq_ring_virt_mem = mz->addr;
-	PMD_INIT_LOG(DEBUG, "vq->vq_ring_mem:      0x%"PRIx64, (uint64_t)mz->phys_addr);
-	PMD_INIT_LOG(DEBUG, "vq->vq_ring_virt_mem: 0x%"PRIx64, (uint64_t)(uintptr_t)mz->addr);
+
+	if (dev->dev_type == RTE_ETH_DEV_PCI) {
+		vq->vq_ring_mem = mz->phys_addr;
+
+		/* Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
+		 * and only accepts 32 bit page frame number.
+		 * Check if the allocated physical memory exceeds 16TB.
+		 */
+		uint64_t last_physaddr = vq->vq_ring_mem + vq->vq_ring_size - 1;
+		if (last_physaddr >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
+			PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
+			rte_free(vq);
+			return -ENOMEM;
+		}
+	}
+#ifdef RTE_VIRTIO_VDEV
+	else
+		vq->vq_ring_mem = (phys_addr_t)mz->addr; /* Use vaddr!!! */
+#endif
+
+	PMD_INIT_LOG(DEBUG, "vq->vq_ring_mem:      0x%"PRIx64,
+			(uint64_t)vq->vq_ring_mem);
+	PMD_INIT_LOG(DEBUG, "vq->vq_ring_virt_mem: 0x%"PRIx64,
+			(uint64_t)(uintptr_t)vq->vq_ring_virt_mem);
 	vq->virtio_net_hdr_mz  = NULL;
 	vq->virtio_net_hdr_mem = 0;
 
+	uint64_t hdr_size = 0;
 	if (queue_type == VTNET_TQ) {
 		/*
 		 * For each xmit packet, allocate a virtio_net_hdr
 		 */
 		snprintf(vq_name, sizeof(vq_name), "port%d_tvq%d_hdrzone",
 			dev->data->port_id, queue_idx);
-		vq->virtio_net_hdr_mz = rte_memzone_reserve_aligned(vq_name,
-			vq_size * hw->vtnet_hdr_size,
-			socket_id, 0, RTE_CACHE_LINE_SIZE);
-		if (vq->virtio_net_hdr_mz == NULL) {
-			if (rte_errno == EEXIST)
-				vq->virtio_net_hdr_mz =
-					rte_memzone_lookup(vq_name);
-			if (vq->virtio_net_hdr_mz == NULL) {
-				rte_free(vq);
-				return -ENOMEM;
-			}
-		}
-		vq->virtio_net_hdr_mem =
-			vq->virtio_net_hdr_mz->phys_addr;
-		memset(vq->virtio_net_hdr_mz->addr, 0,
-			vq_size * hw->vtnet_hdr_size);
+		hdr_size = vq_size * hw->vtnet_hdr_size;
 	} else if (queue_type == VTNET_CQ) {
 		/* Allocate a page for control vq command, data and status */
 		snprintf(vq_name, sizeof(vq_name), "port%d_cvq_hdrzone",
 			dev->data->port_id);
-		vq->virtio_net_hdr_mz = rte_memzone_reserve_aligned(vq_name,
-			PAGE_SIZE, socket_id, 0, RTE_CACHE_LINE_SIZE);
-		if (vq->virtio_net_hdr_mz == NULL) {
+		hdr_size = PAGE_SIZE;
+	}
+
+	if (hdr_size) { /* queue_type is VTNET_TQ or VTNET_CQ */
+		mz = rte_memzone_reserve_aligned(vq_name,
+				hdr_size, socket_id, 0, RTE_CACHE_LINE_SIZE);
+		if (mz == NULL) {
 			if (rte_errno == EEXIST)
-				vq->virtio_net_hdr_mz =
-					rte_memzone_lookup(vq_name);
-			if (vq->virtio_net_hdr_mz == NULL) {
+				mz = rte_memzone_lookup(vq_name);
+			if (mz == NULL) {
 				rte_free(vq);
 				return -ENOMEM;
 			}
 		}
-		vq->virtio_net_hdr_mem =
-			vq->virtio_net_hdr_mz->phys_addr;
-		memset(vq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
+		vq->virtio_net_hdr_mz = mz;
+		vq->virtio_net_hdr_vaddr = mz->addr;
+		memset(vq->virtio_net_hdr_vaddr, 0, hdr_size);
+
+		if (dev->dev_type == RTE_ETH_DEV_PCI)
+			vq->virtio_net_hdr_mem = mz->phys_addr;
+#ifdef RTE_VIRTIO_VDEV
+		else
+			vq->virtio_net_hdr_mem = (phys_addr_t)mz->addr; /* Use vaddr!!! */
+#endif
 	}
 
+	struct rte_mbuf *m = NULL;
+	if (dev->dev_type == RTE_ETH_DEV_PCI)
+		vq->offset = (uintptr_t)&m->buf_addr;
+#ifdef RTE_VIRTIO_VDEV
+	else {
+		vq->offset = (uintptr_t)&m->buf_physaddr;
+#if (RTE_BYTE_ORDER == RTE_BIG_ENDIAN) && (__WORDSIZE == 32)
+		vq->offset += 4;
+#endif
+	}
+#endif
 	/*
 	 * Set guest physical address of the virtqueue
 	 * in VIRTIO_PCI_QUEUE_PFN config register of device
@@ -491,8 +509,10 @@ virtio_dev_close(struct rte_eth_dev *dev)
 	PMD_INIT_LOG(DEBUG, "virtio_dev_close");
 
 	/* reset the NIC */
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		vtpci_irq_config(hw, VIRTIO_MSI_NO_VECTOR);
+	if (dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			vtpci_irq_config(hw, VIRTIO_MSI_NO_VECTOR);
+	}
 	vtpci_reset(hw);
 	hw->started = 0;
 	virtio_dev_free_mbufs(dev);
@@ -1233,8 +1253,9 @@ virtio_interrupt_handler(__rte_unused struct rte_intr_handle *handle,
 	isr = vtpci_isr(hw);
 	PMD_DRV_LOG(INFO, "interrupt status = %#x", isr);
 
-	if (rte_intr_enable(&dev->pci_dev->intr_handle) < 0)
-		PMD_DRV_LOG(ERR, "interrupt enable failed");
+	if (dev->dev_type == RTE_ETH_DEV_PCI)
+		if (rte_intr_enable(&dev->pci_dev->intr_handle) < 0)
+			PMD_DRV_LOG(ERR, "interrupt enable failed");
 
 	if (isr & VIRTIO_PCI_ISR_CONFIG) {
 		if (virtio_dev_link_update(dev, 0) == 0)
@@ -1287,11 +1308,18 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	pci_dev = eth_dev->pci_dev;
 
-	if (virtio_resource_init(pci_dev) < 0)
-		return -1;
-
-	hw->use_msix = virtio_has_msix(&pci_dev->addr);
-	hw->io_base = (uint32_t)(uintptr_t)pci_dev->mem_resource[0].addr;
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (virtio_resource_init(pci_dev) < 0)
+			return -1;
+		hw->use_msix = virtio_has_msix(&pci_dev->addr);
+		hw->io_base = (uint32_t)(uintptr_t)pci_dev->mem_resource[0].addr;
+	}
+#ifdef RTE_VIRTIO_VDEV
+	else {
+		hw->use_msix = 0;
+		hw->io_base = VIRTIO_VDEV_IO_BASE;
+	}
+#endif
 
 	/* Reset the device although not necessary at startup */
 	vtpci_reset(hw);
@@ -1304,10 +1332,12 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 	virtio_negotiate_features(hw);
 
 	/* If host does not support status then disable LSC */
-	if (!vtpci_with_feature(hw, VIRTIO_NET_F_STATUS))
-		pci_dev->driver->drv_flags &= ~RTE_PCI_DRV_INTR_LSC;
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (!vtpci_with_feature(hw, VIRTIO_NET_F_STATUS))
+			pci_dev->driver->drv_flags &= ~RTE_PCI_DRV_INTR_LSC;
 
-	rte_eth_copy_pci_info(eth_dev, pci_dev);
+		rte_eth_copy_pci_info(eth_dev, pci_dev);
+	}
 
 	rx_func_get(eth_dev);
 
@@ -1383,15 +1413,16 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	PMD_INIT_LOG(DEBUG, "hw->max_rx_queues=%d   hw->max_tx_queues=%d",
 			hw->max_rx_queues, hw->max_tx_queues);
-	PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
-			eth_dev->data->port_id, pci_dev->id.vendor_id,
-			pci_dev->id.device_id);
-
-	/* Setup interrupt callback  */
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		rte_intr_callback_register(&pci_dev->intr_handle,
-				   virtio_interrupt_handler, eth_dev);
-
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
+				eth_dev->data->port_id, pci_dev->id.vendor_id,
+				pci_dev->id.device_id);
+
+		/* Setup interrupt callback  */
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			rte_intr_callback_register(&pci_dev->intr_handle,
+					virtio_interrupt_handler, eth_dev);
+	}
 	virtio_dev_cq_start(eth_dev);
 
 	return 0;
@@ -1424,10 +1455,12 @@ eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev)
 	eth_dev->data->mac_addrs = NULL;
 
 	/* reset interrupt callback  */
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		rte_intr_callback_unregister(&pci_dev->intr_handle,
-						virtio_interrupt_handler,
-						eth_dev);
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			rte_intr_callback_unregister(&pci_dev->intr_handle,
+					virtio_interrupt_handler,
+					eth_dev);
+	}
 
 	PMD_INIT_LOG(DEBUG, "dev_uninit completed");
 
@@ -1491,11 +1524,13 @@ virtio_dev_configure(struct rte_eth_dev *dev)
 		return -ENOTSUP;
 	}
 
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		if (vtpci_irq_config(hw, 0) == VIRTIO_MSI_NO_VECTOR) {
-			PMD_DRV_LOG(ERR, "failed to set config vector");
-			return -EBUSY;
-		}
+	if (dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			if (vtpci_irq_config(hw, 0) == VIRTIO_MSI_NO_VECTOR) {
+				PMD_DRV_LOG(ERR, "failed to set config vector");
+				return -EBUSY;
+			}
+	}
 
 	return 0;
 }
@@ -1689,3 +1724,162 @@ static struct rte_driver rte_virtio_driver = {
 };
 
 PMD_REGISTER_DRIVER(rte_virtio_driver);
+
+#ifdef RTE_VIRTIO_VDEV
+
+static const char *valid_args[] = {
+#define ETH_CVIO_ARG_RX_NUM		"rx"
+	ETH_CVIO_ARG_RX_NUM,
+#define ETH_CVIO_ARG_TX_NUM		"tx"
+	ETH_CVIO_ARG_TX_NUM,
+#define ETH_CVIO_ARG_CQ_NUM		"cq"
+	ETH_CVIO_ARG_CQ_NUM,
+#define ETH_CVIO_ARG_MAC		"mac"
+	ETH_CVIO_ARG_MAC,
+#define ETH_CVIO_ARG_PATH		"path"
+	ETH_CVIO_ARG_PATH,
+#define ETH_CVIO_ARG_QUEUE_SIZE	"queue_num"
+	ETH_CVIO_ARG_QUEUE_SIZE,
+	NULL
+};
+
+static int
+get_string_arg(const char *key __rte_unused,
+		const char *value, void *extra_args)
+{
+	if ((value == NULL) || (extra_args == NULL))
+		return -EINVAL;
+
+	strcpy(extra_args, value);
+
+	return 0;
+}
+
+static int
+get_integer_arg(const char *key __rte_unused,
+		const char *value, void *extra_args)
+{
+	uint64_t *p_u64 = extra_args;
+
+	if ((value == NULL) || (extra_args == NULL))
+		return -EINVAL;
+
+	*p_u64 = (uint64_t)strtoull(value, NULL, 0);
+
+	return 0;
+}
+
+static struct rte_eth_dev *
+cvio_eth_dev_alloc(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct rte_eth_dev_data *data;
+	struct virtio_hw *hw;
+
+	eth_dev = rte_eth_dev_allocate(name, RTE_ETH_DEV_VIRTUAL);
+	if (eth_dev == NULL)
+		rte_panic("cannot alloc rte_eth_dev\n");
+
+	data = eth_dev->data;
+
+	hw = rte_zmalloc(NULL, sizeof(*hw), 0);
+	if (!hw)
+		rte_panic("malloc virtio_hw failed\n");
+
+	data->dev_private = hw;
+	data->numa_node = SOCKET_ID_ANY;
+	eth_dev->pci_dev = NULL;
+	/* will be used in virtio_dev_info_get() */
+	eth_dev->driver = &rte_virtio_pmd;
+	/* TODO: eth_dev->link_intr_cbs */
+	return eth_dev;
+}
+
+#define CVIO_DEF_CQ_EN		0
+#define CVIO_DEF_Q_NUM		1
+#define CVIO_DEF_Q_SZ		256
+/*
+ * Dev initialization routine. Invoked once for each virtio vdev at
+ * EAL init time, see rte_eal_dev_init().
+ * Returns 0 on success.
+ */
+static int
+rte_cvio_pmd_devinit(const char *name, const char *params)
+{
+	struct rte_kvargs *kvlist = NULL;
+	struct rte_eth_dev *eth_dev = NULL;
+	uint64_t nb_rx = CVIO_DEF_Q_NUM;
+	uint64_t nb_tx = CVIO_DEF_Q_NUM;
+	uint64_t nb_cq = CVIO_DEF_CQ_EN;
+	uint64_t queue_num = CVIO_DEF_Q_SZ;
+	char sock_path[256];
+	char mac_addr[32];
+	int flag_mac = 0;
+
+	if (params == NULL || params[0] == '\0')
+		rte_panic("arg %s is mandatory for eth_cvio\n",
+				ETH_CVIO_ARG_QUEUE_SIZE);
+
+	kvlist = rte_kvargs_parse(params, valid_args);
+	if (!kvlist)
+		rte_panic("error when parsing param\n");
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_PATH) == 1)
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_PATH,
+				&get_string_arg, sock_path);
+	else
+		rte_panic("arg %s is mandatory for eth_cvio\n",
+				ETH_CVIO_ARG_QUEUE_SIZE);
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_MAC) == 1) {
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_MAC,
+				&get_string_arg, mac_addr);
+		flag_mac = 1;
+	}
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_QUEUE_SIZE) == 1)
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_QUEUE_SIZE,
+			&get_integer_arg, &queue_num);
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_RX_NUM) == 1)
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_RX_NUM,
+			&get_integer_arg, &nb_rx);
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_TX_NUM) == 1)
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_TX_NUM,
+			&get_integer_arg, &nb_tx);
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_CQ_NUM) == 1)
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_CQ_NUM,
+			&get_integer_arg, &nb_cq);
+
+	eth_dev = cvio_eth_dev_alloc(name);
+
+	virtio_vdev_init(eth_dev->data, sock_path,
+			nb_rx, nb_tx, nb_cq, queue_num,
+			(flag_mac) ? mac_addr : NULL);
+
+	/* originally, this will be called in rte_eal_pci_probe() */
+	eth_virtio_dev_init(eth_dev);
+
+	return 0;
+}
+
+static int
+rte_cvio_pmd_devuninit(const char *name)
+{
+	/* TODO: if it's last one, memory init, free memory */
+	rte_panic("%s: %s", __func__, name);
+	return 0;
+}
+
+static struct rte_driver rte_cvio_driver = {
+	.name   = "eth_cvio",
+	.type   = PMD_VDEV,
+	.init   = rte_cvio_pmd_devinit,
+	.uninit = rte_cvio_pmd_devuninit,
+};
+
+PMD_REGISTER_DRIVER(rte_cvio_driver);
+
+#endif
diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
index 9e1ecb3..90890b4 100644
--- a/drivers/net/virtio/virtio_ethdev.h
+++ b/drivers/net/virtio/virtio_ethdev.h
@@ -126,4 +126,5 @@ uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 void virtio_vdev_init(struct rte_eth_dev_data *data, const char *path,
 		int nb_rx, int nb_tx, int nb_cq, int queue_num, char *mac);
 #endif
+
 #endif /* _VIRTIO_ETHDEV_H_ */
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index af05ae2..d79bd05 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -249,31 +249,31 @@ uint32_t virtio_ioport_read(struct virtio_hw *, uint64_t);
 void virtio_ioport_write(struct virtio_hw *, uint64_t, uint32_t);
 
 #define VIRTIO_READ_REG_1(hw, reg) \
-	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
+	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
 	inb((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
-	:virtio_ioport_read(hw, reg)
+	:virtio_ioport_read(hw, reg))
 #define VIRTIO_WRITE_REG_1(hw, reg, value) \
-	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
+	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
 	outb_p((unsigned char)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
-	:virtio_ioport_write(hw, reg, value)
+	:virtio_ioport_write(hw, reg, value))
 
 #define VIRTIO_READ_REG_2(hw, reg) \
-	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
+	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
 	inw((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
-	:virtio_ioport_read(hw, reg)
+	:virtio_ioport_read(hw, reg))
 #define VIRTIO_WRITE_REG_2(hw, reg, value) \
-	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
+	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
 	outw_p((unsigned short)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
-	:virtio_ioport_write(hw, reg, value)
+	:virtio_ioport_write(hw, reg, value))
 
 #define VIRTIO_READ_REG_4(hw, reg) \
-	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
+	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
 	inl((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
-	:virtio_ioport_read(hw, reg)
+	:virtio_ioport_read(hw, reg))
 #define VIRTIO_WRITE_REG_4(hw, reg, value) \
-	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
+	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
 	outl_p((unsigned int)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
-	:virtio_ioport_write(hw, reg, value)
+	:virtio_ioport_write(hw, reg, value))
 
 #else /* RTE_VIRTIO_VDEV */
 
diff --git a/drivers/net/virtio/virtio_rxtx.c b/drivers/net/virtio/virtio_rxtx.c
index 74b39ef..dd07ba7 100644
--- a/drivers/net/virtio/virtio_rxtx.c
+++ b/drivers/net/virtio/virtio_rxtx.c
@@ -191,8 +191,7 @@ virtqueue_enqueue_recv_refill(struct virtqueue *vq, struct rte_mbuf *cookie)
 
 	start_dp = vq->vq_ring.desc;
 	start_dp[idx].addr =
-		(uint64_t)(cookie->buf_physaddr + RTE_PKTMBUF_HEADROOM
-		- hw->vtnet_hdr_size);
+		RTE_MBUF_DATA_DMA_ADDR(cookie, vq->offset) - hw->vtnet_hdr_size;
 	start_dp[idx].len =
 		cookie->buf_len - RTE_PKTMBUF_HEADROOM + hw->vtnet_hdr_size;
 	start_dp[idx].flags =  VRING_DESC_F_WRITE;
@@ -237,7 +236,7 @@ virtqueue_enqueue_xmit(struct virtqueue *txvq, struct rte_mbuf *cookie)
 
 	for (; ((seg_num > 0) && (cookie != NULL)); seg_num--) {
 		idx = start_dp[idx].next;
-		start_dp[idx].addr  = RTE_MBUF_DATA_DMA_ADDR(cookie);
+		start_dp[idx].addr  = RTE_MBUF_DATA_DMA_ADDR(cookie, txvq->offset);
 		start_dp[idx].len   = cookie->data_len;
 		start_dp[idx].flags = VRING_DESC_F_NEXT;
 		cookie = cookie->next;
@@ -343,7 +342,7 @@ virtio_dev_vring_start(struct virtqueue *vq, int queue_type)
 		VIRTIO_WRITE_REG_2(vq->hw, VIRTIO_PCI_QUEUE_SEL,
 			vq->vq_queue_index);
 		VIRTIO_WRITE_REG_4(vq->hw, VIRTIO_PCI_QUEUE_PFN,
-			vq->mz->phys_addr >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
+			vq->vq_ring_mem >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
 	} else if (queue_type == VTNET_TQ) {
 		if (use_simple_rxtx) {
 			int mid_idx  = vq->vq_nentries >> 1;
@@ -366,12 +365,12 @@ virtio_dev_vring_start(struct virtqueue *vq, int queue_type)
 		VIRTIO_WRITE_REG_2(vq->hw, VIRTIO_PCI_QUEUE_SEL,
 			vq->vq_queue_index);
 		VIRTIO_WRITE_REG_4(vq->hw, VIRTIO_PCI_QUEUE_PFN,
-			vq->mz->phys_addr >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
+			vq->vq_ring_mem >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
 	} else {
 		VIRTIO_WRITE_REG_2(vq->hw, VIRTIO_PCI_QUEUE_SEL,
 			vq->vq_queue_index);
 		VIRTIO_WRITE_REG_4(vq->hw, VIRTIO_PCI_QUEUE_PFN,
-			vq->mz->phys_addr >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
+			vq->vq_ring_mem >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
 	}
 }
 
diff --git a/drivers/net/virtio/virtio_rxtx_simple.c b/drivers/net/virtio/virtio_rxtx_simple.c
index ff3c11a..3a14a4e 100644
--- a/drivers/net/virtio/virtio_rxtx_simple.c
+++ b/drivers/net/virtio/virtio_rxtx_simple.c
@@ -80,8 +80,8 @@ virtqueue_enqueue_recv_refill_simple(struct virtqueue *vq,
 	vq->sw_ring[desc_idx] = cookie;
 
 	start_dp = vq->vq_ring.desc;
-	start_dp[desc_idx].addr = (uint64_t)((uintptr_t)cookie->buf_physaddr +
-		RTE_PKTMBUF_HEADROOM - sizeof(struct virtio_net_hdr));
+	start_dp[desc_idx].addr = RTE_MBUF_DATA_DMA_ADDR(cookie, vq->offset)
+		- sizeof(struct virtio_net_hdr);
 	start_dp[desc_idx].len = cookie->buf_len -
 		RTE_PKTMBUF_HEADROOM + sizeof(struct virtio_net_hdr);
 
@@ -118,9 +118,8 @@ virtio_rxq_rearm_vec(struct virtqueue *rxvq)
 		p = (uintptr_t)&sw_ring[i]->rearm_data;
 		*(uint64_t *)p = rxvq->mbuf_initializer;
 
-		start_dp[i].addr =
-			(uint64_t)((uintptr_t)sw_ring[i]->buf_physaddr +
-			RTE_PKTMBUF_HEADROOM - sizeof(struct virtio_net_hdr));
+		start_dp[i].addr = RTE_MBUF_DATA_DMA_ADDR(sw_ring[i], rxvq->offset)
+			- sizeof(struct virtio_net_hdr);
 		start_dp[i].len = sw_ring[i]->buf_len -
 			RTE_PKTMBUF_HEADROOM + sizeof(struct virtio_net_hdr);
 	}
@@ -366,7 +365,7 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 			txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 		for (i = 0; i < nb_tail; i++) {
 			start_dp[desc_idx].addr =
-				RTE_MBUF_DATA_DMA_ADDR(*tx_pkts);
+				RTE_MBUF_DATA_DMA_ADDR(*tx_pkts, txvq->offset);
 			start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 			tx_pkts++;
 			desc_idx++;
@@ -377,7 +376,8 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 	for (i = 0; i < nb_commit; i++)
 		txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 	for (i = 0; i < nb_commit; i++) {
-		start_dp[desc_idx].addr = RTE_MBUF_DATA_DMA_ADDR(*tx_pkts);
+		start_dp[desc_idx].addr = RTE_MBUF_DATA_DMA_ADDR(*tx_pkts,
+				txvq->offset);
 		start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 		tx_pkts++;
 		desc_idx++;
diff --git a/drivers/net/virtio/virtqueue.h b/drivers/net/virtio/virtqueue.h
index 61b3137..dc0b656 100644
--- a/drivers/net/virtio/virtqueue.h
+++ b/drivers/net/virtio/virtqueue.h
@@ -66,8 +66,14 @@ struct rte_mbuf;
 
 #define VIRTQUEUE_MAX_NAME_SZ 32
 
-#define RTE_MBUF_DATA_DMA_ADDR(mb) \
+#ifdef RTE_VIRTIO_VDEV
+#define RTE_MBUF_DATA_DMA_ADDR(mb, offset) \
+	(uint64_t)((uintptr_t)(*(void **)((uintptr_t)mb + offset)) \
+			+ (mb)->data_off)
+#else
+#define RTE_MBUF_DATA_DMA_ADDR(mb, offset) \
 	(uint64_t) ((mb)->buf_physaddr + (mb)->data_off)
+#endif /* RTE_VIRTIO_VDEV */
 
 #define VTNET_SQ_RQ_QUEUE_IDX 0
 #define VTNET_SQ_TQ_QUEUE_IDX 1
@@ -167,7 +173,8 @@ struct virtqueue {
 
 	void        *vq_ring_virt_mem;    /**< linear address of vring*/
 	unsigned int vq_ring_size;
-	phys_addr_t vq_ring_mem;          /**< physical address of vring */
+	phys_addr_t vq_ring_mem;          /**< phys address of vring for pci dev,
+										virt addr of vring for vdev */
 
 	struct vring vq_ring;    /**< vring keeping desc, used and avail */
 	uint16_t    vq_free_cnt; /**< num of desc available */
@@ -186,8 +193,10 @@ struct virtqueue {
 	 */
 	uint16_t vq_used_cons_idx;
 	uint16_t vq_avail_idx;
+	uint16_t offset; /**< relative offset to obtain addr in mbuf */
 	uint64_t mbuf_initializer; /**< value to init mbufs. */
 	phys_addr_t virtio_net_hdr_mem; /**< hdr for each xmit packet */
+	void        *virtio_net_hdr_vaddr;    /**< linear address of vring*/
 
 	struct rte_mbuf **sw_ring; /**< RX software ring. */
 	/* dummy mbuf, for wraparound when processing RX ring. */
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* Re: [PATCH 3/4] virtio/vdev: add ways to interact with vhost
  2016-01-10 11:43   ` [PATCH 3/4] virtio/vdev: add ways to interact with vhost Jianfeng Tan
@ 2016-01-11 10:42     ` Pavel Fedin
  2016-01-11 14:02     ` Pavel Fedin
  2016-01-21  2:18     ` Xie, Huawei
  2 siblings, 0 replies; 196+ messages in thread
From: Pavel Fedin @ 2016-01-11 10:42 UTC (permalink / raw)
  To: 'Jianfeng Tan', dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

 Hello!

 Please, see inline

> -----Original Message-----
> From: Jianfeng Tan [mailto:jianfeng.tan@intel.com]
> Sent: Sunday, January 10, 2016 2:43 PM
> To: dev@dpdk.org
> Cc: rich.lane@bigswitch.com; yuanhan.liu@linux.intel.com; mst@redhat.com;
> nakajima.yoshihiro@lab.ntt.co.jp; huawei.xie@intel.com; mukawa@igel.co.jp;
> p.fedin@samsung.com; michael.qiu@intel.com; ann.zhuangyanying@huawei.com; Jianfeng Tan
> Subject: [PATCH 3/4] virtio/vdev: add ways to interact with vhost
> 
> Depends on the type of vhost file: vhost-user is used if the given
> path points to a unix socket; vhost-net is used if the given path
> points to a char device.
> 
> NOTE: we now keep CONFIG_RTE_VIRTIO_VDEV undefined by default, need
> to be uncommented when in use.
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ---
>  config/common_linuxapp             |   5 +
>  drivers/net/virtio/Makefile        |   4 +
>  drivers/net/virtio/vhost.c         | 734 +++++++++++++++++++++++++++++++++++++
>  drivers/net/virtio/vhost.h         | 192 ++++++++++
>  drivers/net/virtio/virtio_ethdev.h |   5 +-
>  drivers/net/virtio/virtio_pci.h    |  52 ++-
>  6 files changed, 990 insertions(+), 2 deletions(-)
>  create mode 100644 drivers/net/virtio/vhost.c
>  create mode 100644 drivers/net/virtio/vhost.h
> 
> diff --git a/config/common_linuxapp b/config/common_linuxapp
> index 74bc515..f76e162 100644
> --- a/config/common_linuxapp
> +++ b/config/common_linuxapp
> @@ -534,3 +534,8 @@ CONFIG_RTE_APP_TEST=y
>  CONFIG_RTE_TEST_PMD=y
>  CONFIG_RTE_TEST_PMD_RECORD_CORE_CYCLES=n
>  CONFIG_RTE_TEST_PMD_RECORD_BURST_STATS=n
> +
> +#
> +# Enable virtio support for container
> +#
> +CONFIG_RTE_VIRTIO_VDEV=y
> diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
> index 43835ba..0877023 100644
> --- a/drivers/net/virtio/Makefile
> +++ b/drivers/net/virtio/Makefile
> @@ -52,6 +52,10 @@ SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx.c
>  SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_ethdev.c
>  SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx_simple.c
> 
> +ifeq ($(CONFIG_RTE_VIRTIO_VDEV),y)
> +	SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += vhost.c
> +endif
> +
>  # this lib depends upon:
>  DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_eal lib/librte_ether
>  DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_mempool lib/librte_mbuf
> diff --git a/drivers/net/virtio/vhost.c b/drivers/net/virtio/vhost.c
> new file mode 100644
> index 0000000..e423e02
> --- /dev/null
> +++ b/drivers/net/virtio/vhost.c
> @@ -0,0 +1,734 @@
> +/*-
> + *   BSD LICENSE
> + *
> + *   Copyright(c) 2010-2015 Intel Corporation. All rights reserved.
> + *   All rights reserved.
> + *
> + *   Redistribution and use in source and binary forms, with or without
> + *   modification, are permitted provided that the following conditions
> + *   are met:
> + *
> + *     * Redistributions of source code must retain the above copyright
> + *       notice, this list of conditions and the following disclaimer.
> + *     * Redistributions in binary form must reproduce the above copyright
> + *       notice, this list of conditions and the following disclaimer in
> + *       the documentation and/or other materials provided with the
> + *       distribution.
> + *     * Neither the name of Intel Corporation nor the names of its
> + *       contributors may be used to endorse or promote products derived
> + *       from this software without specific prior written permission.
> + *
> + *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
> + *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
> + *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
> + *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
> + *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
> + *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
> + *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
> + *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
> + *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
> + *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
> + *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
> + */
> +
> +#include <stdint.h>
> +#include <sys/types.h>
> +#include <sys/socket.h>
> +#include <sys/un.h>
> +#include <stdio.h>
> +#include <sys/stat.h>
> +#include <fcntl.h>
> +#include <string.h>
> +#include <errno.h>
> +#include <assert.h>
> +#include <sys/mman.h>
> +#include <unistd.h>
> +#include <sys/eventfd.h>
> +#include <sys/ioctl.h>
> +#include <net/if.h>
> +
> +#include <rte_mbuf.h>
> +#include <rte_memory.h>
> +#include <rte_eal_memconfig.h>
> +
> +#include "virtio_pci.h"
> +#include "virtio_logs.h"
> +#include "virtio_ethdev.h"
> +#include "virtqueue.h"
> +#include "vhost.h"
> +
> +static int
> +vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
> +{
> +	struct msghdr msgh;
> +	struct iovec iov;
> +	int r;
> +
> +	size_t fd_size = fd_num * sizeof(int);
> +	char control[CMSG_SPACE(fd_size)];
> +	struct cmsghdr *cmsg;
> +
> +	memset(&msgh, 0, sizeof(msgh));
> +	memset(control, 0, sizeof(control));
> +
> +	iov.iov_base = (uint8_t *)buf;
> +	iov.iov_len = len;
> +
> +	msgh.msg_iov = &iov;
> +	msgh.msg_iovlen = 1;
> +
> +	msgh.msg_control = control;
> +	msgh.msg_controllen = sizeof(control);
> +
> +	cmsg = CMSG_FIRSTHDR(&msgh);
> +
> +	cmsg->cmsg_len = CMSG_LEN(fd_size);
> +	cmsg->cmsg_level = SOL_SOCKET;
> +	cmsg->cmsg_type = SCM_RIGHTS;
> +	memcpy(CMSG_DATA(cmsg), fds, fd_size);
> +
> +	do {
> +		r = sendmsg(fd, &msgh, 0);
> +	} while (r < 0 && errno == EINTR);
> +
> +	return r;
> +}
> +
> +static int
> +vhost_user_read(int fd, VhostUserMsg *msg)
> +{
> +	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
> +	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
> +
> +	ret = recv(fd, (void *)msg, sz_hdr, 0);
> +	if (ret < sz_hdr) {
> +		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
> +				ret, sz_hdr);
> +		goto fail;
> +	}
> +
> +	/* validate msg flags */
> +	if (msg->flags != (valid_flags)) {
> +		PMD_DRV_LOG(ERR, "Failed to recv msg: flags 0x%x instead of 0x%x.",
> +				msg->flags, valid_flags);
> +		goto fail;
> +	}
> +
> +	sz_payload = msg->size;
> +	if (sz_payload) {
> +		ret = recv(fd, (void *)((uint8_t *)msg + sz_hdr), sz_payload, 0);
> +		if (ret < sz_payload) {
> +			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
> +					ret, msg->size);
> +			goto fail;
> +		}
> +	}
> +
> +	return 0;
> +
> +fail:
> +	return -1;
> +}
> +
> +static VhostUserMsg m __attribute__ ((unused));
> +
> +static void
> +prepare_vhost_memory_user(VhostUserMsg *msg, int fds[])
> +{
> +	int i, num;
> +	struct back_file *huges;
> +	struct vhost_memory_region *mr;
> +
> +	num = rte_eal_get_backfile_info(&huges);
> +
> +	if (num > VHOST_MEMORY_MAX_NREGIONS)
> +		rte_panic("%d hugepage files exceed the maximum of %d for "
> +				"vhost-user\n", num, VHOST_MEMORY_MAX_NREGIONS);
> +
> +	for (i = 0; i < num; ++i) {
> +		mr = &msg->payload.memory.regions[i];
> +		mr->guest_phys_addr = (uint64_t)huges[i].addr; /* use vaddr! */
> +		mr->userspace_addr = (uint64_t)huges[i].addr;
> +		mr->memory_size = huges[i].size;
> +		mr->mmap_offset = 0;
> +		fds[i] = open(huges[i].filepath, O_RDWR);
> +	}
> +
> +	msg->payload.memory.nregions = num;
> +	msg->payload.memory.padding = 0;
> +	free(huges);
> +}
> +
> +static int
> +vhost_user_sock(struct virtio_hw *hw, unsigned long int req, void *arg)
> +{
> +	VhostUserMsg msg;
> +	struct vhost_vring_file *file = 0;
> +	int need_reply = 0;
> +	int fds[VHOST_MEMORY_MAX_NREGIONS];
> +	int fd_num = 0;
> +	int i, len;
> +
> +	msg.request = req;
> +	msg.flags = VHOST_USER_VERSION;
> +	msg.size = 0;
> +
> +	switch (req) {
> +	case VHOST_USER_GET_FEATURES:
> +		need_reply = 1;
> +		break;
> +
> +	case VHOST_USER_SET_FEATURES:
> +	case VHOST_USER_SET_LOG_BASE:
> +		msg.payload.u64 = *((__u64 *)arg);
> +		msg.size = sizeof(m.payload.u64);
> +		break;
> +
> +	case VHOST_USER_SET_OWNER:
> +	case VHOST_USER_RESET_OWNER:
> +		break;
> +
> +	case VHOST_USER_SET_MEM_TABLE:
> +		prepare_vhost_memory_user(&msg, fds);
> +		fd_num = msg.payload.memory.nregions;
> +		msg.size = sizeof(m.payload.memory.nregions);
> +		msg.size += sizeof(m.payload.memory.padding);
> +		msg.size += fd_num * sizeof(struct vhost_memory_region);
> +		break;
> +
> +	case VHOST_USER_SET_LOG_FD:
> +		fds[fd_num++] = *((int *)arg);
> +		break;
> +
> +	case VHOST_USER_SET_VRING_NUM:
> +	case VHOST_USER_SET_VRING_BASE:
> +		memcpy(&msg.payload.state, arg, sizeof(struct vhost_vring_state));
> +		msg.size = sizeof(m.payload.state);
> +		break;
> +
> +	case VHOST_USER_GET_VRING_BASE:
> +		memcpy(&msg.payload.state, arg, sizeof(struct vhost_vring_state));
> +		msg.size = sizeof(m.payload.state);
> +		need_reply = 1;
> +		break;
> +
> +	case VHOST_USER_SET_VRING_ADDR:
> +		memcpy(&msg.payload.addr, arg, sizeof(struct vhost_vring_addr));
> +		msg.size = sizeof(m.payload.addr);
> +		break;
> +
> +	case VHOST_USER_SET_VRING_KICK:
> +	case VHOST_USER_SET_VRING_CALL:
> +	case VHOST_USER_SET_VRING_ERR:
> +		file = arg;
> +		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
> +		msg.size = sizeof(m.payload.u64);
> +		if (file->fd > 0)
> +			fds[fd_num++] = file->fd;
> +		else
> +			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
> +		break;
> +
> +	default:
> +		PMD_DRV_LOG(ERR, "vhost-user trying to send unhandled msg type");
> +		return -1;
> +	}
> +
> +	len = VHOST_USER_HDR_SIZE + msg.size;
> +	if (vhost_user_write(hw->vhostfd, &msg, len, fds, fd_num) < 0)
> +		return 0;
> +
> +	if (req == VHOST_USER_SET_MEM_TABLE)
> +		for (i = 0; i < fd_num; ++i)
> +			close(fds[i]);
> +
> +	if (need_reply) {
> +		if (vhost_user_read(hw->vhostfd, &msg) < 0)
> +			return -1;
> +
> +		if (req != msg.request) {
> +			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
> +			return -1;
> +		}
> +
> +		switch (req) {
> +		case VHOST_USER_GET_FEATURES:
> +			if (msg.size != sizeof(m.payload.u64)) {
> +				PMD_DRV_LOG(ERR, "Received bad msg size.");
> +				return -1;
> +			}
> +			*((__u64 *)arg) = msg.payload.u64;
> +			break;
> +		case VHOST_USER_GET_VRING_BASE:
> +			if (msg.size != sizeof(m.payload.state)) {
> +				PMD_DRV_LOG(ERR, "Received bad msg size.");
> +				return -1;
> +			}
> +			memcpy(arg, &msg.payload.state, sizeof(struct vhost_vring_state));
> +			break;
> +		default:
> +			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
> +			return -1;
> +		}
> +	}
> +
> +	return 0;
> +}
> +
> +static int
> +vhost_kernel_ioctl(struct virtio_hw *hw, unsigned long int req, void *arg)
> +{
> +	return ioctl(hw->vhostfd, req, arg);
> +}
> +
> +enum {
> +	VHOST_MSG_SET_OWNER,
> +	VHOST_MSG_SET_FEATURES,
> +	VHOST_MSG_GET_FEATURES,
> +	VHOST_MSG_SET_VRING_CALL,
> +	VHOST_MSG_SET_VRING_NUM,
> +	VHOST_MSG_SET_VRING_BASE,
> +	VHOST_MSG_SET_VRING_ADDR,
> +	VHOST_MSG_SET_VRING_KICK,
> +	VHOST_MSG_SET_MEM_TABLE,
> +	VHOST_MSG_MAX,
> +};
> +
> +static const char *vhost_msg_strings[] = {
> +	"VHOST_MSG_SET_OWNER",
> +	"VHOST_MSG_SET_FEATURES",
> +	"VHOST_MSG_GET_FEATURES",
> +	"VHOST_MSG_SET_VRING_CALL",
> +	"VHOST_MSG_SET_VRING_NUM",
> +	"VHOST_MSG_SET_VRING_BASE",
> +	"VHOST_MSG_SET_VRING_ADDR",
> +	"VHOST_MSG_SET_VRING_KICK",
> +	"VHOST_MSG_SET_MEM_TABLE",
> +	NULL,
> +};
> +
> +static unsigned long int vhost_req_map[][2] = {
> +	{VHOST_SET_OWNER,		VHOST_USER_SET_OWNER},
> +	{VHOST_SET_FEATURES,	VHOST_USER_SET_FEATURES},
> +	{VHOST_GET_FEATURES,	VHOST_USER_GET_FEATURES},
> +	{VHOST_SET_VRING_CALL,	VHOST_USER_SET_VRING_CALL},
> +	{VHOST_SET_VRING_NUM,	VHOST_USER_SET_VRING_NUM},
> +	{VHOST_SET_VRING_BASE,	VHOST_USER_SET_VRING_BASE},
> +	{VHOST_SET_VRING_ADDR,	VHOST_USER_SET_VRING_ADDR},
> +	{VHOST_SET_VRING_KICK,	VHOST_USER_SET_VRING_KICK},
> +	{VHOST_SET_MEM_TABLE,	VHOST_USER_SET_MEM_TABLE},
> +};
> +
> +static int
> +vhost_call(struct virtio_hw *hw, unsigned long int req_orig, void *arg)
> +{
> +	int req_new;
> +	int ret = 0;
> +
> +	if (req_orig >= VHOST_MSG_MAX)
> +		rte_panic("invalid req: %lu\n", req_orig);
> +
> +	PMD_DRV_LOG(INFO, "%s\n", vhost_msg_strings[req_orig]);
> +	req_new = vhost_req_map[req_orig][hw->type];
> +	if (hw->type == VHOST_USER)
> +		ret = vhost_user_sock(hw, req_new, arg);
> +	else
> +		ret = vhost_kernel_ioctl(hw, req_new, arg);
> +
> +	if (ret < 0)
> +		rte_panic("vhost_call %s failed: %s\n",
> +				vhost_msg_strings[req_orig], strerror(errno));
> +
> +	return ret;
> +}
> +
> +static void
> +kick_one_vq(struct virtio_hw *hw, struct virtqueue *vq, unsigned queue_sel)
> +{
> +	int callfd, kickfd;
> +	struct vhost_vring_file file;
> +
> +	/* or use invalid flag to disable it, but vhost-dpdk uses this to judge
> +	 * if dev is alive. so finally we need two real event_fds.
> +	 */
> +	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL comes
> +	 * firstly because vhost depends on this msg to allocate virtqueue
> +	 * pair.
> +	 */
> +	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
> +	if (callfd < 0)
> +		rte_panic("callfd error, %s\n", strerror(errno));
> +
> +	file.index = queue_sel;
> +	file.fd = callfd;
> +	vhost_call(hw, VHOST_MSG_SET_VRING_CALL, &file);
> +	hw->callfds[queue_sel] = callfd;
> +
> +	struct vhost_vring_state state;
> +	state.index = queue_sel;
> +	state.num = vq->vq_ring.num;
> +	vhost_call(hw, VHOST_MSG_SET_VRING_NUM, &state);
> +
> +	state.num = 0; /* no reservation */
> +	vhost_call(hw, VHOST_MSG_SET_VRING_BASE, &state);
> +
> +	struct vhost_vring_addr addr = {
> +		.index = queue_sel,
> +		.desc_user_addr = (uint64_t)vq->vq_ring.desc,
> +		.avail_user_addr = (uint64_t)vq->vq_ring.avail,
> +		.used_user_addr = (uint64_t)vq->vq_ring.used,
> +		.log_guest_addr = 0,
> +		.flags = 0, /* disable log */
> +	};
> +	vhost_call(hw, VHOST_MSG_SET_VRING_ADDR, &addr);
> +
> +	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_KICK comes
> +	 * lastly because vhost depends on this msg to judge if
> +	 * virtio_is_ready().
> +	 */
> +
> +	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
> +	if (kickfd < 0)
> +		rte_panic("kickfd error, %s\n", strerror(errno));
> +
> +	file.fd = kickfd;
> +	vhost_call(hw, VHOST_MSG_SET_VRING_KICK, &file);
> +	hw->kickfds[queue_sel] = kickfd;
> +}
> +
> +/**
> + * Merge those virtually adjacent memsegs into one region.
> + */
> +static void
> +prepare_vhost_memory_kernel(struct vhost_memory_kernel **p_vm)
> +{
> +	unsigned i, j, k = 0;
> +	struct rte_memseg *seg;
> +	struct vhost_memory_region *mr;
> +	struct vhost_memory_kernel *vm;
> +
> +	vm = malloc(sizeof(struct vhost_memory_kernel)
> +			+ RTE_MAX_MEMSEG * sizeof(struct vhost_memory_region));
> +
> +	for (i = 0; i < RTE_MAX_MEMSEG; ++i) {
> +		seg = &rte_eal_get_configuration()->mem_config->memseg[i];
> +		if (seg->addr == NULL)
> +			break;
> +
> +		int new_region = 1;
> +		for (j = 0; j < k; ++j) {
> +			mr = &vm->regions[j];
> +
> +			if (mr->userspace_addr + mr->memory_size
> +					== (uint64_t)seg->addr) {
> +				mr->memory_size += seg->len;
> +				new_region = 0;
> +				break;
> +			}
> +
> +			if ((uint64_t)seg->addr + seg->len
> +					== mr->userspace_addr) {
> +				mr->guest_phys_addr = (uint64_t)seg->addr;
> +				mr->userspace_addr = (uint64_t)seg->addr;
> +				mr->memory_size += seg->len;
> +				new_region = 0;
> +				break;
> +			}
> +		}
> +
> +		if (new_region == 0)
> +			continue;
> +
> +		mr = &vm->regions[k++];
> +		mr->guest_phys_addr = (uint64_t)seg->addr; /* use vaddr here! */
> +		mr->userspace_addr = (uint64_t)seg->addr;
> +		mr->memory_size = seg->len;
> +		mr->mmap_offset = 0;
> +	}
> +
> +	vm->nregions = k;
> +	vm->padding = 0;
> +	*p_vm = vm;
> +}
> +
> +static void kick_all_vq(struct virtio_hw *hw)
> +{
> +	int ret;
> +	unsigned i, queue_sel, nvqs;
> +	struct rte_eth_dev_data *data = hw->data;
> +
> +	if (hw->type == VHOST_KERNEL) {
> +		struct vhost_memory_kernel *vm = NULL;
> +		prepare_vhost_memory_kernel(&vm);
> +		vhost_call(hw, VHOST_MSG_SET_MEM_TABLE, vm);
> +		free(vm);
> +	} else {
> +		/* construct vhost_memory inside prepare_vhost_memory_user() */
> +		vhost_call(hw, VHOST_MSG_SET_MEM_TABLE, NULL);
> +	}
> +
> +	for (i = 0; i < data->nb_rx_queues; ++i) {
> +		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
> +		kick_one_vq(hw, data->rx_queues[i], queue_sel);
> +	}
> +	for (i = 0; i < data->nb_tx_queues; ++i) {
> +		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
> +		kick_one_vq(hw, data->tx_queues[i], queue_sel);
> +	}
> +
> +	/* after setup all virtqueues, we need to set_features again
> +	 * so that these features can be set into each virtqueue in
> +	 * vhost side.
> +	 */
> +	uint64_t features = hw->guest_features;
> +	features &= ~(1ull << VIRTIO_NET_F_MAC);
> +	vhost_call(hw, VHOST_MSG_SET_FEATURES, &features);
> +	if (ioctl(hw->backfd, TUNSETVNETHDRSZ, &hw->vtnet_hdr_size) == -1)
> +		rte_panic("TUNSETVNETHDRSZ failed: %s\n", strerror(errno));
> +	PMD_DRV_LOG(INFO, "set features:%"PRIx64"\n", features);
> +
> +	if (hw->type == VHOST_KERNEL) {
> +		struct vhost_vring_file file;
> +
> +		file.fd = hw->backfd;
> +		nvqs = data->nb_rx_queues + data->nb_tx_queues;
> +		for (file.index = 0; file.index < nvqs; ++file.index) {
> +			ret = vhost_kernel_ioctl(hw, VHOST_NET_SET_BACKEND, &file);
> +			if (ret < 0)
> +				rte_panic("VHOST_NET_SET_BACKEND failed, %s\n",
> +						strerror(errno));
> +		}
> +	}
> +
> +	/* TODO: VHOST_SET_LOG_BASE */
> +}
> +
> +void
> +virtio_ioport_write(struct virtio_hw *hw, uint64_t addr, uint32_t val)
> +{
> +	uint64_t guest_features;
> +
> +	switch (addr) {
> +	case VIRTIO_PCI_GUEST_FEATURES:
> +		guest_features = val;
> +		guest_features &= ~(1ull << VIRTIO_NET_F_MAC);
> +		vhost_call(hw, VHOST_MSG_SET_FEATURES, &guest_features);
> +		break;
> +	case VIRTIO_PCI_QUEUE_PFN:
> +		/* do nothing */
> +		break;
> +	case VIRTIO_PCI_QUEUE_SEL:
> +		hw->queue_sel = val;
> +		break;
> +	case VIRTIO_PCI_STATUS:
> +		if (val & VIRTIO_CONFIG_S_DRIVER_OK)
> +			kick_all_vq(hw);
> +		hw->status = val & 0xFF;
> +		break;
> +	case VIRTIO_PCI_QUEUE_NOTIFY:
> +		{
> +			int ret;
> +			uint64_t buf = 1;
> +			ret = write(hw->kickfds[val], &buf, sizeof(uint64_t));
> +			if (ret == -1)
> +				rte_panic("VIRTIO_PCI_QUEUE_NOTIFY failed: %s\n",
> +						strerror(errno));
> +			break;
> +		}
> +	default:
> +		PMD_DRV_LOG(ERR, "unexpected address %"PRIu64" value 0x%x\n",
> +				addr, val);
> +		break;
> +	}
> +}
> +
> +uint32_t
> +virtio_ioport_read(struct virtio_hw *hw, uint64_t addr)
> +{
> +	uint32_t ret = 0xFFFFFFFF;
> +	uint64_t host_features;
> +
> +	PMD_DRV_LOG(INFO, "addr: %"PRIu64"\n", addr);
> +
> +	switch (addr) {
> +	case VIRTIO_PCI_HOST_FEATURES:
> +		vhost_call(hw, VHOST_MSG_GET_FEATURES, &host_features);
> +		PMD_DRV_LOG(INFO, "get_features: %"PRIx64"\n", host_features);
> +		if (hw->mac_specified)
> +			host_features |= (1ull << VIRTIO_NET_F_MAC);
> +		/* disable it until we support CQ */
> +		host_features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
> +		ret = host_features;
> +		break;
> +	case VIRTIO_PCI_GUEST_FEATURES:
> +		ret = hw->guest_features;
> +		break;
> +	case VIRTIO_PCI_QUEUE_NUM:
> +		ret = hw->queue_num;
> +		break;
> +	case VIRTIO_PCI_QUEUE_SEL:
> +		ret = hw->queue_sel;
> +		break;
> +	case VIRTIO_PCI_STATUS:
> +		ret = hw->status;
> +		break;
> +	case 20: /* mac addr: 0~3 */
> +		if (hw->mac_specified) {
> +			uint32_t m0 = hw->mac_addr[0],
> +				 m1 = hw->mac_addr[1],
> +				 m2 = hw->mac_addr[2],
> +				 m3 = hw->mac_addr[3];
> +			ret = (m3 << 24) | (m2 << 16) | (m1 << 8) | m0;
> +		}
> +		break;
> +	case 24: /* mac addr: 4~5 */
> +		if (hw->mac_specified) {
> +			uint32_t m4 = hw->mac_addr[4],
> +				 m5 = hw->mac_addr[5];
> +			ret = (m5 << 8) | m4;
> +		}
> +		break;
> +	default:
> +		PMD_DRV_LOG(ERR, "%"PRIu64" (r) not supported\n", addr);
> +		break;
> +	}
> +
> +	return ret;
> +}
> +
> +#define TUN_DEF_SNDBUF	(1ull << 20)
> +
> +static void
> +vhost_kernel_backend_setup(struct virtio_hw *hw)
> +{
> +	int fd;
> +	int len = sizeof(struct virtio_net_hdr);
> +	int req_mq = 0;
> +	int sndbuf = TUN_DEF_SNDBUF;
> +	unsigned int features;
> +	struct ifreq ifr;
> +
> +	/* TODO:
> +	 * 1. get and set offload capability, tap_probe_has_ufo, tap_fd_set_offload
> +	 * 2. verify we can get and set vnet_hdr_len, tap_probe_vnet_hdr_len
> +
> +	 * 1. get number of memory regions from vhost module parameter
> +	 * max_mem_regions, supported in newer version linux kernel
> +	 */
> +
> +	fd = open(PATH_NET_TUN, O_RDWR);
> +	if (fd < 0)
> +		rte_panic("open %s error, %s\n", PATH_NET_TUN, strerror(errno));
> +
> +	memset(&ifr, 0, sizeof(ifr));
> +	ifr.ifr_flags = IFF_TAP | IFF_NO_PI;
> +
> +	if (ioctl(fd, TUNGETFEATURES, &features) == -1)
> +		rte_panic("TUNGETFEATURES failed: %s", strerror(errno));
> +
> +	if (features & IFF_ONE_QUEUE)
> +		ifr.ifr_flags |= IFF_ONE_QUEUE;
> +
> +	if (features & IFF_VNET_HDR)
> +		ifr.ifr_flags |= IFF_VNET_HDR;
> +	else
> +		rte_panic("vnet_hdr requested, but kernel does not support\n");
> +
> +	if (req_mq) {
> +		if (features & IFF_MULTI_QUEUE)
> +			ifr.ifr_flags |= IFF_MULTI_QUEUE;
> +		else
> +			rte_panic("multiqueue requested, but kernel does not support\n");
> +	}
> +
> +	strncpy(ifr.ifr_name, "tap%d", IFNAMSIZ);
> +	if (ioctl(fd, TUNSETIFF, (void *) &ifr) == -1)
> +		rte_panic("TUNSETIFF failed: %s", strerror(errno));
> +	fcntl(fd, F_SETFL, O_NONBLOCK);
> +
> +	if (ioctl(fd, TUNSETVNETHDRSZ, &len) == -1)
> +		rte_panic("TUNSETVNETHDRSZ failed: %s\n", strerror(errno));
> +
> +	if (ioctl(fd, TUNSETSNDBUF, &sndbuf) == -1)
> +		rte_panic("TUNSETSNDBUF failed: %s", strerror(errno));
> +
> +	hw->backfd = fd;
> +
> +	hw->vhostfd = open(hw->path, O_RDWR);
> +	if (hw->vhostfd == -1)
> +		rte_panic("open %s failed: %s\n", hw->path, strerror(errno));
> +}
> +
> +static void
> +vhost_user_backend_setup(struct virtio_hw *hw)
> +{
> +	int fd;
> +	int flag;
> +	struct sockaddr_un un;
> +
> +	fd = socket(AF_UNIX, SOCK_STREAM, 0);
> +	if (fd < 0)
> +		rte_panic("socket error, %s\n", strerror(errno));
> +
> +	flag = fcntl(fd, F_GETFD);
> +	fcntl(fd, F_SETFD, flag | FD_CLOEXEC);
> +
> +	memset(&un, 0, sizeof(un));
> +	un.sun_family = AF_UNIX;
> +	snprintf(un.sun_path, sizeof(un.sun_path), "%s", hw->path);
> +	if (connect(fd, (struct sockaddr *)&un, sizeof(un)) < 0) {
> +		PMD_DRV_LOG(ERR, "connect error, %s\n", strerror(errno));
> +		exit(-1);
> +	}
> +
> +	hw->vhostfd = fd;
> +}
> +
> +void
> +virtio_vdev_init(struct rte_eth_dev_data *data, const char *path,
> +		int nb_rx, int nb_tx, int nb_cq __attribute__ ((unused)),
> +		int queue_num, char *mac)
> +{
> +	int i;
> +	int ret;
> +	struct stat s;
> +	uint32_t tmp[ETHER_ADDR_LEN];
> +	struct virtio_hw *hw = data->dev_private;
> +
> +	hw->data = data;
> +	hw->path = strdup(path);
> +	hw->max_rx_queues = nb_rx;
> +	hw->max_tx_queues = nb_tx;
> +	hw->queue_num = queue_num;
> +	hw->mac_specified = 0;
> +	if (mac) {
> +		ret = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0], &tmp[1],
> +				&tmp[2], &tmp[3], &tmp[4], &tmp[5]);
> +		if (ret == ETHER_ADDR_LEN) {
> +			for (i = 0; i < ETHER_ADDR_LEN; ++i)
> +				hw->mac_addr[i] = (uint8_t)tmp[i];
> +			hw->mac_specified = 1;
> +		}
> +	}
> +
> +	/* TODO: cq */
> +
> +	ret = stat(hw->path, &s);
> +	if (ret < 0)
> +		rte_panic("stat: %s failed, %s\n", hw->path, strerror(errno));
> +
> +	switch (s.st_mode & S_IFMT) {
> +		case S_IFCHR:
> +			hw->type = VHOST_KERNEL;
> +			vhost_kernel_backend_setup(hw);
> +			break;
> +		case S_IFSOCK:
> +			hw->type = VHOST_USER;
> +			vhost_user_backend_setup(hw);
> +			break;
> +		default:
> +			rte_panic("unknown file type of %s\n", hw->path);
> +	}
> +	if (vhost_call(hw, VHOST_MSG_SET_OWNER, NULL) == -1)
> +		rte_panic("vhost set_owner failed: %s\n", strerror(errno));
> +}
> diff --git a/drivers/net/virtio/vhost.h b/drivers/net/virtio/vhost.h
> new file mode 100644
> index 0000000..c7517f6
> --- /dev/null
> +++ b/drivers/net/virtio/vhost.h
> @@ -0,0 +1,192 @@
> +/*-
> + *   BSD LICENSE
> + *
> + *   Copyright(c) 2010-2014 Intel Corporation. All rights reserved.
> + *   All rights reserved.
> + *
> + *   Redistribution and use in source and binary forms, with or without
> + *   modification, are permitted provided that the following conditions
> + *   are met:
> + *
> + *     * Redistributions of source code must retain the above copyright
> + *       notice, this list of conditions and the following disclaimer.
> + *     * Redistributions in binary form must reproduce the above copyright
> + *       notice, this list of conditions and the following disclaimer in
> + *       the documentation and/or other materials provided with the
> + *       distribution.
> + *     * Neither the name of Intel Corporation nor the names of its
> + *       contributors may be used to endorse or promote products derived
> + *       from this software without specific prior written permission.
> + *
> + *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
> + *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
> + *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
> + *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
> + *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
> + *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
> + *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
> + *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
> + *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
> + *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
> + *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
> + */
> +
> +#ifndef _VHOST_NET_USER_H
> +#define _VHOST_NET_USER_H
> +
> +#include <stdint.h>
> +#include <linux/types.h>
> +#include <linux/ioctl.h>
> +
> +#define VHOST_MEMORY_MAX_NREGIONS 8
> +
> +struct vhost_vring_state {
> +	unsigned int index;
> +	unsigned int num;
> +};
> +
> +struct vhost_vring_file {
> +	unsigned int index;
> +	int fd;
> +};
> +
> +struct vhost_vring_addr {
> +	unsigned int index;
> +	/* Option flags. */
> +	unsigned int flags;
> +	/* Flag values: */
> +	/* Whether log address is valid. If set enables logging. */
> +#define VHOST_VRING_F_LOG 0
> +
> +	/* Start of array of descriptors (virtually contiguous) */
> +	uint64_t desc_user_addr;
> +	/* Used structure address. Must be 32 bit aligned */
> +	uint64_t used_user_addr;
> +	/* Available structure address. Must be 16 bit aligned */
> +	uint64_t avail_user_addr;
> +	/* Logging support. */
> +	/* Log writes to used structure, at offset calculated from specified
> +	 * address. Address must be 32 bit aligned. */
> +	uint64_t log_guest_addr;
> +};
> +
> +#define VIRTIO_CONFIG_S_DRIVER_OK   4
> +
> +typedef enum VhostUserRequest {
> +	VHOST_USER_NONE = 0,
> +	VHOST_USER_GET_FEATURES = 1,
> +	VHOST_USER_SET_FEATURES = 2,
> +	VHOST_USER_SET_OWNER = 3,
> +	VHOST_USER_RESET_OWNER = 4,
> +	VHOST_USER_SET_MEM_TABLE = 5,
> +	VHOST_USER_SET_LOG_BASE = 6,
> +	VHOST_USER_SET_LOG_FD = 7,
> +	VHOST_USER_SET_VRING_NUM = 8,
> +	VHOST_USER_SET_VRING_ADDR = 9,
> +	VHOST_USER_SET_VRING_BASE = 10,
> +	VHOST_USER_GET_VRING_BASE = 11,
> +	VHOST_USER_SET_VRING_KICK = 12,
> +	VHOST_USER_SET_VRING_CALL = 13,
> +	VHOST_USER_SET_VRING_ERR = 14,
> +	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
> +	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
> +	VHOST_USER_GET_QUEUE_NUM = 17,
> +	VHOST_USER_SET_VRING_ENABLE = 18,
> +	VHOST_USER_MAX
> +} VhostUserRequest;
> +
> +struct vhost_memory_region {
> +	uint64_t guest_phys_addr;
> +	uint64_t memory_size; /* bytes */
> +	uint64_t userspace_addr;
> +	uint64_t mmap_offset;
> +};
> +struct vhost_memory_kernel {
> +	uint32_t nregions;
> +	uint32_t padding;
> +	struct vhost_memory_region regions[0];
> +};
> +
> +struct vhost_memory {
> +	uint32_t nregions;
> +	uint32_t padding;
> +	struct vhost_memory_region regions[VHOST_MEMORY_MAX_NREGIONS];
> +};
> +
> +typedef struct VhostUserMsg {
> +	VhostUserRequest request;
> +
> +#define VHOST_USER_VERSION_MASK     0x3
> +#define VHOST_USER_REPLY_MASK       (0x1 << 2)
> +	uint32_t flags;
> +	uint32_t size; /* the following payload size */
> +	union {
> +#define VHOST_USER_VRING_IDX_MASK   0xff
> +#define VHOST_USER_VRING_NOFD_MASK  (0x1 << 8)
> +		uint64_t u64;
> +		struct vhost_vring_state state;
> +		struct vhost_vring_addr addr;
> +		struct vhost_memory memory;
> +	} payload;
> +	int fds[VHOST_MEMORY_MAX_NREGIONS];
> +} __attribute((packed)) VhostUserMsg;
> +
> +#define VHOST_USER_HDR_SIZE offsetof(VhostUserMsg, payload.u64)
> +#define VHOST_USER_PAYLOAD_SIZE (sizeof(VhostUserMsg) - VHOST_USER_HDR_SIZE)
> +
> +/* The version of the protocol we support */
> +#define VHOST_USER_VERSION    0x1
> +
> +/* ioctls */
> +
> +#define VHOST_VIRTIO 0xAF
> +
> +#define VHOST_GET_FEATURES	_IOR(VHOST_VIRTIO, 0x00, __u64)
> +#define VHOST_SET_FEATURES	_IOW(VHOST_VIRTIO, 0x00, __u64)
> +#define VHOST_SET_OWNER _IO(VHOST_VIRTIO, 0x01)
> +#define VHOST_RESET_OWNER _IO(VHOST_VIRTIO, 0x02)
> +#define VHOST_SET_MEM_TABLE	_IOW(VHOST_VIRTIO, 0x03, struct vhost_memory_kernel)
> +#define VHOST_SET_LOG_BASE _IOW(VHOST_VIRTIO, 0x04, __u64)
> +#define VHOST_SET_LOG_FD _IOW(VHOST_VIRTIO, 0x07, int)
> +#define VHOST_SET_VRING_NUM _IOW(VHOST_VIRTIO, 0x10, struct vhost_vring_state)
> +#define VHOST_SET_VRING_ADDR _IOW(VHOST_VIRTIO, 0x11, struct vhost_vring_addr)
> +#define VHOST_SET_VRING_BASE _IOW(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
> +#define VHOST_GET_VRING_BASE _IOWR(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
> +#define VHOST_SET_VRING_KICK _IOW(VHOST_VIRTIO, 0x20, struct vhost_vring_file)
> +#define VHOST_SET_VRING_CALL _IOW(VHOST_VIRTIO, 0x21, struct vhost_vring_file)
> +#define VHOST_SET_VRING_ERR _IOW(VHOST_VIRTIO, 0x22, struct vhost_vring_file)
> +#define VHOST_NET_SET_BACKEND _IOW(VHOST_VIRTIO, 0x30, struct vhost_vring_file)
> +
> +/*****************************************************************************/
> +
> +/* Ioctl defines */
> +#define TUNSETIFF     _IOW('T', 202, int)
> +#define TUNGETFEATURES _IOR('T', 207, unsigned int)
> +#define TUNSETOFFLOAD  _IOW('T', 208, unsigned int)
> +#define TUNGETIFF      _IOR('T', 210, unsigned int)
> +#define TUNSETSNDBUF   _IOW('T', 212, int)
> +#define TUNGETVNETHDRSZ _IOR('T', 215, int)
> +#define TUNSETVNETHDRSZ _IOW('T', 216, int)
> +#define TUNSETQUEUE  _IOW('T', 217, int)
> +#define TUNSETVNETLE _IOW('T', 220, int)
> +#define TUNSETVNETBE _IOW('T', 222, int)
> +
> +/* TUNSETIFF ifr flags */
> +#define IFF_TAP          0x0002
> +#define IFF_NO_PI        0x1000
> +#define IFF_ONE_QUEUE    0x2000
> +#define IFF_VNET_HDR     0x4000
> +#define IFF_MULTI_QUEUE  0x0100
> +#define IFF_ATTACH_QUEUE 0x0200
> +#define IFF_DETACH_QUEUE 0x0400
> +
> +/* Features for GSO (TUNSETOFFLOAD). */
> +#define TUN_F_CSUM	0x01	/* You can hand me unchecksummed packets. */
> +#define TUN_F_TSO4	0x02	/* I can handle TSO for IPv4 packets */
> +#define TUN_F_TSO6	0x04	/* I can handle TSO for IPv6 packets */
> +#define TUN_F_TSO_ECN	0x08	/* I can handle TSO with ECN bits. */
> +#define TUN_F_UFO	0x10	/* I can handle UFO packets */
> +
> +#define PATH_NET_TUN "/dev/net/tun"
> +
> +#endif
> diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
> index ae2d47d..9e1ecb3 100644
> --- a/drivers/net/virtio/virtio_ethdev.h
> +++ b/drivers/net/virtio/virtio_ethdev.h
> @@ -122,5 +122,8 @@ uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf
> **tx_pkts,
>  #define VTNET_LRO_FEATURES (VIRTIO_NET_F_GUEST_TSO4 | \
>  			    VIRTIO_NET_F_GUEST_TSO6 | VIRTIO_NET_F_GUEST_ECN)
> 
> -
> +#ifdef RTE_VIRTIO_VDEV
> +void virtio_vdev_init(struct rte_eth_dev_data *data, const char *path,
> +		int nb_rx, int nb_tx, int nb_cq, int queue_num, char *mac);
> +#endif
>  #endif /* _VIRTIO_ETHDEV_H_ */
> diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
> index 47f722a..af05ae2 100644
> --- a/drivers/net/virtio/virtio_pci.h
> +++ b/drivers/net/virtio/virtio_pci.h
> @@ -147,7 +147,6 @@ struct virtqueue;
>   * rest are per-device feature bits.
>   */
>  #define VIRTIO_TRANSPORT_F_START 28
> -#define VIRTIO_TRANSPORT_F_END   32

 I understand that this #define is not used, but... May be we should do this cleanup as a separate patch? Otherwise it's hard to
track this change (i believe this definition had some use in the past).

> 
>  /* The Guest publishes the used index for which it expects an interrupt
>   * at the end of the avail ring. Host should ignore the avail->flags field. */
> @@ -165,6 +164,7 @@ struct virtqueue;
> 
>  struct virtio_hw {
>  	struct virtqueue *cvq;
> +#define VIRTIO_VDEV_IO_BASE	0xffffffff
>  	uint32_t    io_base;
>  	uint32_t    guest_features;
>  	uint32_t    max_tx_queues;
> @@ -174,6 +174,21 @@ struct virtio_hw {
>  	uint8_t	    use_msix;
>  	uint8_t     started;
>  	uint8_t     mac_addr[ETHER_ADDR_LEN];
> +#ifdef RTE_VIRTIO_VDEV
> +#define VHOST_KERNEL	0
> +#define VHOST_USER		1
> +	int			type; /* type of backend */
> +	uint32_t	queue_num;
> +	char		*path;
> +	int			mac_specified;
> +	int			vhostfd;
> +	int			backfd; /* tap device used in vhost-net */
> +	int			callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
> +	int			kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
> +	uint32_t	queue_sel;
> +	uint8_t		status;
> +	struct rte_eth_dev_data *data;
> +#endif

 Actually i am currently working on this too, and i decided to use different approach. I made these extra fields into a separate
structure, changed 'io_base' to a pointer, and now i can store there a pointer to this extra structure. Device type can easily be
determined by (dev->dev_type == RTE_ETH_DEV_PCI) check, so you don't need VIRTIO_VDEV_IO_BASE magic value.

>  };
> 
>  /*
> @@ -229,6 +244,39 @@ outl_p(unsigned int data, unsigned int port)
>  #define VIRTIO_PCI_REG_ADDR(hw, reg) \
>  	(unsigned short)((hw)->io_base + (reg))
> 
> +#ifdef RTE_VIRTIO_VDEV
> +uint32_t virtio_ioport_read(struct virtio_hw *, uint64_t);
> +void virtio_ioport_write(struct virtio_hw *, uint64_t, uint32_t);
> +
> +#define VIRTIO_READ_REG_1(hw, reg) \
> +	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	inb((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> +	:virtio_ioport_read(hw, reg)
> +#define VIRTIO_WRITE_REG_1(hw, reg, value) \
> +	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	outb_p((unsigned char)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> +	:virtio_ioport_write(hw, reg, value)
> +
> +#define VIRTIO_READ_REG_2(hw, reg) \
> +	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	inw((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> +	:virtio_ioport_read(hw, reg)
> +#define VIRTIO_WRITE_REG_2(hw, reg, value) \
> +	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	outw_p((unsigned short)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> +	:virtio_ioport_write(hw, reg, value)
> +
> +#define VIRTIO_READ_REG_4(hw, reg) \
> +	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	inl((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> +	:virtio_ioport_read(hw, reg)
> +#define VIRTIO_WRITE_REG_4(hw, reg, value) \
> +	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	outl_p((unsigned int)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> +	:virtio_ioport_write(hw, reg, value)

 I also decided to add two fields to 'hw', where pointers to these accessors are stored. I think this should be faster, however,
yes, this is not performance-critical code because it's executed only during initialization.

> +
> +#else /* RTE_VIRTIO_VDEV */
> +
>  #define VIRTIO_READ_REG_1(hw, reg) \
>  	inb((VIRTIO_PCI_REG_ADDR((hw), (reg))))
>  #define VIRTIO_WRITE_REG_1(hw, reg, value) \
> @@ -244,6 +292,8 @@ outl_p(unsigned int data, unsigned int port)
>  #define VIRTIO_WRITE_REG_4(hw, reg, value) \
>  	outl_p((unsigned int)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg))))
> 
> +#endif /* RTE_VIRTIO_VDEV */
> +
>  static inline int
>  vtpci_with_feature(struct virtio_hw *hw, uint32_t bit)
>  {
> --
> 2.1.4

Kind regards,
Pavel Fedin
Expert Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-10 11:43   ` [PATCH 2/4] mem: add API to obstain memory-backed file info Jianfeng Tan
@ 2016-01-11 11:43     ` Pavel Fedin
  2016-01-11 20:26     ` Rich Lane
  1 sibling, 0 replies; 196+ messages in thread
From: Pavel Fedin @ 2016-01-11 11:43 UTC (permalink / raw)
  To: 'Jianfeng Tan', dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

 Hello!

> -----Original Message-----
> From: Jianfeng Tan [mailto:jianfeng.tan@intel.com]
> Sent: Sunday, January 10, 2016 2:43 PM
> To: dev@dpdk.org
> Cc: rich.lane@bigswitch.com; yuanhan.liu@linux.intel.com; mst@redhat.com;
> nakajima.yoshihiro@lab.ntt.co.jp; huawei.xie@intel.com; mukawa@igel.co.jp;
> p.fedin@samsung.com; michael.qiu@intel.com; ann.zhuangyanying@huawei.com; Jianfeng Tan
> Subject: [PATCH 2/4] mem: add API to obstain memory-backed file info

 "obtain" - typo in subject

> 
> A new API named rte_eal_get_backfile_info() and a new data
> struct back_file is added to obstain information of memory-
> backed file info.
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ---
>  lib/librte_eal/common/include/rte_memory.h | 16 +++++++++++++
>  lib/librte_eal/linuxapp/eal/eal_memory.c   | 37 ++++++++++++++++++++++++++++++
>  2 files changed, 53 insertions(+)
> 
> diff --git a/lib/librte_eal/common/include/rte_memory.h
> b/lib/librte_eal/common/include/rte_memory.h
> index 9c9e40f..75ef8db 100644
> --- a/lib/librte_eal/common/include/rte_memory.h
> +++ b/lib/librte_eal/common/include/rte_memory.h
> @@ -109,6 +109,22 @@ struct rte_memseg {
>  } __rte_packed;
> 
>  /**
> + * This struct is used to store information about memory-backed file that
> + * we mapped in memory initialization.
> + */
> +struct back_file {
> +	void *addr;         /**< virtual addr */
> +	size_t size;        /**< the page size */
> +	char filepath[PATH_MAX]; /**< path to backing file on filesystem */
> +};
> +
> +/**
> +  * Get the hugepage file information. Caller to free.
> +  * Return number of hugepage files used.
> +  */
> +int rte_eal_get_backfile_info(struct back_file **);
> +
> +/**
>   * Lock page in physical memory and prevent from swapping.
>   *
>   * @param virt
> diff --git a/lib/librte_eal/linuxapp/eal/eal_memory.c
> b/lib/librte_eal/linuxapp/eal/eal_memory.c
> index 2bb1163..6ca1404 100644
> --- a/lib/librte_eal/linuxapp/eal/eal_memory.c
> +++ b/lib/librte_eal/linuxapp/eal/eal_memory.c
> @@ -758,6 +758,9 @@ sort_by_physaddr(struct hugepage_file *hugepg_tbl, struct hugepage_info
> *hpi)
>  	return 0;
>  }
> 
> +static struct hugepage_file *hugepage_files;
> +static int num_hugepage_files;
> +
>  /*
>   * Uses mmap to create a shared memory area for storage of data
>   * Used in this file to store the hugepage file map on disk
> @@ -776,9 +779,29 @@ create_shared_memory(const char *filename, const size_t mem_size)
>  	retval = mmap(NULL, mem_size, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0);
>  	close(fd);
> 
> +	hugepage_files = retval;
> +	num_hugepage_files = mem_size / (sizeof(struct hugepage_file));
> +
>  	return retval;
>  }
> 
> +int
> +rte_eal_get_backfile_info(struct back_file **p)
> +{
> +	struct back_file *backfiles;
> +	int i, num_backfiles = num_hugepage_files;
> +
> +	backfiles = malloc(sizeof(struct back_file) * num_backfiles);
> +	for (i = 0; i < num_backfiles; ++i) {
> +		backfiles[i].addr = hugepage_files[i].final_va;
> +		backfiles[i].size = hugepage_files[i].size;
> +		strcpy(backfiles[i].filepath, hugepage_files[i].filepath);
> +	}
> +
> +	*p = backfiles;
> +	return num_backfiles;
> +}
> +
>  /*
>   * this copies *active* hugepages from one hugepage table to another.
>   * destination is typically the shared memory.
> @@ -1157,6 +1180,20 @@ rte_eal_hugepage_init(void)
>  		mcfg->memseg[0].len = internal_config.memory;
>  		mcfg->memseg[0].socket_id = socket_id;
> 
> +		hugepage = create_shared_memory(eal_hugepage_info_path(),
> +				sizeof(struct hugepage_file));
> +		hugepage->orig_va = addr;
> +		hugepage->final_va = addr;
> +		hugepage->physaddr = rte_mem_virt2phy(addr);
> +		hugepage->size = pagesize;
> +		hugepage->socket_id = socket_id;
> +		hugepage->file_id = 0;
> +		hugepage->memseg_id = 0;
> +#ifdef RTE_EAL_SINGLE_FILE_SEGMENTS
> +		hugepage->repeated = internal_config.memory / pagesize;
> +#endif
> +		strncpy(hugepage->filepath, filepath, MAX_HUGEPAGE_PATH);
> +
>  		close(fd);
> 
>  		return 0;
> --
> 2.1.4

Kind regards,
Pavel Fedin
Expert Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 3/4] virtio/vdev: add ways to interact with vhost
  2016-01-10 11:43   ` [PATCH 3/4] virtio/vdev: add ways to interact with vhost Jianfeng Tan
  2016-01-11 10:42     ` Pavel Fedin
@ 2016-01-11 14:02     ` Pavel Fedin
  2016-01-21  2:18     ` Xie, Huawei
  2 siblings, 0 replies; 196+ messages in thread
From: Pavel Fedin @ 2016-01-11 14:02 UTC (permalink / raw)
  To: 'Jianfeng Tan', dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

 Hello! There's one more problem (see inline).

> -----Original Message-----
> From: Jianfeng Tan [mailto:jianfeng.tan@intel.com]
> Sent: Sunday, January 10, 2016 2:43 PM
> To: dev@dpdk.org
> Cc: rich.lane@bigswitch.com; yuanhan.liu@linux.intel.com; mst@redhat.com;
> nakajima.yoshihiro@lab.ntt.co.jp; huawei.xie@intel.com; mukawa@igel.co.jp;
> p.fedin@samsung.com; michael.qiu@intel.com; ann.zhuangyanying@huawei.com; Jianfeng Tan
> Subject: [PATCH 3/4] virtio/vdev: add ways to interact with vhost
> 
> Depends on the type of vhost file: vhost-user is used if the given
> path points to a unix socket; vhost-net is used if the given path
> points to a char device.
> 
> NOTE: we now keep CONFIG_RTE_VIRTIO_VDEV undefined by default, need
> to be uncommented when in use.
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ---
>  config/common_linuxapp             |   5 +
>  drivers/net/virtio/Makefile        |   4 +
>  drivers/net/virtio/vhost.c         | 734 +++++++++++++++++++++++++++++++++++++
>  drivers/net/virtio/vhost.h         | 192 ++++++++++
>  drivers/net/virtio/virtio_ethdev.h |   5 +-
>  drivers/net/virtio/virtio_pci.h    |  52 ++-
>  6 files changed, 990 insertions(+), 2 deletions(-)
>  create mode 100644 drivers/net/virtio/vhost.c
>  create mode 100644 drivers/net/virtio/vhost.h
> 
> diff --git a/config/common_linuxapp b/config/common_linuxapp
> index 74bc515..f76e162 100644
> --- a/config/common_linuxapp
> +++ b/config/common_linuxapp
> @@ -534,3 +534,8 @@ CONFIG_RTE_APP_TEST=y
>  CONFIG_RTE_TEST_PMD=y
>  CONFIG_RTE_TEST_PMD_RECORD_CORE_CYCLES=n
>  CONFIG_RTE_TEST_PMD_RECORD_BURST_STATS=n
> +
> +#
> +# Enable virtio support for container
> +#
> +CONFIG_RTE_VIRTIO_VDEV=y
> diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
> index 43835ba..0877023 100644
> --- a/drivers/net/virtio/Makefile
> +++ b/drivers/net/virtio/Makefile
> @@ -52,6 +52,10 @@ SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx.c
>  SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_ethdev.c
>  SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx_simple.c
> 
> +ifeq ($(CONFIG_RTE_VIRTIO_VDEV),y)
> +	SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += vhost.c
> +endif
> +
>  # this lib depends upon:
>  DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_eal lib/librte_ether
>  DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_mempool lib/librte_mbuf
> diff --git a/drivers/net/virtio/vhost.c b/drivers/net/virtio/vhost.c
> new file mode 100644
> index 0000000..e423e02
> --- /dev/null
> +++ b/drivers/net/virtio/vhost.c
> @@ -0,0 +1,734 @@
> +/*-
> + *   BSD LICENSE
> + *
> + *   Copyright(c) 2010-2015 Intel Corporation. All rights reserved.
> + *   All rights reserved.
> + *
> + *   Redistribution and use in source and binary forms, with or without
> + *   modification, are permitted provided that the following conditions
> + *   are met:
> + *
> + *     * Redistributions of source code must retain the above copyright
> + *       notice, this list of conditions and the following disclaimer.
> + *     * Redistributions in binary form must reproduce the above copyright
> + *       notice, this list of conditions and the following disclaimer in
> + *       the documentation and/or other materials provided with the
> + *       distribution.
> + *     * Neither the name of Intel Corporation nor the names of its
> + *       contributors may be used to endorse or promote products derived
> + *       from this software without specific prior written permission.
> + *
> + *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
> + *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
> + *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
> + *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
> + *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
> + *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
> + *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
> + *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
> + *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
> + *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
> + *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
> + */
> +
> +#include <stdint.h>
> +#include <sys/types.h>
> +#include <sys/socket.h>
> +#include <sys/un.h>
> +#include <stdio.h>
> +#include <sys/stat.h>
> +#include <fcntl.h>
> +#include <string.h>
> +#include <errno.h>
> +#include <assert.h>
> +#include <sys/mman.h>
> +#include <unistd.h>
> +#include <sys/eventfd.h>
> +#include <sys/ioctl.h>
> +#include <net/if.h>
> +
> +#include <rte_mbuf.h>
> +#include <rte_memory.h>
> +#include <rte_eal_memconfig.h>
> +
> +#include "virtio_pci.h"
> +#include "virtio_logs.h"
> +#include "virtio_ethdev.h"
> +#include "virtqueue.h"
> +#include "vhost.h"
> +
> +static int
> +vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
> +{
> +	struct msghdr msgh;
> +	struct iovec iov;
> +	int r;
> +
> +	size_t fd_size = fd_num * sizeof(int);
> +	char control[CMSG_SPACE(fd_size)];
> +	struct cmsghdr *cmsg;
> +
> +	memset(&msgh, 0, sizeof(msgh));
> +	memset(control, 0, sizeof(control));
> +
> +	iov.iov_base = (uint8_t *)buf;
> +	iov.iov_len = len;
> +
> +	msgh.msg_iov = &iov;
> +	msgh.msg_iovlen = 1;
> +
> +	msgh.msg_control = control;
> +	msgh.msg_controllen = sizeof(control);
> +
> +	cmsg = CMSG_FIRSTHDR(&msgh);
> +
> +	cmsg->cmsg_len = CMSG_LEN(fd_size);
> +	cmsg->cmsg_level = SOL_SOCKET;
> +	cmsg->cmsg_type = SCM_RIGHTS;
> +	memcpy(CMSG_DATA(cmsg), fds, fd_size);
> +
> +	do {
> +		r = sendmsg(fd, &msgh, 0);
> +	} while (r < 0 && errno == EINTR);
> +
> +	return r;
> +}
> +
> +static int
> +vhost_user_read(int fd, VhostUserMsg *msg)
> +{
> +	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
> +	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
> +
> +	ret = recv(fd, (void *)msg, sz_hdr, 0);
> +	if (ret < sz_hdr) {
> +		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
> +				ret, sz_hdr);
> +		goto fail;
> +	}
> +
> +	/* validate msg flags */
> +	if (msg->flags != (valid_flags)) {
> +		PMD_DRV_LOG(ERR, "Failed to recv msg: flags 0x%x instead of 0x%x.",
> +				msg->flags, valid_flags);
> +		goto fail;
> +	}
> +
> +	sz_payload = msg->size;
> +	if (sz_payload) {
> +		ret = recv(fd, (void *)((uint8_t *)msg + sz_hdr), sz_payload, 0);
> +		if (ret < sz_payload) {
> +			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
> +					ret, msg->size);
> +			goto fail;
> +		}
> +	}
> +
> +	return 0;
> +
> +fail:
> +	return -1;
> +}
> +
> +static VhostUserMsg m __attribute__ ((unused));
> +
> +static void
> +prepare_vhost_memory_user(VhostUserMsg *msg, int fds[])
> +{
> +	int i, num;
> +	struct back_file *huges;
> +	struct vhost_memory_region *mr;
> +
> +	num = rte_eal_get_backfile_info(&huges);
> +
> +	if (num > VHOST_MEMORY_MAX_NREGIONS)
> +		rte_panic("%d hugepage files exceed the maximum of %d for "
> +				"vhost-user\n", num, VHOST_MEMORY_MAX_NREGIONS);
> +
> +	for (i = 0; i < num; ++i) {
> +		mr = &msg->payload.memory.regions[i];
> +		mr->guest_phys_addr = (uint64_t)huges[i].addr; /* use vaddr! */
> +		mr->userspace_addr = (uint64_t)huges[i].addr;
> +		mr->memory_size = huges[i].size;
> +		mr->mmap_offset = 0;
> +		fds[i] = open(huges[i].filepath, O_RDWR);
> +	}
> +
> +	msg->payload.memory.nregions = num;
> +	msg->payload.memory.padding = 0;
> +	free(huges);
> +}
> +
> +static int
> +vhost_user_sock(struct virtio_hw *hw, unsigned long int req, void *arg)
> +{
> +	VhostUserMsg msg;
> +	struct vhost_vring_file *file = 0;
> +	int need_reply = 0;
> +	int fds[VHOST_MEMORY_MAX_NREGIONS];
> +	int fd_num = 0;
> +	int i, len;
> +
> +	msg.request = req;
> +	msg.flags = VHOST_USER_VERSION;
> +	msg.size = 0;
> +
> +	switch (req) {
> +	case VHOST_USER_GET_FEATURES:
> +		need_reply = 1;
> +		break;
> +
> +	case VHOST_USER_SET_FEATURES:
> +	case VHOST_USER_SET_LOG_BASE:
> +		msg.payload.u64 = *((__u64 *)arg);
> +		msg.size = sizeof(m.payload.u64);
> +		break;
> +
> +	case VHOST_USER_SET_OWNER:
> +	case VHOST_USER_RESET_OWNER:
> +		break;
> +
> +	case VHOST_USER_SET_MEM_TABLE:
> +		prepare_vhost_memory_user(&msg, fds);
> +		fd_num = msg.payload.memory.nregions;
> +		msg.size = sizeof(m.payload.memory.nregions);
> +		msg.size += sizeof(m.payload.memory.padding);
> +		msg.size += fd_num * sizeof(struct vhost_memory_region);
> +		break;
> +
> +	case VHOST_USER_SET_LOG_FD:
> +		fds[fd_num++] = *((int *)arg);
> +		break;
> +
> +	case VHOST_USER_SET_VRING_NUM:
> +	case VHOST_USER_SET_VRING_BASE:
> +		memcpy(&msg.payload.state, arg, sizeof(struct vhost_vring_state));
> +		msg.size = sizeof(m.payload.state);
> +		break;
> +
> +	case VHOST_USER_GET_VRING_BASE:
> +		memcpy(&msg.payload.state, arg, sizeof(struct vhost_vring_state));
> +		msg.size = sizeof(m.payload.state);
> +		need_reply = 1;
> +		break;
> +
> +	case VHOST_USER_SET_VRING_ADDR:
> +		memcpy(&msg.payload.addr, arg, sizeof(struct vhost_vring_addr));
> +		msg.size = sizeof(m.payload.addr);
> +		break;
> +
> +	case VHOST_USER_SET_VRING_KICK:
> +	case VHOST_USER_SET_VRING_CALL:
> +	case VHOST_USER_SET_VRING_ERR:
> +		file = arg;
> +		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
> +		msg.size = sizeof(m.payload.u64);
> +		if (file->fd > 0)
> +			fds[fd_num++] = file->fd;
> +		else
> +			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
> +		break;
> +
> +	default:
> +		PMD_DRV_LOG(ERR, "vhost-user trying to send unhandled msg type");
> +		return -1;
> +	}
> +
> +	len = VHOST_USER_HDR_SIZE + msg.size;
> +	if (vhost_user_write(hw->vhostfd, &msg, len, fds, fd_num) < 0)
> +		return 0;
> +
> +	if (req == VHOST_USER_SET_MEM_TABLE)
> +		for (i = 0; i < fd_num; ++i)
> +			close(fds[i]);
> +
> +	if (need_reply) {
> +		if (vhost_user_read(hw->vhostfd, &msg) < 0)
> +			return -1;
> +
> +		if (req != msg.request) {
> +			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
> +			return -1;
> +		}
> +
> +		switch (req) {
> +		case VHOST_USER_GET_FEATURES:
> +			if (msg.size != sizeof(m.payload.u64)) {
> +				PMD_DRV_LOG(ERR, "Received bad msg size.");
> +				return -1;
> +			}
> +			*((__u64 *)arg) = msg.payload.u64;
> +			break;
> +		case VHOST_USER_GET_VRING_BASE:
> +			if (msg.size != sizeof(m.payload.state)) {
> +				PMD_DRV_LOG(ERR, "Received bad msg size.");
> +				return -1;
> +			}
> +			memcpy(arg, &msg.payload.state, sizeof(struct vhost_vring_state));
> +			break;
> +		default:
> +			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
> +			return -1;
> +		}
> +	}
> +
> +	return 0;
> +}
> +
> +static int
> +vhost_kernel_ioctl(struct virtio_hw *hw, unsigned long int req, void *arg)
> +{
> +	return ioctl(hw->vhostfd, req, arg);
> +}
> +
> +enum {
> +	VHOST_MSG_SET_OWNER,
> +	VHOST_MSG_SET_FEATURES,
> +	VHOST_MSG_GET_FEATURES,
> +	VHOST_MSG_SET_VRING_CALL,
> +	VHOST_MSG_SET_VRING_NUM,
> +	VHOST_MSG_SET_VRING_BASE,
> +	VHOST_MSG_SET_VRING_ADDR,
> +	VHOST_MSG_SET_VRING_KICK,
> +	VHOST_MSG_SET_MEM_TABLE,
> +	VHOST_MSG_MAX,
> +};
> +
> +static const char *vhost_msg_strings[] = {
> +	"VHOST_MSG_SET_OWNER",
> +	"VHOST_MSG_SET_FEATURES",
> +	"VHOST_MSG_GET_FEATURES",
> +	"VHOST_MSG_SET_VRING_CALL",
> +	"VHOST_MSG_SET_VRING_NUM",
> +	"VHOST_MSG_SET_VRING_BASE",
> +	"VHOST_MSG_SET_VRING_ADDR",
> +	"VHOST_MSG_SET_VRING_KICK",
> +	"VHOST_MSG_SET_MEM_TABLE",
> +	NULL,
> +};
> +
> +static unsigned long int vhost_req_map[][2] = {
> +	{VHOST_SET_OWNER,		VHOST_USER_SET_OWNER},
> +	{VHOST_SET_FEATURES,	VHOST_USER_SET_FEATURES},
> +	{VHOST_GET_FEATURES,	VHOST_USER_GET_FEATURES},
> +	{VHOST_SET_VRING_CALL,	VHOST_USER_SET_VRING_CALL},
> +	{VHOST_SET_VRING_NUM,	VHOST_USER_SET_VRING_NUM},
> +	{VHOST_SET_VRING_BASE,	VHOST_USER_SET_VRING_BASE},
> +	{VHOST_SET_VRING_ADDR,	VHOST_USER_SET_VRING_ADDR},
> +	{VHOST_SET_VRING_KICK,	VHOST_USER_SET_VRING_KICK},
> +	{VHOST_SET_MEM_TABLE,	VHOST_USER_SET_MEM_TABLE},
> +};
> +
> +static int
> +vhost_call(struct virtio_hw *hw, unsigned long int req_orig, void *arg)
> +{
> +	int req_new;
> +	int ret = 0;
> +
> +	if (req_orig >= VHOST_MSG_MAX)
> +		rte_panic("invalid req: %lu\n", req_orig);
> +
> +	PMD_DRV_LOG(INFO, "%s\n", vhost_msg_strings[req_orig]);
> +	req_new = vhost_req_map[req_orig][hw->type];
> +	if (hw->type == VHOST_USER)
> +		ret = vhost_user_sock(hw, req_new, arg);
> +	else
> +		ret = vhost_kernel_ioctl(hw, req_new, arg);
> +
> +	if (ret < 0)
> +		rte_panic("vhost_call %s failed: %s\n",
> +				vhost_msg_strings[req_orig], strerror(errno));
> +
> +	return ret;
> +}
> +
> +static void
> +kick_one_vq(struct virtio_hw *hw, struct virtqueue *vq, unsigned queue_sel)
> +{
> +	int callfd, kickfd;
> +	struct vhost_vring_file file;
> +
> +	/* or use invalid flag to disable it, but vhost-dpdk uses this to judge
> +	 * if dev is alive. so finally we need two real event_fds.
> +	 */
> +	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL comes
> +	 * firstly because vhost depends on this msg to allocate virtqueue
> +	 * pair.
> +	 */
> +	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
> +	if (callfd < 0)
> +		rte_panic("callfd error, %s\n", strerror(errno));
> +
> +	file.index = queue_sel;
> +	file.fd = callfd;
> +	vhost_call(hw, VHOST_MSG_SET_VRING_CALL, &file);
> +	hw->callfds[queue_sel] = callfd;
> +
> +	struct vhost_vring_state state;
> +	state.index = queue_sel;
> +	state.num = vq->vq_ring.num;
> +	vhost_call(hw, VHOST_MSG_SET_VRING_NUM, &state);
> +
> +	state.num = 0; /* no reservation */
> +	vhost_call(hw, VHOST_MSG_SET_VRING_BASE, &state);
> +
> +	struct vhost_vring_addr addr = {
> +		.index = queue_sel,
> +		.desc_user_addr = (uint64_t)vq->vq_ring.desc,
> +		.avail_user_addr = (uint64_t)vq->vq_ring.avail,
> +		.used_user_addr = (uint64_t)vq->vq_ring.used,
> +		.log_guest_addr = 0,
> +		.flags = 0, /* disable log */
> +	};
> +	vhost_call(hw, VHOST_MSG_SET_VRING_ADDR, &addr);
> +
> +	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_KICK comes
> +	 * lastly because vhost depends on this msg to judge if
> +	 * virtio_is_ready().
> +	 */
> +
> +	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
> +	if (kickfd < 0)
> +		rte_panic("kickfd error, %s\n", strerror(errno));
> +
> +	file.fd = kickfd;
> +	vhost_call(hw, VHOST_MSG_SET_VRING_KICK, &file);
> +	hw->kickfds[queue_sel] = kickfd;
> +}
> +
> +/**
> + * Merge those virtually adjacent memsegs into one region.
> + */
> +static void
> +prepare_vhost_memory_kernel(struct vhost_memory_kernel **p_vm)
> +{
> +	unsigned i, j, k = 0;
> +	struct rte_memseg *seg;
> +	struct vhost_memory_region *mr;
> +	struct vhost_memory_kernel *vm;
> +
> +	vm = malloc(sizeof(struct vhost_memory_kernel)
> +			+ RTE_MAX_MEMSEG * sizeof(struct vhost_memory_region));
> +
> +	for (i = 0; i < RTE_MAX_MEMSEG; ++i) {
> +		seg = &rte_eal_get_configuration()->mem_config->memseg[i];
> +		if (seg->addr == NULL)
> +			break;
> +
> +		int new_region = 1;
> +		for (j = 0; j < k; ++j) {
> +			mr = &vm->regions[j];
> +
> +			if (mr->userspace_addr + mr->memory_size
> +					== (uint64_t)seg->addr) {
> +				mr->memory_size += seg->len;
> +				new_region = 0;
> +				break;
> +			}
> +
> +			if ((uint64_t)seg->addr + seg->len
> +					== mr->userspace_addr) {
> +				mr->guest_phys_addr = (uint64_t)seg->addr;
> +				mr->userspace_addr = (uint64_t)seg->addr;
> +				mr->memory_size += seg->len;
> +				new_region = 0;
> +				break;
> +			}
> +		}
> +
> +		if (new_region == 0)
> +			continue;
> +
> +		mr = &vm->regions[k++];
> +		mr->guest_phys_addr = (uint64_t)seg->addr; /* use vaddr here! */
> +		mr->userspace_addr = (uint64_t)seg->addr;
> +		mr->memory_size = seg->len;
> +		mr->mmap_offset = 0;
> +	}
> +
> +	vm->nregions = k;
> +	vm->padding = 0;
> +	*p_vm = vm;
> +}
> +
> +static void kick_all_vq(struct virtio_hw *hw)
> +{
> +	int ret;
> +	unsigned i, queue_sel, nvqs;
> +	struct rte_eth_dev_data *data = hw->data;
> +
> +	if (hw->type == VHOST_KERNEL) {
> +		struct vhost_memory_kernel *vm = NULL;
> +		prepare_vhost_memory_kernel(&vm);
> +		vhost_call(hw, VHOST_MSG_SET_MEM_TABLE, vm);
> +		free(vm);
> +	} else {
> +		/* construct vhost_memory inside prepare_vhost_memory_user() */
> +		vhost_call(hw, VHOST_MSG_SET_MEM_TABLE, NULL);
> +	}
> +
> +	for (i = 0; i < data->nb_rx_queues; ++i) {
> +		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
> +		kick_one_vq(hw, data->rx_queues[i], queue_sel);
> +	}
> +	for (i = 0; i < data->nb_tx_queues; ++i) {
> +		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
> +		kick_one_vq(hw, data->tx_queues[i], queue_sel);
> +	}
> +
> +	/* after setup all virtqueues, we need to set_features again
> +	 * so that these features can be set into each virtqueue in
> +	 * vhost side.
> +	 */
> +	uint64_t features = hw->guest_features;
> +	features &= ~(1ull << VIRTIO_NET_F_MAC);
> +	vhost_call(hw, VHOST_MSG_SET_FEATURES, &features);
> +	if (ioctl(hw->backfd, TUNSETVNETHDRSZ, &hw->vtnet_hdr_size) == -1)
> +		rte_panic("TUNSETVNETHDRSZ failed: %s\n", strerror(errno));

 This is a bug. With VHOST_USER hw->backfd is not initialized (i suppose contains 0), and attemting this on stdin does nothing good.

> +	PMD_DRV_LOG(INFO, "set features:%"PRIx64"\n", features);
> +
> +	if (hw->type == VHOST_KERNEL) {
> +		struct vhost_vring_file file;
> +
> +		file.fd = hw->backfd;
> +		nvqs = data->nb_rx_queues + data->nb_tx_queues;
> +		for (file.index = 0; file.index < nvqs; ++file.index) {
> +			ret = vhost_kernel_ioctl(hw, VHOST_NET_SET_BACKEND, &file);
> +			if (ret < 0)
> +				rte_panic("VHOST_NET_SET_BACKEND failed, %s\n",
> +						strerror(errno));
> +		}
> +	}
> +
> +	/* TODO: VHOST_SET_LOG_BASE */
> +}
> +
> +void
> +virtio_ioport_write(struct virtio_hw *hw, uint64_t addr, uint32_t val)
> +{
> +	uint64_t guest_features;
> +
> +	switch (addr) {
> +	case VIRTIO_PCI_GUEST_FEATURES:
> +		guest_features = val;
> +		guest_features &= ~(1ull << VIRTIO_NET_F_MAC);
> +		vhost_call(hw, VHOST_MSG_SET_FEATURES, &guest_features);
> +		break;
> +	case VIRTIO_PCI_QUEUE_PFN:
> +		/* do nothing */
> +		break;
> +	case VIRTIO_PCI_QUEUE_SEL:
> +		hw->queue_sel = val;
> +		break;
> +	case VIRTIO_PCI_STATUS:
> +		if (val & VIRTIO_CONFIG_S_DRIVER_OK)
> +			kick_all_vq(hw);
> +		hw->status = val & 0xFF;
> +		break;
> +	case VIRTIO_PCI_QUEUE_NOTIFY:
> +		{
> +			int ret;
> +			uint64_t buf = 1;
> +			ret = write(hw->kickfds[val], &buf, sizeof(uint64_t));
> +			if (ret == -1)
> +				rte_panic("VIRTIO_PCI_QUEUE_NOTIFY failed: %s\n",
> +						strerror(errno));
> +			break;
> +		}
> +	default:
> +		PMD_DRV_LOG(ERR, "unexpected address %"PRIu64" value 0x%x\n",
> +				addr, val);
> +		break;
> +	}
> +}
> +
> +uint32_t
> +virtio_ioport_read(struct virtio_hw *hw, uint64_t addr)
> +{
> +	uint32_t ret = 0xFFFFFFFF;
> +	uint64_t host_features;
> +
> +	PMD_DRV_LOG(INFO, "addr: %"PRIu64"\n", addr);
> +
> +	switch (addr) {
> +	case VIRTIO_PCI_HOST_FEATURES:
> +		vhost_call(hw, VHOST_MSG_GET_FEATURES, &host_features);
> +		PMD_DRV_LOG(INFO, "get_features: %"PRIx64"\n", host_features);
> +		if (hw->mac_specified)
> +			host_features |= (1ull << VIRTIO_NET_F_MAC);
> +		/* disable it until we support CQ */
> +		host_features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
> +		ret = host_features;
> +		break;
> +	case VIRTIO_PCI_GUEST_FEATURES:
> +		ret = hw->guest_features;
> +		break;
> +	case VIRTIO_PCI_QUEUE_NUM:
> +		ret = hw->queue_num;
> +		break;
> +	case VIRTIO_PCI_QUEUE_SEL:
> +		ret = hw->queue_sel;
> +		break;
> +	case VIRTIO_PCI_STATUS:
> +		ret = hw->status;
> +		break;
> +	case 20: /* mac addr: 0~3 */
> +		if (hw->mac_specified) {
> +			uint32_t m0 = hw->mac_addr[0],
> +				 m1 = hw->mac_addr[1],
> +				 m2 = hw->mac_addr[2],
> +				 m3 = hw->mac_addr[3];
> +			ret = (m3 << 24) | (m2 << 16) | (m1 << 8) | m0;
> +		}
> +		break;
> +	case 24: /* mac addr: 4~5 */
> +		if (hw->mac_specified) {
> +			uint32_t m4 = hw->mac_addr[4],
> +				 m5 = hw->mac_addr[5];
> +			ret = (m5 << 8) | m4;
> +		}
> +		break;
> +	default:
> +		PMD_DRV_LOG(ERR, "%"PRIu64" (r) not supported\n", addr);
> +		break;
> +	}
> +
> +	return ret;
> +}
> +
> +#define TUN_DEF_SNDBUF	(1ull << 20)
> +
> +static void
> +vhost_kernel_backend_setup(struct virtio_hw *hw)
> +{
> +	int fd;
> +	int len = sizeof(struct virtio_net_hdr);
> +	int req_mq = 0;
> +	int sndbuf = TUN_DEF_SNDBUF;
> +	unsigned int features;
> +	struct ifreq ifr;
> +
> +	/* TODO:
> +	 * 1. get and set offload capability, tap_probe_has_ufo, tap_fd_set_offload
> +	 * 2. verify we can get and set vnet_hdr_len, tap_probe_vnet_hdr_len
> +
> +	 * 1. get number of memory regions from vhost module parameter
> +	 * max_mem_regions, supported in newer version linux kernel
> +	 */
> +
> +	fd = open(PATH_NET_TUN, O_RDWR);
> +	if (fd < 0)
> +		rte_panic("open %s error, %s\n", PATH_NET_TUN, strerror(errno));
> +
> +	memset(&ifr, 0, sizeof(ifr));
> +	ifr.ifr_flags = IFF_TAP | IFF_NO_PI;
> +
> +	if (ioctl(fd, TUNGETFEATURES, &features) == -1)
> +		rte_panic("TUNGETFEATURES failed: %s", strerror(errno));
> +
> +	if (features & IFF_ONE_QUEUE)
> +		ifr.ifr_flags |= IFF_ONE_QUEUE;
> +
> +	if (features & IFF_VNET_HDR)
> +		ifr.ifr_flags |= IFF_VNET_HDR;
> +	else
> +		rte_panic("vnet_hdr requested, but kernel does not support\n");
> +
> +	if (req_mq) {
> +		if (features & IFF_MULTI_QUEUE)
> +			ifr.ifr_flags |= IFF_MULTI_QUEUE;
> +		else
> +			rte_panic("multiqueue requested, but kernel does not support\n");
> +	}
> +
> +	strncpy(ifr.ifr_name, "tap%d", IFNAMSIZ);
> +	if (ioctl(fd, TUNSETIFF, (void *) &ifr) == -1)
> +		rte_panic("TUNSETIFF failed: %s", strerror(errno));
> +	fcntl(fd, F_SETFL, O_NONBLOCK);
> +
> +	if (ioctl(fd, TUNSETVNETHDRSZ, &len) == -1)
> +		rte_panic("TUNSETVNETHDRSZ failed: %s\n", strerror(errno));
> +
> +	if (ioctl(fd, TUNSETSNDBUF, &sndbuf) == -1)
> +		rte_panic("TUNSETSNDBUF failed: %s", strerror(errno));
> +
> +	hw->backfd = fd;
> +
> +	hw->vhostfd = open(hw->path, O_RDWR);
> +	if (hw->vhostfd == -1)
> +		rte_panic("open %s failed: %s\n", hw->path, strerror(errno));
> +}
> +
> +static void
> +vhost_user_backend_setup(struct virtio_hw *hw)
> +{
> +	int fd;
> +	int flag;
> +	struct sockaddr_un un;
> +
> +	fd = socket(AF_UNIX, SOCK_STREAM, 0);
> +	if (fd < 0)
> +		rte_panic("socket error, %s\n", strerror(errno));
> +
> +	flag = fcntl(fd, F_GETFD);
> +	fcntl(fd, F_SETFD, flag | FD_CLOEXEC);
> +
> +	memset(&un, 0, sizeof(un));
> +	un.sun_family = AF_UNIX;
> +	snprintf(un.sun_path, sizeof(un.sun_path), "%s", hw->path);
> +	if (connect(fd, (struct sockaddr *)&un, sizeof(un)) < 0) {
> +		PMD_DRV_LOG(ERR, "connect error, %s\n", strerror(errno));
> +		exit(-1);
> +	}
> +
> +	hw->vhostfd = fd;
> +}
> +
> +void
> +virtio_vdev_init(struct rte_eth_dev_data *data, const char *path,
> +		int nb_rx, int nb_tx, int nb_cq __attribute__ ((unused)),
> +		int queue_num, char *mac)
> +{
> +	int i;
> +	int ret;
> +	struct stat s;
> +	uint32_t tmp[ETHER_ADDR_LEN];
> +	struct virtio_hw *hw = data->dev_private;
> +
> +	hw->data = data;
> +	hw->path = strdup(path);
> +	hw->max_rx_queues = nb_rx;
> +	hw->max_tx_queues = nb_tx;
> +	hw->queue_num = queue_num;
> +	hw->mac_specified = 0;
> +	if (mac) {
> +		ret = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0], &tmp[1],
> +				&tmp[2], &tmp[3], &tmp[4], &tmp[5]);
> +		if (ret == ETHER_ADDR_LEN) {
> +			for (i = 0; i < ETHER_ADDR_LEN; ++i)
> +				hw->mac_addr[i] = (uint8_t)tmp[i];
> +			hw->mac_specified = 1;
> +		}
> +	}
> +
> +	/* TODO: cq */
> +
> +	ret = stat(hw->path, &s);
> +	if (ret < 0)
> +		rte_panic("stat: %s failed, %s\n", hw->path, strerror(errno));
> +
> +	switch (s.st_mode & S_IFMT) {
> +		case S_IFCHR:
> +			hw->type = VHOST_KERNEL;
> +			vhost_kernel_backend_setup(hw);
> +			break;
> +		case S_IFSOCK:
> +			hw->type = VHOST_USER;
> +			vhost_user_backend_setup(hw);
> +			break;
> +		default:
> +			rte_panic("unknown file type of %s\n", hw->path);
> +	}
> +	if (vhost_call(hw, VHOST_MSG_SET_OWNER, NULL) == -1)
> +		rte_panic("vhost set_owner failed: %s\n", strerror(errno));
> +}
> diff --git a/drivers/net/virtio/vhost.h b/drivers/net/virtio/vhost.h
> new file mode 100644
> index 0000000..c7517f6
> --- /dev/null
> +++ b/drivers/net/virtio/vhost.h
> @@ -0,0 +1,192 @@
> +/*-
> + *   BSD LICENSE
> + *
> + *   Copyright(c) 2010-2014 Intel Corporation. All rights reserved.
> + *   All rights reserved.
> + *
> + *   Redistribution and use in source and binary forms, with or without
> + *   modification, are permitted provided that the following conditions
> + *   are met:
> + *
> + *     * Redistributions of source code must retain the above copyright
> + *       notice, this list of conditions and the following disclaimer.
> + *     * Redistributions in binary form must reproduce the above copyright
> + *       notice, this list of conditions and the following disclaimer in
> + *       the documentation and/or other materials provided with the
> + *       distribution.
> + *     * Neither the name of Intel Corporation nor the names of its
> + *       contributors may be used to endorse or promote products derived
> + *       from this software without specific prior written permission.
> + *
> + *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
> + *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
> + *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
> + *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
> + *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
> + *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
> + *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
> + *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
> + *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
> + *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
> + *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
> + */
> +
> +#ifndef _VHOST_NET_USER_H
> +#define _VHOST_NET_USER_H
> +
> +#include <stdint.h>
> +#include <linux/types.h>
> +#include <linux/ioctl.h>
> +
> +#define VHOST_MEMORY_MAX_NREGIONS 8
> +
> +struct vhost_vring_state {
> +	unsigned int index;
> +	unsigned int num;
> +};
> +
> +struct vhost_vring_file {
> +	unsigned int index;
> +	int fd;
> +};
> +
> +struct vhost_vring_addr {
> +	unsigned int index;
> +	/* Option flags. */
> +	unsigned int flags;
> +	/* Flag values: */
> +	/* Whether log address is valid. If set enables logging. */
> +#define VHOST_VRING_F_LOG 0
> +
> +	/* Start of array of descriptors (virtually contiguous) */
> +	uint64_t desc_user_addr;
> +	/* Used structure address. Must be 32 bit aligned */
> +	uint64_t used_user_addr;
> +	/* Available structure address. Must be 16 bit aligned */
> +	uint64_t avail_user_addr;
> +	/* Logging support. */
> +	/* Log writes to used structure, at offset calculated from specified
> +	 * address. Address must be 32 bit aligned. */
> +	uint64_t log_guest_addr;
> +};
> +
> +#define VIRTIO_CONFIG_S_DRIVER_OK   4
> +
> +typedef enum VhostUserRequest {
> +	VHOST_USER_NONE = 0,
> +	VHOST_USER_GET_FEATURES = 1,
> +	VHOST_USER_SET_FEATURES = 2,
> +	VHOST_USER_SET_OWNER = 3,
> +	VHOST_USER_RESET_OWNER = 4,
> +	VHOST_USER_SET_MEM_TABLE = 5,
> +	VHOST_USER_SET_LOG_BASE = 6,
> +	VHOST_USER_SET_LOG_FD = 7,
> +	VHOST_USER_SET_VRING_NUM = 8,
> +	VHOST_USER_SET_VRING_ADDR = 9,
> +	VHOST_USER_SET_VRING_BASE = 10,
> +	VHOST_USER_GET_VRING_BASE = 11,
> +	VHOST_USER_SET_VRING_KICK = 12,
> +	VHOST_USER_SET_VRING_CALL = 13,
> +	VHOST_USER_SET_VRING_ERR = 14,
> +	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
> +	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
> +	VHOST_USER_GET_QUEUE_NUM = 17,
> +	VHOST_USER_SET_VRING_ENABLE = 18,
> +	VHOST_USER_MAX
> +} VhostUserRequest;
> +
> +struct vhost_memory_region {
> +	uint64_t guest_phys_addr;
> +	uint64_t memory_size; /* bytes */
> +	uint64_t userspace_addr;
> +	uint64_t mmap_offset;
> +};
> +struct vhost_memory_kernel {
> +	uint32_t nregions;
> +	uint32_t padding;
> +	struct vhost_memory_region regions[0];
> +};
> +
> +struct vhost_memory {
> +	uint32_t nregions;
> +	uint32_t padding;
> +	struct vhost_memory_region regions[VHOST_MEMORY_MAX_NREGIONS];
> +};
> +
> +typedef struct VhostUserMsg {
> +	VhostUserRequest request;
> +
> +#define VHOST_USER_VERSION_MASK     0x3
> +#define VHOST_USER_REPLY_MASK       (0x1 << 2)
> +	uint32_t flags;
> +	uint32_t size; /* the following payload size */
> +	union {
> +#define VHOST_USER_VRING_IDX_MASK   0xff
> +#define VHOST_USER_VRING_NOFD_MASK  (0x1 << 8)
> +		uint64_t u64;
> +		struct vhost_vring_state state;
> +		struct vhost_vring_addr addr;
> +		struct vhost_memory memory;
> +	} payload;
> +	int fds[VHOST_MEMORY_MAX_NREGIONS];
> +} __attribute((packed)) VhostUserMsg;
> +
> +#define VHOST_USER_HDR_SIZE offsetof(VhostUserMsg, payload.u64)
> +#define VHOST_USER_PAYLOAD_SIZE (sizeof(VhostUserMsg) - VHOST_USER_HDR_SIZE)
> +
> +/* The version of the protocol we support */
> +#define VHOST_USER_VERSION    0x1
> +
> +/* ioctls */
> +
> +#define VHOST_VIRTIO 0xAF
> +
> +#define VHOST_GET_FEATURES	_IOR(VHOST_VIRTIO, 0x00, __u64)
> +#define VHOST_SET_FEATURES	_IOW(VHOST_VIRTIO, 0x00, __u64)
> +#define VHOST_SET_OWNER _IO(VHOST_VIRTIO, 0x01)
> +#define VHOST_RESET_OWNER _IO(VHOST_VIRTIO, 0x02)
> +#define VHOST_SET_MEM_TABLE	_IOW(VHOST_VIRTIO, 0x03, struct vhost_memory_kernel)
> +#define VHOST_SET_LOG_BASE _IOW(VHOST_VIRTIO, 0x04, __u64)
> +#define VHOST_SET_LOG_FD _IOW(VHOST_VIRTIO, 0x07, int)
> +#define VHOST_SET_VRING_NUM _IOW(VHOST_VIRTIO, 0x10, struct vhost_vring_state)
> +#define VHOST_SET_VRING_ADDR _IOW(VHOST_VIRTIO, 0x11, struct vhost_vring_addr)
> +#define VHOST_SET_VRING_BASE _IOW(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
> +#define VHOST_GET_VRING_BASE _IOWR(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
> +#define VHOST_SET_VRING_KICK _IOW(VHOST_VIRTIO, 0x20, struct vhost_vring_file)
> +#define VHOST_SET_VRING_CALL _IOW(VHOST_VIRTIO, 0x21, struct vhost_vring_file)
> +#define VHOST_SET_VRING_ERR _IOW(VHOST_VIRTIO, 0x22, struct vhost_vring_file)
> +#define VHOST_NET_SET_BACKEND _IOW(VHOST_VIRTIO, 0x30, struct vhost_vring_file)
> +
> +/*****************************************************************************/
> +
> +/* Ioctl defines */
> +#define TUNSETIFF     _IOW('T', 202, int)
> +#define TUNGETFEATURES _IOR('T', 207, unsigned int)
> +#define TUNSETOFFLOAD  _IOW('T', 208, unsigned int)
> +#define TUNGETIFF      _IOR('T', 210, unsigned int)
> +#define TUNSETSNDBUF   _IOW('T', 212, int)
> +#define TUNGETVNETHDRSZ _IOR('T', 215, int)
> +#define TUNSETVNETHDRSZ _IOW('T', 216, int)
> +#define TUNSETQUEUE  _IOW('T', 217, int)
> +#define TUNSETVNETLE _IOW('T', 220, int)
> +#define TUNSETVNETBE _IOW('T', 222, int)
> +
> +/* TUNSETIFF ifr flags */
> +#define IFF_TAP          0x0002
> +#define IFF_NO_PI        0x1000
> +#define IFF_ONE_QUEUE    0x2000
> +#define IFF_VNET_HDR     0x4000
> +#define IFF_MULTI_QUEUE  0x0100
> +#define IFF_ATTACH_QUEUE 0x0200
> +#define IFF_DETACH_QUEUE 0x0400
> +
> +/* Features for GSO (TUNSETOFFLOAD). */
> +#define TUN_F_CSUM	0x01	/* You can hand me unchecksummed packets. */
> +#define TUN_F_TSO4	0x02	/* I can handle TSO for IPv4 packets */
> +#define TUN_F_TSO6	0x04	/* I can handle TSO for IPv6 packets */
> +#define TUN_F_TSO_ECN	0x08	/* I can handle TSO with ECN bits. */
> +#define TUN_F_UFO	0x10	/* I can handle UFO packets */
> +
> +#define PATH_NET_TUN "/dev/net/tun"
> +
> +#endif
> diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
> index ae2d47d..9e1ecb3 100644
> --- a/drivers/net/virtio/virtio_ethdev.h
> +++ b/drivers/net/virtio/virtio_ethdev.h
> @@ -122,5 +122,8 @@ uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf
> **tx_pkts,
>  #define VTNET_LRO_FEATURES (VIRTIO_NET_F_GUEST_TSO4 | \
>  			    VIRTIO_NET_F_GUEST_TSO6 | VIRTIO_NET_F_GUEST_ECN)
> 
> -
> +#ifdef RTE_VIRTIO_VDEV
> +void virtio_vdev_init(struct rte_eth_dev_data *data, const char *path,
> +		int nb_rx, int nb_tx, int nb_cq, int queue_num, char *mac);
> +#endif
>  #endif /* _VIRTIO_ETHDEV_H_ */
> diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
> index 47f722a..af05ae2 100644
> --- a/drivers/net/virtio/virtio_pci.h
> +++ b/drivers/net/virtio/virtio_pci.h
> @@ -147,7 +147,6 @@ struct virtqueue;
>   * rest are per-device feature bits.
>   */
>  #define VIRTIO_TRANSPORT_F_START 28
> -#define VIRTIO_TRANSPORT_F_END   32
> 
>  /* The Guest publishes the used index for which it expects an interrupt
>   * at the end of the avail ring. Host should ignore the avail->flags field. */
> @@ -165,6 +164,7 @@ struct virtqueue;
> 
>  struct virtio_hw {
>  	struct virtqueue *cvq;
> +#define VIRTIO_VDEV_IO_BASE	0xffffffff
>  	uint32_t    io_base;
>  	uint32_t    guest_features;
>  	uint32_t    max_tx_queues;
> @@ -174,6 +174,21 @@ struct virtio_hw {
>  	uint8_t	    use_msix;
>  	uint8_t     started;
>  	uint8_t     mac_addr[ETHER_ADDR_LEN];
> +#ifdef RTE_VIRTIO_VDEV
> +#define VHOST_KERNEL	0
> +#define VHOST_USER		1
> +	int			type; /* type of backend */
> +	uint32_t	queue_num;
> +	char		*path;
> +	int			mac_specified;
> +	int			vhostfd;
> +	int			backfd; /* tap device used in vhost-net */
> +	int			callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
> +	int			kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
> +	uint32_t	queue_sel;
> +	uint8_t		status;
> +	struct rte_eth_dev_data *data;
> +#endif
>  };
> 
>  /*
> @@ -229,6 +244,39 @@ outl_p(unsigned int data, unsigned int port)
>  #define VIRTIO_PCI_REG_ADDR(hw, reg) \
>  	(unsigned short)((hw)->io_base + (reg))
> 
> +#ifdef RTE_VIRTIO_VDEV
> +uint32_t virtio_ioport_read(struct virtio_hw *, uint64_t);
> +void virtio_ioport_write(struct virtio_hw *, uint64_t, uint32_t);
> +
> +#define VIRTIO_READ_REG_1(hw, reg) \
> +	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	inb((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> +	:virtio_ioport_read(hw, reg)
> +#define VIRTIO_WRITE_REG_1(hw, reg, value) \
> +	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	outb_p((unsigned char)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> +	:virtio_ioport_write(hw, reg, value)
> +
> +#define VIRTIO_READ_REG_2(hw, reg) \
> +	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	inw((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> +	:virtio_ioport_read(hw, reg)
> +#define VIRTIO_WRITE_REG_2(hw, reg, value) \
> +	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	outw_p((unsigned short)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> +	:virtio_ioport_write(hw, reg, value)
> +
> +#define VIRTIO_READ_REG_4(hw, reg) \
> +	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	inl((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> +	:virtio_ioport_read(hw, reg)
> +#define VIRTIO_WRITE_REG_4(hw, reg, value) \
> +	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	outl_p((unsigned int)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> +	:virtio_ioport_write(hw, reg, value)
> +
> +#else /* RTE_VIRTIO_VDEV */
> +
>  #define VIRTIO_READ_REG_1(hw, reg) \
>  	inb((VIRTIO_PCI_REG_ADDR((hw), (reg))))
>  #define VIRTIO_WRITE_REG_1(hw, reg, value) \
> @@ -244,6 +292,8 @@ outl_p(unsigned int data, unsigned int port)
>  #define VIRTIO_WRITE_REG_4(hw, reg, value) \
>  	outl_p((unsigned int)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg))))
> 
> +#endif /* RTE_VIRTIO_VDEV */
> +
>  static inline int
>  vtpci_with_feature(struct virtio_hw *hw, uint32_t bit)
>  {
> --
> 2.1.4

Kind regards,
Pavel Fedin
Expert Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-10 11:42 ` [PATCH 0/4] " Jianfeng Tan
                     ` (3 preceding siblings ...)
  2016-01-10 11:43   ` [PATCH 4/4] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
@ 2016-01-11 14:21   ` Pavel Fedin
  2016-01-11 15:53     ` Tan, Jianfeng
  2016-01-12  5:36   ` Tetsuya Mukawa
  2016-01-26  6:02   ` Qiu, Michael
  6 siblings, 1 reply; 196+ messages in thread
From: Pavel Fedin @ 2016-01-11 14:21 UTC (permalink / raw)
  To: 'Jianfeng Tan', dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

 Hello!


> This patchset is to provide high performance networking interface (virtio)
> for container-based DPDK applications. The way of starting DPDK apps in
> containers with ownership of NIC devices exclusively is beyond the scope.
> The basic idea here is to present a new virtual device (named eth_cvio),
> which can be discovered and initialized in container-based DPDK apps using
> rte_eal_init(). To minimize the change, we reuse already-existing virtio
> frontend driver code (driver/net/virtio/).

 With the aforementioned fixes i tried to run it inside libvirt-lxc. I got the following:
a) With hugepages - "abort with 256 hugepage files exceed the maximum of 8 for vhost-user" - i set -m 512
b) With --single-file - ovs runs, but doesn't get any packets at all. When i try to ping the container from within host side, it
counts drops on vhost-user port.

Kind regards,
Pavel Fedin
Expert Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-11 14:21   ` [PATCH 0/4] virtio support for container Pavel Fedin
@ 2016-01-11 15:53     ` Tan, Jianfeng
  2016-01-12  7:38       ` Pavel Fedin
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-11 15:53 UTC (permalink / raw)
  To: Pavel Fedin, dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying



Hi Fedin,
>   With the aforementioned fixes i tried to run it inside libvirt-lxc. I got the following:
> a) With hugepages - "abort with 256 hugepage files exceed the maximum of 8 for vhost-user" - i set -m 512
This is currently a known issue, we have discussed in another thread 
with Tetsuya.
> b) With --single-file - ovs runs, but doesn't get any packets at all. When i try to ping the container from within host side, it
> counts drops on vhost-user port.
Can you check the OVS in host side, if it prints out message of "virtio 
is now ready for processing"?

Thanks,
Jianfeng

>
> Kind regards,
> Pavel Fedin
> Expert Engineer
> Samsung Electronics Research center Russia
>
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-10 11:43   ` [PATCH 2/4] mem: add API to obstain memory-backed file info Jianfeng Tan
  2016-01-11 11:43     ` Pavel Fedin
@ 2016-01-11 20:26     ` Rich Lane
  2016-01-12  9:12       ` Tan, Jianfeng
  1 sibling, 1 reply; 196+ messages in thread
From: Rich Lane @ 2016-01-11 20:26 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

On Sun, Jan 10, 2016 at 3:43 AM, Jianfeng Tan <jianfeng.tan@intel.com>
wrote:

> @@ -1157,6 +1180,20 @@ rte_eal_hugepage_init(void)
>                 mcfg->memseg[0].len = internal_config.memory;
>                 mcfg->memseg[0].socket_id = socket_id;
>
> +               hugepage = create_shared_memory(eal_hugepage_info_path(),
> +                               sizeof(struct hugepage_file));
> +               hugepage->orig_va = addr;
> +               hugepage->final_va = addr;
> +               hugepage->physaddr = rte_mem_virt2phy(addr);
> +               hugepage->size = pagesize;
>

Should this be "hugepage->size = internal_config.memory"? Otherwise the
vhost-user
memtable entry has a size of only 2MB.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-10 11:42 ` [PATCH 0/4] " Jianfeng Tan
                     ` (4 preceding siblings ...)
  2016-01-11 14:21   ` [PATCH 0/4] virtio support for container Pavel Fedin
@ 2016-01-12  5:36   ` Tetsuya Mukawa
  2016-01-12  5:46     ` Tan, Jianfeng
  2016-01-20  3:48     ` Xie, Huawei
  2016-01-26  6:02   ` Qiu, Michael
  6 siblings, 2 replies; 196+ messages in thread
From: Tetsuya Mukawa @ 2016-01-12  5:36 UTC (permalink / raw)
  To: Jianfeng Tan, dev, huawei.xie; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

On 2016/01/10 20:42, Jianfeng Tan wrote:
> This patchset is to provide high performance networking interface (virtio)
> for container-based DPDK applications. The way of starting DPDK apps in
> containers with ownership of NIC devices exclusively is beyond the scope.
> The basic idea here is to present a new virtual device (named eth_cvio),
> which can be discovered and initialized in container-based DPDK apps using
> rte_eal_init(). To minimize the change, we reuse already-existing virtio
> frontend driver code (driver/net/virtio/).
>  
> Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
> operations into unix socket/cuse protocol, which is originally provided in
> QEMU), is integrated in virtio frontend driver. So this converged driver
> actually plays the role of original frontend driver and the role of QEMU
> device framework.
>  
> The major difference lies in how to calculate relative address for vhost.
> The principle of virtio is that: based on one or multiple shared memory
> segments, vhost maintains a reference system with the base addresses and
> length for each segment so that an address from VM comes (usually GPA,
> Guest Physical Address) can be translated into vhost-recognizable address
> (named VVA, Vhost Virtual Address). To decrease the overhead of address
> translation, we should maintain as few segments as possible. In VM's case,
> GPA is always locally continuous. In container's case, CVA (Container
> Virtual Address) can be used. Specifically:
> a. when set_base_addr, CVA address is used;
> b. when preparing RX's descriptors, CVA address is used;
> c. when transmitting packets, CVA is filled in TX's descriptors;
> d. in TX and CQ's header, CVA is used.
>  
> How to share memory? In VM's case, qemu always shares all physical layout
> to backend. But it's not feasible for a container, as a process, to share
> all virtual memory regions to backend. So only specified virtual memory
> regions (with type of shared) are sent to backend. It's a limitation that
> only addresses in these areas can be used to transmit or receive packets.
>
> Known issues
>
> a. When used with vhost-net, root privilege is required to create tap
> device inside.
> b. Control queue and multi-queue are not supported yet.
> c. When --single-file option is used, socket_id of the memory may be
> wrong. (Use "numactl -N x -m x" to work around this for now)
>  
> How to use?
>
> a. Apply this patchset.
>
> b. To compile container apps:
> $: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
>
> c. To build a docker image using Dockerfile below.
> $: cat ./Dockerfile
> FROM ubuntu:latest
> WORKDIR /usr/src/dpdk
> COPY . /usr/src/dpdk
> ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
> $: docker build -t dpdk-app-l2fwd .
>
> d. Used with vhost-user
> $: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
> 	--socket-mem 1024,1024 -- -p 0x1 --stats 1
> $: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
> 	-v /dev/hugepages:/dev/hugepages \
> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> 	--vdev=eth_cvio0,path=/var/run/usvhost -- -p 0x1
>
> f. Used with vhost-net
> $: modprobe vhost
> $: modprobe vhost-net
> $: docker run -i -t --privileged \
> 	-v /dev/vhost-net:/dev/vhost-net \
> 	-v /dev/net/tun:/dev/net/tun \
> 	-v /dev/hugepages:/dev/hugepages \
> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> 	--vdev=eth_cvio0,path=/dev/vhost-net -- -p 0x1
>
> By the way, it's not necessary to run in a container.
>
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
>
> Jianfeng Tan (4):
>   mem: add --single-file to create single mem-backed file
>   mem: add API to obstain memory-backed file info
>   virtio/vdev: add ways to interact with vhost
>   virtio/vdev: add a new vdev named eth_cvio
>
>  config/common_linuxapp                     |   5 +
>  drivers/net/virtio/Makefile                |   4 +
>  drivers/net/virtio/vhost.c                 | 734 +++++++++++++++++++++++++++++
>  drivers/net/virtio/vhost.h                 | 192 ++++++++
>  drivers/net/virtio/virtio_ethdev.c         | 338 ++++++++++---
>  drivers/net/virtio/virtio_ethdev.h         |   4 +
>  drivers/net/virtio/virtio_pci.h            |  52 +-
>  drivers/net/virtio/virtio_rxtx.c           |  11 +-
>  drivers/net/virtio/virtio_rxtx_simple.c    |  14 +-
>  drivers/net/virtio/virtqueue.h             |  13 +-
>  lib/librte_eal/common/eal_common_options.c |  17 +
>  lib/librte_eal/common/eal_internal_cfg.h   |   1 +
>  lib/librte_eal/common/eal_options.h        |   2 +
>  lib/librte_eal/common/include/rte_memory.h |  16 +
>  lib/librte_eal/linuxapp/eal/eal_memory.c   |  82 +++-
>  15 files changed, 1392 insertions(+), 93 deletions(-)
>  create mode 100644 drivers/net/virtio/vhost.c
>  create mode 100644 drivers/net/virtio/vhost.h
>

Hi Jianfeng and Xie,

I guess my implementation and yours have a lot of common code, so I will
try to rebase my patch on yours.

BTW, one thing I need to change your memory allocation way is that
mmaped address should be under 44bit(32 + PAGE_SHIFT) to work with my patch.
This is because VIRTIO_PCI_QUEUE_PFN register only accepts such address.
(I may need to add one more EAL parameter like "--mmap-under <address>")

Thanks,
Tetsuya

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12  5:36   ` Tetsuya Mukawa
@ 2016-01-12  5:46     ` Tan, Jianfeng
  2016-01-12  6:01       ` Tetsuya Mukawa
  2016-01-20  3:48     ` Xie, Huawei
  1 sibling, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-12  5:46 UTC (permalink / raw)
  To: Tetsuya Mukawa, dev, huawei.xie
  Cc: nakajima.yoshihiro, mst, ann.zhuangyanying


Hi Tetsuya,
  

> Hi Jianfeng and Xie,
>
> I guess my implementation and yours have a lot of common code, so I will
> try to rebase my patch on yours.

We also think so. And before you rebase your code, I think we can rely 
on Yuanhan's
struct virtio_pci_ops to make the code structure brief and clear, as 
discussed in your
patch's thread, i.e., we both rebase our code according to Yuanhan's 
code. Is that OK?


>
> BTW, one thing I need to change your memory allocation way is that
> mmaped address should be under 44bit(32 + PAGE_SHIFT) to work with my patch.
> This is because VIRTIO_PCI_QUEUE_PFN register only accepts such address.
> (I may need to add one more EAL parameter like "--mmap-under <address>")

It makes sense.

Thanks,
Jianfeng

>
> Thanks,
> Tetsuya

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12  5:46     ` Tan, Jianfeng
@ 2016-01-12  6:01       ` Tetsuya Mukawa
  2016-01-12  6:14         ` Yuanhan Liu
  0 siblings, 1 reply; 196+ messages in thread
From: Tetsuya Mukawa @ 2016-01-12  6:01 UTC (permalink / raw)
  To: Tan, Jianfeng, dev, huawei.xie; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

On 2016/01/12 14:46, Tan, Jianfeng wrote:
>
> Hi Tetsuya,
>  
>
>> Hi Jianfeng and Xie,
>>
>> I guess my implementation and yours have a lot of common code, so I will
>> try to rebase my patch on yours.
>
> We also think so. And before you rebase your code, I think we can rely
> on Yuanhan's
> struct virtio_pci_ops to make the code structure brief and clear, as
> discussed in your
> patch's thread, i.e., we both rebase our code according to Yuanhan's
> code. Is that OK?
>

Yes, I agree with it.

Thanks,
Tetsuya

>
>>
>> BTW, one thing I need to change your memory allocation way is that
>> mmaped address should be under 44bit(32 + PAGE_SHIFT) to work with my
>> patch.
>> This is because VIRTIO_PCI_QUEUE_PFN register only accepts such address.
>> (I may need to add one more EAL parameter like "--mmap-under <address>")
>
> It makes sense.
>
> Thanks,
> Jianfeng
>
>>
>> Thanks,
>> Tetsuya
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12  6:01       ` Tetsuya Mukawa
@ 2016-01-12  6:14         ` Yuanhan Liu
  2016-01-12  6:26           ` Tetsuya Mukawa
  0 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2016-01-12  6:14 UTC (permalink / raw)
  To: Tetsuya Mukawa, Thomas Monjalon, Xie, Huawei, Xu, Qian Q
  Cc: nakajima.yoshihiro, mst, dev, ann.zhuangyanying

On Tue, Jan 12, 2016 at 03:01:01PM +0900, Tetsuya Mukawa wrote:
> On 2016/01/12 14:46, Tan, Jianfeng wrote:
> >
> > Hi Tetsuya,
> >  
> >
> >> Hi Jianfeng and Xie,
> >>
> >> I guess my implementation and yours have a lot of common code, so I will
> >> try to rebase my patch on yours.
> >
> > We also think so. And before you rebase your code, I think we can rely
> > on Yuanhan's
> > struct virtio_pci_ops to make the code structure brief and clear, as
> > discussed in your
> > patch's thread, i.e., we both rebase our code according to Yuanhan's
> > code. Is that OK?
> >
> 
> Yes, I agree with it.

I will send v2 out today, and hopefully someone will ACK and test it
soon.  After that, I'm also hoping Thomas could do a quick merge then.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12  6:14         ` Yuanhan Liu
@ 2016-01-12  6:26           ` Tetsuya Mukawa
  2016-01-12  6:29             ` Yuanhan Liu
  0 siblings, 1 reply; 196+ messages in thread
From: Tetsuya Mukawa @ 2016-01-12  6:26 UTC (permalink / raw)
  To: Yuanhan Liu, Thomas Monjalon, Xie, Huawei, Xu, Qian Q
  Cc: nakajima.yoshihiro, mst, dev, ann.zhuangyanying

On 2016/01/12 15:14, Yuanhan Liu wrote:
> On Tue, Jan 12, 2016 at 03:01:01PM +0900, Tetsuya Mukawa wrote:
>> On 2016/01/12 14:46, Tan, Jianfeng wrote:
>>> Hi Tetsuya,
>>>  
>>>
>>>> Hi Jianfeng and Xie,
>>>>
>>>> I guess my implementation and yours have a lot of common code, so I will
>>>> try to rebase my patch on yours.
>>> We also think so. And before you rebase your code, I think we can rely
>>> on Yuanhan's
>>> struct virtio_pci_ops to make the code structure brief and clear, as
>>> discussed in your
>>> patch's thread, i.e., we both rebase our code according to Yuanhan's
>>> code. Is that OK?
>>>
>> Yes, I agree with it.
> I will send v2 out today, and hopefully someone will ACK and test it
> soon.  After that, I'm also hoping Thomas could do a quick merge then.
>
> 	--yliu
>

Hi Yuanhan,

Thanks, I will review and test it also.

Tetsuya

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12  6:26           ` Tetsuya Mukawa
@ 2016-01-12  6:29             ` Yuanhan Liu
  0 siblings, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-01-12  6:29 UTC (permalink / raw)
  To: Tetsuya Mukawa; +Cc: nakajima.yoshihiro, mst, dev, ann.zhuangyanying

On Tue, Jan 12, 2016 at 03:26:47PM +0900, Tetsuya Mukawa wrote:
> On 2016/01/12 15:14, Yuanhan Liu wrote:
> > On Tue, Jan 12, 2016 at 03:01:01PM +0900, Tetsuya Mukawa wrote:
> >> On 2016/01/12 14:46, Tan, Jianfeng wrote:
> >>> Hi Tetsuya,
> >>>  
> >>>
> >>>> Hi Jianfeng and Xie,
> >>>>
> >>>> I guess my implementation and yours have a lot of common code, so I will
> >>>> try to rebase my patch on yours.
> >>> We also think so. And before you rebase your code, I think we can rely
> >>> on Yuanhan's
> >>> struct virtio_pci_ops to make the code structure brief and clear, as
> >>> discussed in your
> >>> patch's thread, i.e., we both rebase our code according to Yuanhan's
> >>> code. Is that OK?
> >>>
> >> Yes, I agree with it.
> > I will send v2 out today, and hopefully someone will ACK and test it
> > soon.  After that, I'm also hoping Thomas could do a quick merge then.
> >
> > 	--yliu
> >
> 
> Hi Yuanhan,
> 
> Thanks, I will review and test it also.

Appreciate that!

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-11 15:53     ` Tan, Jianfeng
@ 2016-01-12  7:38       ` Pavel Fedin
  2016-01-12  8:14         ` Rich Lane
  0 siblings, 1 reply; 196+ messages in thread
From: Pavel Fedin @ 2016-01-12  7:38 UTC (permalink / raw)
  To: 'Tan, Jianfeng', dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

 Hello!

> > b) With --single-file - ovs runs, but doesn't get any packets at all. When i try to ping
> the container from within host side, it
> > counts drops on vhost-user port.
> Can you check the OVS in host side, if it prints out message of "virtio
> is now ready for processing"?

 No, i get errors:
--- cut ---
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: new virtio connection is 38
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: new device, handle is 0
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: read message VHOST_USER_SET_OWNER
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: read message VHOST_USER_GET_FEATURES
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: read message VHOST_USER_SET_FEATURES
Jan 12 10:27:43 nfv_test_x86_64 kernel: device ovs-netdev entered promiscuous mode
Jan 12 10:27:43 nfv_test_x86_64 kernel: device ovs0 entered promiscuous mode
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: read message VHOST_USER_SET_MEM_TABLE
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: mapped region 0 fd:39 to:0x7f079c600000 sz:0x200000 off:0x0
align:0x200000
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: vring call idx:0 file:49
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 10:27:43 nfv_test_x86_64 ovs-vswitchd[18858]: VHOST_CONFIG: (0) Failed to find desc ring address.
--- cut ---

Kind regards,
Pavel Fedin
Expert Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 4/4] virtio/vdev: add a new vdev named eth_cvio
  2016-01-10 11:43   ` [PATCH 4/4] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
@ 2016-01-12  7:45     ` Pavel Fedin
  2016-01-12  7:59       ` Yuanhan Liu
  2016-01-12  8:39       ` Tan, Jianfeng
  2016-01-27  3:10     ` Qiu, Michael
  1 sibling, 2 replies; 196+ messages in thread
From: Pavel Fedin @ 2016-01-12  7:45 UTC (permalink / raw)
  To: 'Jianfeng Tan', dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

 Hello!

 See inline

> -----Original Message-----
> From: Jianfeng Tan [mailto:jianfeng.tan@intel.com]
> Sent: Sunday, January 10, 2016 2:43 PM
> To: dev@dpdk.org
> Cc: rich.lane@bigswitch.com; yuanhan.liu@linux.intel.com; mst@redhat.com;
> nakajima.yoshihiro@lab.ntt.co.jp; huawei.xie@intel.com; mukawa@igel.co.jp;
> p.fedin@samsung.com; michael.qiu@intel.com; ann.zhuangyanying@huawei.com; Jianfeng Tan
> Subject: [PATCH 4/4] virtio/vdev: add a new vdev named eth_cvio
> 
> Add a new virtual device named eth_cvio, it can be used just like
> eth_ring, eth_null, etc.
> 
> Configured parameters include:
>     - rx (optional, 1 by default): number of rx, only allowed to be
> 				   1 for now.
>     - tx (optional, 1 by default): number of tx, only allowed to be
> 				   1 for now.
>     - cq (optional, 0 by default): if ctrl queue is enabled, not
> 				   supported for now.
>     - mac (optional): mac address, random value will be given if not
> 		      specified.
>     - queue_num (optional, 256 by default): size of virtqueue.
>     - path (madatory): path of vhost, depends on the file type:
> 		       vhost-user is used if the given path points to
> 	               a unix socket; vhost-net is used if the given
> 		       path points to a char device.
> 
> The major difference with original virtio for vm is that, here we
> use virtual address instead of physical address for vhost to
> calculate relative address.
> 
> When enable CONFIG_RTE_VIRTIO_VDEV (enabled by default), the compiled
> library can be used in both VM and container environment.
> 
> Examples:
> a. Use vhost-net as a backend
> sudo numactl -N 1 -m 1 ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
>     -m 1024 --no-pci --single-file --file-prefix=l2fwd \
>     --vdev=eth_cvio0,mac=00:01:02:03:04:05,path=/dev/vhost-net \
>     -- -p 0x1
> 
> b. Use vhost-user as a backend
> numactl -N 1 -m 1 ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 -m 1024 \
>     --no-pci --single-file --file-prefix=l2fwd \
>     --vdev=eth_cvio0,mac=00:01:02:03:04:05,path=<path_to_vhost_user> \
>     -- -p 0x1
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ---
>  drivers/net/virtio/virtio_ethdev.c      | 338 +++++++++++++++++++++++++-------
>  drivers/net/virtio/virtio_ethdev.h      |   1 +
>  drivers/net/virtio/virtio_pci.h         |  24 +--
>  drivers/net/virtio/virtio_rxtx.c        |  11 +-
>  drivers/net/virtio/virtio_rxtx_simple.c |  14 +-
>  drivers/net/virtio/virtqueue.h          |  13 +-
>  6 files changed, 302 insertions(+), 99 deletions(-)
> 
> diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
> index d928339..6e46060 100644
> --- a/drivers/net/virtio/virtio_ethdev.c
> +++ b/drivers/net/virtio/virtio_ethdev.c
> @@ -56,6 +56,7 @@
>  #include <rte_memory.h>
>  #include <rte_eal.h>
>  #include <rte_dev.h>
> +#include <rte_kvargs.h>
> 
>  #include "virtio_ethdev.h"
>  #include "virtio_pci.h"
> @@ -174,14 +175,14 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
>  	 * One RX packet for ACK.
>  	 */
>  	vq->vq_ring.desc[head].flags = VRING_DESC_F_NEXT;
> -	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mz->phys_addr;
> +	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mem;
>  	vq->vq_ring.desc[head].len = sizeof(struct virtio_net_ctrl_hdr);
>  	vq->vq_free_cnt--;
>  	i = vq->vq_ring.desc[head].next;
> 
>  	for (k = 0; k < pkt_num; k++) {
>  		vq->vq_ring.desc[i].flags = VRING_DESC_F_NEXT;
> -		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
> +		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
>  			+ sizeof(struct virtio_net_ctrl_hdr)
>  			+ sizeof(ctrl->status) + sizeof(uint8_t)*sum;
>  		vq->vq_ring.desc[i].len = dlen[k];
> @@ -191,7 +192,7 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
>  	}
> 
>  	vq->vq_ring.desc[i].flags = VRING_DESC_F_WRITE;
> -	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
> +	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
>  			+ sizeof(struct virtio_net_ctrl_hdr);
>  	vq->vq_ring.desc[i].len = sizeof(ctrl->status);
>  	vq->vq_free_cnt--;
> @@ -374,68 +375,85 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
>  		}
>  	}
> 
> -	/*
> -	 * Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
> -	 * and only accepts 32 bit page frame number.
> -	 * Check if the allocated physical memory exceeds 16TB.
> -	 */
> -	if ((mz->phys_addr + vq->vq_ring_size - 1) >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
> -		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
> -		rte_free(vq);
> -		return -ENOMEM;
> -	}
> -
>  	memset(mz->addr, 0, sizeof(mz->len));
>  	vq->mz = mz;
> -	vq->vq_ring_mem = mz->phys_addr;
>  	vq->vq_ring_virt_mem = mz->addr;
> -	PMD_INIT_LOG(DEBUG, "vq->vq_ring_mem:      0x%"PRIx64, (uint64_t)mz->phys_addr);
> -	PMD_INIT_LOG(DEBUG, "vq->vq_ring_virt_mem: 0x%"PRIx64, (uint64_t)(uintptr_t)mz->addr);
> +
> +	if (dev->dev_type == RTE_ETH_DEV_PCI) {
> +		vq->vq_ring_mem = mz->phys_addr;
> +
> +		/* Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
> +		 * and only accepts 32 bit page frame number.
> +		 * Check if the allocated physical memory exceeds 16TB.
> +		 */
> +		uint64_t last_physaddr = vq->vq_ring_mem + vq->vq_ring_size - 1;
> +		if (last_physaddr >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
> +			PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
> +			rte_free(vq);
> +			return -ENOMEM;
> +		}
> +	}
> +#ifdef RTE_VIRTIO_VDEV
> +	else
> +		vq->vq_ring_mem = (phys_addr_t)mz->addr; /* Use vaddr!!! */
> +#endif
> +
> +	PMD_INIT_LOG(DEBUG, "vq->vq_ring_mem:      0x%"PRIx64,
> +			(uint64_t)vq->vq_ring_mem);
> +	PMD_INIT_LOG(DEBUG, "vq->vq_ring_virt_mem: 0x%"PRIx64,
> +			(uint64_t)(uintptr_t)vq->vq_ring_virt_mem);
>  	vq->virtio_net_hdr_mz  = NULL;
>  	vq->virtio_net_hdr_mem = 0;
> 
> +	uint64_t hdr_size = 0;
>  	if (queue_type == VTNET_TQ) {
>  		/*
>  		 * For each xmit packet, allocate a virtio_net_hdr
>  		 */
>  		snprintf(vq_name, sizeof(vq_name), "port%d_tvq%d_hdrzone",
>  			dev->data->port_id, queue_idx);
> -		vq->virtio_net_hdr_mz = rte_memzone_reserve_aligned(vq_name,
> -			vq_size * hw->vtnet_hdr_size,
> -			socket_id, 0, RTE_CACHE_LINE_SIZE);
> -		if (vq->virtio_net_hdr_mz == NULL) {
> -			if (rte_errno == EEXIST)
> -				vq->virtio_net_hdr_mz =
> -					rte_memzone_lookup(vq_name);
> -			if (vq->virtio_net_hdr_mz == NULL) {
> -				rte_free(vq);
> -				return -ENOMEM;
> -			}
> -		}
> -		vq->virtio_net_hdr_mem =
> -			vq->virtio_net_hdr_mz->phys_addr;
> -		memset(vq->virtio_net_hdr_mz->addr, 0,
> -			vq_size * hw->vtnet_hdr_size);
> +		hdr_size = vq_size * hw->vtnet_hdr_size;
>  	} else if (queue_type == VTNET_CQ) {
>  		/* Allocate a page for control vq command, data and status */
>  		snprintf(vq_name, sizeof(vq_name), "port%d_cvq_hdrzone",
>  			dev->data->port_id);
> -		vq->virtio_net_hdr_mz = rte_memzone_reserve_aligned(vq_name,
> -			PAGE_SIZE, socket_id, 0, RTE_CACHE_LINE_SIZE);
> -		if (vq->virtio_net_hdr_mz == NULL) {
> +		hdr_size = PAGE_SIZE;
> +	}
> +
> +	if (hdr_size) { /* queue_type is VTNET_TQ or VTNET_CQ */
> +		mz = rte_memzone_reserve_aligned(vq_name,
> +				hdr_size, socket_id, 0, RTE_CACHE_LINE_SIZE);
> +		if (mz == NULL) {
>  			if (rte_errno == EEXIST)
> -				vq->virtio_net_hdr_mz =
> -					rte_memzone_lookup(vq_name);
> -			if (vq->virtio_net_hdr_mz == NULL) {
> +				mz = rte_memzone_lookup(vq_name);
> +			if (mz == NULL) {
>  				rte_free(vq);
>  				return -ENOMEM;
>  			}
>  		}
> -		vq->virtio_net_hdr_mem =
> -			vq->virtio_net_hdr_mz->phys_addr;
> -		memset(vq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
> +		vq->virtio_net_hdr_mz = mz;
> +		vq->virtio_net_hdr_vaddr = mz->addr;
> +		memset(vq->virtio_net_hdr_vaddr, 0, hdr_size);
> +
> +		if (dev->dev_type == RTE_ETH_DEV_PCI)
> +			vq->virtio_net_hdr_mem = mz->phys_addr;
> +#ifdef RTE_VIRTIO_VDEV
> +		else
> +			vq->virtio_net_hdr_mem = (phys_addr_t)mz->addr; /* Use vaddr!!! */
> +#endif
>  	}
> 
> +	struct rte_mbuf *m = NULL;
> +	if (dev->dev_type == RTE_ETH_DEV_PCI)
> +		vq->offset = (uintptr_t)&m->buf_addr;
> +#ifdef RTE_VIRTIO_VDEV
> +	else {
> +		vq->offset = (uintptr_t)&m->buf_physaddr;

 Not sure, but shouldn't these be swapped? Originally, for PCI devices, we used buf_physaddr.

> +#if (RTE_BYTE_ORDER == RTE_BIG_ENDIAN) && (__WORDSIZE == 32)
> +		vq->offset += 4;
> +#endif
> +	}
> +#endif
>  	/*
>  	 * Set guest physical address of the virtqueue
>  	 * in VIRTIO_PCI_QUEUE_PFN config register of device
> @@ -491,8 +509,10 @@ virtio_dev_close(struct rte_eth_dev *dev)
>  	PMD_INIT_LOG(DEBUG, "virtio_dev_close");
> 
>  	/* reset the NIC */
> -	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
> -		vtpci_irq_config(hw, VIRTIO_MSI_NO_VECTOR);
> +	if (dev->dev_type == RTE_ETH_DEV_PCI) {
> +		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
> +			vtpci_irq_config(hw, VIRTIO_MSI_NO_VECTOR);
> +	}
>  	vtpci_reset(hw);
>  	hw->started = 0;
>  	virtio_dev_free_mbufs(dev);
> @@ -1233,8 +1253,9 @@ virtio_interrupt_handler(__rte_unused struct rte_intr_handle *handle,
>  	isr = vtpci_isr(hw);
>  	PMD_DRV_LOG(INFO, "interrupt status = %#x", isr);
> 
> -	if (rte_intr_enable(&dev->pci_dev->intr_handle) < 0)
> -		PMD_DRV_LOG(ERR, "interrupt enable failed");
> +	if (dev->dev_type == RTE_ETH_DEV_PCI)
> +		if (rte_intr_enable(&dev->pci_dev->intr_handle) < 0)
> +			PMD_DRV_LOG(ERR, "interrupt enable failed");
> 
>  	if (isr & VIRTIO_PCI_ISR_CONFIG) {
>  		if (virtio_dev_link_update(dev, 0) == 0)
> @@ -1287,11 +1308,18 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
> 
>  	pci_dev = eth_dev->pci_dev;
> 
> -	if (virtio_resource_init(pci_dev) < 0)
> -		return -1;
> -
> -	hw->use_msix = virtio_has_msix(&pci_dev->addr);
> -	hw->io_base = (uint32_t)(uintptr_t)pci_dev->mem_resource[0].addr;
> +	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
> +		if (virtio_resource_init(pci_dev) < 0)
> +			return -1;
> +		hw->use_msix = virtio_has_msix(&pci_dev->addr);
> +		hw->io_base = (uint32_t)(uintptr_t)pci_dev->mem_resource[0].addr;
> +	}
> +#ifdef RTE_VIRTIO_VDEV
> +	else {
> +		hw->use_msix = 0;
> +		hw->io_base = VIRTIO_VDEV_IO_BASE;
> +	}
> +#endif
> 
>  	/* Reset the device although not necessary at startup */
>  	vtpci_reset(hw);
> @@ -1304,10 +1332,12 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
>  	virtio_negotiate_features(hw);
> 
>  	/* If host does not support status then disable LSC */
> -	if (!vtpci_with_feature(hw, VIRTIO_NET_F_STATUS))
> -		pci_dev->driver->drv_flags &= ~RTE_PCI_DRV_INTR_LSC;
> +	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
> +		if (!vtpci_with_feature(hw, VIRTIO_NET_F_STATUS))
> +			pci_dev->driver->drv_flags &= ~RTE_PCI_DRV_INTR_LSC;
> 
> -	rte_eth_copy_pci_info(eth_dev, pci_dev);
> +		rte_eth_copy_pci_info(eth_dev, pci_dev);
> +	}
> 
>  	rx_func_get(eth_dev);
> 
> @@ -1383,15 +1413,16 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
> 
>  	PMD_INIT_LOG(DEBUG, "hw->max_rx_queues=%d   hw->max_tx_queues=%d",
>  			hw->max_rx_queues, hw->max_tx_queues);
> -	PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
> -			eth_dev->data->port_id, pci_dev->id.vendor_id,
> -			pci_dev->id.device_id);
> -
> -	/* Setup interrupt callback  */
> -	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
> -		rte_intr_callback_register(&pci_dev->intr_handle,
> -				   virtio_interrupt_handler, eth_dev);
> -
> +	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
> +		PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
> +				eth_dev->data->port_id, pci_dev->id.vendor_id,
> +				pci_dev->id.device_id);
> +
> +		/* Setup interrupt callback  */
> +		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
> +			rte_intr_callback_register(&pci_dev->intr_handle,
> +					virtio_interrupt_handler, eth_dev);
> +	}
>  	virtio_dev_cq_start(eth_dev);
> 
>  	return 0;
> @@ -1424,10 +1455,12 @@ eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev)
>  	eth_dev->data->mac_addrs = NULL;
> 
>  	/* reset interrupt callback  */
> -	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
> -		rte_intr_callback_unregister(&pci_dev->intr_handle,
> -						virtio_interrupt_handler,
> -						eth_dev);
> +	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
> +		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
> +			rte_intr_callback_unregister(&pci_dev->intr_handle,
> +					virtio_interrupt_handler,
> +					eth_dev);
> +	}
> 
>  	PMD_INIT_LOG(DEBUG, "dev_uninit completed");
> 
> @@ -1491,11 +1524,13 @@ virtio_dev_configure(struct rte_eth_dev *dev)
>  		return -ENOTSUP;
>  	}
> 
> -	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
> -		if (vtpci_irq_config(hw, 0) == VIRTIO_MSI_NO_VECTOR) {
> -			PMD_DRV_LOG(ERR, "failed to set config vector");
> -			return -EBUSY;
> -		}
> +	if (dev->dev_type == RTE_ETH_DEV_PCI) {
> +		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
> +			if (vtpci_irq_config(hw, 0) == VIRTIO_MSI_NO_VECTOR) {
> +				PMD_DRV_LOG(ERR, "failed to set config vector");
> +				return -EBUSY;
> +			}
> +	}
> 
>  	return 0;
>  }
> @@ -1689,3 +1724,162 @@ static struct rte_driver rte_virtio_driver = {
>  };
> 
>  PMD_REGISTER_DRIVER(rte_virtio_driver);
> +
> +#ifdef RTE_VIRTIO_VDEV
> +
> +static const char *valid_args[] = {
> +#define ETH_CVIO_ARG_RX_NUM		"rx"
> +	ETH_CVIO_ARG_RX_NUM,
> +#define ETH_CVIO_ARG_TX_NUM		"tx"
> +	ETH_CVIO_ARG_TX_NUM,
> +#define ETH_CVIO_ARG_CQ_NUM		"cq"
> +	ETH_CVIO_ARG_CQ_NUM,
> +#define ETH_CVIO_ARG_MAC		"mac"
> +	ETH_CVIO_ARG_MAC,
> +#define ETH_CVIO_ARG_PATH		"path"
> +	ETH_CVIO_ARG_PATH,
> +#define ETH_CVIO_ARG_QUEUE_SIZE	"queue_num"
> +	ETH_CVIO_ARG_QUEUE_SIZE,
> +	NULL
> +};
> +
> +static int
> +get_string_arg(const char *key __rte_unused,
> +		const char *value, void *extra_args)
> +{
> +	if ((value == NULL) || (extra_args == NULL))
> +		return -EINVAL;
> +
> +	strcpy(extra_args, value);
> +
> +	return 0;
> +}
> +
> +static int
> +get_integer_arg(const char *key __rte_unused,
> +		const char *value, void *extra_args)
> +{
> +	uint64_t *p_u64 = extra_args;
> +
> +	if ((value == NULL) || (extra_args == NULL))
> +		return -EINVAL;
> +
> +	*p_u64 = (uint64_t)strtoull(value, NULL, 0);
> +
> +	return 0;
> +}
> +
> +static struct rte_eth_dev *
> +cvio_eth_dev_alloc(const char *name)
> +{
> +	struct rte_eth_dev *eth_dev;
> +	struct rte_eth_dev_data *data;
> +	struct virtio_hw *hw;
> +
> +	eth_dev = rte_eth_dev_allocate(name, RTE_ETH_DEV_VIRTUAL);
> +	if (eth_dev == NULL)
> +		rte_panic("cannot alloc rte_eth_dev\n");
> +
> +	data = eth_dev->data;
> +
> +	hw = rte_zmalloc(NULL, sizeof(*hw), 0);
> +	if (!hw)
> +		rte_panic("malloc virtio_hw failed\n");
> +
> +	data->dev_private = hw;
> +	data->numa_node = SOCKET_ID_ANY;
> +	eth_dev->pci_dev = NULL;
> +	/* will be used in virtio_dev_info_get() */
> +	eth_dev->driver = &rte_virtio_pmd;
> +	/* TODO: eth_dev->link_intr_cbs */
> +	return eth_dev;
> +}
> +
> +#define CVIO_DEF_CQ_EN		0
> +#define CVIO_DEF_Q_NUM		1
> +#define CVIO_DEF_Q_SZ		256
> +/*
> + * Dev initialization routine. Invoked once for each virtio vdev at
> + * EAL init time, see rte_eal_dev_init().
> + * Returns 0 on success.
> + */
> +static int
> +rte_cvio_pmd_devinit(const char *name, const char *params)
> +{
> +	struct rte_kvargs *kvlist = NULL;
> +	struct rte_eth_dev *eth_dev = NULL;
> +	uint64_t nb_rx = CVIO_DEF_Q_NUM;
> +	uint64_t nb_tx = CVIO_DEF_Q_NUM;
> +	uint64_t nb_cq = CVIO_DEF_CQ_EN;
> +	uint64_t queue_num = CVIO_DEF_Q_SZ;
> +	char sock_path[256];
> +	char mac_addr[32];
> +	int flag_mac = 0;
> +
> +	if (params == NULL || params[0] == '\0')
> +		rte_panic("arg %s is mandatory for eth_cvio\n",
> +				ETH_CVIO_ARG_QUEUE_SIZE);
> +
> +	kvlist = rte_kvargs_parse(params, valid_args);
> +	if (!kvlist)
> +		rte_panic("error when parsing param\n");
> +
> +	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_PATH) == 1)
> +		rte_kvargs_process(kvlist, ETH_CVIO_ARG_PATH,
> +				&get_string_arg, sock_path);
> +	else
> +		rte_panic("arg %s is mandatory for eth_cvio\n",
> +				ETH_CVIO_ARG_QUEUE_SIZE);
> +
> +	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_MAC) == 1) {
> +		rte_kvargs_process(kvlist, ETH_CVIO_ARG_MAC,
> +				&get_string_arg, mac_addr);
> +		flag_mac = 1;
> +	}
> +
> +	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_QUEUE_SIZE) == 1)
> +		rte_kvargs_process(kvlist, ETH_CVIO_ARG_QUEUE_SIZE,
> +			&get_integer_arg, &queue_num);
> +
> +	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_RX_NUM) == 1)
> +		rte_kvargs_process(kvlist, ETH_CVIO_ARG_RX_NUM,
> +			&get_integer_arg, &nb_rx);
> +
> +	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_TX_NUM) == 1)
> +		rte_kvargs_process(kvlist, ETH_CVIO_ARG_TX_NUM,
> +			&get_integer_arg, &nb_tx);
> +
> +	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_CQ_NUM) == 1)
> +		rte_kvargs_process(kvlist, ETH_CVIO_ARG_CQ_NUM,
> +			&get_integer_arg, &nb_cq);
> +
> +	eth_dev = cvio_eth_dev_alloc(name);
> +
> +	virtio_vdev_init(eth_dev->data, sock_path,
> +			nb_rx, nb_tx, nb_cq, queue_num,
> +			(flag_mac) ? mac_addr : NULL);
> +
> +	/* originally, this will be called in rte_eal_pci_probe() */
> +	eth_virtio_dev_init(eth_dev);
> +
> +	return 0;
> +}
> +
> +static int
> +rte_cvio_pmd_devuninit(const char *name)
> +{
> +	/* TODO: if it's last one, memory init, free memory */
> +	rte_panic("%s: %s", __func__, name);
> +	return 0;
> +}
> +
> +static struct rte_driver rte_cvio_driver = {
> +	.name   = "eth_cvio",
> +	.type   = PMD_VDEV,
> +	.init   = rte_cvio_pmd_devinit,
> +	.uninit = rte_cvio_pmd_devuninit,
> +};
> +
> +PMD_REGISTER_DRIVER(rte_cvio_driver);
> +
> +#endif
> diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
> index 9e1ecb3..90890b4 100644
> --- a/drivers/net/virtio/virtio_ethdev.h
> +++ b/drivers/net/virtio/virtio_ethdev.h
> @@ -126,4 +126,5 @@ uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf
> **tx_pkts,
>  void virtio_vdev_init(struct rte_eth_dev_data *data, const char *path,
>  		int nb_rx, int nb_tx, int nb_cq, int queue_num, char *mac);
>  #endif
> +
>  #endif /* _VIRTIO_ETHDEV_H_ */
> diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
> index af05ae2..d79bd05 100644
> --- a/drivers/net/virtio/virtio_pci.h
> +++ b/drivers/net/virtio/virtio_pci.h
> @@ -249,31 +249,31 @@ uint32_t virtio_ioport_read(struct virtio_hw *, uint64_t);
>  void virtio_ioport_write(struct virtio_hw *, uint64_t, uint32_t);
> 
>  #define VIRTIO_READ_REG_1(hw, reg) \
> -	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>  	inb((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> -	:virtio_ioport_read(hw, reg)
> +	:virtio_ioport_read(hw, reg))
>  #define VIRTIO_WRITE_REG_1(hw, reg, value) \
> -	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>  	outb_p((unsigned char)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> -	:virtio_ioport_write(hw, reg, value)
> +	:virtio_ioport_write(hw, reg, value))
> 
>  #define VIRTIO_READ_REG_2(hw, reg) \
> -	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>  	inw((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> -	:virtio_ioport_read(hw, reg)
> +	:virtio_ioport_read(hw, reg))
>  #define VIRTIO_WRITE_REG_2(hw, reg, value) \
> -	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>  	outw_p((unsigned short)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> -	:virtio_ioport_write(hw, reg, value)
> +	:virtio_ioport_write(hw, reg, value))
> 
>  #define VIRTIO_READ_REG_4(hw, reg) \
> -	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>  	inl((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> -	:virtio_ioport_read(hw, reg)
> +	:virtio_ioport_read(hw, reg))
>  #define VIRTIO_WRITE_REG_4(hw, reg, value) \
> -	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
> +	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>  	outl_p((unsigned int)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
> -	:virtio_ioport_write(hw, reg, value)
> +	:virtio_ioport_write(hw, reg, value))

 These bracket fixups should be squashed into #3

> 
>  #else /* RTE_VIRTIO_VDEV */
> 
> diff --git a/drivers/net/virtio/virtio_rxtx.c b/drivers/net/virtio/virtio_rxtx.c
> index 74b39ef..dd07ba7 100644
> --- a/drivers/net/virtio/virtio_rxtx.c
> +++ b/drivers/net/virtio/virtio_rxtx.c
> @@ -191,8 +191,7 @@ virtqueue_enqueue_recv_refill(struct virtqueue *vq, struct rte_mbuf
> *cookie)
> 
>  	start_dp = vq->vq_ring.desc;
>  	start_dp[idx].addr =
> -		(uint64_t)(cookie->buf_physaddr + RTE_PKTMBUF_HEADROOM
> -		- hw->vtnet_hdr_size);
> +		RTE_MBUF_DATA_DMA_ADDR(cookie, vq->offset) - hw->vtnet_hdr_size;
>  	start_dp[idx].len =
>  		cookie->buf_len - RTE_PKTMBUF_HEADROOM + hw->vtnet_hdr_size;
>  	start_dp[idx].flags =  VRING_DESC_F_WRITE;
> @@ -237,7 +236,7 @@ virtqueue_enqueue_xmit(struct virtqueue *txvq, struct rte_mbuf *cookie)
> 
>  	for (; ((seg_num > 0) && (cookie != NULL)); seg_num--) {
>  		idx = start_dp[idx].next;
> -		start_dp[idx].addr  = RTE_MBUF_DATA_DMA_ADDR(cookie);
> +		start_dp[idx].addr  = RTE_MBUF_DATA_DMA_ADDR(cookie, txvq->offset);
>  		start_dp[idx].len   = cookie->data_len;
>  		start_dp[idx].flags = VRING_DESC_F_NEXT;
>  		cookie = cookie->next;
> @@ -343,7 +342,7 @@ virtio_dev_vring_start(struct virtqueue *vq, int queue_type)
>  		VIRTIO_WRITE_REG_2(vq->hw, VIRTIO_PCI_QUEUE_SEL,
>  			vq->vq_queue_index);
>  		VIRTIO_WRITE_REG_4(vq->hw, VIRTIO_PCI_QUEUE_PFN,
> -			vq->mz->phys_addr >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
> +			vq->vq_ring_mem >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
>  	} else if (queue_type == VTNET_TQ) {
>  		if (use_simple_rxtx) {
>  			int mid_idx  = vq->vq_nentries >> 1;
> @@ -366,12 +365,12 @@ virtio_dev_vring_start(struct virtqueue *vq, int queue_type)
>  		VIRTIO_WRITE_REG_2(vq->hw, VIRTIO_PCI_QUEUE_SEL,
>  			vq->vq_queue_index);
>  		VIRTIO_WRITE_REG_4(vq->hw, VIRTIO_PCI_QUEUE_PFN,
> -			vq->mz->phys_addr >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
> +			vq->vq_ring_mem >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
>  	} else {
>  		VIRTIO_WRITE_REG_2(vq->hw, VIRTIO_PCI_QUEUE_SEL,
>  			vq->vq_queue_index);
>  		VIRTIO_WRITE_REG_4(vq->hw, VIRTIO_PCI_QUEUE_PFN,
> -			vq->mz->phys_addr >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
> +			vq->vq_ring_mem >> VIRTIO_PCI_QUEUE_ADDR_SHIFT);
>  	}
>  }
> 
> diff --git a/drivers/net/virtio/virtio_rxtx_simple.c
> b/drivers/net/virtio/virtio_rxtx_simple.c
> index ff3c11a..3a14a4e 100644
> --- a/drivers/net/virtio/virtio_rxtx_simple.c
> +++ b/drivers/net/virtio/virtio_rxtx_simple.c
> @@ -80,8 +80,8 @@ virtqueue_enqueue_recv_refill_simple(struct virtqueue *vq,
>  	vq->sw_ring[desc_idx] = cookie;
> 
>  	start_dp = vq->vq_ring.desc;
> -	start_dp[desc_idx].addr = (uint64_t)((uintptr_t)cookie->buf_physaddr +
> -		RTE_PKTMBUF_HEADROOM - sizeof(struct virtio_net_hdr));
> +	start_dp[desc_idx].addr = RTE_MBUF_DATA_DMA_ADDR(cookie, vq->offset)
> +		- sizeof(struct virtio_net_hdr);
>  	start_dp[desc_idx].len = cookie->buf_len -
>  		RTE_PKTMBUF_HEADROOM + sizeof(struct virtio_net_hdr);
> 
> @@ -118,9 +118,8 @@ virtio_rxq_rearm_vec(struct virtqueue *rxvq)
>  		p = (uintptr_t)&sw_ring[i]->rearm_data;
>  		*(uint64_t *)p = rxvq->mbuf_initializer;
> 
> -		start_dp[i].addr =
> -			(uint64_t)((uintptr_t)sw_ring[i]->buf_physaddr +
> -			RTE_PKTMBUF_HEADROOM - sizeof(struct virtio_net_hdr));
> +		start_dp[i].addr = RTE_MBUF_DATA_DMA_ADDR(sw_ring[i], rxvq->offset)
> +			- sizeof(struct virtio_net_hdr);
>  		start_dp[i].len = sw_ring[i]->buf_len -
>  			RTE_PKTMBUF_HEADROOM + sizeof(struct virtio_net_hdr);
>  	}
> @@ -366,7 +365,7 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
>  			txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
>  		for (i = 0; i < nb_tail; i++) {
>  			start_dp[desc_idx].addr =
> -				RTE_MBUF_DATA_DMA_ADDR(*tx_pkts);
> +				RTE_MBUF_DATA_DMA_ADDR(*tx_pkts, txvq->offset);
>  			start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
>  			tx_pkts++;
>  			desc_idx++;
> @@ -377,7 +376,8 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
>  	for (i = 0; i < nb_commit; i++)
>  		txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
>  	for (i = 0; i < nb_commit; i++) {
> -		start_dp[desc_idx].addr = RTE_MBUF_DATA_DMA_ADDR(*tx_pkts);
> +		start_dp[desc_idx].addr = RTE_MBUF_DATA_DMA_ADDR(*tx_pkts,
> +				txvq->offset);
>  		start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
>  		tx_pkts++;
>  		desc_idx++;
> diff --git a/drivers/net/virtio/virtqueue.h b/drivers/net/virtio/virtqueue.h
> index 61b3137..dc0b656 100644
> --- a/drivers/net/virtio/virtqueue.h
> +++ b/drivers/net/virtio/virtqueue.h
> @@ -66,8 +66,14 @@ struct rte_mbuf;
> 
>  #define VIRTQUEUE_MAX_NAME_SZ 32
> 
> -#define RTE_MBUF_DATA_DMA_ADDR(mb) \
> +#ifdef RTE_VIRTIO_VDEV
> +#define RTE_MBUF_DATA_DMA_ADDR(mb, offset) \
> +	(uint64_t)((uintptr_t)(*(void **)((uintptr_t)mb + offset)) \
> +			+ (mb)->data_off)
> +#else
> +#define RTE_MBUF_DATA_DMA_ADDR(mb, offset) \
>  	(uint64_t) ((mb)->buf_physaddr + (mb)->data_off)
> +#endif /* RTE_VIRTIO_VDEV */
> 
>  #define VTNET_SQ_RQ_QUEUE_IDX 0
>  #define VTNET_SQ_TQ_QUEUE_IDX 1
> @@ -167,7 +173,8 @@ struct virtqueue {
> 
>  	void        *vq_ring_virt_mem;    /**< linear address of vring*/
>  	unsigned int vq_ring_size;
> -	phys_addr_t vq_ring_mem;          /**< physical address of vring */
> +	phys_addr_t vq_ring_mem;          /**< phys address of vring for pci dev,
> +										virt addr of vring for vdev */
> 
>  	struct vring vq_ring;    /**< vring keeping desc, used and avail */
>  	uint16_t    vq_free_cnt; /**< num of desc available */
> @@ -186,8 +193,10 @@ struct virtqueue {
>  	 */
>  	uint16_t vq_used_cons_idx;
>  	uint16_t vq_avail_idx;
> +	uint16_t offset; /**< relative offset to obtain addr in mbuf */
>  	uint64_t mbuf_initializer; /**< value to init mbufs. */
>  	phys_addr_t virtio_net_hdr_mem; /**< hdr for each xmit packet */
> +	void        *virtio_net_hdr_vaddr;    /**< linear address of vring*/
> 
>  	struct rte_mbuf **sw_ring; /**< RX software ring. */
>  	/* dummy mbuf, for wraparound when processing RX ring. */
> --
> 2.1.4

Kind regards,
Pavel Fedin
Senior Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 4/4] virtio/vdev: add a new vdev named eth_cvio
  2016-01-12  7:45     ` Pavel Fedin
@ 2016-01-12  7:59       ` Yuanhan Liu
  2016-01-12  8:39       ` Tan, Jianfeng
  1 sibling, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-01-12  7:59 UTC (permalink / raw)
  To: Pavel Fedin; +Cc: nakajima.yoshihiro, mst, dev, ann.zhuangyanying

On Tue, Jan 12, 2016 at 10:45:59AM +0300, Pavel Fedin wrote:
>  Hello!
> 
>  See inline

Hi,

Please strip unrelated context, so that people could reach to your
comments as quick as possible, otherwise, people could easily get
lost from the long patch.

> 
> > -----Original Message-----
> > From: Jianfeng Tan [mailto:jianfeng.tan@intel.com]
> > +	struct rte_mbuf *m = NULL;
> > +	if (dev->dev_type == RTE_ETH_DEV_PCI)
> > +		vq->offset = (uintptr_t)&m->buf_addr;
> > +#ifdef RTE_VIRTIO_VDEV
> > +	else {
> > +		vq->offset = (uintptr_t)&m->buf_physaddr;
> 
>  Not sure, but shouldn't these be swapped? Originally, for PCI devices, we used buf_physaddr.

And this reply just servers as an example only :)

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12  7:38       ` Pavel Fedin
@ 2016-01-12  8:14         ` Rich Lane
  2016-01-12  8:39           ` Pavel Fedin
  0 siblings, 1 reply; 196+ messages in thread
From: Rich Lane @ 2016-01-12  8:14 UTC (permalink / raw)
  To: Pavel Fedin
  Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

See my reply to "mem: add API to obstain memory-backed file info" for a
workaround. With fixes for that and the TUNSETVNETHDRSZ issue I was able to
get traffic running over vhost-user.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12  8:14         ` Rich Lane
@ 2016-01-12  8:39           ` Pavel Fedin
  2016-01-12  8:51             ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Pavel Fedin @ 2016-01-12  8:39 UTC (permalink / raw)
  To: 'Rich Lane'
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

 Hello!

> See my reply to "mem: add API to obstain memory-backed file info" for a workaround. With fixes for that and the TUNSETVNETHDRSZ issue I was able to
> get traffic running over vhost-user.

 With ovs or test apps? I still have problems with ovs after this. Packets go from host to container, but not back. Here is host-side log (i added also GPA display in order to debug the problem you pointed at):
--- cut ---
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: new virtio connection is 38
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: new device, handle is 0
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_OWNER
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_GET_FEATURES
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_FEATURES
Jan 12 11:23:32 nfv_test_x86_64 kernel: device ovs-netdev entered promiscuous mode
Jan 12 11:23:32 nfv_test_x86_64 kernel: device ovs0 entered promiscuous mode
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_MEM_TABLE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: mapped region 0 fd:39 to:0x7f0ddea00000 sz:0x20000000 off:0x0 GPA:0x7f7159000000 align:0x200000
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring call idx:0 file:49
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring kick idx:0 file:50
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: virtio is not ready for processing.
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring call idx:1 file:51
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring kick idx:1 file:52
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: virtio is now ready for processing.
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_FEATURES
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_MEM_TABLE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: mapped region 0 fd:53 to:0x7f0ddea00000 sz:0x20000000 off:0x0 GPA:0x7f7159000000 align:0x200000
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring call idx:0 file:39
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring kick idx:0 file:49
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: virtio is now ready for processing.
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring call idx:1 file:50
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring kick idx:1 file:51
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: virtio is now ready for processing.
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring call idx:3 file:52
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring kick idx:3 file:56
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: virtio is not ready for processing.
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring call idx:5 file:57
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring kick idx:5 file:58
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: virtio is not ready for processing.
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring call idx:7 file:59
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring kick idx:7 file:60
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: virtio is not ready for processing.
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring call idx:9 file:61
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring kick idx:9 file:62
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: virtio is not ready for processing.
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring call idx:11 file:63
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring kick idx:11 file:64
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: virtio is not ready for processing.
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring call idx:13 file:65
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring kick idx:13 file:66
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: virtio is not ready for processing.
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring call idx:15 file:67
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring kick idx:15 file:68
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: virtio is not ready for processing.
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring call idx:17 file:69
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: vring kick idx:17 file:70
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: virtio is not ready for processing.
Jan 12 11:23:32 nfv_test_x86_64 ovs-vswitchd[3461]: VHOST_CONFIG: read message VHOST_USER_SET_FEATURES
--- cut ---

 Note that during multiqueue setup host state reverts back from "now ready for processing" to "not ready for processing". I guess this is the reason for the problem.

Kind regards,
Pavel Fedin
Expert Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 4/4] virtio/vdev: add a new vdev named eth_cvio
  2016-01-12  7:45     ` Pavel Fedin
  2016-01-12  7:59       ` Yuanhan Liu
@ 2016-01-12  8:39       ` Tan, Jianfeng
  2016-01-12  9:15         ` Tan, Jianfeng
  1 sibling, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-12  8:39 UTC (permalink / raw)
  To: Pavel Fedin, dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

Hi Fedin,

On 1/12/2016 3:45 PM, Pavel Fedin wrote:
>   Hello!
>
>   See inline
>
>> ...
>>   	}
>>
>> +	struct rte_mbuf *m = NULL;
>> +	if (dev->dev_type == RTE_ETH_DEV_PCI)
>> +		vq->offset = (uintptr_t)&m->buf_addr;
>> +#ifdef RTE_VIRTIO_VDEV
>> +	else {
>> +		vq->offset = (uintptr_t)&m->buf_physaddr;
>   Not sure, but shouldn't these be swapped? Originally, for PCI devices, we used buf_physaddr.

Oops, seems that you are right. I'm trying to figure out why I can rx/tx 
pkts using the wrong version.

>>   #define VIRTIO_READ_REG_1(hw, reg) \
>> -	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>> +	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>>   	inb((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
>> -	:virtio_ioport_read(hw, reg)
>> +	:virtio_ioport_read(hw, reg))
>>   #define VIRTIO_WRITE_REG_1(hw, reg, value) \
>> -	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>> +	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>>   	outb_p((unsigned char)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
>> -	:virtio_ioport_write(hw, reg, value)
>> +	:virtio_ioport_write(hw, reg, value))
>>
>>   #define VIRTIO_READ_REG_2(hw, reg) \
>> -	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>> +	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>>   	inw((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
>> -	:virtio_ioport_read(hw, reg)
>> +	:virtio_ioport_read(hw, reg))
>>   #define VIRTIO_WRITE_REG_2(hw, reg, value) \
>> -	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>> +	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>>   	outw_p((unsigned short)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
>> -	:virtio_ioport_write(hw, reg, value)
>> +	:virtio_ioport_write(hw, reg, value))
>>
>>   #define VIRTIO_READ_REG_4(hw, reg) \
>> -	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>> +	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>>   	inl((VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
>> -	:virtio_ioport_read(hw, reg)
>> +	:virtio_ioport_read(hw, reg))
>>   #define VIRTIO_WRITE_REG_4(hw, reg, value) \
>> -	(hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>> +	((hw->io_base != VIRTIO_VDEV_IO_BASE) ? \
>>   	outl_p((unsigned int)(value), (VIRTIO_PCI_REG_ADDR((hw), (reg)))) \
>> -	:virtio_ioport_write(hw, reg, value)
>> +	:virtio_ioport_write(hw, reg, value))
>   These bracket fixups should be squashed into #3
>

I'll rewrite this into function pointers according to Yuanhan's patch 
for virtio 1.0.

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12  8:39           ` Pavel Fedin
@ 2016-01-12  8:51             ` Tan, Jianfeng
  2016-01-12 10:48               ` Pavel Fedin
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-12  8:51 UTC (permalink / raw)
  To: Pavel Fedin, 'Rich Lane'
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

Hi Fedin,

On 1/12/2016 4:39 PM, Pavel Fedin wrote:
>   Hello!
>
>> See my reply to "mem: add API to obstain memory-backed file info" for a workaround. With fixes for that and the TUNSETVNETHDRSZ issue I was able to
>> get traffic running over vhost-user.
>   With ovs or test apps? I still have problems with ovs after this. Packets go from host to container, but not back. Here is host-side log (i added also GPA display in order to debug the problem you pointed at):
> --- cut ---
> ...
> --- cut ---
>
>   Note that during multiqueue setup host state reverts back from "now ready for processing" to "not ready for processing". I guess this is the reason for the problem.

Your guess makes sense because current implementation does not support 
multi-queues.

 From you log, only 0 and 1 are "ready for processing"; others are "not 
ready for processing".

Thanks,
Jianfeng


> Kind regards,
> Pavel Fedin
> Expert Engineer
> Samsung Electronics Research center Russia
>
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-11 20:26     ` Rich Lane
@ 2016-01-12  9:12       ` Tan, Jianfeng
  2016-01-12 10:04         ` Pavel Fedin
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-12  9:12 UTC (permalink / raw)
  To: Rich Lane; +Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying


Hi!

On 1/12/2016 4:26 AM, Rich Lane wrote:
> On Sun, Jan 10, 2016 at 3:43 AM, Jianfeng Tan <jianfeng.tan@intel.com 
> <mailto:jianfeng.tan@intel.com>> wrote:
>
>     @@ -1157,6 +1180,20 @@ rte_eal_hugepage_init(void)
>                     mcfg->memseg[0].len = internal_config.memory;
>                     mcfg->memseg[0].socket_id = socket_id;
>
>     +               hugepage =
>     create_shared_memory(eal_hugepage_info_path(),
>     +                               sizeof(struct hugepage_file));
>     +               hugepage->orig_va = addr;
>     +               hugepage->final_va = addr;
>     +               hugepage->physaddr = rte_mem_virt2phy(addr);
>     +               hugepage->size = pagesize;
>
>
> Should this be "hugepage->size = internal_config.memory"? Otherwise 
> the vhost-user
> memtable entry has a size of only 2MB.

I don't think so. See the definition:

  47 struct hugepage_file {
  48         void *orig_va;      /**< virtual addr of first mmap() */
  49         void *final_va;     /**< virtual addr of 2nd mmap() */
  50         uint64_t physaddr;  /**< physical addr */
  51         size_t size;        /**< the page size */
  52         int socket_id;      /**< NUMA socket ID */
  53         int file_id;        /**< the '%d' in HUGEFILE_FMT */
  54         int memseg_id;      /**< the memory segment to which page 
belongs */
  55 #ifdef RTE_EAL_SINGLE_FILE_SEGMENTS
  56         int repeated;           /**< number of times the page size 
is repeated */
  57 #endif
  58         char filepath[MAX_HUGEPAGE_PATH]; /**< path to backing file 
on filesystem */
  59 };

size stands for the page size instead of total size.

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 4/4] virtio/vdev: add a new vdev named eth_cvio
  2016-01-12  8:39       ` Tan, Jianfeng
@ 2016-01-12  9:15         ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-12  9:15 UTC (permalink / raw)
  To: Pavel Fedin, dev; +Cc: nakajima.yoshihiro, ann.zhuangyanying, mst


Hi Fedin,

On 1/12/2016 4:39 PM, Tan, Jianfeng wrote:
> Hi Fedin,
>
> On 1/12/2016 3:45 PM, Pavel Fedin wrote:
>>   Hello!
>>
>>   See inline
>>
>>> ...
>>>       }
>>>
>>> +    struct rte_mbuf *m = NULL;
>>> +    if (dev->dev_type == RTE_ETH_DEV_PCI)
>>> +        vq->offset = (uintptr_t)&m->buf_addr;
>>> +#ifdef RTE_VIRTIO_VDEV
>>> +    else {
>>> +        vq->offset = (uintptr_t)&m->buf_physaddr;
>>   Not sure, but shouldn't these be swapped? Originally, for PCI 
>> devices, we used buf_physaddr.
>
> Oops, seems that you are right. I'm trying to figure out why I can 
> rx/tx pkts using the wrong version.
>

I figure out why. When we run apps without root privilege, mempool's 
elt_pa is assigned the same of elt_va_start. So it happens to be right 
value to translate addresses. But it's definitely a bug. Thanks for 
pointing this out.

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12  9:12       ` Tan, Jianfeng
@ 2016-01-12 10:04         ` Pavel Fedin
  2016-01-12 10:48           ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Pavel Fedin @ 2016-01-12 10:04 UTC (permalink / raw)
  To: 'Tan, Jianfeng', 'Rich Lane'
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

 Hello!

>> Should this be "hugepage->size = internal_config.memory"? Otherwise the vhost-user
>> memtable entry has a size of only 2MB.

> I don't think so. See the definition:

> 47 struct hugepage_file {
> 48         void *orig_va;      /**< virtual addr of first mmap() */
> 49         void *final_va;     /**< virtual addr of 2nd mmap() */
> 50         uint64_t physaddr;  /**< physical addr */
> 51         size_t size;        /**< the page size */
> 52         int socket_id;      /**< NUMA socket ID */
> 53         int file_id;        /**< the '%d' in HUGEFILE_FMT */
> 54         int memseg_id;      /**< the memory segment to which page belongs */                                                                                            
> 55 #ifdef RTE_EAL_SINGLE_FILE_SEGMENTS    
> 56         int repeated;           /**< number of times the page size is repeated */                                                                                       
> 57 #endif                    
> 58         char filepath[MAX_HUGEPAGE_PATH]; /**< path to backing file on filesystem */                                                                                    
> 59 };

> size stands for the page size instead of total size.

 But in this case host gets this page size for total region size, therefore qva_to_vva() fails.
 I haven't worked with hugepages, but i guess that with real hugepages we get one file per page, therefore page size == mapping size. With newly introduced --single-file we now have something that pretends to be a single "uber-huge-page", so we need to specify total size of the mapping here.

 BTW, i'm still unhappy about ABI breakage here. I think we could easily add --shared-mem option, which would simply change mapping mode to SHARED. So, we could use it with both hugepages (default) and plain mmap (with --no-hugepages).

Kind regards,
Pavel Fedin
Expert Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12 10:04         ` Pavel Fedin
@ 2016-01-12 10:48           ` Tan, Jianfeng
  2016-01-12 11:00             ` Pavel Fedin
  2016-01-12 11:22             ` Pavel Fedin
  0 siblings, 2 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-12 10:48 UTC (permalink / raw)
  To: Pavel Fedin, 'Rich Lane'
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

Hello!

>   But in this case host gets this page size for total region size, therefore qva_to_vva() fails.
>   I haven't worked with hugepages, but i guess that with real hugepages we get one file per page, therefore page size == mapping size. With newly introduced --single-file we now have something that pretends to be a single "uber-huge-page", so we need to specify total size of the mapping here.

Oh I get it and recognize the problem here. The actual problem lies in 
the API rte_eal_get_backfile_info().
backfiles[i].size = hugepage_files[i].size;
Should use statfs or hugepage_files[i].size * hugepage_files[i].repeated 
to calculate the total size.

>
>   BTW, i'm still unhappy about ABI breakage here. I think we could easily add --shared-mem option, which would simply change mapping mode to SHARED. So, we could use it with both hugepages (default) and plain mmap (with --no-hugepages).

You mean, use "--no-hugepages --shared-mem" together, right?
That makes sense to me.

Thanks,
Jianfeng

>
> Kind regards,
> Pavel Fedin
> Expert Engineer
> Samsung Electronics Research center Russia
>
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12  8:51             ` Tan, Jianfeng
@ 2016-01-12 10:48               ` Pavel Fedin
  2016-01-12 14:45                 ` Amit Tomer
  0 siblings, 1 reply; 196+ messages in thread
From: Pavel Fedin @ 2016-01-12 10:48 UTC (permalink / raw)
  To: 'Tan, Jianfeng', 'Rich Lane'
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

 Hello!

> Your guess makes sense because current implementation does not support
> multi-queues.
> 
>  From you log, only 0 and 1 are "ready for processing"; others are "not
> ready for processing".

 Yes, and if study it even more carefully, we see that we initialize all tx queues but only a single rx queue (#0).
 After some more code browsing and comparing the two patchsets i figured out that the problem is caused by inappropriate VIRTIO_NET_F_CTRL_VQ flag. In your RFC you used different capability set, while in v1 you seem to have forgotten about this.
 I suggest to temporarily move hw->guest_features assignment out of virtio_negotiate_features() into the caller, where we have eth_dev->dev_type, and can choose the right set depending on it.

 With all mentioned fixes i've got the ping running.
 Tested-by: Pavel Fedin <p.fedin@samsung.com>

Kind regards,
Pavel Fedin
Expert Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12 10:48           ` Tan, Jianfeng
@ 2016-01-12 11:00             ` Pavel Fedin
  2016-01-12 11:07               ` Sergio Gonzalez Monroy
  2016-01-12 11:22             ` Pavel Fedin
  1 sibling, 1 reply; 196+ messages in thread
From: Pavel Fedin @ 2016-01-12 11:00 UTC (permalink / raw)
  To: 'Tan, Jianfeng', 'Rich Lane'
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

 Hello!

> >   BTW, i'm still unhappy about ABI breakage here. I think we could easily add --shared-mem
> option, which would simply change mapping mode to SHARED. So, we could use it with both
> hugepages (default) and plain mmap (with --no-hugepages).
> 
> You mean, use "--no-hugepages --shared-mem" together, right?

 Yes. This would be perfectly backwards-compatible because.

Kind regards,
Pavel Fedin
Senior Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12 11:00             ` Pavel Fedin
@ 2016-01-12 11:07               ` Sergio Gonzalez Monroy
  2016-01-12 11:37                 ` Pavel Fedin
  2016-01-12 11:44                 ` Sergio Gonzalez Monroy
  0 siblings, 2 replies; 196+ messages in thread
From: Sergio Gonzalez Monroy @ 2016-01-12 11:07 UTC (permalink / raw)
  To: Pavel Fedin
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

Hi Pavel,

On 12/01/2016 11:00, Pavel Fedin wrote:
>   Hello!
>
>>>    BTW, i'm still unhappy about ABI breakage here. I think we could easily add --shared-mem
>> option, which would simply change mapping mode to SHARED. So, we could use it with both
>> hugepages (default) and plain mmap (with --no-hugepages).
>>
>> You mean, use "--no-hugepages --shared-mem" together, right?
>   Yes. This would be perfectly backwards-compatible because.

So are you suggesting to not introduce --single-file option but instead 
--shared-mem?
AFAIK --single-file was trying to workaround the limitation of just 
being able to map 8 fds.

Sergio
> Kind regards,
> Pavel Fedin
> Senior Engineer
> Samsung Electronics Research center Russia
>
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12 10:48           ` Tan, Jianfeng
  2016-01-12 11:00             ` Pavel Fedin
@ 2016-01-12 11:22             ` Pavel Fedin
  2016-01-12 11:33               ` Sergio Gonzalez Monroy
  1 sibling, 1 reply; 196+ messages in thread
From: Pavel Fedin @ 2016-01-12 11:22 UTC (permalink / raw)
  To: 'Tan, Jianfeng', 'Rich Lane'
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

 Hello!

> Oh I get it and recognize the problem here. The actual problem lies in
> the API rte_eal_get_backfile_info().
> backfiles[i].size = hugepage_files[i].size;
> Should use statfs or hugepage_files[i].size * hugepage_files[i].repeated
> to calculate the total size.

 .repeated depends on CONFIG_RTE_EAL_SIGLE_FILE_SEGMENTS. By the way, looks like it does the same thing as you are trying to do with --single-file, but with hugepages, doesn't it? I see it's currently used by ivshmem (which is AFAIK very immature and half-abandoned).
 Or should we just move .repeated out of the #ifdef ?

Kind regards,
Pavel Fedin
Senior Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12 11:22             ` Pavel Fedin
@ 2016-01-12 11:33               ` Sergio Gonzalez Monroy
  2016-01-12 12:01                 ` Pavel Fedin
  0 siblings, 1 reply; 196+ messages in thread
From: Sergio Gonzalez Monroy @ 2016-01-12 11:33 UTC (permalink / raw)
  To: Pavel Fedin
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

On 12/01/2016 11:22, Pavel Fedin wrote:
>   Hello!
>
>> Oh I get it and recognize the problem here. The actual problem lies in
>> the API rte_eal_get_backfile_info().
>> backfiles[i].size = hugepage_files[i].size;
>> Should use statfs or hugepage_files[i].size * hugepage_files[i].repeated
>> to calculate the total size.
>   .repeated depends on CONFIG_RTE_EAL_SIGLE_FILE_SEGMENTS. By the way, looks like it does the same thing as you are trying to do with --single-file, but with hugepages, doesn't it? I see it's currently used by ivshmem (which is AFAIK very immature and half-abandoned).

Similar but not the same.
--single-file: a single file for all mapped hugepages.
SINGLE_FILE_SEGMENTS: a file per set of physically contiguous mapped 
hugepages (what DPDK calls memseg , memory segment). So there could be 
more than one file.

Sergio
>   Or should we just move .repeated out of the #ifdef ?
>
> Kind regards,
> Pavel Fedin
> Senior Engineer
> Samsung Electronics Research center Russia
>
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12 11:07               ` Sergio Gonzalez Monroy
@ 2016-01-12 11:37                 ` Pavel Fedin
  2016-01-12 12:12                   ` Sergio Gonzalez Monroy
  2016-01-12 11:44                 ` Sergio Gonzalez Monroy
  1 sibling, 1 reply; 196+ messages in thread
From: Pavel Fedin @ 2016-01-12 11:37 UTC (permalink / raw)
  To: 'Sergio Gonzalez Monroy'
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

 Hello!

> So are you suggesting to not introduce --single-file option but instead
> --shared-mem?
> AFAIK --single-file was trying to workaround the limitation of just
> being able to map 8 fds.

 Heh, yes, you're right... Indeed, sorry, i was not patient enough, i see it uses hpi->hugedir instead of using /dev/shm... I was confused by the code path... It seemed that --single-file is an alias to --no-hugepages.
 And the patch still changes mmap() mode to SHARED unconditionally, which is not good in terms of backwards compability (and this is explicitly noticed in the cover letter).

 So, let's try to sort out...
 a) By default we should still have MAP_PRIVATE
 b) Let's say that we need --shared-mem in order to make it MAP_SHARED. This can be combined with --no-hugepages if necessary (this is what i tried to implement based on the old RFC).
 c) Let's say that --single-file uses hugetlbfs but maps everything via single file. This still can be combined with --shared-mem.

 wouldn't this be more clear, more straightforward and implication-free?

 And if we agree on that, we could now try to decrease number of options:
 a) We could imply MAP_SHARED if cvio is used, because shared memory is mandatory in this case.
 b) (c) above again raises a question: doesn't it make CONFIG_RTE_EAL_SIGLE_FILE_SEGMENTS obsolete? Or may be we could use that one instead of --single-file (however i'm not a fan of compile-time configuration like this)?

Kind regards,
Pavel Fedin
Senior Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12 11:07               ` Sergio Gonzalez Monroy
  2016-01-12 11:37                 ` Pavel Fedin
@ 2016-01-12 11:44                 ` Sergio Gonzalez Monroy
  1 sibling, 0 replies; 196+ messages in thread
From: Sergio Gonzalez Monroy @ 2016-01-12 11:44 UTC (permalink / raw)
  To: Pavel Fedin
  Cc: dev, nakajima.yoshihiro, ann.zhuangyanying, 'Michael S. Tsirkin'

On 12/01/2016 11:07, Sergio Gonzalez Monroy wrote:
> Hi Pavel,
>
> On 12/01/2016 11:00, Pavel Fedin wrote:
>>   Hello!
>>
>>>>    BTW, i'm still unhappy about ABI breakage here. I think we could 
>>>> easily add --shared-mem

Could you elaborate a bit more on your concerns regarding ABI breakage ?

>>> option, which would simply change mapping mode to SHARED. So, we 
>>> could use it with both
>>> hugepages (default) and plain mmap (with --no-hugepages).
>>>
>>> You mean, use "--no-hugepages --shared-mem" together, right?
>>   Yes. This would be perfectly backwards-compatible because.
>
> So are you suggesting to not introduce --single-file option but 
> instead --shared-mem?
> AFAIK --single-file was trying to workaround the limitation of just 
> being able to map 8 fds.
>

My bad, I misread the posts.
Jianfeng pointed out that you are suggesting to have --shared-mem to 
have same functionality
with or without hugepages.

Sergio

> Sergio
>> Kind regards,
>> Pavel Fedin
>> Senior Engineer
>> Samsung Electronics Research center Russia
>>
>>
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12 11:33               ` Sergio Gonzalez Monroy
@ 2016-01-12 12:01                 ` Pavel Fedin
  2016-01-12 13:39                   ` Sergio Gonzalez Monroy
  0 siblings, 1 reply; 196+ messages in thread
From: Pavel Fedin @ 2016-01-12 12:01 UTC (permalink / raw)
  To: 'Sergio Gonzalez Monroy'
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

 Hello!

> >   .repeated depends on CONFIG_RTE_EAL_SIGLE_FILE_SEGMENTS. By the way, looks like it does
> the same thing as you are trying to do with --single-file, but with hugepages, doesn't it? I
> see it's currently used by ivshmem (which is AFAIK very immature and half-abandoned).
> 
> Similar but not the same.
> --single-file: a single file for all mapped hugepages.
> SINGLE_FILE_SEGMENTS: a file per set of physically contiguous mapped
> hugepages (what DPDK calls memseg , memory segment). So there could be
> more than one file.

 Thank you for the explanation.

 By this time, i've done more testing. Current patchset breaks --no-huge. I did not study why:
--- cut ---
Program received signal SIGBUS, Bus error.
malloc_elem_init (elem=elem@entry=0x7fffe51e6000, heap=0x7ffff7fe5a1c, ms=ms@entry=0x7ffff7fb301c, size=size@entry=268435392) at /home/p.fedin/dpdk/lib/librte_eal/common/malloc_elem.c:62
62	/home/p.fedin/dpdk/lib/librte_eal/common/malloc_elem.c: No such file or directory.
Missing separate debuginfos, use: dnf debuginfo-install keyutils-libs-1.5.9-7.fc23.x86_64 krb5-libs-1.13.2-11.fc23.x86_64 libcap-ng-0.7.7-2.fc23.x86_64 libcom_err-1.42.13-3.fc23.x86_64 libselinux-2.4-4.fc23.x86_64 openssl-libs-1.0.2d-2.fc23.x86_64 pcre-8.37-4.fc23.x86_64 zlib-1.2.8-9.fc23.x86_64
(gdb) where
#0  malloc_elem_init (elem=elem@entry=0x7fffe51e6000, heap=0x7ffff7fe5a1c, ms=ms@entry=0x7ffff7fb301c, size=size@entry=268435392)
    at /home/p.fedin/dpdk/lib/librte_eal/common/malloc_elem.c:62
#1  0x00000000004a50b5 in malloc_heap_add_memseg (ms=0x7ffff7fb301c, heap=<optimized out>) at /home/p.fedin/dpdk/lib/librte_eal/common/malloc_heap.c:109
#2  rte_eal_malloc_heap_init () at /home/p.fedin/dpdk/lib/librte_eal/common/malloc_heap.c:232
#3  0x00000000004be896 in rte_eal_memzone_init () at /home/p.fedin/dpdk/lib/librte_eal/common/eal_common_memzone.c:427
#4  0x000000000042ab02 in rte_eal_init (argc=argc@entry=11, argv=argv@entry=0x7fffffffeb80) at /home/p.fedin/dpdk/lib/librte_eal/linuxapp/eal/eal.c:799
#5  0x000000000066dfb9 in dpdk_init (argc=11, argv=0x7fffffffeb80) at lib/netdev-dpdk.c:2192
#6  0x000000000040ddd9 in main (argc=12, argv=0x7fffffffeb78) at vswitchd/ovs-vswitchd.c:74
--- cut ---

 And now i tend to think that we do not need --single-file at all. Because:
a) It's just a temporary workaround for "more than 8 regions" problem.
b) It's not compatible with physical hardware anyway.

 So i think that we could easily use "--no-huge --shared-mem" combination. We could address hugepages compatibility problem later.

Kind regards,
Pavel Fedin
Senior Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12 11:37                 ` Pavel Fedin
@ 2016-01-12 12:12                   ` Sergio Gonzalez Monroy
  2016-01-12 13:57                     ` Pavel Fedin
  0 siblings, 1 reply; 196+ messages in thread
From: Sergio Gonzalez Monroy @ 2016-01-12 12:12 UTC (permalink / raw)
  To: Pavel Fedin
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

On 12/01/2016 11:37, Pavel Fedin wrote:
>   Hello!
>
>> So are you suggesting to not introduce --single-file option but instead
>> --shared-mem?
>> AFAIK --single-file was trying to workaround the limitation of just
>> being able to map 8 fds.
>   Heh, yes, you're right... Indeed, sorry, i was not patient enough, i see it uses hpi->hugedir instead of using /dev/shm... I was confused by the code path... It seemed that --single-file is an alias to --no-hugepages.
>   And the patch still changes mmap() mode to SHARED unconditionally, which is not good in terms of backwards compability (and this is explicitly noticed in the cover letter).

I might be missing something obvious here but, aside from having memory 
SHARED which most DPDK apps using hugepages will have anyway, what is 
the backward compatibility issues that you see here?

>
>   So, let's try to sort out...
>   a) By default we should still have MAP_PRIVATE
>   b) Let's say that we need --shared-mem in order to make it MAP_SHARED. This can be combined with --no-hugepages if necessary (this is what i tried to implement based on the old RFC).

--share-mem would only have meaning with --no-huge, right?

>   c) Let's say that --single-file uses hugetlbfs but maps everything via single file. This still can be combined with --shared-mem.

By default, when using hugepages all mappings are SHARED for 
multiprocess model.
IMHO If you really want to have the ability to have private memory 
instead because you are not considering that model, then it might be 
more appropriate to have --private-mem or --no-shared-mem option instead.

Sergio
>   wouldn't this be more clear, more straightforward and implication-free?
>
>   And if we agree on that, we could now try to decrease number of options:
>   a) We could imply MAP_SHARED if cvio is used, because shared memory is mandatory in this case.
>   b) (c) above again raises a question: doesn't it make CONFIG_RTE_EAL_SIGLE_FILE_SEGMENTS obsolete? Or may be we could use that one instead of --single-file (however i'm not a fan of compile-time configuration like this)?
>
> Kind regards,
> Pavel Fedin
> Senior Engineer
> Samsung Electronics Research center Russia
>
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12 12:01                 ` Pavel Fedin
@ 2016-01-12 13:39                   ` Sergio Gonzalez Monroy
  0 siblings, 0 replies; 196+ messages in thread
From: Sergio Gonzalez Monroy @ 2016-01-12 13:39 UTC (permalink / raw)
  To: Pavel Fedin
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

On 12/01/2016 12:01, Pavel Fedin wrote:
>   Hello!
>
>>>    .repeated depends on CONFIG_RTE_EAL_SIGLE_FILE_SEGMENTS. By the way, looks like it does
>> the same thing as you are trying to do with --single-file, but with hugepages, doesn't it? I
>> see it's currently used by ivshmem (which is AFAIK very immature and half-abandoned).
>>
>> Similar but not the same.
>> --single-file: a single file for all mapped hugepages.
>> SINGLE_FILE_SEGMENTS: a file per set of physically contiguous mapped
>> hugepages (what DPDK calls memseg , memory segment). So there could be
>> more than one file.
>   Thank you for the explanation.
>
>   By this time, i've done more testing. Current patchset breaks --no-huge. I did not study why:
> --- cut ---
> Program received signal SIGBUS, Bus error.
> malloc_elem_init (elem=elem@entry=0x7fffe51e6000, heap=0x7ffff7fe5a1c, ms=ms@entry=0x7ffff7fb301c, size=size@entry=268435392) at /home/p.fedin/dpdk/lib/librte_eal/common/malloc_elem.c:62
> 62	/home/p.fedin/dpdk/lib/librte_eal/common/malloc_elem.c: No such file or directory.
> Missing separate debuginfos, use: dnf debuginfo-install keyutils-libs-1.5.9-7.fc23.x86_64 krb5-libs-1.13.2-11.fc23.x86_64 libcap-ng-0.7.7-2.fc23.x86_64 libcom_err-1.42.13-3.fc23.x86_64 libselinux-2.4-4.fc23.x86_64 openssl-libs-1.0.2d-2.fc23.x86_64 pcre-8.37-4.fc23.x86_64 zlib-1.2.8-9.fc23.x86_64
> (gdb) where
> #0  malloc_elem_init (elem=elem@entry=0x7fffe51e6000, heap=0x7ffff7fe5a1c, ms=ms@entry=0x7ffff7fb301c, size=size@entry=268435392)
>      at /home/p.fedin/dpdk/lib/librte_eal/common/malloc_elem.c:62
> #1  0x00000000004a50b5 in malloc_heap_add_memseg (ms=0x7ffff7fb301c, heap=<optimized out>) at /home/p.fedin/dpdk/lib/librte_eal/common/malloc_heap.c:109
> #2  rte_eal_malloc_heap_init () at /home/p.fedin/dpdk/lib/librte_eal/common/malloc_heap.c:232
> #3  0x00000000004be896 in rte_eal_memzone_init () at /home/p.fedin/dpdk/lib/librte_eal/common/eal_common_memzone.c:427
> #4  0x000000000042ab02 in rte_eal_init (argc=argc@entry=11, argv=argv@entry=0x7fffffffeb80) at /home/p.fedin/dpdk/lib/librte_eal/linuxapp/eal/eal.c:799
> #5  0x000000000066dfb9 in dpdk_init (argc=11, argv=0x7fffffffeb80) at lib/netdev-dpdk.c:2192
> #6  0x000000000040ddd9 in main (argc=12, argv=0x7fffffffeb78) at vswitchd/ovs-vswitchd.c:74
> --- cut ---
>
>   And now i tend to think that we do not need --single-file at all. Because:
> a) It's just a temporary workaround for "more than 8 regions" problem.
> b) It's not compatible with physical hardware anyway.

That's a good summary.
I think --single-file was mostly solving the limit of vhost only mapping 
8 fds. We end up with a single memseg as we do with --no-huge except 
that they are hugepages (well, also in this patch mapped with shared 
instead of private).
Also, It would be compatible with physical hardware if using iommu and vfio.

Sergio

>   So i think that we could easily use "--no-huge --shared-mem" combination. We could address hugepages compatibility problem later.
>
> Kind regards,
> Pavel Fedin
> Senior Engineer
> Samsung Electronics Research center Russia
>
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12 12:12                   ` Sergio Gonzalez Monroy
@ 2016-01-12 13:57                     ` Pavel Fedin
  2016-01-12 14:13                       ` Sergio Gonzalez Monroy
  0 siblings, 1 reply; 196+ messages in thread
From: Pavel Fedin @ 2016-01-12 13:57 UTC (permalink / raw)
  To: 'Sergio Gonzalez Monroy'
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

 Hello!

> I might be missing something obvious here but, aside from having memory
> SHARED which most DPDK apps using hugepages will have anyway, what is
> the backward compatibility issues that you see here?

 Heh, sorry once again for confusing. Indeed, with hugepages we always get MAP_SHARED. I missed that. So, we indeed need
--shared-mem only in addition to --no-huge.

 Backwards compatibility issue is stated in the description of PATCH 1/4:
--- cut ---
b. possible ABI break, originally, --no-huge uses anonymous memory
instead of file-backed way to create memory.
--- cut ---
 The patch unconditionally changes that to SHARED. That's all.

Kind regards,
Pavel Fedin
Senior Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 2/4] mem: add API to obstain memory-backed file info
  2016-01-12 13:57                     ` Pavel Fedin
@ 2016-01-12 14:13                       ` Sergio Gonzalez Monroy
  0 siblings, 0 replies; 196+ messages in thread
From: Sergio Gonzalez Monroy @ 2016-01-12 14:13 UTC (permalink / raw)
  To: Pavel Fedin
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

On 12/01/2016 13:57, Pavel Fedin wrote:
>   Hello!
>
>> I might be missing something obvious here but, aside from having memory
>> SHARED which most DPDK apps using hugepages will have anyway, what is
>> the backward compatibility issues that you see here?
>   Heh, sorry once again for confusing. Indeed, with hugepages we always get MAP_SHARED. I missed that. So, we indeed need
> --shared-mem only in addition to --no-huge.
>
>   Backwards compatibility issue is stated in the description of PATCH 1/4:
> --- cut ---
> b. possible ABI break, originally, --no-huge uses anonymous memory
> instead of file-backed way to create memory.
> --- cut ---
>   The patch unconditionally changes that to SHARED. That's all.

I should read more carefully!
Sorry about that, I thought you were the one with the ABI concerns.

Regarding ABI, I don't think there is any ABI issue with the change, we 
just have our memory file-backed and SHARED but we do that when using 
hugepages so I don't think it would be a huge issue.
But if folks have concerns about it, we could always keep old behavior 
by default and, as you suggest, introduce another option for changing 
the flag.

Sergio
> Kind regards,
> Pavel Fedin
> Senior Engineer
> Samsung Electronics Research center Russia
>
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12 10:48               ` Pavel Fedin
@ 2016-01-12 14:45                 ` Amit Tomer
  2016-01-12 14:50                   ` Pavel Fedin
  2016-01-12 14:53                   ` Tan, Jianfeng
  0 siblings, 2 replies; 196+ messages in thread
From: Amit Tomer @ 2016-01-12 14:45 UTC (permalink / raw)
  To: Pavel Fedin
  Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hello,

I run l2fwd from inside docker with following logs:

But, don't see Port statistics gets updated ?

#/home/ubuntu/dpdk# sudo docker run -i -t -v
/home/ubuntu/dpdk/usvhost:/usr/src/dpdk/usvhost l4
EAL: Detected lcore 0 as core 0 on socket 0
EAL: Detected lcore 1 as core 1 on socket 0
EAL: Detected lcore 2 as core 2 on socket 0
EAL: Detected lcore 3 as core 3 on socket 0
EAL: Detected lcore 4 as core 4 on socket 0
EAL: Detected lcore 5 as core 5 on socket 0
EAL: Detected lcore 6 as core 6 on socket 0
EAL: Detected lcore 7 as core 7 on socket 0
EAL: Detected lcore 8 as core 8 on socket 0
EAL: Setting up physically contiguous memory...
EAL: TSC frequency is ~99999 KHz
EAL: Master lcore 1 is ready (tid=b5968000;cpuset=[1])
Notice: odd number of ports in portmask.
Lcore 1: RX port 0
Initializing port 0... done:
Port 0, MAC address: F6:9F:7A:47:A4:99

Checking link statusdone
Port 0 Link Up - speed 10000 Mbps - full-duplex
L2FWD: entering main loop on lcore 1
L2FWD:  -- lcoreid=1 portid=0


Port statistics ====================================
Statistics for port 0 ------------------------------
Packets sent:                        0
Packets received:                    0
Packets dropped:                     0
Aggregate statistics ===============================
Total packets sent:                  0
Total packets received:              0
Total packets dropped:               0
====================================================

Host side logs after running

# ./vhost-switch -c 0x3 f -n 4 --socket-mem 2048 --huge-dir
/dev/hugepages -- -p 0x1  --dev-basename usvhost

PMD: eth_ixgbe_dev_init(): MAC: 4, PHY: 3
PMD: eth_ixgbe_dev_init(): port 1 vendorID=0x8086 deviceID=0x1528
pf queue num: 0, configured vmdq pool num: 64, each vmdq pool has 2 queues
VHOST_PORT: Max virtio devices supported: 64
VHOST_PORT: Port 0 MAC: d8 9d 67 ee 55 f0
VHOST_PORT: Skipping disabled port 1
VHOST_DATA: Procesing on Core 1 started
VHOST_CONFIG: socket created, fd:20
VHOST_CONFIG: bind to usvhost
VHOST_CONFIG: new virtio connection is 21
VHOST_CONFIG: new device, handle is 0
VHOST_CONFIG: read message VHOST_USER_GET_FEATURES
VHOST_CONFIG: read message VHOST_USER_SET_FEATURES
VHOST_CONFIG: read message VHOST_USER_SET_MEM_TABLE
VHOST_CONFIG: mapped region 0 fd:22 to 0x7f34000000 sz:0x4000000 off:0x0
VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
VHOST_CONFIG: vring call idx:0 file:23
VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
VHOST_CONFIG: vring kick idx:0 file:24
VHOST_CONFIG: virtio isn't ready for processing.
VHOST_CONFIG: read message VHOST_USER_SET_VRING_CALL
VHOST_CONFIG: vring call idx:1 file:25
VHOST_CONFIG: read message VHOST_USER_SET_VRING_NUM
VHOST_CONFIG: read message VHOST_USER_SET_VRING_BASE
VHOST_CONFIG: read message VHOST_USER_SET_VRING_ADDR
VHOST_CONFIG: read message VHOST_USER_SET_VRING_KICK
VHOST_CONFIG: vring kick idx:1 file:26
VHOST_CONFIG: virtio is now ready for processing.
VHOST_DATA: (0) Device has been added to data core 1

Could anyone please point out, how it can be tested further(how can
traffic be sent across host and container)  ?

Thanks,
Amit.

On Tue, Jan 12, 2016 at 4:18 PM, Pavel Fedin <p.fedin@samsung.com> wrote:
>  Hello!
>
>> Your guess makes sense because current implementation does not support
>> multi-queues.
>>
>>  From you log, only 0 and 1 are "ready for processing"; others are "not
>> ready for processing".
>
>  Yes, and if study it even more carefully, we see that we initialize all tx queues but only a single rx queue (#0).
>  After some more code browsing and comparing the two patchsets i figured out that the problem is caused by inappropriate VIRTIO_NET_F_CTRL_VQ flag. In your RFC you used different capability set, while in v1 you seem to have forgotten about this.
>  I suggest to temporarily move hw->guest_features assignment out of virtio_negotiate_features() into the caller, where we have eth_dev->dev_type, and can choose the right set depending on it.
>
>  With all mentioned fixes i've got the ping running.
>  Tested-by: Pavel Fedin <p.fedin@samsung.com>
>
> Kind regards,
> Pavel Fedin
> Expert Engineer
> Samsung Electronics Research center Russia
>
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12 14:45                 ` Amit Tomer
@ 2016-01-12 14:50                   ` Pavel Fedin
  2016-01-12 14:58                     ` Amit Tomer
  2016-01-12 14:53                   ` Tan, Jianfeng
  1 sibling, 1 reply; 196+ messages in thread
From: Pavel Fedin @ 2016-01-12 14:50 UTC (permalink / raw)
  To: 'Amit Tomer'
  Cc: nakajima.yoshihiro, 'Michael S. Tsirkin', dev, ann.zhuangyanying

 Hello!

> Could anyone please point out, how it can be tested further(how can
> traffic be sent across host and container)  ?

 Have you applied all three fixes discussed here?

Kind regards,
Pavel Fedin
Senior Engineer
Samsung Electronics Research center Russia

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12 14:45                 ` Amit Tomer
  2016-01-12 14:50                   ` Pavel Fedin
@ 2016-01-12 14:53                   ` Tan, Jianfeng
  2016-01-12 15:11                     ` Amit Tomer
  1 sibling, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-12 14:53 UTC (permalink / raw)
  To: Amit Tomer, Pavel Fedin
  Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hello!

On 1/12/2016 10:45 PM, Amit Tomer wrote:
> Hello,
>
> I run l2fwd from inside docker with following logs:
>
> But, don't see Port statistics gets updated ?
>

In vhost-switch, it judges if a virtio device is ready for processing 
after receiving
a pkt from virtio device. So you'd better construct a pkt, and send it 
out firstly
in l2fwd.

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12 14:50                   ` Pavel Fedin
@ 2016-01-12 14:58                     ` Amit Tomer
  0 siblings, 0 replies; 196+ messages in thread
From: Amit Tomer @ 2016-01-12 14:58 UTC (permalink / raw)
  To: Pavel Fedin
  Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hello,

>  Have you applied all three fixes discussed here?

I am running it with, only RFC patches applied with "--no-huge" in l2fwd.

Thanks
Amit.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12 14:53                   ` Tan, Jianfeng
@ 2016-01-12 15:11                     ` Amit Tomer
  2016-01-12 16:18                       ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Amit Tomer @ 2016-01-12 15:11 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hello,

> In vhost-switch, it judges if a virtio device is ready for processing after
> receiving
> a pkt from virtio device. So you'd better construct a pkt, and send it out
> firstly
> in l2fwd.

I tried to ping the socket interface from host for the same purpose
but it didn't work.

Could you please suggest some other approach for achieving same(how
pkt can be sent out to l2fwd)?

Also, before trying this, I have verified that vhost-switch is working
ok with testpmd .

Thanks,
Amit.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12 15:11                     ` Amit Tomer
@ 2016-01-12 16:18                       ` Tan, Jianfeng
  2016-01-13 15:00                         ` Amit Tomer
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-12 16:18 UTC (permalink / raw)
  To: Amit Tomer; +Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hello,

On 1/12/2016 11:11 PM, Amit Tomer wrote:
> Hello,
>
>> In vhost-switch, it judges if a virtio device is ready for processing after
>> receiving
>> a pkt from virtio device. So you'd better construct a pkt, and send it out
>> firstly
>> in l2fwd.
> I tried to ping the socket interface from host for the same purpose
> but it didn't work.
>
> Could you please suggest some other approach for achieving same(how
> pkt can be sent out to l2fwd)?
>
> Also, before trying this, I have verified that vhost-switch is working
> ok with testpmd .
>
> Thanks,
> Amit.

You can use below patch for l2fwd to send out an arp packet when it gets 
started.

diff --git a/examples/l2fwd/main.c b/examples/l2fwd/main.c
index 720fd5a..572b1ac 100644
--- a/examples/l2fwd/main.c
+++ b/examples/l2fwd/main.c
@@ -69,6 +69,8 @@
  #include <rte_mempool.h>
  #include <rte_mbuf.h>

+#define SEND_ARP
+
  #define RTE_LOGTYPE_L2FWD RTE_LOGTYPE_USER1

  #define NB_MBUF   8192
@@ -185,6 +187,53 @@ print_stats(void)
printf("\n====================================================\n");
  }

+#ifdef SEND_ARP
+static void
+dpdk_send_arp(int portid, struct rte_mempool *mp)
+{
+    /*
+     * len = 14 + 46
+     * ARP, Request who-has 10.0.0.1 tell 10.0.0.2, length 46
+     */
+    static const uint8_t arp_request[] = {
+        /*0x0000:*/ 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xec, 0xa8, 
0x6b, 0xfd, 0x02, 0x29, 0x08, 0x06, 0x00, 0x01,
+        /*0x0010:*/ 0x08, 0x00, 0x06, 0x04, 0x00, 0x01, 0xec, 0xa8, 
0x6b, 0xfd, 0x02, 0x29, 0x0a, 0x00, 0x00, 0x01,
+        /*0x0020:*/ 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x0a, 0x00, 
0x00, 0x02, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
+        /*0x0030:*/ 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 
0x00, 0x00, 0x00, 0x00
+    };
+    int ret;
+    struct rte_mbuf *m;
+    struct ether_addr mac_addr;
+    int pkt_len = sizeof(arp_request) - 1;
+
+    m = rte_pktmbuf_alloc(mp);
+
+    memcpy((void *)((uint64_t)m->buf_addr + m->data_off), arp_request, 
pkt_len);
+    rte_pktmbuf_pkt_len(m) = pkt_len;
+    rte_pktmbuf_data_len(m) = pkt_len;
+
+    rte_eth_macaddr_get(portid, &mac_addr);
+    memcpy((void *)((uint64_t)m->buf_addr + m->data_off + 6), 
&mac_addr, 6);
+
+    ret = rte_eth_tx_burst(portid, 0, &m, 1);
+    if (ret == 1) {
+        printf("arp sent: ok\n");
+        printf("%02x:%02x:%02x:%02x:%02x:%02x\n",
+                mac_addr.addr_bytes[0],
+                mac_addr.addr_bytes[1],
+                mac_addr.addr_bytes[2],
+                mac_addr.addr_bytes[3],
+                mac_addr.addr_bytes[4],
+                mac_addr.addr_bytes[5]);
+    } else {
+        printf("arp sent: fail\n");
+    }
+
+    rte_pktmbuf_free(m);
+}
+#endif
+
+
  /* Send the burst of packets on an output interface */
  static int
  l2fwd_send_burst(struct lcore_queue_conf *qconf, unsigned n, uint8_t port)
@@ -281,6 +330,9 @@ l2fwd_main_loop(void)
          portid = qconf->rx_port_list[i];
          RTE_LOG(INFO, L2FWD, " -- lcoreid=%u portid=%u\n", lcore_id,
              portid);
+#ifdef SEND_ARP
+        dpdk_send_arp(portid, l2fwd_pktmbuf_pool);
+#endif
      }

      while (1) {

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12 16:18                       ` Tan, Jianfeng
@ 2016-01-13 15:00                         ` Amit Tomer
  2016-01-13 18:41                           ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Amit Tomer @ 2016-01-13 15:00 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hello,

>
> You can use below patch for l2fwd to send out an arp packet when it gets
> started.

I tried to send out arp packet using this patch but buffer allocation
for arp packets itself gets failed:

 m = rte_pktmbuf_alloc(mp);

Return a NULL Value.

Thanks,
Amit.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-13 15:00                         ` Amit Tomer
@ 2016-01-13 18:41                           ` Tan, Jianfeng
  2016-01-14  9:34                             ` Amit Tomer
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-13 18:41 UTC (permalink / raw)
  To: Amit Tomer; +Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hi Amit,

On 1/13/2016 11:00 PM, Amit Tomer wrote:
> Hello,
>
>> You can use below patch for l2fwd to send out an arp packet when it gets
>> started.
> I tried to send out arp packet using this patch but buffer allocation
> for arp packets itself gets failed:
>
>   m = rte_pktmbuf_alloc(mp);
>
> Return a NULL Value.

Can you send out how you start this l2fwd program?

Thanks,
Jianfeng


>
> Thanks,
> Amit.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-13 18:41                           ` Tan, Jianfeng
@ 2016-01-14  9:34                             ` Amit Tomer
  2016-01-14 11:41                               ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Amit Tomer @ 2016-01-14  9:34 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hello,

> Can you send out how you start this l2fwd program?

This is how, I run l2fwd program.

CMD ["/usr/src/dpdk/examples/l2fwd/build/l2fwd", "-c", "0x3", "-n",
"4","--no-pci",
,"--no-huge","--vdev=eth_cvio0,queue_num=256,rx=1,tx=1,cq=0,path=/usr/src/dpdk/usvhost",
"--", "-p", "0x1"]

I tried passing "-m 1024" to it but It causes l2fwd killed even before
it could connect to usvhost socket.

Do I need to create Hugepages from Inside Docker container to make use
of Hugepages?

Thanks,
Amit.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-14  9:34                             ` Amit Tomer
@ 2016-01-14 11:41                               ` Tan, Jianfeng
  2016-01-14 12:03                                 ` Amit Tomer
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-14 11:41 UTC (permalink / raw)
  To: Amit Tomer; +Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hi Amit,

On 1/14/2016 5:34 PM, Amit Tomer wrote:
> Hello,
>
>> Can you send out how you start this l2fwd program?
> This is how, I run l2fwd program.
>
> CMD ["/usr/src/dpdk/examples/l2fwd/build/l2fwd", "-c", "0x3", "-n",
> "4","--no-pci",
> ,"--no-huge","--vdev=eth_cvio0,queue_num=256,rx=1,tx=1,cq=0,path=/usr/src/dpdk/usvhost",
> "--", "-p", "0x1"]

In this way, you can only get 64M memory. I believe it's too small to 
create a l2fwd_pktmbuf_pool in l2fwd.

> I tried passing "-m 1024" to it but It causes l2fwd killed even before
> it could connect to usvhost socket.

In my patch, when --no-huge is specified, I change previous anonymous 
mmap into file-baked memory in /dev/shm. And usually, Docker mounts a 
64MB-size tmpfs there, so you cannot use -m 1024. If you want to do 
that, use -v to substitute the 64MB tmpfs with a bigger tmpfs.


>
> Do I need to create Hugepages from Inside Docker container to make use
> of Hugepages?

Not necessary. But if you want to use hugepages inside Docker, use -v 
option to map a hugetlbfs into containers.

Most importantly, you indeed uncover a bug here. Current implementation 
cannot work with tmpfs, because it lacks ftruncate() between open() and 
mmap(). It turns out that although mmap() succeeds, the memory cannot be 
touched. However, this is not a problem for hugetlbfs. I don't why they 
differ like that way. In all, if you want to use no-huge, please add 
ftruncate(), I'll fix this in next version.

Thanks,
Jianfeng

>
> Thanks,
> Amit.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-14 11:41                               ` Tan, Jianfeng
@ 2016-01-14 12:03                                 ` Amit Tomer
  2016-01-15  6:39                                   ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Amit Tomer @ 2016-01-14 12:03 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hello,

>
> Not necessary. But if you want to use hugepages inside Docker, use -v option
> to map a hugetlbfs into containers.

I modified Docker command line in order to make use of Hugetlbfs:

CMD ["/usr/src/dpdk/examples/l2fwd/build/l2fwd", "-c", "0x3", "-n",
"4","--no-pci", "--socket-mem","512",
"--vdev=eth_cvio0,queue_num=256,rx=1,tx=1,cq=0,path=/var/run/usvhost",
"--", "-p", "0x1"]

Then, I run docker :

 docker run -i -t --privileged  -v /dev/hugepages:/dev/hugepages  -v
/home/ubuntu/backup/usvhost:/var/run/usvhost  l6

But this is what I see:

EAL: Support maximum 128 logical core(s) by configuration.
EAL: Detected 48 lcore(s)
EAL: Setting up physically contiguous memory...
EAL: Failed to find phys addr for 2 MB pages
PANIC in rte_eal_init():
Cannot init memory
1: [/usr/src/dpdk/examples/l2fwd/build/l2fwd(rte_dump_stack+0x20) [0x48ea78]]

This is from Host:

# mount | grep hugetlbfs
hugetlbfs on /dev/hugepages type hugetlbfs (rw,relatime)
none on /dev/hugepages type hugetlbfs (rw,relatime)

 #cat /proc/meminfo | grep Huge
AnonHugePages:    548864 kB
HugePages_Total:    4096
HugePages_Free:     1024
HugePages_Rsvd:        0
HugePages_Surp:        0
Hugepagesize:       2048 kB

What is it, I'm doing wrong here?

Thanks,
Amit

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-14 12:03                                 ` Amit Tomer
@ 2016-01-15  6:39                                   ` Tan, Jianfeng
  2016-01-20 15:19                                     ` Amit Tomer
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-15  6:39 UTC (permalink / raw)
  To: Amit Tomer; +Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hi Amit,

On 1/14/2016 8:03 PM, Amit Tomer wrote:
> Hello,
>
>> Not necessary. But if you want to use hugepages inside Docker, use -v option
>> to map a hugetlbfs into containers.
> I modified Docker command line in order to make use of Hugetlbfs:
>
> CMD ["/usr/src/dpdk/examples/l2fwd/build/l2fwd", "-c", "0x3", "-n",
> "4","--no-pci", "--socket-mem","512",
> "--vdev=eth_cvio0,queue_num=256,rx=1,tx=1,cq=0,path=/var/run/usvhost",
> "--", "-p", "0x1"]



For this case, please use --single-file option because it creates much 
more than 8 fds, which can be handled by vhost-user sendmsg().


>
> Then, I run docker :
>
>   docker run -i -t --privileged  -v /dev/hugepages:/dev/hugepages  -v
> /home/ubuntu/backup/usvhost:/var/run/usvhost  l6
>
> But this is what I see:
>
> EAL: Support maximum 128 logical core(s) by configuration.
> EAL: Detected 48 lcore(s)
> EAL: Setting up physically contiguous memory...
> EAL: Failed to find phys addr for 2 MB pages
> PANIC in rte_eal_init():
> Cannot init memory
> 1: [/usr/src/dpdk/examples/l2fwd/build/l2fwd(rte_dump_stack+0x20) [0x48ea78]]

 From the log, it's caused by that it still cannot open 
/proc/self/pagemap. But it's strange that you already specify --privileged).

Thanks,
Jianfeng

>
> This is from Host:
>
> # mount | grep hugetlbfs
> hugetlbfs on /dev/hugepages type hugetlbfs (rw,relatime)
> none on /dev/hugepages type hugetlbfs (rw,relatime)
>
>   #cat /proc/meminfo | grep Huge
> AnonHugePages:    548864 kB
> HugePages_Total:    4096
> HugePages_Free:     1024
> HugePages_Rsvd:        0
> HugePages_Surp:        0
> Hugepagesize:       2048 kB
>
> What is it, I'm doing wrong here?
>
> Thanks,
> Amit

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-12  5:36   ` Tetsuya Mukawa
  2016-01-12  5:46     ` Tan, Jianfeng
@ 2016-01-20  3:48     ` Xie, Huawei
  1 sibling, 0 replies; 196+ messages in thread
From: Xie, Huawei @ 2016-01-20  3:48 UTC (permalink / raw)
  To: Tetsuya Mukawa, Tan, Jianfeng, dev
  Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

On 1/12/2016 1:37 PM, Tetsuya Mukawa wrote:
> Hi Jianfeng and Xie,
>
> I guess my implementation and yours have a lot of common code, so I will
> try to rebase my patch on yours.
>
> BTW, one thing I need to change your memory allocation way is that
> mmaped address should be under 44bit(32 + PAGE_SHIFT) to work with my patch.
> This is because VIRTIO_PCI_QUEUE_PFN register only accepts such address.
> (I may need to add one more EAL parameter like "--mmap-under <address>")

I believe it is OK to mmap under 44bit, but better check the user space
address space layout.

>
> Thanks,
> Tetsuya


^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-15  6:39                                   ` Tan, Jianfeng
@ 2016-01-20 15:19                                     ` Amit Tomer
  2016-01-22  6:04                                       ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Amit Tomer @ 2016-01-20 15:19 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hello,

> For this case, please use --single-file option because it creates much more
> than 8 fds, which can be handled by vhost-user sendmsg().

Thanks, I'm able to verify it by sending ARP packet from container to
host on arm64. But sometimes, I do see following message while running
l2fwd in container(pointed by Rich).

EAL: Master lcore 0 is ready (tid=8a7a3000;cpuset=[0])
EAL: lcore 1 is ready (tid=89cdf050;cpuset=[1])
Notice: odd number of ports in portmask.
Lcore 0: RX port 0
Initializing port 0... PANIC in kick_all_vq():
TUNSETVNETHDRSZ failed: Inappropriate ioctl for device

How it could be avoided?

Thanks,
Amit.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 1/4] mem: add --single-file to create single mem-backed file
  2016-01-10 11:42   ` [PATCH 1/4] mem: add --single-file to create single mem-backed file Jianfeng Tan
@ 2016-01-21  1:57     ` Xie, Huawei
  0 siblings, 0 replies; 196+ messages in thread
From: Xie, Huawei @ 2016-01-21  1:57 UTC (permalink / raw)
  To: Tan, Jianfeng; +Cc: nakajima.yoshihiro, mst, dev, ann.zhuangyanying

On 1/11/2016 2:43 AM, Tan, Jianfeng wrote:
[snip]
> +#include <mntent.h>
> +#include <sys/mman.h>
> +#include <sys/file.h>
> +#include <sys/vfs.h>

Please remove unreferenced header files.

>  
>  #include <rte_log.h>
>  #include <rte_memory.h>
> @@ -92,6 +96,9 @@
>  #include <rte_common.h>
>  #include <rte_string_fns.h>
>  
> +#define _GNU_SOURCE
> +#include <sys/syscall.h>
> +
>  #include "eal_private.h"

[snip]

> +		char filepath[MAX_HUGEPAGE_PATH];
> +
> +		syscall(SYS_getcpu, NULL, &socket_id, NULL);
> +

[snip]

>  		mcfg->memseg[0].addr = addr;
> -		mcfg->memseg[0].hugepage_sz = RTE_PGSIZE_4K;
> +		mcfg->memseg[0].hugepage_sz = pagesize;
> 		mcfg->memseg[0].len = internal_config.memory;
> -		mcfg->memseg[0].socket_id = 0;
> +		mcfg->memseg[0].socket_id = socket_id;

Anyway the socket_id here doesn't make sense. We could remove the
syscall which relies on _GNU_SOURCE.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 3/4] virtio/vdev: add ways to interact with vhost
  2016-01-10 11:43   ` [PATCH 3/4] virtio/vdev: add ways to interact with vhost Jianfeng Tan
  2016-01-11 10:42     ` Pavel Fedin
  2016-01-11 14:02     ` Pavel Fedin
@ 2016-01-21  2:18     ` Xie, Huawei
  2 siblings, 0 replies; 196+ messages in thread
From: Xie, Huawei @ 2016-01-21  2:18 UTC (permalink / raw)
  To: Tan, Jianfeng, dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

On 1/11/2016 2:43 AM, Tan, Jianfeng wrote:
> +	if (hw->type == VHOST_KERNEL) {
> +		struct vhost_vring_file file;
> +
> +		file.fd = hw->backfd;
> +		nvqs = data->nb_rx_queues + data->nb_tx_queues;
> +		for (file.index = 0; file.index < nvqs; ++file.index) {
> +			ret = vhost_kernel_ioctl(hw, VHOST_NET_SET_BACKEND, &file);
> +			if (ret < 0)
> +				rte_panic("VHOST_NET_SET_BACKEND failed, %s\n",
> +						strerror(errno));
> +		}
> +	}
> +
> +	/* TODO: VHOST_SET_LOG_BASE */

We needn't support VHOST_SET_LOG_BASE.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-20 15:19                                     ` Amit Tomer
@ 2016-01-22  6:04                                       ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-22  6:04 UTC (permalink / raw)
  To: Amit Tomer; +Cc: nakajima.yoshihiro, Michael S. Tsirkin, dev, ann.zhuangyanying

Hi Amit,

On 1/20/2016 11:19 PM, Amit Tomer wrote:
> Hello,
>
>> For this case, please use --single-file option because it creates much more
>> than 8 fds, which can be handled by vhost-user sendmsg().
> Thanks, I'm able to verify it by sending ARP packet from container to
> host on arm64. But sometimes, I do see following message while running
> l2fwd in container(pointed by Rich).
>
> EAL: Master lcore 0 is ready (tid=8a7a3000;cpuset=[0])
> EAL: lcore 1 is ready (tid=89cdf050;cpuset=[1])
> Notice: odd number of ports in portmask.
> Lcore 0: RX port 0
> Initializing port 0... PANIC in kick_all_vq():
> TUNSETVNETHDRSZ failed: Inappropriate ioctl for device
>
> How it could be avoided?
>
> Thanks,
> Amit.

Thanks for pointing out this bug. Actually it's caused by one of my 
fault. So vhost-user cannot work well.
Below change can help start vhost-user.

diff --git a/drivers/net/virtio/vhost.c b/drivers/net/virtio/vhost.c
index e423e02..dbca374 100644
--- a/drivers/net/virtio/vhost.c
+++ b/drivers/net/virtio/vhost.c
@@ -483,8 +483,9 @@ static void kick_all_vq(struct virtio_hw *hw)
         uint64_t features = hw->guest_features;
         features &= ~(1ull << VIRTIO_NET_F_MAC);
         vhost_call(hw, VHOST_MSG_SET_FEATURES, &features);
-       if (ioctl(hw->backfd, TUNSETVNETHDRSZ, &hw->vtnet_hdr_size) == -1)
-               rte_panic("TUNSETVNETHDRSZ failed: %s\n", strerror(errno));
+       if (hw->type == VHOST_KERNEL)
+               if (ioctl(hw->backfd, TUNSETVNETHDRSZ, 
&hw->vtnet_hdr_size) == -1)
+                       rte_panic("TUNSETVNETHDRSZ failed: %s\n", 
strerror(errno));
         PMD_DRV_LOG(INFO, "set features:%"PRIx64"\n", features);


Thanks,
Jianfeng

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-10 11:42 ` [PATCH 0/4] " Jianfeng Tan
                     ` (5 preceding siblings ...)
  2016-01-12  5:36   ` Tetsuya Mukawa
@ 2016-01-26  6:02   ` Qiu, Michael
  2016-01-26  6:09     ` Tan, Jianfeng
  6 siblings, 1 reply; 196+ messages in thread
From: Qiu, Michael @ 2016-01-26  6:02 UTC (permalink / raw)
  To: Tan, Jianfeng, dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

On 1/11/2016 2:43 AM, Tan, Jianfeng wrote:
> This patchset is to provide high performance networking interface (virtio)
> for container-based DPDK applications. The way of starting DPDK apps in
> containers with ownership of NIC devices exclusively is beyond the scope.
> The basic idea here is to present a new virtual device (named eth_cvio),
> which can be discovered and initialized in container-based DPDK apps using
> rte_eal_init(). To minimize the change, we reuse already-existing virtio
> frontend driver code (driver/net/virtio/).
>  
> Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
> operations into unix socket/cuse protocol, which is originally provided in
> QEMU), is integrated in virtio frontend driver. So this converged driver
> actually plays the role of original frontend driver and the role of QEMU
> device framework.
>  
> The major difference lies in how to calculate relative address for vhost.
> The principle of virtio is that: based on one or multiple shared memory
> segments, vhost maintains a reference system with the base addresses and
> length for each segment so that an address from VM comes (usually GPA,
> Guest Physical Address) can be translated into vhost-recognizable address
> (named VVA, Vhost Virtual Address). To decrease the overhead of address
> translation, we should maintain as few segments as possible. In VM's case,
> GPA is always locally continuous. In container's case, CVA (Container
> Virtual Address) can be used. Specifically:
> a. when set_base_addr, CVA address is used;
> b. when preparing RX's descriptors, CVA address is used;
> c. when transmitting packets, CVA is filled in TX's descriptors;
> d. in TX and CQ's header, CVA is used.
>  
> How to share memory? In VM's case, qemu always shares all physical layout
> to backend. But it's not feasible for a container, as a process, to share
> all virtual memory regions to backend. So only specified virtual memory
> regions (with type of shared) are sent to backend. It's a limitation that
> only addresses in these areas can be used to transmit or receive packets.
>
> Known issues
>
> a. When used with vhost-net, root privilege is required to create tap
> device inside.
> b. Control queue and multi-queue are not supported yet.
> c. When --single-file option is used, socket_id of the memory may be
> wrong. (Use "numactl -N x -m x" to work around this for now)
>  
> How to use?
>
> a. Apply this patchset.
>
> b. To compile container apps:
> $: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
>
> c. To build a docker image using Dockerfile below.
> $: cat ./Dockerfile
> FROM ubuntu:latest
> WORKDIR /usr/src/dpdk
> COPY . /usr/src/dpdk
> ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
> $: docker build -t dpdk-app-l2fwd .
>
> d. Used with vhost-user
> $: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
> 	--socket-mem 1024,1024 -- -p 0x1 --stats 1
> $: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
> 	-v /dev/hugepages:/dev/hugepages \
> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> 	--vdev=eth_cvio0,path=/var/run/usvhost -- -p 0x1
>
> f. Used with vhost-net
> $: modprobe vhost
> $: modprobe vhost-net
> $: docker run -i -t --privileged \
> 	-v /dev/vhost-net:/dev/vhost-net \
> 	-v /dev/net/tun:/dev/net/tun \
> 	-v /dev/hugepages:/dev/hugepages \
> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> 	--vdev=eth_cvio0,path=/dev/vhost-net -- -p 0x1

We'd better add a ifname, like
--vdev=eth_cvio0,path=/dev/vhost-net,ifname=tap0, so that user could add
the tap to the bridge first.

Thanks,
Michael
>
> By the way, it's not necessary to run in a container.
>
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
>
> Jianfeng Tan (4):
>   mem: add --single-file to create single mem-backed file
>   mem: add API to obstain memory-backed file info
>   virtio/vdev: add ways to interact with vhost
>   virtio/vdev: add a new vdev named eth_cvio
>
>  config/common_linuxapp                     |   5 +
>  drivers/net/virtio/Makefile                |   4 +
>  drivers/net/virtio/vhost.c                 | 734 +++++++++++++++++++++++++++++
>  drivers/net/virtio/vhost.h                 | 192 ++++++++
>  drivers/net/virtio/virtio_ethdev.c         | 338 ++++++++++---
>  drivers/net/virtio/virtio_ethdev.h         |   4 +
>  drivers/net/virtio/virtio_pci.h            |  52 +-
>  drivers/net/virtio/virtio_rxtx.c           |  11 +-
>  drivers/net/virtio/virtio_rxtx_simple.c    |  14 +-
>  drivers/net/virtio/virtqueue.h             |  13 +-
>  lib/librte_eal/common/eal_common_options.c |  17 +
>  lib/librte_eal/common/eal_internal_cfg.h   |   1 +
>  lib/librte_eal/common/eal_options.h        |   2 +
>  lib/librte_eal/common/include/rte_memory.h |  16 +
>  lib/librte_eal/linuxapp/eal/eal_memory.c   |  82 +++-
>  15 files changed, 1392 insertions(+), 93 deletions(-)
>  create mode 100644 drivers/net/virtio/vhost.c
>  create mode 100644 drivers/net/virtio/vhost.h
>


^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 0/4] virtio support for container
  2016-01-26  6:02   ` Qiu, Michael
@ 2016-01-26  6:09     ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-01-26  6:09 UTC (permalink / raw)
  To: Qiu, Michael, dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

Hi Michael,

On 1/26/2016 2:02 PM, Qiu, Michael wrote:
> On 1/11/2016 2:43 AM, Tan, Jianfeng wrote:
...
>>
>> f. Used with vhost-net
>> $: modprobe vhost
>> $: modprobe vhost-net
>> $: docker run -i -t --privileged \
>> 	-v /dev/vhost-net:/dev/vhost-net \
>> 	-v /dev/net/tun:/dev/net/tun \
>> 	-v /dev/hugepages:/dev/hugepages \
>> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
>> 	--vdev=eth_cvio0,path=/dev/vhost-net -- -p 0x1
> We'd better add a ifname, like
> --vdev=eth_cvio0,path=/dev/vhost-net,ifname=tap0, so that user could add
> the tap to the bridge first.

That's an awesome suggestion.

Thanks,
Jianfeng

>
> Thanks,
> Michael

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH 4/4] virtio/vdev: add a new vdev named eth_cvio
  2016-01-10 11:43   ` [PATCH 4/4] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
  2016-01-12  7:45     ` Pavel Fedin
@ 2016-01-27  3:10     ` Qiu, Michael
  1 sibling, 0 replies; 196+ messages in thread
From: Qiu, Michael @ 2016-01-27  3:10 UTC (permalink / raw)
  To: Tan, Jianfeng, dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

On 1/11/2016 2:43 AM, Tan, Jianfeng wrote:
> Add a new virtual device named eth_cvio, it can be used just like
> eth_ring, eth_null, etc.
>
> Configured parameters include:
>     - rx (optional, 1 by default): number of rx, only allowed to be
> 				   1 for now.
>     - tx (optional, 1 by default): number of tx, only allowed to be
> 				   1 for now.


>From APP side, virtio is something HW, in your implementation rx/tx is
max queue numbers virtio supported. Does it make sense?

Why need user tell HW, how much queues it support? We'd better make it
un-configurable, only let users query it like the real HW, and then
decide how much queues it need to enable.


>     - cq (optional, 0 by default): if ctrl queue is enabled, not
> 				   supported for now.
>     - mac (optional): mac address, random value will be given if not
> 		      specified.
>     - queue_num (optional, 256 by default): size of virtqueue.

Better change it to queue_size.

Thanks,
Michael

>     - path (madatory): path of vhost, depends on the file type:
> 		       vhost-user is used if the given path points to
> 	               a unix socket; vhost-net is used if the given
> 		       path points to a char device.
>
> The major difference with original virtio for vm is that, here we
> use virtual address instead of physical address for vhost to
> calculate relative address.
>
> When enable CONFIG_RTE_VIRTIO_VDEV (enabled by default), the compiled
> library can be used in both VM and container environment.
>
> Examples:
> a. Use vhost-net as a backend
> sudo numactl -N 1 -m 1 ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
>     -m 1024 --no-pci --single-file --file-prefix=l2fwd \
>     --vdev=eth_cvio0,mac=00:01:02:03:04:05,path=/dev/vhost-net \
>     -- -p 0x1
>
> b. Use vhost-user as a backend
> numactl -N 1 -m 1 ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 -m 1024 \
>     --no-pci --single-file --file-prefix=l2fwd \
>     --vdev=eth_cvio0,mac=00:01:02:03:04:05,path=<path_to_vhost_user> \
>     -- -p 0x1
>
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ---
>


^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v2 0/5] virtio support for container
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
                   ` (6 preceding siblings ...)
  2016-01-10 11:42 ` [PATCH 0/4] " Jianfeng Tan
@ 2016-02-05 11:20 ` Jianfeng Tan
  2016-02-05 11:20   ` [PATCH v2 1/5] mem: add --single-file to create single mem-backed file Jianfeng Tan
                     ` (6 more replies)
  2016-04-21  2:56 ` [PATCH v3 0/2] " Jianfeng Tan
                   ` (6 subsequent siblings)
  14 siblings, 7 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-02-05 11:20 UTC (permalink / raw)
  To: dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

v1->v2:
 - Rebase on the patchset of virtio 1.0 support.
 - Fix cannot create non-hugepage memory.
 - Fix wrong size of memory region when "single-file" is used.
 - Fix setting of offset in virtqueue to use virtual address.
 - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
 - Add mac option to specify the mac address of this virtual device.
 - Update doc.

This patchset is to provide high performance networking interface (virtio)
for container-based DPDK applications. The way of starting DPDK apps in
containers with ownership of NIC devices exclusively is beyond the scope.
The basic idea here is to present a new virtual device (named eth_cvio),
which can be discovered and initialized in container-based DPDK apps using
rte_eal_init(). To minimize the change, we reuse already-existing virtio
frontend driver code (driver/net/virtio/).
 
Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
operations into unix socket/cuse protocol, which is originally provided in
QEMU), is integrated in virtio frontend driver. So this converged driver
actually plays the role of original frontend driver and the role of QEMU
device framework.
 
The major difference lies in how to calculate relative address for vhost.
The principle of virtio is that: based on one or multiple shared memory
segments, vhost maintains a reference system with the base addresses and
length for each segment so that an address from VM comes (usually GPA,
Guest Physical Address) can be translated into vhost-recognizable address
(named VVA, Vhost Virtual Address). To decrease the overhead of address
translation, we should maintain as few segments as possible. In VM's case,
GPA is always locally continuous. In container's case, CVA (Container
Virtual Address) can be used. Specifically:
a. when set_base_addr, CVA address is used;
b. when preparing RX's descriptors, CVA address is used;
c. when transmitting packets, CVA is filled in TX's descriptors;
d. in TX and CQ's header, CVA is used.
 
How to share memory? In VM's case, qemu always shares all physical layout
to backend. But it's not feasible for a container, as a process, to share
all virtual memory regions to backend. So only specified virtual memory
regions (with type of shared) are sent to backend. It's a limitation that
only addresses in these areas can be used to transmit or receive packets.

Known issues

a. When used with vhost-net, root privilege is required to create tap
device inside.
b. Control queue and multi-queue are not supported yet.
c. When --single-file option is used, socket_id of the memory may be
wrong. (Use "numactl -N x -m x" to work around this for now)
 
How to use?

a. Apply this patchset.

b. To compile container apps:
$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc

c. To build a docker image using Dockerfile below.
$: cat ./Dockerfile
FROM ubuntu:latest
WORKDIR /usr/src/dpdk
COPY . /usr/src/dpdk
ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
$: docker build -t dpdk-app-l2fwd .

d. Used with vhost-user
$: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
	--socket-mem 1024,1024 -- -p 0x1 --stats 1
$: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=eth_cvio0,path=/var/run/usvhost -- -p 0x1

f. Used with vhost-net
$: modprobe vhost
$: modprobe vhost-net
$: docker run -i -t --privileged \
	-v /dev/vhost-net:/dev/vhost-net \
	-v /dev/net/tun:/dev/net/tun \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=eth_cvio0,path=/dev/vhost-net -- -p 0x1

By the way, it's not necessary to run in a container.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>

Jianfeng Tan (5):
  mem: add --single-file to create single mem-backed file
  mem: add API to obtain memory-backed file info
  virtio/vdev: add embeded device emulation
  virtio/vdev: add a new vdev named eth_cvio
  docs: add release note for virtio for container

 config/common_linuxapp                     |   5 +
 doc/guides/rel_notes/release_2_3.rst       |   4 +
 drivers/net/virtio/Makefile                |   4 +
 drivers/net/virtio/vhost.h                 | 194 +++++++
 drivers/net/virtio/vhost_embedded.c        | 809 +++++++++++++++++++++++++++++
 drivers/net/virtio/virtio_ethdev.c         | 329 +++++++++---
 drivers/net/virtio/virtio_ethdev.h         |   6 +-
 drivers/net/virtio/virtio_pci.h            |  15 +-
 drivers/net/virtio/virtio_rxtx.c           |   6 +-
 drivers/net/virtio/virtio_rxtx_simple.c    |  13 +-
 drivers/net/virtio/virtqueue.h             |  15 +-
 lib/librte_eal/common/eal_common_options.c |  17 +
 lib/librte_eal/common/eal_internal_cfg.h   |   1 +
 lib/librte_eal/common/eal_options.h        |   2 +
 lib/librte_eal/common/include/rte_memory.h |  16 +
 lib/librte_eal/linuxapp/eal/eal.c          |   4 +-
 lib/librte_eal/linuxapp/eal/eal_memory.c   |  88 +++-
 17 files changed, 1435 insertions(+), 93 deletions(-)
 create mode 100644 drivers/net/virtio/vhost.h
 create mode 100644 drivers/net/virtio/vhost_embedded.c

-- 
2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v2 1/5] mem: add --single-file to create single mem-backed file
  2016-02-05 11:20 ` [PATCH v2 0/5] " Jianfeng Tan
@ 2016-02-05 11:20   ` Jianfeng Tan
  2016-03-07 13:13     ` Yuanhan Liu
  2016-02-05 11:20   ` [PATCH v2 2/5] mem: add API to obtain memory-backed file info Jianfeng Tan
                     ` (5 subsequent siblings)
  6 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-02-05 11:20 UTC (permalink / raw)
  To: dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

Originally, there're two cons in using hugepage: a. needs root
privilege to touch /proc/self/pagemap, which is a premise to
alllocate physically contiguous memseg; b. possibly too many
hugepage file are created, especially used with 2M hugepage.

For virtual devices, they don't care about physical-contiguity
of allocated hugepages at all. Option --single-file is to
provide a way to allocate all hugepages into single mem-backed
file.

Known issue:
a. single-file option relys on kernel to allocate numa-affinitive
memory.
b. possible ABI break, originally, --no-huge uses anonymous memory
instead of file-backed way to create memory.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 lib/librte_eal/common/eal_common_options.c | 17 ++++++++++
 lib/librte_eal/common/eal_internal_cfg.h   |  1 +
 lib/librte_eal/common/eal_options.h        |  2 ++
 lib/librte_eal/linuxapp/eal/eal.c          |  4 +--
 lib/librte_eal/linuxapp/eal/eal_memory.c   | 50 +++++++++++++++++++++++++-----
 5 files changed, 64 insertions(+), 10 deletions(-)

diff --git a/lib/librte_eal/common/eal_common_options.c b/lib/librte_eal/common/eal_common_options.c
index 29942ea..65bccbd 100644
--- a/lib/librte_eal/common/eal_common_options.c
+++ b/lib/librte_eal/common/eal_common_options.c
@@ -95,6 +95,7 @@ eal_long_options[] = {
 	{OPT_VFIO_INTR,         1, NULL, OPT_VFIO_INTR_NUM        },
 	{OPT_VMWARE_TSC_MAP,    0, NULL, OPT_VMWARE_TSC_MAP_NUM   },
 	{OPT_XEN_DOM0,          0, NULL, OPT_XEN_DOM0_NUM         },
+	{OPT_SINGLE_FILE,       0, NULL, OPT_SINGLE_FILE_NUM      },
 	{0,                     0, NULL, 0                        }
 };
 
@@ -897,6 +898,10 @@ eal_parse_common_option(int opt, const char *optarg,
 		}
 		break;
 
+	case OPT_SINGLE_FILE_NUM:
+		conf->single_file = 1;
+		break;
+
 	/* don't know what to do, leave this to caller */
 	default:
 		return 1;
@@ -956,6 +961,16 @@ eal_check_common_options(struct internal_config *internal_cfg)
 			"be specified together with --"OPT_NO_HUGE"\n");
 		return -1;
 	}
+	if (internal_cfg->single_file && internal_cfg->force_sockets == 1) {
+		RTE_LOG(ERR, EAL, "Option --"OPT_SINGLE_FILE" cannot "
+			"be specified together with --"OPT_SOCKET_MEM"\n");
+		return -1;
+	}
+	if (internal_cfg->single_file && internal_cfg->hugepage_unlink) {
+		RTE_LOG(ERR, EAL, "Option --"OPT_HUGE_UNLINK" cannot "
+			"be specified together with --"OPT_SINGLE_FILE"\n");
+		return -1;
+	}
 
 	if (internal_cfg->no_hugetlbfs && internal_cfg->hugepage_unlink) {
 		RTE_LOG(ERR, EAL, "Option --"OPT_HUGE_UNLINK" cannot "
@@ -994,6 +1009,8 @@ eal_common_usage(void)
 	       "  -n CHANNELS         Number of memory channels\n"
 	       "  -m MB               Memory to allocate (see also --"OPT_SOCKET_MEM")\n"
 	       "  -r RANKS            Force number of memory ranks (don't detect)\n"
+	       "  --"OPT_SINGLE_FILE" Create just single file for shared memory, and \n"
+	       "                      do not promise physical contiguity of memseg\n"
 	       "  -b, --"OPT_PCI_BLACKLIST" Add a PCI device in black list.\n"
 	       "                      Prevent EAL from using this PCI device. The argument\n"
 	       "                      format is <domain:bus:devid.func>.\n"
diff --git a/lib/librte_eal/common/eal_internal_cfg.h b/lib/librte_eal/common/eal_internal_cfg.h
index 5f1367e..9117ed9 100644
--- a/lib/librte_eal/common/eal_internal_cfg.h
+++ b/lib/librte_eal/common/eal_internal_cfg.h
@@ -61,6 +61,7 @@ struct hugepage_info {
  */
 struct internal_config {
 	volatile size_t memory;           /**< amount of asked memory */
+	volatile unsigned single_file;    /**< mmap all hugepages in single file */
 	volatile unsigned force_nchannel; /**< force number of channels */
 	volatile unsigned force_nrank;    /**< force number of ranks */
 	volatile unsigned no_hugetlbfs;   /**< true to disable hugetlbfs */
diff --git a/lib/librte_eal/common/eal_options.h b/lib/librte_eal/common/eal_options.h
index a881c62..e5da14a 100644
--- a/lib/librte_eal/common/eal_options.h
+++ b/lib/librte_eal/common/eal_options.h
@@ -83,6 +83,8 @@ enum {
 	OPT_VMWARE_TSC_MAP_NUM,
 #define OPT_XEN_DOM0          "xen-dom0"
 	OPT_XEN_DOM0_NUM,
+#define OPT_SINGLE_FILE       "single-file"
+	OPT_SINGLE_FILE_NUM,
 	OPT_LONG_MAX_NUM
 };
 
diff --git a/lib/librte_eal/linuxapp/eal/eal.c b/lib/librte_eal/linuxapp/eal/eal.c
index 635ec36..2bc84f7 100644
--- a/lib/librte_eal/linuxapp/eal/eal.c
+++ b/lib/librte_eal/linuxapp/eal/eal.c
@@ -790,6 +790,8 @@ rte_eal_init(int argc, char **argv)
 		rte_panic("Cannot init IVSHMEM\n");
 #endif
 
+	eal_thread_init_master(rte_config.master_lcore);
+
 	if (rte_eal_memory_init() < 0)
 		rte_panic("Cannot init memory\n");
 
@@ -823,8 +825,6 @@ rte_eal_init(int argc, char **argv)
 	if (eal_plugins_init() < 0)
 		rte_panic("Cannot init plugins\n");
 
-	eal_thread_init_master(rte_config.master_lcore);
-
 	ret = eal_thread_dump_affinity(cpuset, RTE_CPU_AFFINITY_STR_LEN);
 
 	RTE_LOG(DEBUG, EAL, "Master lcore %u is ready (tid=%x;cpuset=[%s%s])\n",
diff --git a/lib/librte_eal/linuxapp/eal/eal_memory.c b/lib/librte_eal/linuxapp/eal/eal_memory.c
index 6008533..68ef49a 100644
--- a/lib/librte_eal/linuxapp/eal/eal_memory.c
+++ b/lib/librte_eal/linuxapp/eal/eal_memory.c
@@ -1102,20 +1102,54 @@ rte_eal_hugepage_init(void)
 	/* get pointer to global configuration */
 	mcfg = rte_eal_get_configuration()->mem_config;
 
-	/* hugetlbfs can be disabled */
-	if (internal_config.no_hugetlbfs) {
-		addr = mmap(NULL, internal_config.memory, PROT_READ | PROT_WRITE,
-				MAP_PRIVATE | MAP_ANONYMOUS, 0, 0);
+	/* when hugetlbfs is disabled or single-file option is specified */
+	if (internal_config.no_hugetlbfs || internal_config.single_file) {
+		int fd;
+		uint64_t pagesize;
+		unsigned socket_id = rte_socket_id();
+		char filepath[MAX_HUGEPAGE_PATH];
+
+		if (internal_config.no_hugetlbfs) {
+			eal_get_hugefile_path(filepath, sizeof(filepath),
+					      "/dev/shm", 0);
+			pagesize = RTE_PGSIZE_4K;
+		} else {
+			struct hugepage_info *hpi;
+
+			hpi = &internal_config.hugepage_info[0];
+			eal_get_hugefile_path(filepath, sizeof(filepath),
+					      hpi->hugedir, 0);
+			pagesize = hpi->hugepage_sz;
+		}
+		fd = open(filepath, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR);
+		if (fd < 0) {
+			RTE_LOG(ERR, EAL, "%s: open %s failed: %s\n",
+				__func__, filepath, strerror(errno));
+			return -1;
+		}
+
+		if (ftruncate(fd, internal_config.memory) < 0) {
+			RTE_LOG(ERR, EAL, "ftuncate %s failed: %s\n",
+				filepath, strerror(errno));
+			return -1;
+		}
+
+		addr = mmap(NULL, internal_config.memory,
+			    PROT_READ | PROT_WRITE,
+			    MAP_SHARED | MAP_POPULATE, fd, 0);
 		if (addr == MAP_FAILED) {
-			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n", __func__,
-					strerror(errno));
+			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n",
+				__func__, strerror(errno));
 			return -1;
 		}
 		mcfg->memseg[0].phys_addr = (phys_addr_t)(uintptr_t)addr;
 		mcfg->memseg[0].addr = addr;
-		mcfg->memseg[0].hugepage_sz = RTE_PGSIZE_4K;
+		mcfg->memseg[0].hugepage_sz = pagesize;
 		mcfg->memseg[0].len = internal_config.memory;
-		mcfg->memseg[0].socket_id = 0;
+		mcfg->memseg[0].socket_id = socket_id;
+
+		close(fd);
+
 		return 0;
 	}
 
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v2 2/5] mem: add API to obtain memory-backed file info
  2016-02-05 11:20 ` [PATCH v2 0/5] " Jianfeng Tan
  2016-02-05 11:20   ` [PATCH v2 1/5] mem: add --single-file to create single mem-backed file Jianfeng Tan
@ 2016-02-05 11:20   ` Jianfeng Tan
  2016-03-07 13:22     ` Yuanhan Liu
  2016-02-05 11:20   ` [PATCH v2 3/5] virtio/vdev: add embeded device emulation Jianfeng Tan
                     ` (4 subsequent siblings)
  6 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-02-05 11:20 UTC (permalink / raw)
  To: dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

A new API named rte_eal_get_backfile_info() and a new data
struct back_file is added to obstain information of memory-
backed file info.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 lib/librte_eal/common/include/rte_memory.h | 16 ++++++++++++
 lib/librte_eal/linuxapp/eal/eal_memory.c   | 40 +++++++++++++++++++++++++++++-
 2 files changed, 55 insertions(+), 1 deletion(-)

diff --git a/lib/librte_eal/common/include/rte_memory.h b/lib/librte_eal/common/include/rte_memory.h
index 587a25d..b09397e 100644
--- a/lib/librte_eal/common/include/rte_memory.h
+++ b/lib/librte_eal/common/include/rte_memory.h
@@ -109,6 +109,22 @@ struct rte_memseg {
 } __rte_packed;
 
 /**
+ * This struct is used to store information about memory-backed file that
+ * we mapped in memory initialization.
+ */
+struct back_file {
+	void *addr;         /**< virtual addr */
+	size_t size;        /**< the page size */
+	char filepath[PATH_MAX]; /**< path to backing file on filesystem */
+};
+
+/**
+  * Get the hugepage file information. Caller to free.
+  * Return number of hugepage files used.
+  */
+int rte_eal_get_backfile_info(struct back_file **);
+
+/**
  * Lock page in physical memory and prevent from swapping.
  *
  * @param virt
diff --git a/lib/librte_eal/linuxapp/eal/eal_memory.c b/lib/librte_eal/linuxapp/eal/eal_memory.c
index 68ef49a..a6b3616 100644
--- a/lib/librte_eal/linuxapp/eal/eal_memory.c
+++ b/lib/librte_eal/linuxapp/eal/eal_memory.c
@@ -743,6 +743,9 @@ sort_by_physaddr(struct hugepage_file *hugepg_tbl, struct hugepage_info *hpi)
 	return 0;
 }
 
+static struct hugepage_file *hugepage_files;
+static int num_hugepage_files;
+
 /*
  * Uses mmap to create a shared memory area for storage of data
  * Used in this file to store the hugepage file map on disk
@@ -760,9 +763,30 @@ create_shared_memory(const char *filename, const size_t mem_size)
 	}
 	retval = mmap(NULL, mem_size, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0);
 	close(fd);
+
+	hugepage_files = retval;
+	num_hugepage_files = mem_size / (sizeof(struct hugepage_file));
+
 	return retval;
 }
 
+int
+rte_eal_get_backfile_info(struct back_file **p)
+{
+	struct back_file *backfiles;
+	int i, num_backfiles = num_hugepage_files;
+
+	backfiles = malloc(sizeof(struct back_file) * num_backfiles);
+	for (i = 0; i < num_backfiles; ++i) {
+		backfiles[i].addr = hugepage_files[i].final_va;
+		backfiles[i].size = hugepage_files[i].size;
+		strcpy(backfiles[i].filepath, hugepage_files[i].filepath);
+	}
+
+	*p = backfiles;
+	return num_backfiles;
+}
+
 /*
  * this copies *active* hugepages from one hugepage table to another.
  * destination is typically the shared memory.
@@ -1148,8 +1172,22 @@ rte_eal_hugepage_init(void)
 		mcfg->memseg[0].len = internal_config.memory;
 		mcfg->memseg[0].socket_id = socket_id;
 
-		close(fd);
+		hugepage = create_shared_memory(eal_hugepage_info_path(),
+						sizeof(struct hugepage_file));
+		hugepage->orig_va = addr;
+		hugepage->final_va = addr;
+		hugepage->physaddr = rte_mem_virt2phy(addr);
+		/* Suppose we have a very huge hugefile here */
+		hugepage->size = internal_config.memory;
+		hugepage->socket_id = socket_id;
+		hugepage->file_id = 0;
+		hugepage->memseg_id = 0;
+#ifdef RTE_EAL_SINGLE_FILE_SEGMENTS
+		hugepage->repeated = internal_config.memory / pagesize;
+#endif
+		strncpy(hugepage->filepath, filepath, MAX_HUGEPAGE_PATH);
 
+		close(fd);
 		return 0;
 	}
 
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v2 3/5] virtio/vdev: add embeded device emulation
  2016-02-05 11:20 ` [PATCH v2 0/5] " Jianfeng Tan
  2016-02-05 11:20   ` [PATCH v2 1/5] mem: add --single-file to create single mem-backed file Jianfeng Tan
  2016-02-05 11:20   ` [PATCH v2 2/5] mem: add API to obtain memory-backed file info Jianfeng Tan
@ 2016-02-05 11:20   ` Jianfeng Tan
  2016-02-07 10:47     ` Michael S. Tsirkin
                       ` (2 more replies)
  2016-02-05 11:20   ` [PATCH v2 4/5] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
                     ` (3 subsequent siblings)
  6 siblings, 3 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-02-05 11:20 UTC (permalink / raw)
  To: dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

To implement virtio vdev, we need way to interract with vhost backend.
And more importantly, needs way to emulate a device into DPDK. So this
patch acts as embedded device emulation.

Depends on the type of vhost file: vhost-user is used if the given
path points to a unix socket; vhost-net is used if the given path
points to a char device.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 config/common_linuxapp              |   5 +
 drivers/net/virtio/Makefile         |   4 +
 drivers/net/virtio/vhost.h          | 194 +++++++++
 drivers/net/virtio/vhost_embedded.c | 809 ++++++++++++++++++++++++++++++++++++
 drivers/net/virtio/virtio_ethdev.h  |   6 +-
 drivers/net/virtio/virtio_pci.h     |  15 +-
 6 files changed, 1031 insertions(+), 2 deletions(-)
 create mode 100644 drivers/net/virtio/vhost.h
 create mode 100644 drivers/net/virtio/vhost_embedded.c

diff --git a/config/common_linuxapp b/config/common_linuxapp
index 74bc515..f76e162 100644
--- a/config/common_linuxapp
+++ b/config/common_linuxapp
@@ -534,3 +534,8 @@ CONFIG_RTE_APP_TEST=y
 CONFIG_RTE_TEST_PMD=y
 CONFIG_RTE_TEST_PMD_RECORD_CORE_CYCLES=n
 CONFIG_RTE_TEST_PMD_RECORD_BURST_STATS=n
+
+#
+# Enable virtio support for container
+#
+CONFIG_RTE_VIRTIO_VDEV=y
diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index 43835ba..ef920f9 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -52,6 +52,10 @@ SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_ethdev.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx_simple.c
 
+ifeq ($(CONFIG_RTE_VIRTIO_VDEV),y)
+	SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += vhost_embedded.c
+endif
+
 # this lib depends upon:
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_eal lib/librte_ether
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_mempool lib/librte_mbuf
diff --git a/drivers/net/virtio/vhost.h b/drivers/net/virtio/vhost.h
new file mode 100644
index 0000000..73d4f5c
--- /dev/null
+++ b/drivers/net/virtio/vhost.h
@@ -0,0 +1,194 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VHOST_NET_USER_H
+#define _VHOST_NET_USER_H
+
+#include <stdint.h>
+#include <linux/types.h>
+#include <linux/ioctl.h>
+
+#define VHOST_MEMORY_MAX_NREGIONS 8
+
+struct vhost_vring_state {
+	unsigned int index;
+	unsigned int num;
+};
+
+struct vhost_vring_file {
+	unsigned int index;
+	int fd;
+};
+
+struct vhost_vring_addr {
+	unsigned int index;
+	/* Option flags. */
+	unsigned int flags;
+	/* Flag values: */
+	/* Whether log address is valid. If set enables logging. */
+#define VHOST_VRING_F_LOG 0
+
+	/* Start of array of descriptors (virtually contiguous) */
+	uint64_t desc_user_addr;
+	/* Used structure address. Must be 32 bit aligned */
+	uint64_t used_user_addr;
+	/* Available structure address. Must be 16 bit aligned */
+	uint64_t avail_user_addr;
+	/* Logging support. */
+	/* Log writes to used structure, at offset calculated from specified
+	 * address. Address must be 32 bit aligned.
+	 */
+	uint64_t log_guest_addr;
+};
+
+#define VIRTIO_CONFIG_S_DRIVER_OK   4
+
+enum vhost_user_request {
+	VHOST_USER_NONE = 0,
+	VHOST_USER_GET_FEATURES = 1,
+	VHOST_USER_SET_FEATURES = 2,
+	VHOST_USER_SET_OWNER = 3,
+	VHOST_USER_RESET_OWNER = 4,
+	VHOST_USER_SET_MEM_TABLE = 5,
+	VHOST_USER_SET_LOG_BASE = 6,
+	VHOST_USER_SET_LOG_FD = 7,
+	VHOST_USER_SET_VRING_NUM = 8,
+	VHOST_USER_SET_VRING_ADDR = 9,
+	VHOST_USER_SET_VRING_BASE = 10,
+	VHOST_USER_GET_VRING_BASE = 11,
+	VHOST_USER_SET_VRING_KICK = 12,
+	VHOST_USER_SET_VRING_CALL = 13,
+	VHOST_USER_SET_VRING_ERR = 14,
+	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
+	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
+	VHOST_USER_GET_QUEUE_NUM = 17,
+	VHOST_USER_SET_VRING_ENABLE = 18,
+	VHOST_USER_MAX
+};
+
+struct vhost_memory_region {
+	uint64_t guest_phys_addr;
+	uint64_t memory_size; /* bytes */
+	uint64_t userspace_addr;
+	uint64_t mmap_offset;
+};
+
+struct vhost_memory_kernel {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[0];
+};
+
+struct vhost_memory {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[VHOST_MEMORY_MAX_NREGIONS];
+};
+
+struct vhost_user_msg {
+	enum vhost_user_request request;
+
+#define VHOST_USER_VERSION_MASK     0x3
+#define VHOST_USER_REPLY_MASK       (0x1 << 2)
+	uint32_t flags;
+	uint32_t size; /* the following payload size */
+	union {
+#define VHOST_USER_VRING_IDX_MASK   0xff
+#define VHOST_USER_VRING_NOFD_MASK  (0x1 << 8)
+		uint64_t u64;
+		struct vhost_vring_state state;
+		struct vhost_vring_addr addr;
+		struct vhost_memory memory;
+	} payload;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+} __attribute((packed));
+
+#define VHOST_USER_HDR_SIZE offsetof(struct vhost_user_msg, payload.u64)
+#define VHOST_USER_PAYLOAD_SIZE (sizeof(struct vhost_user_msg) - VHOST_USER_HDR_SIZE)
+
+/* The version of the protocol we support */
+#define VHOST_USER_VERSION    0x1
+
+/* ioctls */
+
+#define VHOST_VIRTIO 0xAF
+
+#define VHOST_GET_FEATURES _IOR(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_FEATURES _IOW(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_OWNER _IO(VHOST_VIRTIO, 0x01)
+#define VHOST_RESET_OWNER _IO(VHOST_VIRTIO, 0x02)
+#define VHOST_SET_MEM_TABLE _IOW(VHOST_VIRTIO, 0x03, struct vhost_memory_kernel)
+#define VHOST_SET_LOG_BASE _IOW(VHOST_VIRTIO, 0x04, __u64)
+#define VHOST_SET_LOG_FD _IOW(VHOST_VIRTIO, 0x07, int)
+#define VHOST_SET_VRING_NUM _IOW(VHOST_VIRTIO, 0x10, struct vhost_vring_state)
+#define VHOST_SET_VRING_ADDR _IOW(VHOST_VIRTIO, 0x11, struct vhost_vring_addr)
+#define VHOST_SET_VRING_BASE _IOW(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_GET_VRING_BASE _IOWR(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_SET_VRING_KICK _IOW(VHOST_VIRTIO, 0x20, struct vhost_vring_file)
+#define VHOST_SET_VRING_CALL _IOW(VHOST_VIRTIO, 0x21, struct vhost_vring_file)
+#define VHOST_SET_VRING_ERR _IOW(VHOST_VIRTIO, 0x22, struct vhost_vring_file)
+#define VHOST_NET_SET_BACKEND _IOW(VHOST_VIRTIO, 0x30, struct vhost_vring_file)
+
+/*****************************************************************************/
+
+/* Ioctl defines */
+#define TUNSETIFF     _IOW('T', 202, int)
+#define TUNGETFEATURES _IOR('T', 207, unsigned int)
+#define TUNSETOFFLOAD  _IOW('T', 208, unsigned int)
+#define TUNGETIFF      _IOR('T', 210, unsigned int)
+#define TUNSETSNDBUF   _IOW('T', 212, int)
+#define TUNGETVNETHDRSZ _IOR('T', 215, int)
+#define TUNSETVNETHDRSZ _IOW('T', 216, int)
+#define TUNSETQUEUE  _IOW('T', 217, int)
+#define TUNSETVNETLE _IOW('T', 220, int)
+#define TUNSETVNETBE _IOW('T', 222, int)
+
+/* TUNSETIFF ifr flags */
+#define IFF_TAP          0x0002
+#define IFF_NO_PI        0x1000
+#define IFF_ONE_QUEUE    0x2000
+#define IFF_VNET_HDR     0x4000
+#define IFF_MULTI_QUEUE  0x0100
+#define IFF_ATTACH_QUEUE 0x0200
+#define IFF_DETACH_QUEUE 0x0400
+
+/* Features for GSO (TUNSETOFFLOAD). */
+#define TUN_F_CSUM	0x01	/* You can hand me unchecksummed packets. */
+#define TUN_F_TSO4	0x02	/* I can handle TSO for IPv4 packets */
+#define TUN_F_TSO6	0x04	/* I can handle TSO for IPv6 packets */
+#define TUN_F_TSO_ECN	0x08	/* I can handle TSO with ECN bits. */
+#define TUN_F_UFO	0x10	/* I can handle UFO packets */
+
+#define PATH_NET_TUN "/dev/net/tun"
+
+#endif
diff --git a/drivers/net/virtio/vhost_embedded.c b/drivers/net/virtio/vhost_embedded.c
new file mode 100644
index 0000000..0073b86
--- /dev/null
+++ b/drivers/net/virtio/vhost_embedded.c
@@ -0,0 +1,809 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <sys/types.h>
+#include <sys/socket.h>
+#include <sys/un.h>
+#include <stdio.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <string.h>
+#include <errno.h>
+#include <assert.h>
+#include <sys/mman.h>
+#include <unistd.h>
+#include <sys/eventfd.h>
+#include <sys/ioctl.h>
+#include <net/if.h>
+
+#include <rte_mbuf.h>
+#include <rte_memory.h>
+#include <rte_eal_memconfig.h>
+
+#include "virtio_pci.h"
+#include "virtio_logs.h"
+#include "virtio_ethdev.h"
+#include "virtqueue.h"
+#include "vhost.h"
+
+static int
+vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
+{
+	int r;
+	struct msghdr msgh;
+	struct iovec iov;
+	size_t fd_size = fd_num * sizeof(int);
+	char control[CMSG_SPACE(fd_size)];
+	struct cmsghdr *cmsg;
+
+	bzero(&msgh, sizeof(msgh));
+	bzero(control, sizeof(control));
+
+	iov.iov_base = (uint8_t *)buf;
+	iov.iov_len = len;
+
+	msgh.msg_iov = &iov;
+	msgh.msg_iovlen = 1;
+	msgh.msg_control = control;
+	msgh.msg_controllen = sizeof(control);
+
+	cmsg = CMSG_FIRSTHDR(&msgh);
+	cmsg->cmsg_len = CMSG_LEN(fd_size);
+	cmsg->cmsg_level = SOL_SOCKET;
+	cmsg->cmsg_type = SCM_RIGHTS;
+	memcpy(CMSG_DATA(cmsg), fds, fd_size);
+
+	do {
+		r = sendmsg(fd, &msgh, 0);
+	} while (r < 0 && errno == EINTR);
+
+	return r;
+}
+
+static int
+vhost_user_read(int fd, struct vhost_user_msg *msg)
+{
+	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
+	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
+
+	ret = recv(fd, (void *)msg, sz_hdr, 0);
+	if (ret < sz_hdr) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
+			    ret, sz_hdr);
+		goto fail;
+	}
+
+	/* validate msg flags */
+	if (msg->flags != (valid_flags)) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg: flags %x instead of %x.",
+			    msg->flags, valid_flags);
+		goto fail;
+	}
+
+	sz_payload = msg->size;
+	if (sz_payload) {
+		ret = recv(fd, (void *)((char *)msg + sz_hdr), sz_payload, 0);
+		if (ret < sz_payload) {
+			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
+				    ret, msg->size);
+			goto fail;
+		}
+	}
+
+	return 0;
+
+fail:
+	return -1;
+}
+
+static struct vhost_user_msg m __rte_unused;
+
+static void
+prepare_vhost_memory_user(struct vhost_user_msg *msg, int fds[])
+{
+	int i, num;
+	struct back_file *huges;
+	struct vhost_memory_region *mr;
+
+	num = rte_eal_get_backfile_info(&huges);
+
+	if (num > VHOST_MEMORY_MAX_NREGIONS)
+		rte_panic("%d files exceed maximum of %d for vhost-user\n",
+			  num, VHOST_MEMORY_MAX_NREGIONS);
+
+	for (i = 0; i < num; ++i) {
+		mr = &msg->payload.memory.regions[i];
+		mr->guest_phys_addr = (uint64_t)huges[i].addr; /* use vaddr! */
+		mr->userspace_addr = (uint64_t)huges[i].addr;
+		mr->memory_size = huges[i].size;
+		mr->mmap_offset = 0;
+		fds[i] = open(huges[i].filepath, O_RDWR);
+	}
+
+	msg->payload.memory.nregions = num;
+	msg->payload.memory.padding = 0;
+	free(huges);
+}
+
+static int
+vhost_user_sock(struct virtio_hw *hw, unsigned long int req, void *arg)
+{
+	struct vhost_user_msg msg;
+	struct vhost_vring_file *file = 0;
+	int need_reply = 0;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+	int fd_num = 0;
+	int i, len;
+
+	msg.request = req;
+	msg.flags = VHOST_USER_VERSION;
+	msg.size = 0;
+
+	switch (req) {
+	case VHOST_USER_GET_FEATURES:
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_FEATURES:
+	case VHOST_USER_SET_LOG_BASE:
+		msg.payload.u64 = *((__u64 *)arg);
+		msg.size = sizeof(m.payload.u64);
+		break;
+
+	case VHOST_USER_SET_OWNER:
+	case VHOST_USER_RESET_OWNER:
+		break;
+
+	case VHOST_USER_SET_MEM_TABLE:
+		prepare_vhost_memory_user(&msg, fds);
+		fd_num = msg.payload.memory.nregions;
+		msg.size = sizeof(m.payload.memory.nregions);
+		msg.size += sizeof(m.payload.memory.padding);
+		msg.size += fd_num * sizeof(struct vhost_memory_region);
+		break;
+
+	case VHOST_USER_SET_LOG_FD:
+		fds[fd_num++] = *((int *)arg);
+		break;
+
+	case VHOST_USER_SET_VRING_NUM:
+	case VHOST_USER_SET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		break;
+
+	case VHOST_USER_GET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_VRING_ADDR:
+		memcpy(&msg.payload.addr, arg, sizeof(msg.payload.addr));
+		msg.size = sizeof(m.payload.addr);
+		break;
+
+	case VHOST_USER_SET_VRING_KICK:
+	case VHOST_USER_SET_VRING_CALL:
+	case VHOST_USER_SET_VRING_ERR:
+		file = arg;
+		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
+		msg.size = sizeof(m.payload.u64);
+		if (file->fd > 0)
+			fds[fd_num++] = file->fd;
+		else
+			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
+		break;
+
+	default:
+		PMD_DRV_LOG(ERR, "vhost-user trying to send unhandled msg type");
+		return -1;
+	}
+
+	len = VHOST_USER_HDR_SIZE + msg.size;
+	if (vhost_user_write(hw->vhostfd, &msg, len, fds, fd_num) < 0)
+		return 0;
+
+	if (req == VHOST_USER_SET_MEM_TABLE)
+		for (i = 0; i < fd_num; ++i)
+			close(fds[i]);
+
+	if (need_reply) {
+		if (vhost_user_read(hw->vhostfd, &msg) < 0)
+			return -1;
+
+		if (req != msg.request) {
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+
+		switch (req) {
+		case VHOST_USER_GET_FEATURES:
+			if (msg.size != sizeof(m.payload.u64)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			*((__u64 *)arg) = msg.payload.u64;
+			break;
+		case VHOST_USER_GET_VRING_BASE:
+			if (msg.size != sizeof(m.payload.state)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			memcpy(arg, &msg.payload.state,
+			       sizeof(struct vhost_vring_state));
+			break;
+		default:
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+	}
+
+	return 0;
+}
+
+static int
+vhost_kernel_ioctl(struct virtio_hw *hw, unsigned long int req, void *arg)
+{
+	return ioctl(hw->vhostfd, req, arg);
+}
+
+enum {
+	VHOST_MSG_SET_OWNER,
+	VHOST_MSG_SET_FEATURES,
+	VHOST_MSG_GET_FEATURES,
+	VHOST_MSG_SET_VRING_CALL,
+	VHOST_MSG_SET_VRING_NUM,
+	VHOST_MSG_SET_VRING_BASE,
+	VHOST_MSG_GET_VRING_BASE,
+	VHOST_MSG_SET_VRING_ADDR,
+	VHOST_MSG_SET_VRING_KICK,
+	VHOST_MSG_SET_MEM_TABLE,
+	VHOST_MSG_MAX,
+};
+
+static const char * const vhost_msg_strings[] = {
+	[VHOST_MSG_SET_OWNER] = "VHOST_MSG_SET_OWNER",
+	[VHOST_MSG_SET_FEATURES] = "VHOST_MSG_SET_FEATURES",
+	[VHOST_MSG_GET_FEATURES] = "VHOST_MSG_GET_FEATURES",
+	[VHOST_MSG_SET_VRING_CALL] = "VHOST_MSG_SET_VRING_CALL",
+	[VHOST_MSG_SET_VRING_NUM] = "VHOST_MSG_SET_VRING_NUM",
+	[VHOST_MSG_SET_VRING_BASE] = "VHOST_MSG_SET_VRING_BASE",
+	[VHOST_MSG_GET_VRING_BASE] = "VHOST_MSG_GET_VRING_BASE",
+	[VHOST_MSG_SET_VRING_ADDR] = "VHOST_MSG_SET_VRING_ADDR",
+	[VHOST_MSG_SET_VRING_KICK] = "VHOST_MSG_SET_VRING_KICK",
+	[VHOST_MSG_SET_MEM_TABLE] = "VHOST_MSG_SET_MEM_TABLE",
+	NULL,
+};
+
+static unsigned long int vhost_req_map[][2] = {
+	[VHOST_MSG_SET_OWNER] = {
+		VHOST_SET_OWNER, VHOST_USER_SET_OWNER
+	},
+	[VHOST_MSG_SET_FEATURES] = {
+		VHOST_SET_FEATURES, VHOST_USER_SET_FEATURES
+	},
+	[VHOST_MSG_GET_FEATURES] = {
+		VHOST_GET_FEATURES, VHOST_USER_GET_FEATURES
+	},
+	[VHOST_MSG_SET_VRING_CALL] = {
+		VHOST_SET_VRING_CALL, VHOST_USER_SET_VRING_CALL
+	},
+	[VHOST_MSG_SET_VRING_NUM] = {
+		VHOST_SET_VRING_NUM, VHOST_USER_SET_VRING_NUM
+	},
+	[VHOST_MSG_SET_VRING_BASE] = {
+		VHOST_SET_VRING_BASE, VHOST_USER_SET_VRING_BASE
+	},
+	[VHOST_MSG_GET_VRING_BASE] = {
+		VHOST_GET_VRING_BASE, VHOST_USER_GET_VRING_BASE
+	},
+	[VHOST_MSG_SET_VRING_ADDR] = {
+		VHOST_SET_VRING_ADDR, VHOST_USER_SET_VRING_ADDR
+	},
+	[VHOST_MSG_SET_VRING_KICK] = {
+		VHOST_SET_VRING_KICK, VHOST_USER_SET_VRING_KICK
+	},
+	[VHOST_MSG_SET_MEM_TABLE] = {
+		VHOST_SET_MEM_TABLE, VHOST_USER_SET_MEM_TABLE
+	},
+};
+
+static int
+vhost_call(struct virtio_hw *hw, unsigned long int req_orig, void *arg)
+{
+	unsigned long int req_new;
+	int ret;
+
+	if (req_orig >= VHOST_MSG_MAX)
+		rte_panic("invalid req: %lu\n", req_orig);
+
+	PMD_DRV_LOG(INFO, "%s\n", vhost_msg_strings[req_orig]);
+	req_new = vhost_req_map[req_orig][hw->type];
+	if (hw->type == VHOST_USER)
+		ret = vhost_user_sock(hw, req_new, arg);
+	else
+		ret = vhost_kernel_ioctl(hw, req_new, arg);
+
+	if (ret < 0)
+		rte_panic("vhost_call %s failed: %s\n",
+			  vhost_msg_strings[req_orig], strerror(errno));
+
+	return ret;
+}
+
+static void
+kick_one_vq(struct virtio_hw *hw, struct virtqueue *vq, unsigned queue_sel)
+{
+	int callfd, kickfd;
+	struct vhost_vring_file file;
+	struct vhost_vring_state state;
+	struct vhost_vring_addr addr = {
+		.index = queue_sel,
+		.desc_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.desc,
+		.avail_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.avail,
+		.used_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.used,
+		.log_guest_addr = 0,
+		.flags = 0, /* disable log */
+	};
+
+	/* or use invalid flag to disable it, but vhost-dpdk uses this to judge
+	 * if dev is alive. so finally we need two real event_fds.
+	 */
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL come
+	 * firstly because vhost depends on this msg to allocate virtqueue
+	 * pair.
+	 */
+	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (callfd < 0)
+		rte_panic("callfd error, %s\n", strerror(errno));
+
+	file.index = queue_sel;
+	file.fd = callfd;
+	vhost_call(hw, VHOST_MSG_SET_VRING_CALL, &file);
+	hw->callfds[queue_sel] = callfd;
+
+	state.index = queue_sel;
+	state.num = vq->vq_ring.num;
+	vhost_call(hw, VHOST_MSG_SET_VRING_NUM, &state);
+
+	state.num = 0; /* no reservation */
+	vhost_call(hw, VHOST_MSG_SET_VRING_BASE, &state);
+
+	vhost_call(hw, VHOST_MSG_SET_VRING_ADDR, &addr);
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_KICK comes
+	 * lastly because vhost depends on this msg to judge if
+	 * virtio_is_ready().
+	 */
+	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (kickfd < 0)
+		rte_panic("kickfd error, %s\n", strerror(errno));
+
+	file.fd = kickfd;
+	vhost_call(hw, VHOST_MSG_SET_VRING_KICK, &file);
+	hw->kickfds[queue_sel] = kickfd;
+}
+
+/**
+ * Merge those virtually adjacent memsegs into one region.
+ */
+static void
+prepare_vhost_memory_kernel(struct vhost_memory_kernel **p_vm)
+{
+	unsigned i, j, k = 0;
+	struct rte_memseg *seg;
+	struct vhost_memory_region *mr;
+	struct vhost_memory_kernel *vm;
+
+	vm = malloc(sizeof(struct vhost_memory_kernel) +
+		    RTE_MAX_MEMSEG * sizeof(struct vhost_memory_region));
+
+	for (i = 0; i < RTE_MAX_MEMSEG; ++i) {
+		seg = &rte_eal_get_configuration()->mem_config->memseg[i];
+		if (!seg->addr)
+			break;
+
+		int new_region = 1;
+
+		for (j = 0; j < k; ++j) {
+			mr = &vm->regions[j];
+
+			if (mr->userspace_addr + mr->memory_size ==
+			    (uint64_t)seg->addr) {
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+
+			if ((uint64_t)seg->addr + seg->len ==
+			    mr->userspace_addr) {
+				mr->guest_phys_addr = (uint64_t)seg->addr;
+				mr->userspace_addr = (uint64_t)seg->addr;
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+		}
+
+		if (new_region == 0)
+			continue;
+
+		mr = &vm->regions[k++];
+		mr->guest_phys_addr = (uint64_t)seg->addr; /* use vaddr here! */
+		mr->userspace_addr = (uint64_t)seg->addr;
+		mr->memory_size = seg->len;
+		mr->mmap_offset = 0;
+	}
+
+	vm->nregions = k;
+	vm->padding = 0;
+	*p_vm = vm;
+}
+
+static void kick_all_vq(struct virtio_hw *hw)
+{
+	uint64_t features;
+	unsigned i, queue_sel, nvqs;
+	struct rte_eth_dev_data *data = hw->data;
+
+	if (hw->type == VHOST_KERNEL) {
+		struct vhost_memory_kernel *vm = NULL;
+
+		prepare_vhost_memory_kernel(&vm);
+		vhost_call(hw, VHOST_MSG_SET_MEM_TABLE, vm);
+		free(vm);
+	} else {
+		/* construct vhost_memory inside prepare_vhost_memory_user() */
+		vhost_call(hw, VHOST_MSG_SET_MEM_TABLE, NULL);
+	}
+
+	for (i = 0; i < data->nb_rx_queues; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
+		kick_one_vq(hw, data->rx_queues[i], queue_sel);
+	}
+	for (i = 0; i < data->nb_tx_queues; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
+		kick_one_vq(hw, data->tx_queues[i], queue_sel);
+	}
+
+	/* after setup all virtqueues, we need to set_features again
+	 * so that these features can be set into each virtqueue in
+	 * vhost side.
+	 */
+	features = hw->guest_features;
+	features &= ~(1ull << VIRTIO_NET_F_MAC);
+	vhost_call(hw, VHOST_MSG_SET_FEATURES, &features);
+	if (hw->type == VHOST_KERNEL)
+		if (ioctl(hw->backfd, TUNSETVNETHDRSZ,
+			  &hw->vtnet_hdr_size) == -1)
+			rte_panic("TUNSETVNETHDRSZ failed: %s\n",
+				  strerror(errno));
+	PMD_DRV_LOG(INFO, "set features:%" PRIx64 "\n", features);
+
+	if (hw->type == VHOST_KERNEL) {
+		struct vhost_vring_file file;
+
+		file.fd = hw->backfd;
+		nvqs = data->nb_rx_queues + data->nb_tx_queues;
+		for (file.index = 0; file.index < nvqs; ++file.index) {
+			if (vhost_kernel_ioctl(hw, VHOST_NET_SET_BACKEND,
+					       &file) < 0)
+				rte_panic("VHOST_NET_SET_BACKEND failed, %s\n",
+					  strerror(errno));
+		}
+	}
+}
+
+static void
+vdev_read_dev_config(struct virtio_hw *hw, uint64_t offset,
+		     void *dst, int length)
+{
+	if (offset == offsetof(struct virtio_net_config, mac) &&
+	    length == ETHER_ADDR_LEN) {
+		int i;
+
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			((uint8_t *)dst)[i] = hw->mac_addr[i];
+		return;
+	}
+
+	if (offset == offsetof(struct virtio_net_config, status))
+		*(uint16_t *)dst = hw->status;
+
+	if (offset == offsetof(struct virtio_net_config, max_virtqueue_pairs))
+		*(uint16_t *)dst = hw->max_tx_queues;
+}
+
+static void
+vdev_write_dev_config(struct virtio_hw *hw, uint64_t offset,
+		      const void *src, int length)
+{
+	int i;
+
+	if ((offset == offsetof(struct virtio_net_config, mac)) &&
+	    (length == ETHER_ADDR_LEN))
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			hw->mac_addr[i] = ((const uint8_t *)src)[i];
+	else
+		rte_panic("offset=%" PRIu64 ", length=%d\n", offset, length);
+}
+
+static void
+vdev_set_status(struct virtio_hw *hw, uint8_t status)
+{
+	if (status & VIRTIO_CONFIG_S_DRIVER_OK)
+		kick_all_vq(hw);
+	hw->status = status;
+}
+
+static void
+vdev_reset(struct virtio_hw *hw __rte_unused)
+{
+	/* do nothing according to qemu vhost user spec */
+}
+
+static uint8_t
+vdev_get_status(struct virtio_hw *hw)
+{
+	return hw->status;
+}
+
+static uint64_t
+vdev_get_features(struct virtio_hw *hw)
+{
+	uint64_t host_features;
+
+	vhost_call(hw, VHOST_MSG_GET_FEATURES, &host_features);
+	if (hw->mac_specified)
+		host_features |= (1ull << VIRTIO_NET_F_MAC);
+	/* disable it until we support CQ */
+	host_features &= ~(1ull << VIRTIO_NET_F_CTRL_VQ);
+	host_features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
+	return host_features;
+}
+
+static void
+vdev_set_features(struct virtio_hw *hw, uint64_t features)
+{
+	features &= ~(1ull << VIRTIO_NET_F_MAC);
+	vhost_call(hw, VHOST_MSG_SET_FEATURES, &features);
+}
+
+static uint8_t
+vdev_get_isr(struct virtio_hw *hw __rte_unused)
+{
+	rte_panic("");
+}
+
+static uint16_t
+vdev_set_config_irq(struct virtio_hw *hw __rte_unused,
+		    uint16_t vec __rte_unused)
+{
+	rte_panic("");
+}
+
+static uint16_t
+vdev_get_queue_num(struct virtio_hw *hw,
+		   uint16_t queue_id __rte_unused)
+{
+	return hw->queue_num;
+}
+
+static void
+vdev_setup_queue(struct virtio_hw *hw __rte_unused,
+		 struct virtqueue *vq __rte_unused)
+{
+	/* do nothing */
+}
+
+static void
+vdev_del_queue(struct virtio_hw *hw __rte_unused,
+	       struct virtqueue *vq)
+{
+	struct vhost_vring_state state = {
+		.index = vq->vq_queue_index,
+	};
+
+	vhost_call(hw, VHOST_MSG_GET_VRING_BASE, &state);
+	PMD_DRV_LOG(DEBUG, "state.num = %d\n", state.num);
+}
+
+static void
+vdev_notify_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	uint64_t buf = 1;
+
+	if (write(hw->kickfds[vq->vq_queue_index],
+		  &buf, sizeof(uint64_t)) == -1)
+		rte_panic("%s\n", strerror(errno));
+}
+
+static const struct virtio_pci_ops vdev_ops = {
+	.read_dev_cfg	= vdev_read_dev_config,
+	.write_dev_cfg	= vdev_write_dev_config,
+	.reset		= vdev_reset,
+	.get_status	= vdev_get_status,
+	.set_status	= vdev_set_status,
+	.get_features	= vdev_get_features,
+	.set_features	= vdev_set_features,
+	.get_isr	= vdev_get_isr,
+	.set_config_irq	= vdev_set_config_irq,
+	.get_queue_num	= vdev_get_queue_num,
+	.setup_queue	= vdev_setup_queue,
+	.del_queue	= vdev_del_queue,
+	.notify_queue	= vdev_notify_queue,
+};
+
+#define TUN_DEF_SNDBUF	(1ull << 20)
+
+static void
+vhost_kernel_backend_setup(struct virtio_hw *hw, char *ifname)
+{
+	int fd;
+	int len = sizeof(struct virtio_net_hdr);
+	int req_mq = 0;
+	int sndbuf = TUN_DEF_SNDBUF;
+	unsigned int features;
+	struct ifreq ifr;
+
+	/* TODO:
+	 * 1. get/set offload capability, tap_probe_has_ufo, tap_fd_set_offload
+	 * 2. verify we can get/set vnet_hdr_len, tap_probe_vnet_hdr_len
+	 * 3. get number of memory regions from vhost module parameter
+	 * max_mem_regions, supported in newer version linux kernel
+	 */
+
+	fd = open(PATH_NET_TUN, O_RDWR);
+	if (fd < 0)
+		rte_panic("open %s error, %s\n", PATH_NET_TUN, strerror(errno));
+
+	memset(&ifr, 0, sizeof(ifr));
+	ifr.ifr_flags = IFF_TAP | IFF_NO_PI;
+
+	if (ioctl(fd, TUNGETFEATURES, &features) == -1)
+		rte_panic("TUNGETFEATURES failed: %s", strerror(errno));
+
+	if (features & IFF_ONE_QUEUE)
+		ifr.ifr_flags |= IFF_ONE_QUEUE;
+
+	if (features & IFF_VNET_HDR)
+		ifr.ifr_flags |= IFF_VNET_HDR;
+	else
+		rte_panic("vnet_hdr requested, but kernel does not support\n");
+
+	if (req_mq) {
+		if (features & IFF_MULTI_QUEUE)
+			ifr.ifr_flags |= IFF_MULTI_QUEUE;
+		else
+			rte_panic("multiqueue requested, but kernel does not support\n");
+	}
+
+	if (ifname)
+		strncpy(ifr.ifr_name, ifname, IFNAMSIZ);
+	else
+		strncpy(ifr.ifr_name, "tap%d", IFNAMSIZ);
+	if (ioctl(fd, TUNSETIFF, (void *)&ifr) == -1)
+		rte_panic("TUNSETIFF failed: %s", strerror(errno));
+	fcntl(fd, F_SETFL, O_NONBLOCK);
+
+	if (ioctl(fd, TUNSETVNETHDRSZ, &len) == -1)
+		rte_panic("TUNSETVNETHDRSZ failed: %s\n", strerror(errno));
+
+	if (ioctl(fd, TUNSETSNDBUF, &sndbuf) == -1)
+		rte_panic("TUNSETSNDBUF failed: %s", strerror(errno));
+
+	hw->backfd = fd;
+	hw->vhostfd = open(hw->path, O_RDWR);
+	if (hw->vhostfd < 0)
+		rte_panic("open %s failed: %s\n", hw->path, strerror(errno));
+}
+
+static void
+vhost_user_backend_setup(struct virtio_hw *hw)
+{
+	int fd;
+	int flag;
+	struct sockaddr_un un;
+
+	fd = socket(AF_UNIX, SOCK_STREAM, 0);
+	if (fd < 0)
+		rte_panic("socket error, %s\n", strerror(errno));
+
+	flag = fcntl(fd, F_GETFD);
+	fcntl(fd, F_SETFD, flag | FD_CLOEXEC);
+
+	memset(&un, 0, sizeof(un));
+	un.sun_family = AF_UNIX;
+	snprintf(un.sun_path, sizeof(un.sun_path), "%s", hw->path);
+	if (connect(fd, (struct sockaddr *)&un, sizeof(un)) < 0) {
+		PMD_DRV_LOG(ERR, "connect error, %s\n", strerror(errno));
+		rte_panic("connect error, %s\n", strerror(errno));
+	}
+
+	hw->vhostfd = fd;
+}
+
+void
+virtio_vdev_init(struct rte_eth_dev_data *data, char *path,
+		 int nb_rx, int nb_tx, int nb_cq __attribute__ ((unused)),
+		 int queue_num, char *mac, char *ifname)
+{
+	int i, r;
+	struct stat s;
+	uint32_t tmp[ETHER_ADDR_LEN];
+	struct virtio_hw *hw = data->dev_private;
+
+	hw->vtpci_ops = &vdev_ops;
+	hw->io_base  = 0;
+	hw->use_msix = 0;
+	hw->modern   = 0;
+
+	hw->data = data;
+	hw->path = strdup(path);
+	hw->max_rx_queues = nb_rx;
+	hw->max_tx_queues = nb_tx;
+	hw->queue_num = queue_num;
+	hw->mac_specified = 0;
+	if (mac) {
+		r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
+			   &tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
+		if (r == ETHER_ADDR_LEN) {
+			for (i = 0; i < ETHER_ADDR_LEN; ++i)
+				hw->mac_addr[i] = (uint8_t)tmp[i];
+			hw->mac_specified = 1;
+		} else
+			PMD_DRV_LOG(WARN, "wrong format of mac: %s", mac);
+	}
+
+	/* TODO: cq */
+
+	if (stat(hw->path, &s) < 0)
+		rte_panic("stat: %s failed, %s\n", hw->path, strerror(errno));
+
+	switch (s.st_mode & S_IFMT) {
+	case S_IFCHR:
+		hw->type = VHOST_KERNEL;
+		vhost_kernel_backend_setup(hw, ifname);
+		break;
+	case S_IFSOCK:
+		hw->type = VHOST_USER;
+		vhost_user_backend_setup(hw);
+		break;
+	default:
+		rte_panic("unknown file type of %s\n", hw->path);
+	}
+	if (vhost_call(hw, VHOST_MSG_SET_OWNER, NULL) == -1)
+		rte_panic("vhost set_owner failed: %s\n", strerror(errno));
+}
diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
index fed9571..fde77ca 100644
--- a/drivers/net/virtio/virtio_ethdev.h
+++ b/drivers/net/virtio/virtio_ethdev.h
@@ -123,5 +123,9 @@ uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 #define VTNET_LRO_FEATURES (VIRTIO_NET_F_GUEST_TSO4 | \
 			    VIRTIO_NET_F_GUEST_TSO6 | VIRTIO_NET_F_GUEST_ECN)
 
-
+#ifdef RTE_VIRTIO_VDEV
+void virtio_vdev_init(struct rte_eth_dev_data *data, char *path, int nb_rx,
+		      int nb_tx, int nb_cq, int queue_num, char *mac,
+		      char *ifname);
+#endif
 #endif /* _VIRTIO_ETHDEV_H_ */
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index 0544a07..a8394f8 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -150,7 +150,6 @@ struct virtqueue;
  * rest are per-device feature bits.
  */
 #define VIRTIO_TRANSPORT_F_START 28
-#define VIRTIO_TRANSPORT_F_END   32
 
 /* The Guest publishes the used index for which it expects an interrupt
  * at the end of the avail ring. Host should ignore the avail->flags field. */
@@ -266,6 +265,20 @@ struct virtio_hw {
 	struct virtio_pci_common_cfg *common_cfg;
 	struct virtio_net_config *dev_cfg;
 	const struct virtio_pci_ops *vtpci_ops;
+#ifdef RTE_VIRTIO_VDEV
+#define VHOST_KERNEL	0
+#define VHOST_USER	1
+	int		type; /* type of backend */
+	uint32_t	queue_num;
+	char		*path;
+	int		mac_specified;
+	int		vhostfd;
+	int		backfd; /* tap device used in vhost-net */
+	int		callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	uint8_t		status;
+	struct rte_eth_dev_data *data;
+#endif
 };
 
 /*
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v2 4/5] virtio/vdev: add a new vdev named eth_cvio
  2016-02-05 11:20 ` [PATCH v2 0/5] " Jianfeng Tan
                     ` (2 preceding siblings ...)
  2016-02-05 11:20   ` [PATCH v2 3/5] virtio/vdev: add embeded device emulation Jianfeng Tan
@ 2016-02-05 11:20   ` Jianfeng Tan
  2016-02-05 11:20   ` [PATCH v2 5/5] docs: add release note for virtio for container Jianfeng Tan
                     ` (2 subsequent siblings)
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-02-05 11:20 UTC (permalink / raw)
  To: dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

Add a new virtual device named eth_cvio, it can be used just like
eth_ring, eth_null, etc.

Configured parameters include:
  - rx (optional, 1 by default), number of rx, not used for now.
  - tx (optional, 1 by default), number of tx, not used for now.
  - cq (optional, 0 by default), if CQ is enabled, not supported for now.
  - mac (optional), random value will be given if not specified.
  - queue_num (optional, 256 by default), size of virtqueue.
  - path (madatory), path of vhost, depends on the file type, vhost-user
  if the given path points to a unix socket; vhost-net if the given
  path points to a char device.
  - ifname (optional), specify the name of backend tap device; only valid
  when backend is vhost-net.

The major difference with original virtio for vm is that, here we use
virtual addr instead of physical addr for vhost to calculate relative
address.

When enable CONFIG_RTE_VIRTIO_VDEV (enabled by default), the compiled
library can be used in both VM and container environment.

Examples:
a. Use vhost-net as a backend
sudo numactl -N 1 -m 1 ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
    -m 1024 --no-pci --single-file --file-prefix=l2fwd \
    --vdev=eth_cvio0,mac=00:01:02:03:04:05,path=/dev/vhost-net \
    -- -p 0x1

b. Use vhost-user as a backend
numactl -N 1 -m 1 ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 -m 1024 \
    --no-pci --single-file --file-prefix=l2fwd \
    --vdev=eth_cvio0,mac=00:01:02:03:04:05,path=<path_to_vhost_user> \
    -- -p 0x1

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 drivers/net/virtio/virtio_ethdev.c      | 329 +++++++++++++++++++++++++-------
 drivers/net/virtio/virtio_rxtx.c        |   6 +-
 drivers/net/virtio/virtio_rxtx_simple.c |  13 +-
 drivers/net/virtio/virtqueue.h          |  15 +-
 4 files changed, 282 insertions(+), 81 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 755503d..b790fd0 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -52,6 +52,7 @@
 #include <rte_memory.h>
 #include <rte_eal.h>
 #include <rte_dev.h>
+#include <rte_kvargs.h>
 
 #include "virtio_ethdev.h"
 #include "virtio_pci.h"
@@ -170,14 +171,14 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	 * One RX packet for ACK.
 	 */
 	vq->vq_ring.desc[head].flags = VRING_DESC_F_NEXT;
-	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mz->phys_addr;
+	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mem;
 	vq->vq_ring.desc[head].len = sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_free_cnt--;
 	i = vq->vq_ring.desc[head].next;
 
 	for (k = 0; k < pkt_num; k++) {
 		vq->vq_ring.desc[i].flags = VRING_DESC_F_NEXT;
-		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr)
 			+ sizeof(ctrl->status) + sizeof(uint8_t)*sum;
 		vq->vq_ring.desc[i].len = dlen[k];
@@ -187,7 +188,7 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	}
 
 	vq->vq_ring.desc[i].flags = VRING_DESC_F_WRITE;
-	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_ring.desc[i].len = sizeof(ctrl->status);
 	vq->vq_free_cnt--;
@@ -366,70 +367,85 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		}
 	}
 
-	/*
-	 * Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
-	 * and only accepts 32 bit page frame number.
-	 * Check if the allocated physical memory exceeds 16TB.
-	 */
-	if ((mz->phys_addr + vq->vq_ring_size - 1) >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
-		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
-		rte_free(vq);
-		return -ENOMEM;
-	}
-
 	memset(mz->addr, 0, sizeof(mz->len));
 	vq->mz = mz;
-	vq->vq_ring_mem = mz->phys_addr;
 	vq->vq_ring_virt_mem = mz->addr;
-	PMD_INIT_LOG(DEBUG, "vq->vq_ring_mem:      0x%"PRIx64, (uint64_t)mz->phys_addr);
-	PMD_INIT_LOG(DEBUG, "vq->vq_ring_virt_mem: 0x%"PRIx64, (uint64_t)(uintptr_t)mz->addr);
+
+	if (dev->dev_type == RTE_ETH_DEV_PCI) {
+		vq->vq_ring_mem = mz->phys_addr;
+
+		/* Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
+		 * and only accepts 32 bit page frame number.
+		 * Check if the allocated physical memory exceeds 16TB.
+		 */
+		uint64_t last_physaddr = vq->vq_ring_mem + vq->vq_ring_size - 1;
+
+		if (last_physaddr >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
+			PMD_INIT_LOG(ERR,
+				     "vring address shouldn't be above 16TB!");
+			rte_free(vq);
+			return -ENOMEM;
+		}
+	}
+#ifdef RTE_VIRTIO_VDEV
+	else
+		vq->vq_ring_mem = (phys_addr_t)mz->addr; /* Use vaddr!!! */
+#endif
+
+	PMD_INIT_LOG(DEBUG, "vq->vq_ring_mem:      0x%" PRIx64,
+		     (uint64_t)vq->vq_ring_mem);
+	PMD_INIT_LOG(DEBUG, "vq->vq_ring_virt_mem: 0x%" PRIx64,
+		     (uint64_t)(uintptr_t)vq->vq_ring_virt_mem);
 	vq->virtio_net_hdr_mz  = NULL;
 	vq->virtio_net_hdr_mem = 0;
 
+	uint64_t hdr_size = 0;
 	if (queue_type == VTNET_TQ) {
 		/*
 		 * For each xmit packet, allocate a virtio_net_hdr
 		 */
 		snprintf(vq_name, sizeof(vq_name), "port%d_tvq%d_hdrzone",
 			dev->data->port_id, queue_idx);
-		vq->virtio_net_hdr_mz = rte_memzone_reserve_aligned(vq_name,
-			vq_size * hw->vtnet_hdr_size,
-			socket_id, 0, RTE_CACHE_LINE_SIZE);
-		if (vq->virtio_net_hdr_mz == NULL) {
-			if (rte_errno == EEXIST)
-				vq->virtio_net_hdr_mz =
-					rte_memzone_lookup(vq_name);
-			if (vq->virtio_net_hdr_mz == NULL) {
-				rte_free(vq);
-				return -ENOMEM;
-			}
-		}
-		vq->virtio_net_hdr_mem =
-			vq->virtio_net_hdr_mz->phys_addr;
-		memset(vq->virtio_net_hdr_mz->addr, 0,
-			vq_size * hw->vtnet_hdr_size);
+		hdr_size = vq_size * hw->vtnet_hdr_size;
 	} else if (queue_type == VTNET_CQ) {
 		/* Allocate a page for control vq command, data and status */
 		snprintf(vq_name, sizeof(vq_name), "port%d_cvq_hdrzone",
 			dev->data->port_id);
-		vq->virtio_net_hdr_mz = rte_memzone_reserve_aligned(vq_name,
-			PAGE_SIZE, socket_id, 0, RTE_CACHE_LINE_SIZE);
-		if (vq->virtio_net_hdr_mz == NULL) {
+		hdr_size = PAGE_SIZE;
+	}
+
+	if (hdr_size) { /* queue_type is VTNET_TQ or VTNET_CQ */
+		mz = rte_memzone_reserve_aligned(vq_name, hdr_size, socket_id,
+						 0, RTE_CACHE_LINE_SIZE);
+		if (!mz) {
 			if (rte_errno == EEXIST)
-				vq->virtio_net_hdr_mz =
-					rte_memzone_lookup(vq_name);
-			if (vq->virtio_net_hdr_mz == NULL) {
+				mz = rte_memzone_lookup(vq_name);
+			if (!mz) {
 				rte_free(vq);
 				return -ENOMEM;
 			}
 		}
-		vq->virtio_net_hdr_mem =
-			vq->virtio_net_hdr_mz->phys_addr;
-		memset(vq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
+		vq->virtio_net_hdr_mz = mz;
+		vq->virtio_net_hdr_vaddr = mz->addr;
+		memset(vq->virtio_net_hdr_vaddr, 0, hdr_size);
+
+		if (dev->dev_type == RTE_ETH_DEV_PCI)
+			vq->virtio_net_hdr_mem = mz->phys_addr;
+#ifdef RTE_VIRTIO_VDEV
+		else
+			vq->virtio_net_hdr_mem = (phys_addr_t)mz->addr;
+#endif
 	}
 
 	hw->vtpci_ops->setup_queue(hw, vq);
 
+	if (dev->dev_type == RTE_ETH_DEV_PCI)
+		vq->offset = offsetof(struct rte_mbuf, buf_physaddr);
+#ifdef RTE_VIRTIO_VDEV
+	else
+		vq->offset = offsetof(struct rte_mbuf, buf_addr);
+#endif
+
 	*pvq = vq;
 	return 0;
 }
@@ -479,8 +495,10 @@ virtio_dev_close(struct rte_eth_dev *dev)
 	PMD_INIT_LOG(DEBUG, "virtio_dev_close");
 
 	/* reset the NIC */
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		vtpci_irq_config(hw, VIRTIO_MSI_NO_VECTOR);
+	if (dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			vtpci_irq_config(hw, VIRTIO_MSI_NO_VECTOR);
+	}
 	vtpci_reset(hw);
 	hw->started = 0;
 	virtio_dev_free_mbufs(dev);
@@ -983,8 +1001,9 @@ virtio_interrupt_handler(__rte_unused struct rte_intr_handle *handle,
 	isr = vtpci_isr(hw);
 	PMD_DRV_LOG(INFO, "interrupt status = %#x", isr);
 
-	if (rte_intr_enable(&dev->pci_dev->intr_handle) < 0)
-		PMD_DRV_LOG(ERR, "interrupt enable failed");
+	if (dev->dev_type == RTE_ETH_DEV_PCI)
+		if (rte_intr_enable(&dev->pci_dev->intr_handle) < 0)
+			PMD_DRV_LOG(ERR, "interrupt enable failed");
 
 	if (isr & VIRTIO_PCI_ISR_CONFIG) {
 		if (virtio_dev_link_update(dev, 0) == 0)
@@ -1037,8 +1056,9 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	pci_dev = eth_dev->pci_dev;
 
-	if (vtpci_init(pci_dev, hw) < 0)
-		return -1;
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI)
+		if (vtpci_init(pci_dev, hw) < 0)
+			return -1;
 
 	/* Reset the device although not necessary at startup */
 	vtpci_reset(hw);
@@ -1052,10 +1072,12 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 		return -1;
 
 	/* If host does not support status then disable LSC */
-	if (!vtpci_with_feature(hw, VIRTIO_NET_F_STATUS))
-		pci_dev->driver->drv_flags &= ~RTE_PCI_DRV_INTR_LSC;
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (!vtpci_with_feature(hw, VIRTIO_NET_F_STATUS))
+			pci_dev->driver->drv_flags &= ~RTE_PCI_DRV_INTR_LSC;
 
-	rte_eth_copy_pci_info(eth_dev, pci_dev);
+		rte_eth_copy_pci_info(eth_dev, pci_dev);
+	}
 
 	rx_func_get(eth_dev);
 
@@ -1132,15 +1154,17 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	PMD_INIT_LOG(DEBUG, "hw->max_rx_queues=%d   hw->max_tx_queues=%d",
 			hw->max_rx_queues, hw->max_tx_queues);
-	PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
-			eth_dev->data->port_id, pci_dev->id.vendor_id,
-			pci_dev->id.device_id);
-
-	/* Setup interrupt callback  */
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		rte_intr_callback_register(&pci_dev->intr_handle,
-				   virtio_interrupt_handler, eth_dev);
-
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
+			     eth_dev->data->port_id, pci_dev->id.vendor_id,
+			     pci_dev->id.device_id);
+
+		/* Setup interrupt callback  */
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			rte_intr_callback_register(&pci_dev->intr_handle,
+						   virtio_interrupt_handler,
+						   eth_dev);
+	}
 	virtio_dev_cq_start(eth_dev);
 
 	return 0;
@@ -1173,10 +1197,11 @@ eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev)
 	eth_dev->data->mac_addrs = NULL;
 
 	/* reset interrupt callback  */
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		rte_intr_callback_unregister(&pci_dev->intr_handle,
-						virtio_interrupt_handler,
-						eth_dev);
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI)
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			rte_intr_callback_unregister(&pci_dev->intr_handle,
+						     virtio_interrupt_handler,
+						     eth_dev);
 	rte_eal_pci_unmap_device(pci_dev);
 
 	PMD_INIT_LOG(DEBUG, "dev_uninit completed");
@@ -1241,11 +1266,13 @@ virtio_dev_configure(struct rte_eth_dev *dev)
 		return -ENOTSUP;
 	}
 
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		if (vtpci_irq_config(hw, 0) == VIRTIO_MSI_NO_VECTOR) {
-			PMD_DRV_LOG(ERR, "failed to set config vector");
-			return -EBUSY;
-		}
+	if (dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			if (vtpci_irq_config(hw, 0) == VIRTIO_MSI_NO_VECTOR) {
+				PMD_DRV_LOG(ERR, "failed to set config vector");
+				return -EBUSY;
+			}
+	}
 
 	return 0;
 }
@@ -1439,3 +1466,167 @@ static struct rte_driver rte_virtio_driver = {
 };
 
 PMD_REGISTER_DRIVER(rte_virtio_driver);
+
+#ifdef RTE_VIRTIO_VDEV
+
+static const char *valid_args[] = {
+#define ETH_CVIO_ARG_RX_NUM		"rx"
+	ETH_CVIO_ARG_RX_NUM,
+#define ETH_CVIO_ARG_TX_NUM		"tx"
+	ETH_CVIO_ARG_TX_NUM,
+#define ETH_CVIO_ARG_CQ_NUM		"cq"
+	ETH_CVIO_ARG_CQ_NUM,
+#define ETH_CVIO_ARG_MAC		"mac"
+	ETH_CVIO_ARG_MAC,
+#define ETH_CVIO_ARG_PATH		"path"
+	ETH_CVIO_ARG_PATH,
+#define ETH_CVIO_ARG_QUEUE_SIZE		"queue_num"
+	ETH_CVIO_ARG_QUEUE_SIZE,
+#define ETH_CVIO_ARG_IFNAME		"ifname"
+	ETH_CVIO_ARG_IFNAME,
+	NULL
+};
+
+static int
+get_string_arg(const char *key __rte_unused,
+	       const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(char **)extra_args = strdup(value);
+
+	return 0;
+}
+
+static int
+get_integer_arg(const char *key __rte_unused,
+		const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(uint64_t *)extra_args = strtoull(value, NULL, 0);
+
+	return 0;
+}
+
+static struct rte_eth_dev *
+cvio_eth_dev_alloc(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct rte_eth_dev_data *data;
+	struct virtio_hw *hw;
+
+	eth_dev = rte_eth_dev_allocate(name, RTE_ETH_DEV_VIRTUAL);
+	if (!eth_dev)
+		rte_panic("cannot alloc rte_eth_dev\n");
+
+	data = eth_dev->data;
+
+	hw = rte_zmalloc(NULL, sizeof(*hw), 0);
+	if (!hw)
+		rte_panic("malloc virtio_hw failed\n");
+
+	data->dev_private = hw;
+	data->numa_node = SOCKET_ID_ANY;
+	eth_dev->pci_dev = NULL;
+	/* will be used in virtio_dev_info_get() */
+	eth_dev->driver = &rte_virtio_pmd;
+	/* TODO: eth_dev->link_intr_cbs */
+	return eth_dev;
+}
+
+#define CVIO_DEF_CQ_EN		0
+#define CVIO_DEF_Q_NUM		1
+#define CVIO_DEF_Q_SZ		256
+/* Dev initialization routine. Invoked once for each virtio vdev at
+ * EAL init time, see rte_eal_dev_init().
+ * Returns 0 on success.
+ */
+static int
+rte_cvio_pmd_devinit(const char *name, const char *params)
+{
+	struct rte_kvargs *kvlist = NULL;
+	struct rte_eth_dev *eth_dev = NULL;
+	uint64_t nb_rx = CVIO_DEF_Q_NUM;
+	uint64_t nb_tx = CVIO_DEF_Q_NUM;
+	uint64_t nb_cq = CVIO_DEF_CQ_EN;
+	uint64_t queue_num = CVIO_DEF_Q_SZ;
+	char *sock_path = NULL;
+	char *mac_addr = NULL;
+	char *ifname = NULL;
+
+	if (!params || params[0] == '\0')
+		rte_panic("arg %s is mandatory for eth_cvio\n",
+			  ETH_CVIO_ARG_QUEUE_SIZE);
+
+	kvlist = rte_kvargs_parse(params, valid_args);
+	if (!kvlist)
+		rte_panic("error when parsing param\n");
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_PATH) == 1)
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_PATH,
+				   &get_string_arg, &sock_path);
+	else
+		rte_panic("arg %s is mandatory for eth_cvio\n",
+			  ETH_CVIO_ARG_QUEUE_SIZE);
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_MAC) == 1)
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_MAC,
+				   &get_string_arg, &mac_addr);
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_IFNAME) == 1)
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_IFNAME,
+				   &get_string_arg, &ifname);
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_QUEUE_SIZE) == 1)
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_QUEUE_SIZE,
+				   &get_integer_arg, &queue_num);
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_RX_NUM) == 1)
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_RX_NUM,
+				   &get_integer_arg, &nb_rx);
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_TX_NUM) == 1)
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_TX_NUM,
+				   &get_integer_arg, &nb_tx);
+
+	if (rte_kvargs_count(kvlist, ETH_CVIO_ARG_CQ_NUM) == 1)
+		rte_kvargs_process(kvlist, ETH_CVIO_ARG_CQ_NUM,
+				   &get_integer_arg, &nb_cq);
+
+	eth_dev = cvio_eth_dev_alloc(name);
+
+	virtio_vdev_init(eth_dev->data, sock_path, nb_rx, nb_tx, nb_cq,
+			 queue_num, mac_addr, ifname);
+	if (sock_path)
+		free(sock_path);
+	if (mac_addr)
+		free(mac_addr);
+	if (ifname)
+		free(ifname);
+
+	/* originally, this will be called in rte_eal_pci_probe() */
+	eth_virtio_dev_init(eth_dev);
+
+	return 0;
+}
+
+static int
+rte_cvio_pmd_devuninit(const char *name)
+{
+	rte_panic("%s", name);
+	return 0;
+}
+
+static struct rte_driver rte_cvio_driver = {
+	.name   = "eth_cvio",
+	.type   = PMD_VDEV,
+	.init   = rte_cvio_pmd_devinit,
+	.uninit = rte_cvio_pmd_devuninit,
+};
+
+PMD_REGISTER_DRIVER(rte_cvio_driver);
+
+#endif
diff --git a/drivers/net/virtio/virtio_rxtx.c b/drivers/net/virtio/virtio_rxtx.c
index 41a1366..cebd75a 100644
--- a/drivers/net/virtio/virtio_rxtx.c
+++ b/drivers/net/virtio/virtio_rxtx.c
@@ -191,8 +191,7 @@ virtqueue_enqueue_recv_refill(struct virtqueue *vq, struct rte_mbuf *cookie)
 
 	start_dp = vq->vq_ring.desc;
 	start_dp[idx].addr =
-		(uint64_t)(cookie->buf_physaddr + RTE_PKTMBUF_HEADROOM
-		- hw->vtnet_hdr_size);
+		RTE_MBUF_DATA_DMA_ADDR(cookie, vq->offset) - hw->vtnet_hdr_size;
 	start_dp[idx].len =
 		cookie->buf_len - RTE_PKTMBUF_HEADROOM + hw->vtnet_hdr_size;
 	start_dp[idx].flags =  VRING_DESC_F_WRITE;
@@ -237,7 +236,8 @@ virtqueue_enqueue_xmit(struct virtqueue *txvq, struct rte_mbuf *cookie)
 
 	for (; ((seg_num > 0) && (cookie != NULL)); seg_num--) {
 		idx = start_dp[idx].next;
-		start_dp[idx].addr  = RTE_MBUF_DATA_DMA_ADDR(cookie);
+		start_dp[idx].addr  =
+			RTE_MBUF_DATA_DMA_ADDR(cookie, txvq->offset);
 		start_dp[idx].len   = cookie->data_len;
 		start_dp[idx].flags = VRING_DESC_F_NEXT;
 		cookie = cookie->next;
diff --git a/drivers/net/virtio/virtio_rxtx_simple.c b/drivers/net/virtio/virtio_rxtx_simple.c
index 3a1de9d..92a6388 100644
--- a/drivers/net/virtio/virtio_rxtx_simple.c
+++ b/drivers/net/virtio/virtio_rxtx_simple.c
@@ -80,8 +80,8 @@ virtqueue_enqueue_recv_refill_simple(struct virtqueue *vq,
 	vq->sw_ring[desc_idx] = cookie;
 
 	start_dp = vq->vq_ring.desc;
-	start_dp[desc_idx].addr = (uint64_t)((uintptr_t)cookie->buf_physaddr +
-		RTE_PKTMBUF_HEADROOM - vq->hw->vtnet_hdr_size);
+	start_dp[desc_idx].addr = RTE_MBUF_DATA_DMA_ADDR(cookie, vq->offset)
+		- vq->hw->vtnet_hdr_size;
 	start_dp[desc_idx].len = cookie->buf_len -
 		RTE_PKTMBUF_HEADROOM + vq->hw->vtnet_hdr_size;
 
@@ -119,8 +119,8 @@ virtio_rxq_rearm_vec(struct virtqueue *rxvq)
 		*(uint64_t *)p = rxvq->mbuf_initializer;
 
 		start_dp[i].addr =
-			(uint64_t)((uintptr_t)sw_ring[i]->buf_physaddr +
-			RTE_PKTMBUF_HEADROOM - rxvq->hw->vtnet_hdr_size);
+			RTE_MBUF_DATA_DMA_ADDR(sw_ring[i], rxvq->offset) -
+			rxvq->hw->vtnet_hdr_size;
 		start_dp[i].len = sw_ring[i]->buf_len -
 			RTE_PKTMBUF_HEADROOM + rxvq->hw->vtnet_hdr_size;
 	}
@@ -366,7 +366,7 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 			txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 		for (i = 0; i < nb_tail; i++) {
 			start_dp[desc_idx].addr =
-				RTE_MBUF_DATA_DMA_ADDR(*tx_pkts);
+				RTE_MBUF_DATA_DMA_ADDR(*tx_pkts, txvq->offset);
 			start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 			tx_pkts++;
 			desc_idx++;
@@ -377,7 +377,8 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 	for (i = 0; i < nb_commit; i++)
 		txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 	for (i = 0; i < nb_commit; i++) {
-		start_dp[desc_idx].addr = RTE_MBUF_DATA_DMA_ADDR(*tx_pkts);
+		start_dp[desc_idx].addr = RTE_MBUF_DATA_DMA_ADDR(*tx_pkts,
+				txvq->offset);
 		start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 		tx_pkts++;
 		desc_idx++;
diff --git a/drivers/net/virtio/virtqueue.h b/drivers/net/virtio/virtqueue.h
index 99d4fa9..057c4ed 100644
--- a/drivers/net/virtio/virtqueue.h
+++ b/drivers/net/virtio/virtqueue.h
@@ -66,8 +66,14 @@ struct rte_mbuf;
 
 #define VIRTQUEUE_MAX_NAME_SZ 32
 
-#define RTE_MBUF_DATA_DMA_ADDR(mb) \
-	(uint64_t) ((mb)->buf_physaddr + (mb)->data_off)
+#ifdef RTE_VIRTIO_VDEV
+#define RTE_MBUF_DATA_DMA_ADDR(mb, offset) \
+	((uint64_t)((uintptr_t)(*(void **)((uintptr_t)mb + offset)) \
+			+ (mb)->data_off))
+#else
+#define RTE_MBUF_DATA_DMA_ADDR(mb, offset) \
+	((uint64_t)((mb)->buf_physaddr + (mb)->data_off))
+#endif /* RTE_VIRTIO_VDEV */
 
 #define VTNET_SQ_RQ_QUEUE_IDX 0
 #define VTNET_SQ_TQ_QUEUE_IDX 1
@@ -167,7 +173,8 @@ struct virtqueue {
 
 	void        *vq_ring_virt_mem;    /**< linear address of vring*/
 	unsigned int vq_ring_size;
-	phys_addr_t vq_ring_mem;          /**< physical address of vring */
+	phys_addr_t vq_ring_mem;          /**< phys addr of vring for pci dev */
+					  /**< virt addr of vring for vdev. */
 
 	struct vring vq_ring;    /**< vring keeping desc, used and avail */
 	uint16_t    vq_free_cnt; /**< num of desc available */
@@ -186,8 +193,10 @@ struct virtqueue {
 	 */
 	uint16_t vq_used_cons_idx;
 	uint16_t vq_avail_idx;
+	uint16_t offset; /**< relative offset to obtain addr in mbuf */
 	uint64_t mbuf_initializer; /**< value to init mbufs. */
 	phys_addr_t virtio_net_hdr_mem; /**< hdr for each xmit packet */
+	void        *virtio_net_hdr_vaddr;    /**< linear address of vring*/
 
 	struct rte_mbuf **sw_ring; /**< RX software ring. */
 	/* dummy mbuf, for wraparound when processing RX ring. */
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v2 5/5] docs: add release note for virtio for container
  2016-02-05 11:20 ` [PATCH v2 0/5] " Jianfeng Tan
                     ` (3 preceding siblings ...)
  2016-02-05 11:20   ` [PATCH v2 4/5] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
@ 2016-02-05 11:20   ` Jianfeng Tan
  2016-03-23 19:17   ` [PATCH v2 0/5] virtio support " Neil Horman
  2016-04-13 16:14   ` Thomas Monjalon
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-02-05 11:20 UTC (permalink / raw)
  To: dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 doc/guides/rel_notes/release_2_3.rst | 4 ++++
 1 file changed, 4 insertions(+)

diff --git a/doc/guides/rel_notes/release_2_3.rst b/doc/guides/rel_notes/release_2_3.rst
index 7945694..1e7d51d 100644
--- a/doc/guides/rel_notes/release_2_3.rst
+++ b/doc/guides/rel_notes/release_2_3.rst
@@ -39,6 +39,10 @@ This section should contain new features added in this release. Sample format:
 
   Enabled virtio 1.0 support for virtio pmd driver.
 
+* **Virtio support for containers.**
+
+  Add a new virtual device, named eth_cvio, to support virtio for containers.
+
 
 Resolved Issues
 ---------------
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 3/5] virtio/vdev: add embeded device emulation
  2016-02-05 11:20   ` [PATCH v2 3/5] virtio/vdev: add embeded device emulation Jianfeng Tan
@ 2016-02-07 10:47     ` Michael S. Tsirkin
  2016-02-08  6:59     ` Tetsuya Mukawa
  2016-02-16  2:40     ` Tan, Jianfeng
  2 siblings, 0 replies; 196+ messages in thread
From: Michael S. Tsirkin @ 2016-02-07 10:47 UTC (permalink / raw)
  To: Jianfeng Tan; +Cc: nakajima.yoshihiro, dev, ann.zhuangyanying

On Fri, Feb 05, 2016 at 07:20:26PM +0800, Jianfeng Tan wrote:
> diff --git a/drivers/net/virtio/vhost.h b/drivers/net/virtio/vhost.h
> new file mode 100644
> index 0000000..73d4f5c
> --- /dev/null
> +++ b/drivers/net/virtio/vhost.h
> @@ -0,0 +1,194 @@
> +/*-
> + *   BSD LICENSE
> + *
> + *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
> + *   All rights reserved.
> + *
> + *   Redistribution and use in source and binary forms, with or without
> + *   modification, are permitted provided that the following conditions
> + *   are met:
> + *
> + *     * Redistributions of source code must retain the above copyright
> + *       notice, this list of conditions and the following disclaimer.
> + *     * Redistributions in binary form must reproduce the above copyright
> + *       notice, this list of conditions and the following disclaimer in
> + *       the documentation and/or other materials provided with the
> + *       distribution.
> + *     * Neither the name of Intel Corporation nor the names of its
> + *       contributors may be used to endorse or promote products derived
> + *       from this software without specific prior written permission.
> + *
> + *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
> + *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
> + *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
> + *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
> + *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
> + *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
> + *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
> + *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
> + *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
> + *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
> + *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
> + */
> +
> +#ifndef _VHOST_NET_USER_H
> +#define _VHOST_NET_USER_H
> +
> +#include <stdint.h>
> +#include <linux/types.h>
> +#include <linux/ioctl.h>
> +
> +#define VHOST_MEMORY_MAX_NREGIONS 8

Don't hard-code this, it's not nice.

> +
> +struct vhost_vring_state {
> +	unsigned int index;
> +	unsigned int num;
> +};
> +
> +struct vhost_vring_file {
> +	unsigned int index;
> +	int fd;
> +};
> +
> +struct vhost_vring_addr {
> +	unsigned int index;
> +	/* Option flags. */
> +	unsigned int flags;
> +	/* Flag values: */
> +	/* Whether log address is valid. If set enables logging. */
> +#define VHOST_VRING_F_LOG 0
> +
> +	/* Start of array of descriptors (virtually contiguous) */
> +	uint64_t desc_user_addr;
> +	/* Used structure address. Must be 32 bit aligned */
> +	uint64_t used_user_addr;
> +	/* Available structure address. Must be 16 bit aligned */
> +	uint64_t avail_user_addr;
> +	/* Logging support. */
> +	/* Log writes to used structure, at offset calculated from specified
> +	 * address. Address must be 32 bit aligned.
> +	 */
> +	uint64_t log_guest_addr;
> +};
> +
> +#define VIRTIO_CONFIG_S_DRIVER_OK   4
> +
> +enum vhost_user_request {
> +	VHOST_USER_NONE = 0,
> +	VHOST_USER_GET_FEATURES = 1,
> +	VHOST_USER_SET_FEATURES = 2,
> +	VHOST_USER_SET_OWNER = 3,
> +	VHOST_USER_RESET_OWNER = 4,
> +	VHOST_USER_SET_MEM_TABLE = 5,
> +	VHOST_USER_SET_LOG_BASE = 6,
> +	VHOST_USER_SET_LOG_FD = 7,
> +	VHOST_USER_SET_VRING_NUM = 8,
> +	VHOST_USER_SET_VRING_ADDR = 9,
> +	VHOST_USER_SET_VRING_BASE = 10,
> +	VHOST_USER_GET_VRING_BASE = 11,
> +	VHOST_USER_SET_VRING_KICK = 12,
> +	VHOST_USER_SET_VRING_CALL = 13,
> +	VHOST_USER_SET_VRING_ERR = 14,
> +	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
> +	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
> +	VHOST_USER_GET_QUEUE_NUM = 17,
> +	VHOST_USER_SET_VRING_ENABLE = 18,
> +	VHOST_USER_MAX
> +};
> +
> +struct vhost_memory_region {
> +	uint64_t guest_phys_addr;
> +	uint64_t memory_size; /* bytes */
> +	uint64_t userspace_addr;
> +	uint64_t mmap_offset;
> +};
> +
> +struct vhost_memory_kernel {
> +	uint32_t nregions;
> +	uint32_t padding;
> +	struct vhost_memory_region regions[0];
> +};
> +
> +struct vhost_memory {
> +	uint32_t nregions;
> +	uint32_t padding;
> +	struct vhost_memory_region regions[VHOST_MEMORY_MAX_NREGIONS];
> +};
> +
> +struct vhost_user_msg {
> +	enum vhost_user_request request;
> +
> +#define VHOST_USER_VERSION_MASK     0x3
> +#define VHOST_USER_REPLY_MASK       (0x1 << 2)
> +	uint32_t flags;
> +	uint32_t size; /* the following payload size */
> +	union {
> +#define VHOST_USER_VRING_IDX_MASK   0xff
> +#define VHOST_USER_VRING_NOFD_MASK  (0x1 << 8)
> +		uint64_t u64;
> +		struct vhost_vring_state state;
> +		struct vhost_vring_addr addr;
> +		struct vhost_memory memory;
> +	} payload;
> +	int fds[VHOST_MEMORY_MAX_NREGIONS];
> +} __attribute((packed));
> +
> +#define VHOST_USER_HDR_SIZE offsetof(struct vhost_user_msg, payload.u64)
> +#define VHOST_USER_PAYLOAD_SIZE (sizeof(struct vhost_user_msg) - VHOST_USER_HDR_SIZE)
> +
> +/* The version of the protocol we support */
> +#define VHOST_USER_VERSION    0x1
> +
> +/* ioctls */

Why do you duplicate ioctls?
Use them from /usr/include/linux/vhost.h, etc.

In fact, what's not coming from linux here
comes from lib/librte_vhost/vhost_user/vhost-net-user.h.

I think you should reuse code, avoid code duplication.


> +
> +#define VHOST_VIRTIO 0xAF
> +
> +#define VHOST_GET_FEATURES _IOR(VHOST_VIRTIO, 0x00, __u64)
> +#define VHOST_SET_FEATURES _IOW(VHOST_VIRTIO, 0x00, __u64)
> +#define VHOST_SET_OWNER _IO(VHOST_VIRTIO, 0x01)
> +#define VHOST_RESET_OWNER _IO(VHOST_VIRTIO, 0x02)
> +#define VHOST_SET_MEM_TABLE _IOW(VHOST_VIRTIO, 0x03, struct vhost_memory_kernel)
> +#define VHOST_SET_LOG_BASE _IOW(VHOST_VIRTIO, 0x04, __u64)
> +#define VHOST_SET_LOG_FD _IOW(VHOST_VIRTIO, 0x07, int)
> +#define VHOST_SET_VRING_NUM _IOW(VHOST_VIRTIO, 0x10, struct vhost_vring_state)
> +#define VHOST_SET_VRING_ADDR _IOW(VHOST_VIRTIO, 0x11, struct vhost_vring_addr)
> +#define VHOST_SET_VRING_BASE _IOW(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
> +#define VHOST_GET_VRING_BASE _IOWR(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
> +#define VHOST_SET_VRING_KICK _IOW(VHOST_VIRTIO, 0x20, struct vhost_vring_file)
> +#define VHOST_SET_VRING_CALL _IOW(VHOST_VIRTIO, 0x21, struct vhost_vring_file)
> +#define VHOST_SET_VRING_ERR _IOW(VHOST_VIRTIO, 0x22, struct vhost_vring_file)
> +#define VHOST_NET_SET_BACKEND _IOW(VHOST_VIRTIO, 0x30, struct vhost_vring_file)
> +
> +/*****************************************************************************/
> +
> +/* Ioctl defines */
> +#define TUNSETIFF     _IOW('T', 202, int)
> +#define TUNGETFEATURES _IOR('T', 207, unsigned int)
> +#define TUNSETOFFLOAD  _IOW('T', 208, unsigned int)
> +#define TUNGETIFF      _IOR('T', 210, unsigned int)
> +#define TUNSETSNDBUF   _IOW('T', 212, int)
> +#define TUNGETVNETHDRSZ _IOR('T', 215, int)
> +#define TUNSETVNETHDRSZ _IOW('T', 216, int)
> +#define TUNSETQUEUE  _IOW('T', 217, int)
> +#define TUNSETVNETLE _IOW('T', 220, int)
> +#define TUNSETVNETBE _IOW('T', 222, int)
> +
> +/* TUNSETIFF ifr flags */
> +#define IFF_TAP          0x0002
> +#define IFF_NO_PI        0x1000
> +#define IFF_ONE_QUEUE    0x2000
> +#define IFF_VNET_HDR     0x4000
> +#define IFF_MULTI_QUEUE  0x0100
> +#define IFF_ATTACH_QUEUE 0x0200
> +#define IFF_DETACH_QUEUE 0x0400
> +
> +/* Features for GSO (TUNSETOFFLOAD). */
> +#define TUN_F_CSUM	0x01	/* You can hand me unchecksummed packets. */
> +#define TUN_F_TSO4	0x02	/* I can handle TSO for IPv4 packets */
> +#define TUN_F_TSO6	0x04	/* I can handle TSO for IPv6 packets */
> +#define TUN_F_TSO_ECN	0x08	/* I can handle TSO with ECN bits. */
> +#define TUN_F_UFO	0x10	/* I can handle UFO packets */
> +
> +#define PATH_NET_TUN "/dev/net/tun"
> +
> +#endif
> diff --git a/drivers/net/virtio/vhost_embedded.c b/drivers/net/virtio/vhost_embedded.c
> new file mode 100644
> index 0000000..0073b86
> --- /dev/null
> +++ b/drivers/net/virtio/vhost_embedded.c
> @@ -0,0 +1,809 @@
> +/*-
> + *   BSD LICENSE
> + *
> + *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
> + *   All rights reserved.
> + *
> + *   Redistribution and use in source and binary forms, with or without
> + *   modification, are permitted provided that the following conditions
> + *   are met:
> + *
> + *     * Redistributions of source code must retain the above copyright
> + *       notice, this list of conditions and the following disclaimer.
> + *     * Redistributions in binary form must reproduce the above copyright
> + *       notice, this list of conditions and the following disclaimer in
> + *       the documentation and/or other materials provided with the
> + *       distribution.
> + *     * Neither the name of Intel Corporation nor the names of its
> + *       contributors may be used to endorse or promote products derived
> + *       from this software without specific prior written permission.
> + *
> + *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
> + *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
> + *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
> + *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
> + *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
> + *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
> + *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
> + *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
> + *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
> + *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
> + *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
> + */
> +
> +#include <stdint.h>
> +#include <sys/types.h>
> +#include <sys/socket.h>
> +#include <sys/un.h>
> +#include <stdio.h>
> +#include <sys/stat.h>
> +#include <fcntl.h>
> +#include <string.h>
> +#include <errno.h>
> +#include <assert.h>
> +#include <sys/mman.h>
> +#include <unistd.h>
> +#include <sys/eventfd.h>
> +#include <sys/ioctl.h>
> +#include <net/if.h>
> +
> +#include <rte_mbuf.h>
> +#include <rte_memory.h>
> +#include <rte_eal_memconfig.h>
> +
> +#include "virtio_pci.h"
> +#include "virtio_logs.h"
> +#include "virtio_ethdev.h"
> +#include "virtqueue.h"
> +#include "vhost.h"
> +
> +static int
> +vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
> +{
> +	int r;
> +	struct msghdr msgh;
> +	struct iovec iov;
> +	size_t fd_size = fd_num * sizeof(int);
> +	char control[CMSG_SPACE(fd_size)];
> +	struct cmsghdr *cmsg;
> +
> +	bzero(&msgh, sizeof(msgh));
> +	bzero(control, sizeof(control));
> +
> +	iov.iov_base = (uint8_t *)buf;
> +	iov.iov_len = len;
> +
> +	msgh.msg_iov = &iov;
> +	msgh.msg_iovlen = 1;
> +	msgh.msg_control = control;
> +	msgh.msg_controllen = sizeof(control);
> +
> +	cmsg = CMSG_FIRSTHDR(&msgh);
> +	cmsg->cmsg_len = CMSG_LEN(fd_size);
> +	cmsg->cmsg_level = SOL_SOCKET;
> +	cmsg->cmsg_type = SCM_RIGHTS;
> +	memcpy(CMSG_DATA(cmsg), fds, fd_size);
> +
> +	do {
> +		r = sendmsg(fd, &msgh, 0);
> +	} while (r < 0 && errno == EINTR);
> +
> +	return r;
> +}
> +
> +static int
> +vhost_user_read(int fd, struct vhost_user_msg *msg)
> +{
> +	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
> +	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
> +
> +	ret = recv(fd, (void *)msg, sz_hdr, 0);
> +	if (ret < sz_hdr) {
> +		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
> +			    ret, sz_hdr);
> +		goto fail;
> +	}
> +
> +	/* validate msg flags */
> +	if (msg->flags != (valid_flags)) {
> +		PMD_DRV_LOG(ERR, "Failed to recv msg: flags %x instead of %x.",
> +			    msg->flags, valid_flags);
> +		goto fail;
> +	}
> +
> +	sz_payload = msg->size;
> +	if (sz_payload) {
> +		ret = recv(fd, (void *)((char *)msg + sz_hdr), sz_payload, 0);
> +		if (ret < sz_payload) {
> +			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
> +				    ret, msg->size);
> +			goto fail;
> +		}
> +	}
> +
> +	return 0;
> +
> +fail:
> +	return -1;
> +}
> +
> +static struct vhost_user_msg m __rte_unused;
> +
> +static void
> +prepare_vhost_memory_user(struct vhost_user_msg *msg, int fds[])
> +{
> +	int i, num;
> +	struct back_file *huges;
> +	struct vhost_memory_region *mr;
> +
> +	num = rte_eal_get_backfile_info(&huges);
> +
> +	if (num > VHOST_MEMORY_MAX_NREGIONS)
> +		rte_panic("%d files exceed maximum of %d for vhost-user\n",
> +			  num, VHOST_MEMORY_MAX_NREGIONS);
> +
> +	for (i = 0; i < num; ++i) {
> +		mr = &msg->payload.memory.regions[i];
> +		mr->guest_phys_addr = (uint64_t)huges[i].addr; /* use vaddr! */
> +		mr->userspace_addr = (uint64_t)huges[i].addr;
> +		mr->memory_size = huges[i].size;
> +		mr->mmap_offset = 0;
> +		fds[i] = open(huges[i].filepath, O_RDWR);
> +	}
> +
> +	msg->payload.memory.nregions = num;
> +	msg->payload.memory.padding = 0;
> +	free(huges);
> +}
> +
> +static int
> +vhost_user_sock(struct virtio_hw *hw, unsigned long int req, void *arg)
> +{
> +	struct vhost_user_msg msg;
> +	struct vhost_vring_file *file = 0;
> +	int need_reply = 0;
> +	int fds[VHOST_MEMORY_MAX_NREGIONS];
> +	int fd_num = 0;
> +	int i, len;
> +
> +	msg.request = req;
> +	msg.flags = VHOST_USER_VERSION;
> +	msg.size = 0;
> +
> +	switch (req) {
> +	case VHOST_USER_GET_FEATURES:
> +		need_reply = 1;
> +		break;
> +
> +	case VHOST_USER_SET_FEATURES:
> +	case VHOST_USER_SET_LOG_BASE:
> +		msg.payload.u64 = *((__u64 *)arg);
> +		msg.size = sizeof(m.payload.u64);
> +		break;
> +
> +	case VHOST_USER_SET_OWNER:
> +	case VHOST_USER_RESET_OWNER:
> +		break;
> +
> +	case VHOST_USER_SET_MEM_TABLE:
> +		prepare_vhost_memory_user(&msg, fds);
> +		fd_num = msg.payload.memory.nregions;
> +		msg.size = sizeof(m.payload.memory.nregions);
> +		msg.size += sizeof(m.payload.memory.padding);
> +		msg.size += fd_num * sizeof(struct vhost_memory_region);
> +		break;
> +
> +	case VHOST_USER_SET_LOG_FD:
> +		fds[fd_num++] = *((int *)arg);
> +		break;
> +
> +	case VHOST_USER_SET_VRING_NUM:
> +	case VHOST_USER_SET_VRING_BASE:
> +		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
> +		msg.size = sizeof(m.payload.state);
> +		break;
> +
> +	case VHOST_USER_GET_VRING_BASE:
> +		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
> +		msg.size = sizeof(m.payload.state);
> +		need_reply = 1;
> +		break;
> +
> +	case VHOST_USER_SET_VRING_ADDR:
> +		memcpy(&msg.payload.addr, arg, sizeof(msg.payload.addr));
> +		msg.size = sizeof(m.payload.addr);
> +		break;
> +
> +	case VHOST_USER_SET_VRING_KICK:
> +	case VHOST_USER_SET_VRING_CALL:
> +	case VHOST_USER_SET_VRING_ERR:
> +		file = arg;
> +		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
> +		msg.size = sizeof(m.payload.u64);
> +		if (file->fd > 0)
> +			fds[fd_num++] = file->fd;
> +		else
> +			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
> +		break;
> +
> +	default:
> +		PMD_DRV_LOG(ERR, "vhost-user trying to send unhandled msg type");
> +		return -1;
> +	}
> +
> +	len = VHOST_USER_HDR_SIZE + msg.size;
> +	if (vhost_user_write(hw->vhostfd, &msg, len, fds, fd_num) < 0)
> +		return 0;
> +
> +	if (req == VHOST_USER_SET_MEM_TABLE)
> +		for (i = 0; i < fd_num; ++i)
> +			close(fds[i]);
> +
> +	if (need_reply) {
> +		if (vhost_user_read(hw->vhostfd, &msg) < 0)
> +			return -1;
> +
> +		if (req != msg.request) {
> +			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
> +			return -1;
> +		}
> +
> +		switch (req) {
> +		case VHOST_USER_GET_FEATURES:
> +			if (msg.size != sizeof(m.payload.u64)) {
> +				PMD_DRV_LOG(ERR, "Received bad msg size.");
> +				return -1;
> +			}
> +			*((__u64 *)arg) = msg.payload.u64;
> +			break;
> +		case VHOST_USER_GET_VRING_BASE:
> +			if (msg.size != sizeof(m.payload.state)) {
> +				PMD_DRV_LOG(ERR, "Received bad msg size.");
> +				return -1;
> +			}
> +			memcpy(arg, &msg.payload.state,
> +			       sizeof(struct vhost_vring_state));
> +			break;
> +		default:
> +			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
> +			return -1;
> +		}
> +	}
> +
> +	return 0;
> +}
> +
> +static int
> +vhost_kernel_ioctl(struct virtio_hw *hw, unsigned long int req, void *arg)
> +{
> +	return ioctl(hw->vhostfd, req, arg);
> +}
> +
> +enum {
> +	VHOST_MSG_SET_OWNER,
> +	VHOST_MSG_SET_FEATURES,
> +	VHOST_MSG_GET_FEATURES,
> +	VHOST_MSG_SET_VRING_CALL,
> +	VHOST_MSG_SET_VRING_NUM,
> +	VHOST_MSG_SET_VRING_BASE,
> +	VHOST_MSG_GET_VRING_BASE,
> +	VHOST_MSG_SET_VRING_ADDR,
> +	VHOST_MSG_SET_VRING_KICK,
> +	VHOST_MSG_SET_MEM_TABLE,
> +	VHOST_MSG_MAX,
> +};
> +
> +static const char * const vhost_msg_strings[] = {
> +	[VHOST_MSG_SET_OWNER] = "VHOST_MSG_SET_OWNER",
> +	[VHOST_MSG_SET_FEATURES] = "VHOST_MSG_SET_FEATURES",
> +	[VHOST_MSG_GET_FEATURES] = "VHOST_MSG_GET_FEATURES",
> +	[VHOST_MSG_SET_VRING_CALL] = "VHOST_MSG_SET_VRING_CALL",
> +	[VHOST_MSG_SET_VRING_NUM] = "VHOST_MSG_SET_VRING_NUM",
> +	[VHOST_MSG_SET_VRING_BASE] = "VHOST_MSG_SET_VRING_BASE",
> +	[VHOST_MSG_GET_VRING_BASE] = "VHOST_MSG_GET_VRING_BASE",
> +	[VHOST_MSG_SET_VRING_ADDR] = "VHOST_MSG_SET_VRING_ADDR",
> +	[VHOST_MSG_SET_VRING_KICK] = "VHOST_MSG_SET_VRING_KICK",
> +	[VHOST_MSG_SET_MEM_TABLE] = "VHOST_MSG_SET_MEM_TABLE",
> +	NULL,
> +};
> +
> +static unsigned long int vhost_req_map[][2] = {
> +	[VHOST_MSG_SET_OWNER] = {
> +		VHOST_SET_OWNER, VHOST_USER_SET_OWNER
> +	},
> +	[VHOST_MSG_SET_FEATURES] = {
> +		VHOST_SET_FEATURES, VHOST_USER_SET_FEATURES
> +	},
> +	[VHOST_MSG_GET_FEATURES] = {
> +		VHOST_GET_FEATURES, VHOST_USER_GET_FEATURES
> +	},
> +	[VHOST_MSG_SET_VRING_CALL] = {
> +		VHOST_SET_VRING_CALL, VHOST_USER_SET_VRING_CALL
> +	},
> +	[VHOST_MSG_SET_VRING_NUM] = {
> +		VHOST_SET_VRING_NUM, VHOST_USER_SET_VRING_NUM
> +	},
> +	[VHOST_MSG_SET_VRING_BASE] = {
> +		VHOST_SET_VRING_BASE, VHOST_USER_SET_VRING_BASE
> +	},
> +	[VHOST_MSG_GET_VRING_BASE] = {
> +		VHOST_GET_VRING_BASE, VHOST_USER_GET_VRING_BASE
> +	},
> +	[VHOST_MSG_SET_VRING_ADDR] = {
> +		VHOST_SET_VRING_ADDR, VHOST_USER_SET_VRING_ADDR
> +	},
> +	[VHOST_MSG_SET_VRING_KICK] = {
> +		VHOST_SET_VRING_KICK, VHOST_USER_SET_VRING_KICK
> +	},
> +	[VHOST_MSG_SET_MEM_TABLE] = {
> +		VHOST_SET_MEM_TABLE, VHOST_USER_SET_MEM_TABLE
> +	},
> +};
> +
> +static int
> +vhost_call(struct virtio_hw *hw, unsigned long int req_orig, void *arg)
> +{
> +	unsigned long int req_new;
> +	int ret;
> +
> +	if (req_orig >= VHOST_MSG_MAX)
> +		rte_panic("invalid req: %lu\n", req_orig);
> +
> +	PMD_DRV_LOG(INFO, "%s\n", vhost_msg_strings[req_orig]);
> +	req_new = vhost_req_map[req_orig][hw->type];
> +	if (hw->type == VHOST_USER)
> +		ret = vhost_user_sock(hw, req_new, arg);
> +	else
> +		ret = vhost_kernel_ioctl(hw, req_new, arg);
> +
> +	if (ret < 0)
> +		rte_panic("vhost_call %s failed: %s\n",
> +			  vhost_msg_strings[req_orig], strerror(errno));
> +
> +	return ret;
> +}
> +
> +static void
> +kick_one_vq(struct virtio_hw *hw, struct virtqueue *vq, unsigned queue_sel)
> +{
> +	int callfd, kickfd;
> +	struct vhost_vring_file file;
> +	struct vhost_vring_state state;
> +	struct vhost_vring_addr addr = {
> +		.index = queue_sel,
> +		.desc_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.desc,
> +		.avail_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.avail,
> +		.used_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.used,
> +		.log_guest_addr = 0,
> +		.flags = 0, /* disable log */
> +	};
> +
> +	/* or use invalid flag to disable it, but vhost-dpdk uses this to judge
> +	 * if dev is alive. so finally we need two real event_fds.
> +	 */
> +	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL come
> +	 * firstly because vhost depends on this msg to allocate virtqueue
> +	 * pair.
> +	 */
> +	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
> +	if (callfd < 0)
> +		rte_panic("callfd error, %s\n", strerror(errno));
> +
> +	file.index = queue_sel;
> +	file.fd = callfd;
> +	vhost_call(hw, VHOST_MSG_SET_VRING_CALL, &file);
> +	hw->callfds[queue_sel] = callfd;
> +
> +	state.index = queue_sel;
> +	state.num = vq->vq_ring.num;
> +	vhost_call(hw, VHOST_MSG_SET_VRING_NUM, &state);
> +
> +	state.num = 0; /* no reservation */
> +	vhost_call(hw, VHOST_MSG_SET_VRING_BASE, &state);
> +
> +	vhost_call(hw, VHOST_MSG_SET_VRING_ADDR, &addr);
> +
> +	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_KICK comes
> +	 * lastly because vhost depends on this msg to judge if
> +	 * virtio_is_ready().
> +	 */
> +	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
> +	if (kickfd < 0)
> +		rte_panic("kickfd error, %s\n", strerror(errno));
> +
> +	file.fd = kickfd;
> +	vhost_call(hw, VHOST_MSG_SET_VRING_KICK, &file);
> +	hw->kickfds[queue_sel] = kickfd;
> +}
> +
> +/**
> + * Merge those virtually adjacent memsegs into one region.
> + */
> +static void
> +prepare_vhost_memory_kernel(struct vhost_memory_kernel **p_vm)
> +{
> +	unsigned i, j, k = 0;
> +	struct rte_memseg *seg;
> +	struct vhost_memory_region *mr;
> +	struct vhost_memory_kernel *vm;
> +
> +	vm = malloc(sizeof(struct vhost_memory_kernel) +
> +		    RTE_MAX_MEMSEG * sizeof(struct vhost_memory_region));
> +
> +	for (i = 0; i < RTE_MAX_MEMSEG; ++i) {
> +		seg = &rte_eal_get_configuration()->mem_config->memseg[i];
> +		if (!seg->addr)
> +			break;
> +
> +		int new_region = 1;
> +
> +		for (j = 0; j < k; ++j) {
> +			mr = &vm->regions[j];
> +
> +			if (mr->userspace_addr + mr->memory_size ==
> +			    (uint64_t)seg->addr) {
> +				mr->memory_size += seg->len;
> +				new_region = 0;
> +				break;
> +			}
> +
> +			if ((uint64_t)seg->addr + seg->len ==
> +			    mr->userspace_addr) {
> +				mr->guest_phys_addr = (uint64_t)seg->addr;
> +				mr->userspace_addr = (uint64_t)seg->addr;
> +				mr->memory_size += seg->len;
> +				new_region = 0;
> +				break;
> +			}
> +		}
> +
> +		if (new_region == 0)
> +			continue;
> +
> +		mr = &vm->regions[k++];
> +		mr->guest_phys_addr = (uint64_t)seg->addr; /* use vaddr here! */
> +		mr->userspace_addr = (uint64_t)seg->addr;
> +		mr->memory_size = seg->len;
> +		mr->mmap_offset = 0;
> +	}
> +
> +	vm->nregions = k;
> +	vm->padding = 0;
> +	*p_vm = vm;
> +}
> +
> +static void kick_all_vq(struct virtio_hw *hw)
> +{
> +	uint64_t features;
> +	unsigned i, queue_sel, nvqs;
> +	struct rte_eth_dev_data *data = hw->data;
> +
> +	if (hw->type == VHOST_KERNEL) {
> +		struct vhost_memory_kernel *vm = NULL;
> +
> +		prepare_vhost_memory_kernel(&vm);
> +		vhost_call(hw, VHOST_MSG_SET_MEM_TABLE, vm);
> +		free(vm);
> +	} else {
> +		/* construct vhost_memory inside prepare_vhost_memory_user() */
> +		vhost_call(hw, VHOST_MSG_SET_MEM_TABLE, NULL);
> +	}
> +
> +	for (i = 0; i < data->nb_rx_queues; ++i) {
> +		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
> +		kick_one_vq(hw, data->rx_queues[i], queue_sel);
> +	}
> +	for (i = 0; i < data->nb_tx_queues; ++i) {
> +		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
> +		kick_one_vq(hw, data->tx_queues[i], queue_sel);
> +	}
> +
> +	/* after setup all virtqueues, we need to set_features again
> +	 * so that these features can be set into each virtqueue in
> +	 * vhost side.
> +	 */
> +	features = hw->guest_features;
> +	features &= ~(1ull << VIRTIO_NET_F_MAC);
> +	vhost_call(hw, VHOST_MSG_SET_FEATURES, &features);
> +	if (hw->type == VHOST_KERNEL)
> +		if (ioctl(hw->backfd, TUNSETVNETHDRSZ,
> +			  &hw->vtnet_hdr_size) == -1)
> +			rte_panic("TUNSETVNETHDRSZ failed: %s\n",
> +				  strerror(errno));
> +	PMD_DRV_LOG(INFO, "set features:%" PRIx64 "\n", features);
> +
> +	if (hw->type == VHOST_KERNEL) {
> +		struct vhost_vring_file file;
> +
> +		file.fd = hw->backfd;
> +		nvqs = data->nb_rx_queues + data->nb_tx_queues;
> +		for (file.index = 0; file.index < nvqs; ++file.index) {
> +			if (vhost_kernel_ioctl(hw, VHOST_NET_SET_BACKEND,
> +					       &file) < 0)
> +				rte_panic("VHOST_NET_SET_BACKEND failed, %s\n",
> +					  strerror(errno));
> +		}
> +	}
> +}
> +
> +static void
> +vdev_read_dev_config(struct virtio_hw *hw, uint64_t offset,
> +		     void *dst, int length)
> +{
> +	if (offset == offsetof(struct virtio_net_config, mac) &&
> +	    length == ETHER_ADDR_LEN) {
> +		int i;
> +
> +		for (i = 0; i < ETHER_ADDR_LEN; ++i)
> +			((uint8_t *)dst)[i] = hw->mac_addr[i];
> +		return;
> +	}
> +
> +	if (offset == offsetof(struct virtio_net_config, status))
> +		*(uint16_t *)dst = hw->status;
> +
> +	if (offset == offsetof(struct virtio_net_config, max_virtqueue_pairs))
> +		*(uint16_t *)dst = hw->max_tx_queues;
> +}
> +
> +static void
> +vdev_write_dev_config(struct virtio_hw *hw, uint64_t offset,
> +		      const void *src, int length)
> +{
> +	int i;
> +
> +	if ((offset == offsetof(struct virtio_net_config, mac)) &&
> +	    (length == ETHER_ADDR_LEN))
> +		for (i = 0; i < ETHER_ADDR_LEN; ++i)
> +			hw->mac_addr[i] = ((const uint8_t *)src)[i];
> +	else
> +		rte_panic("offset=%" PRIu64 ", length=%d\n", offset, length);
> +}
> +
> +static void
> +vdev_set_status(struct virtio_hw *hw, uint8_t status)
> +{
> +	if (status & VIRTIO_CONFIG_S_DRIVER_OK)
> +		kick_all_vq(hw);
> +	hw->status = status;
> +}
> +
> +static void
> +vdev_reset(struct virtio_hw *hw __rte_unused)
> +{
> +	/* do nothing according to qemu vhost user spec */
> +}
> +
> +static uint8_t
> +vdev_get_status(struct virtio_hw *hw)
> +{
> +	return hw->status;
> +}
> +
> +static uint64_t
> +vdev_get_features(struct virtio_hw *hw)
> +{
> +	uint64_t host_features;
> +
> +	vhost_call(hw, VHOST_MSG_GET_FEATURES, &host_features);
> +	if (hw->mac_specified)
> +		host_features |= (1ull << VIRTIO_NET_F_MAC);
> +	/* disable it until we support CQ */
> +	host_features &= ~(1ull << VIRTIO_NET_F_CTRL_VQ);
> +	host_features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
> +	return host_features;
> +}
> +
> +static void
> +vdev_set_features(struct virtio_hw *hw, uint64_t features)
> +{
> +	features &= ~(1ull << VIRTIO_NET_F_MAC);
> +	vhost_call(hw, VHOST_MSG_SET_FEATURES, &features);
> +}
> +
> +static uint8_t
> +vdev_get_isr(struct virtio_hw *hw __rte_unused)
> +{
> +	rte_panic("");
> +}
> +
> +static uint16_t
> +vdev_set_config_irq(struct virtio_hw *hw __rte_unused,
> +		    uint16_t vec __rte_unused)
> +{
> +	rte_panic("");
> +}
> +
> +static uint16_t
> +vdev_get_queue_num(struct virtio_hw *hw,
> +		   uint16_t queue_id __rte_unused)
> +{
> +	return hw->queue_num;
> +}
> +
> +static void
> +vdev_setup_queue(struct virtio_hw *hw __rte_unused,
> +		 struct virtqueue *vq __rte_unused)
> +{
> +	/* do nothing */
> +}
> +
> +static void
> +vdev_del_queue(struct virtio_hw *hw __rte_unused,
> +	       struct virtqueue *vq)
> +{
> +	struct vhost_vring_state state = {
> +		.index = vq->vq_queue_index,
> +	};
> +
> +	vhost_call(hw, VHOST_MSG_GET_VRING_BASE, &state);
> +	PMD_DRV_LOG(DEBUG, "state.num = %d\n", state.num);
> +}
> +
> +static void
> +vdev_notify_queue(struct virtio_hw *hw, struct virtqueue *vq)
> +{
> +	uint64_t buf = 1;
> +
> +	if (write(hw->kickfds[vq->vq_queue_index],
> +		  &buf, sizeof(uint64_t)) == -1)
> +		rte_panic("%s\n", strerror(errno));
> +}
> +
> +static const struct virtio_pci_ops vdev_ops = {
> +	.read_dev_cfg	= vdev_read_dev_config,
> +	.write_dev_cfg	= vdev_write_dev_config,
> +	.reset		= vdev_reset,
> +	.get_status	= vdev_get_status,
> +	.set_status	= vdev_set_status,
> +	.get_features	= vdev_get_features,
> +	.set_features	= vdev_set_features,
> +	.get_isr	= vdev_get_isr,
> +	.set_config_irq	= vdev_set_config_irq,
> +	.get_queue_num	= vdev_get_queue_num,
> +	.setup_queue	= vdev_setup_queue,
> +	.del_queue	= vdev_del_queue,
> +	.notify_queue	= vdev_notify_queue,
> +};
> +
> +#define TUN_DEF_SNDBUF	(1ull << 20)
> +
> +static void
> +vhost_kernel_backend_setup(struct virtio_hw *hw, char *ifname)
> +{
> +	int fd;
> +	int len = sizeof(struct virtio_net_hdr);
> +	int req_mq = 0;
> +	int sndbuf = TUN_DEF_SNDBUF;
> +	unsigned int features;
> +	struct ifreq ifr;
> +
> +	/* TODO:
> +	 * 1. get/set offload capability, tap_probe_has_ufo, tap_fd_set_offload
> +	 * 2. verify we can get/set vnet_hdr_len, tap_probe_vnet_hdr_len
> +	 * 3. get number of memory regions from vhost module parameter
> +	 * max_mem_regions, supported in newer version linux kernel
> +	 */
> +
> +	fd = open(PATH_NET_TUN, O_RDWR);
> +	if (fd < 0)
> +		rte_panic("open %s error, %s\n", PATH_NET_TUN, strerror(errno));
> +
> +	memset(&ifr, 0, sizeof(ifr));
> +	ifr.ifr_flags = IFF_TAP | IFF_NO_PI;
> +
> +	if (ioctl(fd, TUNGETFEATURES, &features) == -1)
> +		rte_panic("TUNGETFEATURES failed: %s", strerror(errno));
> +
> +	if (features & IFF_ONE_QUEUE)
> +		ifr.ifr_flags |= IFF_ONE_QUEUE;
> +
> +	if (features & IFF_VNET_HDR)
> +		ifr.ifr_flags |= IFF_VNET_HDR;
> +	else
> +		rte_panic("vnet_hdr requested, but kernel does not support\n");
> +
> +	if (req_mq) {
> +		if (features & IFF_MULTI_QUEUE)
> +			ifr.ifr_flags |= IFF_MULTI_QUEUE;
> +		else
> +			rte_panic("multiqueue requested, but kernel does not support\n");
> +	}
> +
> +	if (ifname)
> +		strncpy(ifr.ifr_name, ifname, IFNAMSIZ);
> +	else
> +		strncpy(ifr.ifr_name, "tap%d", IFNAMSIZ);
> +	if (ioctl(fd, TUNSETIFF, (void *)&ifr) == -1)
> +		rte_panic("TUNSETIFF failed: %s", strerror(errno));
> +	fcntl(fd, F_SETFL, O_NONBLOCK);
> +
> +	if (ioctl(fd, TUNSETVNETHDRSZ, &len) == -1)
> +		rte_panic("TUNSETVNETHDRSZ failed: %s\n", strerror(errno));
> +
> +	if (ioctl(fd, TUNSETSNDBUF, &sndbuf) == -1)
> +		rte_panic("TUNSETSNDBUF failed: %s", strerror(errno));
> +
> +	hw->backfd = fd;
> +	hw->vhostfd = open(hw->path, O_RDWR);
> +	if (hw->vhostfd < 0)
> +		rte_panic("open %s failed: %s\n", hw->path, strerror(errno));
> +}
> +
> +static void
> +vhost_user_backend_setup(struct virtio_hw *hw)
> +{
> +	int fd;
> +	int flag;
> +	struct sockaddr_un un;
> +
> +	fd = socket(AF_UNIX, SOCK_STREAM, 0);
> +	if (fd < 0)
> +		rte_panic("socket error, %s\n", strerror(errno));
> +
> +	flag = fcntl(fd, F_GETFD);
> +	fcntl(fd, F_SETFD, flag | FD_CLOEXEC);
> +
> +	memset(&un, 0, sizeof(un));
> +	un.sun_family = AF_UNIX;
> +	snprintf(un.sun_path, sizeof(un.sun_path), "%s", hw->path);
> +	if (connect(fd, (struct sockaddr *)&un, sizeof(un)) < 0) {
> +		PMD_DRV_LOG(ERR, "connect error, %s\n", strerror(errno));
> +		rte_panic("connect error, %s\n", strerror(errno));
> +	}
> +
> +	hw->vhostfd = fd;
> +}
> +
> +void
> +virtio_vdev_init(struct rte_eth_dev_data *data, char *path,
> +		 int nb_rx, int nb_tx, int nb_cq __attribute__ ((unused)),
> +		 int queue_num, char *mac, char *ifname)
> +{
> +	int i, r;
> +	struct stat s;
> +	uint32_t tmp[ETHER_ADDR_LEN];
> +	struct virtio_hw *hw = data->dev_private;
> +
> +	hw->vtpci_ops = &vdev_ops;
> +	hw->io_base  = 0;
> +	hw->use_msix = 0;
> +	hw->modern   = 0;
> +
> +	hw->data = data;
> +	hw->path = strdup(path);
> +	hw->max_rx_queues = nb_rx;
> +	hw->max_tx_queues = nb_tx;
> +	hw->queue_num = queue_num;
> +	hw->mac_specified = 0;
> +	if (mac) {
> +		r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
> +			   &tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
> +		if (r == ETHER_ADDR_LEN) {
> +			for (i = 0; i < ETHER_ADDR_LEN; ++i)
> +				hw->mac_addr[i] = (uint8_t)tmp[i];
> +			hw->mac_specified = 1;
> +		} else
> +			PMD_DRV_LOG(WARN, "wrong format of mac: %s", mac);
> +	}
> +
> +	/* TODO: cq */
> +
> +	if (stat(hw->path, &s) < 0)
> +		rte_panic("stat: %s failed, %s\n", hw->path, strerror(errno));
> +
> +	switch (s.st_mode & S_IFMT) {
> +	case S_IFCHR:
> +		hw->type = VHOST_KERNEL;
> +		vhost_kernel_backend_setup(hw, ifname);
> +		break;
> +	case S_IFSOCK:
> +		hw->type = VHOST_USER;
> +		vhost_user_backend_setup(hw);
> +		break;
> +	default:
> +		rte_panic("unknown file type of %s\n", hw->path);
> +	}
> +	if (vhost_call(hw, VHOST_MSG_SET_OWNER, NULL) == -1)
> +		rte_panic("vhost set_owner failed: %s\n", strerror(errno));
> +}
> diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
> index fed9571..fde77ca 100644
> --- a/drivers/net/virtio/virtio_ethdev.h
> +++ b/drivers/net/virtio/virtio_ethdev.h
> @@ -123,5 +123,9 @@ uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
>  #define VTNET_LRO_FEATURES (VIRTIO_NET_F_GUEST_TSO4 | \
>  			    VIRTIO_NET_F_GUEST_TSO6 | VIRTIO_NET_F_GUEST_ECN)
>  
> -
> +#ifdef RTE_VIRTIO_VDEV
> +void virtio_vdev_init(struct rte_eth_dev_data *data, char *path, int nb_rx,
> +		      int nb_tx, int nb_cq, int queue_num, char *mac,
> +		      char *ifname);
> +#endif
>  #endif /* _VIRTIO_ETHDEV_H_ */
> diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
> index 0544a07..a8394f8 100644
> --- a/drivers/net/virtio/virtio_pci.h
> +++ b/drivers/net/virtio/virtio_pci.h
> @@ -150,7 +150,6 @@ struct virtqueue;
>   * rest are per-device feature bits.
>   */
>  #define VIRTIO_TRANSPORT_F_START 28
> -#define VIRTIO_TRANSPORT_F_END   32
>  
>  /* The Guest publishes the used index for which it expects an interrupt
>   * at the end of the avail ring. Host should ignore the avail->flags field. */
> @@ -266,6 +265,20 @@ struct virtio_hw {
>  	struct virtio_pci_common_cfg *common_cfg;
>  	struct virtio_net_config *dev_cfg;
>  	const struct virtio_pci_ops *vtpci_ops;
> +#ifdef RTE_VIRTIO_VDEV
> +#define VHOST_KERNEL	0
> +#define VHOST_USER	1
> +	int		type; /* type of backend */
> +	uint32_t	queue_num;
> +	char		*path;
> +	int		mac_specified;
> +	int		vhostfd;
> +	int		backfd; /* tap device used in vhost-net */
> +	int		callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
> +	int		kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
> +	uint8_t		status;
> +	struct rte_eth_dev_data *data;
> +#endif
>  };
>  
>  /*
> -- 
> 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 3/5] virtio/vdev: add embeded device emulation
  2016-02-05 11:20   ` [PATCH v2 3/5] virtio/vdev: add embeded device emulation Jianfeng Tan
  2016-02-07 10:47     ` Michael S. Tsirkin
@ 2016-02-08  6:59     ` Tetsuya Mukawa
  2016-02-16  2:47       ` Tan, Jianfeng
  2016-02-16  2:40     ` Tan, Jianfeng
  2 siblings, 1 reply; 196+ messages in thread
From: Tetsuya Mukawa @ 2016-02-08  6:59 UTC (permalink / raw)
  To: Jianfeng Tan, dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

On 2016/02/05 20:20, Jianfeng Tan wrote:
> To implement virtio vdev, we need way to interract with vhost backend.
> And more importantly, needs way to emulate a device into DPDK. So this
> patch acts as embedded device emulation.
>
> Depends on the type of vhost file: vhost-user is used if the given
> path points to a unix socket; vhost-net is used if the given path
> points to a char device.
>
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ---
>  
> +void
> +virtio_vdev_init(struct rte_eth_dev_data *data, char *path,
> +		 int nb_rx, int nb_tx, int nb_cq __attribute__ ((unused)),
> +		 int queue_num, char *mac, char *ifname)
> +{
> +	int i, r;
> +	struct stat s;
> +	uint32_t tmp[ETHER_ADDR_LEN];
> +	struct virtio_hw *hw = data->dev_private;
> +
> +	hw->vtpci_ops = &vdev_ops;
> +	hw->io_base  = 0;
> +	hw->use_msix = 0;
> +	hw->modern   = 0;
> +
> +	hw->data = data;
> +	hw->path = strdup(path);
> +	hw->max_rx_queues = nb_rx;
> +	hw->max_tx_queues = nb_tx;
> +	hw->queue_num = queue_num;
> +	hw->mac_specified = 0;
> +	if (mac) {
> +		r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
> +			   &tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
> +		if (r == ETHER_ADDR_LEN) {
> +			for (i = 0; i < ETHER_ADDR_LEN; ++i)
> +				hw->mac_addr[i] = (uint8_t)tmp[i];
> +			hw->mac_specified = 1;
> +		} else
> +			PMD_DRV_LOG(WARN, "wrong format of mac: %s", mac);

It seems you cannot use 'WARN' here.

Thanks,
Tetsuya

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 3/5] virtio/vdev: add embeded device emulation
  2016-02-05 11:20   ` [PATCH v2 3/5] virtio/vdev: add embeded device emulation Jianfeng Tan
  2016-02-07 10:47     ` Michael S. Tsirkin
  2016-02-08  6:59     ` Tetsuya Mukawa
@ 2016-02-16  2:40     ` Tan, Jianfeng
  2 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-02-16  2:40 UTC (permalink / raw)
  To: dev, mst; +Cc: nakajima.yoshihiro, ann.zhuangyanying


Hi Michael,

Don't know why, I have not received the email you comment on this commit.

On 2/5/2016 7:20 PM, Jianfeng Tan wrote:
> To implement virtio vdev, we need way to interract with vhost backend.
> And more importantly, needs way to emulate a device into DPDK. So this
> patch acts as embedded device emulation.
>
> Depends on the type of vhost file: vhost-user is used if the given
> path points to a unix socket; vhost-net is used if the given path
> points to a char device.
>
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ---
>   config/common_linuxapp              |   5 +
>   drivers/net/virtio/Makefile         |   4 +
>   drivers/net/virtio/vhost.h          | 194 +++++++++
>   drivers/net/virtio/vhost_embedded.c | 809 ++++++++++++++++++++++++++++++++++++
>   drivers/net/virtio/virtio_ethdev.h  |   6 +-
>   drivers/net/virtio/virtio_pci.h     |  15 +-
>   6 files changed, 1031 insertions(+), 2 deletions(-)
>   create mode 100644 drivers/net/virtio/vhost.h
>   create mode 100644 drivers/net/virtio/vhost_embedded.c
>
...
> Don't hard-code this, it's not nice.

Actually, it comes from in lib/librte_vhost/rte_virtio_net.h. If we 
follow your suggestion below, it'll be addressed.

> Why do you duplicate ioctls?
> Use them from /usr/include/linux/vhost.h, etc.
>
> In fact, what's not coming from linux here
> comes from lib/librte_vhost/vhost_user/vhost-net-user.h.
>
> I think you should reuse code, avoid code duplication.

The reason I was considering is:
a. If we include /usr/include/linux/vhost.h, then virtio cannot be used 
in FreeBSD.
b. To use definitions in lib/librte_vhost/vhost_user/vhost-net-user.h, 
we need expose this header file outside.

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 3/5] virtio/vdev: add embeded device emulation
  2016-02-08  6:59     ` Tetsuya Mukawa
@ 2016-02-16  2:47       ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-02-16  2:47 UTC (permalink / raw)
  To: Tetsuya Mukawa, dev; +Cc: nakajima.yoshihiro, mst, ann.zhuangyanying

Hi Tetsuya,

On 2/8/2016 2:59 PM, Tetsuya Mukawa wrote:
> On 2016/02/05 20:20, Jianfeng Tan wrote:
>> To implement virtio vdev, we need way to interract with vhost backend.
>> And more importantly, needs way to emulate a device into DPDK. So this
>> patch acts as embedded device emulation.
>>
>> Depends on the type of vhost file: vhost-user is used if the given
>> path points to a unix socket; vhost-net is used if the given path
>> points to a char device.
>>
>> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
>> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
>> ---
>>  
>> +void
>> +virtio_vdev_init(struct rte_eth_dev_data *data, char *path,
>> +		 int nb_rx, int nb_tx, int nb_cq __attribute__ ((unused)),
>> +		 int queue_num, char *mac, char *ifname)
>> +{
>> +	int i, r;
>> +	struct stat s;
>> +	uint32_t tmp[ETHER_ADDR_LEN];
>> +	struct virtio_hw *hw = data->dev_private;
>> +
>> +	hw->vtpci_ops = &vdev_ops;
>> +	hw->io_base  = 0;
>> +	hw->use_msix = 0;
>> +	hw->modern   = 0;
>> +
>> +	hw->data = data;
>> +	hw->path = strdup(path);
>> +	hw->max_rx_queues = nb_rx;
>> +	hw->max_tx_queues = nb_tx;
>> +	hw->queue_num = queue_num;
>> +	hw->mac_specified = 0;
>> +	if (mac) {
>> +		r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
>> +			   &tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
>> +		if (r == ETHER_ADDR_LEN) {
>> +			for (i = 0; i < ETHER_ADDR_LEN; ++i)
>> +				hw->mac_addr[i] = (uint8_t)tmp[i];
>> +			hw->mac_specified = 1;
>> +		} else
>> +			PMD_DRV_LOG(WARN, "wrong format of mac: %s", mac);
> It seems you cannot use 'WARN' here.

Thanks, I'll change it to ERR.

Thanks,
Jianfeng
>
> Thanks,
> Tetsuya
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 1/5] mem: add --single-file to create single mem-backed file
  2016-02-05 11:20   ` [PATCH v2 1/5] mem: add --single-file to create single mem-backed file Jianfeng Tan
@ 2016-03-07 13:13     ` Yuanhan Liu
  2016-03-08  1:55       ` Tan, Jianfeng
  2016-03-08  8:49       ` Panu Matilainen
  0 siblings, 2 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-03-07 13:13 UTC (permalink / raw)
  To: Jianfeng Tan; +Cc: nakajima.yoshihiro, mst, dev, p.fedin, ann.zhuangyanying

CC'ed EAL hugepage maintainer, which is something you should do when
send a patch.

On Fri, Feb 05, 2016 at 07:20:24PM +0800, Jianfeng Tan wrote:
> Originally, there're two cons in using hugepage: a. needs root
> privilege to touch /proc/self/pagemap, which is a premise to
> alllocate physically contiguous memseg; b. possibly too many
> hugepage file are created, especially used with 2M hugepage.
> 
> For virtual devices, they don't care about physical-contiguity
> of allocated hugepages at all. Option --single-file is to
> provide a way to allocate all hugepages into single mem-backed
> file.
> 
> Known issue:
> a. single-file option relys on kernel to allocate numa-affinitive
> memory.
> b. possible ABI break, originally, --no-huge uses anonymous memory
> instead of file-backed way to create memory.
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
...
> @@ -956,6 +961,16 @@ eal_check_common_options(struct internal_config *internal_cfg)
>  			"be specified together with --"OPT_NO_HUGE"\n");
>  		return -1;
>  	}
> +	if (internal_cfg->single_file && internal_cfg->force_sockets == 1) {
> +		RTE_LOG(ERR, EAL, "Option --"OPT_SINGLE_FILE" cannot "
> +			"be specified together with --"OPT_SOCKET_MEM"\n");
> +		return -1;
> +	}
> +	if (internal_cfg->single_file && internal_cfg->hugepage_unlink) {
> +		RTE_LOG(ERR, EAL, "Option --"OPT_HUGE_UNLINK" cannot "
> +			"be specified together with --"OPT_SINGLE_FILE"\n");
> +		return -1;
> +	}

The two limitation doesn't make sense to me.

> diff --git a/lib/librte_eal/linuxapp/eal/eal_memory.c b/lib/librte_eal/linuxapp/eal/eal_memory.c
> index 6008533..68ef49a 100644
> --- a/lib/librte_eal/linuxapp/eal/eal_memory.c
> +++ b/lib/librte_eal/linuxapp/eal/eal_memory.c
> @@ -1102,20 +1102,54 @@ rte_eal_hugepage_init(void)
>  	/* get pointer to global configuration */
>  	mcfg = rte_eal_get_configuration()->mem_config;
>  
> -	/* hugetlbfs can be disabled */
> -	if (internal_config.no_hugetlbfs) {
> -		addr = mmap(NULL, internal_config.memory, PROT_READ | PROT_WRITE,
> -				MAP_PRIVATE | MAP_ANONYMOUS, 0, 0);
> +	/* when hugetlbfs is disabled or single-file option is specified */
> +	if (internal_config.no_hugetlbfs || internal_config.single_file) {
> +		int fd;
> +		uint64_t pagesize;
> +		unsigned socket_id = rte_socket_id();
> +		char filepath[MAX_HUGEPAGE_PATH];
> +
> +		if (internal_config.no_hugetlbfs) {
> +			eal_get_hugefile_path(filepath, sizeof(filepath),
> +					      "/dev/shm", 0);
> +			pagesize = RTE_PGSIZE_4K;
> +		} else {
> +			struct hugepage_info *hpi;
> +
> +			hpi = &internal_config.hugepage_info[0];
> +			eal_get_hugefile_path(filepath, sizeof(filepath),
> +					      hpi->hugedir, 0);
> +			pagesize = hpi->hugepage_sz;
> +		}
> +		fd = open(filepath, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR);
> +		if (fd < 0) {
> +			RTE_LOG(ERR, EAL, "%s: open %s failed: %s\n",
> +				__func__, filepath, strerror(errno));
> +			return -1;
> +		}
> +
> +		if (ftruncate(fd, internal_config.memory) < 0) {
> +			RTE_LOG(ERR, EAL, "ftuncate %s failed: %s\n",
> +				filepath, strerror(errno));
> +			return -1;
> +		}
> +
> +		addr = mmap(NULL, internal_config.memory,
> +			    PROT_READ | PROT_WRITE,
> +			    MAP_SHARED | MAP_POPULATE, fd, 0);
>  		if (addr == MAP_FAILED) {
> -			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n", __func__,
> -					strerror(errno));
> +			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n",
> +				__func__, strerror(errno));
>  			return -1;
>  		}
>  		mcfg->memseg[0].phys_addr = (phys_addr_t)(uintptr_t)addr;
>  		mcfg->memseg[0].addr = addr;
> -		mcfg->memseg[0].hugepage_sz = RTE_PGSIZE_4K;
> +		mcfg->memseg[0].hugepage_sz = pagesize;
>  		mcfg->memseg[0].len = internal_config.memory;
> -		mcfg->memseg[0].socket_id = 0;
> +		mcfg->memseg[0].socket_id = socket_id;

I saw quite few issues:

- Assume I have a system with two hugepage sizes: 1G (x4) and 2M (x512),
  mounted at /dev/hugepages and /mnt, respectively.

  Here we then got an 5G internal_config.memory, and your code will
  try to mmap 5G on the first mount point (/dev/hugepages) due to the
  hardcode logic in your code:

      hpi = &internal_config.hugepage_info[0];
      eal_get_hugefile_path(filepath, sizeof(filepath),
      		      hpi->hugedir, 0);

  But it has 4G in total, therefore, it will fails.

- As you stated, socket_id is hardcoded, which could be wrong.

- As stated in above, the option limitation doesn't seem right to me.

  I mean, --single-file should be able to work with --socket-mem option
  in semantic.


And I have been thinking how to deal with those issues properly, and a
__very immature__ solution come to my mind (which could be simply not
working), but anyway, here is FYI: we go through the same process to
handle normal huge page initilization to --single-file option as well.
But we take different actions or no actions at all at some stages when
that option is given, which is a bit similiar with the way of handling
RTE_EAL_SINGLE_FILE_SEGMENTS.

And we create one hugepage file for each node, each page size. For a
system like mine above (2 nodes), it may populate files like following:

- 1G x 2 on node0
- 1G x 2 on node1
- 2M x 256 on node0
- 2M x 256 on node1

That could normally fit your case. Though 4 nodes looks like the maximum
node number, --socket-mem option may relieve the limit a bit.

And if we "could" not care the socket_id being set correctly, we could
just simply allocate one file for each hugepage size. That would work
well for your container enabling.

BTW, since we already have SINGLE_FILE_SEGMENTS (config) option, adding
another option --single-file looks really confusing to me.

To me, maybe you could base the SINGLE_FILE_SEGMENTS option, and add
another option, say --no-sort (I confess this name sucks, but you get
my point). With that, we could make sure to create as least huge page
files as possible, to fit your case.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 2/5] mem: add API to obtain memory-backed file info
  2016-02-05 11:20   ` [PATCH v2 2/5] mem: add API to obtain memory-backed file info Jianfeng Tan
@ 2016-03-07 13:22     ` Yuanhan Liu
  2016-03-08  2:31       ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2016-03-07 13:22 UTC (permalink / raw)
  To: Jianfeng Tan; +Cc: nakajima.yoshihiro, mst, dev, p.fedin, ann.zhuangyanying

On Fri, Feb 05, 2016 at 07:20:25PM +0800, Jianfeng Tan wrote:
> A new API named rte_eal_get_backfile_info() and a new data
> struct back_file is added to obstain information of memory-
> backed file info.

I would normally suggest to try hard to find some solution else, instead
of introducing yet another new API, espeically when you just came up with
one user only.

> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ---
>  lib/librte_eal/common/include/rte_memory.h | 16 ++++++++++++
>  lib/librte_eal/linuxapp/eal/eal_memory.c   | 40 +++++++++++++++++++++++++++++-
>  2 files changed, 55 insertions(+), 1 deletion(-)
> 
> diff --git a/lib/librte_eal/common/include/rte_memory.h b/lib/librte_eal/common/include/rte_memory.h
> index 587a25d..b09397e 100644
> --- a/lib/librte_eal/common/include/rte_memory.h
> +++ b/lib/librte_eal/common/include/rte_memory.h
> @@ -109,6 +109,22 @@ struct rte_memseg {
>  } __rte_packed;
>  
>  /**
> + * This struct is used to store information about memory-backed file that
> + * we mapped in memory initialization.
> + */
> +struct back_file {
> +	void *addr;         /**< virtual addr */
> +	size_t size;        /**< the page size */
> +	char filepath[PATH_MAX]; /**< path to backing file on filesystem */
> +};

So, that's all the info you'd like to get. I'm thinking you may don't
need another new API to retrieve them at all:

Say, you can get the filepath and fd from /proc/self/fd (by filtering it
with "rtemap_"):

    $ ls /proc/3487/fd -l
    total 0
    lrwx------ 1 root root 64 Mar  7 20:37 0 -> /dev/pts/2
    lrwx------ 1 root root 64 Mar  7 20:37 1 -> /dev/pts/2
    lrwx------ 1 root root 64 Mar  7 20:37 2 -> /dev/pts/2
    lrwx------ 1 root root 64 Mar  7 20:37 3 -> /run/.rte_config
    lr-x------ 1 root root 64 Mar  7 20:37 4 -> /dev/hugepages
    lr-x------ 1 root root 64 Mar  7 20:37 5 -> /mnt
==> lrwx------ 1 root root 64 Mar  7 20:37 6 -> /dev/hugepages/rtemap_0


Which could also save you an extra "open" at caller side for that
file as well.

And you can get the virtual addr and size from /proc/self/maps:

    $ grep rtemap_ /proc/3487/maps
    7fff40000000-7fffc0000000 rw-s 00000000 00:22 21082 /dev/hugepages/rtemap_0


Will that work for you?

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 1/5] mem: add --single-file to create single mem-backed file
  2016-03-07 13:13     ` Yuanhan Liu
@ 2016-03-08  1:55       ` Tan, Jianfeng
  2016-03-08  2:44         ` Yuanhan Liu
  2016-03-08  8:49       ` Panu Matilainen
  1 sibling, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-03-08  1:55 UTC (permalink / raw)
  To: Yuanhan Liu; +Cc: nakajima.yoshihiro, mst, dev, p.fedin, ann.zhuangyanying

Hi Yuanhan,

On 3/7/2016 9:13 PM, Yuanhan Liu wrote:
> CC'ed EAL hugepage maintainer, which is something you should do when
> send a patch.

Thanks for doing this.

>
> On Fri, Feb 05, 2016 at 07:20:24PM +0800, Jianfeng Tan wrote:
>> Originally, there're two cons in using hugepage: a. needs root
>> privilege to touch /proc/self/pagemap, which is a premise to
>> alllocate physically contiguous memseg; b. possibly too many
>> hugepage file are created, especially used with 2M hugepage.
>>
>> For virtual devices, they don't care about physical-contiguity
>> of allocated hugepages at all. Option --single-file is to
>> provide a way to allocate all hugepages into single mem-backed
>> file.
>>
>> Known issue:
>> a. single-file option relys on kernel to allocate numa-affinitive
>> memory.
>> b. possible ABI break, originally, --no-huge uses anonymous memory
>> instead of file-backed way to create memory.
>>
>> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
>> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ...
>> @@ -956,6 +961,16 @@ eal_check_common_options(struct internal_config *internal_cfg)
>>   			"be specified together with --"OPT_NO_HUGE"\n");
>>   		return -1;
>>   	}
>> +	if (internal_cfg->single_file && internal_cfg->force_sockets == 1) {
>> +		RTE_LOG(ERR, EAL, "Option --"OPT_SINGLE_FILE" cannot "
>> +			"be specified together with --"OPT_SOCKET_MEM"\n");
>> +		return -1;
>> +	}
>> +	if (internal_cfg->single_file && internal_cfg->hugepage_unlink) {
>> +		RTE_LOG(ERR, EAL, "Option --"OPT_HUGE_UNLINK" cannot "
>> +			"be specified together with --"OPT_SINGLE_FILE"\n");
>> +		return -1;
>> +	}
> The two limitation doesn't make sense to me.

For the force_sockets option, my original thought on --single-file 
option is, we don't sort those pages (require root/cap_sys_admin) and 
even don't look up numa information because it may contain both sockets' 
memory.

For the hugepage_unlink option, those hugepage files get closed in the 
end of memory initialization, if we even unlink those hugepage files, so 
we cannot share those with other processes (say backend).

>
>> diff --git a/lib/librte_eal/linuxapp/eal/eal_memory.c b/lib/librte_eal/linuxapp/eal/eal_memory.c
>> index 6008533..68ef49a 100644
>> --- a/lib/librte_eal/linuxapp/eal/eal_memory.c
>> +++ b/lib/librte_eal/linuxapp/eal/eal_memory.c
>> @@ -1102,20 +1102,54 @@ rte_eal_hugepage_init(void)
>>   	/* get pointer to global configuration */
>>   	mcfg = rte_eal_get_configuration()->mem_config;
>>   
>> -	/* hugetlbfs can be disabled */
>> -	if (internal_config.no_hugetlbfs) {
>> -		addr = mmap(NULL, internal_config.memory, PROT_READ | PROT_WRITE,
>> -				MAP_PRIVATE | MAP_ANONYMOUS, 0, 0);
>> +	/* when hugetlbfs is disabled or single-file option is specified */
>> +	if (internal_config.no_hugetlbfs || internal_config.single_file) {
>> +		int fd;
>> +		uint64_t pagesize;
>> +		unsigned socket_id = rte_socket_id();
>> +		char filepath[MAX_HUGEPAGE_PATH];
>> +
>> +		if (internal_config.no_hugetlbfs) {
>> +			eal_get_hugefile_path(filepath, sizeof(filepath),
>> +					      "/dev/shm", 0);
>> +			pagesize = RTE_PGSIZE_4K;
>> +		} else {
>> +			struct hugepage_info *hpi;
>> +
>> +			hpi = &internal_config.hugepage_info[0];
>> +			eal_get_hugefile_path(filepath, sizeof(filepath),
>> +					      hpi->hugedir, 0);
>> +			pagesize = hpi->hugepage_sz;
>> +		}
>> +		fd = open(filepath, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR);
>> +		if (fd < 0) {
>> +			RTE_LOG(ERR, EAL, "%s: open %s failed: %s\n",
>> +				__func__, filepath, strerror(errno));
>> +			return -1;
>> +		}
>> +
>> +		if (ftruncate(fd, internal_config.memory) < 0) {
>> +			RTE_LOG(ERR, EAL, "ftuncate %s failed: %s\n",
>> +				filepath, strerror(errno));
>> +			return -1;
>> +		}
>> +
>> +		addr = mmap(NULL, internal_config.memory,
>> +			    PROT_READ | PROT_WRITE,
>> +			    MAP_SHARED | MAP_POPULATE, fd, 0);
>>   		if (addr == MAP_FAILED) {
>> -			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n", __func__,
>> -					strerror(errno));
>> +			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n",
>> +				__func__, strerror(errno));
>>   			return -1;
>>   		}
>>   		mcfg->memseg[0].phys_addr = (phys_addr_t)(uintptr_t)addr;
>>   		mcfg->memseg[0].addr = addr;
>> -		mcfg->memseg[0].hugepage_sz = RTE_PGSIZE_4K;
>> +		mcfg->memseg[0].hugepage_sz = pagesize;
>>   		mcfg->memseg[0].len = internal_config.memory;
>> -		mcfg->memseg[0].socket_id = 0;
>> +		mcfg->memseg[0].socket_id = socket_id;
> I saw quite few issues:
>
> - Assume I have a system with two hugepage sizes: 1G (x4) and 2M (x512),
>    mounted at /dev/hugepages and /mnt, respectively.
>
>    Here we then got an 5G internal_config.memory, and your code will
>    try to mmap 5G on the first mount point (/dev/hugepages) due to the
>    hardcode logic in your code:
>
>        hpi = &internal_config.hugepage_info[0];
>        eal_get_hugefile_path(filepath, sizeof(filepath),
>        		      hpi->hugedir, 0);
>
>    But it has 4G in total, therefore, it will fails.

As mentioned above, this case is not for original design of --single-file.

>
> - As you stated, socket_id is hardcoded, which could be wrong.

We rely on OS to allocate hugepages, and cannot promise physical 
hugepages in the big hugepage file are from the same socket.

>
> - As stated in above, the option limitation doesn't seem right to me.
>
>    I mean, --single-file should be able to work with --socket-mem option
>    in semantic.

If we'd like to work well with --socket-mem option, we need to use 
syscalls like set_mempolicy(), mbind(). So it'll bring bigger change 
related to current one. I don't know if it's acceptable?

>
>
> And I have been thinking how to deal with those issues properly, and a
> __very immature__ solution come to my mind (which could be simply not
> working), but anyway, here is FYI: we go through the same process to
> handle normal huge page initilization to --single-file option as well.
> But we take different actions or no actions at all at some stages when
> that option is given, which is a bit similiar with the way of handling
> RTE_EAL_SINGLE_FILE_SEGMENTS.
>
> And we create one hugepage file for each node, each page size. For a
> system like mine above (2 nodes), it may populate files like following:
>
> - 1G x 2 on node0
> - 1G x 2 on node1
> - 2M x 256 on node0
> - 2M x 256 on node1
>
> That could normally fit your case. Though 4 nodes looks like the maximum
> node number, --socket-mem option may relieve the limit a bit.
>
> And if we "could" not care the socket_id being set correctly, we could
> just simply allocate one file for each hugepage size. That would work
> well for your container enabling.

This way seems a good option at first sight. Let's compare this new way 
with original design.

The original design just covers the simplest scenario:
a. just one hugetlbfs (new way can provide support for multiple number 
of hugetlbfs)
b. does not require a root privilege (new way can achieve this by using 
above-mentioned mind() or set_mempolicy() syscall)
c. no sorting (both way are OK)
d. performance, from the perspective of virtio for container, we take 
more consideration about the performance of address translation in the 
vhost. In the vhost, now we adopt a O(n) linear comparison to translate 
address (this can be optimized to O(logn) using segment tree, or even 
better using a cache, sorry, it's just another problem), so we should 
maintain as few files as possible. (new way can achieve this by used 
with --socket-mem, --huge-dir)
e. numa aware is not required (and it's complex). (new way can solve 
this without promise)

In all, this new way seems great for me.

Another thing is if "go through the same process to handle normal huge 
page initilization", my consideration is: RTE_EAL_SINGLE_FILE_SEGMENTS 
goes such way to maximize code reuse. But the new way has few common 
code with original ways. And mixing these options together leads to bad 
readability. How do you think?

>
> BTW, since we already have SINGLE_FILE_SEGMENTS (config) option, adding
> another option --single-file looks really confusing to me.
>
> To me, maybe you could base the SINGLE_FILE_SEGMENTS option, and add
> another option, say --no-sort (I confess this name sucks, but you get
> my point). With that, we could make sure to create as least huge page
> files as possible, to fit your case.

This is a great advice. So how do you think of --converged, or 
--no-scattered-mem, or any better idea?

Thanks for valuable input.

Jianfeng

>
> 	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 2/5] mem: add API to obtain memory-backed file info
  2016-03-07 13:22     ` Yuanhan Liu
@ 2016-03-08  2:31       ` Tan, Jianfeng
  2016-03-08  2:53         ` Yuanhan Liu
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-03-08  2:31 UTC (permalink / raw)
  To: Yuanhan Liu; +Cc: nakajima.yoshihiro, mst, dev, p.fedin, ann.zhuangyanying



On 3/7/2016 9:22 PM, Yuanhan Liu wrote:
> On Fri, Feb 05, 2016 at 07:20:25PM +0800, Jianfeng Tan wrote:
>> A new API named rte_eal_get_backfile_info() and a new data
>> struct back_file is added to obstain information of memory-
>> backed file info.
> I would normally suggest to try hard to find some solution else, instead
> of introducing yet another new API, espeically when you just came up with
> one user only.

Actually, Tetsuya's qtest patchset will make it two.

>
>> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
>> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
>> ---
>>   lib/librte_eal/common/include/rte_memory.h | 16 ++++++++++++
>>   lib/librte_eal/linuxapp/eal/eal_memory.c   | 40 +++++++++++++++++++++++++++++-
>>   2 files changed, 55 insertions(+), 1 deletion(-)
>>
>> diff --git a/lib/librte_eal/common/include/rte_memory.h b/lib/librte_eal/common/include/rte_memory.h
>> index 587a25d..b09397e 100644
>> --- a/lib/librte_eal/common/include/rte_memory.h
>> +++ b/lib/librte_eal/common/include/rte_memory.h
>> @@ -109,6 +109,22 @@ struct rte_memseg {
>>   } __rte_packed;
>>   
>>   /**
>> + * This struct is used to store information about memory-backed file that
>> + * we mapped in memory initialization.
>> + */
>> +struct back_file {
>> +	void *addr;         /**< virtual addr */
>> +	size_t size;        /**< the page size */
>> +	char filepath[PATH_MAX]; /**< path to backing file on filesystem */
>> +};
> So, that's all the info you'd like to get. I'm thinking you may don't
> need another new API to retrieve them at all:
>
> Say, you can get the filepath and fd from /proc/self/fd (by filtering it
> with "rtemap_"):
>
>      $ ls /proc/3487/fd -l
>      total 0
>      lrwx------ 1 root root 64 Mar  7 20:37 0 -> /dev/pts/2
>      lrwx------ 1 root root 64 Mar  7 20:37 1 -> /dev/pts/2
>      lrwx------ 1 root root 64 Mar  7 20:37 2 -> /dev/pts/2
>      lrwx------ 1 root root 64 Mar  7 20:37 3 -> /run/.rte_config
>      lr-x------ 1 root root 64 Mar  7 20:37 4 -> /dev/hugepages
>      lr-x------ 1 root root 64 Mar  7 20:37 5 -> /mnt
> ==> lrwx------ 1 root root 64 Mar  7 20:37 6 -> /dev/hugepages/rtemap_0

I guess this rtemap_xxx has been closed after memory initialization and 
cannot be obtained from /proc/xxx/fd. I believe /proc/xxx/maps is what 
you want to say.

>
>
> Which could also save you an extra "open" at caller side for that
> file as well.

Same reason, we cannot save extra "open".

>
> And you can get the virtual addr and size from /proc/self/maps:
>
>      $ grep rtemap_ /proc/3487/maps
>      7fff40000000-7fffc0000000 rw-s 00000000 00:22 21082 /dev/hugepages/rtemap_0
>
>
> Will that work for you?

Yes, from function's side, it works for me. But it needs some string 
processing. Another way is to just exposed an global variable pointing 
to the address of /run/.rte_config, so that callers extract needed 
information by themselves using "struct hugepage_file". How do you think?

Thanks,
Jianfeng

>
> 	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 1/5] mem: add --single-file to create single mem-backed file
  2016-03-08  1:55       ` Tan, Jianfeng
@ 2016-03-08  2:44         ` Yuanhan Liu
  2016-03-09 14:44           ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2016-03-08  2:44 UTC (permalink / raw)
  To: Tan, Jianfeng; +Cc: nakajima.yoshihiro, mst, dev, p.fedin, ann.zhuangyanying

On Tue, Mar 08, 2016 at 09:55:10AM +0800, Tan, Jianfeng wrote:
> Hi Yuanhan,
> 
> On 3/7/2016 9:13 PM, Yuanhan Liu wrote:
> >CC'ed EAL hugepage maintainer, which is something you should do when
> >send a patch.
> 
> Thanks for doing this.
> 
> >
> >On Fri, Feb 05, 2016 at 07:20:24PM +0800, Jianfeng Tan wrote:
> >>Originally, there're two cons in using hugepage: a. needs root
> >>privilege to touch /proc/self/pagemap, which is a premise to
> >>alllocate physically contiguous memseg; b. possibly too many
> >>hugepage file are created, especially used with 2M hugepage.
> >>
> >>For virtual devices, they don't care about physical-contiguity
> >>of allocated hugepages at all. Option --single-file is to
> >>provide a way to allocate all hugepages into single mem-backed
> >>file.
> >>
> >>Known issue:
> >>a. single-file option relys on kernel to allocate numa-affinitive
> >>memory.
> >>b. possible ABI break, originally, --no-huge uses anonymous memory
> >>instead of file-backed way to create memory.
> >>
> >>Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> >>Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> >...
> >>@@ -956,6 +961,16 @@ eal_check_common_options(struct internal_config *internal_cfg)
> >>  			"be specified together with --"OPT_NO_HUGE"\n");
> >>  		return -1;
> >>  	}
> >>+	if (internal_cfg->single_file && internal_cfg->force_sockets == 1) {
> >>+		RTE_LOG(ERR, EAL, "Option --"OPT_SINGLE_FILE" cannot "
> >>+			"be specified together with --"OPT_SOCKET_MEM"\n");
> >>+		return -1;
> >>+	}
> >>+	if (internal_cfg->single_file && internal_cfg->hugepage_unlink) {
> >>+		RTE_LOG(ERR, EAL, "Option --"OPT_HUGE_UNLINK" cannot "
> >>+			"be specified together with --"OPT_SINGLE_FILE"\n");
> >>+		return -1;
> >>+	}
> >The two limitation doesn't make sense to me.
> 
> For the force_sockets option, my original thought on --single-file option
> is, we don't sort those pages (require root/cap_sys_admin) and even don't
> look up numa information because it may contain both sockets' memory.
> 
> For the hugepage_unlink option, those hugepage files get closed in the end
> of memory initialization, if we even unlink those hugepage files, so we
> cannot share those with other processes (say backend).

Yeah, I know how the two limitations come, from your implementation. I
was just wondering if they both are __truly__ the limitations. I mean,
can we get rid of them somehow?

For --socket-mem option, if we can't handle it well, or if we could
ignore the socket_id for allocated huge page, yes, the limitation is
a true one.

But for the second option, no, we should be able to co-work it with
well. One extra action is you should not invoke "close(fd)" for those
huge page files. And then you can get all the informations as I stated
in a reply to your 2nd patch.

> >
> >>diff --git a/lib/librte_eal/linuxapp/eal/eal_memory.c b/lib/librte_eal/linuxapp/eal/eal_memory.c
> >>index 6008533..68ef49a 100644
> >>--- a/lib/librte_eal/linuxapp/eal/eal_memory.c
> >>+++ b/lib/librte_eal/linuxapp/eal/eal_memory.c
> >>@@ -1102,20 +1102,54 @@ rte_eal_hugepage_init(void)
> >>  	/* get pointer to global configuration */
> >>  	mcfg = rte_eal_get_configuration()->mem_config;
> >>-	/* hugetlbfs can be disabled */
> >>-	if (internal_config.no_hugetlbfs) {
> >>-		addr = mmap(NULL, internal_config.memory, PROT_READ | PROT_WRITE,
> >>-				MAP_PRIVATE | MAP_ANONYMOUS, 0, 0);
> >>+	/* when hugetlbfs is disabled or single-file option is specified */
> >>+	if (internal_config.no_hugetlbfs || internal_config.single_file) {
> >>+		int fd;
> >>+		uint64_t pagesize;
> >>+		unsigned socket_id = rte_socket_id();
> >>+		char filepath[MAX_HUGEPAGE_PATH];
> >>+
> >>+		if (internal_config.no_hugetlbfs) {
> >>+			eal_get_hugefile_path(filepath, sizeof(filepath),
> >>+					      "/dev/shm", 0);
> >>+			pagesize = RTE_PGSIZE_4K;
> >>+		} else {
> >>+			struct hugepage_info *hpi;
> >>+
> >>+			hpi = &internal_config.hugepage_info[0];
> >>+			eal_get_hugefile_path(filepath, sizeof(filepath),
> >>+					      hpi->hugedir, 0);
> >>+			pagesize = hpi->hugepage_sz;
> >>+		}
> >>+		fd = open(filepath, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR);
> >>+		if (fd < 0) {
> >>+			RTE_LOG(ERR, EAL, "%s: open %s failed: %s\n",
> >>+				__func__, filepath, strerror(errno));
> >>+			return -1;
> >>+		}
> >>+
> >>+		if (ftruncate(fd, internal_config.memory) < 0) {
> >>+			RTE_LOG(ERR, EAL, "ftuncate %s failed: %s\n",
> >>+				filepath, strerror(errno));
> >>+			return -1;
> >>+		}
> >>+
> >>+		addr = mmap(NULL, internal_config.memory,
> >>+			    PROT_READ | PROT_WRITE,
> >>+			    MAP_SHARED | MAP_POPULATE, fd, 0);
> >>  		if (addr == MAP_FAILED) {
> >>-			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n", __func__,
> >>-					strerror(errno));
> >>+			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n",
> >>+				__func__, strerror(errno));
> >>  			return -1;
> >>  		}
> >>  		mcfg->memseg[0].phys_addr = (phys_addr_t)(uintptr_t)addr;
> >>  		mcfg->memseg[0].addr = addr;
> >>-		mcfg->memseg[0].hugepage_sz = RTE_PGSIZE_4K;
> >>+		mcfg->memseg[0].hugepage_sz = pagesize;
> >>  		mcfg->memseg[0].len = internal_config.memory;
> >>-		mcfg->memseg[0].socket_id = 0;
> >>+		mcfg->memseg[0].socket_id = socket_id;
> >I saw quite few issues:
> >
> >- Assume I have a system with two hugepage sizes: 1G (x4) and 2M (x512),
> >   mounted at /dev/hugepages and /mnt, respectively.
> >
> >   Here we then got an 5G internal_config.memory, and your code will
> >   try to mmap 5G on the first mount point (/dev/hugepages) due to the
> >   hardcode logic in your code:
> >
> >       hpi = &internal_config.hugepage_info[0];
> >       eal_get_hugefile_path(filepath, sizeof(filepath),
> >       		      hpi->hugedir, 0);
> >
> >   But it has 4G in total, therefore, it will fails.
> 
> As mentioned above, this case is not for original design of --single-file.

But it's a so common case, isn't it?

> >
> >- As you stated, socket_id is hardcoded, which could be wrong.
> 
> We rely on OS to allocate hugepages, and cannot promise physical hugepages
> in the big hugepage file are from the same socket.
> 
> >
> >- As stated in above, the option limitation doesn't seem right to me.
> >
> >   I mean, --single-file should be able to work with --socket-mem option
> >   in semantic.
> 
> If we'd like to work well with --socket-mem option, we need to use syscalls
> like set_mempolicy(), mbind(). So it'll bring bigger change related to
> current one. I don't know if it's acceptable?

Yes, if that's the right way to go. But also as you stated, I doubt we
really need handle the numa affinitive here, due to it's complex.

> >
> >
> >And I have been thinking how to deal with those issues properly, and a
> >__very immature__ solution come to my mind (which could be simply not
> >working), but anyway, here is FYI: we go through the same process to
> >handle normal huge page initilization to --single-file option as well.
> >But we take different actions or no actions at all at some stages when
> >that option is given, which is a bit similiar with the way of handling
> >RTE_EAL_SINGLE_FILE_SEGMENTS.
> >
> >And we create one hugepage file for each node, each page size. For a
> >system like mine above (2 nodes), it may populate files like following:
> >
> >- 1G x 2 on node0
> >- 1G x 2 on node1
> >- 2M x 256 on node0
> >- 2M x 256 on node1
> >
> >That could normally fit your case. Though 4 nodes looks like the maximum
> >node number, --socket-mem option may relieve the limit a bit.
> >
> >And if we "could" not care the socket_id being set correctly, we could
> >just simply allocate one file for each hugepage size. That would work
> >well for your container enabling.
> 
> This way seems a good option at first sight. Let's compare this new way with
> original design.
> 
> The original design just covers the simplest scenario:
> a. just one hugetlbfs (new way can provide support for multiple number of
> hugetlbfs)
> b. does not require a root privilege (new way can achieve this by using
> above-mentioned mind() or set_mempolicy() syscall)
> c. no sorting (both way are OK)
> d. performance, from the perspective of virtio for container, we take more
> consideration about the performance of address translation in the vhost. In
> the vhost, now we adopt a O(n) linear comparison to translate address (this
> can be optimized to O(logn) using segment tree, or even better using a
> cache, sorry, it's just another problem), so we should maintain as few files
> as possible. (new way can achieve this by used with --socket-mem,
> --huge-dir)
> e. numa aware is not required (and it's complex). (new way can solve this
> without promise)
> 
> In all, this new way seems great for me.
> 
> Another thing is if "go through the same process to handle normal huge page
> initilization", my consideration is: RTE_EAL_SINGLE_FILE_SEGMENTS goes such
> way to maximize code reuse. But the new way has few common code with
> original ways. And mixing these options together leads to bad readability.
> How do you think?

Indeed. I've already found that the code is a bit hard to read, due to
many "#ifdef ... #else .. #endif" blocks, for RTE_EAL_SINGLE_FILE_SEGMENTS
as well as some special archs.

Therefore, I would suggest to do it as below: add another option based
on the SINGLE_FILE_SEGMENTS implementation.

I mean SINGLE_FILE_SEGMENTS already tries to generate as few files as
possible. If we add another option, say --no-sort (or --no-phys-continuity),
we could add just few lines of code to let it generate one file for
each huge page size (if we don't consider the numa affinity).

> >
> >BTW, since we already have SINGLE_FILE_SEGMENTS (config) option, adding
> >another option --single-file looks really confusing to me.
> >
> >To me, maybe you could base the SINGLE_FILE_SEGMENTS option, and add
> >another option, say --no-sort (I confess this name sucks, but you get
> >my point). With that, we could make sure to create as least huge page
> >files as possible, to fit your case.
> 
> This is a great advice. So how do you think of --converged, or
> --no-scattered-mem, or any better idea?

TBH, none of them looks great to me, either. But I have no better
options. Well, --no-phys-continuity looks like the best option to
me so far :)

	--yliu

> 
> Thanks for valuable input.
> 
> Jianfeng
> 
> >
> >	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 2/5] mem: add API to obtain memory-backed file info
  2016-03-08  2:31       ` Tan, Jianfeng
@ 2016-03-08  2:53         ` Yuanhan Liu
  0 siblings, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-03-08  2:53 UTC (permalink / raw)
  To: Tan, Jianfeng; +Cc: nakajima.yoshihiro, mst, dev, p.fedin, ann.zhuangyanying

On Tue, Mar 08, 2016 at 10:31:10AM +0800, Tan, Jianfeng wrote:
> 
> 
> On 3/7/2016 9:22 PM, Yuanhan Liu wrote:
> >On Fri, Feb 05, 2016 at 07:20:25PM +0800, Jianfeng Tan wrote:
> >>A new API named rte_eal_get_backfile_info() and a new data
> >>struct back_file is added to obstain information of memory-
> >>backed file info.
> >I would normally suggest to try hard to find some solution else, instead
> >of introducing yet another new API, espeically when you just came up with
> >one user only.
> 
> Actually, Tetsuya's qtest patchset will make it two.

Well, it's actually a same story. So, still one user to me.

> >
> >>Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> >>Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> >>---
> >>  lib/librte_eal/common/include/rte_memory.h | 16 ++++++++++++
> >>  lib/librte_eal/linuxapp/eal/eal_memory.c   | 40 +++++++++++++++++++++++++++++-
> >>  2 files changed, 55 insertions(+), 1 deletion(-)
> >>
> >>diff --git a/lib/librte_eal/common/include/rte_memory.h b/lib/librte_eal/common/include/rte_memory.h
> >>index 587a25d..b09397e 100644
> >>--- a/lib/librte_eal/common/include/rte_memory.h
> >>+++ b/lib/librte_eal/common/include/rte_memory.h
> >>@@ -109,6 +109,22 @@ struct rte_memseg {
> >>  } __rte_packed;
> >>  /**
> >>+ * This struct is used to store information about memory-backed file that
> >>+ * we mapped in memory initialization.
> >>+ */
> >>+struct back_file {
> >>+	void *addr;         /**< virtual addr */
> >>+	size_t size;        /**< the page size */
> >>+	char filepath[PATH_MAX]; /**< path to backing file on filesystem */
> >>+};
> >So, that's all the info you'd like to get. I'm thinking you may don't
> >need another new API to retrieve them at all:
> >
> >Say, you can get the filepath and fd from /proc/self/fd (by filtering it
> >with "rtemap_"):
> >
> >     $ ls /proc/3487/fd -l
> >     total 0
> >     lrwx------ 1 root root 64 Mar  7 20:37 0 -> /dev/pts/2
> >     lrwx------ 1 root root 64 Mar  7 20:37 1 -> /dev/pts/2
> >     lrwx------ 1 root root 64 Mar  7 20:37 2 -> /dev/pts/2
> >     lrwx------ 1 root root 64 Mar  7 20:37 3 -> /run/.rte_config
> >     lr-x------ 1 root root 64 Mar  7 20:37 4 -> /dev/hugepages
> >     lr-x------ 1 root root 64 Mar  7 20:37 5 -> /mnt
> >==> lrwx------ 1 root root 64 Mar  7 20:37 6 -> /dev/hugepages/rtemap_0
> 
> I guess this rtemap_xxx has been closed after memory initialization and
> cannot be obtained from /proc/xxx/fd. I believe /proc/xxx/maps is what you
> want to say.

Yes, I forgot to mention that you need keep that file open.
So, you just need one line or two to not close that file
in this case.

> 
> >
> >
> >Which could also save you an extra "open" at caller side for that
> >file as well.
> 
> Same reason, we cannot save extra "open".

We could, if we keep the file open.

> >
> >And you can get the virtual addr and size from /proc/self/maps:
> >
> >     $ grep rtemap_ /proc/3487/maps
> >     7fff40000000-7fffc0000000 rw-s 00000000 00:22 21082 /dev/hugepages/rtemap_0
> >
> >
> >Will that work for you?
> 
> Yes, from function's side, it works for me. But it needs some string
> processing.

What's wrong of the string processing? I have seen many string
processings in DPDK code, even in rte_memory.c.

> Another way is to just exposed an global variable pointing to
> the address of /run/.rte_config, so that callers extract needed information
> by themselves using "struct hugepage_file". How do you think?

That doens't seem elegant to me.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 1/5] mem: add --single-file to create single mem-backed file
  2016-03-07 13:13     ` Yuanhan Liu
  2016-03-08  1:55       ` Tan, Jianfeng
@ 2016-03-08  8:49       ` Panu Matilainen
  2016-03-08  9:04         ` Yuanhan Liu
  1 sibling, 1 reply; 196+ messages in thread
From: Panu Matilainen @ 2016-03-08  8:49 UTC (permalink / raw)
  To: Yuanhan Liu, Jianfeng Tan
  Cc: nakajima.yoshihiro, mst, dev, p.fedin, ann.zhuangyanying

On 03/07/2016 03:13 PM, Yuanhan Liu wrote:
> CC'ed EAL hugepage maintainer, which is something you should do when
> send a patch.
>
> On Fri, Feb 05, 2016 at 07:20:24PM +0800, Jianfeng Tan wrote:
>> Originally, there're two cons in using hugepage: a. needs root
>> privilege to touch /proc/self/pagemap, which is a premise to
>> alllocate physically contiguous memseg; b. possibly too many
>> hugepage file are created, especially used with 2M hugepage.
>>
>> For virtual devices, they don't care about physical-contiguity
>> of allocated hugepages at all. Option --single-file is to
>> provide a way to allocate all hugepages into single mem-backed
>> file.
>>
>> Known issue:
>> a. single-file option relys on kernel to allocate numa-affinitive
>> memory.
>> b. possible ABI break, originally, --no-huge uses anonymous memory
>> instead of file-backed way to create memory.
>>
>> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
>> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ...
>> @@ -956,6 +961,16 @@ eal_check_common_options(struct internal_config *internal_cfg)
>>   			"be specified together with --"OPT_NO_HUGE"\n");
>>   		return -1;
>>   	}
>> +	if (internal_cfg->single_file && internal_cfg->force_sockets == 1) {
>> +		RTE_LOG(ERR, EAL, "Option --"OPT_SINGLE_FILE" cannot "
>> +			"be specified together with --"OPT_SOCKET_MEM"\n");
>> +		return -1;
>> +	}
>> +	if (internal_cfg->single_file && internal_cfg->hugepage_unlink) {
>> +		RTE_LOG(ERR, EAL, "Option --"OPT_HUGE_UNLINK" cannot "
>> +			"be specified together with --"OPT_SINGLE_FILE"\n");
>> +		return -1;
>> +	}
>
> The two limitation doesn't make sense to me.
>
>> diff --git a/lib/librte_eal/linuxapp/eal/eal_memory.c b/lib/librte_eal/linuxapp/eal/eal_memory.c
>> index 6008533..68ef49a 100644
>> --- a/lib/librte_eal/linuxapp/eal/eal_memory.c
>> +++ b/lib/librte_eal/linuxapp/eal/eal_memory.c
>> @@ -1102,20 +1102,54 @@ rte_eal_hugepage_init(void)
>>   	/* get pointer to global configuration */
>>   	mcfg = rte_eal_get_configuration()->mem_config;
>>
>> -	/* hugetlbfs can be disabled */
>> -	if (internal_config.no_hugetlbfs) {
>> -		addr = mmap(NULL, internal_config.memory, PROT_READ | PROT_WRITE,
>> -				MAP_PRIVATE | MAP_ANONYMOUS, 0, 0);
>> +	/* when hugetlbfs is disabled or single-file option is specified */
>> +	if (internal_config.no_hugetlbfs || internal_config.single_file) {
>> +		int fd;
>> +		uint64_t pagesize;
>> +		unsigned socket_id = rte_socket_id();
>> +		char filepath[MAX_HUGEPAGE_PATH];
>> +
>> +		if (internal_config.no_hugetlbfs) {
>> +			eal_get_hugefile_path(filepath, sizeof(filepath),
>> +					      "/dev/shm", 0);
>> +			pagesize = RTE_PGSIZE_4K;
>> +		} else {
>> +			struct hugepage_info *hpi;
>> +
>> +			hpi = &internal_config.hugepage_info[0];
>> +			eal_get_hugefile_path(filepath, sizeof(filepath),
>> +					      hpi->hugedir, 0);
>> +			pagesize = hpi->hugepage_sz;
>> +		}
>> +		fd = open(filepath, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR);
>> +		if (fd < 0) {
>> +			RTE_LOG(ERR, EAL, "%s: open %s failed: %s\n",
>> +				__func__, filepath, strerror(errno));
>> +			return -1;
>> +		}
>> +
>> +		if (ftruncate(fd, internal_config.memory) < 0) {
>> +			RTE_LOG(ERR, EAL, "ftuncate %s failed: %s\n",
>> +				filepath, strerror(errno));
>> +			return -1;
>> +		}
>> +
>> +		addr = mmap(NULL, internal_config.memory,
>> +			    PROT_READ | PROT_WRITE,
>> +			    MAP_SHARED | MAP_POPULATE, fd, 0);
>>   		if (addr == MAP_FAILED) {
>> -			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n", __func__,
>> -					strerror(errno));
>> +			RTE_LOG(ERR, EAL, "%s: mmap() failed: %s\n",
>> +				__func__, strerror(errno));
>>   			return -1;
>>   		}
>>   		mcfg->memseg[0].phys_addr = (phys_addr_t)(uintptr_t)addr;
>>   		mcfg->memseg[0].addr = addr;
>> -		mcfg->memseg[0].hugepage_sz = RTE_PGSIZE_4K;
>> +		mcfg->memseg[0].hugepage_sz = pagesize;
>>   		mcfg->memseg[0].len = internal_config.memory;
>> -		mcfg->memseg[0].socket_id = 0;
>> +		mcfg->memseg[0].socket_id = socket_id;
>
> I saw quite few issues:
>
> - Assume I have a system with two hugepage sizes: 1G (x4) and 2M (x512),
>    mounted at /dev/hugepages and /mnt, respectively.
>
>    Here we then got an 5G internal_config.memory, and your code will
>    try to mmap 5G on the first mount point (/dev/hugepages) due to the
>    hardcode logic in your code:
>
>        hpi = &internal_config.hugepage_info[0];
>        eal_get_hugefile_path(filepath, sizeof(filepath),
>        		      hpi->hugedir, 0);
>
>    But it has 4G in total, therefore, it will fails.
>
> - As you stated, socket_id is hardcoded, which could be wrong.
>
> - As stated in above, the option limitation doesn't seem right to me.
>
>    I mean, --single-file should be able to work with --socket-mem option
>    in semantic.
>
>
> And I have been thinking how to deal with those issues properly, and a
> __very immature__ solution come to my mind (which could be simply not
> working), but anyway, here is FYI: we go through the same process to
> handle normal huge page initilization to --single-file option as well.
> But we take different actions or no actions at all at some stages when
> that option is given, which is a bit similiar with the way of handling
> RTE_EAL_SINGLE_FILE_SEGMENTS.
>
> And we create one hugepage file for each node, each page size. For a
> system like mine above (2 nodes), it may populate files like following:
>
> - 1G x 2 on node0
> - 1G x 2 on node1
> - 2M x 256 on node0
> - 2M x 256 on node1
>
> That could normally fit your case. Though 4 nodes looks like the maximum
> node number, --socket-mem option may relieve the limit a bit.
>
> And if we "could" not care the socket_id being set correctly, we could
> just simply allocate one file for each hugepage size. That would work
> well for your container enabling.
>
> BTW, since we already have SINGLE_FILE_SEGMENTS (config) option, adding
> another option --single-file looks really confusing to me.
>
> To me, maybe you could base the SINGLE_FILE_SEGMENTS option, and add
> another option, say --no-sort (I confess this name sucks, but you get
> my point). With that, we could make sure to create as least huge page
> files as possible, to fit your case.

Note that SINGLE_FILE_SEGMENTS is a nasty hack that only the IVSHMEM 
config uses, getting rid of it (by replacing with a runtime switch) 
would be great. OTOH IVSHMEM itself seems to have fallen out of the 
fashion since the memnic driver is unmaintained and broken since dpdk 
2.0... CC'ing the IVSHMEM maintainer in case he has thoughts on this.

	- Panu -

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 1/5] mem: add --single-file to create single mem-backed file
  2016-03-08  8:49       ` Panu Matilainen
@ 2016-03-08  9:04         ` Yuanhan Liu
  2016-03-08 10:30           ` Thomas Monjalon
  0 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2016-03-08  9:04 UTC (permalink / raw)
  To: Panu Matilainen; +Cc: nakajima.yoshihiro, mst, dev, p.fedin, ann.zhuangyanying

On Tue, Mar 08, 2016 at 10:49:30AM +0200, Panu Matilainen wrote:
> On 03/07/2016 03:13 PM, Yuanhan Liu wrote:
> >To me, maybe you could base the SINGLE_FILE_SEGMENTS option, and add
> >another option, say --no-sort (I confess this name sucks, but you get
> >my point). With that, we could make sure to create as least huge page
> >files as possible, to fit your case.
> 
> Note that SINGLE_FILE_SEGMENTS is a nasty hack that only the IVSHMEM config
> uses, getting rid of it (by replacing with a runtime switch) would be great.

Can't agree more.

BTW, FYI, Jianfeng and I had a private talk, and we came to agree that
it might be better to handle it outside the normal huge page init stage,
just like this patch does, but adding the support of multiple huge page
sizes. Let's not add more messy code there.

	--yliu

> OTOH IVSHMEM itself seems to have fallen out of the fashion since the memnic
> driver is unmaintained and broken since dpdk 2.0... CC'ing the IVSHMEM
> maintainer in case he has thoughts on this.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 1/5] mem: add --single-file to create single mem-backed file
  2016-03-08  9:04         ` Yuanhan Liu
@ 2016-03-08 10:30           ` Thomas Monjalon
  2016-03-08 10:57             ` Burakov, Anatoly
  2016-03-14 13:53             ` Traynor, Kevin
  0 siblings, 2 replies; 196+ messages in thread
From: Thomas Monjalon @ 2016-03-08 10:30 UTC (permalink / raw)
  To: dev; +Cc: nakajima.yoshihiro, mst, p.fedin, ann.zhuangyanying

2016-03-08 17:04, Yuanhan Liu:
> On Tue, Mar 08, 2016 at 10:49:30AM +0200, Panu Matilainen wrote:
> > On 03/07/2016 03:13 PM, Yuanhan Liu wrote:
> > >To me, maybe you could base the SINGLE_FILE_SEGMENTS option, and add
> > >another option, say --no-sort (I confess this name sucks, but you get
> > >my point). With that, we could make sure to create as least huge page
> > >files as possible, to fit your case.
> > 
> > Note that SINGLE_FILE_SEGMENTS is a nasty hack that only the IVSHMEM config
> > uses, getting rid of it (by replacing with a runtime switch) would be great.
> 
> Can't agree more.

+1

> BTW, FYI, Jianfeng and I had a private talk, and we came to agree that
> it might be better to handle it outside the normal huge page init stage,
> just like this patch does, but adding the support of multiple huge page
> sizes. Let's not add more messy code there.
> 
> 	--yliu
> 
> > OTOH IVSHMEM itself seems to have fallen out of the fashion since the memnic
> > driver is unmaintained and broken since dpdk 2.0... CC'ing the IVSHMEM
> > maintainer in case he has thoughts on this.

The ivshmem config was not used for memnic which was using ivshmem only for
data path.
CONFIG_RTE_LIBRTE_IVSHMEM and CONFIG_RTE_EAL_SINGLE_FILE_SEGMENTS are more
about full memory sharing.
I have the feeling it could be dropped.
It there are some users, I'd like to see a justification and a rework to
remove these build options.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 1/5] mem: add --single-file to create single mem-backed file
  2016-03-08 10:30           ` Thomas Monjalon
@ 2016-03-08 10:57             ` Burakov, Anatoly
  2016-03-14 13:53             ` Traynor, Kevin
  1 sibling, 0 replies; 196+ messages in thread
From: Burakov, Anatoly @ 2016-03-08 10:57 UTC (permalink / raw)
  To: Thomas Monjalon, dev; +Cc: nakajima.yoshihiro, mst, p.fedin, ann.zhuangyanying

Hi Thomas,

> 2016-03-08 17:04, Yuanhan Liu:
> > On Tue, Mar 08, 2016 at 10:49:30AM +0200, Panu Matilainen wrote:
> > > On 03/07/2016 03:13 PM, Yuanhan Liu wrote:
> > > >To me, maybe you could base the SINGLE_FILE_SEGMENTS option, and
> > > >add another option, say --no-sort (I confess this name sucks, but
> > > >you get my point). With that, we could make sure to create as least
> > > >huge page files as possible, to fit your case.
> > >
> > > Note that SINGLE_FILE_SEGMENTS is a nasty hack that only the
> IVSHMEM
> > > config uses, getting rid of it (by replacing with a runtime switch) would be
> great.
> >
> > Can't agree more.
> 
> +1
> 
> > BTW, FYI, Jianfeng and I had a private talk, and we came to agree that
> > it might be better to handle it outside the normal huge page init
> > stage, just like this patch does, but adding the support of multiple
> > huge page sizes. Let's not add more messy code there.
> >
> > 	--yliu
> >
> > > OTOH IVSHMEM itself seems to have fallen out of the fashion since
> > > the memnic driver is unmaintained and broken since dpdk 2.0...
> > > CC'ing the IVSHMEM maintainer in case he has thoughts on this.
> 
> The ivshmem config was not used for memnic which was using ivshmem only
> for data path.
> CONFIG_RTE_LIBRTE_IVSHMEM and
> CONFIG_RTE_EAL_SINGLE_FILE_SEGMENTS are more about full memory
> sharing.
> I have the feeling it could be dropped.
> It there are some users, I'd like to see a justification and a rework to remove
> these build options.

Just to add my opinion to it - if there are no users for both of these, I'd like for those to be removed as well. Less maintenance is always better than more maintenance, especially for things that no one uses :)

Thanks,
Anatoly

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 1/5] mem: add --single-file to create single mem-backed file
  2016-03-08  2:44         ` Yuanhan Liu
@ 2016-03-09 14:44           ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-03-09 14:44 UTC (permalink / raw)
  To: Yuanhan Liu; +Cc: nakajima.yoshihiro, mst, dev, p.fedin, ann.zhuangyanying

Hi,

On 3/8/2016 10:44 AM, Yuanhan Liu wrote:
> On Tue, Mar 08, 2016 at 09:55:10AM +0800, Tan, Jianfeng wrote:
>> Hi Yuanhan,
>>
>> On 3/7/2016 9:13 PM, Yuanhan Liu wrote:
>>> CC'ed EAL hugepage maintainer, which is something you should do when
>>> send a patch.
>> Thanks for doing this.
>>
>>> On Fri, Feb 05, 2016 at 07:20:24PM +0800, Jianfeng Tan wrote:
>>>> Originally, there're two cons in using hugepage: a. needs root
>>>> privilege to touch /proc/self/pagemap, which is a premise to
>>>> alllocate physically contiguous memseg; b. possibly too many
>>>> hugepage file are created, especially used with 2M hugepage.
>>>>
>>>> For virtual devices, they don't care about physical-contiguity
>>>> of allocated hugepages at all. Option --single-file is to
>>>> provide a way to allocate all hugepages into single mem-backed
>>>> file.
>>>>
>>>> Known issue:
>>>> a. single-file option relys on kernel to allocate numa-affinitive
>>>> memory.
>>>> b. possible ABI break, originally, --no-huge uses anonymous memory
>>>> instead of file-backed way to create memory.
>>>>
>>>> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
>>>> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
>>> ...
>>>> @@ -956,6 +961,16 @@ eal_check_common_options(struct internal_config *internal_cfg)
>>>>   			"be specified together with --"OPT_NO_HUGE"\n");
>>>>   		return -1;
>>>>   	}
>>>> +	if (internal_cfg->single_file && internal_cfg->force_sockets == 1) {
>>>> +		RTE_LOG(ERR, EAL, "Option --"OPT_SINGLE_FILE" cannot "
>>>> +			"be specified together with --"OPT_SOCKET_MEM"\n");
>>>> +		return -1;
>>>> +	}
>>>> +	if (internal_cfg->single_file && internal_cfg->hugepage_unlink) {
>>>> +		RTE_LOG(ERR, EAL, "Option --"OPT_HUGE_UNLINK" cannot "
>>>> +			"be specified together with --"OPT_SINGLE_FILE"\n");
>>>> +		return -1;
>>>> +	}
>>> The two limitation doesn't make sense to me.
>> For the force_sockets option, my original thought on --single-file option
>> is, we don't sort those pages (require root/cap_sys_admin) and even don't
>> look up numa information because it may contain both sockets' memory.
>>
>> For the hugepage_unlink option, those hugepage files get closed in the end
>> of memory initialization, if we even unlink those hugepage files, so we
>> cannot share those with other processes (say backend).
> Yeah, I know how the two limitations come, from your implementation. I
> was just wondering if they both are __truly__ the limitations. I mean,
> can we get rid of them somehow?
>
> For --socket-mem option, if we can't handle it well, or if we could
> ignore the socket_id for allocated huge page, yes, the limitation is
> a true one.

To make it work with --socket-mem option, we need to call 
mbind()/set_mempolicy(), which leads to including "LDFLAGS += -lnuma" a 
mandatory line in mk file. Don't know if it's  acceptable to bring in 
dependency on libnuma.so?


>
> But for the second option, no, we should be able to co-work it with
> well. One extra action is you should not invoke "close(fd)" for those
> huge page files. And then you can get all the informations as I stated
> in a reply to your 2nd patch.

As discussed yesterday, I think there's a open files limitation for each 
process, if we keep those FDs open, it will bring failure to those 
existing programs. If others treat it as a problem?
...
>>> BTW, since we already have SINGLE_FILE_SEGMENTS (config) option, adding
>>> another option --single-file looks really confusing to me.
>>>
>>> To me, maybe you could base the SINGLE_FILE_SEGMENTS option, and add
>>> another option, say --no-sort (I confess this name sucks, but you get
>>> my point). With that, we could make sure to create as least huge page
>>> files as possible, to fit your case.
>> This is a great advice. So how do you think of --converged, or
>> --no-scattered-mem, or any better idea?
> TBH, none of them looks great to me, either. But I have no better
> options. Well, --no-phys-continuity looks like the best option to
> me so far :)

I'd like to make it a little more concise, how about --no-phys-contig? 
In addition, Yuanhan thinks there's still no literal meaning that just 
create one file for each hugetlbfs (or socket). But from my side, 
there's an indirect meaning, because if no need to promise 
physically-contig, then no need to create hugepages one by one. Anyone 
can give your option here? Thanks.

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 1/5] mem: add --single-file to create single mem-backed file
  2016-03-08 10:30           ` Thomas Monjalon
  2016-03-08 10:57             ` Burakov, Anatoly
@ 2016-03-14 13:53             ` Traynor, Kevin
  2016-03-14 14:45               ` Thomas Monjalon
  1 sibling, 1 reply; 196+ messages in thread
From: Traynor, Kevin @ 2016-03-14 13:53 UTC (permalink / raw)
  To: Thomas Monjalon, dev; +Cc: nakajima.yoshihiro, mst, p.fedin, ann.zhuangyanying

> -----Original Message-----
> From: dev [mailto:dev-bounces@dpdk.org] On Behalf Of Thomas Monjalon
> Sent: Tuesday, March 8, 2016 10:31 AM
> To: dev@dpdk.org
> Cc: nakajima.yoshihiro@lab.ntt.co.jp; mst@redhat.com; p.fedin@samsung.com;
> ann.zhuangyanying@huawei.com
> Subject: Re: [dpdk-dev] [PATCH v2 1/5] mem: add --single-file to create
> single mem-backed file
> 
> 2016-03-08 17:04, Yuanhan Liu:
> > On Tue, Mar 08, 2016 at 10:49:30AM +0200, Panu Matilainen wrote:
> > > On 03/07/2016 03:13 PM, Yuanhan Liu wrote:
> > > >To me, maybe you could base the SINGLE_FILE_SEGMENTS option, and add
> > > >another option, say --no-sort (I confess this name sucks, but you get
> > > >my point). With that, we could make sure to create as least huge page
> > > >files as possible, to fit your case.
> > >
> > > Note that SINGLE_FILE_SEGMENTS is a nasty hack that only the IVSHMEM
> config
> > > uses, getting rid of it (by replacing with a runtime switch) would be
> great.
> >
> > Can't agree more.
> 
> +1
> 
> > BTW, FYI, Jianfeng and I had a private talk, and we came to agree that
> > it might be better to handle it outside the normal huge page init stage,
> > just like this patch does, but adding the support of multiple huge page
> > sizes. Let's not add more messy code there.
> >
> > 	--yliu
> >
> > > OTOH IVSHMEM itself seems to have fallen out of the fashion since the
> memnic
> > > driver is unmaintained and broken since dpdk 2.0... CC'ing the IVSHMEM
> > > maintainer in case he has thoughts on this.
> 
> The ivshmem config was not used for memnic which was using ivshmem only for
> data path.
> CONFIG_RTE_LIBRTE_IVSHMEM and CONFIG_RTE_EAL_SINGLE_FILE_SEGMENTS are more
> about full memory sharing.
> I have the feeling it could be dropped.
> It there are some users, I'd like to see a justification and a rework to
> remove these build options.

Just to clarify - is this suggesting the removal of the IVSHMEM library itself,
or just some of the config options?

The reason I ask is that although we don't currently use it in OVS with DPDK,
I've seen at least one person using it in conjunction with the ring interface.
There may be others, so I want to cross-post if there's a deprecation discussion. 

Kevin.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 1/5] mem: add --single-file to create single mem-backed file
  2016-03-14 13:53             ` Traynor, Kevin
@ 2016-03-14 14:45               ` Thomas Monjalon
  2016-03-14 18:21                 ` Traynor, Kevin
  0 siblings, 1 reply; 196+ messages in thread
From: Thomas Monjalon @ 2016-03-14 14:45 UTC (permalink / raw)
  To: Traynor, Kevin; +Cc: dev, nakajima.yoshihiro, mst, p.fedin, ann.zhuangyanying

2016-03-14 13:53, Traynor, Kevin:
> From: Thomas Monjalon
> > 2016-03-08 17:04, Yuanhan Liu:
> > > On Tue, Mar 08, 2016 at 10:49:30AM +0200, Panu Matilainen wrote:
> > > > On 03/07/2016 03:13 PM, Yuanhan Liu wrote:
> > > > Note that SINGLE_FILE_SEGMENTS is a nasty hack that only the IVSHMEM
> > config
> > > > uses, getting rid of it (by replacing with a runtime switch) would be
> > great.
> > >
> > > Can't agree more.
> > 
> > +1
> > 
> > > > OTOH IVSHMEM itself seems to have fallen out of the fashion since the
> > memnic
> > > > driver is unmaintained and broken since dpdk 2.0... CC'ing the IVSHMEM
> > > > maintainer in case he has thoughts on this.
> > 
> > The ivshmem config was not used for memnic which was using ivshmem only for
> > data path.
> > CONFIG_RTE_LIBRTE_IVSHMEM and CONFIG_RTE_EAL_SINGLE_FILE_SEGMENTS are more
> > about full memory sharing.
> > I have the feeling it could be dropped.
> > It there are some users, I'd like to see a justification and a rework to
> > remove these build options.
> 
> Just to clarify - is this suggesting the removal of the IVSHMEM library itself,
> or just some of the config options?

I have no strong opinion about the library.
About the config options, yes they should be removed. Note that they are not
documented, so we don't really know the motivation to have them.

> The reason I ask is that although we don't currently use it in OVS with DPDK,
> I've seen at least one person using it in conjunction with the ring interface.
> There may be others, so I want to cross-post if there's a deprecation discussion. 

Thank you for sharing.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 1/5] mem: add --single-file to create single mem-backed file
  2016-03-14 14:45               ` Thomas Monjalon
@ 2016-03-14 18:21                 ` Traynor, Kevin
  0 siblings, 0 replies; 196+ messages in thread
From: Traynor, Kevin @ 2016-03-14 18:21 UTC (permalink / raw)
  To: Thomas Monjalon; +Cc: dev, nakajima.yoshihiro, mst, p.fedin, ann.zhuangyanying

> -----Original Message-----
> From: Thomas Monjalon [mailto:thomas.monjalon@6wind.com]
> Sent: Monday, March 14, 2016 2:45 PM
> To: Traynor, Kevin <kevin.traynor@intel.com>
> Cc: dev@dpdk.org; nakajima.yoshihiro@lab.ntt.co.jp; mst@redhat.com;
> p.fedin@samsung.com; ann.zhuangyanying@huawei.com
> Subject: Re: [dpdk-dev] [PATCH v2 1/5] mem: add --single-file to create
> single mem-backed file
> 
> 2016-03-14 13:53, Traynor, Kevin:
> > From: Thomas Monjalon
> > > 2016-03-08 17:04, Yuanhan Liu:
> > > > On Tue, Mar 08, 2016 at 10:49:30AM +0200, Panu Matilainen wrote:
> > > > > On 03/07/2016 03:13 PM, Yuanhan Liu wrote:
> > > > > Note that SINGLE_FILE_SEGMENTS is a nasty hack that only the IVSHMEM
> > > config
> > > > > uses, getting rid of it (by replacing with a runtime switch) would be
> > > great.
> > > >
> > > > Can't agree more.
> > >
> > > +1
> > >
> > > > > OTOH IVSHMEM itself seems to have fallen out of the fashion since the
> > > memnic
> > > > > driver is unmaintained and broken since dpdk 2.0... CC'ing the
> IVSHMEM
> > > > > maintainer in case he has thoughts on this.
> > >
> > > The ivshmem config was not used for memnic which was using ivshmem only
> for
> > > data path.
> > > CONFIG_RTE_LIBRTE_IVSHMEM and CONFIG_RTE_EAL_SINGLE_FILE_SEGMENTS are
> more
> > > about full memory sharing.
> > > I have the feeling it could be dropped.
> > > It there are some users, I'd like to see a justification and a rework to
> > > remove these build options.
> >
> > Just to clarify - is this suggesting the removal of the IVSHMEM library
> itself,
> > or just some of the config options?
> 
> I have no strong opinion about the library.
> About the config options, yes they should be removed. Note that they are not
> documented, so we don't really know the motivation to have them.

ok, thanks for clarifying. As there's no imminent plans to remove the library,
I won't cross post. 

> 
> > The reason I ask is that although we don't currently use it in OVS with
> DPDK,
> > I've seen at least one person using it in conjunction with the ring
> interface.
> > There may be others, so I want to cross-post if there's a deprecation
> discussion.
> 
> Thank you for sharing.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 0/5] virtio support for container
  2016-02-05 11:20 ` [PATCH v2 0/5] " Jianfeng Tan
                     ` (4 preceding siblings ...)
  2016-02-05 11:20   ` [PATCH v2 5/5] docs: add release note for virtio for container Jianfeng Tan
@ 2016-03-23 19:17   ` Neil Horman
  2016-03-24  3:10     ` Tan, Jianfeng
  2016-04-13 16:14   ` Thomas Monjalon
  6 siblings, 1 reply; 196+ messages in thread
From: Neil Horman @ 2016-03-23 19:17 UTC (permalink / raw)
  To: jianfeng.tan; +Cc: dev

On Fri, Feb 05, 2016 at 07:20:23PM +0800, Jianfeng Tan wrote:
> v1->v2:
>  - Rebase on the patchset of virtio 1.0 support.
>  - Fix cannot create non-hugepage memory.
>  - Fix wrong size of memory region when "single-file" is used.
>  - Fix setting of offset in virtqueue to use virtual address.
>  - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
>  - Add mac option to specify the mac address of this virtual device.
>  - Update doc.
> 
> This patchset is to provide high performance networking interface (virtio)
> for container-based DPDK applications. The way of starting DPDK apps in
> containers with ownership of NIC devices exclusively is beyond the scope.
> The basic idea here is to present a new virtual device (named eth_cvio),
> which can be discovered and initialized in container-based DPDK apps using
> rte_eal_init(). To minimize the change, we reuse already-existing virtio
> frontend driver code (driver/net/virtio/).
>  
> Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
> operations into unix socket/cuse protocol, which is originally provided in
> QEMU), is integrated in virtio frontend driver. So this converged driver
> actually plays the role of original frontend driver and the role of QEMU
> device framework.
>  
> The major difference lies in how to calculate relative address for vhost.
> The principle of virtio is that: based on one or multiple shared memory
> segments, vhost maintains a reference system with the base addresses and
> length for each segment so that an address from VM comes (usually GPA,
> Guest Physical Address) can be translated into vhost-recognizable address
> (named VVA, Vhost Virtual Address). To decrease the overhead of address
> translation, we should maintain as few segments as possible. In VM's case,
> GPA is always locally continuous. In container's case, CVA (Container
> Virtual Address) can be used. Specifically:
> a. when set_base_addr, CVA address is used;
> b. when preparing RX's descriptors, CVA address is used;
> c. when transmitting packets, CVA is filled in TX's descriptors;
> d. in TX and CQ's header, CVA is used.
>  
> How to share memory? In VM's case, qemu always shares all physical layout
> to backend. But it's not feasible for a container, as a process, to share
> all virtual memory regions to backend. So only specified virtual memory
> regions (with type of shared) are sent to backend. It's a limitation that
> only addresses in these areas can be used to transmit or receive packets.
> 
> Known issues
> 
> a. When used with vhost-net, root privilege is required to create tap
> device inside.
> b. Control queue and multi-queue are not supported yet.
> c. When --single-file option is used, socket_id of the memory may be
> wrong. (Use "numactl -N x -m x" to work around this for now)
>  
> How to use?
> 
> a. Apply this patchset.
> 
> b. To compile container apps:
> $: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> 
> c. To build a docker image using Dockerfile below.
> $: cat ./Dockerfile
> FROM ubuntu:latest
> WORKDIR /usr/src/dpdk
> COPY . /usr/src/dpdk
> ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
> $: docker build -t dpdk-app-l2fwd .
> 
> d. Used with vhost-user
> $: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
> 	--socket-mem 1024,1024 -- -p 0x1 --stats 1
> $: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
> 	-v /dev/hugepages:/dev/hugepages \
> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> 	--vdev=eth_cvio0,path=/var/run/usvhost -- -p 0x1
> 
> f. Used with vhost-net
> $: modprobe vhost
> $: modprobe vhost-net
> $: docker run -i -t --privileged \
> 	-v /dev/vhost-net:/dev/vhost-net \
> 	-v /dev/net/tun:/dev/net/tun \
> 	-v /dev/hugepages:/dev/hugepages \
> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> 	--vdev=eth_cvio0,path=/dev/vhost-net -- -p 0x1
> 
> By the way, it's not necessary to run in a container.
> 
> Signed-off-by: Huawei Xie <huawei.xie at intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan at intel.com>
> 
> Jianfeng Tan (5):
>   mem: add --single-file to create single mem-backed file
>   mem: add API to obtain memory-backed file info
>   virtio/vdev: add embeded device emulation
>   virtio/vdev: add a new vdev named eth_cvio
>   docs: add release note for virtio for container
> 
>  config/common_linuxapp                     |   5 +
>  doc/guides/rel_notes/release_2_3.rst       |   4 +
>  drivers/net/virtio/Makefile                |   4 +
>  drivers/net/virtio/vhost.h                 | 194 +++++++
>  drivers/net/virtio/vhost_embedded.c        | 809 +++++++++++++++++++++++++++++
>  drivers/net/virtio/virtio_ethdev.c         | 329 +++++++++---
>  drivers/net/virtio/virtio_ethdev.h         |   6 +-
>  drivers/net/virtio/virtio_pci.h            |  15 +-
>  drivers/net/virtio/virtio_rxtx.c           |   6 +-
>  drivers/net/virtio/virtio_rxtx_simple.c    |  13 +-
>  drivers/net/virtio/virtqueue.h             |  15 +-
>  lib/librte_eal/common/eal_common_options.c |  17 +
>  lib/librte_eal/common/eal_internal_cfg.h   |   1 +
>  lib/librte_eal/common/eal_options.h        |   2 +
>  lib/librte_eal/common/include/rte_memory.h |  16 +
>  lib/librte_eal/linuxapp/eal/eal.c          |   4 +-
>  lib/librte_eal/linuxapp/eal/eal_memory.c   |  88 +++-
>  17 files changed, 1435 insertions(+), 93 deletions(-)
>  create mode 100644 drivers/net/virtio/vhost.h
>  create mode 100644 drivers/net/virtio/vhost_embedded.c
> 
> -- 
> 2.1.4
> 
So, first off, apologies for being so late to review this patch, its been on my
todo list forever, and I've just not gotten to it.

I've taken a cursory look at the code, and I can't find anything glaringly wrong
with it.

That said, I'm a bit confused about the overall purpose of this PMD.  I've read
the description several times now, and I _think_ I understand the purpose and
construction of the PMD. Please correct me if this is not the (admittedly very
generalized) overview:

1) You've created a vdev PMD that is generally named eth_cvio%n, which serves as
a virtual NIC suitable for use in a containerized space

2) The PMD in (1) establishes a connection to the host via the vhost backend
(which is either a socket or a character device), which it uses to forward data
from the containerized dpdk application

3) The system hosting the containerized dpdk application ties the other end of
the tun/tap interface established in (2) to some other forwarding mechanism
(ostensibly a host based dpdk forwarder) to send the frame out on the physical
wire.

If I understand that, it seems reasonable, but I have to ask why?  It feels a
bit like a re-invention of the wheel to me.  That is to say, for whatever
optimization this PMD may have, the by-far larger bottleneck is the tun/tap
interface in step (2).  If thats the case, then why create a new PMD at all? Why
not instead just use a tun/tap interface into the container, along with the
af_packet PMD for communication.  That has the ability to do memory mapping of
an interface for relatively fast packet writes, so I expect it will be just as
performant as this solution, and without the need to write and maintain a new
PMD's worth of code.

I feel like I'm missing something here, so please clarify if I am, but at the
moment, I'm having a hard time seeing the advantage to a new PMD here

Regards
Neil

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 0/5] virtio support for container
  2016-03-23 19:17   ` [PATCH v2 0/5] virtio support " Neil Horman
@ 2016-03-24  3:10     ` Tan, Jianfeng
  2016-03-24 13:45       ` Neil Horman
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-03-24  3:10 UTC (permalink / raw)
  To: Neil Horman; +Cc: dev

Hi Neil,

On 3/24/2016 3:17 AM, Neil Horman wrote:
> On Fri, Feb 05, 2016 at 07:20:23PM +0800, Jianfeng Tan wrote:
>> v1->v2:
>>   - Rebase on the patchset of virtio 1.0 support.
>>   - Fix cannot create non-hugepage memory.
>>   - Fix wrong size of memory region when "single-file" is used.
>>   - Fix setting of offset in virtqueue to use virtual address.
>>   - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
>>   - Add mac option to specify the mac address of this virtual device.
>>   - Update doc.
>>
>> This patchset is to provide high performance networking interface (virtio)
>> for container-based DPDK applications. The way of starting DPDK apps in
>> containers with ownership of NIC devices exclusively is beyond the scope.
>> The basic idea here is to present a new virtual device (named eth_cvio),
>> which can be discovered and initialized in container-based DPDK apps using
>> rte_eal_init(). To minimize the change, we reuse already-existing virtio
>> frontend driver code (driver/net/virtio/).
>>   
>> Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
>> operations into unix socket/cuse protocol, which is originally provided in
>> QEMU), is integrated in virtio frontend driver. So this converged driver
>> actually plays the role of original frontend driver and the role of QEMU
>> device framework.
>>   
>> The major difference lies in how to calculate relative address for vhost.
>> The principle of virtio is that: based on one or multiple shared memory
>> segments, vhost maintains a reference system with the base addresses and
>> length for each segment so that an address from VM comes (usually GPA,
>> Guest Physical Address) can be translated into vhost-recognizable address
>> (named VVA, Vhost Virtual Address). To decrease the overhead of address
>> translation, we should maintain as few segments as possible. In VM's case,
>> GPA is always locally continuous. In container's case, CVA (Container
>> Virtual Address) can be used. Specifically:
>> a. when set_base_addr, CVA address is used;
>> b. when preparing RX's descriptors, CVA address is used;
>> c. when transmitting packets, CVA is filled in TX's descriptors;
>> d. in TX and CQ's header, CVA is used.
>>   
>> How to share memory? In VM's case, qemu always shares all physical layout
>> to backend. But it's not feasible for a container, as a process, to share
>> all virtual memory regions to backend. So only specified virtual memory
>> regions (with type of shared) are sent to backend. It's a limitation that
>> only addresses in these areas can be used to transmit or receive packets.
>>
>> Known issues
>>
>> a. When used with vhost-net, root privilege is required to create tap
>> device inside.
>> b. Control queue and multi-queue are not supported yet.
>> c. When --single-file option is used, socket_id of the memory may be
>> wrong. (Use "numactl -N x -m x" to work around this for now)
>>   
>> How to use?
>>
>> a. Apply this patchset.
>>
>> b. To compile container apps:
>> $: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
>> $: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
>> $: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
>> $: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
>>
>> c. To build a docker image using Dockerfile below.
>> $: cat ./Dockerfile
>> FROM ubuntu:latest
>> WORKDIR /usr/src/dpdk
>> COPY . /usr/src/dpdk
>> ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
>> $: docker build -t dpdk-app-l2fwd .
>>
>> d. Used with vhost-user
>> $: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
>> 	--socket-mem 1024,1024 -- -p 0x1 --stats 1
>> $: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
>> 	-v /dev/hugepages:/dev/hugepages \
>> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
>> 	--vdev=eth_cvio0,path=/var/run/usvhost -- -p 0x1
>>
>> f. Used with vhost-net
>> $: modprobe vhost
>> $: modprobe vhost-net
>> $: docker run -i -t --privileged \
>> 	-v /dev/vhost-net:/dev/vhost-net \
>> 	-v /dev/net/tun:/dev/net/tun \
>> 	-v /dev/hugepages:/dev/hugepages \
>> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
>> 	--vdev=eth_cvio0,path=/dev/vhost-net -- -p 0x1
>>
>> By the way, it's not necessary to run in a container.
>>
>> Signed-off-by: Huawei Xie <huawei.xie at intel.com>
>> Signed-off-by: Jianfeng Tan <jianfeng.tan at intel.com>
>>
>> Jianfeng Tan (5):
>>    mem: add --single-file to create single mem-backed file
>>    mem: add API to obtain memory-backed file info
>>    virtio/vdev: add embeded device emulation
>>    virtio/vdev: add a new vdev named eth_cvio
>>    docs: add release note for virtio for container
>>
>>   config/common_linuxapp                     |   5 +
>>   doc/guides/rel_notes/release_2_3.rst       |   4 +
>>   drivers/net/virtio/Makefile                |   4 +
>>   drivers/net/virtio/vhost.h                 | 194 +++++++
>>   drivers/net/virtio/vhost_embedded.c        | 809 +++++++++++++++++++++++++++++
>>   drivers/net/virtio/virtio_ethdev.c         | 329 +++++++++---
>>   drivers/net/virtio/virtio_ethdev.h         |   6 +-
>>   drivers/net/virtio/virtio_pci.h            |  15 +-
>>   drivers/net/virtio/virtio_rxtx.c           |   6 +-
>>   drivers/net/virtio/virtio_rxtx_simple.c    |  13 +-
>>   drivers/net/virtio/virtqueue.h             |  15 +-
>>   lib/librte_eal/common/eal_common_options.c |  17 +
>>   lib/librte_eal/common/eal_internal_cfg.h   |   1 +
>>   lib/librte_eal/common/eal_options.h        |   2 +
>>   lib/librte_eal/common/include/rte_memory.h |  16 +
>>   lib/librte_eal/linuxapp/eal/eal.c          |   4 +-
>>   lib/librte_eal/linuxapp/eal/eal_memory.c   |  88 +++-
>>   17 files changed, 1435 insertions(+), 93 deletions(-)
>>   create mode 100644 drivers/net/virtio/vhost.h
>>   create mode 100644 drivers/net/virtio/vhost_embedded.c
>>
>> -- 
>> 2.1.4
>>
> So, first off, apologies for being so late to review this patch, its been on my
> todo list forever, and I've just not gotten to it.
>
> I've taken a cursory look at the code, and I can't find anything glaringly wrong
> with it.

Thanks very much for reviewing this series.

>
> That said, I'm a bit confused about the overall purpose of this PMD.  I've read
> the description several times now, and I _think_ I understand the purpose and
> construction of the PMD. Please correct me if this is not the (admittedly very
> generalized) overview:
>
> 1) You've created a vdev PMD that is generally named eth_cvio%n, which serves as
> a virtual NIC suitable for use in a containerized space
>
> 2) The PMD in (1) establishes a connection to the host via the vhost backend
> (which is either a socket or a character device), which it uses to forward data
> from the containerized dpdk application

The socket or the character device is used just for control plane 
messages to setting up the datapath. The data does not go through the 
socket or the character device.

>
> 3) The system hosting the containerized dpdk application ties the other end of
> the tun/tap interface established in (2) to some other forwarding mechanism
> (ostensibly a host based dpdk forwarder) to send the frame out on the physical
> wire.

There are two kinds of vhost backend:
(1) vhost-user, no need to leverage a tun/tap. the cvio PMD connects to 
the backend socket, and communicate memory region information with the 
vhost-user backend (the backend is another DPDK application using vhost 
PMD by Tetsuya, or using vhost library like vhost example).
(2) vhost-net, here we need a tun/tap. When we open the /dev/vhost-net 
char device, and some ioctl on it, it just starts a kthread (backend). 
We need an interface (tun/tap) as an agent to blend into kernel 
networking, so that the kthread knows where to send those packets (sent 
by frontend), and where to receive packets to send to frontend.

To be honest, vhost-user is the preferred way to achieve high 
performance. As far as vhost-net is concerned, it goes through a kernel 
network stack, which is the performance bottleneck.

>
> If I understand that, it seems reasonable, but I have to ask why?  It feels a
> bit like a re-invention of the wheel to me.  That is to say, for whatever
> optimization this PMD may have, the by-far larger bottleneck is the tun/tap
> interface in step (2).  If thats the case, then why create a new PMD at all? Why
> not instead just use a tun/tap interface into the container, along with the
> af_packet PMD for communication.

There's a gap here. We don't write/read pkts directly from tun/tap. The 
basic idea of virtio/vhost is a frontend/backend architecture on shared 
memory. The difference lies in:
Frontend and backend are working at the same time at different cores. 
Although AF_PACKRET can leverage memory mapping (shared memory between 
user space and kernel space as I understand), but there's a wake-up 
sequence. Use tx as an example, application puts some packets into 
shared memory, and then uses sendto(..., MSG_DONTWAIT, ...) to kick off 
transmit.

IMO, this is why we can achieve better performance than af_packet or pcap.

Thanks,
Jianfeng


> That has the ability to do memory mapping of
> an interface for relatively fast packet writes, so I expect it will be just as
> performant as this solution, and without the need to write and maintain a new
> PMD's worth of code.
>
> I feel like I'm missing something here, so please clarify if I am, but at the
> moment, I'm having a hard time seeing the advantage to a new PMD here
>
> Regards
> Neil
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 0/5] virtio support for container
  2016-03-24  3:10     ` Tan, Jianfeng
@ 2016-03-24 13:45       ` Neil Horman
  2016-03-25  1:25         ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Neil Horman @ 2016-03-24 13:45 UTC (permalink / raw)
  To: Tan, Jianfeng; +Cc: Neil Horman, dev

On Thu, Mar 24, 2016 at 11:10:50AM +0800, Tan, Jianfeng wrote:
> Hi Neil,
> 
> On 3/24/2016 3:17 AM, Neil Horman wrote:
> >On Fri, Feb 05, 2016 at 07:20:23PM +0800, Jianfeng Tan wrote:
> >>v1->v2:
> >>  - Rebase on the patchset of virtio 1.0 support.
> >>  - Fix cannot create non-hugepage memory.
> >>  - Fix wrong size of memory region when "single-file" is used.
> >>  - Fix setting of offset in virtqueue to use virtual address.
> >>  - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
> >>  - Add mac option to specify the mac address of this virtual device.
> >>  - Update doc.
> >>
> >>This patchset is to provide high performance networking interface (virtio)
> >>for container-based DPDK applications. The way of starting DPDK apps in
> >>containers with ownership of NIC devices exclusively is beyond the scope.
> >>The basic idea here is to present a new virtual device (named eth_cvio),
> >>which can be discovered and initialized in container-based DPDK apps using
> >>rte_eal_init(). To minimize the change, we reuse already-existing virtio
> >>frontend driver code (driver/net/virtio/).
> >>Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
> >>operations into unix socket/cuse protocol, which is originally provided in
> >>QEMU), is integrated in virtio frontend driver. So this converged driver
> >>actually plays the role of original frontend driver and the role of QEMU
> >>device framework.
> >>The major difference lies in how to calculate relative address for vhost.
> >>The principle of virtio is that: based on one or multiple shared memory
> >>segments, vhost maintains a reference system with the base addresses and
> >>length for each segment so that an address from VM comes (usually GPA,
> >>Guest Physical Address) can be translated into vhost-recognizable address
> >>(named VVA, Vhost Virtual Address). To decrease the overhead of address
> >>translation, we should maintain as few segments as possible. In VM's case,
> >>GPA is always locally continuous. In container's case, CVA (Container
> >>Virtual Address) can be used. Specifically:
> >>a. when set_base_addr, CVA address is used;
> >>b. when preparing RX's descriptors, CVA address is used;
> >>c. when transmitting packets, CVA is filled in TX's descriptors;
> >>d. in TX and CQ's header, CVA is used.
> >>How to share memory? In VM's case, qemu always shares all physical layout
> >>to backend. But it's not feasible for a container, as a process, to share
> >>all virtual memory regions to backend. So only specified virtual memory
> >>regions (with type of shared) are sent to backend. It's a limitation that
> >>only addresses in these areas can be used to transmit or receive packets.
> >>
> >>Known issues
> >>
> >>a. When used with vhost-net, root privilege is required to create tap
> >>device inside.
> >>b. Control queue and multi-queue are not supported yet.
> >>c. When --single-file option is used, socket_id of the memory may be
> >>wrong. (Use "numactl -N x -m x" to work around this for now)
> >>How to use?
> >>
> >>a. Apply this patchset.
> >>
> >>b. To compile container apps:
> >>$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> >>$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> >>$: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> >>$: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> >>
> >>c. To build a docker image using Dockerfile below.
> >>$: cat ./Dockerfile
> >>FROM ubuntu:latest
> >>WORKDIR /usr/src/dpdk
> >>COPY . /usr/src/dpdk
> >>ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
> >>$: docker build -t dpdk-app-l2fwd .
> >>
> >>d. Used with vhost-user
> >>$: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
> >>	--socket-mem 1024,1024 -- -p 0x1 --stats 1
> >>$: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
> >>	-v /dev/hugepages:/dev/hugepages \
> >>	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> >>	--vdev=eth_cvio0,path=/var/run/usvhost -- -p 0x1
> >>
> >>f. Used with vhost-net
> >>$: modprobe vhost
> >>$: modprobe vhost-net
> >>$: docker run -i -t --privileged \
> >>	-v /dev/vhost-net:/dev/vhost-net \
> >>	-v /dev/net/tun:/dev/net/tun \
> >>	-v /dev/hugepages:/dev/hugepages \
> >>	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> >>	--vdev=eth_cvio0,path=/dev/vhost-net -- -p 0x1
> >>
> >>By the way, it's not necessary to run in a container.
> >>
> >>Signed-off-by: Huawei Xie <huawei.xie at intel.com>
> >>Signed-off-by: Jianfeng Tan <jianfeng.tan at intel.com>
> >>
> >>Jianfeng Tan (5):
> >>   mem: add --single-file to create single mem-backed file
> >>   mem: add API to obtain memory-backed file info
> >>   virtio/vdev: add embeded device emulation
> >>   virtio/vdev: add a new vdev named eth_cvio
> >>   docs: add release note for virtio for container
> >>
> >>  config/common_linuxapp                     |   5 +
> >>  doc/guides/rel_notes/release_2_3.rst       |   4 +
> >>  drivers/net/virtio/Makefile                |   4 +
> >>  drivers/net/virtio/vhost.h                 | 194 +++++++
> >>  drivers/net/virtio/vhost_embedded.c        | 809 +++++++++++++++++++++++++++++
> >>  drivers/net/virtio/virtio_ethdev.c         | 329 +++++++++---
> >>  drivers/net/virtio/virtio_ethdev.h         |   6 +-
> >>  drivers/net/virtio/virtio_pci.h            |  15 +-
> >>  drivers/net/virtio/virtio_rxtx.c           |   6 +-
> >>  drivers/net/virtio/virtio_rxtx_simple.c    |  13 +-
> >>  drivers/net/virtio/virtqueue.h             |  15 +-
> >>  lib/librte_eal/common/eal_common_options.c |  17 +
> >>  lib/librte_eal/common/eal_internal_cfg.h   |   1 +
> >>  lib/librte_eal/common/eal_options.h        |   2 +
> >>  lib/librte_eal/common/include/rte_memory.h |  16 +
> >>  lib/librte_eal/linuxapp/eal/eal.c          |   4 +-
> >>  lib/librte_eal/linuxapp/eal/eal_memory.c   |  88 +++-
> >>  17 files changed, 1435 insertions(+), 93 deletions(-)
> >>  create mode 100644 drivers/net/virtio/vhost.h
> >>  create mode 100644 drivers/net/virtio/vhost_embedded.c
> >>
> >>-- 
> >>2.1.4
> >>
> >So, first off, apologies for being so late to review this patch, its been on my
> >todo list forever, and I've just not gotten to it.
> >
> >I've taken a cursory look at the code, and I can't find anything glaringly wrong
> >with it.
> 
> Thanks very much for reviewing this series.
> 
> >
> >That said, I'm a bit confused about the overall purpose of this PMD.  I've read
> >the description several times now, and I _think_ I understand the purpose and
> >construction of the PMD. Please correct me if this is not the (admittedly very
> >generalized) overview:
> >
> >1) You've created a vdev PMD that is generally named eth_cvio%n, which serves as
> >a virtual NIC suitable for use in a containerized space
> >
> >2) The PMD in (1) establishes a connection to the host via the vhost backend
> >(which is either a socket or a character device), which it uses to forward data
> >from the containerized dpdk application
> 
> The socket or the character device is used just for control plane messages
> to setting up the datapath. The data does not go through the socket or the
> character device.
> 
> >
> >3) The system hosting the containerized dpdk application ties the other end of
> >the tun/tap interface established in (2) to some other forwarding mechanism
> >(ostensibly a host based dpdk forwarder) to send the frame out on the physical
> >wire.
> 
> There are two kinds of vhost backend:
> (1) vhost-user, no need to leverage a tun/tap. the cvio PMD connects to the
> backend socket, and communicate memory region information with the
> vhost-user backend (the backend is another DPDK application using vhost PMD
> by Tetsuya, or using vhost library like vhost example).
> (2) vhost-net, here we need a tun/tap. When we open the /dev/vhost-net char
> device, and some ioctl on it, it just starts a kthread (backend). We need an
> interface (tun/tap) as an agent to blend into kernel networking, so that the
> kthread knows where to send those packets (sent by frontend), and where to
> receive packets to send to frontend.
> 
> To be honest, vhost-user is the preferred way to achieve high performance.
> As far as vhost-net is concerned, it goes through a kernel network stack,
> which is the performance bottleneck.
> 
Sure, that makes sense.  So in the vhost-user case, we just read/write to a
shared memory region?  I.e. no user/kernel space transition for the nominal data
path?  If thats the case, than thats the piece I'm missing
Neil

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 0/5] virtio support for container
  2016-03-24 13:45       ` Neil Horman
@ 2016-03-25  1:25         ` Tan, Jianfeng
  2016-03-25 11:06           ` Neil Horman
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-03-25  1:25 UTC (permalink / raw)
  To: Neil Horman; +Cc: Neil Horman, dev



On 3/24/2016 9:45 PM, Neil Horman wrote:
> On Thu, Mar 24, 2016 at 11:10:50AM +0800, Tan, Jianfeng wrote:
>> Hi Neil,
>>
>> On 3/24/2016 3:17 AM, Neil Horman wrote:
>>> On Fri, Feb 05, 2016 at 07:20:23PM +0800, Jianfeng Tan wrote:
>>>> v1->v2:
>>>>   - Rebase on the patchset of virtio 1.0 support.
>>>>   - Fix cannot create non-hugepage memory.
>>>>   - Fix wrong size of memory region when "single-file" is used.
>>>>   - Fix setting of offset in virtqueue to use virtual address.
>>>>   - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
>>>>   - Add mac option to specify the mac address of this virtual device.
>>>>   - Update doc.
>>>>
>>>> This patchset is to provide high performance networking interface (virtio)
>>>> for container-based DPDK applications. The way of starting DPDK apps in
>>>> containers with ownership of NIC devices exclusively is beyond the scope.
>>>> The basic idea here is to present a new virtual device (named eth_cvio),
>>>> which can be discovered and initialized in container-based DPDK apps using
>>>> rte_eal_init(). To minimize the change, we reuse already-existing virtio
>>>> frontend driver code (driver/net/virtio/).
>>>> Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
>>>> operations into unix socket/cuse protocol, which is originally provided in
>>>> QEMU), is integrated in virtio frontend driver. So this converged driver
>>>> actually plays the role of original frontend driver and the role of QEMU
>>>> device framework.
>>>> The major difference lies in how to calculate relative address for vhost.
>>>> The principle of virtio is that: based on one or multiple shared memory
>>>> segments, vhost maintains a reference system with the base addresses and
>>>> length for each segment so that an address from VM comes (usually GPA,
>>>> Guest Physical Address) can be translated into vhost-recognizable address
>>>> (named VVA, Vhost Virtual Address). To decrease the overhead of address
>>>> translation, we should maintain as few segments as possible. In VM's case,
>>>> GPA is always locally continuous. In container's case, CVA (Container
>>>> Virtual Address) can be used. Specifically:
>>>> a. when set_base_addr, CVA address is used;
>>>> b. when preparing RX's descriptors, CVA address is used;
>>>> c. when transmitting packets, CVA is filled in TX's descriptors;
>>>> d. in TX and CQ's header, CVA is used.
>>>> How to share memory? In VM's case, qemu always shares all physical layout
>>>> to backend. But it's not feasible for a container, as a process, to share
>>>> all virtual memory regions to backend. So only specified virtual memory
>>>> regions (with type of shared) are sent to backend. It's a limitation that
>>>> only addresses in these areas can be used to transmit or receive packets.
>>>>
>>>> Known issues
>>>>
>>>> a. When used with vhost-net, root privilege is required to create tap
>>>> device inside.
>>>> b. Control queue and multi-queue are not supported yet.
>>>> c. When --single-file option is used, socket_id of the memory may be
>>>> wrong. (Use "numactl -N x -m x" to work around this for now)
>>>> How to use?
>>>>
>>>> a. Apply this patchset.
>>>>
>>>> b. To compile container apps:
>>>> $: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
>>>> $: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
>>>> $: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
>>>> $: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
>>>>
>>>> c. To build a docker image using Dockerfile below.
>>>> $: cat ./Dockerfile
>>> >FROM ubuntu:latest
>>>> WORKDIR /usr/src/dpdk
>>>> COPY . /usr/src/dpdk
>>>> ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
>>>> $: docker build -t dpdk-app-l2fwd .
>>>>
>>>> d. Used with vhost-user
>>>> $: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
>>>> 	--socket-mem 1024,1024 -- -p 0x1 --stats 1
>>>> $: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
>>>> 	-v /dev/hugepages:/dev/hugepages \
>>>> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
>>>> 	--vdev=eth_cvio0,path=/var/run/usvhost -- -p 0x1
>>>>
>>>> f. Used with vhost-net
>>>> $: modprobe vhost
>>>> $: modprobe vhost-net
>>>> $: docker run -i -t --privileged \
>>>> 	-v /dev/vhost-net:/dev/vhost-net \
>>>> 	-v /dev/net/tun:/dev/net/tun \
>>>> 	-v /dev/hugepages:/dev/hugepages \
>>>> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
>>>> 	--vdev=eth_cvio0,path=/dev/vhost-net -- -p 0x1
>>>>
>>>> By the way, it's not necessary to run in a container.
>>>>
>>>> Signed-off-by: Huawei Xie <huawei.xie at intel.com>
>>>> Signed-off-by: Jianfeng Tan <jianfeng.tan at intel.com>
>>>>
>>>> Jianfeng Tan (5):
>>>>    mem: add --single-file to create single mem-backed file
>>>>    mem: add API to obtain memory-backed file info
>>>>    virtio/vdev: add embeded device emulation
>>>>    virtio/vdev: add a new vdev named eth_cvio
>>>>    docs: add release note for virtio for container
>>>>
>>>>   config/common_linuxapp                     |   5 +
>>>>   doc/guides/rel_notes/release_2_3.rst       |   4 +
>>>>   drivers/net/virtio/Makefile                |   4 +
>>>>   drivers/net/virtio/vhost.h                 | 194 +++++++
>>>>   drivers/net/virtio/vhost_embedded.c        | 809 +++++++++++++++++++++++++++++
>>>>   drivers/net/virtio/virtio_ethdev.c         | 329 +++++++++---
>>>>   drivers/net/virtio/virtio_ethdev.h         |   6 +-
>>>>   drivers/net/virtio/virtio_pci.h            |  15 +-
>>>>   drivers/net/virtio/virtio_rxtx.c           |   6 +-
>>>>   drivers/net/virtio/virtio_rxtx_simple.c    |  13 +-
>>>>   drivers/net/virtio/virtqueue.h             |  15 +-
>>>>   lib/librte_eal/common/eal_common_options.c |  17 +
>>>>   lib/librte_eal/common/eal_internal_cfg.h   |   1 +
>>>>   lib/librte_eal/common/eal_options.h        |   2 +
>>>>   lib/librte_eal/common/include/rte_memory.h |  16 +
>>>>   lib/librte_eal/linuxapp/eal/eal.c          |   4 +-
>>>>   lib/librte_eal/linuxapp/eal/eal_memory.c   |  88 +++-
>>>>   17 files changed, 1435 insertions(+), 93 deletions(-)
>>>>   create mode 100644 drivers/net/virtio/vhost.h
>>>>   create mode 100644 drivers/net/virtio/vhost_embedded.c
>>>>
>>>> -- 
>>>> 2.1.4
>>>>
>>> So, first off, apologies for being so late to review this patch, its been on my
>>> todo list forever, and I've just not gotten to it.
>>>
>>> I've taken a cursory look at the code, and I can't find anything glaringly wrong
>>> with it.
>> Thanks very much for reviewing this series.
>>
>>> That said, I'm a bit confused about the overall purpose of this PMD.  I've read
>>> the description several times now, and I _think_ I understand the purpose and
>>> construction of the PMD. Please correct me if this is not the (admittedly very
>>> generalized) overview:
>>>
>>> 1) You've created a vdev PMD that is generally named eth_cvio%n, which serves as
>>> a virtual NIC suitable for use in a containerized space
>>>
>>> 2) The PMD in (1) establishes a connection to the host via the vhost backend
>>> (which is either a socket or a character device), which it uses to forward data
>> >from the containerized dpdk application
>>
>> The socket or the character device is used just for control plane messages
>> to setting up the datapath. The data does not go through the socket or the
>> character device.
>>
>>> 3) The system hosting the containerized dpdk application ties the other end of
>>> the tun/tap interface established in (2) to some other forwarding mechanism
>>> (ostensibly a host based dpdk forwarder) to send the frame out on the physical
>>> wire.
>> There are two kinds of vhost backend:
>> (1) vhost-user, no need to leverage a tun/tap. the cvio PMD connects to the
>> backend socket, and communicate memory region information with the
>> vhost-user backend (the backend is another DPDK application using vhost PMD
>> by Tetsuya, or using vhost library like vhost example).
>> (2) vhost-net, here we need a tun/tap. When we open the /dev/vhost-net char
>> device, and some ioctl on it, it just starts a kthread (backend). We need an
>> interface (tun/tap) as an agent to blend into kernel networking, so that the
>> kthread knows where to send those packets (sent by frontend), and where to
>> receive packets to send to frontend.
>>
>> To be honest, vhost-user is the preferred way to achieve high performance.
>> As far as vhost-net is concerned, it goes through a kernel network stack,
>> which is the performance bottleneck.
>>
> Sure, that makes sense.  So in the vhost-user case, we just read/write to a
> shared memory region?  I.e. no user/kernel space transition for the nominal data
> path?  If thats the case, than thats the piece I'm missing
> Neil

Yes, exactly for now (both sides is in polling mode). Plus, we are 
trying to add interrupt mode so that large amount of containers can run 
with this new PMD. At interrupt mode, "user/kernel transition" would be 
smart because its the other side's responsibility to tell this side if 
the other side needs to be waken up, so user/kernel space transition 
happens only wakeup is necessary.

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 0/5] virtio support for container
  2016-03-25  1:25         ` Tan, Jianfeng
@ 2016-03-25 11:06           ` Neil Horman
  0 siblings, 0 replies; 196+ messages in thread
From: Neil Horman @ 2016-03-25 11:06 UTC (permalink / raw)
  To: Tan, Jianfeng; +Cc: Neil Horman, dev

On Fri, Mar 25, 2016 at 09:25:49AM +0800, Tan, Jianfeng wrote:
> 
> 
> On 3/24/2016 9:45 PM, Neil Horman wrote:
> >On Thu, Mar 24, 2016 at 11:10:50AM +0800, Tan, Jianfeng wrote:
> >>Hi Neil,
> >>
> >>On 3/24/2016 3:17 AM, Neil Horman wrote:
> >>>On Fri, Feb 05, 2016 at 07:20:23PM +0800, Jianfeng Tan wrote:
> >>>>v1->v2:
> >>>>  - Rebase on the patchset of virtio 1.0 support.
> >>>>  - Fix cannot create non-hugepage memory.
> >>>>  - Fix wrong size of memory region when "single-file" is used.
> >>>>  - Fix setting of offset in virtqueue to use virtual address.
> >>>>  - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
> >>>>  - Add mac option to specify the mac address of this virtual device.
> >>>>  - Update doc.
> >>>>
> >>>>This patchset is to provide high performance networking interface (virtio)
> >>>>for container-based DPDK applications. The way of starting DPDK apps in
> >>>>containers with ownership of NIC devices exclusively is beyond the scope.
> >>>>The basic idea here is to present a new virtual device (named eth_cvio),
> >>>>which can be discovered and initialized in container-based DPDK apps using
> >>>>rte_eal_init(). To minimize the change, we reuse already-existing virtio
> >>>>frontend driver code (driver/net/virtio/).
> >>>>Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
> >>>>operations into unix socket/cuse protocol, which is originally provided in
> >>>>QEMU), is integrated in virtio frontend driver. So this converged driver
> >>>>actually plays the role of original frontend driver and the role of QEMU
> >>>>device framework.
> >>>>The major difference lies in how to calculate relative address for vhost.
> >>>>The principle of virtio is that: based on one or multiple shared memory
> >>>>segments, vhost maintains a reference system with the base addresses and
> >>>>length for each segment so that an address from VM comes (usually GPA,
> >>>>Guest Physical Address) can be translated into vhost-recognizable address
> >>>>(named VVA, Vhost Virtual Address). To decrease the overhead of address
> >>>>translation, we should maintain as few segments as possible. In VM's case,
> >>>>GPA is always locally continuous. In container's case, CVA (Container
> >>>>Virtual Address) can be used. Specifically:
> >>>>a. when set_base_addr, CVA address is used;
> >>>>b. when preparing RX's descriptors, CVA address is used;
> >>>>c. when transmitting packets, CVA is filled in TX's descriptors;
> >>>>d. in TX and CQ's header, CVA is used.
> >>>>How to share memory? In VM's case, qemu always shares all physical layout
> >>>>to backend. But it's not feasible for a container, as a process, to share
> >>>>all virtual memory regions to backend. So only specified virtual memory
> >>>>regions (with type of shared) are sent to backend. It's a limitation that
> >>>>only addresses in these areas can be used to transmit or receive packets.
> >>>>
> >>>>Known issues
> >>>>
> >>>>a. When used with vhost-net, root privilege is required to create tap
> >>>>device inside.
> >>>>b. Control queue and multi-queue are not supported yet.
> >>>>c. When --single-file option is used, socket_id of the memory may be
> >>>>wrong. (Use "numactl -N x -m x" to work around this for now)
> >>>>How to use?
> >>>>
> >>>>a. Apply this patchset.
> >>>>
> >>>>b. To compile container apps:
> >>>>$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> >>>>$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> >>>>$: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> >>>>$: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> >>>>
> >>>>c. To build a docker image using Dockerfile below.
> >>>>$: cat ./Dockerfile
> >>>>FROM ubuntu:latest
> >>>>WORKDIR /usr/src/dpdk
> >>>>COPY . /usr/src/dpdk
> >>>>ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
> >>>>$: docker build -t dpdk-app-l2fwd .
> >>>>
> >>>>d. Used with vhost-user
> >>>>$: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
> >>>>	--socket-mem 1024,1024 -- -p 0x1 --stats 1
> >>>>$: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
> >>>>	-v /dev/hugepages:/dev/hugepages \
> >>>>	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> >>>>	--vdev=eth_cvio0,path=/var/run/usvhost -- -p 0x1
> >>>>
> >>>>f. Used with vhost-net
> >>>>$: modprobe vhost
> >>>>$: modprobe vhost-net
> >>>>$: docker run -i -t --privileged \
> >>>>	-v /dev/vhost-net:/dev/vhost-net \
> >>>>	-v /dev/net/tun:/dev/net/tun \
> >>>>	-v /dev/hugepages:/dev/hugepages \
> >>>>	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> >>>>	--vdev=eth_cvio0,path=/dev/vhost-net -- -p 0x1
> >>>>
> >>>>By the way, it's not necessary to run in a container.
> >>>>
> >>>>Signed-off-by: Huawei Xie <huawei.xie at intel.com>
> >>>>Signed-off-by: Jianfeng Tan <jianfeng.tan at intel.com>
> >>>>
> >>>>Jianfeng Tan (5):
> >>>>   mem: add --single-file to create single mem-backed file
> >>>>   mem: add API to obtain memory-backed file info
> >>>>   virtio/vdev: add embeded device emulation
> >>>>   virtio/vdev: add a new vdev named eth_cvio
> >>>>   docs: add release note for virtio for container
> >>>>
> >>>>  config/common_linuxapp                     |   5 +
> >>>>  doc/guides/rel_notes/release_2_3.rst       |   4 +
> >>>>  drivers/net/virtio/Makefile                |   4 +
> >>>>  drivers/net/virtio/vhost.h                 | 194 +++++++
> >>>>  drivers/net/virtio/vhost_embedded.c        | 809 +++++++++++++++++++++++++++++
> >>>>  drivers/net/virtio/virtio_ethdev.c         | 329 +++++++++---
> >>>>  drivers/net/virtio/virtio_ethdev.h         |   6 +-
> >>>>  drivers/net/virtio/virtio_pci.h            |  15 +-
> >>>>  drivers/net/virtio/virtio_rxtx.c           |   6 +-
> >>>>  drivers/net/virtio/virtio_rxtx_simple.c    |  13 +-
> >>>>  drivers/net/virtio/virtqueue.h             |  15 +-
> >>>>  lib/librte_eal/common/eal_common_options.c |  17 +
> >>>>  lib/librte_eal/common/eal_internal_cfg.h   |   1 +
> >>>>  lib/librte_eal/common/eal_options.h        |   2 +
> >>>>  lib/librte_eal/common/include/rte_memory.h |  16 +
> >>>>  lib/librte_eal/linuxapp/eal/eal.c          |   4 +-
> >>>>  lib/librte_eal/linuxapp/eal/eal_memory.c   |  88 +++-
> >>>>  17 files changed, 1435 insertions(+), 93 deletions(-)
> >>>>  create mode 100644 drivers/net/virtio/vhost.h
> >>>>  create mode 100644 drivers/net/virtio/vhost_embedded.c
> >>>>
> >>>>-- 
> >>>>2.1.4
> >>>>
> >>>So, first off, apologies for being so late to review this patch, its been on my
> >>>todo list forever, and I've just not gotten to it.
> >>>
> >>>I've taken a cursory look at the code, and I can't find anything glaringly wrong
> >>>with it.
> >>Thanks very much for reviewing this series.
> >>
> >>>That said, I'm a bit confused about the overall purpose of this PMD.  I've read
> >>>the description several times now, and I _think_ I understand the purpose and
> >>>construction of the PMD. Please correct me if this is not the (admittedly very
> >>>generalized) overview:
> >>>
> >>>1) You've created a vdev PMD that is generally named eth_cvio%n, which serves as
> >>>a virtual NIC suitable for use in a containerized space
> >>>
> >>>2) The PMD in (1) establishes a connection to the host via the vhost backend
> >>>(which is either a socket or a character device), which it uses to forward data
> >>>from the containerized dpdk application
> >>
> >>The socket or the character device is used just for control plane messages
> >>to setting up the datapath. The data does not go through the socket or the
> >>character device.
> >>
> >>>3) The system hosting the containerized dpdk application ties the other end of
> >>>the tun/tap interface established in (2) to some other forwarding mechanism
> >>>(ostensibly a host based dpdk forwarder) to send the frame out on the physical
> >>>wire.
> >>There are two kinds of vhost backend:
> >>(1) vhost-user, no need to leverage a tun/tap. the cvio PMD connects to the
> >>backend socket, and communicate memory region information with the
> >>vhost-user backend (the backend is another DPDK application using vhost PMD
> >>by Tetsuya, or using vhost library like vhost example).
> >>(2) vhost-net, here we need a tun/tap. When we open the /dev/vhost-net char
> >>device, and some ioctl on it, it just starts a kthread (backend). We need an
> >>interface (tun/tap) as an agent to blend into kernel networking, so that the
> >>kthread knows where to send those packets (sent by frontend), and where to
> >>receive packets to send to frontend.
> >>
> >>To be honest, vhost-user is the preferred way to achieve high performance.
> >>As far as vhost-net is concerned, it goes through a kernel network stack,
> >>which is the performance bottleneck.
> >>
> >Sure, that makes sense.  So in the vhost-user case, we just read/write to a
> >shared memory region?  I.e. no user/kernel space transition for the nominal data
> >path?  If thats the case, than thats the piece I'm missing
> >Neil
> 
> Yes, exactly for now (both sides is in polling mode). Plus, we are trying to
> add interrupt mode so that large amount of containers can run with this new
> PMD. At interrupt mode, "user/kernel transition" would be smart because its
> the other side's responsibility to tell this side if the other side needs to
> be waken up, so user/kernel space transition happens only wakeup is
> necessary.
> 
> Thanks,
> Jianfeng
> 

Ok, thank you for the clarification

Acked-By: Neil Horman <nhorman@tuxdrver.com>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 0/5] virtio support for container
  2016-02-05 11:20 ` [PATCH v2 0/5] " Jianfeng Tan
                     ` (5 preceding siblings ...)
  2016-03-23 19:17   ` [PATCH v2 0/5] virtio support " Neil Horman
@ 2016-04-13 16:14   ` Thomas Monjalon
  2016-04-14  6:08     ` Tan, Jianfeng
  6 siblings, 1 reply; 196+ messages in thread
From: Thomas Monjalon @ 2016-04-13 16:14 UTC (permalink / raw)
  To: Jianfeng Tan; +Cc: dev, nakajima.yoshihiro, mst, ann.zhuangyanying

Hi Jianfeng,

Thanks for raising the container issues and proposing some solutions.
General comments below.

2016-02-05 19:20, Jianfeng Tan:
> This patchset is to provide high performance networking interface (virtio)
> for container-based DPDK applications. The way of starting DPDK apps in
> containers with ownership of NIC devices exclusively is beyond the scope.
> The basic idea here is to present a new virtual device (named eth_cvio),
> which can be discovered and initialized in container-based DPDK apps using
> rte_eal_init(). To minimize the change, we reuse already-existing virtio
> frontend driver code (driver/net/virtio/).
>  
> Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
> operations into unix socket/cuse protocol, which is originally provided in
> QEMU), is integrated in virtio frontend driver. So this converged driver
> actually plays the role of original frontend driver and the role of QEMU
> device framework.
>  
> The major difference lies in how to calculate relative address for vhost.
> The principle of virtio is that: based on one or multiple shared memory
> segments, vhost maintains a reference system with the base addresses and
> length for each segment so that an address from VM comes (usually GPA,
> Guest Physical Address) can be translated into vhost-recognizable address
> (named VVA, Vhost Virtual Address). To decrease the overhead of address
> translation, we should maintain as few segments as possible. In VM's case,
> GPA is always locally continuous. In container's case, CVA (Container
> Virtual Address) can be used. Specifically:
> a. when set_base_addr, CVA address is used;
> b. when preparing RX's descriptors, CVA address is used;
> c. when transmitting packets, CVA is filled in TX's descriptors;
> d. in TX and CQ's header, CVA is used.
>  
> How to share memory? In VM's case, qemu always shares all physical layout
> to backend. But it's not feasible for a container, as a process, to share
> all virtual memory regions to backend. So only specified virtual memory
> regions (with type of shared) are sent to backend. It's a limitation that
> only addresses in these areas can be used to transmit or receive packets.
> 
> Known issues
> 
> a. When used with vhost-net, root privilege is required to create tap
> device inside.
> b. Control queue and multi-queue are not supported yet.
> c. When --single-file option is used, socket_id of the memory may be
> wrong. (Use "numactl -N x -m x" to work around this for now)

There are 2 different topics in this patchset:
1/ How to provide networking in containers
2/ How to provide memory in containers

1/ You have decided to use the virtio spec to bridge the host
with its containers. But there is no virtio device in a container
and no vhost interface in the host (except the kernel one).
So you are extending virtio to work as a vdev inside the container.
Could you explain what is the datapath between virtio and the host app?
Does it need to use a fake device from Qemu as Tetsuya has done?

Do you think there can be some alternatives to vhost/virtio in containers?

2/ The memory management is already a mess and it's going worst.
I think we need to think the requirements first and then write a proper
implementation to cover every identified needs.
I have started a new thread to cover this part:
	http://thread.gmane.org/gmane.comp.networking.dpdk.devel/37445

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v2 0/5] virtio support for container
  2016-04-13 16:14   ` Thomas Monjalon
@ 2016-04-14  6:08     ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-04-14  6:08 UTC (permalink / raw)
  To: Thomas Monjalon; +Cc: dev, nakajima.yoshihiro, mst, ann.zhuangyanying

Hi Thomas,

On 4/14/2016 12:14 AM, Thomas Monjalon wrote:
> Hi Jianfeng,
>
> Thanks for raising the container issues and proposing some solutions.
> General comments below.
>
> 2016-02-05 19:20, Jianfeng Tan:
>> This patchset is to provide high performance networking interface (virtio)
>> for container-based DPDK applications. The way of starting DPDK apps in
>> containers with ownership of NIC devices exclusively is beyond the scope.
>> The basic idea here is to present a new virtual device (named eth_cvio),
>> which can be discovered and initialized in container-based DPDK apps using
>> rte_eal_init(). To minimize the change, we reuse already-existing virtio
>> frontend driver code (driver/net/virtio/).
>>   
>> Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
>> operations into unix socket/cuse protocol, which is originally provided in
>> QEMU), is integrated in virtio frontend driver. So this converged driver
>> actually plays the role of original frontend driver and the role of QEMU
>> device framework.
>>   
>> The major difference lies in how to calculate relative address for vhost.
>> The principle of virtio is that: based on one or multiple shared memory
>> segments, vhost maintains a reference system with the base addresses and
>> length for each segment so that an address from VM comes (usually GPA,
>> Guest Physical Address) can be translated into vhost-recognizable address
>> (named VVA, Vhost Virtual Address). To decrease the overhead of address
>> translation, we should maintain as few segments as possible. In VM's case,
>> GPA is always locally continuous. In container's case, CVA (Container
>> Virtual Address) can be used. Specifically:
>> a. when set_base_addr, CVA address is used;
>> b. when preparing RX's descriptors, CVA address is used;
>> c. when transmitting packets, CVA is filled in TX's descriptors;
>> d. in TX and CQ's header, CVA is used.
>>   
>> How to share memory? In VM's case, qemu always shares all physical layout
>> to backend. But it's not feasible for a container, as a process, to share
>> all virtual memory regions to backend. So only specified virtual memory
>> regions (with type of shared) are sent to backend. It's a limitation that
>> only addresses in these areas can be used to transmit or receive packets.
>>
>> Known issues
>>
>> a. When used with vhost-net, root privilege is required to create tap
>> device inside.
>> b. Control queue and multi-queue are not supported yet.
>> c. When --single-file option is used, socket_id of the memory may be
>> wrong. (Use "numactl -N x -m x" to work around this for now)
> There are 2 different topics in this patchset:
> 1/ How to provide networking in containers
> 2/ How to provide memory in containers
>
> 1/ You have decided to use the virtio spec to bridge the host
> with its containers. But there is no virtio device in a container
> and no vhost interface in the host (except the kernel one).
> So you are extending virtio to work as a vdev inside the container.
> Could you explain what is the datapath between virtio and the host app?

The datapath is based on the shared memory, which is determined using 
vhost-user protocol through a unix socket. So the key condition in this 
approach is to map the unix socket into container.


> Does it need to use a fake device from Qemu as Tetsuya has done?

In this implementation, we don't need a fake device from Qemu as Tetsuya 
is doing. We just maintain a virtual virtio device in DPDK EAL layer, 
and talk to vhost via unix socket. OK, I think it's necessary to point 
out the implementation difference between the two implementation: this 
approach gets involved existing virtio PMD at the layer of struct 
virtio_pci_ops, but Tetsuya's solution intercepts r/w toioport or pci 
configuration space.


>
> Do you think there can be some alternatives to vhost/virtio in containers?

Yeah, we were considering another way to create virtual virtio in kernel 
space, which is driven by a new kernel module (instead of virtio-net) 
and a new library (maybe in DPDK). Then control path goes from app -> 
library -> kernel -> vhost user (or vhost-net), and data path is still 
based on the negotiated shared memory and some vring structures inside 
the memory. However, this involves another new kernel module, I don't 
think it's a easy way to go.


>
> 2/ The memory management is already a mess and it's going worst.
> I think we need to think the requirements first and then write a proper
> implementation to cover every identified needs.
> I have started a new thread to cover this part:
> 	http://thread.gmane.org/gmane.comp.networking.dpdk.devel/37445

I agree we should isolate the memory problem from network interface 
problem. And the memory problem is not a blocker issue for this patch, 
we can go without changing the memory part, however, it makes it hard to 
use. We'll go to the thread to discuss this more.

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v3 0/2] virtio support for container
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
                   ` (7 preceding siblings ...)
  2016-02-05 11:20 ` [PATCH v2 0/5] " Jianfeng Tan
@ 2016-04-21  2:56 ` Jianfeng Tan
  2016-04-21  2:56   ` [PATCH v3 1/2] virtio/vdev: add embeded device emulation Jianfeng Tan
  2016-04-21  2:56   ` [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
  2016-04-29  1:18 ` [PATCH v4 0/8] virtio support for container Jianfeng Tan
                   ` (5 subsequent siblings)
  14 siblings, 2 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-04-21  2:56 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, michael.qiu, ann.zhuangyanying,
	mukawa, nhorman

v3:
 - Remove --single-file option; do no change at EAL memory.
 - Remove the added API rte_eal_get_backfile_info(), instead we check all
   opened files with HUGEFILE_FMT to find hugepage files owned by DPDK.
 - Accordingly, add more restrictions at "Known issue" section.
 - Rename parameter from queue_num to queue_size for confusion.
 - Rename vhost_embedded.c to rte_eth_virtio_vdev.c.
 - Move code related to the newly added vdev to rte_eth_virtio_vdev.c, to
   reuse eth_virtio_dev_init(), remove its static declaration.
 - Implement dev_uninit() for rte_eth_dev_detach().
 - WARN -> ERR, in vhost_embedded.c
 - Add more commit message for clarify the model.

v2:
 - Rebase on the patchset of virtio 1.0 support.
 - Fix cannot create non-hugepage memory.
 - Fix wrong size of memory region when "single-file" is used.
 - Fix setting of offset in virtqueue to use virtual address.
 - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
 - Add mac option to specify the mac address of this virtual device.
 - Update doc.

This patchset is to provide high performance networking interface (virtio)
for container-based DPDK applications. The way of starting DPDK apps in
containers with ownership of NIC devices exclusively is beyond the scope.
The basic idea here is to present a new virtual device (named eth_cvio),
which can be discovered and initialized in container-based DPDK apps using
rte_eal_init(). To minimize the change, we reuse already-existing virtio
frontend driver code (driver/net/virtio/).
 
Compared to QEMU/VM case, virtio device framework (translates I/O port r/w
operations into unix socket/cuse protocol, which is originally provided in
QEMU), is integrated in virtio frontend driver. So this converged driver
actually plays the role of original frontend driver and the role of QEMU
device framework.
 
The major difference lies in how to calculate relative address for vhost.
The principle of virtio is that: based on one or multiple shared memory
segments, vhost maintains a reference system with the base addresses and
length for each segment so that an address from VM comes (usually GPA,
Guest Physical Address) can be translated into vhost-recognizable address
(named VVA, Vhost Virtual Address). To decrease the overhead of address
translation, we should maintain as few segments as possible. In VM's case,
GPA is always locally continuous. In container's case, CVA (Container
Virtual Address) can be used. Specifically:
a. when set_base_addr, CVA address is used;
b. when preparing RX's descriptors, CVA address is used;
c. when transmitting packets, CVA is filled in TX's descriptors;
d. in TX and CQ's header, CVA is used.
 
How to share memory? In VM's case, qemu always shares all physical layout
to backend. But it's not feasible for a container, as a process, to share
all virtual memory regions to backend. So only specified virtual memory
regions (with type of shared) are sent to backend. It's a limitation that
only addresses in these areas can be used to transmit or receive packets.

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").

How to use?

a. Apply this patchset.

b. To compile container apps:
$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc

c. To build a docker image using Dockerfile below.
$: cat ./Dockerfile
FROM ubuntu:latest
WORKDIR /usr/src/dpdk
COPY . /usr/src/dpdk
ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
$: docker build -t dpdk-app-l2fwd .

d. Used with vhost-user
$: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
	--socket-mem 1024,1024 -- -p 0x1 --stats 1
$: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=eth_cvio0,path=/var/run/usvhost -- -p 0x1

f. Used with vhost-net
$: modprobe vhost
$: modprobe vhost-net
$: docker run -i -t --privileged \
	-v /dev/vhost-net:/dev/vhost-net \
	-v /dev/net/tun:/dev/net/tun \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=eth_cvio0,path=/dev/vhost-net -- -p 0x1

By the way, it's not necessary to run in a container.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-By: Neil Horman <nhorman@tuxdrver.com>

Jianfeng Tan (2):
  virtio/vdev: add embeded device emulation
  virtio/vdev: add a new vdev named eth_cvio

 config/common_linuxapp                   |    3 +
 doc/guides/nics/overview.rst             |   58 +-
 doc/guides/rel_notes/release_16_07.rst   |    4 +
 drivers/net/virtio/Makefile              |    4 +
 drivers/net/virtio/rte_eth_virtio_vdev.c | 1079 ++++++++++++++++++++++++++++++
 drivers/net/virtio/vhost.h               |  194 ++++++
 drivers/net/virtio/virtio_ethdev.c       |  134 ++--
 drivers/net/virtio/virtio_ethdev.h       |    2 +
 drivers/net/virtio/virtio_pci.h          |   14 +
 drivers/net/virtio/virtio_rxtx.c         |    5 +-
 drivers/net/virtio/virtio_rxtx_simple.c  |   13 +-
 drivers/net/virtio/virtqueue.h           |   10 +
 12 files changed, 1433 insertions(+), 87 deletions(-)
 create mode 100644 drivers/net/virtio/rte_eth_virtio_vdev.c
 create mode 100644 drivers/net/virtio/vhost.h

-- 
2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v3 1/2] virtio/vdev: add embeded device emulation
  2016-04-21  2:56 ` [PATCH v3 0/2] " Jianfeng Tan
@ 2016-04-21  2:56   ` Jianfeng Tan
  2016-04-21 22:01     ` Yuanhan Liu
  2016-04-21  2:56   ` [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
  1 sibling, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-04-21  2:56 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, michael.qiu, ann.zhuangyanying,
	mukawa, nhorman

Background: Previously, we usually use a virtio device in QEMU/VM's
context as below pic shows. Virtio nic is emulated in QEMU, and usually
presented in VM as a PCI device.

|-----------|
|     vm    |
|-----------| (over PCI bus or MMIO or Channel I/O)
|    QEMU   | -> device emulation
|-----------|
      |
      | (vhost-user protocol or vhost-net ioctls)
      |
|-----------|
|   vhost   |
|-----------|

Then we come to the topic that how to present a virtio device in an app
or container, which uses virtio device to do inter process communication
with vhost backend process. To achieve that, first of all, we need way
in DPDK to interract with vhost backend. And then emulate a virtual
virtio device in DPDK (which is addressed in following patch).

|-----------|
|  DPDK app |
|-----------|
|  DPDK lib | -> device emulation (addressed by following patch)
|-----------|
      |
      | (vhost-user protocol or vhost-net ioctls), addressed by this patch
      |
|-----------|
|   vhost   |
|-----------|

How: we implement another instance of struct virtio_pci_ops to intercept
the communications between VM and QEMU. Instead of rd/wr ioport or PCI
configuration space, here we directly talk with backend through the vhost
file. Depending on the type of vhost file,
   - vhost-user is used if the given path points to a unix socket;
   - vhost-net is used if the given path points to a char device.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-By: Neil Horman <nhorman@tuxdrver.com>
---
 config/common_linuxapp                   |   3 +
 drivers/net/virtio/Makefile              |   4 +
 drivers/net/virtio/rte_eth_virtio_vdev.c | 897 +++++++++++++++++++++++++++++++
 drivers/net/virtio/vhost.h               | 194 +++++++
 drivers/net/virtio/virtio_ethdev.h       |   1 -
 drivers/net/virtio/virtio_pci.h          |  14 +
 6 files changed, 1112 insertions(+), 1 deletion(-)
 create mode 100644 drivers/net/virtio/rte_eth_virtio_vdev.c
 create mode 100644 drivers/net/virtio/vhost.h

diff --git a/config/common_linuxapp b/config/common_linuxapp
index 7e698e2..e69f71c 100644
--- a/config/common_linuxapp
+++ b/config/common_linuxapp
@@ -43,3 +43,6 @@ CONFIG_RTE_LIBRTE_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_AF_PACKET=y
 CONFIG_RTE_LIBRTE_POWER=y
+
+# Enable virtio support for containers
+CONFIG_RTE_VIRTIO_VDEV=y
diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index ef84f60..4b5ba32 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -55,6 +55,10 @@ ifeq ($(findstring RTE_MACHINE_CPUFLAG_SSSE3,$(CFLAGS)),RTE_MACHINE_CPUFLAG_SSSE
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx_simple.c
 endif
 
+ifeq ($(CONFIG_RTE_VIRTIO_VDEV),y)
+	SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += rte_eth_virtio_vdev.c
+endif
+
 # this lib depends upon:
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_eal lib/librte_ether
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_mempool lib/librte_mbuf
diff --git a/drivers/net/virtio/rte_eth_virtio_vdev.c b/drivers/net/virtio/rte_eth_virtio_vdev.c
new file mode 100644
index 0000000..419acef
--- /dev/null
+++ b/drivers/net/virtio/rte_eth_virtio_vdev.c
@@ -0,0 +1,897 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <sys/types.h>
+#include <sys/socket.h>
+#include <sys/un.h>
+#include <stdio.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <string.h>
+#include <errno.h>
+#include <assert.h>
+#include <sys/mman.h>
+#include <unistd.h>
+#include <sys/eventfd.h>
+#include <sys/ioctl.h>
+#include <net/if.h>
+
+#include <rte_mbuf.h>
+#include <rte_memory.h>
+#include <rte_eal_memconfig.h>
+
+#include "virtio_pci.h"
+#include "virtio_logs.h"
+#include "virtio_ethdev.h"
+#include "virtqueue.h"
+#include "vhost.h"
+
+static int
+vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
+{
+	int r;
+	struct msghdr msgh;
+	struct iovec iov;
+	size_t fd_size = fd_num * sizeof(int);
+	char control[CMSG_SPACE(fd_size)];
+	struct cmsghdr *cmsg;
+
+	bzero(&msgh, sizeof(msgh));
+	bzero(control, sizeof(control));
+
+	iov.iov_base = (uint8_t *)buf;
+	iov.iov_len = len;
+
+	msgh.msg_iov = &iov;
+	msgh.msg_iovlen = 1;
+	msgh.msg_control = control;
+	msgh.msg_controllen = sizeof(control);
+
+	cmsg = CMSG_FIRSTHDR(&msgh);
+	cmsg->cmsg_len = CMSG_LEN(fd_size);
+	cmsg->cmsg_level = SOL_SOCKET;
+	cmsg->cmsg_type = SCM_RIGHTS;
+	memcpy(CMSG_DATA(cmsg), fds, fd_size);
+
+	do {
+		r = sendmsg(fd, &msgh, 0);
+	} while (r < 0 && errno == EINTR);
+
+	return r;
+}
+
+static int
+vhost_user_read(int fd, struct vhost_user_msg *msg)
+{
+	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
+	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
+
+	ret = recv(fd, (void *)msg, sz_hdr, 0);
+	if (ret < sz_hdr) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
+			    ret, sz_hdr);
+		goto fail;
+	}
+
+	/* validate msg flags */
+	if (msg->flags != (valid_flags)) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg: flags %x instead of %x.",
+			    msg->flags, valid_flags);
+		goto fail;
+	}
+
+	sz_payload = msg->size;
+	if (sz_payload) {
+		ret = recv(fd, (void *)((char *)msg + sz_hdr), sz_payload, 0);
+		if (ret < sz_payload) {
+			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
+				    ret, msg->size);
+			goto fail;
+		}
+	}
+
+	return 0;
+
+fail:
+	return -1;
+}
+
+struct hugepage_file_info {
+	uint64_t addr;            /**< virtual addr */
+	size_t   size;            /**< the file size */
+	char     path[PATH_MAX];  /**< path to backing file */
+};
+
+/* Two possible solutions:
+ * 1. Match HUGEPAGE_INFO_FMT to find the file storing struct hugepage_file
+ * array. This is simple but cannot be used in secondary process because
+ * secondary process will close and munmap that file.
+ * 2. Match HUGEFILE_FMT to find hugepage files directly.
+ *
+ * We choose option 2.
+ */
+static int
+get_hugepage_file_info(struct hugepage_file_info huges[], int max)
+{
+	int idx;
+	FILE *f;
+	char buf[BUFSIZ], *tmp, *tail;
+	char *str_underline, *str_start;
+	int huge_index;
+	uint64_t v_start, v_end;
+
+	f = fopen("/proc/self/maps", "r");
+	if (!f) {
+		PMD_DRV_LOG(ERR, "cannot open /proc/self/maps");
+		return -1;
+	}
+
+	idx = 0;
+	while (fgets(buf, sizeof(buf), f) != NULL) {
+		sscanf(buf, "%" PRIx64 "-%" PRIx64, &v_start, &v_end);
+
+		tmp = strchr(buf, ' ') + 1; /** skip address */
+		tmp = strchr(tmp, ' ') + 1; /** skip perm */
+		tmp = strchr(tmp, ' ') + 1; /** skip offset */
+		tmp = strchr(tmp, ' ') + 1; /** skip dev */
+		tmp = strchr(tmp, ' ') + 1; /** skip inode */
+		while (*tmp == ' ')         /** skip spaces */
+			tmp++;
+		tail = strrchr(tmp, '\n');  /** remove newline if exists */
+		if (tail)
+			*tail = '\0';
+
+		/* Match HUGEFILE_FMT, aka "%s/%smap_%d",
+		 * which is defined in eal_filesystem.h
+		 */
+		str_underline = strrchr(tmp, '_');
+		if (!str_underline)
+			continue;
+
+		str_start = str_underline - strlen("map");
+		if (str_start < tmp)
+			continue;
+
+		if (sscanf(str_start, "map_%d", &huge_index) != 1)
+			continue;
+
+		if (idx >= max) {
+			PMD_DRV_LOG(ERR, "Exceed maximum of %d", max);
+			goto error;
+		}
+		huges[idx].addr = v_start;
+		huges[idx].size = v_end - v_start;
+		strcpy(huges[idx].path, tmp);
+		idx++;
+	}
+
+	fclose(f);
+	return idx;
+
+error:
+	fclose(f);
+	return -1;
+}
+
+static struct vhost_user_msg m __rte_unused;
+
+static void
+prepare_vhost_memory_user(struct vhost_user_msg *msg, int fds[])
+{
+	int i, num;
+	struct hugepage_file_info huges[VHOST_MEMORY_MAX_NREGIONS];
+	struct vhost_memory_region *mr;
+
+	num = get_hugepage_file_info(huges, VHOST_MEMORY_MAX_NREGIONS);
+	if (num < 0)
+		rte_panic("Failed to prepare memory for vhost-user\n");
+
+	for (i = 0; i < num; ++i) {
+		mr = &msg->payload.memory.regions[i];
+		mr->guest_phys_addr = huges[i].addr; /* use vaddr! */
+		mr->userspace_addr = huges[i].addr;
+		mr->memory_size = huges[i].size;
+		mr->mmap_offset = 0;
+		fds[i] = open(huges[i].path, O_RDWR);
+	}
+
+	msg->payload.memory.nregions = num;
+	msg->payload.memory.padding = 0;
+}
+
+static int
+vhost_user_sock(struct virtio_hw *hw, unsigned long int req, void *arg)
+{
+	struct vhost_user_msg msg;
+	struct vhost_vring_file *file = 0;
+	int need_reply = 0;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+	int fd_num = 0;
+	int i, len;
+
+	msg.request = req;
+	msg.flags = VHOST_USER_VERSION;
+	msg.size = 0;
+
+	switch (req) {
+	case VHOST_USER_GET_FEATURES:
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_FEATURES:
+	case VHOST_USER_SET_LOG_BASE:
+		msg.payload.u64 = *((__u64 *)arg);
+		msg.size = sizeof(m.payload.u64);
+		break;
+
+	case VHOST_USER_SET_OWNER:
+	case VHOST_USER_RESET_OWNER:
+		break;
+
+	case VHOST_USER_SET_MEM_TABLE:
+		prepare_vhost_memory_user(&msg, fds);
+		fd_num = msg.payload.memory.nregions;
+		msg.size = sizeof(m.payload.memory.nregions);
+		msg.size += sizeof(m.payload.memory.padding);
+		msg.size += fd_num * sizeof(struct vhost_memory_region);
+		break;
+
+	case VHOST_USER_SET_LOG_FD:
+		fds[fd_num++] = *((int *)arg);
+		break;
+
+	case VHOST_USER_SET_VRING_NUM:
+	case VHOST_USER_SET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		break;
+
+	case VHOST_USER_GET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_VRING_ADDR:
+		memcpy(&msg.payload.addr, arg, sizeof(msg.payload.addr));
+		msg.size = sizeof(m.payload.addr);
+		break;
+
+	case VHOST_USER_SET_VRING_KICK:
+	case VHOST_USER_SET_VRING_CALL:
+	case VHOST_USER_SET_VRING_ERR:
+		file = arg;
+		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
+		msg.size = sizeof(m.payload.u64);
+		if (file->fd > 0)
+			fds[fd_num++] = file->fd;
+		else
+			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
+		break;
+
+	default:
+		PMD_DRV_LOG(ERR, "trying to send unhandled msg type");
+		return -1;
+	}
+
+	len = VHOST_USER_HDR_SIZE + msg.size;
+	if (vhost_user_write(hw->vhostfd, &msg, len, fds, fd_num) < 0)
+		return 0;
+
+	if (req == VHOST_USER_SET_MEM_TABLE)
+		for (i = 0; i < fd_num; ++i)
+			close(fds[i]);
+
+	if (need_reply) {
+		if (vhost_user_read(hw->vhostfd, &msg) < 0)
+			return -1;
+
+		if (req != msg.request) {
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+
+		switch (req) {
+		case VHOST_USER_GET_FEATURES:
+			if (msg.size != sizeof(m.payload.u64)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			*((__u64 *)arg) = msg.payload.u64;
+			break;
+		case VHOST_USER_GET_VRING_BASE:
+			if (msg.size != sizeof(m.payload.state)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			memcpy(arg, &msg.payload.state,
+			       sizeof(struct vhost_vring_state));
+			break;
+		default:
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+	}
+
+	return 0;
+}
+
+static int
+vhost_kernel_ioctl(struct virtio_hw *hw, unsigned long int req, void *arg)
+{
+	return ioctl(hw->vhostfd, req, arg);
+}
+
+enum {
+	VHOST_MSG_SET_OWNER,
+	VHOST_MSG_SET_FEATURES,
+	VHOST_MSG_GET_FEATURES,
+	VHOST_MSG_SET_VRING_CALL,
+	VHOST_MSG_SET_VRING_NUM,
+	VHOST_MSG_SET_VRING_BASE,
+	VHOST_MSG_GET_VRING_BASE,
+	VHOST_MSG_SET_VRING_ADDR,
+	VHOST_MSG_SET_VRING_KICK,
+	VHOST_MSG_SET_MEM_TABLE,
+	VHOST_MSG_MAX,
+};
+
+static const char * const vhost_msg_strings[] = {
+	[VHOST_MSG_SET_OWNER] = "VHOST_MSG_SET_OWNER",
+	[VHOST_MSG_SET_FEATURES] = "VHOST_MSG_SET_FEATURES",
+	[VHOST_MSG_GET_FEATURES] = "VHOST_MSG_GET_FEATURES",
+	[VHOST_MSG_SET_VRING_CALL] = "VHOST_MSG_SET_VRING_CALL",
+	[VHOST_MSG_SET_VRING_NUM] = "VHOST_MSG_SET_VRING_NUM",
+	[VHOST_MSG_SET_VRING_BASE] = "VHOST_MSG_SET_VRING_BASE",
+	[VHOST_MSG_GET_VRING_BASE] = "VHOST_MSG_GET_VRING_BASE",
+	[VHOST_MSG_SET_VRING_ADDR] = "VHOST_MSG_SET_VRING_ADDR",
+	[VHOST_MSG_SET_VRING_KICK] = "VHOST_MSG_SET_VRING_KICK",
+	[VHOST_MSG_SET_MEM_TABLE] = "VHOST_MSG_SET_MEM_TABLE",
+	NULL,
+};
+
+static unsigned long int vhost_req_map[][2] = {
+	[VHOST_MSG_SET_OWNER] = {
+		VHOST_SET_OWNER, VHOST_USER_SET_OWNER
+	},
+	[VHOST_MSG_SET_FEATURES] = {
+		VHOST_SET_FEATURES, VHOST_USER_SET_FEATURES
+	},
+	[VHOST_MSG_GET_FEATURES] = {
+		VHOST_GET_FEATURES, VHOST_USER_GET_FEATURES
+	},
+	[VHOST_MSG_SET_VRING_CALL] = {
+		VHOST_SET_VRING_CALL, VHOST_USER_SET_VRING_CALL
+	},
+	[VHOST_MSG_SET_VRING_NUM] = {
+		VHOST_SET_VRING_NUM, VHOST_USER_SET_VRING_NUM
+	},
+	[VHOST_MSG_SET_VRING_BASE] = {
+		VHOST_SET_VRING_BASE, VHOST_USER_SET_VRING_BASE
+	},
+	[VHOST_MSG_GET_VRING_BASE] = {
+		VHOST_GET_VRING_BASE, VHOST_USER_GET_VRING_BASE
+	},
+	[VHOST_MSG_SET_VRING_ADDR] = {
+		VHOST_SET_VRING_ADDR, VHOST_USER_SET_VRING_ADDR
+	},
+	[VHOST_MSG_SET_VRING_KICK] = {
+		VHOST_SET_VRING_KICK, VHOST_USER_SET_VRING_KICK
+	},
+	[VHOST_MSG_SET_MEM_TABLE] = {
+		VHOST_SET_MEM_TABLE, VHOST_USER_SET_MEM_TABLE
+	},
+};
+
+static int
+vhost_call(struct virtio_hw *hw, unsigned long int req_orig, void *arg)
+{
+	unsigned long int req_new;
+	int ret;
+
+	if (req_orig >= VHOST_MSG_MAX)
+		rte_panic("invalid req: %lu\n", req_orig);
+
+	PMD_DRV_LOG(INFO, "%s", vhost_msg_strings[req_orig]);
+	req_new = vhost_req_map[req_orig][hw->type];
+	if (hw->type == VHOST_USER)
+		ret = vhost_user_sock(hw, req_new, arg);
+	else
+		ret = vhost_kernel_ioctl(hw, req_new, arg);
+
+	if (ret < 0)
+		rte_panic("vhost_call %s failed: %s\n",
+			  vhost_msg_strings[req_orig], strerror(errno));
+
+	return ret;
+}
+
+static void
+kick_one_vq(struct virtio_hw *hw, struct virtqueue *vq, unsigned queue_sel)
+{
+	int callfd, kickfd;
+	struct vhost_vring_file file;
+	struct vhost_vring_state state;
+	struct vhost_vring_addr addr = {
+		.index = queue_sel,
+		.desc_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.desc,
+		.avail_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.avail,
+		.used_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.used,
+		.log_guest_addr = 0,
+		.flags = 0, /* disable log */
+	};
+
+	/* or use invalid flag to disable it, but vhost-dpdk uses this to judge
+	 * if dev is alive. so finally we need two real event_fds.
+	 */
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL come
+	 * firstly because vhost depends on this msg to allocate virtqueue
+	 * pair.
+	 */
+	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (callfd < 0)
+		rte_panic("callfd error, %s\n", strerror(errno));
+
+	file.index = queue_sel;
+	file.fd = callfd;
+	vhost_call(hw, VHOST_MSG_SET_VRING_CALL, &file);
+	hw->callfds[queue_sel] = callfd;
+
+	state.index = queue_sel;
+	state.num = vq->vq_ring.num;
+	vhost_call(hw, VHOST_MSG_SET_VRING_NUM, &state);
+
+	state.num = 0; /* no reservation */
+	vhost_call(hw, VHOST_MSG_SET_VRING_BASE, &state);
+
+	vhost_call(hw, VHOST_MSG_SET_VRING_ADDR, &addr);
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_KICK comes
+	 * lastly because vhost depends on this msg to judge if
+	 * virtio_is_ready().
+	 */
+	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (kickfd < 0)
+		rte_panic("kickfd error, %s\n", strerror(errno));
+
+	file.fd = kickfd;
+	vhost_call(hw, VHOST_MSG_SET_VRING_KICK, &file);
+	hw->kickfds[queue_sel] = kickfd;
+}
+
+/**
+ * Merge those virtually adjacent memsegs into one region.
+ */
+static void
+prepare_vhost_memory_kernel(struct vhost_memory_kernel **p_vm)
+{
+	unsigned i, j, k = 0;
+	struct rte_memseg *seg;
+	struct vhost_memory_region *mr;
+	struct vhost_memory_kernel *vm;
+
+	vm = malloc(sizeof(struct vhost_memory_kernel) +
+		    RTE_MAX_MEMSEG * sizeof(struct vhost_memory_region));
+
+	for (i = 0; i < RTE_MAX_MEMSEG; ++i) {
+		seg = &rte_eal_get_configuration()->mem_config->memseg[i];
+		if (!seg->addr)
+			break;
+
+		int new_region = 1;
+
+		for (j = 0; j < k; ++j) {
+			mr = &vm->regions[j];
+
+			if (mr->userspace_addr + mr->memory_size ==
+			    (uint64_t)seg->addr) {
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+
+			if ((uint64_t)seg->addr + seg->len ==
+			    mr->userspace_addr) {
+				mr->guest_phys_addr = (uint64_t)seg->addr;
+				mr->userspace_addr = (uint64_t)seg->addr;
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+		}
+
+		if (new_region == 0)
+			continue;
+
+		mr = &vm->regions[k++];
+		mr->guest_phys_addr = (uint64_t)seg->addr; /* use vaddr here! */
+		mr->userspace_addr = (uint64_t)seg->addr;
+		mr->memory_size = seg->len;
+		mr->mmap_offset = 0;
+	}
+
+	vm->nregions = k;
+	vm->padding = 0;
+	*p_vm = vm;
+}
+
+static void kick_all_vq(struct virtio_hw *hw)
+{
+	uint64_t features;
+	unsigned i, queue_sel, nvqs;
+	struct vhost_vring_file file;
+	struct rte_eth_dev_data *data = hw->data;
+
+	if (hw->type == VHOST_KERNEL) {
+		struct vhost_memory_kernel *vm = NULL;
+
+		prepare_vhost_memory_kernel(&vm);
+		vhost_call(hw, VHOST_MSG_SET_MEM_TABLE, vm);
+		free(vm);
+	} else {
+		/* construct vhost_memory inside prepare_vhost_memory_user() */
+		vhost_call(hw, VHOST_MSG_SET_MEM_TABLE, NULL);
+	}
+
+	for (i = 0; i < data->nb_rx_queues; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
+		kick_one_vq(hw, data->rx_queues[i], queue_sel);
+	}
+	for (i = 0; i < data->nb_tx_queues; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
+		kick_one_vq(hw, data->tx_queues[i], queue_sel);
+	}
+
+	/* after setup all virtqueues, we need to set_features so that
+	 * these features can be set into each virtqueue in vhost side.
+	 */
+	features = hw->guest_features;
+	features &= ~(1ull << VIRTIO_NET_F_MAC);
+	vhost_call(hw, VHOST_MSG_SET_FEATURES, &features);
+	PMD_DRV_LOG(INFO, "set features: %" PRIx64, features);
+
+	if (hw->type == VHOST_USER)
+		return;
+
+	/* Special for VHOST_KERNEL */
+
+	if (ioctl(hw->backfd, TUNSETVNETHDRSZ, &hw->vtnet_hdr_size) == -1)
+		rte_panic("TUNSETVNETHDRSZ failed: %s\n", strerror(errno));
+
+	file.fd = hw->backfd;
+	nvqs = data->nb_rx_queues + data->nb_tx_queues;
+	for (file.index = 0; file.index < nvqs; ++file.index) {
+		if (vhost_kernel_ioctl(hw, VHOST_NET_SET_BACKEND, &file) < 0)
+			rte_panic("VHOST_NET_SET_BACKEND failed, %s\n",
+				  strerror(errno));
+	}
+}
+
+static void
+vdev_read_dev_config(struct virtio_hw *hw, uint64_t offset,
+		     void *dst, int length)
+{
+	int i;
+
+	if (offset == offsetof(struct virtio_net_config, mac) &&
+	    length == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			((uint8_t *)dst)[i] = hw->mac_addr[i];
+		return;
+	}
+
+	if (offset == offsetof(struct virtio_net_config, status))
+		*(uint16_t *)dst = hw->status;
+
+	if (offset == offsetof(struct virtio_net_config, max_virtqueue_pairs))
+		*(uint16_t *)dst = hw->max_tx_queues;
+}
+
+static void
+vdev_write_dev_config(struct virtio_hw *hw, uint64_t offset,
+		      const void *src, int length)
+{
+	int i;
+
+	if ((offset == offsetof(struct virtio_net_config, mac)) &&
+	    (length == ETHER_ADDR_LEN))
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			hw->mac_addr[i] = ((const uint8_t *)src)[i];
+	else
+		rte_panic("offset=%" PRIu64 ", length=%d\n", offset, length);
+}
+
+static void
+vdev_set_status(struct virtio_hw *hw, uint8_t status)
+{
+	if (status & VIRTIO_CONFIG_S_DRIVER_OK)
+		kick_all_vq(hw);
+	hw->status = status;
+}
+
+static void
+vdev_reset(struct virtio_hw *hw __rte_unused)
+{
+	/* do nothing according to qemu vhost user spec */
+}
+
+static uint8_t
+vdev_get_status(struct virtio_hw *hw)
+{
+	return hw->status;
+}
+
+static uint64_t
+vdev_get_features(struct virtio_hw *hw)
+{
+	uint64_t host_features;
+
+	vhost_call(hw, VHOST_MSG_GET_FEATURES, &host_features);
+	if (hw->mac_specified)
+		host_features |= (1ull << VIRTIO_NET_F_MAC);
+	/* disable it until we support CQ */
+	host_features &= ~(1ull << VIRTIO_NET_F_CTRL_VQ);
+	host_features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
+	return host_features;
+}
+
+static void
+vdev_set_features(struct virtio_hw *hw __rte_unused,
+		  uint64_t features __rte_unused)
+{
+	/* We hold to send this message, VHOST_MSG_SET_FEATURES,
+	 * until all VirtQueues have been kicked.
+	 */
+}
+
+static uint8_t
+vdev_get_isr(struct virtio_hw *hw __rte_unused)
+{
+	rte_panic("");
+}
+
+static uint16_t
+vdev_set_config_irq(struct virtio_hw *hw __rte_unused,
+		    uint16_t vec __rte_unused)
+{
+	rte_panic("");
+}
+
+static uint16_t
+vdev_get_queue_num(struct virtio_hw *hw,
+		   uint16_t queue_id __rte_unused)
+{
+	return hw->queue_num;
+}
+
+static void
+vdev_setup_queue(struct virtio_hw *hw __rte_unused,
+		 struct virtqueue *vq __rte_unused)
+{
+	/* do nothing */
+}
+
+static void
+vdev_del_queue(struct virtio_hw *hw __rte_unused,
+	       struct virtqueue *vq)
+{
+	struct vhost_vring_state state = {
+		.index = vq->vq_queue_index,
+	};
+
+	vhost_call(hw, VHOST_MSG_GET_VRING_BASE, &state);
+	PMD_DRV_LOG(DEBUG, "state.num = %d", state.num);
+}
+
+static void
+vdev_notify_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	uint64_t buf = 1;
+
+	if (write(hw->kickfds[vq->vq_queue_index],
+		  &buf, sizeof(uint64_t)) == -1)
+		rte_panic("%s\n", strerror(errno));
+}
+
+static const struct virtio_pci_ops vdev_ops = {
+	.read_dev_cfg	= vdev_read_dev_config,
+	.write_dev_cfg	= vdev_write_dev_config,
+	.reset		= vdev_reset,
+	.get_status	= vdev_get_status,
+	.set_status	= vdev_set_status,
+	.get_features	= vdev_get_features,
+	.set_features	= vdev_set_features,
+	.get_isr	= vdev_get_isr,
+	.set_config_irq	= vdev_set_config_irq,
+	.get_queue_num	= vdev_get_queue_num,
+	.setup_queue	= vdev_setup_queue,
+	.del_queue	= vdev_del_queue,
+	.notify_queue	= vdev_notify_queue,
+};
+
+#define TUN_DEF_SNDBUF	(1ull << 20)
+
+static void
+vhost_kernel_backend_setup(struct virtio_hw *hw, char *ifname)
+{
+	int fd;
+	int len = sizeof(struct virtio_net_hdr);
+	int req_mq = 0;
+	int sndbuf = TUN_DEF_SNDBUF;
+	unsigned int features;
+	struct ifreq ifr;
+
+	/* TODO:
+	 * 1. get/set offload capability, tap_probe_has_ufo, tap_fd_set_offload
+	 * 2. verify we can get/set vnet_hdr_len, tap_probe_vnet_hdr_len
+	 * 3. get number of memory regions from vhost module parameter
+	 * max_mem_regions, supported in newer version linux kernel
+	 */
+
+	fd = open(PATH_NET_TUN, O_RDWR);
+	if (fd < 0)
+		rte_panic("open %s error, %s\n", PATH_NET_TUN, strerror(errno));
+
+	memset(&ifr, 0, sizeof(ifr));
+	ifr.ifr_flags = IFF_TAP | IFF_NO_PI;
+
+	if (ioctl(fd, TUNGETFEATURES, &features) == -1)
+		rte_panic("TUNGETFEATURES failed: %s", strerror(errno));
+
+	if (features & IFF_ONE_QUEUE)
+		ifr.ifr_flags |= IFF_ONE_QUEUE;
+
+	if (features & IFF_VNET_HDR)
+		ifr.ifr_flags |= IFF_VNET_HDR;
+	else
+		rte_panic("vnet_hdr requested, but kernel does not support\n");
+
+	if (req_mq) {
+		if (features & IFF_MULTI_QUEUE)
+			ifr.ifr_flags |= IFF_MULTI_QUEUE;
+		else
+			rte_panic("multi queue requested, but not support\n");
+	}
+
+	if (ifname)
+		strncpy(ifr.ifr_name, ifname, IFNAMSIZ);
+	else
+		strncpy(ifr.ifr_name, "tap%d", IFNAMSIZ);
+	if (ioctl(fd, TUNSETIFF, (void *)&ifr) == -1)
+		rte_panic("TUNSETIFF failed: %s", strerror(errno));
+	fcntl(fd, F_SETFL, O_NONBLOCK);
+
+	if (ioctl(fd, TUNSETVNETHDRSZ, &len) == -1)
+		rte_panic("TUNSETVNETHDRSZ failed: %s\n", strerror(errno));
+
+	if (ioctl(fd, TUNSETSNDBUF, &sndbuf) == -1)
+		rte_panic("TUNSETSNDBUF failed: %s", strerror(errno));
+
+	hw->backfd = fd;
+	hw->vhostfd = open(hw->path, O_RDWR);
+	if (hw->vhostfd < 0)
+		rte_panic("open %s failed: %s\n", hw->path, strerror(errno));
+}
+
+static void
+vhost_user_backend_setup(struct virtio_hw *hw)
+{
+	int fd;
+	int flag;
+	struct sockaddr_un un;
+
+	fd = socket(AF_UNIX, SOCK_STREAM, 0);
+	if (fd < 0)
+		rte_panic("socket error, %s\n", strerror(errno));
+
+	flag = fcntl(fd, F_GETFD);
+	fcntl(fd, F_SETFD, flag | FD_CLOEXEC);
+
+	memset(&un, 0, sizeof(un));
+	un.sun_family = AF_UNIX;
+	snprintf(un.sun_path, sizeof(un.sun_path), "%s", hw->path);
+	if (connect(fd, (struct sockaddr *)&un, sizeof(un)) < 0) {
+		PMD_DRV_LOG(ERR, "connect error, %s", strerror(errno));
+		rte_panic("connect error, %s\n", strerror(errno));
+	}
+
+	hw->vhostfd = fd;
+}
+
+static void
+virtio_vdev_init(struct rte_eth_dev_data *data, char *path,
+		 int nb_rx, int nb_tx, int nb_cq __attribute__ ((unused)),
+		 int queue_num, char *mac, char *ifname)
+{
+	int i, r;
+	struct stat s;
+	uint32_t tmp[ETHER_ADDR_LEN];
+	struct virtio_hw *hw = data->dev_private;
+
+	hw->vtpci_ops = &vdev_ops;
+	hw->use_msix = 0;
+	hw->modern   = 0;
+
+	hw->data = data;
+	hw->path = strdup(path);
+	hw->max_rx_queues = nb_rx;
+	hw->max_tx_queues = nb_tx;
+	hw->queue_num = queue_num;
+	hw->mac_specified = 0;
+	if (mac) {
+		r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
+			   &tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
+		if (r == ETHER_ADDR_LEN) {
+			for (i = 0; i < ETHER_ADDR_LEN; ++i)
+				hw->mac_addr[i] = (uint8_t)tmp[i];
+			hw->mac_specified = 1;
+		} else
+			PMD_DRV_LOG(ERR, "wrong format of mac: %s", mac);
+	}
+
+	/* TODO: cq */
+
+	if (stat(hw->path, &s) < 0)
+		rte_panic("stat: %s failed, %s\n", hw->path, strerror(errno));
+
+	switch (s.st_mode & S_IFMT) {
+	case S_IFCHR:
+		hw->type = VHOST_KERNEL;
+		vhost_kernel_backend_setup(hw, ifname);
+		break;
+	case S_IFSOCK:
+		hw->type = VHOST_USER;
+		vhost_user_backend_setup(hw);
+		break;
+	default:
+		rte_panic("unknown file type of %s\n", hw->path);
+	}
+	if (vhost_call(hw, VHOST_MSG_SET_OWNER, NULL) == -1)
+		rte_panic("vhost set_owner failed: %s\n", strerror(errno));
+}
+
+static void
+virtio_vdev_uninit(struct rte_eth_dev_data *data)
+{
+	struct virtio_hw *hw = data->dev_private;
+
+	free(hw->path);
+	if (hw->type == VHOST_KERNEL)
+		close(hw->backfd);
+	/* TODO: need update when enable mq */
+	close(hw->callfds[0]);
+	close(hw->kickfds[0]);
+
+	close(hw->vhostfd);
+}
diff --git a/drivers/net/virtio/vhost.h b/drivers/net/virtio/vhost.h
new file mode 100644
index 0000000..73d4f5c
--- /dev/null
+++ b/drivers/net/virtio/vhost.h
@@ -0,0 +1,194 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VHOST_NET_USER_H
+#define _VHOST_NET_USER_H
+
+#include <stdint.h>
+#include <linux/types.h>
+#include <linux/ioctl.h>
+
+#define VHOST_MEMORY_MAX_NREGIONS 8
+
+struct vhost_vring_state {
+	unsigned int index;
+	unsigned int num;
+};
+
+struct vhost_vring_file {
+	unsigned int index;
+	int fd;
+};
+
+struct vhost_vring_addr {
+	unsigned int index;
+	/* Option flags. */
+	unsigned int flags;
+	/* Flag values: */
+	/* Whether log address is valid. If set enables logging. */
+#define VHOST_VRING_F_LOG 0
+
+	/* Start of array of descriptors (virtually contiguous) */
+	uint64_t desc_user_addr;
+	/* Used structure address. Must be 32 bit aligned */
+	uint64_t used_user_addr;
+	/* Available structure address. Must be 16 bit aligned */
+	uint64_t avail_user_addr;
+	/* Logging support. */
+	/* Log writes to used structure, at offset calculated from specified
+	 * address. Address must be 32 bit aligned.
+	 */
+	uint64_t log_guest_addr;
+};
+
+#define VIRTIO_CONFIG_S_DRIVER_OK   4
+
+enum vhost_user_request {
+	VHOST_USER_NONE = 0,
+	VHOST_USER_GET_FEATURES = 1,
+	VHOST_USER_SET_FEATURES = 2,
+	VHOST_USER_SET_OWNER = 3,
+	VHOST_USER_RESET_OWNER = 4,
+	VHOST_USER_SET_MEM_TABLE = 5,
+	VHOST_USER_SET_LOG_BASE = 6,
+	VHOST_USER_SET_LOG_FD = 7,
+	VHOST_USER_SET_VRING_NUM = 8,
+	VHOST_USER_SET_VRING_ADDR = 9,
+	VHOST_USER_SET_VRING_BASE = 10,
+	VHOST_USER_GET_VRING_BASE = 11,
+	VHOST_USER_SET_VRING_KICK = 12,
+	VHOST_USER_SET_VRING_CALL = 13,
+	VHOST_USER_SET_VRING_ERR = 14,
+	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
+	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
+	VHOST_USER_GET_QUEUE_NUM = 17,
+	VHOST_USER_SET_VRING_ENABLE = 18,
+	VHOST_USER_MAX
+};
+
+struct vhost_memory_region {
+	uint64_t guest_phys_addr;
+	uint64_t memory_size; /* bytes */
+	uint64_t userspace_addr;
+	uint64_t mmap_offset;
+};
+
+struct vhost_memory_kernel {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[0];
+};
+
+struct vhost_memory {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[VHOST_MEMORY_MAX_NREGIONS];
+};
+
+struct vhost_user_msg {
+	enum vhost_user_request request;
+
+#define VHOST_USER_VERSION_MASK     0x3
+#define VHOST_USER_REPLY_MASK       (0x1 << 2)
+	uint32_t flags;
+	uint32_t size; /* the following payload size */
+	union {
+#define VHOST_USER_VRING_IDX_MASK   0xff
+#define VHOST_USER_VRING_NOFD_MASK  (0x1 << 8)
+		uint64_t u64;
+		struct vhost_vring_state state;
+		struct vhost_vring_addr addr;
+		struct vhost_memory memory;
+	} payload;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+} __attribute((packed));
+
+#define VHOST_USER_HDR_SIZE offsetof(struct vhost_user_msg, payload.u64)
+#define VHOST_USER_PAYLOAD_SIZE (sizeof(struct vhost_user_msg) - VHOST_USER_HDR_SIZE)
+
+/* The version of the protocol we support */
+#define VHOST_USER_VERSION    0x1
+
+/* ioctls */
+
+#define VHOST_VIRTIO 0xAF
+
+#define VHOST_GET_FEATURES _IOR(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_FEATURES _IOW(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_OWNER _IO(VHOST_VIRTIO, 0x01)
+#define VHOST_RESET_OWNER _IO(VHOST_VIRTIO, 0x02)
+#define VHOST_SET_MEM_TABLE _IOW(VHOST_VIRTIO, 0x03, struct vhost_memory_kernel)
+#define VHOST_SET_LOG_BASE _IOW(VHOST_VIRTIO, 0x04, __u64)
+#define VHOST_SET_LOG_FD _IOW(VHOST_VIRTIO, 0x07, int)
+#define VHOST_SET_VRING_NUM _IOW(VHOST_VIRTIO, 0x10, struct vhost_vring_state)
+#define VHOST_SET_VRING_ADDR _IOW(VHOST_VIRTIO, 0x11, struct vhost_vring_addr)
+#define VHOST_SET_VRING_BASE _IOW(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_GET_VRING_BASE _IOWR(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_SET_VRING_KICK _IOW(VHOST_VIRTIO, 0x20, struct vhost_vring_file)
+#define VHOST_SET_VRING_CALL _IOW(VHOST_VIRTIO, 0x21, struct vhost_vring_file)
+#define VHOST_SET_VRING_ERR _IOW(VHOST_VIRTIO, 0x22, struct vhost_vring_file)
+#define VHOST_NET_SET_BACKEND _IOW(VHOST_VIRTIO, 0x30, struct vhost_vring_file)
+
+/*****************************************************************************/
+
+/* Ioctl defines */
+#define TUNSETIFF     _IOW('T', 202, int)
+#define TUNGETFEATURES _IOR('T', 207, unsigned int)
+#define TUNSETOFFLOAD  _IOW('T', 208, unsigned int)
+#define TUNGETIFF      _IOR('T', 210, unsigned int)
+#define TUNSETSNDBUF   _IOW('T', 212, int)
+#define TUNGETVNETHDRSZ _IOR('T', 215, int)
+#define TUNSETVNETHDRSZ _IOW('T', 216, int)
+#define TUNSETQUEUE  _IOW('T', 217, int)
+#define TUNSETVNETLE _IOW('T', 220, int)
+#define TUNSETVNETBE _IOW('T', 222, int)
+
+/* TUNSETIFF ifr flags */
+#define IFF_TAP          0x0002
+#define IFF_NO_PI        0x1000
+#define IFF_ONE_QUEUE    0x2000
+#define IFF_VNET_HDR     0x4000
+#define IFF_MULTI_QUEUE  0x0100
+#define IFF_ATTACH_QUEUE 0x0200
+#define IFF_DETACH_QUEUE 0x0400
+
+/* Features for GSO (TUNSETOFFLOAD). */
+#define TUN_F_CSUM	0x01	/* You can hand me unchecksummed packets. */
+#define TUN_F_TSO4	0x02	/* I can handle TSO for IPv4 packets */
+#define TUN_F_TSO6	0x04	/* I can handle TSO for IPv6 packets */
+#define TUN_F_TSO_ECN	0x08	/* I can handle TSO with ECN bits. */
+#define TUN_F_UFO	0x10	/* I can handle UFO packets */
+
+#define PATH_NET_TUN "/dev/net/tun"
+
+#endif
diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
index 66423a0..3f3b032 100644
--- a/drivers/net/virtio/virtio_ethdev.h
+++ b/drivers/net/virtio/virtio_ethdev.h
@@ -121,5 +121,4 @@ uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 #define VTNET_LRO_FEATURES (VIRTIO_NET_F_GUEST_TSO4 | \
 			    VIRTIO_NET_F_GUEST_TSO6 | VIRTIO_NET_F_GUEST_ECN)
 
-
 #endif /* _VIRTIO_ETHDEV_H_ */
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index b69785e..68097e6 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -260,6 +260,20 @@ struct virtio_hw {
 	struct virtio_pci_common_cfg *common_cfg;
 	struct virtio_net_config *dev_cfg;
 	const struct virtio_pci_ops *vtpci_ops;
+#ifdef RTE_VIRTIO_VDEV
+#define VHOST_KERNEL	0
+#define VHOST_USER	1
+	int		type; /* type of backend */
+	uint32_t	queue_num;
+	char		*path;
+	int		mac_specified;
+	int		vhostfd;
+	int		backfd; /* tap device used in vhost-net */
+	int		callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	uint8_t		status;
+	struct rte_eth_dev_data *data;
+#endif
 };
 
 /*
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio
  2016-04-21  2:56 ` [PATCH v3 0/2] " Jianfeng Tan
  2016-04-21  2:56   ` [PATCH v3 1/2] virtio/vdev: add embeded device emulation Jianfeng Tan
@ 2016-04-21  2:56   ` Jianfeng Tan
  2016-04-21  8:51     ` David Marchand
                       ` (2 more replies)
  1 sibling, 3 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-04-21  2:56 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, michael.qiu, ann.zhuangyanying,
	mukawa, nhorman

Add a new virtual device named eth_cvio, it can be used just like
eth_ring, eth_null, etc.

Configured parameters include:
  - rx (optional, 1 by default), number of rx, not used for now.
  - tx (optional, 1 by default), number of tx, not used for now.
  - cq (optional, 0 by default), not supported for now.
  - mac (optional), random value will be given if not specified.
  - queue_size (optional, 256 by default), size of virtqueue.
  - path (madatory), path of vhost, depends on the file type, vhost
    user if the given path points to a unix socket; vhost-net if the
    given path points to a char device.
  - ifname (optional), specify the name of backend tap device; only
    valid when backend is vhost-net.

The major difference with original virtio for vm is that, here we use
virtual addr instead of physical addr for vhost to calculate relative
address.

When enable CONFIG_RTE_VIRTIO_VDEV (enabled by default), the compiled
library can be used in both VM and container environment.

Examples:
path_vhost=/dev/vhost-net # use vhost-net as a backend
path_vhost=<path_to_vhost_user> # use vhost-user as a backend

sudo ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
    --socket-mem 0,1024 --no-pci --file-prefix=l2fwd \
    --vdev=eth_cvio0,mac=00:01:02:03:04:05,path=$path_vhost -- -p 0x1

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-By: Neil Horman <nhorman@tuxdrver.com>
---
 doc/guides/nics/overview.rst             |  58 +++++-----
 doc/guides/rel_notes/release_16_07.rst   |   4 +
 drivers/net/virtio/rte_eth_virtio_vdev.c | 188 ++++++++++++++++++++++++++++++-
 drivers/net/virtio/virtio_ethdev.c       | 134 ++++++++++++++--------
 drivers/net/virtio/virtio_ethdev.h       |   3 +
 drivers/net/virtio/virtio_pci.h          |   2 +-
 drivers/net/virtio/virtio_rxtx.c         |   5 +-
 drivers/net/virtio/virtio_rxtx_simple.c  |  13 ++-
 drivers/net/virtio/virtqueue.h           |  10 ++
 9 files changed, 326 insertions(+), 91 deletions(-)

diff --git a/doc/guides/nics/overview.rst b/doc/guides/nics/overview.rst
index ed116e3..1ff72fb 100644
--- a/doc/guides/nics/overview.rst
+++ b/doc/guides/nics/overview.rst
@@ -74,40 +74,40 @@ Most of these differences are summarized below.
 
 .. table:: Features availability in networking drivers
 
-   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
-   Feature              a b b b c e e e i i i i i i i i i i f f f f m m m n n p r s v v v v x
-                        f n n o x 1 n n 4 4 4 4 g g x x x x m m m m l l p f u c i z h i i m e
-                        p x x n g 0 a i 0 0 0 0 b b g g g g 1 1 1 1 x x i p l a n e o r r x n
-                        a 2 2 d b 0   c e e e e   v b b b b 0 0 0 0 4 5 p   l p g d s t t n v
-                        c x x i e 0       . v v   f e e e e k k k k     e         a t i i e i
-                        k   v n           . f f       . v v   . v v               t   o o t r
-                        e   f g           .   .       . f f   . f f               a     . 3 t
+   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
+   Feature              a b b b c e e e i i i i i i i i i i f f f f m m m n n p r s v v v v x c
+                        f n n o x 1 n n 4 4 4 4 g g x x x x m m m m l l p f u c i z h i i m e v
+                        p x x n g 0 a i 0 0 0 0 b b g g g g 1 1 1 1 x x i p l a n e o r r x n i
+                        a 2 2 d b 0   c e e e e   v b b b b 0 0 0 0 4 5 p   l p g d s t t n v r
+                        c x x i e 0       . v v   f e e e e k k k k     e         a t i i e i t
+                        k   v n           . f f       . v v   . v v               t   o o t r i
+                        e   f g           .   .       . f f   . f f               a     . 3 t o
                         t                 v   v       v   v   v   v               2     v
                                           e   e       e   e   e   e                     e
                                           c   c       c   c   c   c                     c
-   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
+   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
    speed capabilities
-   link status            X X   X X   X X X     X   X X X X         X X           X X X X
+   link status            X X   X X   X X X     X   X X X X         X X           X X X X     X
    link status event      X X     X     X X     X   X X             X X             X
    queue status event                                                               X
    Rx interrupt                   X     X X X X X X X X X X X X X X
-   queue start/stop             X   X X X X X X     X X     X X X X X X           X   X X
+   queue start/stop             X   X X X X X X     X X     X X X X X X           X   X X     X
    MTU update                   X X X           X   X X X X         X X
    jumbo frame                  X X X X X X X X X   X X X X X X X X X X       X
-   scattered Rx                 X X X   X X X X X X X X X X X X X X X X           X   X
+   scattered Rx                 X X X   X X X X X X X X X X X X X X X X           X   X       X
    LRO                                              X X X X
    TSO                          X   X   X X X X X X X X X X X X X X
-   promiscuous mode       X X   X X   X X X X X X X X X     X X     X X           X   X X
-   allmulticast mode            X X     X X X X X X X X X X X X     X X           X   X X
-   unicast MAC filter     X X     X   X X X X X X X X X X X X X     X X               X X
-   multicast MAC filter   X X         X X X X X             X X     X X               X X
+   promiscuous mode       X X   X X   X X X X X X X X X     X X     X X           X   X X     X
+   allmulticast mode            X X     X X X X X X X X X X X X     X X           X   X X     X
+   unicast MAC filter     X X     X   X X X X X X X X X X X X X     X X               X X     X
+   multicast MAC filter   X X         X X X X X             X X     X X               X X     X
    RSS hash                     X   X X X X X X X   X X X X X X X X X X
    RSS key update                   X   X X X X X   X X X X X X X X   X
    RSS reta update                  X   X X X X X   X X X X X X X X   X
    VMDq                                 X X     X   X X     X X
    SR-IOV                   X       X   X X     X   X X             X X
    DCB                                  X X     X   X X
-   VLAN filter                    X   X X X X X X X X X X X X X     X X               X X
+   VLAN filter                    X   X X X X X X X X X X X X X     X X               X X     X
    ethertype filter                     X X     X   X X
    n-tuple filter                               X   X X
    SYN filter                                   X   X X
@@ -127,23 +127,23 @@ Most of these differences are summarized below.
    inner L4 checksum                X   X   X       X   X           X
    packet type parsing          X     X X   X   X X X   X   X X X X X X
    timesync                             X X     X   X X
-   basic stats            X X   X X X X X X X X X X X X X X X X X X X X       X   X X X X
-   extended stats                   X   X X X X X X X X X X X X X X                   X X
-   stats per queue              X                   X X     X X X X X X           X   X X
+   basic stats            X X   X X X X X X X X X X X X X X X X X X X X       X   X X X X     X
+   extended stats                   X   X X X X X X X X X X X X X X                   X X     X
+   stats per queue              X                   X X     X X X X X X           X   X X     X
    EEPROM dump                                  X   X X
    registers dump                               X X X X X X
    multiprocess aware                   X X X X     X X X X X X X X X X       X
-   BSD nic_uio                  X X   X X X X X X X X X X X X X X X                   X X
-   Linux UIO              X X   X X X X X X X X X X X X X X X X X X                   X X
-   Linux VFIO                   X X   X X X X X X X X X X X X X X X                   X X
+   BSD nic_uio                  X X   X X X X X X X X X X X X X X X                   X X     X
+   Linux UIO              X X   X X X X X X X X X X X X X X X X X X                   X X     X
+   Linux VFIO                   X X   X X X X X X X X X X X X X X X                   X X     X
    other kdrv                                                       X X           X
-   ARMv7                                                                      X       X X
-   ARMv8                                                                      X       X X
+   ARMv7                                                                      X       X X     X
+   ARMv8                                                                      X       X X     X
    Power8                                                           X X       X
    TILE-Gx                                                                    X
-   x86-32                       X X X X X X X X X X X X X X X X X X X X       X     X X X
-   x86-64                 X X   X X X X X X X X X X X X X X X X X X X X       X   X X X X
-   usage doc              X X   X     X                             X X       X   X   X
+   x86-32                       X X X X X X X X X X X X X X X X X X X X       X     X X X     X
+   x86-64                 X X   X X X X X X X X X X X X X X X X X X X X       X   X X X X     X
+   usage doc              X X   X     X                             X X       X   X   X       X
    design doc
    perf doc
-   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
+   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
diff --git a/doc/guides/rel_notes/release_16_07.rst b/doc/guides/rel_notes/release_16_07.rst
index 701e827..f26639c 100644
--- a/doc/guides/rel_notes/release_16_07.rst
+++ b/doc/guides/rel_notes/release_16_07.rst
@@ -34,6 +34,10 @@ This section should contain new features added in this release. Sample format:
 
   Refer to the previous release notes for examples.
 
+* **Virtio support for containers.**
+
+  Add a new virtual device, named eth_cvio, to support virtio for containers.
+
 
 Resolved Issues
 ---------------
diff --git a/drivers/net/virtio/rte_eth_virtio_vdev.c b/drivers/net/virtio/rte_eth_virtio_vdev.c
index 419acef..282c9ae 100644
--- a/drivers/net/virtio/rte_eth_virtio_vdev.c
+++ b/drivers/net/virtio/rte_eth_virtio_vdev.c
@@ -50,6 +50,8 @@
 #include <rte_mbuf.h>
 #include <rte_memory.h>
 #include <rte_eal_memconfig.h>
+#include <rte_malloc.h>
+#include <rte_kvargs.h>
 
 #include "virtio_pci.h"
 #include "virtio_logs.h"
@@ -692,7 +694,7 @@ static uint16_t
 vdev_get_queue_num(struct virtio_hw *hw,
 		   uint16_t queue_id __rte_unused)
 {
-	return hw->queue_num;
+	return hw->queue_size;
 }
 
 static void
@@ -832,7 +834,7 @@ vhost_user_backend_setup(struct virtio_hw *hw)
 static void
 virtio_vdev_init(struct rte_eth_dev_data *data, char *path,
 		 int nb_rx, int nb_tx, int nb_cq __attribute__ ((unused)),
-		 int queue_num, char *mac, char *ifname)
+		 int queue_size, char *mac, char *ifname)
 {
 	int i, r;
 	struct stat s;
@@ -847,7 +849,7 @@ virtio_vdev_init(struct rte_eth_dev_data *data, char *path,
 	hw->path = strdup(path);
 	hw->max_rx_queues = nb_rx;
 	hw->max_tx_queues = nb_tx;
-	hw->queue_num = queue_num;
+	hw->queue_size = queue_size;
 	hw->mac_specified = 0;
 	if (mac) {
 		r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
@@ -895,3 +897,183 @@ virtio_vdev_uninit(struct rte_eth_dev_data *data)
 
 	close(hw->vhostfd);
 }
+
+static const char *valid_args[] = {
+#define CVIO_ARG_RX_NUM         "rx"
+	CVIO_ARG_RX_NUM,
+#define CVIO_ARG_TX_NUM         "tx"
+	CVIO_ARG_TX_NUM,
+#define CVIO_ARG_CQ_NUM         "cq"
+	CVIO_ARG_CQ_NUM,
+#define CVIO_ARG_MAC            "mac"
+	CVIO_ARG_MAC,
+#define CVIO_ARG_PATH           "path"
+	CVIO_ARG_PATH,
+#define CVIO_ARG_QUEUE_SIZE     "queue_size"
+	CVIO_ARG_QUEUE_SIZE,
+#define CVIO_ARG_IFNAME         "ifname"
+	CVIO_ARG_IFNAME,
+	NULL
+};
+
+#define CVIO_DEF_CQ_EN	0
+#define CVIO_DEF_Q_NUM	1
+#define CVIO_DEF_Q_SZ	256
+
+static int
+get_string_arg(const char *key __rte_unused,
+	       const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(char **)extra_args = strdup(value);
+
+	return 0;
+}
+
+static int
+get_integer_arg(const char *key __rte_unused,
+		const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(uint64_t *)extra_args = strtoull(value, NULL, 0);
+
+	return 0;
+}
+
+static struct rte_eth_dev *
+cvio_eth_dev_alloc(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct rte_eth_dev_data *data;
+	struct virtio_hw *hw;
+
+	eth_dev = rte_eth_dev_allocate(name, RTE_ETH_DEV_VIRTUAL);
+	if (!eth_dev)
+		rte_panic("cannot alloc rte_eth_dev\n");
+
+	data = eth_dev->data;
+
+	hw = rte_zmalloc(NULL, sizeof(*hw), 0);
+	if (!hw)
+		rte_panic("malloc virtio_hw failed\n");
+
+	data->dev_private = hw;
+	data->numa_node = SOCKET_ID_ANY;
+	data->kdrv = RTE_KDRV_NONE;
+	data->dev_flags = RTE_ETH_DEV_DETACHABLE;
+	eth_dev->pci_dev = NULL;
+	eth_dev->driver = NULL;
+	return eth_dev;
+}
+
+/* Dev initialization routine. Invoked once for each virtio vdev at
+ * EAL init time, see rte_eal_dev_init().
+ * Returns 0 on success.
+ */
+static int
+rte_cvio_pmd_devinit(const char *name, const char *params)
+{
+	struct rte_kvargs *kvlist = NULL;
+	struct rte_eth_dev *eth_dev = NULL;
+	uint64_t nb_rx = CVIO_DEF_Q_NUM;
+	uint64_t nb_tx = CVIO_DEF_Q_NUM;
+	uint64_t nb_cq = CVIO_DEF_CQ_EN;
+	uint64_t queue_size = CVIO_DEF_Q_SZ;
+	char *sock_path = NULL;
+	char *mac_addr = NULL;
+	char *ifname = NULL;
+
+	if (!params || params[0] == '\0')
+		rte_panic("arg %s is mandatory for eth_cvio\n",
+			  CVIO_ARG_QUEUE_SIZE);
+
+	kvlist = rte_kvargs_parse(params, valid_args);
+	if (!kvlist)
+		rte_panic("error when parsing param\n");
+
+	if (rte_kvargs_count(kvlist, CVIO_ARG_PATH) == 1)
+		rte_kvargs_process(kvlist, CVIO_ARG_PATH,
+				   &get_string_arg, &sock_path);
+	else
+		rte_panic("arg %s is mandatory for eth_cvio\n",
+			  CVIO_ARG_QUEUE_SIZE);
+
+	if (rte_kvargs_count(kvlist, CVIO_ARG_MAC) == 1)
+		rte_kvargs_process(kvlist, CVIO_ARG_MAC,
+				   &get_string_arg, &mac_addr);
+
+	if (rte_kvargs_count(kvlist, CVIO_ARG_IFNAME) == 1)
+		rte_kvargs_process(kvlist, CVIO_ARG_IFNAME,
+				   &get_string_arg, &ifname);
+
+	if (rte_kvargs_count(kvlist, CVIO_ARG_QUEUE_SIZE) == 1)
+		rte_kvargs_process(kvlist, CVIO_ARG_QUEUE_SIZE,
+				   &get_integer_arg, &queue_size);
+
+	if (rte_kvargs_count(kvlist, CVIO_ARG_RX_NUM) == 1)
+		rte_kvargs_process(kvlist, CVIO_ARG_RX_NUM,
+				   &get_integer_arg, &nb_rx);
+
+	if (rte_kvargs_count(kvlist, CVIO_ARG_TX_NUM) == 1)
+		rte_kvargs_process(kvlist, CVIO_ARG_TX_NUM,
+				   &get_integer_arg, &nb_tx);
+
+	if (rte_kvargs_count(kvlist, CVIO_ARG_CQ_NUM) == 1)
+		rte_kvargs_process(kvlist, CVIO_ARG_CQ_NUM,
+				   &get_integer_arg, &nb_cq);
+
+	eth_dev = cvio_eth_dev_alloc(name);
+
+	virtio_vdev_init(eth_dev->data, sock_path, nb_rx, nb_tx, nb_cq,
+			 queue_size, mac_addr, ifname);
+	if (sock_path)
+		free(sock_path);
+	if (mac_addr)
+		free(mac_addr);
+	if (ifname)
+		free(ifname);
+
+	/* originally, this will be called in rte_eal_pci_probe() */
+	eth_virtio_dev_init(eth_dev);
+
+	return 0;
+}
+
+/** Called by rte_eth_dev_detach() */
+static int
+rte_cvio_pmd_devuninit(const char *name)
+{
+	struct rte_eth_dev *eth_dev = NULL;
+
+	if (!name)
+		return -EINVAL;
+
+	PMD_DRV_LOG(INFO, PMD, "Un-Initializing %s\n", name);
+	eth_dev = rte_eth_dev_allocated(name);
+	if (!eth_dev)
+		return -ENODEV;
+
+	/* make sure the device is stopped, queues freed */
+	rte_eth_dev_close(eth_dev->data->port_id);
+
+	virtio_vdev_uninit(eth_dev->data);
+
+	rte_free(eth_dev->data->dev_private);
+	rte_free(eth_dev->data);
+	rte_eth_dev_release_port(eth_dev);
+
+	return 0;
+}
+
+static struct rte_driver rte_cvio_driver = {
+	.name   = "eth_cvio",
+	.type   = PMD_VDEV,
+	.init   = rte_cvio_pmd_devinit,
+	.uninit = rte_cvio_pmd_devuninit,
+};
+
+PMD_REGISTER_DRIVER(rte_cvio_driver);
diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 63a368a..9d28384 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -60,7 +60,6 @@
 #include "virtio_rxtx.h"
 
 
-static int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
 static int eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev);
 static int  virtio_dev_configure(struct rte_eth_dev *dev);
 static int  virtio_dev_start(struct rte_eth_dev *dev);
@@ -168,14 +167,14 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	 * One RX packet for ACK.
 	 */
 	vq->vq_ring.desc[head].flags = VRING_DESC_F_NEXT;
-	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mz->phys_addr;
+	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mem;
 	vq->vq_ring.desc[head].len = sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_free_cnt--;
 	i = vq->vq_ring.desc[head].next;
 
 	for (k = 0; k < pkt_num; k++) {
 		vq->vq_ring.desc[i].flags = VRING_DESC_F_NEXT;
-		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr)
 			+ sizeof(ctrl->status) + sizeof(uint8_t)*sum;
 		vq->vq_ring.desc[i].len = dlen[k];
@@ -185,7 +184,7 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	}
 
 	vq->vq_ring.desc[i].flags = VRING_DESC_F_WRITE;
-	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_ring.desc[i].len = sizeof(ctrl->status);
 	vq->vq_free_cnt--;
@@ -363,20 +362,27 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		}
 	}
 
-	/*
-	 * Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
-	 * and only accepts 32 bit page frame number.
-	 * Check if the allocated physical memory exceeds 16TB.
-	 */
-	if ((mz->phys_addr + vq->vq_ring_size - 1) >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
-		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
-		rte_free(vq);
-		return -ENOMEM;
-	}
-
 	memset(mz->addr, 0, sizeof(mz->len));
 	vq->mz = mz;
-	vq->vq_ring_mem = mz->phys_addr;
+	if (dev->dev_type == RTE_ETH_DEV_PCI) {
+		vq->vq_ring_mem = mz->phys_addr;
+
+		/*
+		 * Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
+		 * and only accepts 32 bit page frame number.
+		 * Check if the allocated physical memory exceeds 16TB.
+		 */
+		if ((mz->phys_addr + vq->vq_ring_size - 1) >>
+				(VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
+			PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
+			rte_free(vq);
+			return -ENOMEM;
+		}
+	}
+#ifdef RTE_VIRTIO_VDEV
+	else
+		vq->vq_ring_mem = (phys_addr_t)mz->addr; /* Use vaddr!!! */
+#endif
 	vq->vq_ring_virt_mem = mz->addr;
 	PMD_INIT_LOG(DEBUG, "vq->vq_ring_mem:      0x%"PRIx64, (uint64_t)mz->phys_addr);
 	PMD_INIT_LOG(DEBUG, "vq->vq_ring_virt_mem: 0x%"PRIx64, (uint64_t)(uintptr_t)mz->addr);
@@ -407,7 +413,12 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 			}
 		}
 		vq->virtio_net_hdr_mz = hdr_mz;
-		vq->virtio_net_hdr_mem = hdr_mz->phys_addr;
+		if (dev->dev_type == RTE_ETH_DEV_PCI)
+			vq->virtio_net_hdr_mem = hdr_mz->phys_addr;
+#ifdef RTE_VIRTIO_VDEV
+		else
+			vq->virtio_net_hdr_mem = (phys_addr_t)hdr_mz->addr;
+#endif
 
 		txr = hdr_mz->addr;
 		memset(txr, 0, vq_size * sizeof(*txr));
@@ -440,13 +451,24 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 				return -ENOMEM;
 			}
 		}
-		vq->virtio_net_hdr_mem =
-			vq->virtio_net_hdr_mz->phys_addr;
+		if (dev->dev_type == RTE_ETH_DEV_PCI)
+			vq->virtio_net_hdr_mem = mz->phys_addr;
+#ifdef RTE_VIRTIO_VDEV
+		else
+			vq->virtio_net_hdr_mem = (phys_addr_t)mz->addr;
+#endif
 		memset(vq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
 	}
 
 	hw->vtpci_ops->setup_queue(hw, vq);
 
+	if (dev->dev_type == RTE_ETH_DEV_PCI)
+		vq->offset = offsetof(struct rte_mbuf, buf_physaddr);
+#ifdef RTE_VIRTIO_VDEV
+	else
+		vq->offset = offsetof(struct rte_mbuf, buf_addr);
+#endif
+
 	*pvq = vq;
 	return 0;
 }
@@ -499,8 +521,9 @@ virtio_dev_close(struct rte_eth_dev *dev)
 		virtio_dev_stop(dev);
 
 	/* reset the NIC */
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		vtpci_irq_config(hw, VIRTIO_MSI_NO_VECTOR);
+	if (dev->dev_type == RTE_ETH_DEV_PCI)
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			vtpci_irq_config(hw, VIRTIO_MSI_NO_VECTOR);
 	vtpci_reset(hw);
 	virtio_dev_free_mbufs(dev);
 	virtio_free_queues(dev);
@@ -1002,8 +1025,9 @@ virtio_interrupt_handler(__rte_unused struct rte_intr_handle *handle,
 	isr = vtpci_isr(hw);
 	PMD_DRV_LOG(INFO, "interrupt status = %#x", isr);
 
-	if (rte_intr_enable(&dev->pci_dev->intr_handle) < 0)
-		PMD_DRV_LOG(ERR, "interrupt enable failed");
+	if (dev->dev_type == RTE_ETH_DEV_PCI)
+		if (rte_intr_enable(&dev->pci_dev->intr_handle) < 0)
+			PMD_DRV_LOG(ERR, "interrupt enable failed");
 
 	if (isr & VIRTIO_PCI_ISR_CONFIG) {
 		if (virtio_dev_link_update(dev, 0) == 0)
@@ -1027,7 +1051,7 @@ rx_func_get(struct rte_eth_dev *eth_dev)
  * This function is based on probe() function in virtio_pci.c
  * It returns 0 on success.
  */
-static int
+int
 eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 {
 	struct virtio_hw *hw = eth_dev->data->dev_private;
@@ -1057,9 +1081,11 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	pci_dev = eth_dev->pci_dev;
 
-	ret = vtpci_init(pci_dev, hw);
-	if (ret)
-		return ret;
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		ret = vtpci_init(pci_dev, hw);
+		if (ret)
+			return ret;
+	}
 
 	/* Reset the device although not necessary at startup */
 	vtpci_reset(hw);
@@ -1073,10 +1099,12 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 		return -1;
 
 	/* If host does not support status then disable LSC */
-	if (!vtpci_with_feature(hw, VIRTIO_NET_F_STATUS))
-		pci_dev->driver->drv_flags &= ~RTE_PCI_DRV_INTR_LSC;
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (!vtpci_with_feature(hw, VIRTIO_NET_F_STATUS))
+			pci_dev->driver->drv_flags &= ~RTE_PCI_DRV_INTR_LSC;
 
-	rte_eth_copy_pci_info(eth_dev, pci_dev);
+		rte_eth_copy_pci_info(eth_dev, pci_dev);
+	}
 
 	rx_func_get(eth_dev);
 
@@ -1150,15 +1178,17 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	PMD_INIT_LOG(DEBUG, "hw->max_rx_queues=%d   hw->max_tx_queues=%d",
 			hw->max_rx_queues, hw->max_tx_queues);
-	PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
-			eth_dev->data->port_id, pci_dev->id.vendor_id,
-			pci_dev->id.device_id);
-
-	/* Setup interrupt callback  */
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		rte_intr_callback_register(&pci_dev->intr_handle,
-				   virtio_interrupt_handler, eth_dev);
-
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI) {
+		PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
+			     eth_dev->data->port_id, pci_dev->id.vendor_id,
+			     pci_dev->id.device_id);
+
+		/* Setup interrupt callback  */
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			rte_intr_callback_register(&pci_dev->intr_handle,
+						   virtio_interrupt_handler,
+						   eth_dev);
+	}
 	virtio_dev_cq_start(eth_dev);
 
 	return 0;
@@ -1190,10 +1220,11 @@ eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev)
 	eth_dev->data->mac_addrs = NULL;
 
 	/* reset interrupt callback  */
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		rte_intr_callback_unregister(&pci_dev->intr_handle,
-						virtio_interrupt_handler,
-						eth_dev);
+	if (eth_dev->dev_type == RTE_ETH_DEV_PCI)
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			rte_intr_callback_unregister(&pci_dev->intr_handle,
+						     virtio_interrupt_handler,
+						     eth_dev);
 	rte_eal_pci_unmap_device(pci_dev);
 
 	PMD_INIT_LOG(DEBUG, "dev_uninit completed");
@@ -1258,11 +1289,13 @@ virtio_dev_configure(struct rte_eth_dev *dev)
 		return -ENOTSUP;
 	}
 
-	if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
-		if (vtpci_irq_config(hw, 0) == VIRTIO_MSI_NO_VECTOR) {
-			PMD_DRV_LOG(ERR, "failed to set config vector");
-			return -EBUSY;
-		}
+	if (dev->dev_type == RTE_ETH_DEV_PCI) {
+		if (pci_dev->driver->drv_flags & RTE_PCI_DRV_INTR_LSC)
+			if (vtpci_irq_config(hw, 0) == VIRTIO_MSI_NO_VECTOR) {
+				PMD_DRV_LOG(ERR, "failed to set config vector");
+				return -EBUSY;
+			}
+	}
 
 	return 0;
 }
@@ -1431,7 +1464,10 @@ virtio_dev_info_get(struct rte_eth_dev *dev, struct rte_eth_dev_info *dev_info)
 {
 	struct virtio_hw *hw = dev->data->dev_private;
 
-	dev_info->driver_name = dev->driver->pci_drv.name;
+	if (dev->dev_type == RTE_ETH_DEV_PCI)
+		dev_info->driver_name = dev->driver->pci_drv.name;
+	else
+		dev_info->driver_name = "cvirtio PMD";
 	dev_info->max_rx_queues = (uint16_t)hw->max_rx_queues;
 	dev_info->max_tx_queues = (uint16_t)hw->max_tx_queues;
 	dev_info->min_rx_bufsize = VIRTIO_MIN_RX_BUFSIZE;
diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
index 3f3b032..284afaa 100644
--- a/drivers/net/virtio/virtio_ethdev.h
+++ b/drivers/net/virtio/virtio_ethdev.h
@@ -113,6 +113,8 @@ uint16_t virtio_recv_pkts_vec(void *rx_queue, struct rte_mbuf **rx_pkts,
 uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 		uint16_t nb_pkts);
 
+int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
+
 /*
  * The VIRTIO_NET_F_GUEST_TSO[46] features permit the host to send us
  * frames larger than 1514 bytes. We do not yet support software LRO
@@ -121,4 +123,5 @@ uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 #define VTNET_LRO_FEATURES (VIRTIO_NET_F_GUEST_TSO4 | \
 			    VIRTIO_NET_F_GUEST_TSO6 | VIRTIO_NET_F_GUEST_ECN)
 
+
 #endif /* _VIRTIO_ETHDEV_H_ */
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index 68097e6..3b47332 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -264,7 +264,7 @@ struct virtio_hw {
 #define VHOST_KERNEL	0
 #define VHOST_USER	1
 	int		type; /* type of backend */
-	uint32_t	queue_num;
+	uint32_t	queue_size;
 	char		*path;
 	int		mac_specified;
 	int		vhostfd;
diff --git a/drivers/net/virtio/virtio_rxtx.c b/drivers/net/virtio/virtio_rxtx.c
index ef21d8e..9d7e537 100644
--- a/drivers/net/virtio/virtio_rxtx.c
+++ b/drivers/net/virtio/virtio_rxtx.c
@@ -193,8 +193,7 @@ virtqueue_enqueue_recv_refill(struct virtqueue *vq, struct rte_mbuf *cookie)
 
 	start_dp = vq->vq_ring.desc;
 	start_dp[idx].addr =
-		(uint64_t)(cookie->buf_physaddr + RTE_PKTMBUF_HEADROOM
-		- hw->vtnet_hdr_size);
+		MBUF_DATA_DMA_ADDR(cookie, vq->offset) - hw->vtnet_hdr_size;
 	start_dp[idx].len =
 		cookie->buf_len - RTE_PKTMBUF_HEADROOM + hw->vtnet_hdr_size;
 	start_dp[idx].flags =  VRING_DESC_F_WRITE;
@@ -265,7 +264,7 @@ virtqueue_enqueue_xmit(struct virtqueue *txvq, struct rte_mbuf *cookie,
 	}
 
 	do {
-		start_dp[idx].addr  = rte_mbuf_data_dma_addr(cookie);
+		start_dp[idx].addr  = MBUF_DATA_DMA_ADDR(cookie, txvq->offset);
 		start_dp[idx].len   = cookie->data_len;
 		start_dp[idx].flags = cookie->next ? VRING_DESC_F_NEXT : 0;
 		idx = start_dp[idx].next;
diff --git a/drivers/net/virtio/virtio_rxtx_simple.c b/drivers/net/virtio/virtio_rxtx_simple.c
index 8f5293d..599c42e 100644
--- a/drivers/net/virtio/virtio_rxtx_simple.c
+++ b/drivers/net/virtio/virtio_rxtx_simple.c
@@ -80,8 +80,8 @@ virtqueue_enqueue_recv_refill_simple(struct virtqueue *vq,
 	vq->sw_ring[desc_idx] = cookie;
 
 	start_dp = vq->vq_ring.desc;
-	start_dp[desc_idx].addr = (uint64_t)((uintptr_t)cookie->buf_physaddr +
-		RTE_PKTMBUF_HEADROOM - vq->hw->vtnet_hdr_size);
+	start_dp[desc_idx].addr = MBUF_DATA_DMA_ADDR(cookie, vq->offset) -
+		vq->hw->vtnet_hdr_size;
 	start_dp[desc_idx].len = cookie->buf_len -
 		RTE_PKTMBUF_HEADROOM + vq->hw->vtnet_hdr_size;
 
@@ -119,8 +119,8 @@ virtio_rxq_rearm_vec(struct virtqueue *rxvq)
 		*(uint64_t *)p = rxvq->mbuf_initializer;
 
 		start_dp[i].addr =
-			(uint64_t)((uintptr_t)sw_ring[i]->buf_physaddr +
-			RTE_PKTMBUF_HEADROOM - rxvq->hw->vtnet_hdr_size);
+			MBUF_DATA_DMA_ADDR(sw_ring[i], rxvq->offset) -
+			rxvq->hw->vtnet_hdr_size;
 		start_dp[i].len = sw_ring[i]->buf_len -
 			RTE_PKTMBUF_HEADROOM + rxvq->hw->vtnet_hdr_size;
 	}
@@ -366,7 +366,7 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 			txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 		for (i = 0; i < nb_tail; i++) {
 			start_dp[desc_idx].addr =
-				rte_mbuf_data_dma_addr(*tx_pkts);
+				MBUF_DATA_DMA_ADDR(*tx_pkts, txvq->offset);
 			start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 			tx_pkts++;
 			desc_idx++;
@@ -377,7 +377,8 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 	for (i = 0; i < nb_commit; i++)
 		txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 	for (i = 0; i < nb_commit; i++) {
-		start_dp[desc_idx].addr = rte_mbuf_data_dma_addr(*tx_pkts);
+		start_dp[desc_idx].addr =
+			MBUF_DATA_DMA_ADDR(*tx_pkts, txvq->offset);
 		start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 		tx_pkts++;
 		desc_idx++;
diff --git a/drivers/net/virtio/virtqueue.h b/drivers/net/virtio/virtqueue.h
index 4e9239e..81b5283 100644
--- a/drivers/net/virtio/virtqueue.h
+++ b/drivers/net/virtio/virtqueue.h
@@ -66,6 +66,14 @@ struct rte_mbuf;
 
 #define VIRTQUEUE_MAX_NAME_SZ 32
 
+#ifdef RTE_VIRTIO_VDEV
+#define MBUF_DATA_DMA_ADDR(mb, offset) \
+	((uint64_t)((uintptr_t)(*(void **)((uintptr_t)mb + offset)) \
+			+ (mb)->data_off))
+#else /* RTE_VIRTIO_VDEV */
+#define MBUF_DATA_DMA_ADDR(mb, offset) rte_mbuf_data_dma_addr(mb)
+#endif /* RTE_VIRTIO_VDEV */
+
 #define VTNET_SQ_RQ_QUEUE_IDX 0
 #define VTNET_SQ_TQ_QUEUE_IDX 1
 #define VTNET_SQ_CQ_QUEUE_IDX 2
@@ -165,6 +173,7 @@ struct virtqueue {
 	void        *vq_ring_virt_mem;    /**< linear address of vring*/
 	unsigned int vq_ring_size;
 	phys_addr_t vq_ring_mem;          /**< physical address of vring */
+					  /**< virtual address for vdev. */
 
 	struct vring vq_ring;    /**< vring keeping desc, used and avail */
 	uint16_t    vq_free_cnt; /**< num of desc available */
@@ -183,6 +192,7 @@ struct virtqueue {
 	 */
 	uint16_t vq_used_cons_idx;
 	uint16_t vq_avail_idx;
+	uint16_t offset; /**< relative offset to obtain addr in mbuf */
 	uint64_t mbuf_initializer; /**< value to init mbufs. */
 	phys_addr_t virtio_net_hdr_mem; /**< hdr for each xmit packet */
 
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio
  2016-04-21  2:56   ` [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
@ 2016-04-21  8:51     ` David Marchand
  2016-04-22  5:15       ` Tan, Jianfeng
  2016-04-21 10:05     ` Thomas Monjalon
  2016-04-21 22:14     ` Yuanhan Liu
  2 siblings, 1 reply; 196+ messages in thread
From: David Marchand @ 2016-04-21  8:51 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, Yuanhan Liu, Michael S. Tsirkin,
	nakajima.yoshihiro, p.fedin, Qiu, Michael, ann.zhuangyanying,
	Tetsuya Mukawa, nhorman

Hello,

On Thu, Apr 21, 2016 at 4:56 AM, Jianfeng Tan <jianfeng.tan@intel.com> wrote:
> Add a new virtual device named eth_cvio, it can be used just like
> eth_ring, eth_null, etc.
>
> Configured parameters include:
>   - rx (optional, 1 by default), number of rx, not used for now.
>   - tx (optional, 1 by default), number of tx, not used for now.
>   - cq (optional, 0 by default), not supported for now.
>   - mac (optional), random value will be given if not specified.
>   - queue_size (optional, 256 by default), size of virtqueue.
>   - path (madatory), path of vhost, depends on the file type, vhost
>     user if the given path points to a unix socket; vhost-net if the
>     given path points to a char device.
>   - ifname (optional), specify the name of backend tap device; only
>     valid when backend is vhost-net.
>
> The major difference with original virtio for vm is that, here we use
> virtual addr instead of physical addr for vhost to calculate relative
> address.
>
> When enable CONFIG_RTE_VIRTIO_VDEV (enabled by default), the compiled
> library can be used in both VM and container environment.

This implementation heavily relies on dev_type to keep as much code
shared between pci / vdev path as possible.

virtio code relies on drv_flags (even updating it while this should be
per-device).
So first, virtio should rely on dev_flags.

The rest needs to be astracted in some virtio ops ?


Thanks.

-- 
David Marchand

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio
  2016-04-21  2:56   ` [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
  2016-04-21  8:51     ` David Marchand
@ 2016-04-21 10:05     ` Thomas Monjalon
  2016-04-22  7:26       ` Tan, Jianfeng
  2016-04-21 22:14     ` Yuanhan Liu
  2 siblings, 1 reply; 196+ messages in thread
From: Thomas Monjalon @ 2016-04-21 10:05 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, yuanhan.liu, mst, nakajima.yoshihiro,
	p.fedin, michael.qiu, ann.zhuangyanying, mukawa, nhorman

2016-04-21 02:56, Jianfeng Tan:
> Add a new virtual device named eth_cvio, it can be used just like
> eth_ring, eth_null, etc.

Why this name eth_cvio?
Why the prefix eth_?
The virtio-net driver uses a kernel device. Here it is a userland device.
Why not virtio-user?

>  .. table:: Features availability in networking drivers
>  
> -   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
> -   Feature              a b b b c e e e i i i i i i i i i i f f f f m m m n n p r s v v v v x
> -                        f n n o x 1 n n 4 4 4 4 g g x x x x m m m m l l p f u c i z h i i m e
> -                        p x x n g 0 a i 0 0 0 0 b b g g g g 1 1 1 1 x x i p l a n e o r r x n
> -                        a 2 2 d b 0   c e e e e   v b b b b 0 0 0 0 4 5 p   l p g d s t t n v
> -                        c x x i e 0       . v v   f e e e e k k k k     e         a t i i e i
> -                        k   v n           . f f       . v v   . v v               t   o o t r
> -                        e   f g           .   .       . f f   . f f               a     . 3 t
> +   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
> +   Feature              a b b b c e e e i i i i i i i i i i f f f f m m m n n p r s v v v v x c
> +                        f n n o x 1 n n 4 4 4 4 g g x x x x m m m m l l p f u c i z h i i m e v
> +                        p x x n g 0 a i 0 0 0 0 b b g g g g 1 1 1 1 x x i p l a n e o r r x n i
> +                        a 2 2 d b 0   c e e e e   v b b b b 0 0 0 0 4 5 p   l p g d s t t n v r
> +                        c x x i e 0       . v v   f e e e e k k k k     e         a t i i e i t
> +                        k   v n           . f f       . v v   . v v               t   o o t r i
> +                        e   f g           .   .       . f f   . f f               a     . 3 t o
>                          t                 v   v       v   v   v   v               2     v
>                                            e   e       e   e   e   e                     e
>                                            c   c       c   c   c   c                     c
> -   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =

Please keep the alphabetical order.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 1/2] virtio/vdev: add embeded device emulation
  2016-04-21  2:56   ` [PATCH v3 1/2] virtio/vdev: add embeded device emulation Jianfeng Tan
@ 2016-04-21 22:01     ` Yuanhan Liu
  2016-04-22 10:12       ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2016-04-21 22:01 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	michael.qiu, ann.zhuangyanying, mukawa, nhorman, Thomas Monjalon

On Thu, Apr 21, 2016 at 02:56:35AM +0000, Jianfeng Tan wrote:
> Background: Previously, we usually use a virtio device in QEMU/VM's
> context as below pic shows. Virtio nic is emulated in QEMU, and usually
> presented in VM as a PCI device.
> 
> |-----------|
> |     vm    |
> |-----------| (over PCI bus or MMIO or Channel I/O)
> |    QEMU   | -> device emulation
> |-----------|
>       |
>       | (vhost-user protocol or vhost-net ioctls)
>       |
> |-----------|
> |   vhost   |
> |-----------|
> 
> Then we come to the topic that how to present a virtio device in an app
> or container, which uses virtio device to do inter process communication
> with vhost backend process. To achieve that, first of all, we need way
> in DPDK to interract with vhost backend. And then emulate a virtual
> virtio device in DPDK (which is addressed in following patch).
> 
> |-----------|
> |  DPDK app |
> |-----------|
> |  DPDK lib | -> device emulation (addressed by following patch)
> |-----------|
>       |
>       | (vhost-user protocol or vhost-net ioctls), addressed by this patch
>       |
> |-----------|
> |   vhost   |
> |-----------|
> 
> How: we implement another instance of struct virtio_pci_ops to intercept
> the communications between VM and QEMU. Instead of rd/wr ioport or PCI
> configuration space, here we directly talk with backend through the vhost
> file.

Nope, that's wrong, and here becomes a bit subtle. I will try to make
some explanation here.

Let's talk about the normal case (with QEMU) first. Where, virtio PMD
is a driver, and virito device is emulated inside QEMU, and exposed by
PCI. So, virtio PMD talks to the device with ioport rd/wr (or MMIO for
virtio 1.0).

Till now, you are right.

However, vhost-user socket is for establishing a connection, providing
HOST with enough information, so that host can directly manipulate the
vring, to dequeue/enqueue buffers.

So, what you were saying about "directly talk with backend (the virtual
virtio device you created) through vhost file" is not right. Instead,
in your case, the (virtual) virtio device and the PMD driver is in
the same process space, therefore, you could actually access or the
device info simply by normal read/write.

As you can see, It's a bit messy to mix all of them (virtio PMD driver,
virtio device emulation, and vhost-uesr frontend) in one single directory
(or even in one single file as you did). Therefore, I'd suggest you to
make a new dir, say "virtio-user" (a good name from Thomas), and put all
files related to virtio device emulation and vhost-user frontend there.

Further more, I'd suggest to divide the code into following files:

- virtio-user/virtio.c

  All virtio device emulation goes here.

- virtio-user/vhost-user.c

  The vhost-user frontend implementation

- virtio-user/vhost-kernel.c

  vhost kernel hanldings, including setting the tap device.

- And, __maybe__ another standalone file for handling the talk
  between the driver and the device. (See more for the comments
  about virtio_pci_ops below).


That would make it much clearer, IMO.

Besides that, I came up with few minor nits below. You might want to
fix them all in the next version.

> +static int
> +vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
> +{
> +	int r;
> +	struct msghdr msgh;
> +	struct iovec iov;
> +	size_t fd_size = fd_num * sizeof(int);
> +	char control[CMSG_SPACE(fd_size)];
> +	struct cmsghdr *cmsg;
> +
> +	bzero(&msgh, sizeof(msgh));
> +	bzero(control, sizeof(control));

bzero is marked as deprecated (see the man page), use memset instead.

> +
> +static struct vhost_user_msg m __rte_unused;

Hmm, if it's not used, why define it. If it's used, why decorate it
with __rte_unused?

> +
> +static void
> +prepare_vhost_memory_user(struct vhost_user_msg *msg, int fds[])
> +{
> +	int i, num;
> +	struct hugepage_file_info huges[VHOST_MEMORY_MAX_NREGIONS];
> +	struct vhost_memory_region *mr;
> +
> +	num = get_hugepage_file_info(huges, VHOST_MEMORY_MAX_NREGIONS);
> +	if (num < 0)
> +		rte_panic("Failed to prepare memory for vhost-user\n");

Do not use rte_panic, unless it's really needed. I see no good reason
to use it in a driver. If something we need is out of order, just
return and print some log and tell the user that this driver will not
work. This would keep other components work. You may then argue that
we have only one driver in container usage, but still, it's not a
good habit.

> +static void
> +vdev_reset(struct virtio_hw *hw __rte_unused)
> +{
> +	/* do nothing according to qemu vhost user spec */

That's not the right way to quote spec, it barely tells us anything
useful. So, you should quote the content here.

> +
> +static const struct virtio_pci_ops vdev_ops = {
> +	.read_dev_cfg	= vdev_read_dev_config,
> +	.write_dev_cfg	= vdev_write_dev_config,
> +	.reset		= vdev_reset,
> +	.get_status	= vdev_get_status,
> +	.set_status	= vdev_set_status,
> +	.get_features	= vdev_get_features,
> +	.set_features	= vdev_set_features,
> +	.get_isr	= vdev_get_isr,
> +	.set_config_irq	= vdev_set_config_irq,
> +	.get_queue_num	= vdev_get_queue_num,
> +	.setup_queue	= vdev_setup_queue,
> +	.del_queue	= vdev_del_queue,
> +	.notify_queue	= vdev_notify_queue,
> +};

As stated above, this acutally does NOT belong to the virtual virtio
device emulation. It should be part of the code of the PMD driver.
You should seperate them.

> diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
> index b69785e..68097e6 100644
> --- a/drivers/net/virtio/virtio_pci.h
> +++ b/drivers/net/virtio/virtio_pci.h
> @@ -260,6 +260,20 @@ struct virtio_hw {
>  	struct virtio_pci_common_cfg *common_cfg;
>  	struct virtio_net_config *dev_cfg;
>  	const struct virtio_pci_ops *vtpci_ops;
> +#ifdef RTE_VIRTIO_VDEV
> +#define VHOST_KERNEL	0
> +#define VHOST_USER	1
> +	int		type; /* type of backend */
> +	uint32_t	queue_num;
> +	char		*path;
> +	int		mac_specified;
> +	int		vhostfd;
> +	int		backfd; /* tap device used in vhost-net */
> +	int		callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
> +	int		kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
> +	uint8_t		status;
> +	struct rte_eth_dev_data *data;
> +#endif

And put all of them to the virtio "device" context, in the virtio-user/
then.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio
  2016-04-21  2:56   ` [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
  2016-04-21  8:51     ` David Marchand
  2016-04-21 10:05     ` Thomas Monjalon
@ 2016-04-21 22:14     ` Yuanhan Liu
  2016-04-22 10:12       ` Tan, Jianfeng
  2 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2016-04-21 22:14 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	michael.qiu, ann.zhuangyanying, mukawa, nhorman

On Thu, Apr 21, 2016 at 02:56:36AM +0000, Jianfeng Tan wrote:
> Add a new virtual device named eth_cvio, it can be used just like
> eth_ring, eth_null, etc.
> 
> Configured parameters include:
>   - rx (optional, 1 by default), number of rx, not used for now.
>   - tx (optional, 1 by default), number of tx, not used for now.
>   - cq (optional, 0 by default), not supported for now.
>   - mac (optional), random value will be given if not specified.
>   - queue_size (optional, 256 by default), size of virtqueue.
>   - path (madatory), path of vhost, depends on the file type, vhost
>     user if the given path points to a unix socket; vhost-net if the
>     given path points to a char device.
>   - ifname (optional), specify the name of backend tap device; only
>     valid when backend is vhost-net.
> 
> The major difference with original virtio for vm is that, here we use
> virtual addr instead of physical addr for vhost to calculate relative
> address.
> 
> When enable CONFIG_RTE_VIRTIO_VDEV (enabled by default), the compiled
> library can be used in both VM and container environment.
> 
> Examples:
> path_vhost=/dev/vhost-net # use vhost-net as a backend
> path_vhost=<path_to_vhost_user> # use vhost-user as a backend
> 
> sudo ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
>     --socket-mem 0,1024 --no-pci --file-prefix=l2fwd \
>     --vdev=eth_cvio0,mac=00:01:02:03:04:05,path=$path_vhost -- -p 0x1
> 
> Known issues:
>  - Control queue and multi-queue are not supported yet.
>  - Cannot work with --huge-unlink.
>  - Cannot work with no-huge.
>  - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
>    hugepages.
>  - Root privilege is a must (mainly becase of sorting hugepages according
>    to physical address).
>  - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> Acked-By: Neil Horman <nhorman@tuxdrver.com>
> ---
>  doc/guides/nics/overview.rst             |  58 +++++-----
>  doc/guides/rel_notes/release_16_07.rst   |   4 +
>  drivers/net/virtio/rte_eth_virtio_vdev.c | 188 ++++++++++++++++++++++++++++++-

Why prefixing it with "rte_eth_..."?

> -   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
> -   Feature              a b b b c e e e i i i i i i i i i i f f f f m m m n n p r s v v v v x
> -                        f n n o x 1 n n 4 4 4 4 g g x x x x m m m m l l p f u c i z h i i m e
> -                        p x x n g 0 a i 0 0 0 0 b b g g g g 1 1 1 1 x x i p l a n e o r r x n
> -                        a 2 2 d b 0   c e e e e   v b b b b 0 0 0 0 4 5 p   l p g d s t t n v
> -                        c x x i e 0       . v v   f e e e e k k k k     e         a t i i e i
> -                        k   v n           . f f       . v v   . v v               t   o o t r
> -                        e   f g           .   .       . f f   . f f               a     . 3 t
> +   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
> +   Feature              a b b b c e e e i i i i i i i i i i f f f f m m m n n p r s v v v v x c
> +                        f n n o x 1 n n 4 4 4 4 g g x x x x m m m m l l p f u c i z h i i m e v
> +                        p x x n g 0 a i 0 0 0 0 b b g g g g 1 1 1 1 x x i p l a n e o r r x n i
> +                        a 2 2 d b 0   c e e e e   v b b b b 0 0 0 0 4 5 p   l p g d s t t n v r
> +                        c x x i e 0       . v v   f e e e e k k k k     e         a t i i e i t
> +                        k   v n           . f f       . v v   . v v               t   o o t r i
> +                        e   f g           .   .       . f f   . f f               a     . 3 t o
>                          t                 v   v       v   v   v   v               2     v


I would wish we have a diff that could do compare by columns but not by
rows :)


> diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
> index 68097e6..3b47332 100644
> --- a/drivers/net/virtio/virtio_pci.h
> +++ b/drivers/net/virtio/virtio_pci.h
> @@ -264,7 +264,7 @@ struct virtio_hw {
>  #define VHOST_KERNEL	0
>  #define VHOST_USER	1
>  	int		type; /* type of backend */
> -	uint32_t	queue_num;
> +	uint32_t	queue_size;

Hmm, this kind of change should not be squeezed here, stealthily. I
would agree that the rename in decreases the stealthily, which is a
good thing. You should submit a standalone patch instead.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio
  2016-04-21  8:51     ` David Marchand
@ 2016-04-22  5:15       ` Tan, Jianfeng
  2016-04-22  7:36         ` David Marchand
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-04-22  5:15 UTC (permalink / raw)
  To: David Marchand
  Cc: dev, Huawei Xie, rich.lane, Yuanhan Liu, Michael S. Tsirkin,
	nakajima.yoshihiro, p.fedin, Qiu, Michael, ann.zhuangyanying,
	Tetsuya Mukawa, nhorman

Hi,

On 4/21/2016 4:51 PM, David Marchand wrote:
> Hello,
>
> On Thu, Apr 21, 2016 at 4:56 AM, Jianfeng Tan <jianfeng.tan@intel.com> wrote:
>> Add a new virtual device named eth_cvio, it can be used just like
>> eth_ring, eth_null, etc.
>>
>> Configured parameters include:
>>    - rx (optional, 1 by default), number of rx, not used for now.
>>    - tx (optional, 1 by default), number of tx, not used for now.
>>    - cq (optional, 0 by default), not supported for now.
>>    - mac (optional), random value will be given if not specified.
>>    - queue_size (optional, 256 by default), size of virtqueue.
>>    - path (madatory), path of vhost, depends on the file type, vhost
>>      user if the given path points to a unix socket; vhost-net if the
>>      given path points to a char device.
>>    - ifname (optional), specify the name of backend tap device; only
>>      valid when backend is vhost-net.
>>
>> The major difference with original virtio for vm is that, here we use
>> virtual addr instead of physical addr for vhost to calculate relative
>> address.
>>
>> When enable CONFIG_RTE_VIRTIO_VDEV (enabled by default), the compiled
>> library can be used in both VM and container environment.
> This implementation heavily relies on dev_type to keep as much code
> shared between pci / vdev path as possible.

Yes, I still have no method to make it more clear.

>
> virtio code relies on drv_flags (even updating it while this should be
> per-device).
> So first, virtio should rely on dev_flags.

Mainly drv_flags's RTE_PCI_DRV_INTR_LSC, and RTE_PCI_DRV_DETACHABLE bit 
is used. I understand the issue, pointed out by you here, that if two 
virtio devices are used in a VM, one with feature VIRTIO_NET_F_STATUS, 
and the other without feature VIRTIO_NET_F_STATUS (under the case that 
two vhost backends are used). Then it leads to uncertainty of the behavior.

Since the flags has been copied into dev_flags after features 
negotiated, I believe we should use dev_flags instead of drv_flags. A 
patch to fix this will be sent.


>
> The rest needs to be astracted in some virtio ops ?

So with that fix goes, we may make it more clear now. Do you mean this?

Thanks,
Jianfeng

>
>
> Thanks.
>

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio
  2016-04-21 10:05     ` Thomas Monjalon
@ 2016-04-22  7:26       ` Tan, Jianfeng
  2016-04-22  8:30         ` Thomas Monjalon
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-04-22  7:26 UTC (permalink / raw)
  To: Thomas Monjalon
  Cc: dev, Huawei Xie, rich.lane, yuanhan.liu, mst, nakajima.yoshihiro,
	p.fedin, michael.qiu, ann.zhuangyanying, mukawa, nhorman

Hi Thomas,

On 4/21/2016 6:05 PM, Thomas Monjalon wrote:
> 2016-04-21 02:56, Jianfeng Tan:
>> Add a new virtual device named eth_cvio, it can be used just like
>> eth_ring, eth_null, etc.
> Why this name eth_cvio?
> Why the prefix eth_?
> The virtio-net driver uses a kernel device. Here it is a userland device.
> Why not virtio-user?

I was looking for an appropriate name for this device; have tried a lot, 
but none is good enough. Thank you for your advice virtio-user.
The prefix eth_ is to keep the same style with eth_ring, eth_null.
So how about eth_virtio_user?

>
>>   .. table:: Features availability in networking drivers
>>   
>> -   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
>> -   Feature              a b b b c e e e i i i i i i i i i i f f f f m m m n n p r s v v v v x
>> -                        f n n o x 1 n n 4 4 4 4 g g x x x x m m m m l l p f u c i z h i i m e
>> -                        p x x n g 0 a i 0 0 0 0 b b g g g g 1 1 1 1 x x i p l a n e o r r x n
>> -                        a 2 2 d b 0   c e e e e   v b b b b 0 0 0 0 4 5 p   l p g d s t t n v
>> -                        c x x i e 0       . v v   f e e e e k k k k     e         a t i i e i
>> -                        k   v n           . f f       . v v   . v v               t   o o t r
>> -                        e   f g           .   .       . f f   . f f               a     . 3 t
>> +   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
>> +   Feature              a b b b c e e e i i i i i i i i i i f f f f m m m n n p r s v v v v x c
>> +                        f n n o x 1 n n 4 4 4 4 g g x x x x m m m m l l p f u c i z h i i m e v
>> +                        p x x n g 0 a i 0 0 0 0 b b g g g g 1 1 1 1 x x i p l a n e o r r x n i
>> +                        a 2 2 d b 0   c e e e e   v b b b b 0 0 0 0 4 5 p   l p g d s t t n v r
>> +                        c x x i e 0       . v v   f e e e e k k k k     e         a t i i e i t
>> +                        k   v n           . f f       . v v   . v v               t   o o t r i
>> +                        e   f g           .   .       . f f   . f f               a     . 3 t o
>>                           t                 v   v       v   v   v   v               2     v
>>                                             e   e       e   e   e   e                     e
>>                                             c   c       c   c   c   c                     c
>> -   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
> Please keep the alphabetical order.

OK, I'll do it in next version.

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio
  2016-04-22  5:15       ` Tan, Jianfeng
@ 2016-04-22  7:36         ` David Marchand
  2016-04-22 10:25           ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: David Marchand @ 2016-04-22  7:36 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: dev, Huawei Xie, Rich Lane, Yuanhan Liu, Michael S. Tsirkin,
	nakajima.yoshihiro, p.fedin, Qiu, Michael, ann.zhuangyanying,
	Tetsuya Mukawa, nhorman

Hello,

On Fri, Apr 22, 2016 at 7:15 AM, Tan, Jianfeng <jianfeng.tan@intel.com> wrote:
> On 4/21/2016 4:51 PM, David Marchand wrote:
>> virtio code relies on drv_flags (even updating it while this should be
>> per-device).
>> So first, virtio should rely on dev_flags.
>
>
> Mainly drv_flags's RTE_PCI_DRV_INTR_LSC, and RTE_PCI_DRV_DETACHABLE bit is
> used. I understand the issue, pointed out by you here, that if two virtio
> devices are used in a VM, one with feature VIRTIO_NET_F_STATUS, and the
> other without feature VIRTIO_NET_F_STATUS (under the case that two vhost
> backends are used). Then it leads to uncertainty of the behavior.
>
> Since the flags has been copied into dev_flags after features negotiated, I
> believe we should use dev_flags instead of drv_flags. A patch to fix this
> will be sent.

Ok.

>> The rest needs to be astracted in some virtio ops ?
> So with that fix goes, we may make it more clear now. Do you mean this?

Well, here, we have what looks like to be two drivers (one pci and one vdev).
You tried to keep all this code together, to avoid duplicating it,
which sounds sane.
But in the end, you are trying to make this work by adding checks
where this can't.
I am not saying we should duplicate the code, but maybe having some
internal virtio ops / abstraction would do the trick and avoid those
checks.


The reason of those comments is that dev_type in ethdev is going to
disappear, see [1] and [2].
Drivers are called through their own specific ethdev/crypto ops and
so, those drivers know implicitely that their are either pci or vdev
(or whatever in the future) drivers.


[1]: http://dpdk.org/ml/archives/dev/2016-April/037686.html
[2]: http://dpdk.org/ml/archives/dev/2016-January/031390.html


-- 
David Marchand

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio
  2016-04-22  7:26       ` Tan, Jianfeng
@ 2016-04-22  8:30         ` Thomas Monjalon
  0 siblings, 0 replies; 196+ messages in thread
From: Thomas Monjalon @ 2016-04-22  8:30 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: dev, Huawei Xie, rich.lane, yuanhan.liu, mst, nakajima.yoshihiro,
	p.fedin, michael.qiu, ann.zhuangyanying, mukawa, nhorman

2016-04-22 15:26, Tan, Jianfeng:
> Hi Thomas,
> 
> On 4/21/2016 6:05 PM, Thomas Monjalon wrote:
> > 2016-04-21 02:56, Jianfeng Tan:
> >> Add a new virtual device named eth_cvio, it can be used just like
> >> eth_ring, eth_null, etc.
> > Why this name eth_cvio?
> > Why the prefix eth_?
> > The virtio-net driver uses a kernel device. Here it is a userland device.
> > Why not virtio-user?
> 
> I was looking for an appropriate name for this device; have tried a lot, 
> but none is good enough. Thank you for your advice virtio-user.
> The prefix eth_ is to keep the same style with eth_ring, eth_null.
> So how about eth_virtio_user?

I prefer not adding eth_. Just my opinion.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 1/2] virtio/vdev: add embeded device emulation
  2016-04-21 22:01     ` Yuanhan Liu
@ 2016-04-22 10:12       ` Tan, Jianfeng
  2016-04-22 10:17         ` Thomas Monjalon
  2016-04-22 17:27         ` Yuanhan Liu
  0 siblings, 2 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-04-22 10:12 UTC (permalink / raw)
  To: Yuanhan Liu
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	michael.qiu, ann.zhuangyanying, mukawa, nhorman, Thomas Monjalon

Hi Yuanhan,

On 4/22/2016 6:01 AM, Yuanhan Liu wrote:
> On Thu, Apr 21, 2016 at 02:56:35AM +0000, Jianfeng Tan wrote:
>> Background: Previously, we usually use a virtio device in QEMU/VM's
>> context as below pic shows. Virtio nic is emulated in QEMU, and usually
>> presented in VM as a PCI device.
>>
>> |-----------|
>> |     vm    |
>> |-----------| (over PCI bus or MMIO or Channel I/O)
>> |    QEMU   | -> device emulation
>> |-----------|
>>        |
>>        | (vhost-user protocol or vhost-net ioctls)
>>        |
>> |-----------|
>> |   vhost   |
>> |-----------|
>>
>> Then we come to the topic that how to present a virtio device in an app
>> or container, which uses virtio device to do inter process communication
>> with vhost backend process. To achieve that, first of all, we need way
>> in DPDK to interract with vhost backend. And then emulate a virtual
>> virtio device in DPDK (which is addressed in following patch).
>>
>> |-----------|
>> |  DPDK app |
>> |-----------|
>> |  DPDK lib | -> device emulation (addressed by following patch)
>> |-----------|
>>        |
>>        | (vhost-user protocol or vhost-net ioctls), addressed by this patch
>>        |
>> |-----------|
>> |   vhost   |
>> |-----------|
>>
>> How: we implement another instance of struct virtio_pci_ops to intercept
>> the communications between VM and QEMU. Instead of rd/wr ioport or PCI
>> configuration space, here we directly talk with backend through the vhost
>> file.
> Nope, that's wrong, and here becomes a bit subtle. I will try to make
> some explanation here.
>
> Let's talk about the normal case (with QEMU) first. Where, virtio PMD
> is a driver, and virito device is emulated inside QEMU, and exposed by
> PCI. So, virtio PMD talks to the device with ioport rd/wr (or MMIO for
> virtio 1.0).
>
> Till now, you are right.
>
> However, vhost-user socket is for establishing a connection, providing
> HOST with enough information, so that host can directly manipulate the
> vring, to dequeue/enqueue buffers.
>
> So, what you were saying about "directly talk with backend (the virtual
> virtio device you created) through vhost file" is not right. Instead,
> in your case, the (virtual) virtio device and the PMD driver is in
> the same process space, therefore, you could actually access or the
> device info simply by normal read/write.

Here by _backend_, I mean vhost. I know you take backend as the device 
emulation. I ask Huawei, he thinks backend = device emulation + vhost. 
So maybe I should use the phrase vhost for accuracy.

>
> As you can see, It's a bit messy to mix all of them (virtio PMD driver,
> virtio device emulation, and vhost-uesr frontend) in one single directory
> (or even in one single file as you did). Therefore, I'd suggest you to
> make a new dir, say "virtio-user" (a good name from Thomas), and put all
> files related to virtio device emulation and vhost-user frontend there.
>
> Further more, I'd suggest to divide the code into following files:
>
> - virtio-user/virtio.c
>
>    All virtio device emulation goes here.
>
> - virtio-user/vhost-user.c
>
>    The vhost-user frontend implementation
>
> - virtio-user/vhost-kernel.c
>
>    vhost kernel hanldings, including setting the tap device.
>
> - And, __maybe__ another standalone file for handling the talk
>    between the driver and the device. (See more for the comments
>    about virtio_pci_ops below).
>
>
> That would make it much clearer, IMO.

Got your point here, but to be honest, I'm a little concerned to split a 
1k-lined file into 5+ files. Previously I'd like to make them converged 
together. But your suggestion is great for clean code. If any other 
could give some comments?

>
> Besides that, I came up with few minor nits below. You might want to
> fix them all in the next version.
>
>> +static int
>> +vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
>> +{
>> +	int r;
>> +	struct msghdr msgh;
>> +	struct iovec iov;
>> +	size_t fd_size = fd_num * sizeof(int);
>> +	char control[CMSG_SPACE(fd_size)];
>> +	struct cmsghdr *cmsg;
>> +
>> +	bzero(&msgh, sizeof(msgh));
>> +	bzero(control, sizeof(control));
> bzero is marked as deprecated (see the man page), use memset instead.

I should apologize about this, you actually told me once before, but I 
failed to fix it in this version.

>
>> +
>> +static struct vhost_user_msg m __rte_unused;
> Hmm, if it's not used, why define it. If it's used, why decorate it
> with __rte_unused?

This variable is to make it easy to calculate length of some fields 
inside this struct. But seems that removing it does not leading to any 
compiling error. I'll remove it.

>
>> +
>> +static void
>> +prepare_vhost_memory_user(struct vhost_user_msg *msg, int fds[])
>> +{
>> +	int i, num;
>> +	struct hugepage_file_info huges[VHOST_MEMORY_MAX_NREGIONS];
>> +	struct vhost_memory_region *mr;
>> +
>> +	num = get_hugepage_file_info(huges, VHOST_MEMORY_MAX_NREGIONS);
>> +	if (num < 0)
>> +		rte_panic("Failed to prepare memory for vhost-user\n");
> Do not use rte_panic, unless it's really needed. I see no good reason
> to use it in a driver. If something we need is out of order, just
> return and print some log and tell the user that this driver will not
> work. This would keep other components work. You may then argue that
> we have only one driver in container usage, but still, it's not a
> good habit.

Just want to make it "fail early, fair loudly". But I agree to keep same 
stype with other driver.

>
>> +static void
>> +vdev_reset(struct virtio_hw *hw __rte_unused)
>> +{
>> +	/* do nothing according to qemu vhost user spec */
> That's not the right way to quote spec, it barely tells us anything
> useful. So, you should quote the content here.

OK, I'll add more info here.

>
>> +
>> +static const struct virtio_pci_ops vdev_ops = {
>> +	.read_dev_cfg	= vdev_read_dev_config,
>> +	.write_dev_cfg	= vdev_write_dev_config,
>> +	.reset		= vdev_reset,
>> +	.get_status	= vdev_get_status,
>> +	.set_status	= vdev_set_status,
>> +	.get_features	= vdev_get_features,
>> +	.set_features	= vdev_set_features,
>> +	.get_isr	= vdev_get_isr,
>> +	.set_config_irq	= vdev_set_config_irq,
>> +	.get_queue_num	= vdev_get_queue_num,
>> +	.setup_queue	= vdev_setup_queue,
>> +	.del_queue	= vdev_del_queue,
>> +	.notify_queue	= vdev_notify_queue,
>> +};
> As stated above, this acutally does NOT belong to the virtual virtio
> device emulation. It should be part of the code of the PMD driver.
> You should seperate them.



>
>> diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
>> index b69785e..68097e6 100644
>> --- a/drivers/net/virtio/virtio_pci.h
>> +++ b/drivers/net/virtio/virtio_pci.h
>> @@ -260,6 +260,20 @@ struct virtio_hw {
>>   	struct virtio_pci_common_cfg *common_cfg;
>>   	struct virtio_net_config *dev_cfg;
>>   	const struct virtio_pci_ops *vtpci_ops;
>> +#ifdef RTE_VIRTIO_VDEV
>> +#define VHOST_KERNEL	0
>> +#define VHOST_USER	1
>> +	int		type; /* type of backend */
>> +	uint32_t	queue_num;
>> +	char		*path;
>> +	int		mac_specified;
>> +	int		vhostfd;
>> +	int		backfd; /* tap device used in vhost-net */
>> +	int		callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
>> +	int		kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
>> +	uint8_t		status;
>> +	struct rte_eth_dev_data *data;
>> +#endif
> And put all of them to the virtio "device" context, in the virtio-user/
> then.

OK, I'll define a new struct to store these fields. I consider there are 
two methods to refer these data. One is to store a pointer to the 
instance of the new struct in the struct virtio_hw; the other is to main 
them totally closed inside virtio-user, such as using a linked chain. 
Which do you prefer?

Thanks,
Jianfeng

>
> 	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio
  2016-04-21 22:14     ` Yuanhan Liu
@ 2016-04-22 10:12       ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-04-22 10:12 UTC (permalink / raw)
  To: Yuanhan Liu
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	michael.qiu, ann.zhuangyanying, mukawa

Hi Yuanhan,

On 4/22/2016 6:14 AM, Yuanhan Liu wrote:
> On Thu, Apr 21, 2016 at 02:56:36AM +0000, Jianfeng Tan wrote:
>> Add a new virtual device named eth_cvio, it can be used just like
>> eth_ring, eth_null, etc.
>>
>> Configured parameters include:
>>    - rx (optional, 1 by default), number of rx, not used for now.
>>    - tx (optional, 1 by default), number of tx, not used for now.
>>    - cq (optional, 0 by default), not supported for now.
>>    - mac (optional), random value will be given if not specified.
>>    - queue_size (optional, 256 by default), size of virtqueue.
>>    - path (madatory), path of vhost, depends on the file type, vhost
>>      user if the given path points to a unix socket; vhost-net if the
>>      given path points to a char device.
>>    - ifname (optional), specify the name of backend tap device; only
>>      valid when backend is vhost-net.
>>
>> The major difference with original virtio for vm is that, here we use
>> virtual addr instead of physical addr for vhost to calculate relative
>> address.
>>
>> When enable CONFIG_RTE_VIRTIO_VDEV (enabled by default), the compiled
>> library can be used in both VM and container environment.
>>
>> Examples:
>> path_vhost=/dev/vhost-net # use vhost-net as a backend
>> path_vhost=<path_to_vhost_user> # use vhost-user as a backend
>>
>> sudo ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
>>      --socket-mem 0,1024 --no-pci --file-prefix=l2fwd \
>>      --vdev=eth_cvio0,mac=00:01:02:03:04:05,path=$path_vhost -- -p 0x1
>>
>> Known issues:
>>   - Control queue and multi-queue are not supported yet.
>>   - Cannot work with --huge-unlink.
>>   - Cannot work with no-huge.
>>   - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
>>     hugepages.
>>   - Root privilege is a must (mainly becase of sorting hugepages according
>>     to physical address).
>>   - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").
>>
>> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
>> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
>> Acked-By: Neil Horman <nhorman@tuxdrver.com>
>> ---
>>   doc/guides/nics/overview.rst             |  58 +++++-----
>>   doc/guides/rel_notes/release_16_07.rst   |   4 +
>>   drivers/net/virtio/rte_eth_virtio_vdev.c | 188 ++++++++++++++++++++++++++++++-
> Why prefixing it with "rte_eth_..."?

I was to make align with other virtual devices, like ring, null, etc. 
But like suggested by Thomas and you, I'll rename it.

>
>> -   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
>> -   Feature              a b b b c e e e i i i i i i i i i i f f f f m m m n n p r s v v v v x
>> -                        f n n o x 1 n n 4 4 4 4 g g x x x x m m m m l l p f u c i z h i i m e
>> -                        p x x n g 0 a i 0 0 0 0 b b g g g g 1 1 1 1 x x i p l a n e o r r x n
>> -                        a 2 2 d b 0   c e e e e   v b b b b 0 0 0 0 4 5 p   l p g d s t t n v
>> -                        c x x i e 0       . v v   f e e e e k k k k     e         a t i i e i
>> -                        k   v n           . f f       . v v   . v v               t   o o t r
>> -                        e   f g           .   .       . f f   . f f               a     . 3 t
>> +   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
>> +   Feature              a b b b c e e e i i i i i i i i i i f f f f m m m n n p r s v v v v x c
>> +                        f n n o x 1 n n 4 4 4 4 g g x x x x m m m m l l p f u c i z h i i m e v
>> +                        p x x n g 0 a i 0 0 0 0 b b g g g g 1 1 1 1 x x i p l a n e o r r x n i
>> +                        a 2 2 d b 0   c e e e e   v b b b b 0 0 0 0 4 5 p   l p g d s t t n v r
>> +                        c x x i e 0       . v v   f e e e e k k k k     e         a t i i e i t
>> +                        k   v n           . f f       . v v   . v v               t   o o t r i
>> +                        e   f g           .   .       . f f   . f f               a     . 3 t o
>>                           t                 v   v       v   v   v   v               2     v
>
> I would wish we have a diff that could do compare by columns but not by
> rows :)
>
>
>> diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
>> index 68097e6..3b47332 100644
>> --- a/drivers/net/virtio/virtio_pci.h
>> +++ b/drivers/net/virtio/virtio_pci.h
>> @@ -264,7 +264,7 @@ struct virtio_hw {
>>   #define VHOST_KERNEL	0
>>   #define VHOST_USER	1
>>   	int		type; /* type of backend */
>> -	uint32_t	queue_num;
>> +	uint32_t	queue_size;
> Hmm, this kind of change should not be squeezed here, stealthily. I
> would agree that the rename in decreases the stealthily, which is a
> good thing. You should submit a standalone patch instead.

Nice catch. I should do it in the first commit when it's defined.

Thank,
Jianfeng



>
> 	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 1/2] virtio/vdev: add embeded device emulation
  2016-04-22 10:12       ` Tan, Jianfeng
@ 2016-04-22 10:17         ` Thomas Monjalon
  2016-04-22 17:27         ` Yuanhan Liu
  1 sibling, 0 replies; 196+ messages in thread
From: Thomas Monjalon @ 2016-04-22 10:17 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: Yuanhan Liu, dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro,
	p.fedin, michael.qiu, ann.zhuangyanying, mukawa, nhorman

2016-04-22 18:12, Tan, Jianfeng:
> On 4/22/2016 6:01 AM, Yuanhan Liu wrote:
> > Further more, I'd suggest to divide the code into following files:
> >
> > - virtio-user/virtio.c
> >
> >    All virtio device emulation goes here.
> >
> > - virtio-user/vhost-user.c
> >
> >    The vhost-user frontend implementation
> >
> > - virtio-user/vhost-kernel.c
> >
> >    vhost kernel hanldings, including setting the tap device.
> >
> > - And, __maybe__ another standalone file for handling the talk
> >    between the driver and the device. (See more for the comments
> >    about virtio_pci_ops below).
> >
> >
> > That would make it much clearer, IMO.
> 
> Got your point here, but to be honest, I'm a little concerned to split a 
> 1k-lined file into 5+ files. Previously I'd like to make them converged 
> together. But your suggestion is great for clean code. If any other 
> could give some comments?

I wanted to do the same comment: separating code for guest driver,
guest device and host driver will make things easier to understand.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio
  2016-04-22  7:36         ` David Marchand
@ 2016-04-22 10:25           ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-04-22 10:25 UTC (permalink / raw)
  To: David Marchand
  Cc: dev, Huawei Xie, Rich Lane, Yuanhan Liu, Michael S. Tsirkin,
	nakajima.yoshihiro, p.fedin, Qiu, Michael, ann.zhuangyanying,
	Tetsuya Mukawa, nhorman

Hi,

On 4/22/2016 3:36 PM, David Marchand wrote:
> Hello,
>
> On Fri, Apr 22, 2016 at 7:15 AM, Tan, Jianfeng <jianfeng.tan@intel.com> wrote:
>> On 4/21/2016 4:51 PM, David Marchand wrote:
>>> virtio code relies on drv_flags (even updating it while this should be
>>> per-device).
>>> So first, virtio should rely on dev_flags.
>>
>> Mainly drv_flags's RTE_PCI_DRV_INTR_LSC, and RTE_PCI_DRV_DETACHABLE bit is
>> used. I understand the issue, pointed out by you here, that if two virtio
>> devices are used in a VM, one with feature VIRTIO_NET_F_STATUS, and the
>> other without feature VIRTIO_NET_F_STATUS (under the case that two vhost
>> backends are used). Then it leads to uncertainty of the behavior.
>>
>> Since the flags has been copied into dev_flags after features negotiated, I
>> believe we should use dev_flags instead of drv_flags. A patch to fix this
>> will be sent.
> Ok.
>
>>> The rest needs to be astracted in some virtio ops ?
>> So with that fix goes, we may make it more clear now. Do you mean this?
> Well, here, we have what looks like to be two drivers (one pci and one vdev).
> You tried to keep all this code together, to avoid duplicating it,
> which sounds sane.
> But in the end, you are trying to make this work by adding checks
> where this can't.
> I am not saying we should duplicate the code, but maybe having some
> internal virtio ops / abstraction would do the trick and avoid those
> checks.
>
>
> The reason of those comments is that dev_type in ethdev is going to
> disappear, see [1] and [2].
> Drivers are called through their own specific ethdev/crypto ops and
> so, those drivers know implicitely that their are either pci or vdev
> (or whatever in the future) drivers.
>
>
> [1]: http://dpdk.org/ml/archives/dev/2016-April/037686.html
> [2]: http://dpdk.org/ml/archives/dev/2016-January/031390.html

Thank you for this import information. A quick check, we can remove 
those checks, as you said, to virtio ops. Great!

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v3 1/2] virtio/vdev: add embeded device emulation
  2016-04-22 10:12       ` Tan, Jianfeng
  2016-04-22 10:17         ` Thomas Monjalon
@ 2016-04-22 17:27         ` Yuanhan Liu
  1 sibling, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-04-22 17:27 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	michael.qiu, ann.zhuangyanying, mukawa, nhorman, Thomas Monjalon

On Fri, Apr 22, 2016 at 06:12:21PM +0800, Tan, Jianfeng wrote:
> >>How: we implement another instance of struct virtio_pci_ops to intercept
> >>the communications between VM and QEMU. Instead of rd/wr ioport or PCI
> >>configuration space, here we directly talk with backend through the vhost
> >>file.
> >Nope, that's wrong, and here becomes a bit subtle. I will try to make
> >some explanation here.
> >
> >Let's talk about the normal case (with QEMU) first. Where, virtio PMD
> >is a driver, and virito device is emulated inside QEMU, and exposed by
> >PCI. So, virtio PMD talks to the device with ioport rd/wr (or MMIO for
> >virtio 1.0).
> >
> >Till now, you are right.
> >
> >However, vhost-user socket is for establishing a connection, providing
> >HOST with enough information, so that host can directly manipulate the
> >vring, to dequeue/enqueue buffers.
> >
> >So, what you were saying about "directly talk with backend (the virtual
> >virtio device you created) through vhost file" is not right. Instead,
> >in your case, the (virtual) virtio device and the PMD driver is in
> >the same process space, therefore, you could actually access or the
> >device info simply by normal read/write.
> 
> Here by _backend_, I mean vhost. I know you take backend as the device
> emulation.

Yes, because I stated it explicitly (by saying "the virtual device you
created"). And here is why. Let's check your commit log first:

    Instead of rd/wr ioport or PCI configuration space, here we directly
    talk with backend through the vhost file.

Firstly, assume the backend is "vhost-user backend" as you stated, we
are always using vhost file to talk to vhost-user backend, aren't we?

So, I assume you were stating backend as the "virtual device". However, 
here you were comparing two __different__ things. The first half you
were talking about guest and driver communication, the later half you
were saying the communication between vhost-user frontend and backend.

> I ask Huawei, he thinks backend = device emulation + vhost. So

I will not say he is wrong. But they are actually two different thing.
Device emulation is for, as it's name explains, emulating the virtio
device, yet it handles the communication from the driver. Vhost-user
is for implementing the frontend, initiating the negoitation to the
vhost-user backend.

See the difference? It might be subtle somehow, but the difference
is obvious. You should seperate them.

Now let's look at some of your code:

    +static uint64_t
    +vdev_get_features(struct virtio_hw *hw)
    +{
    +       uint64_t host_features;
    +
    +       vhost_call(hw, VHOST_MSG_GET_FEATURES, &host_features);
    +       if (hw->mac_specified)
    +               host_features |= (1ull << VIRTIO_NET_F_MAC);
    +       /* disable it until we support CQ */
    +       host_features &= ~(1ull << VIRTIO_NET_F_CTRL_VQ);
    +       host_features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
    +       return host_features;
    +}


The get_features method is actually for getting the features supported
by the __device__. Therefore, you should maintain a standalone feature
list in your virtual device implementation (like what QEMU does). However,
here you directly fetch the features from the vhost-user backend.

One more example:

    +static void
    +vdev_set_features(struct virtio_hw *hw __rte_unused,
    +                 uint64_t features __rte_unused)
    +{
    +       /* We hold to send this message, VHOST_MSG_SET_FEATURES,
    +        * until all VirtQueues have been kicked.
    +        */
    +}

I know why you were doing that, you also made a comment. The fact of
the matter is that you should not do that (send vhost-user request).
"set_features" method is for setting the negoiated features from driver
and device, and then store it in virtual device. You should not touch
vhost-user backend here.

So, sorry, you are actually doing what you stated: doing device
configuration through vhost-user interface. But that's a wrong
way to make things work.

The right way to go is to maintain a struct inside the virtual device,
to keep all related states, such as features, status. And do vhost-user
negotiation just only when VIRTIO_CONFIG_S_DRIVER_OK is seen.

In this way, you could keep the logic separately. That's one of the
reason why I ask you to separate them in different files.

> maybe I should use the phrase vhost for accuracy.
> 
> >
> >As you can see, It's a bit messy to mix all of them (virtio PMD driver,
> >virtio device emulation, and vhost-uesr frontend) in one single directory
> >(or even in one single file as you did). Therefore, I'd suggest you to
> >make a new dir, say "virtio-user" (a good name from Thomas), and put all
> >files related to virtio device emulation and vhost-user frontend there.
> >
> >Further more, I'd suggest to divide the code into following files:
> >
> >- virtio-user/virtio.c
> >
> >   All virtio device emulation goes here.
> >
> >- virtio-user/vhost-user.c
> >
> >   The vhost-user frontend implementation
> >
> >- virtio-user/vhost-kernel.c
> >
> >   vhost kernel hanldings, including setting the tap device.
> >
> >- And, __maybe__ another standalone file for handling the talk
> >   between the driver and the device. (See more for the comments
> >   about virtio_pci_ops below).
> >
> >
> >That would make it much clearer, IMO.
> 
> Got your point here, but to be honest, I'm a little concerned to split a
> 1k-lined file into 5+ files.

What's wrong doing that? Does it hurt anything? Nope, instead, I see
many benefits while doing that: clean code/logic/design, easier for
review, easier for maintain, and so on...

I just see it hurts a lot when not doing that. Don't even to you say
you are going to add more stuff based on that, say control queue.


> Previously I'd like to make them converged
> together. But your suggestion is great for clean code. If any other could
> give some comments?

Hmmmm, since you were admitting it's "great for clean code", I'm just
wondering why this is still not enough to convince you???

It's more than making things work; it's more important to make things
work in the right/clean way.

> >>+	bzero(&msgh, sizeof(msgh));
> >>+	bzero(control, sizeof(control));
> >bzero is marked as deprecated (see the man page), use memset instead.
> 
> I should apologize about this, you actually told me once before, but I
> failed to fix it in this version.

Never mind. It's been for a while, we (as human) just forgot things.

> >>diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
> >>index b69785e..68097e6 100644
> >>--- a/drivers/net/virtio/virtio_pci.h
> >>+++ b/drivers/net/virtio/virtio_pci.h
> >>@@ -260,6 +260,20 @@ struct virtio_hw {
> >>  	struct virtio_pci_common_cfg *common_cfg;
> >>  	struct virtio_net_config *dev_cfg;
> >>  	const struct virtio_pci_ops *vtpci_ops;
> >>+#ifdef RTE_VIRTIO_VDEV
> >>+#define VHOST_KERNEL	0
> >>+#define VHOST_USER	1
> >>+	int		type; /* type of backend */
> >>+	uint32_t	queue_num;
> >>+	char		*path;
> >>+	int		mac_specified;
> >>+	int		vhostfd;
> >>+	int		backfd; /* tap device used in vhost-net */
> >>+	int		callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
> >>+	int		kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
> >>+	uint8_t		status;
> >>+	struct rte_eth_dev_data *data;
> >>+#endif
> >And put all of them to the virtio "device" context, in the virtio-user/
> >then.
> 
> OK, I'll define a new struct to store these fields. I consider there are two
> methods to refer these data. One is to store a pointer to the instance of
> the new struct in the struct virtio_hw; the other is to main them totally
> closed inside virtio-user, such as using a linked chain. Which do you
> prefer?

Let's try the first one first; it's simpler, and let's see how well it
might work. If not, we could try others.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v4 0/8] virtio support for container
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
                   ` (8 preceding siblings ...)
  2016-04-21  2:56 ` [PATCH v3 0/2] " Jianfeng Tan
@ 2016-04-29  1:18 ` Jianfeng Tan
  2016-04-29  1:18   ` [PATCH v4 1/8] virtio: hide phys addr check inside pci ops Jianfeng Tan
                     ` (8 more replies)
  2016-05-30 10:55 ` [PATCH v5 " Jianfeng Tan
                   ` (4 subsequent siblings)
  14 siblings, 9 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-04-29  1:18 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, michael.qiu, ann.zhuangyanying,
	mukawa, nhorman

v4:
 - Avoid using dev_type, instead use (eth_dev->pci_device is NULL) to
   judge if it's virtual device or physical device.
 - Change the added device name to virtio-user.
 - Split into vhost_user.c, vhost_kernel.c, vhost.c, virtio_user_pci.c,
   virtio_user_dev.c.
 - Move virtio-user specific data from struct virtio_hw into struct
   virtio_user_hw.
 - Add support to send reset_owner message.
 - Change del_queue implementation. (This need more check)
 - Remove rte_panic(), and superseded with log.
 - Add reset_owner into virtio_pci_ops.reset.
 - Merge parameter "rx" and "tx" to "queues" to emliminate confusion.
 - Move get_features to after set_owner.
 - Redefine path in virtio_user_hw from char * to char [].

v3:
 - Remove --single-file option; do no change at EAL memory.
 - Remove the added API rte_eal_get_backfile_info(), instead we check all
   opened files with HUGEFILE_FMT to find hugepage files owned by DPDK.
 - Accordingly, add more restrictions at "Known issue" section.
 - Rename parameter from queue_num to queue_size for confusion.
 - Rename vhost_embedded.c to rte_eth_virtio_vdev.c.
 - Move code related to the newly added vdev to rte_eth_virtio_vdev.c, to
   reuse eth_virtio_dev_init(), remove its static declaration.
 - Implement dev_uninit() for rte_eth_dev_detach().
 - WARN -> ERR, in vhost_embedded.c
 - Add more commit message for clarify the model.

v2:
 - Rebase on the patchset of virtio 1.0 support.
 - Fix cannot create non-hugepage memory.
 - Fix wrong size of memory region when "single-file" is used.
 - Fix setting of offset in virtqueue to use virtual address.
 - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
 - Add mac option to specify the mac address of this virtual device.
 - Update doc.

This patchset is to provide high performance networking interface (virtio)
for container-based DPDK applications. The way of starting DPDK apps in
containers with ownership of NIC devices exclusively is beyond the scope.
The basic idea here is to present a new virtual device (named virtio-user),
which can be discovered and initialized by DPDK. To minimize the change,
we reuse already-existing virtio PMD code (driver/net/virtio/).

Background: Previously, we usually use a virtio device in the context of
QEMU/VM as below pic shows. Virtio nic is emulated in QEMU, and usually
presented in VM as a PCI device.

  ------------------
  |  virtio driver |  ----->  VM
  ------------------
        |
        | ----------> (over PCI bus or MMIO or Channel I/O)
        |
  ------------------
  | device emulate |
  |                |  ----->  QEMU
  | vhost adapter  |
  ------------------
        |
        | ----------> (vhost-user protocol or vhost-net ioctls)
        |
  ------------------
  | vhost backend  |
  ------------------
 
Compared to QEMU/VM case, virtio support for contaner requires to embedded
device framework inside the virtio PMD. So this converged driver actually
plays three roles:
  - virtio driver to drive this new kind of virtual device;
  - device emulation to present this virtual device and reponse to the
    virtio driver, which is originally by QEMU;
  - and the role to communicate with vhost backend, which is also
    originally by QEMU.

The code layout and functionality of each module:
 
  ----------------------
  | ------------------ |
  | | virtio driver  | |----> (virtio_user_pci.c)
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate |-|----> (virtio_user_dev.c)
  | |                | |
  | | vhost adapter  |-|----> (vhost_user.c, vhost_kernel.c, vhost.c)
  | ------------------ |
  ----------------------
         |
         | -------------- --> (vhost-user protocol or vhost-net ioctls)
         |
   ------------------
   | vhost backend  |
   ------------------

How to share memory? In VM's case, qemu always shares all physical layout
to backend. But it's not feasible for a container, as a process, to share
all virtual memory regions to backend. So only specified virtual memory
regions (with type of shared) are sent to backend. It's a limitation that
only addresses in these areas can be used to transmit or receive packets.

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").

How to use?

a. Apply this patchset.

b. To compile container apps:
$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc

c. To build a docker image using Dockerfile below.
$: cat ./Dockerfile
FROM ubuntu:latest
WORKDIR /usr/src/dpdk
COPY . /usr/src/dpdk
ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
$: docker build -t dpdk-app-l2fwd .

d. Used with vhost-user
$: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
	--socket-mem 1024,1024 -- -p 0x1 --stats 1
$: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=virtio-user0,path=/var/run/usvhost -- -p 0x1

f. Used with vhost-net
$: modprobe vhost
$: modprobe vhost-net
$: docker run -i -t --privileged \
	-v /dev/vhost-net:/dev/vhost-net \
	-v /dev/net/tun:/dev/net/tun \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=virtio-user0,path=/dev/vhost-net -- -p 0x1

By the way, it's not necessary to run in a container.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-By: Neil Horman <nhorman@tuxdriver.com>

Jianfeng Tan (8):
  virtio: hide phys addr check inside pci ops
  virtio: abstract vring hdr desc init as a method
  virtio: enable use virtual address to fill desc
  virtio-user: add vhost adapter layer
  virtio-user: add device emulation layer APIs
  virtio-user: add new virtual pci driver for virtio
  virtio-user: add a new virtual device named virtio-user
  doc: update doc for virtio-user

 config/common_linuxapp                           |   3 +
 doc/guides/nics/overview.rst                     |  64 +--
 doc/guides/rel_notes/release_16_07.rst           |   4 +
 drivers/net/virtio/Makefile                      |   8 +
 drivers/net/virtio/virtio_ethdev.c               |  69 ++--
 drivers/net/virtio/virtio_ethdev.h               |   2 +
 drivers/net/virtio/virtio_pci.c                  |  30 +-
 drivers/net/virtio/virtio_pci.h                  |   3 +-
 drivers/net/virtio/virtio_rxtx.c                 |   5 +-
 drivers/net/virtio/virtio_rxtx_simple.c          |  13 +-
 drivers/net/virtio/virtio_user/vhost.c           | 105 +++++
 drivers/net/virtio/virtio_user/vhost.h           | 221 +++++++++++
 drivers/net/virtio/virtio_user/vhost_kernel.c    | 254 ++++++++++++
 drivers/net/virtio/virtio_user/vhost_user.c      | 375 ++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 475 +++++++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.h |  61 +++
 drivers/net/virtio/virtio_user/virtio_user_pci.c | 209 ++++++++++
 drivers/net/virtio/virtqueue.h                   |  33 +-
 18 files changed, 1849 insertions(+), 85 deletions(-)
 create mode 100644 drivers/net/virtio/virtio_user/vhost.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost.h
 create mode 100644 drivers/net/virtio/virtio_user/vhost_kernel.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_pci.c

-- 
2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v4 1/8] virtio: hide phys addr check inside pci ops
  2016-04-29  1:18 ` [PATCH v4 0/8] virtio support for container Jianfeng Tan
@ 2016-04-29  1:18   ` Jianfeng Tan
  2016-05-11 23:05     ` Yuanhan Liu
  2016-04-29  1:18   ` [PATCH v4 2/8] virtio: abstract vring hdr desc init as a method Jianfeng Tan
                     ` (7 subsequent siblings)
  8 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-04-29  1:18 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, michael.qiu, ann.zhuangyanying,
	mukawa, nhorman

This patch is to move phys addr check from virtio_dev_queue_setup
to pci ops. To makt that happen, make sure virtio_ops.setup_queue
return the result if we pass through the check.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-By: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/virtio_ethdev.c | 17 +++++------------
 drivers/net/virtio/virtio_pci.c    | 30 ++++++++++++++++++++++++++++--
 drivers/net/virtio/virtio_pci.h    |  2 +-
 3 files changed, 34 insertions(+), 15 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index bd990ff..534f0e6 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -369,17 +369,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		}
 	}
 
-	/*
-	 * Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
-	 * and only accepts 32 bit page frame number.
-	 * Check if the allocated physical memory exceeds 16TB.
-	 */
-	if ((mz->phys_addr + vq->vq_ring_size - 1) >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
-		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
-		virtio_dev_queue_release(vq);
-		return -ENOMEM;
-	}
-
 	memset(mz->addr, 0, sizeof(mz->len));
 	vq->mz = mz;
 	vq->vq_ring_mem = mz->phys_addr;
@@ -451,7 +440,11 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		memset(vq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
 	}
 
-	hw->vtpci_ops->setup_queue(hw, vq);
+	if (hw->vtpci_ops->setup_queue(hw, vq) < 0) {
+		PMD_INIT_LOG(ERR, "setup_queue failed");
+		virtio_dev_queue_release(vq);
+		return -EINVAL;
+	}
 
 	vq->started = 1;
 	*pvq = vq;
diff --git a/drivers/net/virtio/virtio_pci.c b/drivers/net/virtio/virtio_pci.c
index 9cdca06..6bd239c 100644
--- a/drivers/net/virtio/virtio_pci.c
+++ b/drivers/net/virtio/virtio_pci.c
@@ -55,6 +55,22 @@
  */
 #define VIRTIO_PCI_CONFIG(hw) (((hw)->use_msix) ? 24 : 20)
 
+static inline int
+check_vq_phys_addr_ok(struct virtqueue *vq)
+{
+	/* Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
+	 * and only accepts 32 bit page frame number.
+	 * Check if the allocated physical memory exceeds 16TB.
+	 */
+	if ((vq->vq_ring_mem + vq->vq_ring_size - 1) >>
+			(VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
+		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
+		return 0;
+	}
+
+	return 1;
+}
+
 static void
 legacy_read_dev_config(struct virtio_hw *hw, size_t offset,
 		       void *dst, int length)
@@ -143,15 +159,20 @@ legacy_get_queue_num(struct virtio_hw *hw, uint16_t queue_id)
 	return dst;
 }
 
-static void
+static int
 legacy_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 {
 	uint32_t src;
 
+	if (!check_vq_phys_addr_ok(vq))
+		return -1;
+
 	rte_eal_pci_ioport_write(&hw->io, &vq->vq_queue_index, 2,
 			 VIRTIO_PCI_QUEUE_SEL);
 	src = vq->mz->phys_addr >> VIRTIO_PCI_QUEUE_ADDR_SHIFT;
 	rte_eal_pci_ioport_write(&hw->io, &src, 4, VIRTIO_PCI_QUEUE_PFN);
+
+	return 0;
 }
 
 static void
@@ -367,12 +388,15 @@ modern_get_queue_num(struct virtio_hw *hw, uint16_t queue_id)
 	return io_read16(&hw->common_cfg->queue_size);
 }
 
-static void
+static int
 modern_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 {
 	uint64_t desc_addr, avail_addr, used_addr;
 	uint16_t notify_off;
 
+	if (!check_vq_phys_addr_ok(vq))
+		return -1;
+
 	desc_addr = vq->mz->phys_addr;
 	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
 	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
@@ -400,6 +424,8 @@ modern_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 	PMD_INIT_LOG(DEBUG, "\t used_addr: %" PRIx64, used_addr);
 	PMD_INIT_LOG(DEBUG, "\t notify addr: %p (notify offset: %u)",
 		vq->notify_addr, notify_off);
+
+	return 0;
 }
 
 static void
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index 554efea..a76daf7 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -234,7 +234,7 @@ struct virtio_pci_ops {
 	uint16_t (*set_config_irq)(struct virtio_hw *hw, uint16_t vec);
 
 	uint16_t (*get_queue_num)(struct virtio_hw *hw, uint16_t queue_id);
-	void (*setup_queue)(struct virtio_hw *hw, struct virtqueue *vq);
+	int (*setup_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 	void (*del_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 	void (*notify_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 };
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v4 2/8] virtio: abstract vring hdr desc init as a method
  2016-04-29  1:18 ` [PATCH v4 0/8] virtio support for container Jianfeng Tan
  2016-04-29  1:18   ` [PATCH v4 1/8] virtio: hide phys addr check inside pci ops Jianfeng Tan
@ 2016-04-29  1:18   ` Jianfeng Tan
  2016-04-29  1:18   ` [PATCH v4 3/8] virtio: enable use virtual address to fill desc Jianfeng Tan
                     ` (6 subsequent siblings)
  8 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-04-29  1:18 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, michael.qiu, ann.zhuangyanying,
	mukawa, nhorman

To make it reusable, here we abstract the initialization of vring
header into a method.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-By: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/virtio_ethdev.c | 22 ++++------------------
 drivers/net/virtio/virtqueue.h     | 20 ++++++++++++++++++++
 2 files changed, 24 insertions(+), 18 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 534f0e6..0c20fb9 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -380,8 +380,7 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 
 	if (queue_type == VTNET_TQ) {
 		const struct rte_memzone *hdr_mz;
-		struct virtio_tx_region *txr;
-		unsigned int i;
+		size_t hdr_mz_sz = vq_size * sizeof(struct virtio_tx_region);
 
 		/*
 		 * For each xmit packet, allocate a virtio_net_hdr
@@ -390,7 +389,7 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		snprintf(vq_name, sizeof(vq_name), "port%d_tvq%d_hdrzone",
 			 dev->data->port_id, queue_idx);
 		hdr_mz = rte_memzone_reserve_aligned(vq_name,
-						     vq_size * sizeof(*txr),
+						     hdr_mz_sz,
 						     socket_id, 0,
 						     RTE_CACHE_LINE_SIZE);
 		if (hdr_mz == NULL) {
@@ -404,21 +403,8 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		vq->virtio_net_hdr_mz = hdr_mz;
 		vq->virtio_net_hdr_mem = hdr_mz->phys_addr;
 
-		txr = hdr_mz->addr;
-		memset(txr, 0, vq_size * sizeof(*txr));
-		for (i = 0; i < vq_size; i++) {
-			struct vring_desc *start_dp = txr[i].tx_indir;
-
-			vring_desc_init(start_dp, RTE_DIM(txr[i].tx_indir));
-
-			/* first indirect descriptor is always the tx header */
-			start_dp->addr = vq->virtio_net_hdr_mem
-				+ i * sizeof(*txr)
-				+ offsetof(struct virtio_tx_region, tx_hdr);
-
-			start_dp->len = vq->hw->vtnet_hdr_size;
-			start_dp->flags = VRING_DESC_F_NEXT;
-		}
+		memset(hdr_mz->addr, 0, hdr_mz_sz);
+		vring_hdr_desc_init(vq);
 
 	} else if (queue_type == VTNET_CQ) {
 		/* Allocate a page for control vq command, data and status */
diff --git a/drivers/net/virtio/virtqueue.h b/drivers/net/virtio/virtqueue.h
index 83d89ca..3b19fd1 100644
--- a/drivers/net/virtio/virtqueue.h
+++ b/drivers/net/virtio/virtqueue.h
@@ -264,6 +264,26 @@ vring_desc_init(struct vring_desc *dp, uint16_t n)
 	dp[i].next = VQ_RING_DESC_CHAIN_END;
 }
 
+static inline void
+vring_hdr_desc_init(struct virtqueue *vq)
+{
+	int i;
+	struct virtio_tx_region *txr = vq->virtio_net_hdr_mz->addr;
+
+	for (i = 0; i < vq->vq_nentries; i++) {
+		struct vring_desc *start_dp = txr[i].tx_indir;
+
+		vring_desc_init(start_dp, RTE_DIM(txr[i].tx_indir));
+
+		/* first indirect descriptor is always the tx header */
+		start_dp->addr = vq->virtio_net_hdr_mem + i * sizeof(*txr) +
+				 offsetof(struct virtio_tx_region, tx_hdr);
+
+		start_dp->len = vq->hw->vtnet_hdr_size;
+		start_dp->flags = VRING_DESC_F_NEXT;
+	}
+}
+
 /**
  * Tell the backend not to interrupt us.
  */
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v4 3/8] virtio: enable use virtual address to fill desc
  2016-04-29  1:18 ` [PATCH v4 0/8] virtio support for container Jianfeng Tan
  2016-04-29  1:18   ` [PATCH v4 1/8] virtio: hide phys addr check inside pci ops Jianfeng Tan
  2016-04-29  1:18   ` [PATCH v4 2/8] virtio: abstract vring hdr desc init as a method Jianfeng Tan
@ 2016-04-29  1:18   ` Jianfeng Tan
  2016-04-29  1:18   ` [PATCH v4 4/8] virtio-user: add vhost adapter layer Jianfeng Tan
                     ` (5 subsequent siblings)
  8 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-04-29  1:18 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, michael.qiu, ann.zhuangyanying,
	mukawa, nhorman

This patch is related to how to calculate relative address for vhost
backend.

The principle is that: based on one or multiple shared memory segments,
vhost maintains a reference system with the base addresses and length
for each segment so that an address from VM comes (usually GPA, Guest
Physical Address) can be translated into vhost-recognizable address
(named VVA, Vhost Virtual Address). In VM's case, GPA is always locally
continuous. But for some other case, like virtio-user, virtual address
can be used.

It basically means:
  a. when set_base_addr, VA address is used;
  b. when preparing RX's descriptors, VA address is used;
  c. when transmitting packets, VA is filled in TX's descriptors;
  d. in TX and CQ's header, VA is used.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-By: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/virtio_ethdev.c      | 11 ++++++++---
 drivers/net/virtio/virtio_rxtx.c        |  5 ++---
 drivers/net/virtio/virtio_rxtx_simple.c | 13 +++++++------
 drivers/net/virtio/virtqueue.h          | 13 ++++++++++++-
 4 files changed, 29 insertions(+), 13 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 0c20fb9..16b324d 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -167,14 +167,14 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	 * One RX packet for ACK.
 	 */
 	vq->vq_ring.desc[head].flags = VRING_DESC_F_NEXT;
-	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mz->phys_addr;
+	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mem;
 	vq->vq_ring.desc[head].len = sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_free_cnt--;
 	i = vq->vq_ring.desc[head].next;
 
 	for (k = 0; k < pkt_num; k++) {
 		vq->vq_ring.desc[i].flags = VRING_DESC_F_NEXT;
-		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr)
 			+ sizeof(ctrl->status) + sizeof(uint8_t)*sum;
 		vq->vq_ring.desc[i].len = dlen[k];
@@ -184,7 +184,7 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	}
 
 	vq->vq_ring.desc[i].flags = VRING_DESC_F_WRITE;
-	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_ring.desc[i].len = sizeof(ctrl->status);
 	vq->vq_free_cnt--;
@@ -426,6 +426,11 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		memset(vq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
 	}
 
+	/* Use physical address to fill desc.addr by default,
+	 * and will be changed to use virtual address for vdev.
+	 */
+	vq->offset = offsetof(struct rte_mbuf, buf_physaddr);
+
 	if (hw->vtpci_ops->setup_queue(hw, vq) < 0) {
 		PMD_INIT_LOG(ERR, "setup_queue failed");
 		virtio_dev_queue_release(vq);
diff --git a/drivers/net/virtio/virtio_rxtx.c b/drivers/net/virtio/virtio_rxtx.c
index ef21d8e..9d7e537 100644
--- a/drivers/net/virtio/virtio_rxtx.c
+++ b/drivers/net/virtio/virtio_rxtx.c
@@ -193,8 +193,7 @@ virtqueue_enqueue_recv_refill(struct virtqueue *vq, struct rte_mbuf *cookie)
 
 	start_dp = vq->vq_ring.desc;
 	start_dp[idx].addr =
-		(uint64_t)(cookie->buf_physaddr + RTE_PKTMBUF_HEADROOM
-		- hw->vtnet_hdr_size);
+		MBUF_DATA_DMA_ADDR(cookie, vq->offset) - hw->vtnet_hdr_size;
 	start_dp[idx].len =
 		cookie->buf_len - RTE_PKTMBUF_HEADROOM + hw->vtnet_hdr_size;
 	start_dp[idx].flags =  VRING_DESC_F_WRITE;
@@ -265,7 +264,7 @@ virtqueue_enqueue_xmit(struct virtqueue *txvq, struct rte_mbuf *cookie,
 	}
 
 	do {
-		start_dp[idx].addr  = rte_mbuf_data_dma_addr(cookie);
+		start_dp[idx].addr  = MBUF_DATA_DMA_ADDR(cookie, txvq->offset);
 		start_dp[idx].len   = cookie->data_len;
 		start_dp[idx].flags = cookie->next ? VRING_DESC_F_NEXT : 0;
 		idx = start_dp[idx].next;
diff --git a/drivers/net/virtio/virtio_rxtx_simple.c b/drivers/net/virtio/virtio_rxtx_simple.c
index 8f5293d..83a794e 100644
--- a/drivers/net/virtio/virtio_rxtx_simple.c
+++ b/drivers/net/virtio/virtio_rxtx_simple.c
@@ -80,8 +80,8 @@ virtqueue_enqueue_recv_refill_simple(struct virtqueue *vq,
 	vq->sw_ring[desc_idx] = cookie;
 
 	start_dp = vq->vq_ring.desc;
-	start_dp[desc_idx].addr = (uint64_t)((uintptr_t)cookie->buf_physaddr +
-		RTE_PKTMBUF_HEADROOM - vq->hw->vtnet_hdr_size);
+	start_dp[desc_idx].addr = MBUF_DATA_DMA_ADDR(cookie, vq->offset) -
+				  vq->hw->vtnet_hdr_size;
 	start_dp[desc_idx].len = cookie->buf_len -
 		RTE_PKTMBUF_HEADROOM + vq->hw->vtnet_hdr_size;
 
@@ -119,8 +119,8 @@ virtio_rxq_rearm_vec(struct virtqueue *rxvq)
 		*(uint64_t *)p = rxvq->mbuf_initializer;
 
 		start_dp[i].addr =
-			(uint64_t)((uintptr_t)sw_ring[i]->buf_physaddr +
-			RTE_PKTMBUF_HEADROOM - rxvq->hw->vtnet_hdr_size);
+			MBUF_DATA_DMA_ADDR(sw_ring[i], rxvq->offset) -
+			rxvq->hw->vtnet_hdr_size;
 		start_dp[i].len = sw_ring[i]->buf_len -
 			RTE_PKTMBUF_HEADROOM + rxvq->hw->vtnet_hdr_size;
 	}
@@ -366,7 +366,7 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 			txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 		for (i = 0; i < nb_tail; i++) {
 			start_dp[desc_idx].addr =
-				rte_mbuf_data_dma_addr(*tx_pkts);
+				MBUF_DATA_DMA_ADDR(*tx_pkts, txvq->offset);
 			start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 			tx_pkts++;
 			desc_idx++;
@@ -377,7 +377,8 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 	for (i = 0; i < nb_commit; i++)
 		txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 	for (i = 0; i < nb_commit; i++) {
-		start_dp[desc_idx].addr = rte_mbuf_data_dma_addr(*tx_pkts);
+		start_dp[desc_idx].addr =
+			MBUF_DATA_DMA_ADDR(*tx_pkts, txvq->offset);
 		start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 		tx_pkts++;
 		desc_idx++;
diff --git a/drivers/net/virtio/virtqueue.h b/drivers/net/virtio/virtqueue.h
index 3b19fd1..07e41b9 100644
--- a/drivers/net/virtio/virtqueue.h
+++ b/drivers/net/virtio/virtqueue.h
@@ -66,6 +66,14 @@ struct rte_mbuf;
 
 #define VIRTQUEUE_MAX_NAME_SZ 32
 
+#ifdef RTE_VIRTIO_VDEV
+#define MBUF_DATA_DMA_ADDR(mb, offset) \
+	((uint64_t)((uintptr_t)(*(void **)((uintptr_t)mb + offset)) \
+			+ (mb)->data_off))
+#else /* RTE_VIRTIO_VDEV */
+#define MBUF_DATA_DMA_ADDR(mb, offset) rte_mbuf_data_dma_addr(mb)
+#endif /* RTE_VIRTIO_VDEV */
+
 #define VTNET_SQ_RQ_QUEUE_IDX 0
 #define VTNET_SQ_TQ_QUEUE_IDX 1
 #define VTNET_SQ_CQ_QUEUE_IDX 2
@@ -165,6 +173,7 @@ struct virtqueue {
 	void        *vq_ring_virt_mem;    /**< linear address of vring*/
 	unsigned int vq_ring_size;
 	phys_addr_t vq_ring_mem;          /**< physical address of vring */
+					  /**< use virtual address for vdev. */
 
 	struct vring vq_ring;    /**< vring keeping desc, used and avail */
 	uint16_t    vq_free_cnt; /**< num of desc available */
@@ -183,8 +192,10 @@ struct virtqueue {
 	 */
 	uint16_t vq_used_cons_idx;
 	uint16_t vq_avail_idx;
+	uint16_t offset; /**< relative offset to obtain addr in mbuf */
 	uint64_t mbuf_initializer; /**< value to init mbufs. */
-	phys_addr_t virtio_net_hdr_mem; /**< hdr for each xmit packet */
+	phys_addr_t virtio_net_hdr_mem; /**< phys addr of hdr memzone */
+					/**< use virtual address for vdev */
 
 	struct rte_mbuf **sw_ring; /**< RX software ring. */
 	/* dummy mbuf, for wraparound when processing RX ring. */
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v4 4/8] virtio-user: add vhost adapter layer
  2016-04-29  1:18 ` [PATCH v4 0/8] virtio support for container Jianfeng Tan
                     ` (2 preceding siblings ...)
  2016-04-29  1:18   ` [PATCH v4 3/8] virtio: enable use virtual address to fill desc Jianfeng Tan
@ 2016-04-29  1:18   ` Jianfeng Tan
  2016-04-29  1:18   ` [PATCH v4 5/8] virtio-user: add device emulation layer APIs Jianfeng Tan
                     ` (4 subsequent siblings)
  8 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-04-29  1:18 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, michael.qiu, ann.zhuangyanying,
	mukawa, nhorman

This patch is to provide vhost adapter layer implementations. Instead
of relying on a hypervisor to translate between device emulation and
vhost backend, here we directly talk with vhost backend through the
vhost file. Depending on the type of vhost file,
  - vhost-user is used if the given path points to a unix socket;
  - vhost-kernel is used if the given path points to a char device.

Here three main APIs are provided to upper layer (device emulation):
  - vhost_user_setup(), to set up env to talk to a vhost user backend;
  - vhost_kernel_setup(), to set up env to talk to a vhost kernel backend.
  - vhost_call(), to provide a unified interface to communicate with
    vhost backend.

  ----------------------
  | ------------------ |
  | | virtio driver  | |
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate | |
  | |                | |
  | | vhost adapter  |-|----> (vhost_user.c, vhost_kernel.c, vhost.c)
  | ------------------ |
  ----------------------
            |
            | -------------- --> (vhost-user protocol or vhost-net ioctls)
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-By: Neil Horman <nhorman@tuxdriver.com>
---
 config/common_linuxapp                        |   3 +
 drivers/net/virtio/Makefile                   |   6 +
 drivers/net/virtio/virtio_pci.h               |   1 +
 drivers/net/virtio/virtio_user/vhost.c        | 105 ++++++++
 drivers/net/virtio/virtio_user/vhost.h        | 221 +++++++++++++++
 drivers/net/virtio/virtio_user/vhost_kernel.c | 254 +++++++++++++++++
 drivers/net/virtio/virtio_user/vhost_user.c   | 375 ++++++++++++++++++++++++++
 7 files changed, 965 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/vhost.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost.h
 create mode 100644 drivers/net/virtio/virtio_user/vhost_kernel.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c

diff --git a/config/common_linuxapp b/config/common_linuxapp
index 7e698e2..946a6d4 100644
--- a/config/common_linuxapp
+++ b/config/common_linuxapp
@@ -43,3 +43,6 @@ CONFIG_RTE_LIBRTE_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_AF_PACKET=y
 CONFIG_RTE_LIBRTE_POWER=y
+
+# Enable virtio-user
+CONFIG_RTE_VIRTIO_VDEV=y
diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index ef84f60..c9f2bc0 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -55,6 +55,12 @@ ifeq ($(findstring RTE_MACHINE_CPUFLAG_SSSE3,$(CFLAGS)),RTE_MACHINE_CPUFLAG_SSSE
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx_simple.c
 endif
 
+ifeq ($(CONFIG_RTE_VIRTIO_VDEV),y)
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_kernel.c
+endif
+
 # this lib depends upon:
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_eal lib/librte_ether
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_mempool lib/librte_mbuf
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index a76daf7..b9f1ee5 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -260,6 +260,7 @@ struct virtio_hw {
 	struct virtio_pci_common_cfg *common_cfg;
 	struct virtio_net_config *dev_cfg;
 	const struct virtio_pci_ops *vtpci_ops;
+	void *vdev_private;
 };
 
 /*
diff --git a/drivers/net/virtio/virtio_user/vhost.c b/drivers/net/virtio/virtio_user/vhost.c
new file mode 100644
index 0000000..ff76658
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost.c
@@ -0,0 +1,105 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include "vhost.h"
+
+static const char * const vhost_msg_strings[] = {
+	[VHOST_MSG_SET_OWNER] = "VHOST_MSG_SET_OWNER",
+	[VHOST_MSG_RESET_OWNER] = "VHOST_MSG_RESET_OWNER",
+	[VHOST_MSG_SET_FEATURES] = "VHOST_MSG_SET_FEATURES",
+	[VHOST_MSG_GET_FEATURES] = "VHOST_MSG_GET_FEATURES",
+	[VHOST_MSG_SET_VRING_CALL] = "VHOST_MSG_SET_VRING_CALL",
+	[VHOST_MSG_SET_VRING_NUM] = "VHOST_MSG_SET_VRING_NUM",
+	[VHOST_MSG_SET_VRING_BASE] = "VHOST_MSG_SET_VRING_BASE",
+	[VHOST_MSG_GET_VRING_BASE] = "VHOST_MSG_GET_VRING_BASE",
+	[VHOST_MSG_SET_VRING_ADDR] = "VHOST_MSG_SET_VRING_ADDR",
+	[VHOST_MSG_SET_VRING_KICK] = "VHOST_MSG_SET_VRING_KICK",
+	[VHOST_MSG_SET_MEM_TABLE] = "VHOST_MSG_SET_MEM_TABLE",
+	NULL,
+};
+
+static unsigned long int vhost_req_map[][2] = {
+	[VHOST_MSG_SET_OWNER] = {
+		VHOST_SET_OWNER, VHOST_USER_SET_OWNER
+	},
+	[VHOST_MSG_RESET_OWNER] = {
+		VHOST_RESET_OWNER, VHOST_USER_RESET_OWNER
+	},
+	[VHOST_MSG_SET_FEATURES] = {
+		VHOST_SET_FEATURES, VHOST_USER_SET_FEATURES
+	},
+	[VHOST_MSG_GET_FEATURES] = {
+		VHOST_GET_FEATURES, VHOST_USER_GET_FEATURES
+	},
+	[VHOST_MSG_SET_VRING_CALL] = {
+		VHOST_SET_VRING_CALL, VHOST_USER_SET_VRING_CALL
+	},
+	[VHOST_MSG_SET_VRING_NUM] = {
+		VHOST_SET_VRING_NUM, VHOST_USER_SET_VRING_NUM
+	},
+	[VHOST_MSG_SET_VRING_BASE] = {
+		VHOST_SET_VRING_BASE, VHOST_USER_SET_VRING_BASE
+	},
+	[VHOST_MSG_GET_VRING_BASE] = {
+		VHOST_GET_VRING_BASE, VHOST_USER_GET_VRING_BASE
+	},
+	[VHOST_MSG_SET_VRING_ADDR] = {
+		VHOST_SET_VRING_ADDR, VHOST_USER_SET_VRING_ADDR
+	},
+	[VHOST_MSG_SET_VRING_KICK] = {
+		VHOST_SET_VRING_KICK, VHOST_USER_SET_VRING_KICK
+	},
+	[VHOST_MSG_SET_MEM_TABLE] = {
+		VHOST_SET_MEM_TABLE, VHOST_USER_SET_MEM_TABLE
+	},
+};
+
+int
+vhost_call(int vhostfd, unsigned int type, unsigned long int req, void *arg)
+{
+	unsigned long int req_new;
+	int ret;
+
+	PMD_DRV_LOG(INFO, "%s", vhost_msg_strings[req]);
+
+	req_new = vhost_req_map[req][type];
+	if (type == VHOST_USER)
+		ret = vhost_user_sock(vhostfd, req_new, arg);
+	else
+		ret = vhost_kernel_ioctl(vhostfd, req_new, arg);
+
+	if (ret < 0)
+		PMD_DRV_LOG(ERR, "vhost_call %s failed: %s\n",
+			    vhost_msg_strings[req], strerror(errno));
+	return ret;
+}
diff --git a/drivers/net/virtio/virtio_user/vhost.h b/drivers/net/virtio/virtio_user/vhost.h
new file mode 100644
index 0000000..5cd3543
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost.h
@@ -0,0 +1,221 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VHOST_NET_USER_H
+#define _VHOST_NET_USER_H
+
+#include <stdint.h>
+#include <linux/types.h>
+#include <linux/ioctl.h>
+
+#include "../virtio_pci.h"
+#include "../virtio_logs.h"
+#include "../virtqueue.h"
+
+#define VHOST_MEMORY_MAX_NREGIONS 8
+
+struct vhost_vring_state {
+	unsigned int index;
+	unsigned int num;
+};
+
+struct vhost_vring_file {
+	unsigned int index;
+	int fd;
+};
+
+struct vhost_vring_addr {
+	unsigned int index;
+	/* Option flags. */
+	unsigned int flags;
+	/* Flag values: */
+	/* Whether log address is valid. If set enables logging. */
+#define VHOST_VRING_F_LOG 0
+
+	/* Start of array of descriptors (virtually contiguous) */
+	uint64_t desc_user_addr;
+	/* Used structure address. Must be 32 bit aligned */
+	uint64_t used_user_addr;
+	/* Available structure address. Must be 16 bit aligned */
+	uint64_t avail_user_addr;
+	/* Logging support. */
+	/* Log writes to used structure, at offset calculated from specified
+	 * address. Address must be 32 bit aligned.
+	 */
+	uint64_t log_guest_addr;
+};
+
+enum vhost_user_request {
+	VHOST_USER_NONE = 0,
+	VHOST_USER_GET_FEATURES = 1,
+	VHOST_USER_SET_FEATURES = 2,
+	VHOST_USER_SET_OWNER = 3,
+	VHOST_USER_RESET_OWNER = 4,
+	VHOST_USER_SET_MEM_TABLE = 5,
+	VHOST_USER_SET_LOG_BASE = 6,
+	VHOST_USER_SET_LOG_FD = 7,
+	VHOST_USER_SET_VRING_NUM = 8,
+	VHOST_USER_SET_VRING_ADDR = 9,
+	VHOST_USER_SET_VRING_BASE = 10,
+	VHOST_USER_GET_VRING_BASE = 11,
+	VHOST_USER_SET_VRING_KICK = 12,
+	VHOST_USER_SET_VRING_CALL = 13,
+	VHOST_USER_SET_VRING_ERR = 14,
+	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
+	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
+	VHOST_USER_GET_QUEUE_NUM = 17,
+	VHOST_USER_SET_VRING_ENABLE = 18,
+	VHOST_USER_MAX
+};
+
+struct vhost_memory_region {
+	uint64_t guest_phys_addr;
+	uint64_t memory_size; /* bytes */
+	uint64_t userspace_addr;
+	uint64_t mmap_offset;
+};
+
+struct vhost_memory_kernel {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[0];
+};
+
+struct vhost_memory {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[VHOST_MEMORY_MAX_NREGIONS];
+};
+
+struct vhost_user_msg {
+	enum vhost_user_request request;
+
+#define VHOST_USER_VERSION_MASK     0x3
+#define VHOST_USER_REPLY_MASK       (0x1 << 2)
+	uint32_t flags;
+	uint32_t size; /* the following payload size */
+	union {
+#define VHOST_USER_VRING_IDX_MASK   0xff
+#define VHOST_USER_VRING_NOFD_MASK  (0x1 << 8)
+		uint64_t u64;
+		struct vhost_vring_state state;
+		struct vhost_vring_addr addr;
+		struct vhost_memory memory;
+	} payload;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+} __attribute((packed));
+
+#define VHOST_USER_HDR_SIZE offsetof(struct vhost_user_msg, payload.u64)
+#define VHOST_USER_PAYLOAD_SIZE (sizeof(struct vhost_user_msg) - VHOST_USER_HDR_SIZE)
+
+/* The version of the protocol we support */
+#define VHOST_USER_VERSION    0x1
+
+/* ioctls */
+
+#define VHOST_VIRTIO 0xAF
+
+#define VHOST_GET_FEATURES _IOR(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_FEATURES _IOW(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_OWNER _IO(VHOST_VIRTIO, 0x01)
+#define VHOST_RESET_OWNER _IO(VHOST_VIRTIO, 0x02)
+#define VHOST_SET_MEM_TABLE _IOW(VHOST_VIRTIO, 0x03, struct vhost_memory_kernel)
+#define VHOST_SET_LOG_BASE _IOW(VHOST_VIRTIO, 0x04, __u64)
+#define VHOST_SET_LOG_FD _IOW(VHOST_VIRTIO, 0x07, int)
+#define VHOST_SET_VRING_NUM _IOW(VHOST_VIRTIO, 0x10, struct vhost_vring_state)
+#define VHOST_SET_VRING_ADDR _IOW(VHOST_VIRTIO, 0x11, struct vhost_vring_addr)
+#define VHOST_SET_VRING_BASE _IOW(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_GET_VRING_BASE _IOWR(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_SET_VRING_KICK _IOW(VHOST_VIRTIO, 0x20, struct vhost_vring_file)
+#define VHOST_SET_VRING_CALL _IOW(VHOST_VIRTIO, 0x21, struct vhost_vring_file)
+#define VHOST_SET_VRING_ERR _IOW(VHOST_VIRTIO, 0x22, struct vhost_vring_file)
+#define VHOST_NET_SET_BACKEND _IOW(VHOST_VIRTIO, 0x30, struct vhost_vring_file)
+
+/*****************************************************************************/
+
+/* Ioctl defines */
+#define TUNSETIFF     _IOW('T', 202, int)
+#define TUNGETFEATURES _IOR('T', 207, unsigned int)
+#define TUNSETOFFLOAD  _IOW('T', 208, unsigned int)
+#define TUNGETIFF      _IOR('T', 210, unsigned int)
+#define TUNSETSNDBUF   _IOW('T', 212, int)
+#define TUNGETVNETHDRSZ _IOR('T', 215, int)
+#define TUNSETVNETHDRSZ _IOW('T', 216, int)
+#define TUNSETQUEUE  _IOW('T', 217, int)
+#define TUNSETVNETLE _IOW('T', 220, int)
+#define TUNSETVNETBE _IOW('T', 222, int)
+
+/* TUNSETIFF ifr flags */
+#define IFF_TAP          0x0002
+#define IFF_NO_PI        0x1000
+#define IFF_ONE_QUEUE    0x2000
+#define IFF_VNET_HDR     0x4000
+#define IFF_MULTI_QUEUE  0x0100
+#define IFF_ATTACH_QUEUE 0x0200
+#define IFF_DETACH_QUEUE 0x0400
+
+/* Features for GSO (TUNSETOFFLOAD). */
+#define TUN_F_CSUM	0x01	/* You can hand me unchecksummed packets. */
+#define TUN_F_TSO4	0x02	/* I can handle TSO for IPv4 packets */
+#define TUN_F_TSO6	0x04	/* I can handle TSO for IPv6 packets */
+#define TUN_F_TSO_ECN	0x08	/* I can handle TSO with ECN bits. */
+#define TUN_F_UFO	0x10	/* I can handle UFO packets */
+
+enum {
+	VHOST_MSG_SET_OWNER,
+	VHOST_MSG_RESET_OWNER,
+	VHOST_MSG_SET_FEATURES,
+	VHOST_MSG_GET_FEATURES,
+	VHOST_MSG_SET_VRING_CALL,
+	VHOST_MSG_SET_VRING_NUM,
+	VHOST_MSG_SET_VRING_BASE,
+	VHOST_MSG_GET_VRING_BASE,
+	VHOST_MSG_SET_VRING_ADDR,
+	VHOST_MSG_SET_VRING_KICK,
+	VHOST_MSG_SET_MEM_TABLE,
+	VHOST_MSG_MAX,
+};
+
+#define VHOST_KERNEL	0
+#define VHOST_USER	1
+
+int vhost_user_sock(int vhostfd, unsigned long int req, void *arg);
+int vhost_user_setup(const char *path);
+
+int vhost_kernel_ioctl(int vhostfd, unsigned long int req, void *arg);
+int vhost_kernel_setup(const char *path, const char *ifname, int *p_tapfd);
+int vhost_kernel_post(int vhostfd, int tapfd, uint64_t features, unsigned nvqs);
+
+int vhost_call(int vhostfd, unsigned int type, unsigned long int req, void *arg);
+
+#endif
diff --git a/drivers/net/virtio/virtio_user/vhost_kernel.c b/drivers/net/virtio/virtio_user/vhost_kernel.c
new file mode 100644
index 0000000..8abcf45
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost_kernel.c
@@ -0,0 +1,254 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <sys/ioctl.h>
+#include <net/if.h>
+#include <string.h>
+#include <errno.h>
+
+#include <rte_memory.h>
+#include <rte_eal_memconfig.h>
+
+#include "vhost.h"
+
+#define TUN_DEF_SNDBUF	(1ull << 20)
+#define PATH_NET_TUN	"/dev/net/tun"
+
+/** Merge those virtually adjacent memsegs into one region.
+ */
+static void
+prepare_vhost_memory_kernel(struct vhost_memory_kernel **p_vm)
+{
+	unsigned i, j, k = 0;
+	struct rte_memseg *seg;
+	struct vhost_memory_region *mr;
+	struct vhost_memory_kernel *vm;
+
+	vm = malloc(sizeof(struct vhost_memory_kernel) +
+		    RTE_MAX_MEMSEG * sizeof(struct vhost_memory_region));
+
+	for (i = 0; i < RTE_MAX_MEMSEG; ++i) {
+		seg = &rte_eal_get_configuration()->mem_config->memseg[i];
+		if (!seg->addr)
+			break;
+
+		int new_region = 1;
+
+		for (j = 0; j < k; ++j) {
+			mr = &vm->regions[j];
+
+			if (mr->userspace_addr + mr->memory_size ==
+			    (uint64_t)seg->addr) {
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+
+			if ((uint64_t)seg->addr + seg->len ==
+			    mr->userspace_addr) {
+				mr->guest_phys_addr = (uint64_t)seg->addr;
+				mr->userspace_addr = (uint64_t)seg->addr;
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+		}
+
+		if (new_region == 0)
+			continue;
+
+		mr = &vm->regions[k++];
+		mr->guest_phys_addr = (uint64_t)seg->addr; /* use vaddr here! */
+		mr->userspace_addr = (uint64_t)seg->addr;
+		mr->memory_size = seg->len;
+		mr->mmap_offset = 0;
+	}
+
+	vm->nregions = k;
+	vm->padding = 0;
+	*p_vm = vm;
+}
+
+int
+vhost_kernel_ioctl(int vhostfd, unsigned long int req, void *arg)
+{
+	int ret;
+	struct vhost_memory_kernel *vm = NULL;
+
+	if (req == VHOST_SET_MEM_TABLE) {
+		prepare_vhost_memory_kernel(&vm);
+		arg = (void *)vm;
+	}
+
+	ret = ioctl(vhostfd, req, arg);
+
+	if (req == VHOST_SET_MEM_TABLE)
+		free(vm);
+	else if (req == VHOST_SET_FEATURES) {
+	}
+
+	return ret;
+}
+
+/**
+ * Set up environment to talk with a vhost kernel backend.
+ * @param path
+ *   - The path to vhost net (kernel) character file.
+ *
+ * @param ifname
+ *   - Specify the tap device name if any, or NULL.
+ *
+ * @param p_tapfd
+ *   - Pointer to store the fd of tap device.
+ *
+ * @return
+ *   - (-1) if fail to set up;
+ *   - (>=0) if successful, and it is the fd to vhostfd.
+ */
+int
+vhost_kernel_setup(const char *path, const char *ifname, int *p_tapfd)
+{
+	int vhostfd, tapfd;
+	int len = sizeof(struct virtio_net_hdr_mrg_rxbuf);
+	int req_mq = 0;
+	int sndbuf = TUN_DEF_SNDBUF;
+	unsigned int features;
+	struct ifreq ifr;
+
+	/* TODO:
+	 * 1. get/set offload capability, tap_probe_has_ufo, tap_fd_set_offload
+	 * 2. verify we can get/set vnet_hdr_len, tap_probe_vnet_hdr_len
+	 * 3. get number of memory regions from vhost module parameter
+	 * max_mem_regions, supported in newer version linux kernel
+	 */
+
+	tapfd = open(PATH_NET_TUN, O_RDWR);
+	if (tapfd < 0) {
+		PMD_DRV_LOG(ERR, "fail to open %s: %s",
+			    PATH_NET_TUN, strerror(errno));
+		return -1;
+	}
+
+	memset(&ifr, 0, sizeof(ifr));
+	ifr.ifr_flags = IFF_TAP | IFF_NO_PI;
+
+	if (ioctl(tapfd, TUNGETFEATURES, &features) == -1) {
+		PMD_DRV_LOG(ERR, "TUNGETFEATURES failed: %s", strerror(errno));
+		goto error;
+	}
+
+	if (features & IFF_ONE_QUEUE)
+		ifr.ifr_flags |= IFF_ONE_QUEUE;
+
+	if (features & IFF_VNET_HDR)
+		ifr.ifr_flags |= IFF_VNET_HDR;
+	else {
+		PMD_DRV_LOG(ERR, "vnet_hdr not supported by kernel");
+		goto error;
+	}
+
+	if (req_mq) {
+		if (features & IFF_MULTI_QUEUE)
+			ifr.ifr_flags |= IFF_MULTI_QUEUE;
+		else {
+			PMD_DRV_LOG(ERR, "multiqueue not supported by kernel");
+			goto error;
+		}
+	}
+
+	if (ifname)
+		strncpy(ifr.ifr_name, ifname, IFNAMSIZ);
+	else
+		strncpy(ifr.ifr_name, "tap%d", IFNAMSIZ);
+	if (ioctl(tapfd, TUNSETIFF, (void *)&ifr) == -1) {
+		PMD_DRV_LOG(ERR, "TUNSETIFF failed: %s", strerror(errno));
+		goto error;
+	}
+	fcntl(tapfd, F_SETFL, O_NONBLOCK);
+
+	if (ioctl(tapfd, TUNSETVNETHDRSZ, &len) < 0) {
+		PMD_DRV_LOG(ERR, "TUNSETVNETHDRSZ failed: %s", strerror(errno));
+		goto error;
+	}
+
+	if (ioctl(tapfd, TUNSETSNDBUF, &sndbuf) < 0) {
+		PMD_DRV_LOG(ERR, "TUNSETSNDBUF failed: %s", strerror(errno));
+		goto error;
+	}
+
+	vhostfd = open(path, O_RDWR);
+	if (vhostfd < 0) {
+		PMD_DRV_LOG(ERR, "fail to open %s", path, strerror(errno));
+		goto error;
+	}
+
+	*p_tapfd = tapfd;
+	return vhostfd;
+
+error:
+	close(tapfd);
+	return -1;
+}
+
+int
+vhost_kernel_post(int vhostfd, int tapfd, uint64_t features, unsigned nvqs)
+{
+	struct vhost_vring_file f;
+	uint16_t hdr_size;
+
+	if ((features & (1ULL << VIRTIO_NET_F_MRG_RXBUF)) ||
+	    (features & (1ULL << VIRTIO_F_VERSION_1)))
+		hdr_size = sizeof(struct virtio_net_hdr_mrg_rxbuf);
+	else
+		hdr_size = sizeof(struct virtio_net_hdr);
+
+	if (ioctl(tapfd, TUNSETVNETHDRSZ, &hdr_size) == -1) {
+		PMD_DRV_LOG(ERR, "TUNSETVNETHDRSZ fails, %s", strerror(errno));
+		return -1;
+	}
+
+	f.fd = tapfd;
+	for (f.index = 0; f.index < nvqs; ++f.index) {
+		if (vhost_kernel_ioctl(vhostfd, VHOST_NET_SET_BACKEND, &f) < 0) {
+			PMD_DRV_LOG(ERR, "VHOST_NET_SET_BACKEND fails, %s",
+				    strerror(errno));
+			return -1;
+		}
+	}
+
+	return 0;
+}
diff --git a/drivers/net/virtio/virtio_user/vhost_user.c b/drivers/net/virtio/virtio_user/vhost_user.c
new file mode 100644
index 0000000..6fd648c
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost_user.c
@@ -0,0 +1,375 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <unistd.h>
+#include <fcntl.h>
+#include <sys/un.h>
+#include <string.h>
+#include <errno.h>
+
+#include "vhost.h"
+
+static int
+vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
+{
+	int r;
+	struct msghdr msgh;
+	struct iovec iov;
+	size_t fd_size = fd_num * sizeof(int);
+	char control[CMSG_SPACE(fd_size)];
+	struct cmsghdr *cmsg;
+
+	memset(&msgh, 0, sizeof(msgh));
+	memset(control, 0, sizeof(control));
+
+	iov.iov_base = (uint8_t *)buf;
+	iov.iov_len = len;
+
+	msgh.msg_iov = &iov;
+	msgh.msg_iovlen = 1;
+	msgh.msg_control = control;
+	msgh.msg_controllen = sizeof(control);
+
+	cmsg = CMSG_FIRSTHDR(&msgh);
+	cmsg->cmsg_len = CMSG_LEN(fd_size);
+	cmsg->cmsg_level = SOL_SOCKET;
+	cmsg->cmsg_type = SCM_RIGHTS;
+	memcpy(CMSG_DATA(cmsg), fds, fd_size);
+
+	do {
+		r = sendmsg(fd, &msgh, 0);
+	} while (r < 0 && errno == EINTR);
+
+	return r;
+}
+
+static int
+vhost_user_read(int fd, struct vhost_user_msg *msg)
+{
+	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
+	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
+
+	ret = recv(fd, (void *)msg, sz_hdr, 0);
+	if (ret < sz_hdr) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
+			    ret, sz_hdr);
+		goto fail;
+	}
+
+	/* validate msg flags */
+	if (msg->flags != (valid_flags)) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg: flags %x instead of %x.",
+			    msg->flags, valid_flags);
+		goto fail;
+	}
+
+	sz_payload = msg->size;
+	if (sz_payload) {
+		ret = recv(fd, (void *)((char *)msg + sz_hdr), sz_payload, 0);
+		if (ret < sz_payload) {
+			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
+				    ret, msg->size);
+			goto fail;
+		}
+	}
+
+	return 0;
+
+fail:
+	return -1;
+}
+
+struct hugepage_file_info {
+	uint64_t addr;            /**< virtual addr */
+	size_t   size;            /**< the file size */
+	char     path[PATH_MAX];  /**< path to backing file */
+};
+
+/* Two possible options:
+ * 1. Match HUGEPAGE_INFO_FMT to find the file storing struct hugepage_file
+ * array. This is simple but cannot be used in secondary process because
+ * secondary process will close and munmap that file.
+ * 2. Match HUGEFILE_FMT to find hugepage files directly.
+ *
+ * We choose option 2.
+ */
+static int
+get_hugepage_file_info(struct hugepage_file_info huges[], int max)
+{
+	int idx;
+	FILE *f;
+	char buf[BUFSIZ], *tmp, *tail;
+	char *str_underline, *str_start;
+	int huge_index;
+	uint64_t v_start, v_end;
+
+	f = fopen("/proc/self/maps", "r");
+	if (!f) {
+		PMD_DRV_LOG(ERR, "cannot open /proc/self/maps");
+		return -1;
+	}
+
+	idx = 0;
+	while (fgets(buf, sizeof(buf), f) != NULL) {
+		sscanf(buf, "%" PRIx64 "-%" PRIx64, &v_start, &v_end);
+
+		tmp = strchr(buf, ' ') + 1; /** skip address */
+		tmp = strchr(tmp, ' ') + 1; /** skip perm */
+		tmp = strchr(tmp, ' ') + 1; /** skip offset */
+		tmp = strchr(tmp, ' ') + 1; /** skip dev */
+		tmp = strchr(tmp, ' ') + 1; /** skip inode */
+		while (*tmp == ' ')         /** skip spaces */
+			tmp++;
+		tail = strrchr(tmp, '\n');  /** remove newline if exists */
+		if (tail)
+			*tail = '\0';
+
+		/* Match HUGEFILE_FMT, aka "%s/%smap_%d",
+		 * which is defined in eal_filesystem.h
+		 */
+		str_underline = strrchr(tmp, '_');
+		if (!str_underline)
+			continue;
+
+		str_start = str_underline - strlen("map");
+		if (str_start < tmp)
+			continue;
+
+		if (sscanf(str_start, "map_%d", &huge_index) != 1)
+			continue;
+
+		if (idx >= max) {
+			PMD_DRV_LOG(ERR, "Exceed maximum of %d", max);
+			goto error;
+		}
+		huges[idx].addr = v_start;
+		huges[idx].size = v_end - v_start;
+		strcpy(huges[idx].path, tmp);
+		idx++;
+	}
+
+	fclose(f);
+	return idx;
+
+error:
+	fclose(f);
+	return -1;
+}
+
+static int
+prepare_vhost_memory_user(struct vhost_user_msg *msg, int fds[])
+{
+	int i, num;
+	struct hugepage_file_info huges[VHOST_MEMORY_MAX_NREGIONS];
+	struct vhost_memory_region *mr;
+
+	num = get_hugepage_file_info(huges, VHOST_MEMORY_MAX_NREGIONS);
+	if (num < 0) {
+		PMD_INIT_LOG(ERR, "Failed to prepare memory for vhost-user");
+		return -1;
+	}
+
+	for (i = 0; i < num; ++i) {
+		mr = &msg->payload.memory.regions[i];
+		mr->guest_phys_addr = huges[i].addr; /* use vaddr! */
+		mr->userspace_addr = huges[i].addr;
+		mr->memory_size = huges[i].size;
+		mr->mmap_offset = 0;
+		fds[i] = open(huges[i].path, O_RDWR);
+	}
+
+	msg->payload.memory.nregions = num;
+	msg->payload.memory.padding = 0;
+
+	return 0;
+}
+
+static struct vhost_user_msg m;
+
+int
+vhost_user_sock(int vhostfd, unsigned long int req, void *arg)
+{
+	struct vhost_user_msg msg;
+	struct vhost_vring_file *file = 0;
+	int need_reply = 0;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+	int fd_num = 0;
+	int i, len;
+
+	msg.request = req;
+	msg.flags = VHOST_USER_VERSION;
+	msg.size = 0;
+
+	switch (req) {
+	case VHOST_USER_GET_FEATURES:
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_FEATURES:
+	case VHOST_USER_SET_LOG_BASE:
+		msg.payload.u64 = *((__u64 *)arg);
+		msg.size = sizeof(m.payload.u64);
+		break;
+
+	case VHOST_USER_SET_OWNER:
+	case VHOST_USER_RESET_OWNER:
+		break;
+
+	case VHOST_USER_SET_MEM_TABLE:
+		if (prepare_vhost_memory_user(&msg, fds) < 0)
+			return -1;
+		fd_num = msg.payload.memory.nregions;
+		msg.size = sizeof(m.payload.memory.nregions);
+		msg.size += sizeof(m.payload.memory.padding);
+		msg.size += fd_num * sizeof(struct vhost_memory_region);
+		break;
+
+	case VHOST_USER_SET_LOG_FD:
+		fds[fd_num++] = *((int *)arg);
+		break;
+
+	case VHOST_USER_SET_VRING_NUM:
+	case VHOST_USER_SET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		break;
+
+	case VHOST_USER_GET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_VRING_ADDR:
+		memcpy(&msg.payload.addr, arg, sizeof(msg.payload.addr));
+		msg.size = sizeof(m.payload.addr);
+		break;
+
+	case VHOST_USER_SET_VRING_KICK:
+	case VHOST_USER_SET_VRING_CALL:
+	case VHOST_USER_SET_VRING_ERR:
+		file = arg;
+		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
+		msg.size = sizeof(m.payload.u64);
+		if (file->fd > 0)
+			fds[fd_num++] = file->fd;
+		else
+			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
+		break;
+
+	default:
+		PMD_DRV_LOG(ERR, "trying to send unhandled msg type");
+		return -1;
+	}
+
+	len = VHOST_USER_HDR_SIZE + msg.size;
+	if (vhost_user_write(vhostfd, &msg, len, fds, fd_num) < 0)
+		return 0;
+
+	if (req == VHOST_USER_SET_MEM_TABLE)
+		for (i = 0; i < fd_num; ++i)
+			close(fds[i]);
+
+	if (need_reply) {
+		if (vhost_user_read(vhostfd, &msg) < 0)
+			return -1;
+
+		if (req != msg.request) {
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+
+		switch (req) {
+		case VHOST_USER_GET_FEATURES:
+			if (msg.size != sizeof(m.payload.u64)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			*((__u64 *)arg) = msg.payload.u64;
+			break;
+		case VHOST_USER_GET_VRING_BASE:
+			if (msg.size != sizeof(m.payload.state)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			memcpy(arg, &msg.payload.state,
+			       sizeof(struct vhost_vring_state));
+			break;
+		default:
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+	}
+
+	return 0;
+}
+
+/**
+ * Set up environment to talk with a vhost user backend.
+ * @param path
+ *   - The path to vhost user unix socket file.
+ *
+ * @return
+ *   - (-1) if fail to set up;
+ *   - (>=0) if successful, and it is the fd to vhostfd.
+ */
+int
+vhost_user_setup(const char *path)
+{
+	int fd;
+	int flag;
+	struct sockaddr_un un;
+
+	fd = socket(AF_UNIX, SOCK_STREAM, 0);
+	if (fd < 0) {
+		PMD_DRV_LOG(ERR, "socket() error, %s", strerror(errno));
+		return -1;
+	}
+
+	flag = fcntl(fd, F_GETFD);
+	fcntl(fd, F_SETFD, flag | FD_CLOEXEC);
+
+	memset(&un, 0, sizeof(un));
+	un.sun_family = AF_UNIX;
+	snprintf(un.sun_path, sizeof(un.sun_path), "%s", path);
+	if (connect(fd, (struct sockaddr *)&un, sizeof(un)) < 0) {
+		PMD_DRV_LOG(ERR, "connect error, %s", strerror(errno));
+		close(fd);
+		return -1;
+	}
+
+	return fd;
+}
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v4 5/8] virtio-user: add device emulation layer APIs
  2016-04-29  1:18 ` [PATCH v4 0/8] virtio support for container Jianfeng Tan
                     ` (3 preceding siblings ...)
  2016-04-29  1:18   ` [PATCH v4 4/8] virtio-user: add vhost adapter layer Jianfeng Tan
@ 2016-04-29  1:18   ` Jianfeng Tan
  2016-05-12  1:05     ` Yuanhan Liu
  2016-04-29  1:18   ` [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
                     ` (3 subsequent siblings)
  8 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-04-29  1:18 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, michael.qiu, ann.zhuangyanying,
	mukawa, nhorman

Two device emulation layer APIs are added for virtio driver to call:
  - virtio_user_start_device()
  - virtio_user_stop_device()

These APIs will get called by virtio driver, and they call vhost adapter
layer APIs to implement the functionality. Besides, this patch defines
a struct named virtio_user_hw to help manage the data stands for this
kind of virtual device.

  ----------------------
  | ------------------ |
  | | virtio driver  | |
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate |-|----> (virtio_user_dev.c, virtio_user_dev.h)
  | |                | |
  | | vhost adapter  | |
  | ------------------ |
  ----------------------
            |
            |
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-By: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/Makefile                      |   1 +
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 168 +++++++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.h |  59 ++++++++
 3 files changed, 228 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h

diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index c9f2bc0..68068bd 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -59,6 +59,7 @@ ifeq ($(CONFIG_RTE_VIRTIO_VDEV),y)
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_kernel.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_dev.c
 endif
 
 # this lib depends upon:
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.c b/drivers/net/virtio/virtio_user/virtio_user_dev.c
new file mode 100644
index 0000000..81f7f03
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.c
@@ -0,0 +1,168 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <stdio.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <string.h>
+#include <errno.h>
+#include <sys/mman.h>
+#include <unistd.h>
+#include <sys/eventfd.h>
+
+#include <rte_malloc.h>
+#include <rte_kvargs.h>
+
+#include "vhost.h"
+#include "virtio_user_dev.h"
+#include "../virtio_ethdev.h"
+
+static int
+kick_one_vq(struct virtio_user_hw *hw, struct virtqueue *vq,
+	    unsigned queue_sel)
+{
+	int callfd, kickfd;
+	struct vhost_vring_file file;
+	struct vhost_vring_state state;
+	struct vhost_vring_addr addr = {
+		.index = queue_sel,
+		.desc_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.desc,
+		.avail_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.avail,
+		.used_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.used,
+		.log_guest_addr = 0,
+		.flags = 0, /* disable log */
+	};
+
+	/* May use invalid flag, but some backend leverages kickfd and callfd as
+	 * criteria to judge if dev is alive. so finally we use real event_fd.
+	 */
+	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (callfd < 0) {
+		PMD_DRV_LOG(ERR, "callfd error, %s\n", strerror(errno));
+		return -1;
+	}
+	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (kickfd < 0) {
+		close(callfd);
+		PMD_DRV_LOG(ERR, "kickfd error, %s\n", strerror(errno));
+		return -1;
+	}
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL come
+	 * firstly because vhost depends on this msg to allocate virtqueue
+	 * pair.
+	 */
+	file.index = queue_sel;
+	file.fd = callfd;
+	vhost_call(hw->vhostfd, hw->type, VHOST_MSG_SET_VRING_CALL, &file);
+	hw->callfds[queue_sel] = callfd;
+
+	state.index = queue_sel;
+	state.num = vq->vq_ring.num;
+	vhost_call(hw->vhostfd, hw->type, VHOST_MSG_SET_VRING_NUM, &state);
+
+	state.num = 0; /* no reservation */
+	vhost_call(hw->vhostfd, hw->type, VHOST_MSG_SET_VRING_BASE, &state);
+
+	vhost_call(hw->vhostfd, hw->type, VHOST_MSG_SET_VRING_ADDR, &addr);
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_KICK comes
+	 * lastly because vhost depends on this msg to judge if
+	 * virtio is ready.
+	 */
+	file.fd = kickfd;
+	vhost_call(hw->vhostfd, hw->type, VHOST_MSG_SET_VRING_KICK, &file);
+	hw->kickfds[queue_sel] = kickfd;
+
+	return 0;
+}
+
+int
+virtio_user_start_device(struct virtio_user_hw *hw)
+{
+	uint64_t features;
+	unsigned i, queue_sel;
+	struct rte_eth_dev_data *data = hw->data;
+	int ret;
+
+	/* construct memory region inside each implementation */
+	ret = vhost_call(hw->vhostfd, hw->type, VHOST_MSG_SET_MEM_TABLE, NULL);
+	if (ret < 0)
+		goto error;
+
+	for (i = 0; i < data->nb_rx_queues; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
+		if (kick_one_vq(hw, data->rx_queues[i], queue_sel) < 0) {
+			PMD_DRV_LOG(INFO, "kick rx vq fails: %u", i);
+			goto error;
+		}
+	}
+	for (i = 0; i < data->nb_tx_queues; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
+		if (kick_one_vq(hw, data->tx_queues[i], queue_sel) < 0) {
+			PMD_DRV_LOG(INFO, "kick tx vq fails: %u", i);
+			goto error;
+		}
+	}
+
+	/* After setup all virtqueues, we need to set_features so that
+	 * these features can be set into each virtqueue in vhost side.
+	 * And before that, make sure VIRTIO_NET_F_MAC is stripped.
+	 */
+	features = hw->features;
+	features &= ~(1ull << VIRTIO_NET_F_MAC);
+	ret = vhost_call(hw->vhostfd, hw->type,
+			 VHOST_MSG_SET_FEATURES, &features);
+	if (ret < 0)
+		goto error;
+	PMD_DRV_LOG(INFO, "set features: %" PRIx64, features);
+
+	if (hw->type == VHOST_USER)
+		return 0;
+
+	/* It is tricky here, but so far we have to do that */
+	ret = vhost_kernel_post(hw->vhostfd, hw->tapfd, features,
+				 data->nb_rx_queues + data->nb_tx_queues);
+	if (ret == 0)
+		return 0;
+error:
+	/* TODO: free resource here or caller to check */
+	return -1;
+}
+
+int virtio_user_stop_device(struct virtio_user_hw *hw)
+{
+	return vhost_call(hw->vhostfd, hw->type, VHOST_MSG_RESET_OWNER, NULL);
+}
+
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.h b/drivers/net/virtio/virtio_user/virtio_user_dev.h
new file mode 100644
index 0000000..76250f0
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.h
@@ -0,0 +1,59 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VIRTIO_USER_VDEV_H
+#define _VIRTIO_USER_VDEV_H
+
+#include <limits.h>
+#include "../virtio_pci.h"
+
+struct virtio_user_hw {
+	struct rte_eth_dev_data *data;
+	int		type; /* VHOST_KERNEL or VHOST_USER */
+	int		vhostfd;
+	int		tapfd; /* only used in vhost kernel */
+	int		callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		mac_specified;
+	uint32_t	max_queue_pairs;
+	uint32_t	queue_size;
+	uint64_t	features;
+	uint8_t		status;
+	uint8_t		mac_addr[ETHER_ADDR_LEN];
+	char		path[PATH_MAX];
+};
+
+int virtio_user_start_device(struct virtio_user_hw *hw);
+int virtio_user_stop_device(struct virtio_user_hw *hw);
+
+#endif
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio
  2016-04-29  1:18 ` [PATCH v4 0/8] virtio support for container Jianfeng Tan
                     ` (4 preceding siblings ...)
  2016-04-29  1:18   ` [PATCH v4 5/8] virtio-user: add device emulation layer APIs Jianfeng Tan
@ 2016-04-29  1:18   ` Jianfeng Tan
  2016-05-12  2:12     ` Yuanhan Liu
  2016-04-29  1:18   ` [PATCH v4 7/8] virtio-user: add a new virtual device named virtio-user Jianfeng Tan
                     ` (2 subsequent siblings)
  8 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-04-29  1:18 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, michael.qiu, ann.zhuangyanying,
	mukawa, nhorman

This patch implements another new instance of struct virtio_pci_ops to
drive the virtio-user virtual device. Instead of rd/wr ioport or PCI
configuration space, this virtual pci driver will rd/wr the virtual
device struct virtio_user_hw, and when necessary, invokes APIs provided
by device emulation later to start/stop the device.

  ----------------------
  | ------------------ |
  | | virtio driver  | |----> (virtio_user_pci.c)
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate | |
  | |                | |
  | | vhost adapter  | |
  | ------------------ |
  ----------------------
            |
            |
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-By: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/Makefile                      |   1 +
 drivers/net/virtio/virtio_user/virtio_user_dev.h |   2 +
 drivers/net/virtio/virtio_user/virtio_user_pci.c | 209 +++++++++++++++++++++++
 3 files changed, 212 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_pci.c

diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index 68068bd..13b2b75 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -60,6 +60,7 @@ SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_kernel.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_dev.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_pci.c
 endif
 
 # this lib depends upon:
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.h b/drivers/net/virtio/virtio_user/virtio_user_dev.h
index 76250f0..bc4dc1a 100644
--- a/drivers/net/virtio/virtio_user/virtio_user_dev.h
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.h
@@ -56,4 +56,6 @@ struct virtio_user_hw {
 int virtio_user_start_device(struct virtio_user_hw *hw);
 int virtio_user_stop_device(struct virtio_user_hw *hw);
 
+const struct virtio_pci_ops vdev_ops;
+
 #endif
diff --git a/drivers/net/virtio/virtio_user/virtio_user_pci.c b/drivers/net/virtio/virtio_user/virtio_user_pci.c
new file mode 100644
index 0000000..60351d9
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_pci.c
@@ -0,0 +1,209 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <sys/types.h>
+#include <unistd.h>
+
+#include "../virtio_logs.h"
+#include "../virtio_pci.h"
+#include "../virtqueue.h"
+#include "virtio_user_dev.h"
+
+static void
+vdev_read_dev_config(struct virtio_hw *hw, uint64_t offset,
+		     void *dst, int length)
+{
+	int i;
+	struct virtio_user_hw *uhw = (struct virtio_user_hw *)hw->vdev_private;
+
+	if (offset == offsetof(struct virtio_net_config, mac) &&
+	    length == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			((uint8_t *)dst)[i] = uhw->mac_addr[i];
+		return;
+	}
+
+	if (offset == offsetof(struct virtio_net_config, status))
+		*(uint16_t *)dst = uhw->status;
+
+	if (offset == offsetof(struct virtio_net_config, max_virtqueue_pairs))
+		*(uint16_t *)dst = uhw->max_queue_pairs;
+}
+
+static void
+vdev_write_dev_config(struct virtio_hw *hw, uint64_t offset,
+		      const void *src, int length)
+{
+	int i;
+	struct virtio_user_hw *uhw = (struct virtio_user_hw *)hw->vdev_private;
+
+	if ((offset == offsetof(struct virtio_net_config, mac)) &&
+	    (length == ETHER_ADDR_LEN))
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			uhw->mac_addr[i] = ((const uint8_t *)src)[i];
+	else
+		PMD_DRV_LOG(ERR, "not supported offset=%" PRIu64 ", length=%d",
+			    offset, length);
+}
+
+static void
+vdev_set_status(struct virtio_hw *hw, uint8_t status)
+{
+	struct virtio_user_hw *uhw = (struct virtio_user_hw *)hw->vdev_private;
+
+	if (status & VIRTIO_CONFIG_STATUS_DRIVER_OK)
+		virtio_user_start_device(uhw);
+	uhw->status = status;
+}
+
+static void
+vdev_reset(struct virtio_hw *hw)
+{
+	struct virtio_user_hw *uhw = (struct virtio_user_hw *)hw->vdev_private;
+
+	virtio_user_stop_device(uhw);
+}
+
+static uint8_t
+vdev_get_status(struct virtio_hw *hw)
+{
+	struct virtio_user_hw *uhw = (struct virtio_user_hw *)hw->vdev_private;
+
+	return uhw->status;
+}
+
+static uint64_t
+vdev_get_features(struct virtio_hw *hw)
+{
+	struct virtio_user_hw *uhw = (struct virtio_user_hw *)hw->vdev_private;
+
+	return uhw->features;
+}
+
+static void
+vdev_set_features(struct virtio_hw *hw, uint64_t features)
+{
+	struct virtio_user_hw *uhw = (struct virtio_user_hw *)hw->vdev_private;
+
+	uhw->features = features;
+}
+
+static uint8_t
+vdev_get_isr(struct virtio_hw *hw __rte_unused)
+{
+	/* When config interrupt happens, driver calls this function to query
+	 * what kinds of change happen. Interrupt mode not supported for now.
+	 */
+	return 0;
+}
+
+static uint16_t
+vdev_set_config_irq(struct virtio_hw *hw __rte_unused,
+		    uint16_t vec __rte_unused)
+{
+	return VIRTIO_MSI_NO_VECTOR;
+}
+
+/* This function is to get the queue size, aka, number of descs, of a specified
+ * queue. Different with the VHOST_USER_GET_QUEUE_NUM, which is used to get the
+ * max supported queues.
+ */
+static uint16_t
+vdev_get_queue_num(struct virtio_hw *hw,
+		   uint16_t queue_id __rte_unused)
+{
+	struct virtio_user_hw *uhw = (struct virtio_user_hw *)hw->vdev_private;
+
+	/* Currently, each queue has same queue size */
+	return uhw->queue_size;
+}
+
+static int
+vdev_setup_queue(struct virtio_hw *hw __rte_unused, struct virtqueue *vq)
+{
+	/* Changed to use virtual addr */
+	vq->vq_ring_mem = (phys_addr_t)vq->mz->addr;
+	if (vq->virtio_net_hdr_mz) {
+		vq->virtio_net_hdr_mem =
+			(phys_addr_t)vq->virtio_net_hdr_mz->addr;
+		/* Do it one more time after we reset virtio_net_hdr_mem */
+		vring_hdr_desc_init(vq);
+	}
+	vq->offset = offsetof(struct rte_mbuf, buf_addr);
+	return 0;
+}
+
+static void
+vdev_del_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	/* For legacy devices, write 0 to VIRTIO_PCI_QUEUE_PFN port, QEMU
+	 * correspondingly stops the ioeventfds, and reset the status of
+	 * the device.
+	 * For modern devices, set queue desc, avail, used in PCI bar to 0,
+	 * not see any more behavior in QEMU.
+	 *
+	 * Here we just care about what information to deliver to vhost-user
+	 * or vhost-kernel. And we just close ioeventfd for now.
+	 */
+	struct virtio_user_hw *uhw = (struct virtio_user_hw *)hw->vdev_private;
+
+	close(uhw->callfds[vq->vq_queue_index]);
+	close(uhw->kickfds[vq->vq_queue_index]);
+}
+
+static void
+vdev_notify_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	uint64_t buf = 1;
+	struct virtio_user_hw *uhw = (struct virtio_user_hw *)hw->vdev_private;
+
+	if (write(uhw->kickfds[vq->vq_queue_index], &buf, sizeof(buf)) < 0)
+		PMD_DRV_LOG(ERR, "failed to kick backend: %d", strerror(errno));
+}
+
+const struct virtio_pci_ops vdev_ops = {
+	.read_dev_cfg	= vdev_read_dev_config,
+	.write_dev_cfg	= vdev_write_dev_config,
+	.reset		= vdev_reset,
+	.get_status	= vdev_get_status,
+	.set_status	= vdev_set_status,
+	.get_features	= vdev_get_features,
+	.set_features	= vdev_set_features,
+	.get_isr	= vdev_get_isr,
+	.set_config_irq	= vdev_set_config_irq,
+	.get_queue_num	= vdev_get_queue_num,
+	.setup_queue	= vdev_setup_queue,
+	.del_queue	= vdev_del_queue,
+	.notify_queue	= vdev_notify_queue,
+};
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v4 7/8] virtio-user: add a new virtual device named virtio-user
  2016-04-29  1:18 ` [PATCH v4 0/8] virtio support for container Jianfeng Tan
                     ` (5 preceding siblings ...)
  2016-04-29  1:18   ` [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
@ 2016-04-29  1:18   ` Jianfeng Tan
  2016-04-29  1:18   ` [PATCH v4 8/8] doc: update doc for virtio-user Jianfeng Tan
  2016-04-29  1:35   ` [PATCH v4 0/8] virtio support for container Tan, Jianfeng
  8 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-04-29  1:18 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, michael.qiu, ann.zhuangyanying,
	mukawa, nhorman

Add a new virtual device named vhost-user, which can be used just like
eth_ring, eth_null, etc. To reuse the code of original virtio, we do
some adjustment in virtio_ethdev.c, such as remove key _static_ of
eth_virtio_dev_init() so that it can be reused in virtual device; and
we add some check to make sure it will not crash.

Configured parameters include:
  - queues (optional, 1 by default), number of rx, multi-queue not
    supported for now.
  - cq (optional, 0 by default), not supported for now.
  - mac (optional), random value will be given if not specified.
  - queue_size (optional, 256 by default), size of virtqueues.
  - path (madatory), path of vhost, depends on the file type, vhost
    user if the given path points to a unix socket; vhost-net if the
    given path points to a char device.
  - ifname (optional), specify the name of backend tap device; only
    valid when backend is vhost-net.

When enable CONFIG_RTE_VIRTIO_VDEV (enabled by default), the compiled
library can be used in both VM and container environment.

Examples:
path_vhost=/dev/vhost-net # use vhost-net as a backend
path_vhost=<path_to_vhost_user> # use vhost-user as a backend

sudo ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
    --socket-mem 0,1024 --no-pci --file-prefix=l2fwd \
    --vdev=virtio-user0,mac=00:01:02:03:04:05,path=$path_vhost -- -p 0x1

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-By: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/virtio_ethdev.c               |  19 +-
 drivers/net/virtio/virtio_ethdev.h               |   2 +
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 307 +++++++++++++++++++++++
 3 files changed, 321 insertions(+), 7 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 16b324d..54462a3 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -59,7 +59,6 @@
 #include "virtqueue.h"
 #include "virtio_rxtx.h"
 
-static int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
 static int eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev);
 static int  virtio_dev_configure(struct rte_eth_dev *dev);
 static int  virtio_dev_start(struct rte_eth_dev *dev);
@@ -1017,7 +1016,7 @@ rx_func_get(struct rte_eth_dev *eth_dev)
  * This function is based on probe() function in virtio_pci.c
  * It returns 0 on success.
  */
-static int
+int
 eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 {
 	struct virtio_hw *hw = eth_dev->data->dev_private;
@@ -1048,9 +1047,11 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	pci_dev = eth_dev->pci_dev;
 
-	ret = vtpci_init(pci_dev, hw, &dev_flags);
-	if (ret)
-		return ret;
+	if (pci_dev) {
+		ret = vtpci_init(pci_dev, hw, &dev_flags);
+		if (ret)
+			return ret;
+	}
 
 	/* Reset the device although not necessary at startup */
 	vtpci_reset(hw);
@@ -1147,7 +1148,8 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	PMD_INIT_LOG(DEBUG, "hw->max_rx_queues=%d   hw->max_tx_queues=%d",
 			hw->max_rx_queues, hw->max_tx_queues);
-	PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
+	if (pci_dev)
+		PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
 			eth_dev->data->port_id, pci_dev->id.vendor_id,
 			pci_dev->id.device_id);
 
@@ -1426,7 +1428,10 @@ virtio_dev_info_get(struct rte_eth_dev *dev, struct rte_eth_dev_info *dev_info)
 {
 	struct virtio_hw *hw = dev->data->dev_private;
 
-	dev_info->driver_name = dev->driver->pci_drv.name;
+	if (dev->pci_dev)
+		dev_info->driver_name = dev->driver->pci_drv.name;
+	else
+		dev_info->driver_name = "virtio-user PMD";
 	dev_info->max_rx_queues = (uint16_t)hw->max_rx_queues;
 	dev_info->max_tx_queues = (uint16_t)hw->max_tx_queues;
 	dev_info->min_rx_bufsize = VIRTIO_MIN_RX_BUFSIZE;
diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
index 66423a0..284afaa 100644
--- a/drivers/net/virtio/virtio_ethdev.h
+++ b/drivers/net/virtio/virtio_ethdev.h
@@ -113,6 +113,8 @@ uint16_t virtio_recv_pkts_vec(void *rx_queue, struct rte_mbuf **rx_pkts,
 uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 		uint16_t nb_pkts);
 
+int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
+
 /*
  * The VIRTIO_NET_F_GUEST_TSO[46] features permit the host to send us
  * frames larger than 1514 bytes. We do not yet support software LRO
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.c b/drivers/net/virtio/virtio_user/virtio_user_dev.c
index 81f7f03..a1ea833 100644
--- a/drivers/net/virtio/virtio_user/virtio_user_dev.c
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.c
@@ -166,3 +166,310 @@ int virtio_user_stop_device(struct virtio_user_hw *hw)
 	return vhost_call(hw->vhostfd, hw->type, VHOST_MSG_RESET_OWNER, NULL);
 }
 
+static inline void parse_mac(struct virtio_user_hw *hw, const char *mac)
+{
+	int i, r;
+	uint32_t tmp[ETHER_ADDR_LEN];
+
+	if (!mac)
+		return;
+
+	r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
+			&tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
+	if (r == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			hw->mac_addr[i] = (uint8_t)tmp[i];
+		hw->mac_specified = 1;
+	} else {
+		/* ignore the wrong mac, use random mac */
+		PMD_DRV_LOG(ERR, "wrong format of mac: %s", mac);
+	}
+}
+
+static int
+virtio_vdev_init(struct rte_eth_dev_data *data, char *path,
+		 int queues, int nb_cq __attribute__ ((unused)),
+		 int queue_size, const char *mac, char *ifname)
+{
+	struct stat s;
+	struct virtio_hw *hw = data->dev_private;
+	struct virtio_user_hw *uhw = hw->vdev_private;
+
+	uhw->data = data;
+	strncpy(uhw->path, path, PATH_MAX);
+	uhw->max_queue_pairs = queues;
+	uhw->queue_size = queue_size;
+	uhw->mac_specified = 0;
+	parse_mac(uhw, mac);
+	uhw->vhostfd = -1;
+	uhw->tapfd = -1;
+
+	/* TODO: cq */
+
+	if (stat(uhw->path, &s) < 0) {
+		PMD_INIT_LOG(ERR, "stat: %s failed, %s", uhw->path,
+			     strerror(errno));
+		return -1;
+	}
+
+	switch (s.st_mode & S_IFMT) {
+	case S_IFCHR:
+		uhw->type = VHOST_KERNEL;
+		uhw->vhostfd = vhost_kernel_setup(uhw->path, ifname,
+						  &uhw->tapfd);
+		break;
+	case S_IFSOCK:
+		uhw->type = VHOST_USER;
+		uhw->vhostfd = vhost_user_setup(uhw->path);
+		break;
+	default:
+		PMD_INIT_LOG(ERR, "unknown file type of %s", uhw->path);
+		return -1;
+	}
+	if (uhw->vhostfd < 0) {
+		PMD_INIT_LOG(ERR, "backend set up fails");
+		return -1;
+	}
+	if (vhost_call(uhw->vhostfd, uhw->type,
+			VHOST_MSG_SET_OWNER, NULL) < 0) {
+		PMD_INIT_LOG(ERR, "set_owner fails: %s", strerror(errno));
+		return -1;
+	}
+
+	if (vhost_call(uhw->vhostfd, uhw->type,
+			VHOST_MSG_GET_FEATURES, &uhw->features) < 0) {
+		PMD_INIT_LOG(ERR, "get_features failed: %s", strerror(errno));
+		return -1;
+	}
+	if (uhw->mac_specified)
+		uhw->features |= (1ull << VIRTIO_NET_F_MAC);
+	/* disable it until we support CQ */
+	uhw->features &= ~(1ull << VIRTIO_NET_F_CTRL_VQ);
+	uhw->features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
+
+	return 0;
+
+}
+
+static void
+virtio_vdev_uninit(struct rte_eth_dev_data *data)
+{
+	struct virtio_hw *hw = data->dev_private;
+	struct virtio_user_hw *uhw = hw->vdev_private;
+
+	if (uhw->type == VHOST_KERNEL)
+		close(uhw->tapfd);
+	/* TODO: need update when enable mq */
+	close(uhw->callfds[0]);
+	close(uhw->kickfds[0]);
+
+	close(uhw->vhostfd);
+}
+
+static const char *valid_args[] = {
+#define VIRTIO_USER_ARG_QUEUES_NUM     "queues"
+	VIRTIO_USER_ARG_QUEUES_NUM,
+#define VIRTIO_USER_ARG_CQ_NUM         "cq"
+	VIRTIO_USER_ARG_CQ_NUM,
+#define VIRTIO_USER_ARG_MAC            "mac"
+	VIRTIO_USER_ARG_MAC,
+#define VIRTIO_USER_ARG_PATH           "path"
+	VIRTIO_USER_ARG_PATH,
+#define VIRTIO_USER_ARG_QUEUE_SIZE     "queue_size"
+	VIRTIO_USER_ARG_QUEUE_SIZE,
+#define VIRTIO_USER_ARG_IFNAME         "ifname"
+	VIRTIO_USER_ARG_IFNAME,
+	NULL
+};
+
+#define VIRTIO_USER_DEF_CQ_EN	0
+#define VIRTIO_USER_DEF_Q_NUM	1
+#define VIRTIO_USER_DEF_Q_SZ	256
+
+static int
+get_string_arg(const char *key __rte_unused,
+	       const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(char **)extra_args = strdup(value);
+
+	return 0;
+}
+
+static int
+get_integer_arg(const char *key __rte_unused,
+		const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(uint64_t *)extra_args = strtoull(value, NULL, 0);
+
+	return 0;
+}
+
+static struct rte_eth_dev *
+virtio_user_eth_dev_alloc(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct rte_eth_dev_data *data;
+	struct virtio_hw *hw;
+	struct virtio_user_hw *uhw;
+
+	eth_dev = rte_eth_dev_allocate(name, RTE_ETH_DEV_VIRTUAL);
+	if (!eth_dev) {
+		PMD_INIT_LOG(ERR, "cannot alloc rte_eth_dev");
+		return NULL;
+	}
+
+	data = eth_dev->data;
+
+	hw = rte_zmalloc(NULL, sizeof(*hw), 0);
+	if (!hw) {
+		PMD_INIT_LOG(ERR, "malloc virtio_hw failed");
+		rte_eth_dev_release_port(eth_dev);
+		return NULL;
+	}
+
+	uhw = rte_zmalloc(NULL, sizeof(*uhw), 0);
+	if (!uhw) {
+		PMD_INIT_LOG(ERR, "malloc virtio_user_hw failed");
+		rte_eth_dev_release_port(eth_dev);
+		rte_free(hw);
+		return NULL;
+	}
+
+	hw->vtpci_ops = &vdev_ops;
+	hw->use_msix = 0;
+	hw->modern   = 0;
+	hw->vdev_private = uhw;
+	data->dev_private = hw;
+	data->numa_node = SOCKET_ID_ANY;
+	data->kdrv = RTE_KDRV_NONE;
+	data->dev_flags = RTE_ETH_DEV_DETACHABLE;
+	eth_dev->pci_dev = NULL;
+	eth_dev->driver = NULL;
+	return eth_dev;
+}
+
+/* Dev initialization routine. Invoked once for each virtio vdev at
+ * EAL init time, see rte_eal_dev_init().
+ * Returns 0 on success.
+ */
+static int
+rte_virtio_user_pmd_devinit(const char *name, const char *params)
+{
+	struct rte_kvargs *kvlist = NULL;
+	struct rte_eth_dev *eth_dev = NULL;
+	uint64_t queues = VIRTIO_USER_DEF_Q_NUM;
+	uint64_t nb_cq = VIRTIO_USER_DEF_CQ_EN;
+	uint64_t queue_size = VIRTIO_USER_DEF_Q_SZ;
+	char *path = NULL;
+	char *mac_addr = NULL;
+	char *ifname = NULL;
+	int ret = -1;
+
+	if (!params || params[0] == '\0') {
+		PMD_INIT_LOG(ERR, "arg %s is mandatory for virtio-user",
+			  VIRTIO_USER_ARG_QUEUE_SIZE);
+		goto end;
+	}
+
+	kvlist = rte_kvargs_parse(params, valid_args);
+	if (!kvlist) {
+		PMD_INIT_LOG(ERR, "error when parsing param");
+		goto end;
+	}
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_PATH) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_PATH,
+				   &get_string_arg, &path);
+	else {
+		PMD_INIT_LOG(ERR, "arg %s is mandatory for virtio-user\n",
+			  VIRTIO_USER_ARG_QUEUE_SIZE);
+		goto end;
+	}
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_MAC) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_MAC,
+				   &get_string_arg, &mac_addr);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_IFNAME) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_IFNAME,
+				   &get_string_arg, &ifname);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_QUEUE_SIZE) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_QUEUE_SIZE,
+				   &get_integer_arg, &queue_size);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_QUEUES_NUM) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_QUEUES_NUM,
+				   &get_integer_arg, &queues);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_CQ_NUM) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_CQ_NUM,
+				   &get_integer_arg, &nb_cq);
+
+	eth_dev = virtio_user_eth_dev_alloc(name);
+	if (!eth_dev) {
+		PMD_INIT_LOG(ERR, "virtio-user fails to alloc device");
+		goto end;
+	}
+
+	if (virtio_vdev_init(eth_dev->data, path, queues, nb_cq,
+			     queue_size, mac_addr, ifname) < 0)
+		goto end;
+
+	/* previously called by rte_eal_pci_probe() for physical dev */
+	if (eth_virtio_dev_init(eth_dev) < 0) {
+		PMD_INIT_LOG(ERR, "eth_virtio_dev_init fails");
+		goto end;
+	}
+	ret = 0;
+
+end:
+	if (path)
+		free(path);
+	if (mac_addr)
+		free(mac_addr);
+	if (ifname)
+		free(ifname);
+	return ret;
+}
+
+/** Called by rte_eth_dev_detach() */
+static int
+rte_virtio_user_pmd_devuninit(const char *name)
+{
+	struct rte_eth_dev *eth_dev = NULL;
+
+	if (!name)
+		return -EINVAL;
+
+	PMD_DRV_LOG(INFO, PMD, "Un-Initializing %s\n", name);
+	eth_dev = rte_eth_dev_allocated(name);
+	if (!eth_dev)
+		return -ENODEV;
+
+	/* make sure the device is stopped, queues freed */
+	rte_eth_dev_close(eth_dev->data->port_id);
+
+	virtio_vdev_uninit(eth_dev->data);
+
+	rte_free(eth_dev->data->dev_private);
+	rte_free(eth_dev->data);
+	rte_eth_dev_release_port(eth_dev);
+
+	return 0;
+}
+
+static struct rte_driver rte_virtio_user_driver = {
+	.name   = "virtio-user",
+	.type   = PMD_VDEV,
+	.init   = rte_virtio_user_pmd_devinit,
+	.uninit = rte_virtio_user_pmd_devuninit,
+};
+
+PMD_REGISTER_DRIVER(rte_virtio_user_driver);
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v4 8/8] doc: update doc for virtio-user
  2016-04-29  1:18 ` [PATCH v4 0/8] virtio support for container Jianfeng Tan
                     ` (6 preceding siblings ...)
  2016-04-29  1:18   ` [PATCH v4 7/8] virtio-user: add a new virtual device named virtio-user Jianfeng Tan
@ 2016-04-29  1:18   ` Jianfeng Tan
  2016-04-29  1:35   ` [PATCH v4 0/8] virtio support for container Tan, Jianfeng
  8 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-04-29  1:18 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, michael.qiu, ann.zhuangyanying,
	mukawa, nhorman

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-By: Neil Horman <nhorman@tuxdriver.com>
---
 doc/guides/nics/overview.rst           | 64 +++++++++++++++++-----------------
 doc/guides/rel_notes/release_16_07.rst |  4 +++
 2 files changed, 36 insertions(+), 32 deletions(-)

diff --git a/doc/guides/nics/overview.rst b/doc/guides/nics/overview.rst
index f08039e..92e7468 100644
--- a/doc/guides/nics/overview.rst
+++ b/doc/guides/nics/overview.rst
@@ -74,40 +74,40 @@ Most of these differences are summarized below.
 
 .. table:: Features availability in networking drivers
 
-   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
-   Feature              a b b b c e e e i i i i i i i i i i f f f f m m m n n p r s v v v v x
-                        f n n o x 1 n n 4 4 4 4 g g x x x x m m m m l l p f u c i z h i i m e
-                        p x x n g 0 a i 0 0 0 0 b b g g g g 1 1 1 1 x x i p l a n e o r r x n
-                        a 2 2 d b 0   c e e e e   v b b b b 0 0 0 0 4 5 p   l p g d s t t n v
-                        c x x i e 0       . v v   f e e e e k k k k     e         a t i i e i
-                        k   v n           . f f       . v v   . v v               t   o o t r
-                        e   f g           .   .       . f f   . f f               a     . 3 t
-                        t                 v   v       v   v   v   v               2     v
-                                          e   e       e   e   e   e                     e
-                                          c   c       c   c   c   c                     c
-   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
+   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
+   Feature              a b b b c e e e i i i i i i i i i i f f f f m m m n n p r s v v v v v x
+                        f n n o x 1 n n 4 4 4 4 g g x x x x m m m m l l p f u c i z h i i i m e
+                        p x x n g 0 a i 0 0 0 0 b b g g g g 1 1 1 1 x x i p l a n e o r r r x n
+                        a 2 2 d b 0   c e e e e   v b b b b 0 0 0 0 4 5 p   l p g d s t t t n v
+                        c x x i e 0       . v v   f e e e e k k k k     e         a t i i i e i
+                        k   v n           . f f       . v v   . v v               t   o o o t r
+                        e   f g           .   .       . f f   . f f               a     . u 3 t
+                        t                 v   v       v   v   v   v               2     v s
+                                          e   e       e   e   e   e                     e e
+                                          c   c       c   c   c   c                     c r
+   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
    Speed capabilities
-   Link status            Y Y   Y Y   Y Y Y     Y   Y Y Y Y         Y Y           Y Y Y Y
+   Link status            Y Y   Y Y   Y Y Y     Y   Y Y Y Y         Y Y           Y Y Y Y Y
    Link status event      Y Y     Y     Y Y     Y   Y Y             Y Y             Y
    Queue status event                                                               Y
    Rx interrupt                   Y     Y Y Y Y Y Y Y Y Y Y Y Y Y Y
-   Queue start/stop             Y   Y Y Y Y Y Y     Y Y     Y Y Y Y Y Y           Y   Y Y
+   Queue start/stop             Y   Y Y Y Y Y Y     Y Y     Y Y Y Y Y Y           Y   Y Y Y
    MTU update                   Y Y Y           Y   Y Y Y Y         Y Y
    Jumbo frame                  Y Y Y Y Y Y Y Y Y   Y Y Y Y Y Y Y Y Y Y       Y
-   Scattered Rx                 Y Y Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y           Y   Y
+   Scattered Rx                 Y Y Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y           Y   Y   Y
    LRO                                              Y Y Y Y
    TSO                          Y   Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y
-   Promiscuous mode       Y Y   Y Y   Y Y Y Y Y Y Y Y Y     Y Y     Y Y           Y   Y Y
-   Allmulticast mode            Y Y     Y Y Y Y Y Y Y Y Y Y Y Y     Y Y           Y   Y Y
-   Unicast MAC filter     Y Y     Y   Y Y Y Y Y Y Y Y Y Y Y Y Y     Y Y               Y Y
-   Multicast MAC filter   Y Y         Y Y Y Y Y             Y Y     Y Y               Y Y
+   Promiscuous mode       Y Y   Y Y   Y Y Y Y Y Y Y Y Y     Y Y     Y Y           Y   Y Y Y
+   Allmulticast mode            Y Y     Y Y Y Y Y Y Y Y Y Y Y Y     Y Y           Y   Y Y Y
+   Unicast MAC filter     Y Y     Y   Y Y Y Y Y Y Y Y Y Y Y Y Y     Y Y               Y Y Y
+   Multicast MAC filter   Y Y         Y Y Y Y Y             Y Y     Y Y               Y Y Y
    RSS hash                     Y   Y Y Y Y Y Y Y   Y Y Y Y Y Y Y Y Y Y
    RSS key update                   Y   Y Y Y Y Y   Y Y Y Y Y Y Y Y   Y
    RSS reta update                  Y   Y Y Y Y Y   Y Y Y Y Y Y Y Y   Y
    VMDq                                 Y Y     Y   Y Y     Y Y
    SR-IOV                   Y       Y   Y Y     Y   Y Y             Y Y
    DCB                                  Y Y     Y   Y Y
-   VLAN filter                    Y   Y Y Y Y Y Y Y Y Y Y Y Y Y     Y Y               Y Y
+   VLAN filter                    Y   Y Y Y Y Y Y Y Y Y Y Y Y Y     Y Y               Y Y Y
    Ethertype filter                     Y Y     Y   Y Y
    N-tuple filter                               Y   Y Y
    SYN filter                                   Y   Y Y
@@ -127,26 +127,26 @@ Most of these differences are summarized below.
    Inner L4 checksum                Y   Y   Y       Y   Y           Y
    Packet type parsing          Y     Y Y   Y   Y Y Y   Y   Y Y Y Y Y Y
    Timesync                             Y Y     Y   Y Y
-   Basic stats            Y Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y       Y   Y Y Y Y
-   Extended stats                   Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y                   Y Y
-   Stats per queue              Y                   Y Y     Y Y Y Y Y Y           Y   Y Y
+   Basic stats            Y Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y       Y   Y Y Y Y Y
+   Extended stats                   Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y                   Y Y Y
+   Stats per queue              Y                   Y Y     Y Y Y Y Y Y           Y   Y Y Y
    EEPROM dump                                  Y   Y Y
    Registers dump                               Y Y Y Y Y Y
    Multiprocess aware                   Y Y Y Y     Y Y Y Y Y Y Y Y Y Y       Y
-   BSD nic_uio                  Y Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y                   Y Y
-   Linux UIO              Y Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y                   Y Y
-   Linux VFIO                   Y Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y                   Y Y
+   BSD nic_uio                  Y Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y                   Y Y Y
+   Linux UIO              Y Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y                   Y Y Y
+   Linux VFIO                   Y Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y                   Y Y Y
    Other kdrv                                                       Y Y           Y
-   ARMv7                                                                      Y       Y Y
-   ARMv8                                                                      Y       Y Y
+   ARMv7                                                                      Y       Y Y Y
+   ARMv8                                                                      Y       Y Y Y
    Power8                                                           Y Y       Y
    TILE-Gx                                                                    Y
-   x86-32                       Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y       Y     Y Y Y
-   x86-64                 Y Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y       Y   Y Y Y Y
-   Usage doc              Y Y   Y     Y                             Y Y       Y   Y   Y
+   x86-32                       Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y       Y     Y Y Y Y
+   x86-64                 Y Y   Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y Y       Y   Y Y Y Y Y
+   Usage doc              Y Y   Y     Y                             Y Y       Y   Y   Y   Y
    Design doc
    Perf doc
-   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
+   ==================== = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = = =
 
 .. Note::
 
diff --git a/doc/guides/rel_notes/release_16_07.rst b/doc/guides/rel_notes/release_16_07.rst
index 83c841b..30d74f8 100644
--- a/doc/guides/rel_notes/release_16_07.rst
+++ b/doc/guides/rel_notes/release_16_07.rst
@@ -34,6 +34,10 @@ This section should contain new features added in this release. Sample format:
 
   Refer to the previous release notes for examples.
 
+* **Virtio support for containers.**
+
+  Add a new virtual device, named virtio-user, to support virtio for containers.
+
 
 Resolved Issues
 ---------------
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* Re: [PATCH v4 0/8] virtio support for container
  2016-04-29  1:18 ` [PATCH v4 0/8] virtio support for container Jianfeng Tan
                     ` (7 preceding siblings ...)
  2016-04-29  1:18   ` [PATCH v4 8/8] doc: update doc for virtio-user Jianfeng Tan
@ 2016-04-29  1:35   ` Tan, Jianfeng
  8 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-04-29  1:35 UTC (permalink / raw)
  To: dev
  Cc: Xie, Huawei, rich.lane, yuanhan.liu, mst, nakajima.yoshihiro,
	p.fedin, Qiu, Michael, ann.zhuangyanying, mukawa, nhorman

Sorry, forget to mention, this patchset depends on:
  - [PATCH v2] virtio: fix modify drv_flags for specific device
  - [PATCH v3 0/2] virtio: fix memory leak of virtqueue memzones

Thanks,
Jianfeng


> -----Original Message-----
> From: Tan, Jianfeng
> Sent: Friday, April 29, 2016 9:18 AM
> To: dev@dpdk.org
> Cc: Tan, Jianfeng; Xie, Huawei; rich.lane@bigswitch.com;
> yuanhan.liu@linux.intel.com; mst@redhat.com;
> nakajima.yoshihiro@lab.ntt.co.jp; p.fedin@samsung.com; Qiu, Michael;
> ann.zhuangyanying@huawei.com; mukawa@igel.co.jp;
> nhorman@tuxdriver.com
> Subject: [PATCH v4 0/8] virtio support for container
> 
> v4:
>  - Avoid using dev_type, instead use (eth_dev->pci_device is NULL) to
>    judge if it's virtual device or physical device.
>  - Change the added device name to virtio-user.
>  - Split into vhost_user.c, vhost_kernel.c, vhost.c, virtio_user_pci.c,
>    virtio_user_dev.c.
>  - Move virtio-user specific data from struct virtio_hw into struct
>    virtio_user_hw.
>  - Add support to send reset_owner message.
>  - Change del_queue implementation. (This need more check)
>  - Remove rte_panic(), and superseded with log.
>  - Add reset_owner into virtio_pci_ops.reset.
>  - Merge parameter "rx" and "tx" to "queues" to emliminate confusion.
>  - Move get_features to after set_owner.
>  - Redefine path in virtio_user_hw from char * to char [].
> 
> v3:
>  - Remove --single-file option; do no change at EAL memory.
>  - Remove the added API rte_eal_get_backfile_info(), instead we check all
>    opened files with HUGEFILE_FMT to find hugepage files owned by DPDK.
>  - Accordingly, add more restrictions at "Known issue" section.
>  - Rename parameter from queue_num to queue_size for confusion.
>  - Rename vhost_embedded.c to rte_eth_virtio_vdev.c.
>  - Move code related to the newly added vdev to rte_eth_virtio_vdev.c, to
>    reuse eth_virtio_dev_init(), remove its static declaration.
>  - Implement dev_uninit() for rte_eth_dev_detach().
>  - WARN -> ERR, in vhost_embedded.c
>  - Add more commit message for clarify the model.
> 
> v2:
>  - Rebase on the patchset of virtio 1.0 support.
>  - Fix cannot create non-hugepage memory.
>  - Fix wrong size of memory region when "single-file" is used.
>  - Fix setting of offset in virtqueue to use virtual address.
>  - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
>  - Add mac option to specify the mac address of this virtual device.
>  - Update doc.
> 
> This patchset is to provide high performance networking interface (virtio)
> for container-based DPDK applications. The way of starting DPDK apps in
> containers with ownership of NIC devices exclusively is beyond the scope.
> The basic idea here is to present a new virtual device (named virtio-user),
> which can be discovered and initialized by DPDK. To minimize the change,
> we reuse already-existing virtio PMD code (driver/net/virtio/).
> 
> Background: Previously, we usually use a virtio device in the context of
> QEMU/VM as below pic shows. Virtio nic is emulated in QEMU, and usually
> presented in VM as a PCI device.
> 
>   ------------------
>   |  virtio driver |  ----->  VM
>   ------------------
>         |
>         | ----------> (over PCI bus or MMIO or Channel I/O)
>         |
>   ------------------
>   | device emulate |
>   |                |  ----->  QEMU
>   | vhost adapter  |
>   ------------------
>         |
>         | ----------> (vhost-user protocol or vhost-net ioctls)
>         |
>   ------------------
>   | vhost backend  |
>   ------------------
> 
> Compared to QEMU/VM case, virtio support for contaner requires to
> embedded
> device framework inside the virtio PMD. So this converged driver actually
> plays three roles:
>   - virtio driver to drive this new kind of virtual device;
>   - device emulation to present this virtual device and reponse to the
>     virtio driver, which is originally by QEMU;
>   - and the role to communicate with vhost backend, which is also
>     originally by QEMU.
> 
> The code layout and functionality of each module:
> 
>   ----------------------
>   | ------------------ |
>   | | virtio driver  | |----> (virtio_user_pci.c)
>   | ------------------ |
>   |         |          |
>   | ------------------ | ------>  virtio-user PMD
>   | | device emulate |-|----> (virtio_user_dev.c)
>   | |                | |
>   | | vhost adapter  |-|----> (vhost_user.c, vhost_kernel.c, vhost.c)
>   | ------------------ |
>   ----------------------
>          |
>          | -------------- --> (vhost-user protocol or vhost-net ioctls)
>          |
>    ------------------
>    | vhost backend  |
>    ------------------
> 
> How to share memory? In VM's case, qemu always shares all physical layout
> to backend. But it's not feasible for a container, as a process, to share
> all virtual memory regions to backend. So only specified virtual memory
> regions (with type of shared) are sent to backend. It's a limitation that
> only addresses in these areas can be used to transmit or receive packets.
> 
> Known issues:
>  - Control queue and multi-queue are not supported yet.
>  - Cannot work with --huge-unlink.
>  - Cannot work with no-huge.
>  - Cannot work when there are more than
> VHOST_MEMORY_MAX_NREGIONS(8)
>    hugepages.
>  - Root privilege is a must (mainly becase of sorting hugepages according
>    to physical address).
>  - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").
> 
> How to use?
> 
> a. Apply this patchset.
> 
> b. To compile container apps:
> $: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> 
> c. To build a docker image using Dockerfile below.
> $: cat ./Dockerfile
> FROM ubuntu:latest
> WORKDIR /usr/src/dpdk
> COPY . /usr/src/dpdk
> ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
> $: docker build -t dpdk-app-l2fwd .
> 
> d. Used with vhost-user
> $: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
> 	--socket-mem 1024,1024 -- -p 0x1 --stats 1
> $: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
> 	-v /dev/hugepages:/dev/hugepages \
> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> 	--vdev=virtio-user0,path=/var/run/usvhost -- -p 0x1
> 
> f. Used with vhost-net
> $: modprobe vhost
> $: modprobe vhost-net
> $: docker run -i -t --privileged \
> 	-v /dev/vhost-net:/dev/vhost-net \
> 	-v /dev/net/tun:/dev/net/tun \
> 	-v /dev/hugepages:/dev/hugepages \
> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> 	--vdev=virtio-user0,path=/dev/vhost-net -- -p 0x1
> 
> By the way, it's not necessary to run in a container.
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> Acked-By: Neil Horman <nhorman@tuxdriver.com>
> 
> Jianfeng Tan (8):
>   virtio: hide phys addr check inside pci ops
>   virtio: abstract vring hdr desc init as a method
>   virtio: enable use virtual address to fill desc
>   virtio-user: add vhost adapter layer
>   virtio-user: add device emulation layer APIs
>   virtio-user: add new virtual pci driver for virtio
>   virtio-user: add a new virtual device named virtio-user
>   doc: update doc for virtio-user
> 
>  config/common_linuxapp                           |   3 +
>  doc/guides/nics/overview.rst                     |  64 +--
>  doc/guides/rel_notes/release_16_07.rst           |   4 +
>  drivers/net/virtio/Makefile                      |   8 +
>  drivers/net/virtio/virtio_ethdev.c               |  69 ++--
>  drivers/net/virtio/virtio_ethdev.h               |   2 +
>  drivers/net/virtio/virtio_pci.c                  |  30 +-
>  drivers/net/virtio/virtio_pci.h                  |   3 +-
>  drivers/net/virtio/virtio_rxtx.c                 |   5 +-
>  drivers/net/virtio/virtio_rxtx_simple.c          |  13 +-
>  drivers/net/virtio/virtio_user/vhost.c           | 105 +++++
>  drivers/net/virtio/virtio_user/vhost.h           | 221 +++++++++++
>  drivers/net/virtio/virtio_user/vhost_kernel.c    | 254 ++++++++++++
>  drivers/net/virtio/virtio_user/vhost_user.c      | 375 ++++++++++++++++++
>  drivers/net/virtio/virtio_user/virtio_user_dev.c | 475
> +++++++++++++++++++++++
>  drivers/net/virtio/virtio_user/virtio_user_dev.h |  61 +++
>  drivers/net/virtio/virtio_user/virtio_user_pci.c | 209 ++++++++++
>  drivers/net/virtio/virtqueue.h                   |  33 +-
>  18 files changed, 1849 insertions(+), 85 deletions(-)
>  create mode 100644 drivers/net/virtio/virtio_user/vhost.c
>  create mode 100644 drivers/net/virtio/virtio_user/vhost.h
>  create mode 100644 drivers/net/virtio/virtio_user/vhost_kernel.c
>  create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c
>  create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
>  create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h
>  create mode 100644 drivers/net/virtio/virtio_user/virtio_user_pci.c
> 
> --
> 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v4 1/8] virtio: hide phys addr check inside pci ops
  2016-04-29  1:18   ` [PATCH v4 1/8] virtio: hide phys addr check inside pci ops Jianfeng Tan
@ 2016-05-11 23:05     ` Yuanhan Liu
  0 siblings, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-05-11 23:05 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	michael.qiu, ann.zhuangyanying, mukawa, nhorman

On Fri, Apr 29, 2016 at 01:18:29AM +0000, Jianfeng Tan wrote:
> This patch is to move phys addr check from virtio_dev_queue_setup
> to pci ops. To makt that happen, make sure virtio_ops.setup_queue
> return the result if we pass through the check.
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> Acked-By: Neil Horman <nhorman@tuxdriver.com>

This patch doesn't even exist in your old versions; it doesn't make
sense to carry the Ack here.

Besides that, this patch looks good to me:

Acked-by: Yuanhan Liu <yuanhan.liu@linux.intel.com>

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v4 5/8] virtio-user: add device emulation layer APIs
  2016-04-29  1:18   ` [PATCH v4 5/8] virtio-user: add device emulation layer APIs Jianfeng Tan
@ 2016-05-12  1:05     ` Yuanhan Liu
  0 siblings, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-05-12  1:05 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

On Fri, Apr 29, 2016 at 01:18:33AM +0000, Jianfeng Tan wrote:
> +static int
> +kick_one_vq(struct virtio_user_hw *hw, struct virtqueue *vq,
> +	    unsigned queue_sel)

Firstly, "dev" is a more common word than "hw" here. So, name the struct
to "virtio_user_device", and name the var to "dev".

And you are still mixing virtio driver and virtio device emulation here.
Say, struct virtqueue should not be used here: it belongs to driver.
Instead, you should define your own. It could be fair simple that is
enough for current usage:

struct virtqueue {
	uint32_t num;
	uint64_t desc_addr;
	uint64_t avail_addr;
	uint64_t used_addr;
};

That could let us not depend on any structures (or just few if not
possible) from virtio PMD driver.

	--yliu
> +{
> +	int callfd, kickfd;
> +	struct vhost_vring_file file;
> +	struct vhost_vring_state state;
> +	struct vhost_vring_addr addr = {
> +		.index = queue_sel,
> +		.desc_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.desc,
> +		.avail_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.avail,
> +		.used_user_addr = (uint64_t)(uintptr_t)vq->vq_ring.used,
> +		.log_guest_addr = 0,
> +		.flags = 0, /* disable log */
> +	};

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio
  2016-04-29  1:18   ` [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
@ 2016-05-12  2:12     ` Yuanhan Liu
  2016-05-12  7:08       ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2016-05-12  2:12 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

On Fri, Apr 29, 2016 at 01:18:34AM +0000, Jianfeng Tan wrote:
> +static void
> +vdev_read_dev_config(struct virtio_hw *hw, uint64_t offset,
> +		     void *dst, int length)
> +{
> +	int i;
> +	struct virtio_user_hw *uhw = (struct virtio_user_hw *)hw->vdev_private;

Unnecessary cast.

> +static int
> +vdev_setup_queue(struct virtio_hw *hw __rte_unused, struct virtqueue *vq)
> +{
> +	/* Changed to use virtual addr */
> +	vq->vq_ring_mem = (phys_addr_t)vq->mz->addr;
> +	if (vq->virtio_net_hdr_mz) {
> +		vq->virtio_net_hdr_mem =
> +			(phys_addr_t)vq->virtio_net_hdr_mz->addr;
> +		/* Do it one more time after we reset virtio_net_hdr_mem */
> +		vring_hdr_desc_init(vq);
> +	}
> +	vq->offset = offsetof(struct rte_mbuf, buf_addr);
> +	return 0;

Here as last email said, you should not mix vq stuff. What's more,
why do you invoke vring_hdr_desc_init() here? If it needs a special
handling, do it in driver.

The "setup_queue" method is actually for telling the device where desc,
avail and used vring are located. Hence, the implementation could be simple:
just log them.

> +
> +const struct virtio_pci_ops vdev_ops = {

Note that this is the interface for the driver to talk to the device,
we should put this file into upper layer then, in the driver.

And let me make a summary, trying to make it clear:

- We should not use any structures/functions from the virtio driver
  here, unless it's really a must.

- It's allowed for driver to make *few* special handling for the virtio
  user device. And that's what the driver supposed to do: to handle
  different device variants.

  So, I think it's okay to export the virtio_user_device struct to 
  driver and do all those kind of "fake pci" configration there.

	--yliu

> +	.read_dev_cfg	= vdev_read_dev_config,
> +	.write_dev_cfg	= vdev_write_dev_config,
> +	.reset		= vdev_reset,
> +	.get_status	= vdev_get_status,
> +	.set_status	= vdev_set_status,
> +	.get_features	= vdev_get_features,
> +	.set_features	= vdev_set_features,
> +	.get_isr	= vdev_get_isr,
> +	.set_config_irq	= vdev_set_config_irq,
> +	.get_queue_num	= vdev_get_queue_num,
> +	.setup_queue	= vdev_setup_queue,
> +	.del_queue	= vdev_del_queue,
> +	.notify_queue	= vdev_notify_queue,
> +};
> -- 
> 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio
  2016-05-12  2:12     ` Yuanhan Liu
@ 2016-05-12  7:08       ` Tan, Jianfeng
  2016-05-12 16:40         ` Yuanhan Liu
  2016-05-12 17:02         ` Michael S. Tsirkin
  0 siblings, 2 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-05-12  7:08 UTC (permalink / raw)
  To: Yuanhan Liu
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

Hi yuanhan,


On 5/12/2016 10:12 AM, Yuanhan Liu wrote:
> On Fri, Apr 29, 2016 at 01:18:34AM +0000, Jianfeng Tan wrote:
>> +static void
>> +vdev_read_dev_config(struct virtio_hw *hw, uint64_t offset,
>> +		     void *dst, int length)
>> +{
>> +	int i;
>> +	struct virtio_user_hw *uhw = (struct virtio_user_hw *)hw->vdev_private;
> Unnecessary cast.

Yes.

>
>> +static int
>> +vdev_setup_queue(struct virtio_hw *hw __rte_unused, struct virtqueue *vq)
>> +{
>> +	/* Changed to use virtual addr */
>> +	vq->vq_ring_mem = (phys_addr_t)vq->mz->addr;
>> +	if (vq->virtio_net_hdr_mz) {
>> +		vq->virtio_net_hdr_mem =
>> +			(phys_addr_t)vq->virtio_net_hdr_mz->addr;
>> +		/* Do it one more time after we reset virtio_net_hdr_mem */
>> +		vring_hdr_desc_init(vq);
>> +	}
>> +	vq->offset = offsetof(struct rte_mbuf, buf_addr);
>> +	return 0;
> Here as last email said, you should not mix vq stuff. What's more,
> why do you invoke vring_hdr_desc_init() here?

vring_hdr_desc_init() is to init header desc according to 
vq->virtio_net_hdr_mem, and here we change to use virtual address, so we 
need to invoke this after vq->virtio_net_hdr_mem is decided.

But for this case, you remind me that we can achieve that by: inside 
virtio_dev_queue_setup(), move vring_hdr_desc_init() after setup_queue().

> If it needs a special
> handling, do it in driver.

As discussed in previous mail with David, we should hide special 
handling inside pci ops, such as real virtio device needs to check 
address (patch 1). See below comments for more detail.

>
> The "setup_queue" method is actually for telling the device where desc,
> avail and used vring are located. Hence, the implementation could be simple:
> just log them.

>
>> +
>> +const struct virtio_pci_ops vdev_ops = {
> Note that this is the interface for the driver to talk to the device,
> we should put this file into upper layer then, in the driver.
>
> And let me make a summary, trying to make it clear:
>
> - We should not use any structures/functions from the virtio driver
>    here, unless it's really a must.

Firstly I agree this point (although I see a difference in how we take 
"a must"). My original principle is to maximize the use of existing 
structures instead of maintain any new ones. And I already give up that 
principle when I accept your previous suggestion to use struct 
virtio_user_device to store virtio-user specific fields. So I agree to 
add the new struct virtqueue to avoid use of driver-layer virtqueues.

>
> - It's allowed for driver to make *few* special handling for the virtio
>    user device. And that's what the driver supposed to do: to handle
>    different device variants.

So here are two contradictory ways. Compared to the way you suggest,  
another way is to keep a unified driver and maintain all special 
handling inside struct virtio_pci_ops.

I prefer the latter because:
(1) Special handling for each kind of device will be gather together 
instead of scattered everywhere of driver code.
(2) It's more aligned to previous logic to hide the detail to 
differentiate modern/legacy device.


Thanks,
Jianfeng


>
>    So, I think it's okay to export the virtio_user_device struct to
>    driver and do all those kind of "fake pci" configration there.
>
> 	--yliu
>
>> +	.read_dev_cfg	= vdev_read_dev_config,
>> +	.write_dev_cfg	= vdev_write_dev_config,
>> +	.reset		= vdev_reset,
>> +	.get_status	= vdev_get_status,
>> +	.set_status	= vdev_set_status,
>> +	.get_features	= vdev_get_features,
>> +	.set_features	= vdev_set_features,
>> +	.get_isr	= vdev_get_isr,
>> +	.set_config_irq	= vdev_set_config_irq,
>> +	.get_queue_num	= vdev_get_queue_num,
>> +	.setup_queue	= vdev_setup_queue,
>> +	.del_queue	= vdev_del_queue,
>> +	.notify_queue	= vdev_notify_queue,
>> +};
>> -- 
>> 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio
  2016-05-12  7:08       ` Tan, Jianfeng
@ 2016-05-12 16:40         ` Yuanhan Liu
  2016-05-13  1:54           ` Tan, Jianfeng
  2016-05-12 17:02         ` Michael S. Tsirkin
  1 sibling, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2016-05-12 16:40 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

On Thu, May 12, 2016 at 03:08:05PM +0800, Tan, Jianfeng wrote:
> >>+static int
> >>+vdev_setup_queue(struct virtio_hw *hw __rte_unused, struct virtqueue *vq)
> >>+{
> >>+	/* Changed to use virtual addr */
> >>+	vq->vq_ring_mem = (phys_addr_t)vq->mz->addr;
> >>+	if (vq->virtio_net_hdr_mz) {
> >>+		vq->virtio_net_hdr_mem =
> >>+			(phys_addr_t)vq->virtio_net_hdr_mz->addr;
> >>+		/* Do it one more time after we reset virtio_net_hdr_mem */
> >>+		vring_hdr_desc_init(vq);
> >>+	}
> >>+	vq->offset = offsetof(struct rte_mbuf, buf_addr);
> >>+	return 0;
> >Here as last email said, you should not mix vq stuff. What's more,
> >why do you invoke vring_hdr_desc_init() here?
> 
> vring_hdr_desc_init() is to init header desc according to
> vq->virtio_net_hdr_mem, and here we change to use virtual address, so we
> need to invoke this after vq->virtio_net_hdr_mem is decided.
> 
> But for this case, you remind me that we can achieve that by: inside
> virtio_dev_queue_setup(), move vring_hdr_desc_init() after setup_queue().
> 
> >If it needs a special
> >handling, do it in driver.
> 
> As discussed in previous mail with David, we should hide special handling
> inside pci ops,

Generally speaking, yes.

> such as real virtio device needs to check address (patch 1).

And that's a good one: I've already acked. But here, I doubt it introduces 
any benefits to do that. Firstly, it's a Tx queue specific setting, moving
it to common code path means you have to add a check like the way you did
in this patch. BTW, it's an implicit check, which hurts readability a bit.

Secondly, you have to do this kind of check/settings in 3 different places: 

- legacy queue_setup() method
- modern queue_setup() method
- your vdev queue_setup() method

And another remind is that Huawei planned to split Rx/Tx queue settings,
here you mixed them again, and I don't think Huawei would like it. Don't
even to say that after the split, the Tx specific stuff will be no longer
in the common vq structure.

So, I'd suggest something like following:

	if (is_vdev(..)) {
		/* comment here that we use VA for vdev */
		vq->vq_ring_mem = (phys_addr_t)vq->mz->addr;
		vq->virtio_net_hdr_mem = ...;
		vq->offset = ...;
	} else {
		vq->vq_ring_mem = ...;
		...
	}
	vring_hdr_desc_init(vq);

> See below comments for more detail.
> 
> >
> >The "setup_queue" method is actually for telling the device where desc,
> >avail and used vring are located. Hence, the implementation could be simple:
> >just log them.
> 
> >
> >>+
> >>+const struct virtio_pci_ops vdev_ops = {
> >Note that this is the interface for the driver to talk to the device,
> >we should put this file into upper layer then, in the driver.
> >
> >And let me make a summary, trying to make it clear:
> >
> >- We should not use any structures/functions from the virtio driver
> >   here, unless it's really a must.
> 
> Firstly I agree this point (although I see a difference in how we take "a
> must"). My original principle is to maximize the use of existing structures
> instead of maintain any new ones.

If that could save you a lot of efforts and make the design clean, I
might would say, yes, go for it. But it's obviously NO in this case.

> And I already give up that principle when
> I accept your previous suggestion to use struct virtio_user_device to store
> virtio-user specific fields. So I agree to add the new struct virtqueue to
> avoid use of driver-layer virtqueues.
> 
> >
> >- It's allowed for driver to make *few* special handling for the virtio
> >   user device. And that's what the driver supposed to do: to handle
> >   different device variants.
> 
> So here are two contradictory ways. Compared to the way you suggest,
> another way is to keep a unified driver and maintain all special handling
> inside struct virtio_pci_ops.
> 
> I prefer the latter because:
> (1) Special handling for each kind of device will be gather together instead
> of scattered everywhere of driver code.
> (2) It's more aligned to previous logic to hide the detail to differentiate
> modern/legacy device.

May I ask how many more such handling are needed, excluding the tx queue
header desc setup? And as stated, in generic, yes, we should try that.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio
  2016-05-12  7:08       ` Tan, Jianfeng
  2016-05-12 16:40         ` Yuanhan Liu
@ 2016-05-12 17:02         ` Michael S. Tsirkin
  2016-05-13  2:00           ` Tan, Jianfeng
  1 sibling, 1 reply; 196+ messages in thread
From: Michael S. Tsirkin @ 2016-05-12 17:02 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: Yuanhan Liu, dev, Huawei Xie, rich.lane, nakajima.yoshihiro,
	p.fedin, ann.zhuangyanying, mukawa, nhorman

On Thu, May 12, 2016 at 03:08:05PM +0800, Tan, Jianfeng wrote:
> (2) It's more aligned to previous logic to hide the detail to differentiate
> modern/legacy device.

Why is there a need to support legacy interfaces at all?  It's a container
so if it's in use one can be reasonably sure you have a new kernel.

-- 
MST

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio
  2016-05-12 16:40         ` Yuanhan Liu
@ 2016-05-13  1:54           ` Tan, Jianfeng
  2016-05-13  4:45             ` Yuanhan Liu
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-05-13  1:54 UTC (permalink / raw)
  To: Yuanhan Liu
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman



On 5/13/2016 12:40 AM, Yuanhan Liu wrote:
> On Thu, May 12, 2016 at 03:08:05PM +0800, Tan, Jianfeng wrote:
>>>> +static int
>>>> +vdev_setup_queue(struct virtio_hw *hw __rte_unused, struct virtqueue *vq)
>>>> +{
>>>> +	/* Changed to use virtual addr */
>>>> +	vq->vq_ring_mem = (phys_addr_t)vq->mz->addr;
>>>> +	if (vq->virtio_net_hdr_mz) {
>>>> +		vq->virtio_net_hdr_mem =
>>>> +			(phys_addr_t)vq->virtio_net_hdr_mz->addr;
>>>> +		/* Do it one more time after we reset virtio_net_hdr_mem */
>>>> +		vring_hdr_desc_init(vq);
>>>> +	}
>>>> +	vq->offset = offsetof(struct rte_mbuf, buf_addr);
>>>> +	return 0;
>>> Here as last email said, you should not mix vq stuff. What's more,
>>> why do you invoke vring_hdr_desc_init() here?
>> vring_hdr_desc_init() is to init header desc according to
>> vq->virtio_net_hdr_mem, and here we change to use virtual address, so we
>> need to invoke this after vq->virtio_net_hdr_mem is decided.
>>
>> But for this case, you remind me that we can achieve that by: inside
>> virtio_dev_queue_setup(), move vring_hdr_desc_init() after setup_queue().
>>
>>> If it needs a special
>>> handling, do it in driver.
>> As discussed in previous mail with David, we should hide special handling
>> inside pci ops,
> Generally speaking, yes.
>
>> such as real virtio device needs to check address (patch 1).
> And that's a good one: I've already acked. But here, I doubt it introduces
> any benefits to do that. Firstly, it's a Tx queue specific setting, moving
> it to common code path means you have to add a check like the way you did
> in this patch.

I agree about this point. But is it really big deal? Please go on with 
remaining comments before make decision.

> BTW, it's an implicit check, which hurts readability a bit.

At the point of readability, I think driver does not need to care about 
devices use physical addresses or virtual addresses, and this is why we 
do that inside pci ops to hide those details.

Another side, pci ops is introduced to cover the details of talking to a 
device as I understand, do you think it offends such semantics? (this 
one could be your argument :-))

>
> Secondly, you have to do this kind of check/settings in 3 different places:
>
> - legacy queue_setup() method
> - modern queue_setup() method
> - your vdev queue_setup() method

Each kind of device does such check/settings on its own requirement. So 
only indispensable check/settings are added into these three different 
devices.

>
> And another remind is that Huawei planned to split Rx/Tx queue settings,
> here you mixed them again, and I don't think Huawei would like it. Don't
> even to say that after the split, the Tx specific stuff will be no longer
> in the common vq structure.

Yes, this is a blocker issue of my current implementation. Let's see you 
suggestion firstly.

>
> So, I'd suggest something like following:
>
> 	if (is_vdev(..)) {

The blocker issue of your suggestion is that we have no such condition.

Previously, I use dev_type, but as David's comment said:

    "The reason of those comments is that dev_type in ethdev is going to
    disappear, see [1] and [2].
    Drivers are called through their own specific ethdev/crypto ops and
    so, those drivers know implicitely that their are either pci or vdev
    (or whatever in the future) drivers."

Another consideration is comparing vtpci_ops pointer, I don't think it's 
elegant.

> 		/* comment here that we use VA for vdev */
> 		vq->vq_ring_mem = (phys_addr_t)vq->mz->addr;
> 		vq->virtio_net_hdr_mem = ...;
> 		vq->offset = ...;
> 	} else {
> 		vq->vq_ring_mem = ...;
> 		...
> 	}
> 	vring_hdr_desc_init(vq);
>
>> See below comments for more detail.
>>
>>> The "setup_queue" method is actually for telling the device where desc,
>>> avail and used vring are located. Hence, the implementation could be simple:
>>> just log them.
>>>> +
>>>> +const struct virtio_pci_ops vdev_ops = {
>>> Note that this is the interface for the driver to talk to the device,
>>> we should put this file into upper layer then, in the driver.
>>>
>>> And let me make a summary, trying to make it clear:
>>>
>>> - We should not use any structures/functions from the virtio driver
>>>    here, unless it's really a must.
>> Firstly I agree this point (although I see a difference in how we take "a
>> must"). My original principle is to maximize the use of existing structures
>> instead of maintain any new ones.
> If that could save you a lot of efforts and make the design clean, I
> might would say, yes, go for it. But it's obviously NO in this case.
>
>> And I already give up that principle when
>> I accept your previous suggestion to use struct virtio_user_device to store
>> virtio-user specific fields. So I agree to add the new struct virtqueue to
>> avoid use of driver-layer virtqueues.
>>
>>> - It's allowed for driver to make *few* special handling for the virtio
>>>    user device. And that's what the driver supposed to do: to handle
>>>    different device variants.
>> So here are two contradictory ways. Compared to the way you suggest,
>> another way is to keep a unified driver and maintain all special handling
>> inside struct virtio_pci_ops.
>>
>> I prefer the latter because:
>> (1) Special handling for each kind of device will be gather together instead
>> of scattered everywhere of driver code.
>> (2) It's more aligned to previous logic to hide the detail to differentiate
>> modern/legacy device.
> May I ask how many more such handling are needed, excluding the tx queue
> header desc setup? And as stated, in generic, yes, we should try that.

Those which need special handling:
(1) vq->vq_ring_mem: it is set but never used, so it's out of question.
(2) vq->virtio_net_hdr_mem and vring_hdr_desc_init
(3) vq->offset

Just (2) and (3) so far. And the question is quite clear: where to put 
these two special handling.

Thanks,
Jianfeng

>
> 	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio
  2016-05-12 17:02         ` Michael S. Tsirkin
@ 2016-05-13  2:00           ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-05-13  2:00 UTC (permalink / raw)
  To: Michael S. Tsirkin
  Cc: Yuanhan Liu, dev, Huawei Xie, rich.lane, nakajima.yoshihiro,
	p.fedin, ann.zhuangyanying, mukawa, nhorman


Hi Michael,

On 5/13/2016 1:02 AM, Michael S. Tsirkin wrote:
> On Thu, May 12, 2016 at 03:08:05PM +0800, Tan, Jianfeng wrote:
>> (2) It's more aligned to previous logic to hide the detail to differentiate
>> modern/legacy device.
> Why is there a need to support legacy interfaces at all?  It's a container
> so if it's in use one can be reasonably sure you have a new kernel.
>

No, there's no need. The added device, virtio-user, is parallel to 
legacy and modern device. But there's a feature bit inside vhost user 
protocol for vhost user to decide the length of header, current 
implementation by default set this feature bit.

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio
  2016-05-13  1:54           ` Tan, Jianfeng
@ 2016-05-13  4:45             ` Yuanhan Liu
  2016-05-16  1:48               ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2016-05-13  4:45 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

On Fri, May 13, 2016 at 09:54:33AM +0800, Tan, Jianfeng wrote:
> 
>     So, I'd suggest something like following:
> 
>             if (is_vdev(..)) {
> 
> 
> The blocker issue of your suggestion is that we have no such condition.
> 
> Previously, I use dev_type, but as David's comment said:

That's not the only option. There should be others, for example,
checking the existence of virtio_user_device. Or even, you could
add a new flag inside virtio hw while initiating your vdev.

>     May I ask how many more such handling are needed, excluding the tx queue
>     header desc setup? And as stated, in generic, yes, we should try that.
> 
> 
> Those which need special handling:
> (1) vq->vq_ring_mem: it is set but never used, so it's out of question.
> (2) vq->virtio_net_hdr_mem and vring_hdr_desc_init

vring_hdr_desc_init is common.

> (3) vq->offset
> 
> Just (2) and (3) so far. And the question is quite clear: where to put these
> two special handling.

Apparently, you can't put it into the queue_setup(). And I still think
my proposal works great here.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio
  2016-05-13  4:45             ` Yuanhan Liu
@ 2016-05-16  1:48               ` Tan, Jianfeng
  2016-05-16  2:51                 ` Yuanhan Liu
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-05-16  1:48 UTC (permalink / raw)
  To: Yuanhan Liu
  Cc: dev, Xie, Huawei, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

Hi Yuanhan,

> -----Original Message-----
> From: Yuanhan Liu [mailto:yuanhan.liu@linux.intel.com]
> Sent: Friday, May 13, 2016 12:45 PM
> To: Tan, Jianfeng
> Cc: dev@dpdk.org; Xie, Huawei; rich.lane@bigswitch.com; mst@redhat.com;
> nakajima.yoshihiro@lab.ntt.co.jp; p.fedin@samsung.com;
> ann.zhuangyanying@huawei.com; mukawa@igel.co.jp;
> nhorman@tuxdriver.com
> Subject: Re: [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio
> 
> On Fri, May 13, 2016 at 09:54:33AM +0800, Tan, Jianfeng wrote:
> >
> >     So, I'd suggest something like following:
> >
> >             if (is_vdev(..)) {
> >
> >
> > The blocker issue of your suggestion is that we have no such condition.
> >
> > Previously, I use dev_type, but as David's comment said:
> 
> That's not the only option. There should be others, for example,
> checking the existence of virtio_user_device. Or even, you could
> add a new flag inside virtio hw while initiating your vdev.
> 
> >     May I ask how many more such handling are needed, excluding the tx
> queue
> >     header desc setup? And as stated, in generic, yes, we should try that.
> >
> >
> > Those which need special handling:
> > (1) vq->vq_ring_mem: it is set but never used, so it's out of question.
> > (2) vq->virtio_net_hdr_mem and vring_hdr_desc_init
> 
> vring_hdr_desc_init is common.
> 
> > (3) vq->offset
> >
> > Just (2) and (3) so far. And the question is quite clear: where to put these
> > two special handling.
> 
> Apparently, you can't put it into the queue_setup(). And I still think
> my proposal works great here.

OK, since it's indeed inappropriate to put these special handlings inside queue_setup() from semantic perspective, I'll add them according to if hw->vdev_private == NULL in the driver. Thanks for suggestion.

Thanks,
Jianfeng

> 
> 	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio
  2016-05-16  1:48               ` Tan, Jianfeng
@ 2016-05-16  2:51                 ` Yuanhan Liu
  0 siblings, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-05-16  2:51 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: dev, Xie, Huawei, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

On Mon, May 16, 2016 at 01:48:01AM +0000, Tan, Jianfeng wrote:
> > On Fri, May 13, 2016 at 09:54:33AM +0800, Tan, Jianfeng wrote:
> > >
> > >     So, I'd suggest something like following:
> > >
> > >             if (is_vdev(..)) {
> > >
> > >
> > > The blocker issue of your suggestion is that we have no such condition.
> > >
> > > Previously, I use dev_type, but as David's comment said:
> > 
> > That's not the only option. There should be others, for example,
> > checking the existence of virtio_user_device. Or even, you could
> > add a new flag inside virtio hw while initiating your vdev.
> > 
> > >     May I ask how many more such handling are needed, excluding the tx
> > queue
> > >     header desc setup? And as stated, in generic, yes, we should try that.
> > >
> > >
> > > Those which need special handling:
> > > (1) vq->vq_ring_mem: it is set but never used, so it's out of question.
> > > (2) vq->virtio_net_hdr_mem and vring_hdr_desc_init
> > 
> > vring_hdr_desc_init is common.
> > 
> > > (3) vq->offset
> > >
> > > Just (2) and (3) so far. And the question is quite clear: where to put these
> > > two special handling.
> > 
> > Apparently, you can't put it into the queue_setup(). And I still think
> > my proposal works great here.
> 
> OK, since it's indeed inappropriate to put these special handlings inside queue_setup() from semantic perspective, I'll add them according to if hw->vdev_private == NULL in the driver.

I'm thinking maybe we could rename "vdev_private" to "virtio_user_dev"
(or something like that), to make it explicit that it's for virtio user
device. I mean, there should be no other vdevs after all. OTOH, using
"hw->vdev_private != NULL" to say it's a virtio-user device is a bit
weird; it doesn't even make too much sense.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v5 0/8] virtio support for container
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
                   ` (9 preceding siblings ...)
  2016-04-29  1:18 ` [PATCH v4 0/8] virtio support for container Jianfeng Tan
@ 2016-05-30 10:55 ` Jianfeng Tan
  2016-05-30 10:55   ` [PATCH v5 1/8] virtio: hide phys addr check inside pci ops Jianfeng Tan
                     ` (7 more replies)
  2016-06-02  9:54 ` [PATCH v6 0/7] virtio support for container Jianfeng Tan
                   ` (3 subsequent siblings)
  14 siblings, 8 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-05-30 10:55 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

v5:
 - Rename struct virtio_user_hw to struct virtio_user_dev.
 - Rename "vdev_private" to "virtio_user_dev".
 - Move special handling into virtio_ethdev.c from queue_setup().
 - Add vring in virtio_user_dev (remove rte_eth_dev_data), so that
   device does not depend on driver's data structure (rte_eth_dev_data).
 - Remove update on doc/guides/nics/overview.rst, because virtio-user has
   exact feature set with virtio.
 - Change "unsigned long int" to "uint64_t", "unsigned" to "uint32_t".
 - Remove unnecessary cast in vdev_read_dev_config().
 - Add functions in virtio_user_dev.c with prefix of "virtio_user_".
 - Rebase on virtio-next-virtio.

v4:
 - Avoid using dev_type, instead use (eth_dev->pci_device is NULL) to
   judge if it's virtual device or physical device.
 - Change the added device name to virtio-user.
 - Split into vhost_user.c, vhost_kernel.c, vhost.c, virtio_user_pci.c,
   virtio_user_dev.c.
 - Move virtio-user specific data from struct virtio_hw into struct
   virtio_user_hw.
 - Add support to send reset_owner message.
 - Change del_queue implementation. (This need more check)
 - Remove rte_panic(), and superseded with log.
 - Add reset_owner into virtio_pci_ops.reset.
 - Merge parameter "rx" and "tx" to "queues" to emliminate confusion.
 - Move get_features to after set_owner.
 - Redefine path in virtio_user_hw from char * to char [].

v3:
 - Remove --single-file option; do no change at EAL memory.
 - Remove the added API rte_eal_get_backfile_info(), instead we check all
   opened files with HUGEFILE_FMT to find hugepage files owned by DPDK.
 - Accordingly, add more restrictions at "Known issue" section.
 - Rename parameter from queue_num to queue_size for confusion.
 - Rename vhost_embedded.c to rte_eth_virtio_vdev.c.
 - Move code related to the newly added vdev to rte_eth_virtio_vdev.c, to
   reuse eth_virtio_dev_init(), remove its static declaration.
 - Implement dev_uninit() for rte_eth_dev_detach().
 - WARN -> ERR, in vhost_embedded.c
 - Add more commit message for clarify the model.

v2:
 - Rebase on the patchset of virtio 1.0 support.
 - Fix cannot create non-hugepage memory.
 - Fix wrong size of memory region when "single-file" is used.
 - Fix setting of offset in virtqueue to use virtual address.
 - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
 - Add mac option to specify the mac address of this virtual device.
 - Update doc.

This patchset is to provide high performance networking interface (virtio)
for container-based DPDK applications. The way of starting DPDK apps in
containers with ownership of NIC devices exclusively is beyond the scope.
The basic idea here is to present a new virtual device (named virtio-user),
which can be discovered and initialized by DPDK. To minimize the change,
we reuse already-existing virtio PMD code (driver/net/virtio/).

Background: Previously, we usually use a virtio device in the context of
QEMU/VM as below pic shows. Virtio nic is emulated in QEMU, and usually
presented in VM as a PCI device.

  ------------------
  |  virtio driver |  ----->  VM
  ------------------
        |
        | ----------> (over PCI bus or MMIO or Channel I/O)
        |
  ------------------
  | device emulate |
  |                |  ----->  QEMU
  | vhost adapter  |
  ------------------
        |
        | ----------> (vhost-user protocol or vhost-net ioctls)
        |
  ------------------
  | vhost backend  |
  ------------------
 
Compared to QEMU/VM case, virtio support for contaner requires to embedded
device framework inside the virtio PMD. So this converged driver actually
plays three roles:
  - virtio driver to drive this new kind of virtual device;
  - device emulation to present this virtual device and reponse to the
    virtio driver, which is originally by QEMU;
  - and the role to communicate with vhost backend, which is also
    originally by QEMU.

The code layout and functionality of each module:
 
  ----------------------
  | ------------------ |
  | | virtio driver  | |----> (virtio_user_pci.c)
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate |-|----> (virtio_user_dev.c)
  | |                | |
  | | vhost adapter  |-|----> (vhost_user.c, vhost_kernel.c, vhost.c)
  | ------------------ |
  ----------------------
         |
         | -------------- --> (vhost-user protocol or vhost-net ioctls)
         |
   ------------------
   | vhost backend  |
   ------------------

How to share memory? In VM's case, qemu always shares all physical layout
to backend. But it's not feasible for a container, as a process, to share
all virtual memory regions to backend. So only specified virtual memory
regions (with type of shared) are sent to backend. It's a limitation that
only addresses in these areas can be used to transmit or receive packets.

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").

How to use?

a. Apply this patchset.

b. To compile container apps:
$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc

c. To build a docker image using Dockerfile below.
$: cat ./Dockerfile
FROM ubuntu:latest
WORKDIR /usr/src/dpdk
COPY . /usr/src/dpdk
ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
$: docker build -t dpdk-app-l2fwd .

d. Used with vhost-user
$: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
	--socket-mem 1024,1024 -- -p 0x1 --stats 1
$: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=virtio-user0,path=/var/run/usvhost -- -p 0x1

f. Used with vhost-net
$: modprobe vhost
$: modprobe vhost-net
$: docker run -i -t --privileged \
	-v /dev/vhost-net:/dev/vhost-net \
	-v /dev/net/tun:/dev/net/tun \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=virtio-user0,path=/dev/vhost-net -- -p 0x1

By the way, it's not necessary to run in a container.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>

Jianfeng Tan (8):
  virtio: hide phys addr check inside pci ops
  virtio: clean up virtio_dev_queue_setup
  virtio: enable use virtual address to fill desc
  virtio-user: add vhost adapter layer
  virtio-user: add device emulation layer APIs
  virtio-user: add new virtual pci driver for virtio
  virtio-user: add a new vdev named virtio-user
  doc: update doc for virtio-user

 config/common_linuxapp                           |   3 +
 doc/guides/rel_notes/release_16_07.rst           |   4 +
 drivers/net/virtio/Makefile                      |   8 +
 drivers/net/virtio/virtio_ethdev.c               |  97 +++--
 drivers/net/virtio/virtio_ethdev.h               |   2 +
 drivers/net/virtio/virtio_pci.c                  |  30 +-
 drivers/net/virtio/virtio_pci.h                  |   3 +-
 drivers/net/virtio/virtio_rxtx.c                 |   5 +-
 drivers/net/virtio/virtio_rxtx_simple.c          |  13 +-
 drivers/net/virtio/virtio_user/vhost.c           | 105 +++++
 drivers/net/virtio/virtio_user/vhost.h           | 222 +++++++++++
 drivers/net/virtio/virtio_user/vhost_kernel.c    | 254 ++++++++++++
 drivers/net/virtio/virtio_user/vhost_user.c      | 378 ++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 477 +++++++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.h |  62 +++
 drivers/net/virtio/virtio_user/virtio_user_pci.c | 218 +++++++++++
 drivers/net/virtio/virtqueue.h                   |  13 +-
 17 files changed, 1840 insertions(+), 54 deletions(-)
 create mode 100644 drivers/net/virtio/virtio_user/vhost.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost.h
 create mode 100644 drivers/net/virtio/virtio_user/vhost_kernel.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_pci.c

-- 
2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v5 1/8] virtio: hide phys addr check inside pci ops
  2016-05-30 10:55 ` [PATCH v5 " Jianfeng Tan
@ 2016-05-30 10:55   ` Jianfeng Tan
  2016-05-30 10:55   ` [PATCH v5 2/8] virtio: clean up virtio_dev_queue_setup Jianfeng Tan
                     ` (6 subsequent siblings)
  7 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-05-30 10:55 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is to move phys addr check from virtio_dev_queue_setup
to pci ops. To makt that happen, make sure virtio_ops.setup_queue
return the result if we pass through the check.

Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Acked-by: Yuanhan Liu <yuanhan.liu@linux.intel.com>
---
 drivers/net/virtio/virtio_ethdev.c | 17 +++++------------
 drivers/net/virtio/virtio_pci.c    | 30 ++++++++++++++++++++++++++++--
 drivers/net/virtio/virtio_pci.h    |  2 +-
 3 files changed, 34 insertions(+), 15 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index c3fb628..a3031e4 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -364,17 +364,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		}
 	}
 
-	/*
-	 * Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
-	 * and only accepts 32 bit page frame number.
-	 * Check if the allocated physical memory exceeds 16TB.
-	 */
-	if ((mz->phys_addr + vq->vq_ring_size - 1) >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
-		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
-		virtio_dev_queue_release(vq);
-		return -ENOMEM;
-	}
-
 	memset(mz->addr, 0, sizeof(mz->len));
 	vq->mz = mz;
 	vq->vq_ring_mem = mz->phys_addr;
@@ -446,7 +435,11 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		memset(vq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
 	}
 
-	hw->vtpci_ops->setup_queue(hw, vq);
+	if (hw->vtpci_ops->setup_queue(hw, vq) < 0) {
+		PMD_INIT_LOG(ERR, "setup_queue failed");
+		virtio_dev_queue_release(vq);
+		return -EINVAL;
+	}
 
 	vq->configured = 1;
 	*pvq = vq;
diff --git a/drivers/net/virtio/virtio_pci.c b/drivers/net/virtio/virtio_pci.c
index 9cdca06..6bd239c 100644
--- a/drivers/net/virtio/virtio_pci.c
+++ b/drivers/net/virtio/virtio_pci.c
@@ -55,6 +55,22 @@
  */
 #define VIRTIO_PCI_CONFIG(hw) (((hw)->use_msix) ? 24 : 20)
 
+static inline int
+check_vq_phys_addr_ok(struct virtqueue *vq)
+{
+	/* Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
+	 * and only accepts 32 bit page frame number.
+	 * Check if the allocated physical memory exceeds 16TB.
+	 */
+	if ((vq->vq_ring_mem + vq->vq_ring_size - 1) >>
+			(VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
+		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
+		return 0;
+	}
+
+	return 1;
+}
+
 static void
 legacy_read_dev_config(struct virtio_hw *hw, size_t offset,
 		       void *dst, int length)
@@ -143,15 +159,20 @@ legacy_get_queue_num(struct virtio_hw *hw, uint16_t queue_id)
 	return dst;
 }
 
-static void
+static int
 legacy_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 {
 	uint32_t src;
 
+	if (!check_vq_phys_addr_ok(vq))
+		return -1;
+
 	rte_eal_pci_ioport_write(&hw->io, &vq->vq_queue_index, 2,
 			 VIRTIO_PCI_QUEUE_SEL);
 	src = vq->mz->phys_addr >> VIRTIO_PCI_QUEUE_ADDR_SHIFT;
 	rte_eal_pci_ioport_write(&hw->io, &src, 4, VIRTIO_PCI_QUEUE_PFN);
+
+	return 0;
 }
 
 static void
@@ -367,12 +388,15 @@ modern_get_queue_num(struct virtio_hw *hw, uint16_t queue_id)
 	return io_read16(&hw->common_cfg->queue_size);
 }
 
-static void
+static int
 modern_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 {
 	uint64_t desc_addr, avail_addr, used_addr;
 	uint16_t notify_off;
 
+	if (!check_vq_phys_addr_ok(vq))
+		return -1;
+
 	desc_addr = vq->mz->phys_addr;
 	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
 	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
@@ -400,6 +424,8 @@ modern_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 	PMD_INIT_LOG(DEBUG, "\t used_addr: %" PRIx64, used_addr);
 	PMD_INIT_LOG(DEBUG, "\t notify addr: %p (notify offset: %u)",
 		vq->notify_addr, notify_off);
+
+	return 0;
 }
 
 static void
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index 554efea..a76daf7 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -234,7 +234,7 @@ struct virtio_pci_ops {
 	uint16_t (*set_config_irq)(struct virtio_hw *hw, uint16_t vec);
 
 	uint16_t (*get_queue_num)(struct virtio_hw *hw, uint16_t queue_id);
-	void (*setup_queue)(struct virtio_hw *hw, struct virtqueue *vq);
+	int (*setup_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 	void (*del_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 	void (*notify_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 };
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v5 2/8] virtio: clean up virtio_dev_queue_setup
  2016-05-30 10:55 ` [PATCH v5 " Jianfeng Tan
  2016-05-30 10:55   ` [PATCH v5 1/8] virtio: hide phys addr check inside pci ops Jianfeng Tan
@ 2016-05-30 10:55   ` Jianfeng Tan
  2016-06-01  7:38     ` Yuanhan Liu
  2016-05-30 10:55   ` [PATCH v5 3/8] virtio: enable use virtual address to fill desc Jianfeng Tan
                     ` (5 subsequent siblings)
  7 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-05-30 10:55 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Abstract vring hdr desc init as an inline method.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 drivers/net/virtio/virtio_ethdev.c | 42 ++++++++++++++++++++++----------------
 1 file changed, 24 insertions(+), 18 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index a3031e4..781886d 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -278,6 +278,26 @@ virtio_dev_queue_release(struct virtqueue *vq)
 	}
 }
 
+static void
+vring_hdr_desc_init(struct virtqueue *vq)
+{
+	int i;
+	struct virtio_tx_region *txr = vq->virtio_net_hdr_mz->addr;
+
+	for (i = 0; i < vq->vq_nentries; i++) {
+		struct vring_desc *start_dp = txr[i].tx_indir;
+
+		vring_desc_init(start_dp, RTE_DIM(txr[i].tx_indir));
+
+		/* first indirect descriptor is always the tx header */
+		start_dp->addr = vq->virtio_net_hdr_mem + i * sizeof(*txr) +
+				 offsetof(struct virtio_tx_region, tx_hdr);
+
+		start_dp->len = vq->hw->vtnet_hdr_size;
+		start_dp->flags = VRING_DESC_F_NEXT;
+	}
+}
+
 int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 			int queue_type,
 			uint16_t queue_idx,
@@ -375,8 +395,7 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 
 	if (queue_type == VTNET_TQ) {
 		const struct rte_memzone *hdr_mz;
-		struct virtio_tx_region *txr;
-		unsigned int i;
+		size_t hdr_mz_sz = vq_size * sizeof(struct virtio_tx_region);
 
 		/*
 		 * For each xmit packet, allocate a virtio_net_hdr
@@ -385,7 +404,7 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		snprintf(vq_name, sizeof(vq_name), "port%d_tvq%d_hdrzone",
 			 dev->data->port_id, queue_idx);
 		hdr_mz = rte_memzone_reserve_aligned(vq_name,
-						     vq_size * sizeof(*txr),
+						     hdr_mz_sz,
 						     socket_id, 0,
 						     RTE_CACHE_LINE_SIZE);
 		if (hdr_mz == NULL) {
@@ -399,21 +418,8 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		vq->virtio_net_hdr_mz = hdr_mz;
 		vq->virtio_net_hdr_mem = hdr_mz->phys_addr;
 
-		txr = hdr_mz->addr;
-		memset(txr, 0, vq_size * sizeof(*txr));
-		for (i = 0; i < vq_size; i++) {
-			struct vring_desc *start_dp = txr[i].tx_indir;
-
-			vring_desc_init(start_dp, RTE_DIM(txr[i].tx_indir));
-
-			/* first indirect descriptor is always the tx header */
-			start_dp->addr = vq->virtio_net_hdr_mem
-				+ i * sizeof(*txr)
-				+ offsetof(struct virtio_tx_region, tx_hdr);
-
-			start_dp->len = vq->hw->vtnet_hdr_size;
-			start_dp->flags = VRING_DESC_F_NEXT;
-		}
+		memset(hdr_mz->addr, 0, hdr_mz_sz);
+		vring_hdr_desc_init(vq);
 
 	} else if (queue_type == VTNET_CQ) {
 		/* Allocate a page for control vq command, data and status */
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v5 3/8] virtio: enable use virtual address to fill desc
  2016-05-30 10:55 ` [PATCH v5 " Jianfeng Tan
  2016-05-30 10:55   ` [PATCH v5 1/8] virtio: hide phys addr check inside pci ops Jianfeng Tan
  2016-05-30 10:55   ` [PATCH v5 2/8] virtio: clean up virtio_dev_queue_setup Jianfeng Tan
@ 2016-05-30 10:55   ` Jianfeng Tan
  2016-06-01  8:03     ` Yuanhan Liu
  2016-05-30 10:55   ` [PATCH v5 4/8] virtio-user: add vhost adapter layer Jianfeng Tan
                     ` (4 subsequent siblings)
  7 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-05-30 10:55 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is related to how to calculate relative address for vhost
backend.

The principle is that: based on one or multiple shared memory regions,
vhost maintains a reference system with the frontend start address,
backend start address, and length for each segment, so that each
frontend address (GPA, Guest Physical Address) can be translated into
vhost-recognizable backend address. To make the address translation
efficient, we need to maintain as few regions as possible. In the case
of VM, GPA is always locally continuous. But for some other case, like
virtio-user, we use virtual address here.

It basically means:
  a. when set_base_addr, VA address is used;
  b. when preparing RX's descriptors, VA address is used;
  c. when transmitting packets, VA is filled in TX's descriptors;
  d. in TX and CQ's header, VA is used.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/virtio_ethdev.c      | 21 ++++++++++++++++-----
 drivers/net/virtio/virtio_rxtx.c        |  5 ++---
 drivers/net/virtio/virtio_rxtx_simple.c | 13 +++++++------
 drivers/net/virtio/virtqueue.h          | 13 ++++++++++++-
 4 files changed, 37 insertions(+), 15 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 781886d..1866afd 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -167,14 +167,14 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	 * One RX packet for ACK.
 	 */
 	vq->vq_ring.desc[head].flags = VRING_DESC_F_NEXT;
-	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mz->phys_addr;
+	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mem;
 	vq->vq_ring.desc[head].len = sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_free_cnt--;
 	i = vq->vq_ring.desc[head].next;
 
 	for (k = 0; k < pkt_num; k++) {
 		vq->vq_ring.desc[i].flags = VRING_DESC_F_NEXT;
-		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr)
 			+ sizeof(ctrl->status) + sizeof(uint8_t)*sum;
 		vq->vq_ring.desc[i].len = dlen[k];
@@ -184,7 +184,7 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	}
 
 	vq->vq_ring.desc[i].flags = VRING_DESC_F_WRITE;
-	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_ring.desc[i].len = sizeof(ctrl->status);
 	vq->vq_free_cnt--;
@@ -419,8 +419,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		vq->virtio_net_hdr_mem = hdr_mz->phys_addr;
 
 		memset(hdr_mz->addr, 0, hdr_mz_sz);
-		vring_hdr_desc_init(vq);
-
 	} else if (queue_type == VTNET_CQ) {
 		/* Allocate a page for control vq command, data and status */
 		snprintf(vq_name, sizeof(vq_name), "port%d_cvq_hdrzone",
@@ -441,6 +439,19 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		memset(vq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
 	}
 
+	if (dev->pci_dev)
+		vq->offset = offsetof(struct rte_mbuf, buf_physaddr);
+	else {
+		vq->vq_ring_mem = (phys_addr_t)vq->mz->addr;
+		vq->offset = offsetof(struct rte_mbuf, buf_addr);
+		if (vq->virtio_net_hdr_mz)
+			vq->virtio_net_hdr_mem =
+				(phys_addr_t)vq->virtio_net_hdr_mz->addr;
+	}
+
+	if (queue_type == VTNET_TQ)
+		vring_hdr_desc_init(vq);
+
 	if (hw->vtpci_ops->setup_queue(hw, vq) < 0) {
 		PMD_INIT_LOG(ERR, "setup_queue failed");
 		virtio_dev_queue_release(vq);
diff --git a/drivers/net/virtio/virtio_rxtx.c b/drivers/net/virtio/virtio_rxtx.c
index f326222..5b0c3df 100644
--- a/drivers/net/virtio/virtio_rxtx.c
+++ b/drivers/net/virtio/virtio_rxtx.c
@@ -193,8 +193,7 @@ virtqueue_enqueue_recv_refill(struct virtqueue *vq, struct rte_mbuf *cookie)
 
 	start_dp = vq->vq_ring.desc;
 	start_dp[idx].addr =
-		(uint64_t)(cookie->buf_physaddr + RTE_PKTMBUF_HEADROOM
-		- hw->vtnet_hdr_size);
+		MBUF_DATA_DMA_ADDR(cookie, vq->offset) - hw->vtnet_hdr_size;
 	start_dp[idx].len =
 		cookie->buf_len - RTE_PKTMBUF_HEADROOM + hw->vtnet_hdr_size;
 	start_dp[idx].flags =  VRING_DESC_F_WRITE;
@@ -265,7 +264,7 @@ virtqueue_enqueue_xmit(struct virtqueue *txvq, struct rte_mbuf *cookie,
 	}
 
 	do {
-		start_dp[idx].addr  = rte_mbuf_data_dma_addr(cookie);
+		start_dp[idx].addr  = MBUF_DATA_DMA_ADDR(cookie, txvq->offset);
 		start_dp[idx].len   = cookie->data_len;
 		start_dp[idx].flags = cookie->next ? VRING_DESC_F_NEXT : 0;
 		idx = start_dp[idx].next;
diff --git a/drivers/net/virtio/virtio_rxtx_simple.c b/drivers/net/virtio/virtio_rxtx_simple.c
index 8f5293d..83a794e 100644
--- a/drivers/net/virtio/virtio_rxtx_simple.c
+++ b/drivers/net/virtio/virtio_rxtx_simple.c
@@ -80,8 +80,8 @@ virtqueue_enqueue_recv_refill_simple(struct virtqueue *vq,
 	vq->sw_ring[desc_idx] = cookie;
 
 	start_dp = vq->vq_ring.desc;
-	start_dp[desc_idx].addr = (uint64_t)((uintptr_t)cookie->buf_physaddr +
-		RTE_PKTMBUF_HEADROOM - vq->hw->vtnet_hdr_size);
+	start_dp[desc_idx].addr = MBUF_DATA_DMA_ADDR(cookie, vq->offset) -
+				  vq->hw->vtnet_hdr_size;
 	start_dp[desc_idx].len = cookie->buf_len -
 		RTE_PKTMBUF_HEADROOM + vq->hw->vtnet_hdr_size;
 
@@ -119,8 +119,8 @@ virtio_rxq_rearm_vec(struct virtqueue *rxvq)
 		*(uint64_t *)p = rxvq->mbuf_initializer;
 
 		start_dp[i].addr =
-			(uint64_t)((uintptr_t)sw_ring[i]->buf_physaddr +
-			RTE_PKTMBUF_HEADROOM - rxvq->hw->vtnet_hdr_size);
+			MBUF_DATA_DMA_ADDR(sw_ring[i], rxvq->offset) -
+			rxvq->hw->vtnet_hdr_size;
 		start_dp[i].len = sw_ring[i]->buf_len -
 			RTE_PKTMBUF_HEADROOM + rxvq->hw->vtnet_hdr_size;
 	}
@@ -366,7 +366,7 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 			txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 		for (i = 0; i < nb_tail; i++) {
 			start_dp[desc_idx].addr =
-				rte_mbuf_data_dma_addr(*tx_pkts);
+				MBUF_DATA_DMA_ADDR(*tx_pkts, txvq->offset);
 			start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 			tx_pkts++;
 			desc_idx++;
@@ -377,7 +377,8 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 	for (i = 0; i < nb_commit; i++)
 		txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 	for (i = 0; i < nb_commit; i++) {
-		start_dp[desc_idx].addr = rte_mbuf_data_dma_addr(*tx_pkts);
+		start_dp[desc_idx].addr =
+			MBUF_DATA_DMA_ADDR(*tx_pkts, txvq->offset);
 		start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 		tx_pkts++;
 		desc_idx++;
diff --git a/drivers/net/virtio/virtqueue.h b/drivers/net/virtio/virtqueue.h
index 4e543d2..8ffc366 100644
--- a/drivers/net/virtio/virtqueue.h
+++ b/drivers/net/virtio/virtqueue.h
@@ -66,6 +66,14 @@ struct rte_mbuf;
 
 #define VIRTQUEUE_MAX_NAME_SZ 32
 
+#ifdef RTE_VIRTIO_VDEV
+#define MBUF_DATA_DMA_ADDR(mb, offset) \
+	((uint64_t)((uintptr_t)(*(void **)((uintptr_t)mb + offset)) \
+			+ (mb)->data_off))
+#else /* RTE_VIRTIO_VDEV */
+#define MBUF_DATA_DMA_ADDR(mb, offset) rte_mbuf_data_dma_addr(mb)
+#endif /* RTE_VIRTIO_VDEV */
+
 #define VTNET_SQ_RQ_QUEUE_IDX 0
 #define VTNET_SQ_TQ_QUEUE_IDX 1
 #define VTNET_SQ_CQ_QUEUE_IDX 2
@@ -165,6 +173,7 @@ struct virtqueue {
 	void        *vq_ring_virt_mem;    /**< linear address of vring*/
 	unsigned int vq_ring_size;
 	phys_addr_t vq_ring_mem;          /**< physical address of vring */
+					  /**< use virtual address for vdev. */
 
 	struct vring vq_ring;    /**< vring keeping desc, used and avail */
 	uint16_t    vq_free_cnt; /**< num of desc available */
@@ -183,8 +192,10 @@ struct virtqueue {
 	 */
 	uint16_t vq_used_cons_idx;
 	uint16_t vq_avail_idx;
+	uint16_t offset; /**< relative offset to obtain addr in mbuf */
 	uint64_t mbuf_initializer; /**< value to init mbufs. */
-	phys_addr_t virtio_net_hdr_mem; /**< hdr for each xmit packet */
+	phys_addr_t virtio_net_hdr_mem; /**< phys addr of hdr memzone */
+					/**< use virtual address for vdev */
 
 	struct rte_mbuf **sw_ring; /**< RX software ring. */
 	/* dummy mbuf, for wraparound when processing RX ring. */
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v5 4/8] virtio-user: add vhost adapter layer
  2016-05-30 10:55 ` [PATCH v5 " Jianfeng Tan
                     ` (2 preceding siblings ...)
  2016-05-30 10:55   ` [PATCH v5 3/8] virtio: enable use virtual address to fill desc Jianfeng Tan
@ 2016-05-30 10:55   ` Jianfeng Tan
  2016-05-30 10:55   ` [PATCH v5 5/8] virtio-user: add device emulation layer APIs Jianfeng Tan
                     ` (3 subsequent siblings)
  7 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-05-30 10:55 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is to provide vhost adapter layer implementations. Instead
of relying on a hypervisor to translate between device emulation and
vhost backend, here we directly talk with vhost backend through the
vhost file. Depending on the type of vhost file,
  - vhost-user is used if the given path points to a unix socket;
  - vhost-kernel is used if the given path points to a char device.

Here three main APIs are provided to upper layer (device emulation):
  - vhost_user_setup(), to set up env to talk to a vhost user backend;
  - vhost_kernel_setup(), to set up env to talk to a vhost kernel backend.
  - vhost_call(), to provide a unified interface to communicate with
    vhost backend.

  ----------------------
  | ------------------ |
  | | virtio driver  | |
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate | |
  | |                | |
  | | vhost adapter  |-|----> (vhost_user.c, vhost_kernel.c, vhost.c)
  | ------------------ |
  ----------------------
            |
            | -------------- --> (vhost-user protocol or vhost-net ioctls)
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 config/common_linuxapp                        |   3 +
 drivers/net/virtio/Makefile                   |   6 +
 drivers/net/virtio/virtio_user/vhost.c        | 105 +++++++
 drivers/net/virtio/virtio_user/vhost.h        | 222 +++++++++++++++
 drivers/net/virtio/virtio_user/vhost_kernel.c | 254 +++++++++++++++++
 drivers/net/virtio/virtio_user/vhost_user.c   | 378 ++++++++++++++++++++++++++
 6 files changed, 968 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/vhost.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost.h
 create mode 100644 drivers/net/virtio/virtio_user/vhost_kernel.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c

diff --git a/config/common_linuxapp b/config/common_linuxapp
index 7e698e2..946a6d4 100644
--- a/config/common_linuxapp
+++ b/config/common_linuxapp
@@ -43,3 +43,6 @@ CONFIG_RTE_LIBRTE_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_AF_PACKET=y
 CONFIG_RTE_LIBRTE_POWER=y
+
+# Enable virtio-user
+CONFIG_RTE_VIRTIO_VDEV=y
diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index ef84f60..c9f2bc0 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -55,6 +55,12 @@ ifeq ($(findstring RTE_MACHINE_CPUFLAG_SSSE3,$(CFLAGS)),RTE_MACHINE_CPUFLAG_SSSE
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx_simple.c
 endif
 
+ifeq ($(CONFIG_RTE_VIRTIO_VDEV),y)
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_kernel.c
+endif
+
 # this lib depends upon:
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_eal lib/librte_ether
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_mempool lib/librte_mbuf
diff --git a/drivers/net/virtio/virtio_user/vhost.c b/drivers/net/virtio/virtio_user/vhost.c
new file mode 100644
index 0000000..1944a97
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost.c
@@ -0,0 +1,105 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include "vhost.h"
+
+static const char * const vhost_msg_strings[] = {
+	[VHOST_MSG_SET_OWNER] = "VHOST_MSG_SET_OWNER",
+	[VHOST_MSG_RESET_OWNER] = "VHOST_MSG_RESET_OWNER",
+	[VHOST_MSG_SET_FEATURES] = "VHOST_MSG_SET_FEATURES",
+	[VHOST_MSG_GET_FEATURES] = "VHOST_MSG_GET_FEATURES",
+	[VHOST_MSG_SET_VRING_CALL] = "VHOST_MSG_SET_VRING_CALL",
+	[VHOST_MSG_SET_VRING_NUM] = "VHOST_MSG_SET_VRING_NUM",
+	[VHOST_MSG_SET_VRING_BASE] = "VHOST_MSG_SET_VRING_BASE",
+	[VHOST_MSG_GET_VRING_BASE] = "VHOST_MSG_GET_VRING_BASE",
+	[VHOST_MSG_SET_VRING_ADDR] = "VHOST_MSG_SET_VRING_ADDR",
+	[VHOST_MSG_SET_VRING_KICK] = "VHOST_MSG_SET_VRING_KICK",
+	[VHOST_MSG_SET_MEM_TABLE] = "VHOST_MSG_SET_MEM_TABLE",
+	NULL,
+};
+
+static uint64_t vhost_req_map[][2] = {
+	[VHOST_MSG_SET_OWNER] = {
+		VHOST_SET_OWNER, VHOST_USER_SET_OWNER
+	},
+	[VHOST_MSG_RESET_OWNER] = {
+		VHOST_RESET_OWNER, VHOST_USER_RESET_OWNER
+	},
+	[VHOST_MSG_SET_FEATURES] = {
+		VHOST_SET_FEATURES, VHOST_USER_SET_FEATURES
+	},
+	[VHOST_MSG_GET_FEATURES] = {
+		VHOST_GET_FEATURES, VHOST_USER_GET_FEATURES
+	},
+	[VHOST_MSG_SET_VRING_CALL] = {
+		VHOST_SET_VRING_CALL, VHOST_USER_SET_VRING_CALL
+	},
+	[VHOST_MSG_SET_VRING_NUM] = {
+		VHOST_SET_VRING_NUM, VHOST_USER_SET_VRING_NUM
+	},
+	[VHOST_MSG_SET_VRING_BASE] = {
+		VHOST_SET_VRING_BASE, VHOST_USER_SET_VRING_BASE
+	},
+	[VHOST_MSG_GET_VRING_BASE] = {
+		VHOST_GET_VRING_BASE, VHOST_USER_GET_VRING_BASE
+	},
+	[VHOST_MSG_SET_VRING_ADDR] = {
+		VHOST_SET_VRING_ADDR, VHOST_USER_SET_VRING_ADDR
+	},
+	[VHOST_MSG_SET_VRING_KICK] = {
+		VHOST_SET_VRING_KICK, VHOST_USER_SET_VRING_KICK
+	},
+	[VHOST_MSG_SET_MEM_TABLE] = {
+		VHOST_SET_MEM_TABLE, VHOST_USER_SET_MEM_TABLE
+	},
+};
+
+int
+vhost_call(int vhostfd, int type, uint64_t req, void *arg)
+{
+	uint64_t req_new;
+	int ret;
+
+	PMD_DRV_LOG(INFO, "%s", vhost_msg_strings[req]);
+
+	req_new = vhost_req_map[req][type];
+	if (type == VHOST_USER)
+		ret = vhost_user_sock(vhostfd, req_new, arg);
+	else
+		ret = vhost_kernel_ioctl(vhostfd, req_new, arg);
+
+	if (ret < 0)
+		PMD_DRV_LOG(ERR, "vhost_call %s failed: %s\n",
+			    vhost_msg_strings[req], strerror(errno));
+	return ret;
+}
diff --git a/drivers/net/virtio/virtio_user/vhost.h b/drivers/net/virtio/virtio_user/vhost.h
new file mode 100644
index 0000000..6bac1e8
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost.h
@@ -0,0 +1,222 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VHOST_NET_USER_H
+#define _VHOST_NET_USER_H
+
+#include <stdint.h>
+#include <linux/types.h>
+#include <linux/ioctl.h>
+
+#include "../virtio_pci.h"
+#include "../virtio_logs.h"
+#include "../virtqueue.h"
+
+#define VHOST_MEMORY_MAX_NREGIONS 8
+
+struct vhost_vring_state {
+	unsigned int index;
+	unsigned int num;
+};
+
+struct vhost_vring_file {
+	unsigned int index;
+	int fd;
+};
+
+struct vhost_vring_addr {
+	unsigned int index;
+	/* Option flags. */
+	unsigned int flags;
+	/* Flag values: */
+	/* Whether log address is valid. If set enables logging. */
+#define VHOST_VRING_F_LOG 0
+
+	/* Start of array of descriptors (virtually contiguous) */
+	uint64_t desc_user_addr;
+	/* Used structure address. Must be 32 bit aligned */
+	uint64_t used_user_addr;
+	/* Available structure address. Must be 16 bit aligned */
+	uint64_t avail_user_addr;
+	/* Logging support. */
+	/* Log writes to used structure, at offset calculated from specified
+	 * address. Address must be 32 bit aligned.
+	 */
+	uint64_t log_guest_addr;
+};
+
+enum vhost_user_request {
+	VHOST_USER_NONE = 0,
+	VHOST_USER_GET_FEATURES = 1,
+	VHOST_USER_SET_FEATURES = 2,
+	VHOST_USER_SET_OWNER = 3,
+	VHOST_USER_RESET_OWNER = 4,
+	VHOST_USER_SET_MEM_TABLE = 5,
+	VHOST_USER_SET_LOG_BASE = 6,
+	VHOST_USER_SET_LOG_FD = 7,
+	VHOST_USER_SET_VRING_NUM = 8,
+	VHOST_USER_SET_VRING_ADDR = 9,
+	VHOST_USER_SET_VRING_BASE = 10,
+	VHOST_USER_GET_VRING_BASE = 11,
+	VHOST_USER_SET_VRING_KICK = 12,
+	VHOST_USER_SET_VRING_CALL = 13,
+	VHOST_USER_SET_VRING_ERR = 14,
+	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
+	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
+	VHOST_USER_GET_QUEUE_NUM = 17,
+	VHOST_USER_SET_VRING_ENABLE = 18,
+	VHOST_USER_MAX
+};
+
+struct vhost_memory_region {
+	uint64_t guest_phys_addr;
+	uint64_t memory_size; /* bytes */
+	uint64_t userspace_addr;
+	uint64_t mmap_offset;
+};
+
+struct vhost_memory_kernel {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[0];
+};
+
+struct vhost_memory {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[VHOST_MEMORY_MAX_NREGIONS];
+};
+
+struct vhost_user_msg {
+	enum vhost_user_request request;
+
+#define VHOST_USER_VERSION_MASK     0x3
+#define VHOST_USER_REPLY_MASK       (0x1 << 2)
+	uint32_t flags;
+	uint32_t size; /* the following payload size */
+	union {
+#define VHOST_USER_VRING_IDX_MASK   0xff
+#define VHOST_USER_VRING_NOFD_MASK  (0x1 << 8)
+		uint64_t u64;
+		struct vhost_vring_state state;
+		struct vhost_vring_addr addr;
+		struct vhost_memory memory;
+	} payload;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+} __attribute((packed));
+
+#define VHOST_USER_HDR_SIZE offsetof(struct vhost_user_msg, payload.u64)
+#define VHOST_USER_PAYLOAD_SIZE \
+	(sizeof(struct vhost_user_msg) - VHOST_USER_HDR_SIZE)
+
+/* The version of the protocol we support */
+#define VHOST_USER_VERSION    0x1
+
+/* ioctls */
+
+#define VHOST_VIRTIO 0xAF
+
+#define VHOST_GET_FEATURES _IOR(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_FEATURES _IOW(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_OWNER _IO(VHOST_VIRTIO, 0x01)
+#define VHOST_RESET_OWNER _IO(VHOST_VIRTIO, 0x02)
+#define VHOST_SET_MEM_TABLE _IOW(VHOST_VIRTIO, 0x03, struct vhost_memory_kernel)
+#define VHOST_SET_LOG_BASE _IOW(VHOST_VIRTIO, 0x04, __u64)
+#define VHOST_SET_LOG_FD _IOW(VHOST_VIRTIO, 0x07, int)
+#define VHOST_SET_VRING_NUM _IOW(VHOST_VIRTIO, 0x10, struct vhost_vring_state)
+#define VHOST_SET_VRING_ADDR _IOW(VHOST_VIRTIO, 0x11, struct vhost_vring_addr)
+#define VHOST_SET_VRING_BASE _IOW(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_GET_VRING_BASE _IOWR(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_SET_VRING_KICK _IOW(VHOST_VIRTIO, 0x20, struct vhost_vring_file)
+#define VHOST_SET_VRING_CALL _IOW(VHOST_VIRTIO, 0x21, struct vhost_vring_file)
+#define VHOST_SET_VRING_ERR _IOW(VHOST_VIRTIO, 0x22, struct vhost_vring_file)
+#define VHOST_NET_SET_BACKEND _IOW(VHOST_VIRTIO, 0x30, struct vhost_vring_file)
+
+/*****************************************************************************/
+
+/* Ioctl defines */
+#define TUNSETIFF     _IOW('T', 202, int)
+#define TUNGETFEATURES _IOR('T', 207, unsigned int)
+#define TUNSETOFFLOAD  _IOW('T', 208, unsigned int)
+#define TUNGETIFF      _IOR('T', 210, unsigned int)
+#define TUNSETSNDBUF   _IOW('T', 212, int)
+#define TUNGETVNETHDRSZ _IOR('T', 215, int)
+#define TUNSETVNETHDRSZ _IOW('T', 216, int)
+#define TUNSETQUEUE  _IOW('T', 217, int)
+#define TUNSETVNETLE _IOW('T', 220, int)
+#define TUNSETVNETBE _IOW('T', 222, int)
+
+/* TUNSETIFF ifr flags */
+#define IFF_TAP          0x0002
+#define IFF_NO_PI        0x1000
+#define IFF_ONE_QUEUE    0x2000
+#define IFF_VNET_HDR     0x4000
+#define IFF_MULTI_QUEUE  0x0100
+#define IFF_ATTACH_QUEUE 0x0200
+#define IFF_DETACH_QUEUE 0x0400
+
+/* Features for GSO (TUNSETOFFLOAD). */
+#define TUN_F_CSUM	0x01	/* You can hand me unchecksummed packets. */
+#define TUN_F_TSO4	0x02	/* I can handle TSO for IPv4 packets */
+#define TUN_F_TSO6	0x04	/* I can handle TSO for IPv6 packets */
+#define TUN_F_TSO_ECN	0x08	/* I can handle TSO with ECN bits. */
+#define TUN_F_UFO	0x10	/* I can handle UFO packets */
+
+enum {
+	VHOST_MSG_SET_OWNER,
+	VHOST_MSG_RESET_OWNER,
+	VHOST_MSG_SET_FEATURES,
+	VHOST_MSG_GET_FEATURES,
+	VHOST_MSG_SET_VRING_CALL,
+	VHOST_MSG_SET_VRING_NUM,
+	VHOST_MSG_SET_VRING_BASE,
+	VHOST_MSG_GET_VRING_BASE,
+	VHOST_MSG_SET_VRING_ADDR,
+	VHOST_MSG_SET_VRING_KICK,
+	VHOST_MSG_SET_MEM_TABLE,
+	VHOST_MSG_MAX,
+};
+
+#define VHOST_KERNEL	0
+#define VHOST_USER	1
+
+int vhost_user_sock(int vhostfd, uint64_t req, void *arg);
+int vhost_user_setup(const char *path);
+
+int vhost_kernel_ioctl(int vhostfd, uint64_t req, void *arg);
+int vhost_kernel_setup(const char *path, const char *ifname, int *p_tapfd);
+int vhost_kernel_post(int vhostfd, int tapfd, uint64_t features, uint32_t nvqs);
+
+int vhost_call(int vhostfd, int type, uint64_t req, void *arg);
+
+#endif
diff --git a/drivers/net/virtio/virtio_user/vhost_kernel.c b/drivers/net/virtio/virtio_user/vhost_kernel.c
new file mode 100644
index 0000000..41196cf
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost_kernel.c
@@ -0,0 +1,254 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <sys/ioctl.h>
+#include <net/if.h>
+#include <string.h>
+#include <errno.h>
+
+#include <rte_memory.h>
+#include <rte_eal_memconfig.h>
+
+#include "vhost.h"
+
+#define TUN_DEF_SNDBUF	(1ull << 20)
+#define PATH_NET_TUN	"/dev/net/tun"
+
+/** Merge those virtually adjacent memsegs into one region.
+ */
+static void
+prepare_vhost_memory_kernel(struct vhost_memory_kernel **p_vm)
+{
+	uint32_t i, j, k = 0;
+	struct rte_memseg *seg;
+	struct vhost_memory_region *mr;
+	struct vhost_memory_kernel *vm;
+
+	vm = malloc(sizeof(struct vhost_memory_kernel) +
+		    RTE_MAX_MEMSEG * sizeof(struct vhost_memory_region));
+
+	for (i = 0; i < RTE_MAX_MEMSEG; ++i) {
+		seg = &rte_eal_get_configuration()->mem_config->memseg[i];
+		if (!seg->addr)
+			break;
+
+		int new_region = 1;
+
+		for (j = 0; j < k; ++j) {
+			mr = &vm->regions[j];
+
+			if (mr->userspace_addr + mr->memory_size ==
+			    (uint64_t)seg->addr) {
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+
+			if ((uint64_t)seg->addr + seg->len ==
+			    mr->userspace_addr) {
+				mr->guest_phys_addr = (uint64_t)seg->addr;
+				mr->userspace_addr = (uint64_t)seg->addr;
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+		}
+
+		if (new_region == 0)
+			continue;
+
+		mr = &vm->regions[k++];
+		mr->guest_phys_addr = (uint64_t)seg->addr; /* use vaddr here! */
+		mr->userspace_addr = (uint64_t)seg->addr;
+		mr->memory_size = seg->len;
+		mr->mmap_offset = 0;
+	}
+
+	vm->nregions = k;
+	vm->padding = 0;
+	*p_vm = vm;
+}
+
+int
+vhost_kernel_ioctl(int vhostfd, uint64_t req, void *arg)
+{
+	int ret;
+	struct vhost_memory_kernel *vm = NULL;
+
+	if (req == VHOST_SET_MEM_TABLE) {
+		prepare_vhost_memory_kernel(&vm);
+		arg = (void *)vm;
+	}
+
+	ret = ioctl(vhostfd, req, arg);
+
+	if (req == VHOST_SET_MEM_TABLE)
+		free(vm);
+
+	return ret;
+}
+
+/**
+ * Set up environment to talk with a vhost kernel backend.
+ * @param path
+ *   - The path to vhost net (kernel) character file.
+ *
+ * @param ifname
+ *   - Specify the tap device name if any, or NULL.
+ *
+ * @param p_tapfd
+ *   - Pointer to store the fd of tap device.
+ *
+ * @return
+ *   - (-1) if fail to set up;
+ *   - (>=0) if successful, and it is the fd to vhostfd.
+ */
+int
+vhost_kernel_setup(const char *path, const char *ifname, int *p_tapfd)
+{
+	int vhostfd, tapfd;
+	int len = sizeof(struct virtio_net_hdr_mrg_rxbuf);
+	int req_mq = 0;
+	int sndbuf = TUN_DEF_SNDBUF;
+	unsigned int features;
+	struct ifreq ifr;
+
+	/* TODO:
+	 * 1. get/set offload capability, tap_probe_has_ufo, tap_fd_set_offload
+	 * 2. verify we can get/set vnet_hdr_len, tap_probe_vnet_hdr_len
+	 * 3. get number of memory regions from vhost module parameter
+	 * max_mem_regions, supported in newer version linux kernel
+	 */
+
+	tapfd = open(PATH_NET_TUN, O_RDWR);
+	if (tapfd < 0) {
+		PMD_DRV_LOG(ERR, "fail to open %s: %s",
+			    PATH_NET_TUN, strerror(errno));
+		return -1;
+	}
+
+	memset(&ifr, 0, sizeof(ifr));
+	ifr.ifr_flags = IFF_TAP | IFF_NO_PI;
+
+	if (ioctl(tapfd, TUNGETFEATURES, &features) == -1) {
+		PMD_DRV_LOG(ERR, "TUNGETFEATURES failed: %s", strerror(errno));
+		goto error;
+	}
+
+	if (features & IFF_ONE_QUEUE)
+		ifr.ifr_flags |= IFF_ONE_QUEUE;
+
+	if (features & IFF_VNET_HDR)
+		ifr.ifr_flags |= IFF_VNET_HDR;
+	else {
+		PMD_DRV_LOG(ERR, "vnet_hdr not supported by kernel");
+		goto error;
+	}
+
+	if (req_mq) {
+		if (features & IFF_MULTI_QUEUE)
+			ifr.ifr_flags |= IFF_MULTI_QUEUE;
+		else {
+			PMD_DRV_LOG(ERR, "multiqueue not supported by kernel");
+			goto error;
+		}
+	}
+
+	if (ifname)
+		strncpy(ifr.ifr_name, ifname, IFNAMSIZ);
+	else
+		strncpy(ifr.ifr_name, "tap%d", IFNAMSIZ);
+	if (ioctl(tapfd, TUNSETIFF, (void *)&ifr) == -1) {
+		PMD_DRV_LOG(ERR, "TUNSETIFF failed: %s", strerror(errno));
+		goto error;
+	}
+	fcntl(tapfd, F_SETFL, O_NONBLOCK);
+
+	if (ioctl(tapfd, TUNSETVNETHDRSZ, &len) < 0) {
+		PMD_DRV_LOG(ERR, "TUNSETVNETHDRSZ failed: %s", strerror(errno));
+		goto error;
+	}
+
+	if (ioctl(tapfd, TUNSETSNDBUF, &sndbuf) < 0) {
+		PMD_DRV_LOG(ERR, "TUNSETSNDBUF failed: %s", strerror(errno));
+		goto error;
+	}
+
+	vhostfd = open(path, O_RDWR);
+	if (vhostfd < 0) {
+		PMD_DRV_LOG(ERR, "fail to open %s, %s", path, strerror(errno));
+		goto error;
+	}
+
+	*p_tapfd = tapfd;
+	return vhostfd;
+
+error:
+	close(tapfd);
+	return -1;
+}
+
+int
+vhost_kernel_post(int vhostfd, int tapfd, uint64_t features, uint32_t nvqs)
+{
+	struct vhost_vring_file f;
+	uint16_t hdr_size;
+	int ret;
+
+	if ((features & (1ULL << VIRTIO_NET_F_MRG_RXBUF)) ||
+	    (features & (1ULL << VIRTIO_F_VERSION_1)))
+		hdr_size = sizeof(struct virtio_net_hdr_mrg_rxbuf);
+	else
+		hdr_size = sizeof(struct virtio_net_hdr);
+
+	if (ioctl(tapfd, TUNSETVNETHDRSZ, &hdr_size) == -1) {
+		PMD_DRV_LOG(ERR, "TUNSETVNETHDRSZ fails, %s", strerror(errno));
+		return -1;
+	}
+
+	f.fd = tapfd;
+	for (f.index = 0; f.index < nvqs; ++f.index) {
+		ret = vhost_kernel_ioctl(vhostfd, VHOST_NET_SET_BACKEND, &f);
+		if (ret < 0) {
+			PMD_DRV_LOG(ERR, "VHOST_NET_SET_BACKEND fails, %s",
+				    strerror(errno));
+			return -1;
+		}
+	}
+
+	return 0;
+}
diff --git a/drivers/net/virtio/virtio_user/vhost_user.c b/drivers/net/virtio/virtio_user/vhost_user.c
new file mode 100644
index 0000000..b0a84cf
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost_user.c
@@ -0,0 +1,378 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <unistd.h>
+#include <fcntl.h>
+#include <sys/un.h>
+#include <string.h>
+#include <errno.h>
+
+#include "vhost.h"
+
+static int
+vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
+{
+	int r;
+	struct msghdr msgh;
+	struct iovec iov;
+	size_t fd_size = fd_num * sizeof(int);
+	char control[CMSG_SPACE(fd_size)];
+	struct cmsghdr *cmsg;
+
+	memset(&msgh, 0, sizeof(msgh));
+	memset(control, 0, sizeof(control));
+
+	iov.iov_base = (uint8_t *)buf;
+	iov.iov_len = len;
+
+	msgh.msg_iov = &iov;
+	msgh.msg_iovlen = 1;
+	msgh.msg_control = control;
+	msgh.msg_controllen = sizeof(control);
+
+	cmsg = CMSG_FIRSTHDR(&msgh);
+	cmsg->cmsg_len = CMSG_LEN(fd_size);
+	cmsg->cmsg_level = SOL_SOCKET;
+	cmsg->cmsg_type = SCM_RIGHTS;
+	memcpy(CMSG_DATA(cmsg), fds, fd_size);
+
+	do {
+		r = sendmsg(fd, &msgh, 0);
+	} while (r < 0 && errno == EINTR);
+
+	return r;
+}
+
+static int
+vhost_user_read(int fd, struct vhost_user_msg *msg)
+{
+	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
+	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
+
+	ret = recv(fd, (void *)msg, sz_hdr, 0);
+	if (ret < sz_hdr) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
+			    ret, sz_hdr);
+		goto fail;
+	}
+
+	/* validate msg flags */
+	if (msg->flags != (valid_flags)) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg: flags %x instead of %x.",
+			    msg->flags, valid_flags);
+		goto fail;
+	}
+
+	sz_payload = msg->size;
+	if (sz_payload) {
+		ret = recv(fd, (void *)((char *)msg + sz_hdr), sz_payload, 0);
+		if (ret < sz_payload) {
+			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
+				    ret, msg->size);
+			goto fail;
+		}
+	}
+
+	return 0;
+
+fail:
+	return -1;
+}
+
+struct hugepage_file_info {
+	uint64_t addr;            /**< virtual addr */
+	size_t   size;            /**< the file size */
+	char     path[PATH_MAX];  /**< path to backing file */
+};
+
+/* Two possible options:
+ * 1. Match HUGEPAGE_INFO_FMT to find the file storing struct hugepage_file
+ * array. This is simple but cannot be used in secondary process because
+ * secondary process will close and munmap that file.
+ * 2. Match HUGEFILE_FMT to find hugepage files directly.
+ *
+ * We choose option 2.
+ */
+static int
+get_hugepage_file_info(struct hugepage_file_info huges[], int max)
+{
+	int idx;
+	FILE *f;
+	char buf[BUFSIZ], *tmp, *tail;
+	char *str_underline, *str_start;
+	int huge_index;
+	uint64_t v_start, v_end;
+
+	f = fopen("/proc/self/maps", "r");
+	if (!f) {
+		PMD_DRV_LOG(ERR, "cannot open /proc/self/maps");
+		return -1;
+	}
+
+	idx = 0;
+	while (fgets(buf, sizeof(buf), f) != NULL) {
+		if (sscanf(buf, "%" PRIx64 "-%" PRIx64, &v_start, &v_end) < 2) {
+			PMD_DRV_LOG(ERR, "Failed to parse address");
+			goto error;
+		}
+
+		tmp = strchr(buf, ' ') + 1; /** skip address */
+		tmp = strchr(tmp, ' ') + 1; /** skip perm */
+		tmp = strchr(tmp, ' ') + 1; /** skip offset */
+		tmp = strchr(tmp, ' ') + 1; /** skip dev */
+		tmp = strchr(tmp, ' ') + 1; /** skip inode */
+		while (*tmp == ' ')         /** skip spaces */
+			tmp++;
+		tail = strrchr(tmp, '\n');  /** remove newline if exists */
+		if (tail)
+			*tail = '\0';
+
+		/* Match HUGEFILE_FMT, aka "%s/%smap_%d",
+		 * which is defined in eal_filesystem.h
+		 */
+		str_underline = strrchr(tmp, '_');
+		if (!str_underline)
+			continue;
+
+		str_start = str_underline - strlen("map");
+		if (str_start < tmp)
+			continue;
+
+		if (sscanf(str_start, "map_%d", &huge_index) != 1)
+			continue;
+
+		if (idx >= max) {
+			PMD_DRV_LOG(ERR, "Exceed maximum of %d", max);
+			goto error;
+		}
+		huges[idx].addr = v_start;
+		huges[idx].size = v_end - v_start;
+		strcpy(huges[idx].path, tmp);
+		idx++;
+	}
+
+	fclose(f);
+	return idx;
+
+error:
+	fclose(f);
+	return -1;
+}
+
+static int
+prepare_vhost_memory_user(struct vhost_user_msg *msg, int fds[])
+{
+	int i, num;
+	struct hugepage_file_info huges[VHOST_MEMORY_MAX_NREGIONS];
+	struct vhost_memory_region *mr;
+
+	num = get_hugepage_file_info(huges, VHOST_MEMORY_MAX_NREGIONS);
+	if (num < 0) {
+		PMD_INIT_LOG(ERR, "Failed to prepare memory for vhost-user");
+		return -1;
+	}
+
+	for (i = 0; i < num; ++i) {
+		mr = &msg->payload.memory.regions[i];
+		mr->guest_phys_addr = huges[i].addr; /* use vaddr! */
+		mr->userspace_addr = huges[i].addr;
+		mr->memory_size = huges[i].size;
+		mr->mmap_offset = 0;
+		fds[i] = open(huges[i].path, O_RDWR);
+	}
+
+	msg->payload.memory.nregions = num;
+	msg->payload.memory.padding = 0;
+
+	return 0;
+}
+
+static struct vhost_user_msg m;
+
+int
+vhost_user_sock(int vhostfd, uint64_t req, void *arg)
+{
+	struct vhost_user_msg msg;
+	struct vhost_vring_file *file = 0;
+	int need_reply = 0;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+	int fd_num = 0;
+	int i, len;
+
+	msg.request = req;
+	msg.flags = VHOST_USER_VERSION;
+	msg.size = 0;
+
+	switch (req) {
+	case VHOST_USER_GET_FEATURES:
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_FEATURES:
+	case VHOST_USER_SET_LOG_BASE:
+		msg.payload.u64 = *((__u64 *)arg);
+		msg.size = sizeof(m.payload.u64);
+		break;
+
+	case VHOST_USER_SET_OWNER:
+	case VHOST_USER_RESET_OWNER:
+		break;
+
+	case VHOST_USER_SET_MEM_TABLE:
+		if (prepare_vhost_memory_user(&msg, fds) < 0)
+			return -1;
+		fd_num = msg.payload.memory.nregions;
+		msg.size = sizeof(m.payload.memory.nregions);
+		msg.size += sizeof(m.payload.memory.padding);
+		msg.size += fd_num * sizeof(struct vhost_memory_region);
+		break;
+
+	case VHOST_USER_SET_LOG_FD:
+		fds[fd_num++] = *((int *)arg);
+		break;
+
+	case VHOST_USER_SET_VRING_NUM:
+	case VHOST_USER_SET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		break;
+
+	case VHOST_USER_GET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_VRING_ADDR:
+		memcpy(&msg.payload.addr, arg, sizeof(msg.payload.addr));
+		msg.size = sizeof(m.payload.addr);
+		break;
+
+	case VHOST_USER_SET_VRING_KICK:
+	case VHOST_USER_SET_VRING_CALL:
+	case VHOST_USER_SET_VRING_ERR:
+		file = arg;
+		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
+		msg.size = sizeof(m.payload.u64);
+		if (file->fd > 0)
+			fds[fd_num++] = file->fd;
+		else
+			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
+		break;
+
+	default:
+		PMD_DRV_LOG(ERR, "trying to send unhandled msg type");
+		return -1;
+	}
+
+	len = VHOST_USER_HDR_SIZE + msg.size;
+	if (vhost_user_write(vhostfd, &msg, len, fds, fd_num) < 0)
+		return 0;
+
+	if (req == VHOST_USER_SET_MEM_TABLE)
+		for (i = 0; i < fd_num; ++i)
+			close(fds[i]);
+
+	if (need_reply) {
+		if (vhost_user_read(vhostfd, &msg) < 0)
+			return -1;
+
+		if (req != msg.request) {
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+
+		switch (req) {
+		case VHOST_USER_GET_FEATURES:
+			if (msg.size != sizeof(m.payload.u64)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			*((__u64 *)arg) = msg.payload.u64;
+			break;
+		case VHOST_USER_GET_VRING_BASE:
+			if (msg.size != sizeof(m.payload.state)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			memcpy(arg, &msg.payload.state,
+			       sizeof(struct vhost_vring_state));
+			break;
+		default:
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+	}
+
+	return 0;
+}
+
+/**
+ * Set up environment to talk with a vhost user backend.
+ * @param path
+ *   - The path to vhost user unix socket file.
+ *
+ * @return
+ *   - (-1) if fail to set up;
+ *   - (>=0) if successful, and it is the fd to vhostfd.
+ */
+int
+vhost_user_setup(const char *path)
+{
+	int fd;
+	int flag;
+	struct sockaddr_un un;
+
+	fd = socket(AF_UNIX, SOCK_STREAM, 0);
+	if (fd < 0) {
+		PMD_DRV_LOG(ERR, "socket() error, %s", strerror(errno));
+		return -1;
+	}
+
+	flag = fcntl(fd, F_GETFD);
+	fcntl(fd, F_SETFD, flag | FD_CLOEXEC);
+
+	memset(&un, 0, sizeof(un));
+	un.sun_family = AF_UNIX;
+	snprintf(un.sun_path, sizeof(un.sun_path), "%s", path);
+	if (connect(fd, (struct sockaddr *)&un, sizeof(un)) < 0) {
+		PMD_DRV_LOG(ERR, "connect error, %s", strerror(errno));
+		close(fd);
+		return -1;
+	}
+
+	return fd;
+}
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v5 5/8] virtio-user: add device emulation layer APIs
  2016-05-30 10:55 ` [PATCH v5 " Jianfeng Tan
                     ` (3 preceding siblings ...)
  2016-05-30 10:55   ` [PATCH v5 4/8] virtio-user: add vhost adapter layer Jianfeng Tan
@ 2016-05-30 10:55   ` Jianfeng Tan
  2016-05-30 10:55   ` [PATCH v5 6/8] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
                     ` (2 subsequent siblings)
  7 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-05-30 10:55 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Two device emulation layer APIs are added for virtio driver to call:
  - virtio_user_start_device()
  - virtio_user_stop_device()

These APIs will get called by virtio driver, and they call vhost adapter
layer APIs to implement the functionality. Besides, this patch defines
a struct named virtio_user_dev to help manage the data stands for this
kind of virtual device.

  ----------------------
  | ------------------ |
  | | virtio driver  | |
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate |-|----> (virtio_user_dev.c, virtio_user_dev.h)
  | |                | |
  | | vhost adapter  | |
  | ------------------ |
  ----------------------
            |
            |
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/Makefile                      |   1 +
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 168 +++++++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.h |  60 ++++++++
 3 files changed, 229 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h

diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index c9f2bc0..68068bd 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -59,6 +59,7 @@ ifeq ($(CONFIG_RTE_VIRTIO_VDEV),y)
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_kernel.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_dev.c
 endif
 
 # this lib depends upon:
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.c b/drivers/net/virtio/virtio_user/virtio_user_dev.c
new file mode 100644
index 0000000..41d8ad1
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.c
@@ -0,0 +1,168 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <stdio.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <string.h>
+#include <errno.h>
+#include <sys/mman.h>
+#include <unistd.h>
+#include <sys/eventfd.h>
+
+#include <rte_malloc.h>
+#include <rte_kvargs.h>
+
+#include "vhost.h"
+#include "virtio_user_dev.h"
+#include "../virtio_ethdev.h"
+
+static int
+virtio_user_kick_queue(struct virtio_user_dev *dev, uint32_t queue_sel)
+{
+	int callfd, kickfd;
+	struct vhost_vring_file file;
+	struct vhost_vring_state state;
+	struct vring *vring = &dev->vrings[queue_sel];
+	struct vhost_vring_addr addr = {
+		.index = queue_sel,
+		.desc_user_addr = (uint64_t)(uintptr_t)vring->desc,
+		.avail_user_addr = (uint64_t)(uintptr_t)vring->avail,
+		.used_user_addr = (uint64_t)(uintptr_t)vring->used,
+		.log_guest_addr = 0,
+		.flags = 0, /* disable log */
+	};
+
+	/* May use invalid flag, but some backend leverages kickfd and callfd as
+	 * criteria to judge if dev is alive. so finally we use real event_fd.
+	 */
+	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (callfd < 0) {
+		PMD_DRV_LOG(ERR, "callfd error, %s\n", strerror(errno));
+		return -1;
+	}
+	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (kickfd < 0) {
+		close(callfd);
+		PMD_DRV_LOG(ERR, "kickfd error, %s\n", strerror(errno));
+		return -1;
+	}
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL come
+	 * firstly because vhost depends on this msg to allocate virtqueue
+	 * pair.
+	 */
+	file.index = queue_sel;
+	file.fd = callfd;
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_CALL, &file);
+	dev->callfds[queue_sel] = callfd;
+
+	state.index = queue_sel;
+	state.num = vring->num;
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_NUM, &state);
+
+	state.num = 0; /* no reservation */
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_BASE, &state);
+
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_ADDR, &addr);
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_KICK comes
+	 * lastly because vhost depends on this msg to judge if
+	 * virtio is ready.
+	 */
+	file.fd = kickfd;
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_KICK, &file);
+	dev->kickfds[queue_sel] = kickfd;
+
+	return 0;
+}
+
+int
+virtio_user_start_device(struct virtio_user_dev *dev)
+{
+	uint64_t features;
+	uint32_t i, queue_sel;
+	int ret;
+
+	/* construct memory region inside each implementation */
+	ret = vhost_call(dev->vhostfd, dev->type,
+			 VHOST_MSG_SET_MEM_TABLE, NULL);
+	if (ret < 0)
+		goto error;
+
+	for (i = 0; i < dev->queue_pairs; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
+		if (virtio_user_kick_queue(dev, queue_sel) < 0) {
+			PMD_DRV_LOG(INFO, "kick rx vq fails: %u", i);
+			goto error;
+		}
+	}
+	for (i = 0; i < dev->queue_pairs; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
+		if (virtio_user_kick_queue(dev, queue_sel) < 0) {
+			PMD_DRV_LOG(INFO, "kick tx vq fails: %u", i);
+			goto error;
+		}
+	}
+
+	/* After setup all virtqueues, we need to set_features so that
+	 * these features can be set into each virtqueue in vhost side.
+	 * And before that, make sure VIRTIO_NET_F_MAC is stripped.
+	 */
+	features = dev->features;
+	features &= ~(1ull << VIRTIO_NET_F_MAC);
+	ret = vhost_call(dev->vhostfd, dev->type,
+			 VHOST_MSG_SET_FEATURES, &features);
+	if (ret < 0)
+		goto error;
+	PMD_DRV_LOG(INFO, "set features: %" PRIx64, features);
+
+	if (dev->type == VHOST_USER)
+		return 0;
+
+	/* It is tricky here, but so far we have to do that */
+	ret = vhost_kernel_post(dev->vhostfd, dev->tapfd, features,
+				dev->queue_pairs * 2);
+	if (ret == 0)
+		return 0;
+error:
+	/* TODO: free resource here or caller to check */
+	return -1;
+}
+
+int virtio_user_stop_device(struct virtio_user_dev *dev)
+{
+	return vhost_call(dev->vhostfd, dev->type, VHOST_MSG_RESET_OWNER, NULL);
+}
+
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.h b/drivers/net/virtio/virtio_user/virtio_user_dev.h
new file mode 100644
index 0000000..8ca0095
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.h
@@ -0,0 +1,60 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VIRTIO_USER_DEV_H
+#define _VIRTIO_USER_DEV_H
+
+#include <limits.h>
+#include "../virtio_pci.h"
+
+struct virtio_user_dev {
+	int		type; /* VHOST_KERNEL or VHOST_USER */
+	int		vhostfd;
+	int		tapfd; /* only used in vhost kernel */
+	int		callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		mac_specified;
+	uint32_t	max_queue_pairs;
+	uint32_t	queue_pairs;
+	uint32_t	queue_size;
+	uint64_t	features;
+	uint8_t		status;
+	uint8_t		mac_addr[ETHER_ADDR_LEN];
+	char		path[PATH_MAX];
+	struct vring	vrings[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+};
+
+int virtio_user_start_device(struct virtio_user_dev *dev);
+int virtio_user_stop_device(struct virtio_user_dev *dev);
+
+#endif
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v5 6/8] virtio-user: add new virtual pci driver for virtio
  2016-05-30 10:55 ` [PATCH v5 " Jianfeng Tan
                     ` (4 preceding siblings ...)
  2016-05-30 10:55   ` [PATCH v5 5/8] virtio-user: add device emulation layer APIs Jianfeng Tan
@ 2016-05-30 10:55   ` Jianfeng Tan
  2016-06-01  8:21     ` Yuanhan Liu
  2016-05-30 10:55   ` [PATCH v5 7/8] virtio-user: add a new vdev named virtio-user Jianfeng Tan
  2016-05-30 10:55   ` [PATCH v5 8/8] doc: update doc for virtio-user Jianfeng Tan
  7 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-05-30 10:55 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch implements another new instance of struct virtio_pci_ops to
drive the virtio-user virtual device. Instead of rd/wr ioport or PCI
configuration space, this virtual pci driver will rd/wr the virtual
device struct virtio_user_hw, and when necessary, invokes APIs provided
by device emulation later to start/stop the device.

  ----------------------
  | ------------------ |
  | | virtio driver  | |----> (virtio_user_pci.c)
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate | |
  | |                | |
  | | vhost adapter  | |
  | ------------------ |
  ----------------------
            |
            |
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/Makefile                      |   1 +
 drivers/net/virtio/virtio_pci.h                  |   1 +
 drivers/net/virtio/virtio_user/virtio_user_dev.h |   2 +
 drivers/net/virtio/virtio_user/virtio_user_pci.c | 218 +++++++++++++++++++++++
 4 files changed, 222 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_pci.c

diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index 68068bd..13b2b75 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -60,6 +60,7 @@ SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_kernel.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_dev.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_pci.c
 endif
 
 # this lib depends upon:
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index a76daf7..d10d013 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -260,6 +260,7 @@ struct virtio_hw {
 	struct virtio_pci_common_cfg *common_cfg;
 	struct virtio_net_config *dev_cfg;
 	const struct virtio_pci_ops *vtpci_ops;
+	void	    *virtio_user_dev;
 };
 
 /*
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.h b/drivers/net/virtio/virtio_user/virtio_user_dev.h
index 8ca0095..9ebe440 100644
--- a/drivers/net/virtio/virtio_user/virtio_user_dev.h
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.h
@@ -57,4 +57,6 @@ struct virtio_user_dev {
 int virtio_user_start_device(struct virtio_user_dev *dev);
 int virtio_user_stop_device(struct virtio_user_dev *dev);
 
+const struct virtio_pci_ops vdev_ops;
+
 #endif
diff --git a/drivers/net/virtio/virtio_user/virtio_user_pci.c b/drivers/net/virtio/virtio_user/virtio_user_pci.c
new file mode 100644
index 0000000..b56419b
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_pci.c
@@ -0,0 +1,218 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <sys/types.h>
+#include <unistd.h>
+
+#include "../virtio_logs.h"
+#include "../virtio_pci.h"
+#include "../virtqueue.h"
+#include "virtio_user_dev.h"
+
+#define virtio_user_get_dev(hw) \
+	((struct virtio_user_dev *)(hw)->virtio_user_dev);
+
+static void
+vdev_read_dev_config(struct virtio_hw *hw, uint64_t offset,
+		     void *dst, int length)
+{
+	int i;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (offset == offsetof(struct virtio_net_config, mac) &&
+	    length == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			((uint8_t *)dst)[i] = dev->mac_addr[i];
+		return;
+	}
+
+	if (offset == offsetof(struct virtio_net_config, status))
+		*(uint16_t *)dst = dev->status;
+
+	if (offset == offsetof(struct virtio_net_config, max_virtqueue_pairs))
+		*(uint16_t *)dst = dev->max_queue_pairs;
+}
+
+static void
+vdev_write_dev_config(struct virtio_hw *hw, uint64_t offset,
+		      const void *src, int length)
+{
+	int i;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if ((offset == offsetof(struct virtio_net_config, mac)) &&
+	    (length == ETHER_ADDR_LEN))
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			dev->mac_addr[i] = ((const uint8_t *)src)[i];
+	else
+		PMD_DRV_LOG(ERR, "not supported offset=%" PRIu64 ", len=%d\n",
+			    offset, length);
+}
+
+static void
+vdev_set_status(struct virtio_hw *hw, uint8_t status)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (status & VIRTIO_CONFIG_STATUS_DRIVER_OK)
+		virtio_user_start_device(dev);
+	dev->status = status;
+}
+
+static void
+vdev_reset(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	virtio_user_stop_device(dev);
+}
+
+static uint8_t
+vdev_get_status(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	return dev->status;
+}
+
+static uint64_t
+vdev_get_features(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	return dev->features;
+}
+
+static void
+vdev_set_features(struct virtio_hw *hw, uint64_t features)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	dev->features = features;
+}
+
+static uint8_t
+vdev_get_isr(struct virtio_hw *hw __rte_unused)
+{
+	/* When config interrupt happens, driver calls this function to query
+	 * what kinds of change happen. Interrupt mode not supported for now.
+	 */
+	return 0;
+}
+
+static uint16_t
+vdev_set_config_irq(struct virtio_hw *hw __rte_unused,
+		    uint16_t vec __rte_unused)
+{
+	return VIRTIO_MSI_NO_VECTOR;
+}
+
+/* This function is to get the queue size, aka, number of descs, of a specified
+ * queue. Different with the VHOST_USER_GET_QUEUE_NUM, which is used to get the
+ * max supported queues.
+ */
+static uint16_t
+vdev_get_queue_num(struct virtio_hw *hw, uint16_t queue_id __rte_unused)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	/* Currently, each queue has same queue size */
+	return dev->queue_size;
+}
+
+static int
+vdev_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+	uint16_t queue_idx = vq->vq_queue_index;
+	uint64_t desc_addr, avail_addr, used_addr;
+
+	desc_addr = (uint64_t)vq->mz->addr;
+	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
+	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
+							 ring[vq->vq_nentries]),
+				   VIRTIO_PCI_VRING_ALIGN);
+
+	dev->vrings[queue_idx].num = vq->vq_nentries;
+	dev->vrings[queue_idx].desc = (void *)desc_addr;
+	dev->vrings[queue_idx].avail = (void *)avail_addr;
+	dev->vrings[queue_idx].used = (void *)used_addr;
+
+	return 0;
+}
+
+static void
+vdev_del_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	/* For legacy devices, write 0 to VIRTIO_PCI_QUEUE_PFN port, QEMU
+	 * correspondingly stops the ioeventfds, and reset the status of
+	 * the device.
+	 * For modern devices, set queue desc, avail, used in PCI bar to 0,
+	 * not see any more behavior in QEMU.
+	 *
+	 * Here we just care about what information to deliver to vhost-user
+	 * or vhost-kernel. So we just close ioeventfd for now.
+	 */
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	close(dev->callfds[vq->vq_queue_index]);
+	close(dev->kickfds[vq->vq_queue_index]);
+}
+
+static void
+vdev_notify_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	uint64_t buf = 1;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (write(dev->kickfds[vq->vq_queue_index], &buf, sizeof(buf)) < 0)
+		PMD_DRV_LOG(ERR, "failed to kick backend: %s\n",
+			    strerror(errno));
+}
+
+const struct virtio_pci_ops vdev_ops = {
+	.read_dev_cfg	= vdev_read_dev_config,
+	.write_dev_cfg	= vdev_write_dev_config,
+	.reset		= vdev_reset,
+	.get_status	= vdev_get_status,
+	.set_status	= vdev_set_status,
+	.get_features	= vdev_get_features,
+	.set_features	= vdev_set_features,
+	.get_isr	= vdev_get_isr,
+	.set_config_irq	= vdev_set_config_irq,
+	.get_queue_num	= vdev_get_queue_num,
+	.setup_queue	= vdev_setup_queue,
+	.del_queue	= vdev_del_queue,
+	.notify_queue	= vdev_notify_queue,
+};
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v5 7/8] virtio-user: add a new vdev named virtio-user
  2016-05-30 10:55 ` [PATCH v5 " Jianfeng Tan
                     ` (5 preceding siblings ...)
  2016-05-30 10:55   ` [PATCH v5 6/8] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
@ 2016-05-30 10:55   ` Jianfeng Tan
  2016-06-01  8:26     ` Yuanhan Liu
  2016-05-30 10:55   ` [PATCH v5 8/8] doc: update doc for virtio-user Jianfeng Tan
  7 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-05-30 10:55 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Add a new virtual device named vhost-user, which can be used just like
eth_ring, eth_null, etc. To reuse the code of original virtio, we do
some adjustment in virtio_ethdev.c, such as remove key _static_ of
eth_virtio_dev_init() so that it can be reused in virtual device; and
we add some check to make sure it will not crash.

Configured parameters include:
  - queues (optional, 1 by default), number of queue pairs, multi-queue
    not supported for now.
  - cq (optional, 0 by default), not supported for now.
  - mac (optional), random value will be given if not specified.
  - queue_size (optional, 256 by default), size of virtqueues.
  - path (madatory), path of vhost, depends on the file type, vhost
    user if the given path points to a unix socket; vhost-net if the
    given path points to a char device.
  - ifname (optional), specify the name of backend tap device; only
    valid when backend is vhost-net.

When enable CONFIG_RTE_VIRTIO_VDEV (enabled by default), the compiled
library can be used in both VM and container environment.

Examples:
path_vhost=/dev/vhost-net # use vhost-net as a backend
path_vhost=<path_to_vhost_user> # use vhost-user as a backend

sudo ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
    --socket-mem 0,1024 --no-pci --file-prefix=l2fwd \
    --vdev=virtio-user0,mac=00:01:02:03:04:05,path=$path_vhost -- -p 0x1

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/virtio_ethdev.c               |  19 +-
 drivers/net/virtio/virtio_ethdev.h               |   2 +
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 309 +++++++++++++++++++++++
 3 files changed, 323 insertions(+), 7 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 1866afd..f8972f2 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -59,7 +59,6 @@
 #include "virtqueue.h"
 #include "virtio_rxtx.h"
 
-static int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
 static int eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev);
 static int  virtio_dev_configure(struct rte_eth_dev *dev);
 static int  virtio_dev_start(struct rte_eth_dev *dev);
@@ -1038,7 +1037,7 @@ rx_func_get(struct rte_eth_dev *eth_dev)
  * This function is based on probe() function in virtio_pci.c
  * It returns 0 on success.
  */
-static int
+int
 eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 {
 	struct virtio_hw *hw = eth_dev->data->dev_private;
@@ -1069,9 +1068,11 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	pci_dev = eth_dev->pci_dev;
 
-	ret = vtpci_init(pci_dev, hw, &dev_flags);
-	if (ret)
-		return ret;
+	if (pci_dev) {
+		ret = vtpci_init(pci_dev, hw, &dev_flags);
+		if (ret)
+			return ret;
+	}
 
 	/* Reset the device although not necessary at startup */
 	vtpci_reset(hw);
@@ -1163,7 +1164,8 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	PMD_INIT_LOG(DEBUG, "hw->max_rx_queues=%d   hw->max_tx_queues=%d",
 			hw->max_rx_queues, hw->max_tx_queues);
-	PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
+	if (pci_dev)
+		PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
 			eth_dev->data->port_id, pci_dev->id.vendor_id,
 			pci_dev->id.device_id);
 
@@ -1442,7 +1444,10 @@ virtio_dev_info_get(struct rte_eth_dev *dev, struct rte_eth_dev_info *dev_info)
 {
 	struct virtio_hw *hw = dev->data->dev_private;
 
-	dev_info->driver_name = dev->driver->pci_drv.name;
+	if (dev->pci_dev)
+		dev_info->driver_name = dev->driver->pci_drv.name;
+	else
+		dev_info->driver_name = "virtio-user PMD";
 	dev_info->max_rx_queues = (uint16_t)hw->max_rx_queues;
 	dev_info->max_tx_queues = (uint16_t)hw->max_tx_queues;
 	dev_info->min_rx_bufsize = VIRTIO_MIN_RX_BUFSIZE;
diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
index 66423a0..284afaa 100644
--- a/drivers/net/virtio/virtio_ethdev.h
+++ b/drivers/net/virtio/virtio_ethdev.h
@@ -113,6 +113,8 @@ uint16_t virtio_recv_pkts_vec(void *rx_queue, struct rte_mbuf **rx_pkts,
 uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 		uint16_t nb_pkts);
 
+int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
+
 /*
  * The VIRTIO_NET_F_GUEST_TSO[46] features permit the host to send us
  * frames larger than 1514 bytes. We do not yet support software LRO
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.c b/drivers/net/virtio/virtio_user/virtio_user_dev.c
index 41d8ad1..5e4f60b 100644
--- a/drivers/net/virtio/virtio_user/virtio_user_dev.c
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.c
@@ -166,3 +166,312 @@ int virtio_user_stop_device(struct virtio_user_dev *dev)
 	return vhost_call(dev->vhostfd, dev->type, VHOST_MSG_RESET_OWNER, NULL);
 }
 
+static inline void parse_mac(struct virtio_user_dev *dev, const char *mac)
+{
+	int i, r;
+	uint32_t tmp[ETHER_ADDR_LEN];
+
+	if (!mac)
+		return;
+
+	r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
+			&tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
+	if (r == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			dev->mac_addr[i] = (uint8_t)tmp[i];
+		dev->mac_specified = 1;
+	} else {
+		/* ignore the wrong mac, use random mac */
+		PMD_DRV_LOG(ERR, "wrong format of mac: %s", mac);
+	}
+}
+
+static int
+virtio_user_dev_init(struct virtio_user_dev *dev, char *path, int queues,
+		 int queue_size, const char *mac, char *ifname)
+{
+	struct stat s;
+
+	strncpy(dev->path, path, PATH_MAX);
+	dev->max_queue_pairs = queues;
+	dev->queue_pairs = 1; /* mq disabled by default */
+	dev->queue_size = queue_size;
+	dev->mac_specified = 0;
+	parse_mac(dev, mac);
+	dev->vhostfd = -1;
+	dev->tapfd = -1;
+
+	if (stat(dev->path, &s) < 0) {
+		PMD_INIT_LOG(ERR, "stat: %s failed, %s", dev->path,
+			     strerror(errno));
+		return -1;
+	}
+
+	switch (s.st_mode & S_IFMT) {
+	case S_IFCHR:
+		dev->type = VHOST_KERNEL;
+		dev->vhostfd = vhost_kernel_setup(dev->path, ifname,
+						  &dev->tapfd);
+		break;
+	case S_IFSOCK:
+		dev->type = VHOST_USER;
+		dev->vhostfd = vhost_user_setup(dev->path);
+		break;
+	default:
+		PMD_INIT_LOG(ERR, "unknown file type of %s", dev->path);
+		return -1;
+	}
+	if (dev->vhostfd < 0) {
+		PMD_INIT_LOG(ERR, "backend set up fails");
+		return -1;
+	}
+	if (vhost_call(dev->vhostfd, dev->type,
+			VHOST_MSG_SET_OWNER, NULL) < 0) {
+		PMD_INIT_LOG(ERR, "set_owner fails: %s", strerror(errno));
+		return -1;
+	}
+
+	if (vhost_call(dev->vhostfd, dev->type,
+			VHOST_MSG_GET_FEATURES, &dev->features) < 0) {
+		PMD_INIT_LOG(ERR, "get_features failed: %s", strerror(errno));
+		return -1;
+	}
+	if (dev->mac_specified)
+		dev->features |= (1ull << VIRTIO_NET_F_MAC);
+	/* disable it until we support CQ */
+	dev->features &= ~(1ull << VIRTIO_NET_F_CTRL_VQ);
+	dev->features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
+
+	return 0;
+
+}
+
+static void
+virtio_user_dev_uninit(struct virtio_user_dev *dev)
+{
+	uint32_t i;
+
+	if (dev->type == VHOST_KERNEL)
+		close(dev->tapfd);
+
+	for (i = 0; i < dev->queue_pairs * 2; ++i) {
+		close(dev->callfds[i]);
+		close(dev->kickfds[i]);
+	}
+
+	close(dev->vhostfd);
+}
+
+static const char *valid_args[] = {
+#define VIRTIO_USER_ARG_QUEUES_NUM     "queues"
+	VIRTIO_USER_ARG_QUEUES_NUM,
+#define VIRTIO_USER_ARG_CQ_NUM         "cq"
+	VIRTIO_USER_ARG_CQ_NUM,
+#define VIRTIO_USER_ARG_MAC            "mac"
+	VIRTIO_USER_ARG_MAC,
+#define VIRTIO_USER_ARG_PATH           "path"
+	VIRTIO_USER_ARG_PATH,
+#define VIRTIO_USER_ARG_QUEUE_SIZE     "queue_size"
+	VIRTIO_USER_ARG_QUEUE_SIZE,
+#define VIRTIO_USER_ARG_IFNAME         "ifname"
+	VIRTIO_USER_ARG_IFNAME,
+	NULL
+};
+
+#define VIRTIO_USER_DEF_CQ_EN	0
+#define VIRTIO_USER_DEF_Q_NUM	1
+#define VIRTIO_USER_DEF_Q_SZ	256
+
+static int
+get_string_arg(const char *key __rte_unused,
+	       const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(char **)extra_args = strdup(value);
+
+	return 0;
+}
+
+static int
+get_integer_arg(const char *key __rte_unused,
+		const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(uint64_t *)extra_args = strtoull(value, NULL, 0);
+
+	return 0;
+}
+
+static struct rte_eth_dev *
+virtio_user_eth_dev_alloc(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct rte_eth_dev_data *data;
+	struct virtio_hw *hw;
+	struct virtio_user_dev *dev;
+
+	eth_dev = rte_eth_dev_allocate(name, RTE_ETH_DEV_VIRTUAL);
+	if (!eth_dev) {
+		PMD_INIT_LOG(ERR, "cannot alloc rte_eth_dev");
+		return NULL;
+	}
+
+	data = eth_dev->data;
+
+	hw = rte_zmalloc(NULL, sizeof(*hw), 0);
+	if (!hw) {
+		PMD_INIT_LOG(ERR, "malloc virtio_hw failed");
+		rte_eth_dev_release_port(eth_dev);
+		return NULL;
+	}
+
+	dev = rte_zmalloc(NULL, sizeof(*dev), 0);
+	if (!dev) {
+		PMD_INIT_LOG(ERR, "malloc virtio_user_dev failed");
+		rte_eth_dev_release_port(eth_dev);
+		rte_free(hw);
+		return NULL;
+	}
+
+	hw->vtpci_ops = &vdev_ops;
+	hw->use_msix = 0;
+	hw->modern   = 0;
+	hw->virtio_user_dev = dev;
+	data->dev_private = hw;
+	data->numa_node = SOCKET_ID_ANY;
+	data->kdrv = RTE_KDRV_NONE;
+	data->dev_flags = RTE_ETH_DEV_DETACHABLE;
+	eth_dev->pci_dev = NULL;
+	eth_dev->driver = NULL;
+	return eth_dev;
+}
+
+/* Dev initialization routine. Invoked once for each virtio vdev at
+ * EAL init time, see rte_eal_dev_init().
+ * Returns 0 on success.
+ */
+static int
+virtio_user_pmd_devinit(const char *name, const char *params)
+{
+	struct rte_kvargs *kvlist;
+	struct rte_eth_dev *eth_dev;
+	struct virtio_hw *hw;
+	uint64_t queues = VIRTIO_USER_DEF_Q_NUM;
+	uint64_t nb_cq = VIRTIO_USER_DEF_CQ_EN;
+	uint64_t queue_size = VIRTIO_USER_DEF_Q_SZ;
+	char *path = NULL;
+	char *mac_addr = NULL;
+	char *ifname = NULL;
+	int ret = -1;
+
+	if (!params || params[0] == '\0') {
+		PMD_INIT_LOG(ERR, "arg %s is mandatory for virtio-user",
+			  VIRTIO_USER_ARG_QUEUE_SIZE);
+		goto end;
+	}
+
+	kvlist = rte_kvargs_parse(params, valid_args);
+	if (!kvlist) {
+		PMD_INIT_LOG(ERR, "error when parsing param");
+		goto end;
+	}
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_PATH) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_PATH,
+				   &get_string_arg, &path);
+	else {
+		PMD_INIT_LOG(ERR, "arg %s is mandatory for virtio-user\n",
+			  VIRTIO_USER_ARG_QUEUE_SIZE);
+		goto end;
+	}
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_MAC) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_MAC,
+				   &get_string_arg, &mac_addr);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_IFNAME) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_IFNAME,
+				   &get_string_arg, &ifname);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_QUEUE_SIZE) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_QUEUE_SIZE,
+				   &get_integer_arg, &queue_size);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_QUEUES_NUM) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_QUEUES_NUM,
+				   &get_integer_arg, &queues);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_CQ_NUM) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_CQ_NUM,
+				   &get_integer_arg, &nb_cq);
+
+	eth_dev = virtio_user_eth_dev_alloc(name);
+	if (!eth_dev) {
+		PMD_INIT_LOG(ERR, "virtio-user fails to alloc device");
+		goto end;
+	}
+
+	hw = eth_dev->data->dev_private;
+	if (virtio_user_dev_init(hw->virtio_user_dev, path, queues,
+			     queue_size, mac_addr, ifname) < 0)
+		goto end;
+
+	/* previously called by rte_eal_pci_probe() for physical dev */
+	if (eth_virtio_dev_init(eth_dev) < 0) {
+		PMD_INIT_LOG(ERR, "eth_virtio_dev_init fails");
+		goto end;
+	}
+	ret = 0;
+
+end:
+	if (path)
+		free(path);
+	if (mac_addr)
+		free(mac_addr);
+	if (ifname)
+		free(ifname);
+	return ret;
+}
+
+/** Called by rte_eth_dev_detach() */
+static int
+virtio_user_pmd_devuninit(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct virtio_hw *hw;
+	struct virtio_user_dev *dev;
+
+	if (!name)
+		return -EINVAL;
+
+	PMD_DRV_LOG(INFO, "Un-Initializing %s\n", name);
+	eth_dev = rte_eth_dev_allocated(name);
+	if (!eth_dev)
+		return -ENODEV;
+
+	/* make sure the device is stopped, queues freed */
+	rte_eth_dev_close(eth_dev->data->port_id);
+
+	hw = eth_dev->data->dev_private;
+	dev = hw->virtio_user_dev;
+	virtio_user_dev_uninit(dev);
+
+	rte_free(eth_dev->data->dev_private);
+	rte_free(eth_dev->data);
+	rte_eth_dev_release_port(eth_dev);
+
+	return 0;
+}
+
+static struct rte_driver virtio_user_driver = {
+	.name   = "virtio-user",
+	.type   = PMD_VDEV,
+	.init   = virtio_user_pmd_devinit,
+	.uninit = virtio_user_pmd_devuninit,
+};
+
+PMD_REGISTER_DRIVER(virtio_user_driver);
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v5 8/8] doc: update doc for virtio-user
  2016-05-30 10:55 ` [PATCH v5 " Jianfeng Tan
                     ` (6 preceding siblings ...)
  2016-05-30 10:55   ` [PATCH v5 7/8] virtio-user: add a new vdev named virtio-user Jianfeng Tan
@ 2016-05-30 10:55   ` Jianfeng Tan
  2016-06-01  8:30     ` Yuanhan Liu
  7 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-05-30 10:55 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 doc/guides/rel_notes/release_16_07.rst | 4 ++++
 1 file changed, 4 insertions(+)

diff --git a/doc/guides/rel_notes/release_16_07.rst b/doc/guides/rel_notes/release_16_07.rst
index f6d543c..b1054b6 100644
--- a/doc/guides/rel_notes/release_16_07.rst
+++ b/doc/guides/rel_notes/release_16_07.rst
@@ -34,6 +34,10 @@ This section should contain new features added in this release. Sample format:
 
   Refer to the previous release notes for examples.
 
+* **Virtio support for containers.**
+
+  Add a new virtual device, named virtio-user, to support virtio for containers.
+
 
 Resolved Issues
 ---------------
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* Re: [PATCH v5 2/8] virtio: clean up virtio_dev_queue_setup
  2016-05-30 10:55   ` [PATCH v5 2/8] virtio: clean up virtio_dev_queue_setup Jianfeng Tan
@ 2016-06-01  7:38     ` Yuanhan Liu
  2016-06-01  7:44       ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2016-06-01  7:38 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

On Mon, May 30, 2016 at 10:55:33AM +0000, Jianfeng Tan wrote:
> Abstract vring hdr desc init as an inline method.

What's this patch for then? In your last version, it will be invoked
twice, but it turned out to be wrong. So, why keeping this change?
I didn't see it improves anything.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v5 2/8] virtio: clean up virtio_dev_queue_setup
  2016-06-01  7:38     ` Yuanhan Liu
@ 2016-06-01  7:44       ` Tan, Jianfeng
  2016-06-01  7:58         ` Yuanhan Liu
  0 siblings, 1 reply; 196+ messages in thread
From: Tan, Jianfeng @ 2016-06-01  7:44 UTC (permalink / raw)
  To: Yuanhan Liu
  Cc: dev, Xie, Huawei, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

Hi Yuanhan,

> -----Original Message-----
> From: Yuanhan Liu [mailto:yuanhan.liu@linux.intel.com]
> Sent: Wednesday, June 1, 2016 3:38 PM
> To: Tan, Jianfeng
> Cc: dev@dpdk.org; Xie, Huawei; rich.lane@bigswitch.com; mst@redhat.com;
> nakajima.yoshihiro@lab.ntt.co.jp; p.fedin@samsung.com;
> ann.zhuangyanying@huawei.com; mukawa@igel.co.jp;
> nhorman@tuxdriver.com
> Subject: Re: [PATCH v5 2/8] virtio: clean up virtio_dev_queue_setup
> 
> On Mon, May 30, 2016 at 10:55:33AM +0000, Jianfeng Tan wrote:
> > Abstract vring hdr desc init as an inline method.
> 
> What's this patch for then? In your last version, it will be invoked
> twice, but it turned out to be wrong. So, why keeping this change?
> I didn't see it improves anything.
> 

Yes, for now, only one version is kept because the position to call this function is changed. And I think this segment of code functions as a special purpose, which can be abstracted as a function, make sense?

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v5 2/8] virtio: clean up virtio_dev_queue_setup
  2016-06-01  7:44       ` Tan, Jianfeng
@ 2016-06-01  7:58         ` Yuanhan Liu
  0 siblings, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-06-01  7:58 UTC (permalink / raw)
  To: Tan, Jianfeng
  Cc: dev, Xie, Huawei, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

On Wed, Jun 01, 2016 at 07:44:33AM +0000, Tan, Jianfeng wrote:
> Hi Yuanhan,
> 
> > -----Original Message-----
> > From: Yuanhan Liu [mailto:yuanhan.liu@linux.intel.com]
> > Sent: Wednesday, June 1, 2016 3:38 PM
> > To: Tan, Jianfeng
> > Cc: dev@dpdk.org; Xie, Huawei; rich.lane@bigswitch.com; mst@redhat.com;
> > nakajima.yoshihiro@lab.ntt.co.jp; p.fedin@samsung.com;
> > ann.zhuangyanying@huawei.com; mukawa@igel.co.jp;
> > nhorman@tuxdriver.com
> > Subject: Re: [PATCH v5 2/8] virtio: clean up virtio_dev_queue_setup
> > 
> > On Mon, May 30, 2016 at 10:55:33AM +0000, Jianfeng Tan wrote:
> > > Abstract vring hdr desc init as an inline method.
> > 
> > What's this patch for then? In your last version, it will be invoked
> > twice, but it turned out to be wrong. So, why keeping this change?
> > I didn't see it improves anything.
> > 
> 
> Yes, for now, only one version is kept because the position to call this function is changed. And I think this segment of code functions as a special purpose, which can be abstracted as a function, make sense?

Yeah, maybe. But idealy, we should move it to tx_queue_setup() function.
Let's see what we might do after applying Huawei's rx/tx split patch: it
needs a while (I saw bugs).

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v5 3/8] virtio: enable use virtual address to fill desc
  2016-05-30 10:55   ` [PATCH v5 3/8] virtio: enable use virtual address to fill desc Jianfeng Tan
@ 2016-06-01  8:03     ` Yuanhan Liu
  0 siblings, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-06-01  8:03 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

On Mon, May 30, 2016 at 10:55:34AM +0000, Jianfeng Tan wrote:
> This patch is related to how to calculate relative address for vhost
> backend.
> 
> The principle is that: based on one or multiple shared memory regions,
> vhost maintains a reference system with the frontend start address,
> backend start address, and length for each segment, so that each
> frontend address (GPA, Guest Physical Address) can be translated into
> vhost-recognizable backend address. To make the address translation
> efficient, we need to maintain as few regions as possible. In the case
> of VM, GPA is always locally continuous. But for some other case, like
> virtio-user, we use virtual address here.
> 
> It basically means:
>   a. when set_base_addr, VA address is used;
>   b. when preparing RX's descriptors, VA address is used;
>   c. when transmitting packets, VA is filled in TX's descriptors;
>   d. in TX and CQ's header, VA is used.
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> Acked-by: Neil Horman <nhorman@tuxdriver.com>
> ---
>  drivers/net/virtio/virtio_ethdev.c      | 21 ++++++++++++++++-----
>  drivers/net/virtio/virtio_rxtx.c        |  5 ++---
>  drivers/net/virtio/virtio_rxtx_simple.c | 13 +++++++------
>  drivers/net/virtio/virtqueue.h          | 13 ++++++++++++-
>  4 files changed, 37 insertions(+), 15 deletions(-)
> 
> @@ -419,8 +419,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
>  		vq->virtio_net_hdr_mem = hdr_mz->phys_addr;
>  
>  		memset(hdr_mz->addr, 0, hdr_mz_sz);
> -		vring_hdr_desc_init(vq);
> -
>  	} else if (queue_type == VTNET_CQ) {
>  		/* Allocate a page for control vq command, data and status */
>  		snprintf(vq_name, sizeof(vq_name), "port%d_cvq_hdrzone",
> @@ -441,6 +439,19 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
>  		memset(vq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
>  	}
>  
> +	if (dev->pci_dev)
> +		vq->offset = offsetof(struct rte_mbuf, buf_physaddr);
> +	else {
> +		vq->vq_ring_mem = (phys_addr_t)vq->mz->addr;
> +		vq->offset = offsetof(struct rte_mbuf, buf_addr);
> +		if (vq->virtio_net_hdr_mz)
> +			vq->virtio_net_hdr_mem =
> +				(phys_addr_t)vq->virtio_net_hdr_mz->addr;
> +	}

I guess this piece of code deserves some comments. Say, for virtio-user
case (that is when dev->pci_dev is NULL), we use virtual address,
because, bala, bala ...


> @@ -165,6 +173,7 @@ struct virtqueue {
>  	void        *vq_ring_virt_mem;    /**< linear address of vring*/
>  	unsigned int vq_ring_size;
>  	phys_addr_t vq_ring_mem;          /**< physical address of vring */
> +					  /**< use virtual address for vdev. */

Replace vdev with "virtio-user" is better here?

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v5 6/8] virtio-user: add new virtual pci driver for virtio
  2016-05-30 10:55   ` [PATCH v5 6/8] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
@ 2016-06-01  8:21     ` Yuanhan Liu
  0 siblings, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-06-01  8:21 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

On Mon, May 30, 2016 at 10:55:37AM +0000, Jianfeng Tan wrote:
> This patch implements another new instance of struct virtio_pci_ops to
> drive the virtio-user virtual device. Instead of rd/wr ioport or PCI
> configuration space, this virtual pci driver will rd/wr the virtual
> device struct virtio_user_hw, and when necessary, invokes APIs provided
> by device emulation later to start/stop the device.
> 
>   ----------------------
>   | ------------------ |
>   | | virtio driver  | |----> (virtio_user_pci.c)
>   | ------------------ |
>   |         |          |
>   | ------------------ | ------>  virtio-user PMD
>   | | device emulate | |
>   | |                | |
>   | | vhost adapter  | |
>   | ------------------ |
>   ----------------------
>             |
>             |
>             |
>    ------------------
>    | vhost backend  |
>    ------------------
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> Acked-by: Neil Horman <nhorman@tuxdriver.com>
> ---
>  drivers/net/virtio/Makefile                      |   1 +
>  drivers/net/virtio/virtio_pci.h                  |   1 +
>  drivers/net/virtio/virtio_user/virtio_user_dev.h |   2 +
>  drivers/net/virtio/virtio_user/virtio_user_pci.c | 218 +++++++++++++++++++++++

Jianfeng, this file (virtio_user_pci.c) belongs to virtio driver, but
not virtio-user device.

I thought it a while, I found it's still better to introduce a helper
function from virtio-user device for each ops, just like the below
vdev_reset(). Sorry for noisy.

And I'd suggest to rename it to virtio_user.c, so that we can move
the virtio_user_pmd_devinit/devuninit in the next patch to there,
too. They also belong the driver side, but not the device side.

> +static void
> +vdev_read_dev_config(struct virtio_hw *hw, uint64_t offset,
> +		     void *dst, int length)

Let's user "virtio_user" consistently here, instead of vdev?

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v5 7/8] virtio-user: add a new vdev named virtio-user
  2016-05-30 10:55   ` [PATCH v5 7/8] virtio-user: add a new vdev named virtio-user Jianfeng Tan
@ 2016-06-01  8:26     ` Yuanhan Liu
  2016-06-02  1:27       ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2016-06-01  8:26 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

On Mon, May 30, 2016 at 10:55:38AM +0000, Jianfeng Tan wrote:
> diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.c b/drivers/net/virtio/virtio_user/virtio_user_dev.c
> index 41d8ad1..5e4f60b 100644
> --- a/drivers/net/virtio/virtio_user/virtio_user_dev.c
> +++ b/drivers/net/virtio/virtio_user/virtio_user_dev.c
> @@ -166,3 +166,312 @@ int virtio_user_stop_device(struct virtio_user_dev *dev)
>  	return vhost_call(dev->vhostfd, dev->type, VHOST_MSG_RESET_OWNER, NULL);
>  }
>  
> +static inline void parse_mac(struct virtio_user_dev *dev, const char *mac)

Note that this is a slight coding style offensive.

> +{
> +	int i, r;
> +	uint32_t tmp[ETHER_ADDR_LEN];
> +
> +	if (!mac)
> +		return;
> +
> +	r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
> +			&tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
> +	if (r == ETHER_ADDR_LEN) {
> +		for (i = 0; i < ETHER_ADDR_LEN; ++i)
> +			dev->mac_addr[i] = (uint8_t)tmp[i];
> +		dev->mac_specified = 1;
> +	} else {
> +		/* ignore the wrong mac, use random mac */
> +		PMD_DRV_LOG(ERR, "wrong format of mac: %s", mac);
> +	}
> +}
> +
> +static int
> +virtio_user_dev_init(struct virtio_user_dev *dev, char *path, int queues,
> +		 int queue_size, const char *mac, char *ifname)

As stated in last email, we should move all others (except above 2
functions) to the driver layer, where they belong to.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v5 8/8] doc: update doc for virtio-user
  2016-05-30 10:55   ` [PATCH v5 8/8] doc: update doc for virtio-user Jianfeng Tan
@ 2016-06-01  8:30     ` Yuanhan Liu
  0 siblings, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-06-01  8:30 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

On Mon, May 30, 2016 at 10:55:39AM +0000, Jianfeng Tan wrote:
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> ---
>  doc/guides/rel_notes/release_16_07.rst | 4 ++++
>  1 file changed, 4 insertions(+)
> 
> diff --git a/doc/guides/rel_notes/release_16_07.rst b/doc/guides/rel_notes/release_16_07.rst
> index f6d543c..b1054b6 100644
> --- a/doc/guides/rel_notes/release_16_07.rst
> +++ b/doc/guides/rel_notes/release_16_07.rst
> @@ -34,6 +34,10 @@ This section should contain new features added in this release. Sample format:
>  
>    Refer to the previous release notes for examples.
>  
> +* **Virtio support for containers.**
> +
> +  Add a new virtual device, named virtio-user, to support virtio for containers.

Note that Thomas have stated quite many times, that we should keep the
release note in the same patch that actually enabled the feature.

Besides that, I guess we should state the current limitation here. It
might be even better if we could make a howto doc, say in the vhost
example doc.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v5 7/8] virtio-user: add a new vdev named virtio-user
  2016-06-01  8:26     ` Yuanhan Liu
@ 2016-06-02  1:27       ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-06-02  1:27 UTC (permalink / raw)
  To: Yuanhan Liu
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

Hi Yuanhan,


On 6/1/2016 4:26 PM, Yuanhan Liu wrote:
> On Mon, May 30, 2016 at 10:55:38AM +0000, Jianfeng Tan wrote:
>> diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.c b/drivers/net/virtio/virtio_user/virtio_user_dev.c
>> index 41d8ad1..5e4f60b 100644
>> --- a/drivers/net/virtio/virtio_user/virtio_user_dev.c
>> +++ b/drivers/net/virtio/virtio_user/virtio_user_dev.c
>> @@ -166,3 +166,312 @@ int virtio_user_stop_device(struct virtio_user_dev *dev)
>>   	return vhost_call(dev->vhostfd, dev->type, VHOST_MSG_RESET_OWNER, NULL);
>>   }
>>   
>> +static inline void parse_mac(struct virtio_user_dev *dev, const char *mac)
> Note that this is a slight coding style offensive.

OK, I'll fix it.

>
>> +{
>> +	int i, r;
>> +	uint32_t tmp[ETHER_ADDR_LEN];
>> +
>> +	if (!mac)
>> +		return;
>> +
>> +	r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
>> +			&tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
>> +	if (r == ETHER_ADDR_LEN) {
>> +		for (i = 0; i < ETHER_ADDR_LEN; ++i)
>> +			dev->mac_addr[i] = (uint8_t)tmp[i];
>> +		dev->mac_specified = 1;
>> +	} else {
>> +		/* ignore the wrong mac, use random mac */
>> +		PMD_DRV_LOG(ERR, "wrong format of mac: %s", mac);
>> +	}
>> +}
>> +
>> +static int
>> +virtio_user_dev_init(struct virtio_user_dev *dev, char *path, int queues,
>> +		 int queue_size, const char *mac, char *ifname)
> As stated in last email, we should move all others (except above 2
> functions) to the driver layer, where they belong to.

OK, I'll create a virtio_user_ethdev.c to store these things.

Thanks,
Jianfeng

>
> 	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v6 0/7] virtio support for container
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
                   ` (10 preceding siblings ...)
  2016-05-30 10:55 ` [PATCH v5 " Jianfeng Tan
@ 2016-06-02  9:54 ` Jianfeng Tan
  2016-06-02  9:54   ` [PATCH v6 1/7] virtio: hide phys addr check inside pci ops Jianfeng Tan
                     ` (6 more replies)
  2016-06-12  0:35 ` [PATCH v7 0/6] virtio support for container Jianfeng Tan
                   ` (2 subsequent siblings)
  14 siblings, 7 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-02  9:54 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

v6:
 - Move driver related code into from driver/net/virtio/virtio-user/ to
   driver/net/virtio/ directory, inside virtio_user_ethdev.c.
 - Rename vdev to virtio_user in comments and code.
 - Merge code, which lies in virtio_user_pci.c, into virtio_user_ethdev.c.
 - Add some comments at virtio-user special handling at virtio_dev_ethdev.c.
 - Merge document update into the 7nd commit where virtio-user is added.
 - Add usage with vhost-switch in vhost.rst.

v5:
 - Rename struct virtio_user_hw to struct virtio_user_dev.
 - Rename "vdev_private" to "virtio_user_dev".
 - Move special handling into virtio_ethdev.c from queue_setup().
 - Add vring in virtio_user_dev (remove rte_eth_dev_data), so that
   device does not depend on driver's data structure (rte_eth_dev_data).
 - Remove update on doc/guides/nics/overview.rst, because virtio-user has
   exact feature set with virtio.
 - Change "unsigned long int" to "uint64_t", "unsigned" to "uint32_t".
 - Remove unnecessary cast in vdev_read_dev_config().
 - Add functions in virtio_user_dev.c with prefix of "virtio_user_".
 - Rebase on virtio-next-virtio.

v4:
 - Avoid using dev_type, instead use (eth_dev->pci_device is NULL) to
   judge if it's virtual device or physical device.
 - Change the added device name to virtio-user.
 - Split into vhost_user.c, vhost_kernel.c, vhost.c, virtio_user_pci.c,
   virtio_user_dev.c.
 - Move virtio-user specific data from struct virtio_hw into struct
   virtio_user_hw.
 - Add support to send reset_owner message.
 - Change del_queue implementation. (This need more check)
 - Remove rte_panic(), and superseded with log.
 - Add reset_owner into virtio_pci_ops.reset.
 - Merge parameter "rx" and "tx" to "queues" to emliminate confusion.
 - Move get_features to after set_owner.
 - Redefine path in virtio_user_hw from char * to char [].

v3:
 - Remove --single-file option; do no change at EAL memory.
 - Remove the added API rte_eal_get_backfile_info(), instead we check all
   opened files with HUGEFILE_FMT to find hugepage files owned by DPDK.
 - Accordingly, add more restrictions at "Known issue" section.
 - Rename parameter from queue_num to queue_size for confusion.
 - Rename vhost_embedded.c to rte_eth_virtio_vdev.c.
 - Move code related to the newly added vdev to rte_eth_virtio_vdev.c, to
   reuse eth_virtio_dev_init(), remove its static declaration.
 - Implement dev_uninit() for rte_eth_dev_detach().
 - WARN -> ERR, in vhost_embedded.c
 - Add more commit message for clarify the model.

v2:
 - Rebase on the patchset of virtio 1.0 support.
 - Fix cannot create non-hugepage memory.
 - Fix wrong size of memory region when "single-file" is used.
 - Fix setting of offset in virtqueue to use virtual address.
 - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
 - Add mac option to specify the mac address of this virtual device.
 - Update doc.

This patchset is to provide high performance networking interface (virtio)
for container-based DPDK applications. The way of starting DPDK apps in
containers with ownership of NIC devices exclusively is beyond the scope.
The basic idea here is to present a new virtual device (named virtio-user),
which can be discovered and initialized by DPDK. To minimize the change,
we reuse already-existing virtio PMD code (driver/net/virtio/).

Background: Previously, we usually use a virtio device in the context of
QEMU/VM as below pic shows. Virtio nic is emulated in QEMU, and usually
presented in VM as a PCI device.

  ------------------
  |  virtio driver |  ----->  VM
  ------------------
        |
        | ----------> (over PCI bus or MMIO or Channel I/O)
        |
  ------------------
  | device emulate |
  |                |  ----->  QEMU
  | vhost adapter  |
  ------------------
        |
        | ----------> (vhost-user protocol or vhost-net ioctls)
        |
  ------------------
  | vhost backend  |
  ------------------
 
Compared to QEMU/VM case, virtio support for contaner requires to embedded
device framework inside the virtio PMD. So this converged driver actually
plays three roles:
  - virtio driver to drive this new kind of virtual device;
  - device emulation to present this virtual device and reponse to the
    virtio driver, which is originally by QEMU;
  - and the role to communicate with vhost backend, which is also
    originally by QEMU.

The code layout and functionality of each module:
 
  ----------------------
  | ------------------ |
  | | virtio driver  | |----> (virtio_user_ethdev.c)
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate |-|----> (virtio_user_dev.c)
  | |                | |
  | | vhost adapter  |-|----> (vhost_user.c, vhost_kernel.c, vhost.c)
  | ------------------ |
  ----------------------
         |
         | -------------- --> (vhost-user protocol or vhost-net ioctls)
         |
   ------------------
   | vhost backend  |
   ------------------

How to share memory? In VM's case, qemu always shares all physical layout
to backend. But it's not feasible for a container, as a process, to share
all virtual memory regions to backend. So only specified virtual memory
regions (with type of shared) are sent to backend. It's a limitation that
only addresses in these areas can be used to transmit or receive packets.

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").

How to use?

a. Apply this patchset.

b. To compile container apps:
$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc

c. To build a docker image using Dockerfile below.
$: cat ./Dockerfile
FROM ubuntu:latest
WORKDIR /usr/src/dpdk
COPY . /usr/src/dpdk
ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
$: docker build -t dpdk-app-l2fwd .

d. Used with vhost-user
$: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
	--socket-mem 1024,1024 -- -p 0x1 --stats 1
$: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=virtio-user0,path=/var/run/usvhost -- -p 0x1

f. Used with vhost-net
$: modprobe vhost
$: modprobe vhost-net
$: docker run -i -t --privileged \
	-v /dev/vhost-net:/dev/vhost-net \
	-v /dev/net/tun:/dev/net/tun \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=virtio-user0,path=/dev/vhost-net -- -p 0x1

By the way, it's not necessary to run in a container.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>



Jianfeng Tan (7):
  virtio: hide phys addr check inside pci ops
  virtio: clean up virtio_dev_queue_setup
  virtio: enable use virtual address to fill desc
  virtio-user: add vhost adapter layer
  virtio-user: add device emulation layer APIs
  virtio-user: add new virtual pci driver for virtio
  virtio-user: add a new vdev named virtio-user

 config/common_linuxapp                           |   3 +
 doc/guides/rel_notes/release_16_07.rst           |  11 +
 doc/guides/sample_app_ug/vhost.rst               |  17 +
 drivers/net/virtio/Makefile                      |   8 +
 drivers/net/virtio/virtio_ethdev.c               | 101 +++---
 drivers/net/virtio/virtio_ethdev.h               |   2 +
 drivers/net/virtio/virtio_pci.c                  |  30 +-
 drivers/net/virtio/virtio_pci.h                  |   3 +-
 drivers/net/virtio/virtio_rxtx.c                 |   5 +-
 drivers/net/virtio/virtio_rxtx_simple.c          |  13 +-
 drivers/net/virtio/virtio_user/vhost.c           | 105 ++++++
 drivers/net/virtio/virtio_user/vhost.h           | 222 ++++++++++++
 drivers/net/virtio/virtio_user/vhost_kernel.c    | 254 +++++++++++++
 drivers/net/virtio/virtio_user/vhost_user.c      | 378 ++++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 263 ++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.h |  64 ++++
 drivers/net/virtio/virtio_user_ethdev.c          | 436 +++++++++++++++++++++++
 drivers/net/virtio/virtqueue.h                   |  13 +-
 18 files changed, 1874 insertions(+), 54 deletions(-)
 create mode 100644 drivers/net/virtio/virtio_user/vhost.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost.h
 create mode 100644 drivers/net/virtio/virtio_user/vhost_kernel.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h
 create mode 100644 drivers/net/virtio/virtio_user_ethdev.c

-- 
2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v6 1/7] virtio: hide phys addr check inside pci ops
  2016-06-02  9:54 ` [PATCH v6 0/7] virtio support for container Jianfeng Tan
@ 2016-06-02  9:54   ` Jianfeng Tan
  2016-06-02  9:54   ` [PATCH v6 2/7] virtio: clean up virtio_dev_queue_setup Jianfeng Tan
                     ` (5 subsequent siblings)
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-02  9:54 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is to move phys addr check from virtio_dev_queue_setup
to pci ops. To makt that happen, make sure virtio_ops.setup_queue
return the result if we pass through the check.

Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Acked-by: Yuanhan Liu <yuanhan.liu@linux.intel.com>
---
 drivers/net/virtio/virtio_ethdev.c | 17 +++++------------
 drivers/net/virtio/virtio_pci.c    | 30 ++++++++++++++++++++++++++++--
 drivers/net/virtio/virtio_pci.h    |  2 +-
 3 files changed, 34 insertions(+), 15 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index c3fb628..a3031e4 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -364,17 +364,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		}
 	}
 
-	/*
-	 * Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
-	 * and only accepts 32 bit page frame number.
-	 * Check if the allocated physical memory exceeds 16TB.
-	 */
-	if ((mz->phys_addr + vq->vq_ring_size - 1) >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
-		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
-		virtio_dev_queue_release(vq);
-		return -ENOMEM;
-	}
-
 	memset(mz->addr, 0, sizeof(mz->len));
 	vq->mz = mz;
 	vq->vq_ring_mem = mz->phys_addr;
@@ -446,7 +435,11 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		memset(vq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
 	}
 
-	hw->vtpci_ops->setup_queue(hw, vq);
+	if (hw->vtpci_ops->setup_queue(hw, vq) < 0) {
+		PMD_INIT_LOG(ERR, "setup_queue failed");
+		virtio_dev_queue_release(vq);
+		return -EINVAL;
+	}
 
 	vq->configured = 1;
 	*pvq = vq;
diff --git a/drivers/net/virtio/virtio_pci.c b/drivers/net/virtio/virtio_pci.c
index 9cdca06..6bd239c 100644
--- a/drivers/net/virtio/virtio_pci.c
+++ b/drivers/net/virtio/virtio_pci.c
@@ -55,6 +55,22 @@
  */
 #define VIRTIO_PCI_CONFIG(hw) (((hw)->use_msix) ? 24 : 20)
 
+static inline int
+check_vq_phys_addr_ok(struct virtqueue *vq)
+{
+	/* Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
+	 * and only accepts 32 bit page frame number.
+	 * Check if the allocated physical memory exceeds 16TB.
+	 */
+	if ((vq->vq_ring_mem + vq->vq_ring_size - 1) >>
+			(VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
+		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
+		return 0;
+	}
+
+	return 1;
+}
+
 static void
 legacy_read_dev_config(struct virtio_hw *hw, size_t offset,
 		       void *dst, int length)
@@ -143,15 +159,20 @@ legacy_get_queue_num(struct virtio_hw *hw, uint16_t queue_id)
 	return dst;
 }
 
-static void
+static int
 legacy_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 {
 	uint32_t src;
 
+	if (!check_vq_phys_addr_ok(vq))
+		return -1;
+
 	rte_eal_pci_ioport_write(&hw->io, &vq->vq_queue_index, 2,
 			 VIRTIO_PCI_QUEUE_SEL);
 	src = vq->mz->phys_addr >> VIRTIO_PCI_QUEUE_ADDR_SHIFT;
 	rte_eal_pci_ioport_write(&hw->io, &src, 4, VIRTIO_PCI_QUEUE_PFN);
+
+	return 0;
 }
 
 static void
@@ -367,12 +388,15 @@ modern_get_queue_num(struct virtio_hw *hw, uint16_t queue_id)
 	return io_read16(&hw->common_cfg->queue_size);
 }
 
-static void
+static int
 modern_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 {
 	uint64_t desc_addr, avail_addr, used_addr;
 	uint16_t notify_off;
 
+	if (!check_vq_phys_addr_ok(vq))
+		return -1;
+
 	desc_addr = vq->mz->phys_addr;
 	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
 	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
@@ -400,6 +424,8 @@ modern_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 	PMD_INIT_LOG(DEBUG, "\t used_addr: %" PRIx64, used_addr);
 	PMD_INIT_LOG(DEBUG, "\t notify addr: %p (notify offset: %u)",
 		vq->notify_addr, notify_off);
+
+	return 0;
 }
 
 static void
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index 554efea..a76daf7 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -234,7 +234,7 @@ struct virtio_pci_ops {
 	uint16_t (*set_config_irq)(struct virtio_hw *hw, uint16_t vec);
 
 	uint16_t (*get_queue_num)(struct virtio_hw *hw, uint16_t queue_id);
-	void (*setup_queue)(struct virtio_hw *hw, struct virtqueue *vq);
+	int (*setup_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 	void (*del_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 	void (*notify_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 };
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v6 2/7] virtio: clean up virtio_dev_queue_setup
  2016-06-02  9:54 ` [PATCH v6 0/7] virtio support for container Jianfeng Tan
  2016-06-02  9:54   ` [PATCH v6 1/7] virtio: hide phys addr check inside pci ops Jianfeng Tan
@ 2016-06-02  9:54   ` Jianfeng Tan
  2016-06-02  9:54   ` [PATCH v6 3/7] virtio: enable use virtual address to fill desc Jianfeng Tan
                     ` (4 subsequent siblings)
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-02  9:54 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Abstract vring hdr desc init as an inline method.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
---
 drivers/net/virtio/virtio_ethdev.c | 42 ++++++++++++++++++++++----------------
 1 file changed, 24 insertions(+), 18 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index a3031e4..781886d 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -278,6 +278,26 @@ virtio_dev_queue_release(struct virtqueue *vq)
 	}
 }
 
+static void
+vring_hdr_desc_init(struct virtqueue *vq)
+{
+	int i;
+	struct virtio_tx_region *txr = vq->virtio_net_hdr_mz->addr;
+
+	for (i = 0; i < vq->vq_nentries; i++) {
+		struct vring_desc *start_dp = txr[i].tx_indir;
+
+		vring_desc_init(start_dp, RTE_DIM(txr[i].tx_indir));
+
+		/* first indirect descriptor is always the tx header */
+		start_dp->addr = vq->virtio_net_hdr_mem + i * sizeof(*txr) +
+				 offsetof(struct virtio_tx_region, tx_hdr);
+
+		start_dp->len = vq->hw->vtnet_hdr_size;
+		start_dp->flags = VRING_DESC_F_NEXT;
+	}
+}
+
 int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 			int queue_type,
 			uint16_t queue_idx,
@@ -375,8 +395,7 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 
 	if (queue_type == VTNET_TQ) {
 		const struct rte_memzone *hdr_mz;
-		struct virtio_tx_region *txr;
-		unsigned int i;
+		size_t hdr_mz_sz = vq_size * sizeof(struct virtio_tx_region);
 
 		/*
 		 * For each xmit packet, allocate a virtio_net_hdr
@@ -385,7 +404,7 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		snprintf(vq_name, sizeof(vq_name), "port%d_tvq%d_hdrzone",
 			 dev->data->port_id, queue_idx);
 		hdr_mz = rte_memzone_reserve_aligned(vq_name,
-						     vq_size * sizeof(*txr),
+						     hdr_mz_sz,
 						     socket_id, 0,
 						     RTE_CACHE_LINE_SIZE);
 		if (hdr_mz == NULL) {
@@ -399,21 +418,8 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		vq->virtio_net_hdr_mz = hdr_mz;
 		vq->virtio_net_hdr_mem = hdr_mz->phys_addr;
 
-		txr = hdr_mz->addr;
-		memset(txr, 0, vq_size * sizeof(*txr));
-		for (i = 0; i < vq_size; i++) {
-			struct vring_desc *start_dp = txr[i].tx_indir;
-
-			vring_desc_init(start_dp, RTE_DIM(txr[i].tx_indir));
-
-			/* first indirect descriptor is always the tx header */
-			start_dp->addr = vq->virtio_net_hdr_mem
-				+ i * sizeof(*txr)
-				+ offsetof(struct virtio_tx_region, tx_hdr);
-
-			start_dp->len = vq->hw->vtnet_hdr_size;
-			start_dp->flags = VRING_DESC_F_NEXT;
-		}
+		memset(hdr_mz->addr, 0, hdr_mz_sz);
+		vring_hdr_desc_init(vq);
 
 	} else if (queue_type == VTNET_CQ) {
 		/* Allocate a page for control vq command, data and status */
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v6 3/7] virtio: enable use virtual address to fill desc
  2016-06-02  9:54 ` [PATCH v6 0/7] virtio support for container Jianfeng Tan
  2016-06-02  9:54   ` [PATCH v6 1/7] virtio: hide phys addr check inside pci ops Jianfeng Tan
  2016-06-02  9:54   ` [PATCH v6 2/7] virtio: clean up virtio_dev_queue_setup Jianfeng Tan
@ 2016-06-02  9:54   ` Jianfeng Tan
  2016-06-02  9:54   ` [PATCH v6 4/7] virtio-user: add vhost adapter layer Jianfeng Tan
                     ` (3 subsequent siblings)
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-02  9:54 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is related to how to calculate relative address for vhost
backend.

The principle is that: based on one or multiple shared memory regions,
vhost maintains a reference system with the frontend start address,
backend start address, and length for each segment, so that each
frontend address (GPA, Guest Physical Address) can be translated into
vhost-recognizable backend address. To make the address translation
efficient, we need to maintain as few regions as possible. In the case
of VM, GPA is always locally continuous. But for some other case, like
virtio-user, we use virtual address here.

It basically means:
  a. when set_base_addr, VA address is used;
  b. when preparing RX's descriptors, VA address is used;
  c. when transmitting packets, VA is filled in TX's descriptors;
  d. in TX and CQ's header, VA is used.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/virtio_ethdev.c      | 25 ++++++++++++++++++++-----
 drivers/net/virtio/virtio_rxtx.c        |  5 ++---
 drivers/net/virtio/virtio_rxtx_simple.c | 13 +++++++------
 drivers/net/virtio/virtqueue.h          | 13 ++++++++++++-
 4 files changed, 41 insertions(+), 15 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 781886d..90f1a28 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -167,14 +167,14 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	 * One RX packet for ACK.
 	 */
 	vq->vq_ring.desc[head].flags = VRING_DESC_F_NEXT;
-	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mz->phys_addr;
+	vq->vq_ring.desc[head].addr = vq->virtio_net_hdr_mem;
 	vq->vq_ring.desc[head].len = sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_free_cnt--;
 	i = vq->vq_ring.desc[head].next;
 
 	for (k = 0; k < pkt_num; k++) {
 		vq->vq_ring.desc[i].flags = VRING_DESC_F_NEXT;
-		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+		vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr)
 			+ sizeof(ctrl->status) + sizeof(uint8_t)*sum;
 		vq->vq_ring.desc[i].len = dlen[k];
@@ -184,7 +184,7 @@ virtio_send_command(struct virtqueue *vq, struct virtio_pmd_ctrl *ctrl,
 	}
 
 	vq->vq_ring.desc[i].flags = VRING_DESC_F_WRITE;
-	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mz->phys_addr
+	vq->vq_ring.desc[i].addr = vq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_ring.desc[i].len = sizeof(ctrl->status);
 	vq->vq_free_cnt--;
@@ -419,8 +419,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		vq->virtio_net_hdr_mem = hdr_mz->phys_addr;
 
 		memset(hdr_mz->addr, 0, hdr_mz_sz);
-		vring_hdr_desc_init(vq);
-
 	} else if (queue_type == VTNET_CQ) {
 		/* Allocate a page for control vq command, data and status */
 		snprintf(vq_name, sizeof(vq_name), "port%d_cvq_hdrzone",
@@ -441,6 +439,23 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		memset(vq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
 	}
 
+	/* For virtio-user case (that is when dev->pci_dev is NULL), we use
+	 * virtual address. And we need properly set _offset_, please see
+	 * MBUF_DATA_DMA_ADDR in virtqueue.h for more information.
+	 */
+	if (dev->pci_dev)
+		vq->offset = offsetof(struct rte_mbuf, buf_physaddr);
+	else {
+		vq->vq_ring_mem = (phys_addr_t)vq->mz->addr;
+		vq->offset = offsetof(struct rte_mbuf, buf_addr);
+		if (vq->virtio_net_hdr_mz)
+			vq->virtio_net_hdr_mem =
+				(phys_addr_t)vq->virtio_net_hdr_mz->addr;
+	}
+
+	if (queue_type == VTNET_TQ)
+		vring_hdr_desc_init(vq);
+
 	if (hw->vtpci_ops->setup_queue(hw, vq) < 0) {
 		PMD_INIT_LOG(ERR, "setup_queue failed");
 		virtio_dev_queue_release(vq);
diff --git a/drivers/net/virtio/virtio_rxtx.c b/drivers/net/virtio/virtio_rxtx.c
index f326222..5b0c3df 100644
--- a/drivers/net/virtio/virtio_rxtx.c
+++ b/drivers/net/virtio/virtio_rxtx.c
@@ -193,8 +193,7 @@ virtqueue_enqueue_recv_refill(struct virtqueue *vq, struct rte_mbuf *cookie)
 
 	start_dp = vq->vq_ring.desc;
 	start_dp[idx].addr =
-		(uint64_t)(cookie->buf_physaddr + RTE_PKTMBUF_HEADROOM
-		- hw->vtnet_hdr_size);
+		MBUF_DATA_DMA_ADDR(cookie, vq->offset) - hw->vtnet_hdr_size;
 	start_dp[idx].len =
 		cookie->buf_len - RTE_PKTMBUF_HEADROOM + hw->vtnet_hdr_size;
 	start_dp[idx].flags =  VRING_DESC_F_WRITE;
@@ -265,7 +264,7 @@ virtqueue_enqueue_xmit(struct virtqueue *txvq, struct rte_mbuf *cookie,
 	}
 
 	do {
-		start_dp[idx].addr  = rte_mbuf_data_dma_addr(cookie);
+		start_dp[idx].addr  = MBUF_DATA_DMA_ADDR(cookie, txvq->offset);
 		start_dp[idx].len   = cookie->data_len;
 		start_dp[idx].flags = cookie->next ? VRING_DESC_F_NEXT : 0;
 		idx = start_dp[idx].next;
diff --git a/drivers/net/virtio/virtio_rxtx_simple.c b/drivers/net/virtio/virtio_rxtx_simple.c
index 8f5293d..83a794e 100644
--- a/drivers/net/virtio/virtio_rxtx_simple.c
+++ b/drivers/net/virtio/virtio_rxtx_simple.c
@@ -80,8 +80,8 @@ virtqueue_enqueue_recv_refill_simple(struct virtqueue *vq,
 	vq->sw_ring[desc_idx] = cookie;
 
 	start_dp = vq->vq_ring.desc;
-	start_dp[desc_idx].addr = (uint64_t)((uintptr_t)cookie->buf_physaddr +
-		RTE_PKTMBUF_HEADROOM - vq->hw->vtnet_hdr_size);
+	start_dp[desc_idx].addr = MBUF_DATA_DMA_ADDR(cookie, vq->offset) -
+				  vq->hw->vtnet_hdr_size;
 	start_dp[desc_idx].len = cookie->buf_len -
 		RTE_PKTMBUF_HEADROOM + vq->hw->vtnet_hdr_size;
 
@@ -119,8 +119,8 @@ virtio_rxq_rearm_vec(struct virtqueue *rxvq)
 		*(uint64_t *)p = rxvq->mbuf_initializer;
 
 		start_dp[i].addr =
-			(uint64_t)((uintptr_t)sw_ring[i]->buf_physaddr +
-			RTE_PKTMBUF_HEADROOM - rxvq->hw->vtnet_hdr_size);
+			MBUF_DATA_DMA_ADDR(sw_ring[i], rxvq->offset) -
+			rxvq->hw->vtnet_hdr_size;
 		start_dp[i].len = sw_ring[i]->buf_len -
 			RTE_PKTMBUF_HEADROOM + rxvq->hw->vtnet_hdr_size;
 	}
@@ -366,7 +366,7 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 			txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 		for (i = 0; i < nb_tail; i++) {
 			start_dp[desc_idx].addr =
-				rte_mbuf_data_dma_addr(*tx_pkts);
+				MBUF_DATA_DMA_ADDR(*tx_pkts, txvq->offset);
 			start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 			tx_pkts++;
 			desc_idx++;
@@ -377,7 +377,8 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 	for (i = 0; i < nb_commit; i++)
 		txvq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 	for (i = 0; i < nb_commit; i++) {
-		start_dp[desc_idx].addr = rte_mbuf_data_dma_addr(*tx_pkts);
+		start_dp[desc_idx].addr =
+			MBUF_DATA_DMA_ADDR(*tx_pkts, txvq->offset);
 		start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 		tx_pkts++;
 		desc_idx++;
diff --git a/drivers/net/virtio/virtqueue.h b/drivers/net/virtio/virtqueue.h
index 4e543d2..434aff0 100644
--- a/drivers/net/virtio/virtqueue.h
+++ b/drivers/net/virtio/virtqueue.h
@@ -66,6 +66,14 @@ struct rte_mbuf;
 
 #define VIRTQUEUE_MAX_NAME_SZ 32
 
+#ifdef RTE_VIRTIO_VDEV
+#define MBUF_DATA_DMA_ADDR(mb, offset) \
+	((uint64_t)((uintptr_t)(*(void **)((uintptr_t)mb + offset)) \
+			+ (mb)->data_off))
+#else /* RTE_VIRTIO_VDEV */
+#define MBUF_DATA_DMA_ADDR(mb, offset) rte_mbuf_data_dma_addr(mb)
+#endif /* RTE_VIRTIO_VDEV */
+
 #define VTNET_SQ_RQ_QUEUE_IDX 0
 #define VTNET_SQ_TQ_QUEUE_IDX 1
 #define VTNET_SQ_CQ_QUEUE_IDX 2
@@ -165,6 +173,7 @@ struct virtqueue {
 	void        *vq_ring_virt_mem;    /**< linear address of vring*/
 	unsigned int vq_ring_size;
 	phys_addr_t vq_ring_mem;          /**< physical address of vring */
+				/**< use virtual address for virtio-user. */
 
 	struct vring vq_ring;    /**< vring keeping desc, used and avail */
 	uint16_t    vq_free_cnt; /**< num of desc available */
@@ -183,8 +192,10 @@ struct virtqueue {
 	 */
 	uint16_t vq_used_cons_idx;
 	uint16_t vq_avail_idx;
+	uint16_t offset; /**< relative offset to obtain addr in mbuf */
 	uint64_t mbuf_initializer; /**< value to init mbufs. */
-	phys_addr_t virtio_net_hdr_mem; /**< hdr for each xmit packet */
+	phys_addr_t virtio_net_hdr_mem; /**< phys addr of hdr memzone */
+					/**< use virtual address for vdev */
 
 	struct rte_mbuf **sw_ring; /**< RX software ring. */
 	/* dummy mbuf, for wraparound when processing RX ring. */
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v6 4/7] virtio-user: add vhost adapter layer
  2016-06-02  9:54 ` [PATCH v6 0/7] virtio support for container Jianfeng Tan
                     ` (2 preceding siblings ...)
  2016-06-02  9:54   ` [PATCH v6 3/7] virtio: enable use virtual address to fill desc Jianfeng Tan
@ 2016-06-02  9:54   ` Jianfeng Tan
  2016-06-02  9:54   ` [PATCH v6 5/7] virtio-user: add device emulation layer APIs Jianfeng Tan
                     ` (2 subsequent siblings)
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-02  9:54 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is to provide vhost adapter layer implementations. Instead
of relying on a hypervisor to translate between device emulation and
vhost backend, here we directly talk with vhost backend through the
vhost file. Depending on the type of vhost file,
  - vhost-user is used if the given path points to a unix socket;
  - vhost-kernel is used if the given path points to a char device.

Here three main APIs are provided to upper layer (device emulation):
  - vhost_user_setup(), to set up env to talk to a vhost user backend;
  - vhost_kernel_setup(), to set up env to talk to a vhost kernel backend.
  - vhost_call(), to provide a unified interface to communicate with
    vhost backend.

  ----------------------
  | ------------------ |
  | | virtio driver  | |
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate | |
  | |                | |
  | | vhost adapter  |-|----> (vhost_user.c, vhost_kernel.c, vhost.c)
  | ------------------ |
  ----------------------
            |
            | -------------- --> (vhost-user protocol or vhost-net ioctls)
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 config/common_linuxapp                        |   3 +
 drivers/net/virtio/Makefile                   |   6 +
 drivers/net/virtio/virtio_user/vhost.c        | 105 +++++++
 drivers/net/virtio/virtio_user/vhost.h        | 222 +++++++++++++++
 drivers/net/virtio/virtio_user/vhost_kernel.c | 254 +++++++++++++++++
 drivers/net/virtio/virtio_user/vhost_user.c   | 378 ++++++++++++++++++++++++++
 6 files changed, 968 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/vhost.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost.h
 create mode 100644 drivers/net/virtio/virtio_user/vhost_kernel.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c

diff --git a/config/common_linuxapp b/config/common_linuxapp
index 7e698e2..946a6d4 100644
--- a/config/common_linuxapp
+++ b/config/common_linuxapp
@@ -43,3 +43,6 @@ CONFIG_RTE_LIBRTE_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_AF_PACKET=y
 CONFIG_RTE_LIBRTE_POWER=y
+
+# Enable virtio-user
+CONFIG_RTE_VIRTIO_VDEV=y
diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index ef84f60..c9f2bc0 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -55,6 +55,12 @@ ifeq ($(findstring RTE_MACHINE_CPUFLAG_SSSE3,$(CFLAGS)),RTE_MACHINE_CPUFLAG_SSSE
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx_simple.c
 endif
 
+ifeq ($(CONFIG_RTE_VIRTIO_VDEV),y)
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_kernel.c
+endif
+
 # this lib depends upon:
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_eal lib/librte_ether
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_mempool lib/librte_mbuf
diff --git a/drivers/net/virtio/virtio_user/vhost.c b/drivers/net/virtio/virtio_user/vhost.c
new file mode 100644
index 0000000..1944a97
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost.c
@@ -0,0 +1,105 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include "vhost.h"
+
+static const char * const vhost_msg_strings[] = {
+	[VHOST_MSG_SET_OWNER] = "VHOST_MSG_SET_OWNER",
+	[VHOST_MSG_RESET_OWNER] = "VHOST_MSG_RESET_OWNER",
+	[VHOST_MSG_SET_FEATURES] = "VHOST_MSG_SET_FEATURES",
+	[VHOST_MSG_GET_FEATURES] = "VHOST_MSG_GET_FEATURES",
+	[VHOST_MSG_SET_VRING_CALL] = "VHOST_MSG_SET_VRING_CALL",
+	[VHOST_MSG_SET_VRING_NUM] = "VHOST_MSG_SET_VRING_NUM",
+	[VHOST_MSG_SET_VRING_BASE] = "VHOST_MSG_SET_VRING_BASE",
+	[VHOST_MSG_GET_VRING_BASE] = "VHOST_MSG_GET_VRING_BASE",
+	[VHOST_MSG_SET_VRING_ADDR] = "VHOST_MSG_SET_VRING_ADDR",
+	[VHOST_MSG_SET_VRING_KICK] = "VHOST_MSG_SET_VRING_KICK",
+	[VHOST_MSG_SET_MEM_TABLE] = "VHOST_MSG_SET_MEM_TABLE",
+	NULL,
+};
+
+static uint64_t vhost_req_map[][2] = {
+	[VHOST_MSG_SET_OWNER] = {
+		VHOST_SET_OWNER, VHOST_USER_SET_OWNER
+	},
+	[VHOST_MSG_RESET_OWNER] = {
+		VHOST_RESET_OWNER, VHOST_USER_RESET_OWNER
+	},
+	[VHOST_MSG_SET_FEATURES] = {
+		VHOST_SET_FEATURES, VHOST_USER_SET_FEATURES
+	},
+	[VHOST_MSG_GET_FEATURES] = {
+		VHOST_GET_FEATURES, VHOST_USER_GET_FEATURES
+	},
+	[VHOST_MSG_SET_VRING_CALL] = {
+		VHOST_SET_VRING_CALL, VHOST_USER_SET_VRING_CALL
+	},
+	[VHOST_MSG_SET_VRING_NUM] = {
+		VHOST_SET_VRING_NUM, VHOST_USER_SET_VRING_NUM
+	},
+	[VHOST_MSG_SET_VRING_BASE] = {
+		VHOST_SET_VRING_BASE, VHOST_USER_SET_VRING_BASE
+	},
+	[VHOST_MSG_GET_VRING_BASE] = {
+		VHOST_GET_VRING_BASE, VHOST_USER_GET_VRING_BASE
+	},
+	[VHOST_MSG_SET_VRING_ADDR] = {
+		VHOST_SET_VRING_ADDR, VHOST_USER_SET_VRING_ADDR
+	},
+	[VHOST_MSG_SET_VRING_KICK] = {
+		VHOST_SET_VRING_KICK, VHOST_USER_SET_VRING_KICK
+	},
+	[VHOST_MSG_SET_MEM_TABLE] = {
+		VHOST_SET_MEM_TABLE, VHOST_USER_SET_MEM_TABLE
+	},
+};
+
+int
+vhost_call(int vhostfd, int type, uint64_t req, void *arg)
+{
+	uint64_t req_new;
+	int ret;
+
+	PMD_DRV_LOG(INFO, "%s", vhost_msg_strings[req]);
+
+	req_new = vhost_req_map[req][type];
+	if (type == VHOST_USER)
+		ret = vhost_user_sock(vhostfd, req_new, arg);
+	else
+		ret = vhost_kernel_ioctl(vhostfd, req_new, arg);
+
+	if (ret < 0)
+		PMD_DRV_LOG(ERR, "vhost_call %s failed: %s\n",
+			    vhost_msg_strings[req], strerror(errno));
+	return ret;
+}
diff --git a/drivers/net/virtio/virtio_user/vhost.h b/drivers/net/virtio/virtio_user/vhost.h
new file mode 100644
index 0000000..6bac1e8
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost.h
@@ -0,0 +1,222 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VHOST_NET_USER_H
+#define _VHOST_NET_USER_H
+
+#include <stdint.h>
+#include <linux/types.h>
+#include <linux/ioctl.h>
+
+#include "../virtio_pci.h"
+#include "../virtio_logs.h"
+#include "../virtqueue.h"
+
+#define VHOST_MEMORY_MAX_NREGIONS 8
+
+struct vhost_vring_state {
+	unsigned int index;
+	unsigned int num;
+};
+
+struct vhost_vring_file {
+	unsigned int index;
+	int fd;
+};
+
+struct vhost_vring_addr {
+	unsigned int index;
+	/* Option flags. */
+	unsigned int flags;
+	/* Flag values: */
+	/* Whether log address is valid. If set enables logging. */
+#define VHOST_VRING_F_LOG 0
+
+	/* Start of array of descriptors (virtually contiguous) */
+	uint64_t desc_user_addr;
+	/* Used structure address. Must be 32 bit aligned */
+	uint64_t used_user_addr;
+	/* Available structure address. Must be 16 bit aligned */
+	uint64_t avail_user_addr;
+	/* Logging support. */
+	/* Log writes to used structure, at offset calculated from specified
+	 * address. Address must be 32 bit aligned.
+	 */
+	uint64_t log_guest_addr;
+};
+
+enum vhost_user_request {
+	VHOST_USER_NONE = 0,
+	VHOST_USER_GET_FEATURES = 1,
+	VHOST_USER_SET_FEATURES = 2,
+	VHOST_USER_SET_OWNER = 3,
+	VHOST_USER_RESET_OWNER = 4,
+	VHOST_USER_SET_MEM_TABLE = 5,
+	VHOST_USER_SET_LOG_BASE = 6,
+	VHOST_USER_SET_LOG_FD = 7,
+	VHOST_USER_SET_VRING_NUM = 8,
+	VHOST_USER_SET_VRING_ADDR = 9,
+	VHOST_USER_SET_VRING_BASE = 10,
+	VHOST_USER_GET_VRING_BASE = 11,
+	VHOST_USER_SET_VRING_KICK = 12,
+	VHOST_USER_SET_VRING_CALL = 13,
+	VHOST_USER_SET_VRING_ERR = 14,
+	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
+	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
+	VHOST_USER_GET_QUEUE_NUM = 17,
+	VHOST_USER_SET_VRING_ENABLE = 18,
+	VHOST_USER_MAX
+};
+
+struct vhost_memory_region {
+	uint64_t guest_phys_addr;
+	uint64_t memory_size; /* bytes */
+	uint64_t userspace_addr;
+	uint64_t mmap_offset;
+};
+
+struct vhost_memory_kernel {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[0];
+};
+
+struct vhost_memory {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[VHOST_MEMORY_MAX_NREGIONS];
+};
+
+struct vhost_user_msg {
+	enum vhost_user_request request;
+
+#define VHOST_USER_VERSION_MASK     0x3
+#define VHOST_USER_REPLY_MASK       (0x1 << 2)
+	uint32_t flags;
+	uint32_t size; /* the following payload size */
+	union {
+#define VHOST_USER_VRING_IDX_MASK   0xff
+#define VHOST_USER_VRING_NOFD_MASK  (0x1 << 8)
+		uint64_t u64;
+		struct vhost_vring_state state;
+		struct vhost_vring_addr addr;
+		struct vhost_memory memory;
+	} payload;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+} __attribute((packed));
+
+#define VHOST_USER_HDR_SIZE offsetof(struct vhost_user_msg, payload.u64)
+#define VHOST_USER_PAYLOAD_SIZE \
+	(sizeof(struct vhost_user_msg) - VHOST_USER_HDR_SIZE)
+
+/* The version of the protocol we support */
+#define VHOST_USER_VERSION    0x1
+
+/* ioctls */
+
+#define VHOST_VIRTIO 0xAF
+
+#define VHOST_GET_FEATURES _IOR(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_FEATURES _IOW(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_OWNER _IO(VHOST_VIRTIO, 0x01)
+#define VHOST_RESET_OWNER _IO(VHOST_VIRTIO, 0x02)
+#define VHOST_SET_MEM_TABLE _IOW(VHOST_VIRTIO, 0x03, struct vhost_memory_kernel)
+#define VHOST_SET_LOG_BASE _IOW(VHOST_VIRTIO, 0x04, __u64)
+#define VHOST_SET_LOG_FD _IOW(VHOST_VIRTIO, 0x07, int)
+#define VHOST_SET_VRING_NUM _IOW(VHOST_VIRTIO, 0x10, struct vhost_vring_state)
+#define VHOST_SET_VRING_ADDR _IOW(VHOST_VIRTIO, 0x11, struct vhost_vring_addr)
+#define VHOST_SET_VRING_BASE _IOW(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_GET_VRING_BASE _IOWR(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_SET_VRING_KICK _IOW(VHOST_VIRTIO, 0x20, struct vhost_vring_file)
+#define VHOST_SET_VRING_CALL _IOW(VHOST_VIRTIO, 0x21, struct vhost_vring_file)
+#define VHOST_SET_VRING_ERR _IOW(VHOST_VIRTIO, 0x22, struct vhost_vring_file)
+#define VHOST_NET_SET_BACKEND _IOW(VHOST_VIRTIO, 0x30, struct vhost_vring_file)
+
+/*****************************************************************************/
+
+/* Ioctl defines */
+#define TUNSETIFF     _IOW('T', 202, int)
+#define TUNGETFEATURES _IOR('T', 207, unsigned int)
+#define TUNSETOFFLOAD  _IOW('T', 208, unsigned int)
+#define TUNGETIFF      _IOR('T', 210, unsigned int)
+#define TUNSETSNDBUF   _IOW('T', 212, int)
+#define TUNGETVNETHDRSZ _IOR('T', 215, int)
+#define TUNSETVNETHDRSZ _IOW('T', 216, int)
+#define TUNSETQUEUE  _IOW('T', 217, int)
+#define TUNSETVNETLE _IOW('T', 220, int)
+#define TUNSETVNETBE _IOW('T', 222, int)
+
+/* TUNSETIFF ifr flags */
+#define IFF_TAP          0x0002
+#define IFF_NO_PI        0x1000
+#define IFF_ONE_QUEUE    0x2000
+#define IFF_VNET_HDR     0x4000
+#define IFF_MULTI_QUEUE  0x0100
+#define IFF_ATTACH_QUEUE 0x0200
+#define IFF_DETACH_QUEUE 0x0400
+
+/* Features for GSO (TUNSETOFFLOAD). */
+#define TUN_F_CSUM	0x01	/* You can hand me unchecksummed packets. */
+#define TUN_F_TSO4	0x02	/* I can handle TSO for IPv4 packets */
+#define TUN_F_TSO6	0x04	/* I can handle TSO for IPv6 packets */
+#define TUN_F_TSO_ECN	0x08	/* I can handle TSO with ECN bits. */
+#define TUN_F_UFO	0x10	/* I can handle UFO packets */
+
+enum {
+	VHOST_MSG_SET_OWNER,
+	VHOST_MSG_RESET_OWNER,
+	VHOST_MSG_SET_FEATURES,
+	VHOST_MSG_GET_FEATURES,
+	VHOST_MSG_SET_VRING_CALL,
+	VHOST_MSG_SET_VRING_NUM,
+	VHOST_MSG_SET_VRING_BASE,
+	VHOST_MSG_GET_VRING_BASE,
+	VHOST_MSG_SET_VRING_ADDR,
+	VHOST_MSG_SET_VRING_KICK,
+	VHOST_MSG_SET_MEM_TABLE,
+	VHOST_MSG_MAX,
+};
+
+#define VHOST_KERNEL	0
+#define VHOST_USER	1
+
+int vhost_user_sock(int vhostfd, uint64_t req, void *arg);
+int vhost_user_setup(const char *path);
+
+int vhost_kernel_ioctl(int vhostfd, uint64_t req, void *arg);
+int vhost_kernel_setup(const char *path, const char *ifname, int *p_tapfd);
+int vhost_kernel_post(int vhostfd, int tapfd, uint64_t features, uint32_t nvqs);
+
+int vhost_call(int vhostfd, int type, uint64_t req, void *arg);
+
+#endif
diff --git a/drivers/net/virtio/virtio_user/vhost_kernel.c b/drivers/net/virtio/virtio_user/vhost_kernel.c
new file mode 100644
index 0000000..41196cf
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost_kernel.c
@@ -0,0 +1,254 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <sys/ioctl.h>
+#include <net/if.h>
+#include <string.h>
+#include <errno.h>
+
+#include <rte_memory.h>
+#include <rte_eal_memconfig.h>
+
+#include "vhost.h"
+
+#define TUN_DEF_SNDBUF	(1ull << 20)
+#define PATH_NET_TUN	"/dev/net/tun"
+
+/** Merge those virtually adjacent memsegs into one region.
+ */
+static void
+prepare_vhost_memory_kernel(struct vhost_memory_kernel **p_vm)
+{
+	uint32_t i, j, k = 0;
+	struct rte_memseg *seg;
+	struct vhost_memory_region *mr;
+	struct vhost_memory_kernel *vm;
+
+	vm = malloc(sizeof(struct vhost_memory_kernel) +
+		    RTE_MAX_MEMSEG * sizeof(struct vhost_memory_region));
+
+	for (i = 0; i < RTE_MAX_MEMSEG; ++i) {
+		seg = &rte_eal_get_configuration()->mem_config->memseg[i];
+		if (!seg->addr)
+			break;
+
+		int new_region = 1;
+
+		for (j = 0; j < k; ++j) {
+			mr = &vm->regions[j];
+
+			if (mr->userspace_addr + mr->memory_size ==
+			    (uint64_t)seg->addr) {
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+
+			if ((uint64_t)seg->addr + seg->len ==
+			    mr->userspace_addr) {
+				mr->guest_phys_addr = (uint64_t)seg->addr;
+				mr->userspace_addr = (uint64_t)seg->addr;
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+		}
+
+		if (new_region == 0)
+			continue;
+
+		mr = &vm->regions[k++];
+		mr->guest_phys_addr = (uint64_t)seg->addr; /* use vaddr here! */
+		mr->userspace_addr = (uint64_t)seg->addr;
+		mr->memory_size = seg->len;
+		mr->mmap_offset = 0;
+	}
+
+	vm->nregions = k;
+	vm->padding = 0;
+	*p_vm = vm;
+}
+
+int
+vhost_kernel_ioctl(int vhostfd, uint64_t req, void *arg)
+{
+	int ret;
+	struct vhost_memory_kernel *vm = NULL;
+
+	if (req == VHOST_SET_MEM_TABLE) {
+		prepare_vhost_memory_kernel(&vm);
+		arg = (void *)vm;
+	}
+
+	ret = ioctl(vhostfd, req, arg);
+
+	if (req == VHOST_SET_MEM_TABLE)
+		free(vm);
+
+	return ret;
+}
+
+/**
+ * Set up environment to talk with a vhost kernel backend.
+ * @param path
+ *   - The path to vhost net (kernel) character file.
+ *
+ * @param ifname
+ *   - Specify the tap device name if any, or NULL.
+ *
+ * @param p_tapfd
+ *   - Pointer to store the fd of tap device.
+ *
+ * @return
+ *   - (-1) if fail to set up;
+ *   - (>=0) if successful, and it is the fd to vhostfd.
+ */
+int
+vhost_kernel_setup(const char *path, const char *ifname, int *p_tapfd)
+{
+	int vhostfd, tapfd;
+	int len = sizeof(struct virtio_net_hdr_mrg_rxbuf);
+	int req_mq = 0;
+	int sndbuf = TUN_DEF_SNDBUF;
+	unsigned int features;
+	struct ifreq ifr;
+
+	/* TODO:
+	 * 1. get/set offload capability, tap_probe_has_ufo, tap_fd_set_offload
+	 * 2. verify we can get/set vnet_hdr_len, tap_probe_vnet_hdr_len
+	 * 3. get number of memory regions from vhost module parameter
+	 * max_mem_regions, supported in newer version linux kernel
+	 */
+
+	tapfd = open(PATH_NET_TUN, O_RDWR);
+	if (tapfd < 0) {
+		PMD_DRV_LOG(ERR, "fail to open %s: %s",
+			    PATH_NET_TUN, strerror(errno));
+		return -1;
+	}
+
+	memset(&ifr, 0, sizeof(ifr));
+	ifr.ifr_flags = IFF_TAP | IFF_NO_PI;
+
+	if (ioctl(tapfd, TUNGETFEATURES, &features) == -1) {
+		PMD_DRV_LOG(ERR, "TUNGETFEATURES failed: %s", strerror(errno));
+		goto error;
+	}
+
+	if (features & IFF_ONE_QUEUE)
+		ifr.ifr_flags |= IFF_ONE_QUEUE;
+
+	if (features & IFF_VNET_HDR)
+		ifr.ifr_flags |= IFF_VNET_HDR;
+	else {
+		PMD_DRV_LOG(ERR, "vnet_hdr not supported by kernel");
+		goto error;
+	}
+
+	if (req_mq) {
+		if (features & IFF_MULTI_QUEUE)
+			ifr.ifr_flags |= IFF_MULTI_QUEUE;
+		else {
+			PMD_DRV_LOG(ERR, "multiqueue not supported by kernel");
+			goto error;
+		}
+	}
+
+	if (ifname)
+		strncpy(ifr.ifr_name, ifname, IFNAMSIZ);
+	else
+		strncpy(ifr.ifr_name, "tap%d", IFNAMSIZ);
+	if (ioctl(tapfd, TUNSETIFF, (void *)&ifr) == -1) {
+		PMD_DRV_LOG(ERR, "TUNSETIFF failed: %s", strerror(errno));
+		goto error;
+	}
+	fcntl(tapfd, F_SETFL, O_NONBLOCK);
+
+	if (ioctl(tapfd, TUNSETVNETHDRSZ, &len) < 0) {
+		PMD_DRV_LOG(ERR, "TUNSETVNETHDRSZ failed: %s", strerror(errno));
+		goto error;
+	}
+
+	if (ioctl(tapfd, TUNSETSNDBUF, &sndbuf) < 0) {
+		PMD_DRV_LOG(ERR, "TUNSETSNDBUF failed: %s", strerror(errno));
+		goto error;
+	}
+
+	vhostfd = open(path, O_RDWR);
+	if (vhostfd < 0) {
+		PMD_DRV_LOG(ERR, "fail to open %s, %s", path, strerror(errno));
+		goto error;
+	}
+
+	*p_tapfd = tapfd;
+	return vhostfd;
+
+error:
+	close(tapfd);
+	return -1;
+}
+
+int
+vhost_kernel_post(int vhostfd, int tapfd, uint64_t features, uint32_t nvqs)
+{
+	struct vhost_vring_file f;
+	uint16_t hdr_size;
+	int ret;
+
+	if ((features & (1ULL << VIRTIO_NET_F_MRG_RXBUF)) ||
+	    (features & (1ULL << VIRTIO_F_VERSION_1)))
+		hdr_size = sizeof(struct virtio_net_hdr_mrg_rxbuf);
+	else
+		hdr_size = sizeof(struct virtio_net_hdr);
+
+	if (ioctl(tapfd, TUNSETVNETHDRSZ, &hdr_size) == -1) {
+		PMD_DRV_LOG(ERR, "TUNSETVNETHDRSZ fails, %s", strerror(errno));
+		return -1;
+	}
+
+	f.fd = tapfd;
+	for (f.index = 0; f.index < nvqs; ++f.index) {
+		ret = vhost_kernel_ioctl(vhostfd, VHOST_NET_SET_BACKEND, &f);
+		if (ret < 0) {
+			PMD_DRV_LOG(ERR, "VHOST_NET_SET_BACKEND fails, %s",
+				    strerror(errno));
+			return -1;
+		}
+	}
+
+	return 0;
+}
diff --git a/drivers/net/virtio/virtio_user/vhost_user.c b/drivers/net/virtio/virtio_user/vhost_user.c
new file mode 100644
index 0000000..b0a84cf
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost_user.c
@@ -0,0 +1,378 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <unistd.h>
+#include <fcntl.h>
+#include <sys/un.h>
+#include <string.h>
+#include <errno.h>
+
+#include "vhost.h"
+
+static int
+vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
+{
+	int r;
+	struct msghdr msgh;
+	struct iovec iov;
+	size_t fd_size = fd_num * sizeof(int);
+	char control[CMSG_SPACE(fd_size)];
+	struct cmsghdr *cmsg;
+
+	memset(&msgh, 0, sizeof(msgh));
+	memset(control, 0, sizeof(control));
+
+	iov.iov_base = (uint8_t *)buf;
+	iov.iov_len = len;
+
+	msgh.msg_iov = &iov;
+	msgh.msg_iovlen = 1;
+	msgh.msg_control = control;
+	msgh.msg_controllen = sizeof(control);
+
+	cmsg = CMSG_FIRSTHDR(&msgh);
+	cmsg->cmsg_len = CMSG_LEN(fd_size);
+	cmsg->cmsg_level = SOL_SOCKET;
+	cmsg->cmsg_type = SCM_RIGHTS;
+	memcpy(CMSG_DATA(cmsg), fds, fd_size);
+
+	do {
+		r = sendmsg(fd, &msgh, 0);
+	} while (r < 0 && errno == EINTR);
+
+	return r;
+}
+
+static int
+vhost_user_read(int fd, struct vhost_user_msg *msg)
+{
+	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
+	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
+
+	ret = recv(fd, (void *)msg, sz_hdr, 0);
+	if (ret < sz_hdr) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
+			    ret, sz_hdr);
+		goto fail;
+	}
+
+	/* validate msg flags */
+	if (msg->flags != (valid_flags)) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg: flags %x instead of %x.",
+			    msg->flags, valid_flags);
+		goto fail;
+	}
+
+	sz_payload = msg->size;
+	if (sz_payload) {
+		ret = recv(fd, (void *)((char *)msg + sz_hdr), sz_payload, 0);
+		if (ret < sz_payload) {
+			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
+				    ret, msg->size);
+			goto fail;
+		}
+	}
+
+	return 0;
+
+fail:
+	return -1;
+}
+
+struct hugepage_file_info {
+	uint64_t addr;            /**< virtual addr */
+	size_t   size;            /**< the file size */
+	char     path[PATH_MAX];  /**< path to backing file */
+};
+
+/* Two possible options:
+ * 1. Match HUGEPAGE_INFO_FMT to find the file storing struct hugepage_file
+ * array. This is simple but cannot be used in secondary process because
+ * secondary process will close and munmap that file.
+ * 2. Match HUGEFILE_FMT to find hugepage files directly.
+ *
+ * We choose option 2.
+ */
+static int
+get_hugepage_file_info(struct hugepage_file_info huges[], int max)
+{
+	int idx;
+	FILE *f;
+	char buf[BUFSIZ], *tmp, *tail;
+	char *str_underline, *str_start;
+	int huge_index;
+	uint64_t v_start, v_end;
+
+	f = fopen("/proc/self/maps", "r");
+	if (!f) {
+		PMD_DRV_LOG(ERR, "cannot open /proc/self/maps");
+		return -1;
+	}
+
+	idx = 0;
+	while (fgets(buf, sizeof(buf), f) != NULL) {
+		if (sscanf(buf, "%" PRIx64 "-%" PRIx64, &v_start, &v_end) < 2) {
+			PMD_DRV_LOG(ERR, "Failed to parse address");
+			goto error;
+		}
+
+		tmp = strchr(buf, ' ') + 1; /** skip address */
+		tmp = strchr(tmp, ' ') + 1; /** skip perm */
+		tmp = strchr(tmp, ' ') + 1; /** skip offset */
+		tmp = strchr(tmp, ' ') + 1; /** skip dev */
+		tmp = strchr(tmp, ' ') + 1; /** skip inode */
+		while (*tmp == ' ')         /** skip spaces */
+			tmp++;
+		tail = strrchr(tmp, '\n');  /** remove newline if exists */
+		if (tail)
+			*tail = '\0';
+
+		/* Match HUGEFILE_FMT, aka "%s/%smap_%d",
+		 * which is defined in eal_filesystem.h
+		 */
+		str_underline = strrchr(tmp, '_');
+		if (!str_underline)
+			continue;
+
+		str_start = str_underline - strlen("map");
+		if (str_start < tmp)
+			continue;
+
+		if (sscanf(str_start, "map_%d", &huge_index) != 1)
+			continue;
+
+		if (idx >= max) {
+			PMD_DRV_LOG(ERR, "Exceed maximum of %d", max);
+			goto error;
+		}
+		huges[idx].addr = v_start;
+		huges[idx].size = v_end - v_start;
+		strcpy(huges[idx].path, tmp);
+		idx++;
+	}
+
+	fclose(f);
+	return idx;
+
+error:
+	fclose(f);
+	return -1;
+}
+
+static int
+prepare_vhost_memory_user(struct vhost_user_msg *msg, int fds[])
+{
+	int i, num;
+	struct hugepage_file_info huges[VHOST_MEMORY_MAX_NREGIONS];
+	struct vhost_memory_region *mr;
+
+	num = get_hugepage_file_info(huges, VHOST_MEMORY_MAX_NREGIONS);
+	if (num < 0) {
+		PMD_INIT_LOG(ERR, "Failed to prepare memory for vhost-user");
+		return -1;
+	}
+
+	for (i = 0; i < num; ++i) {
+		mr = &msg->payload.memory.regions[i];
+		mr->guest_phys_addr = huges[i].addr; /* use vaddr! */
+		mr->userspace_addr = huges[i].addr;
+		mr->memory_size = huges[i].size;
+		mr->mmap_offset = 0;
+		fds[i] = open(huges[i].path, O_RDWR);
+	}
+
+	msg->payload.memory.nregions = num;
+	msg->payload.memory.padding = 0;
+
+	return 0;
+}
+
+static struct vhost_user_msg m;
+
+int
+vhost_user_sock(int vhostfd, uint64_t req, void *arg)
+{
+	struct vhost_user_msg msg;
+	struct vhost_vring_file *file = 0;
+	int need_reply = 0;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+	int fd_num = 0;
+	int i, len;
+
+	msg.request = req;
+	msg.flags = VHOST_USER_VERSION;
+	msg.size = 0;
+
+	switch (req) {
+	case VHOST_USER_GET_FEATURES:
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_FEATURES:
+	case VHOST_USER_SET_LOG_BASE:
+		msg.payload.u64 = *((__u64 *)arg);
+		msg.size = sizeof(m.payload.u64);
+		break;
+
+	case VHOST_USER_SET_OWNER:
+	case VHOST_USER_RESET_OWNER:
+		break;
+
+	case VHOST_USER_SET_MEM_TABLE:
+		if (prepare_vhost_memory_user(&msg, fds) < 0)
+			return -1;
+		fd_num = msg.payload.memory.nregions;
+		msg.size = sizeof(m.payload.memory.nregions);
+		msg.size += sizeof(m.payload.memory.padding);
+		msg.size += fd_num * sizeof(struct vhost_memory_region);
+		break;
+
+	case VHOST_USER_SET_LOG_FD:
+		fds[fd_num++] = *((int *)arg);
+		break;
+
+	case VHOST_USER_SET_VRING_NUM:
+	case VHOST_USER_SET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		break;
+
+	case VHOST_USER_GET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_VRING_ADDR:
+		memcpy(&msg.payload.addr, arg, sizeof(msg.payload.addr));
+		msg.size = sizeof(m.payload.addr);
+		break;
+
+	case VHOST_USER_SET_VRING_KICK:
+	case VHOST_USER_SET_VRING_CALL:
+	case VHOST_USER_SET_VRING_ERR:
+		file = arg;
+		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
+		msg.size = sizeof(m.payload.u64);
+		if (file->fd > 0)
+			fds[fd_num++] = file->fd;
+		else
+			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
+		break;
+
+	default:
+		PMD_DRV_LOG(ERR, "trying to send unhandled msg type");
+		return -1;
+	}
+
+	len = VHOST_USER_HDR_SIZE + msg.size;
+	if (vhost_user_write(vhostfd, &msg, len, fds, fd_num) < 0)
+		return 0;
+
+	if (req == VHOST_USER_SET_MEM_TABLE)
+		for (i = 0; i < fd_num; ++i)
+			close(fds[i]);
+
+	if (need_reply) {
+		if (vhost_user_read(vhostfd, &msg) < 0)
+			return -1;
+
+		if (req != msg.request) {
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+
+		switch (req) {
+		case VHOST_USER_GET_FEATURES:
+			if (msg.size != sizeof(m.payload.u64)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			*((__u64 *)arg) = msg.payload.u64;
+			break;
+		case VHOST_USER_GET_VRING_BASE:
+			if (msg.size != sizeof(m.payload.state)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			memcpy(arg, &msg.payload.state,
+			       sizeof(struct vhost_vring_state));
+			break;
+		default:
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+	}
+
+	return 0;
+}
+
+/**
+ * Set up environment to talk with a vhost user backend.
+ * @param path
+ *   - The path to vhost user unix socket file.
+ *
+ * @return
+ *   - (-1) if fail to set up;
+ *   - (>=0) if successful, and it is the fd to vhostfd.
+ */
+int
+vhost_user_setup(const char *path)
+{
+	int fd;
+	int flag;
+	struct sockaddr_un un;
+
+	fd = socket(AF_UNIX, SOCK_STREAM, 0);
+	if (fd < 0) {
+		PMD_DRV_LOG(ERR, "socket() error, %s", strerror(errno));
+		return -1;
+	}
+
+	flag = fcntl(fd, F_GETFD);
+	fcntl(fd, F_SETFD, flag | FD_CLOEXEC);
+
+	memset(&un, 0, sizeof(un));
+	un.sun_family = AF_UNIX;
+	snprintf(un.sun_path, sizeof(un.sun_path), "%s", path);
+	if (connect(fd, (struct sockaddr *)&un, sizeof(un)) < 0) {
+		PMD_DRV_LOG(ERR, "connect error, %s", strerror(errno));
+		close(fd);
+		return -1;
+	}
+
+	return fd;
+}
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v6 5/7] virtio-user: add device emulation layer APIs
  2016-06-02  9:54 ` [PATCH v6 0/7] virtio support for container Jianfeng Tan
                     ` (3 preceding siblings ...)
  2016-06-02  9:54   ` [PATCH v6 4/7] virtio-user: add vhost adapter layer Jianfeng Tan
@ 2016-06-02  9:54   ` Jianfeng Tan
  2016-06-02  9:54   ` [PATCH v6 6/7] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
  2016-06-02  9:54   ` [PATCH v6 7/7] virtio-user: add a new vdev named virtio-user Jianfeng Tan
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-02  9:54 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Two device emulation layer APIs are added for virtio driver to call:
  - virtio_user_start_device()
  - virtio_user_stop_device()
  - virtio_user_dev_init()
  - virtio_user_dev_uninit()

These APIs will get called by virtio driver, and they call vhost adapter
layer APIs to implement the functionality. Besides, this patch defines
a struct named virtio_user_dev to help manage the data stands for this
kind of virtual device.

  ----------------------
  | ------------------ |
  | | virtio driver  | |
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate |-|----> (virtio_user_dev.c, virtio_user_dev.h)
  | |                | |
  | | vhost adapter  | |
  | ------------------ |
  ----------------------
            |
            |
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/Makefile                      |   1 +
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 263 +++++++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.h |  64 ++++++
 3 files changed, 328 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h

diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index c9f2bc0..68068bd 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -59,6 +59,7 @@ ifeq ($(CONFIG_RTE_VIRTIO_VDEV),y)
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_kernel.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_dev.c
 endif
 
 # this lib depends upon:
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.c b/drivers/net/virtio/virtio_user/virtio_user_dev.c
new file mode 100644
index 0000000..b4e53a8
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.c
@@ -0,0 +1,263 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <stdio.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <string.h>
+#include <errno.h>
+#include <sys/mman.h>
+#include <unistd.h>
+#include <sys/eventfd.h>
+
+#include "vhost.h"
+#include "virtio_user_dev.h"
+#include "../virtio_ethdev.h"
+
+static int
+virtio_user_kick_queue(struct virtio_user_dev *dev, uint32_t queue_sel)
+{
+	int callfd, kickfd;
+	struct vhost_vring_file file;
+	struct vhost_vring_state state;
+	struct vring *vring = &dev->vrings[queue_sel];
+	struct vhost_vring_addr addr = {
+		.index = queue_sel,
+		.desc_user_addr = (uint64_t)(uintptr_t)vring->desc,
+		.avail_user_addr = (uint64_t)(uintptr_t)vring->avail,
+		.used_user_addr = (uint64_t)(uintptr_t)vring->used,
+		.log_guest_addr = 0,
+		.flags = 0, /* disable log */
+	};
+
+	/* May use invalid flag, but some backend leverages kickfd and callfd as
+	 * criteria to judge if dev is alive. so finally we use real event_fd.
+	 */
+	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (callfd < 0) {
+		PMD_DRV_LOG(ERR, "callfd error, %s\n", strerror(errno));
+		return -1;
+	}
+	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (kickfd < 0) {
+		close(callfd);
+		PMD_DRV_LOG(ERR, "kickfd error, %s\n", strerror(errno));
+		return -1;
+	}
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL come
+	 * firstly because vhost depends on this msg to allocate virtqueue
+	 * pair.
+	 */
+	file.index = queue_sel;
+	file.fd = callfd;
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_CALL, &file);
+	dev->callfds[queue_sel] = callfd;
+
+	state.index = queue_sel;
+	state.num = vring->num;
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_NUM, &state);
+
+	state.num = 0; /* no reservation */
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_BASE, &state);
+
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_ADDR, &addr);
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_KICK comes
+	 * lastly because vhost depends on this msg to judge if
+	 * virtio is ready.
+	 */
+	file.fd = kickfd;
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_KICK, &file);
+	dev->kickfds[queue_sel] = kickfd;
+
+	return 0;
+}
+
+int
+virtio_user_start_device(struct virtio_user_dev *dev)
+{
+	uint64_t features;
+	uint32_t i, queue_sel;
+	int ret;
+
+	/* construct memory region inside each implementation */
+	ret = vhost_call(dev->vhostfd, dev->type,
+			 VHOST_MSG_SET_MEM_TABLE, NULL);
+	if (ret < 0)
+		goto error;
+
+	for (i = 0; i < dev->queue_pairs; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
+		if (virtio_user_kick_queue(dev, queue_sel) < 0) {
+			PMD_DRV_LOG(INFO, "kick rx vq fails: %u", i);
+			goto error;
+		}
+	}
+	for (i = 0; i < dev->queue_pairs; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
+		if (virtio_user_kick_queue(dev, queue_sel) < 0) {
+			PMD_DRV_LOG(INFO, "kick tx vq fails: %u", i);
+			goto error;
+		}
+	}
+
+	/* After setup all virtqueues, we need to set_features so that
+	 * these features can be set into each virtqueue in vhost side.
+	 * And before that, make sure VIRTIO_NET_F_MAC is stripped.
+	 */
+	features = dev->features;
+	features &= ~(1ull << VIRTIO_NET_F_MAC);
+	ret = vhost_call(dev->vhostfd, dev->type,
+			 VHOST_MSG_SET_FEATURES, &features);
+	if (ret < 0)
+		goto error;
+	PMD_DRV_LOG(INFO, "set features: %" PRIx64, features);
+
+	if (dev->type == VHOST_USER)
+		return 0;
+
+	/* It is tricky here, but so far we have to do that */
+	ret = vhost_kernel_post(dev->vhostfd, dev->tapfd, features,
+				dev->queue_pairs * 2);
+	if (ret == 0)
+		return 0;
+error:
+	/* TODO: free resource here or caller to check */
+	return -1;
+}
+
+int virtio_user_stop_device(struct virtio_user_dev *dev)
+{
+	return vhost_call(dev->vhostfd, dev->type, VHOST_MSG_RESET_OWNER, NULL);
+}
+
+static inline void
+parse_mac(struct virtio_user_dev *dev, const char *mac)
+{
+	int i, r;
+	uint32_t tmp[ETHER_ADDR_LEN];
+
+	if (!mac)
+		return;
+
+	r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
+			&tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
+	if (r == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			dev->mac_addr[i] = (uint8_t)tmp[i];
+		dev->mac_specified = 1;
+	} else {
+		/* ignore the wrong mac, use random mac */
+		PMD_DRV_LOG(ERR, "wrong format of mac: %s", mac);
+	}
+}
+
+int
+virtio_user_dev_init(struct virtio_user_dev *dev, char *path, int queues,
+		     int cq, int queue_size, const char *mac, char *ifname)
+{
+	struct stat s;
+
+	strncpy(dev->path, path, PATH_MAX);
+	dev->max_queue_pairs = queues;
+	dev->queue_pairs = 1; /* mq disabled by default */
+	dev->queue_size = queue_size;
+	dev->mac_specified = 0;
+	parse_mac(dev, mac);
+	dev->vhostfd = -1;
+	dev->tapfd = -1;
+	/* TODO: cq */
+	RTE_SET_USED(cq);
+
+	if (stat(dev->path, &s) < 0) {
+		PMD_INIT_LOG(ERR, "stat: %s failed, %s", dev->path,
+			     strerror(errno));
+		return -1;
+	}
+
+	switch (s.st_mode & S_IFMT) {
+	case S_IFCHR:
+		dev->type = VHOST_KERNEL;
+		dev->vhostfd = vhost_kernel_setup(dev->path, ifname,
+						  &dev->tapfd);
+		break;
+	case S_IFSOCK:
+		dev->type = VHOST_USER;
+		dev->vhostfd = vhost_user_setup(dev->path);
+		break;
+	default:
+		PMD_INIT_LOG(ERR, "unknown file type of %s", dev->path);
+		return -1;
+	}
+	if (dev->vhostfd < 0) {
+		PMD_INIT_LOG(ERR, "backend set up fails");
+		return -1;
+	}
+	if (vhost_call(dev->vhostfd, dev->type,
+			VHOST_MSG_SET_OWNER, NULL) < 0) {
+		PMD_INIT_LOG(ERR, "set_owner fails: %s", strerror(errno));
+		return -1;
+	}
+
+	if (vhost_call(dev->vhostfd, dev->type,
+			VHOST_MSG_GET_FEATURES, &dev->features) < 0) {
+		PMD_INIT_LOG(ERR, "get_features failed: %s", strerror(errno));
+		return -1;
+	}
+	if (dev->mac_specified)
+		dev->features |= (1ull << VIRTIO_NET_F_MAC);
+	/* disable it until we support CQ */
+	dev->features &= ~(1ull << VIRTIO_NET_F_CTRL_VQ);
+	dev->features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
+
+	return 0;
+
+}
+
+void
+virtio_user_dev_uninit(struct virtio_user_dev *dev)
+{
+	uint32_t i;
+
+	if (dev->type == VHOST_KERNEL)
+		close(dev->tapfd);
+
+	for (i = 0; i < dev->queue_pairs * 2; ++i) {
+		close(dev->callfds[i]);
+		close(dev->kickfds[i]);
+	}
+
+	close(dev->vhostfd);
+}
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.h b/drivers/net/virtio/virtio_user/virtio_user_dev.h
new file mode 100644
index 0000000..9987b70
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.h
@@ -0,0 +1,64 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VIRTIO_USER_DEV_H
+#define _VIRTIO_USER_DEV_H
+
+#include <limits.h>
+#include "../virtio_pci.h"
+#include "../virtio_ring.h"
+
+struct virtio_user_dev {
+	int		type; /* VHOST_KERNEL or VHOST_USER */
+	int		vhostfd;
+	int		tapfd; /* only used in vhost kernel */
+	int		callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		mac_specified;
+	uint32_t	max_queue_pairs;
+	uint32_t	queue_pairs;
+	uint32_t	queue_size;
+	uint64_t	features;
+	uint8_t		status;
+	uint8_t		mac_addr[ETHER_ADDR_LEN];
+	char		path[PATH_MAX];
+	struct vring	vrings[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+};
+
+int virtio_user_start_device(struct virtio_user_dev *dev);
+int virtio_user_stop_device(struct virtio_user_dev *dev);
+int virtio_user_dev_init(struct virtio_user_dev *dev, char *path, int queues,
+			 int cq, int queue_size, const char *mac, char *ifname);
+void virtio_user_dev_uninit(struct virtio_user_dev *dev);
+
+#endif
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v6 6/7] virtio-user: add new virtual pci driver for virtio
  2016-06-02  9:54 ` [PATCH v6 0/7] virtio support for container Jianfeng Tan
                     ` (4 preceding siblings ...)
  2016-06-02  9:54   ` [PATCH v6 5/7] virtio-user: add device emulation layer APIs Jianfeng Tan
@ 2016-06-02  9:54   ` Jianfeng Tan
  2016-06-06  8:01     ` Yuanhan Liu
  2016-06-02  9:54   ` [PATCH v6 7/7] virtio-user: add a new vdev named virtio-user Jianfeng Tan
  6 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-02  9:54 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch implements another new instance of struct virtio_pci_ops to
drive the virtio-user virtual device. Instead of rd/wr ioport or PCI
configuration space, this virtual pci driver will rd/wr the virtual
device struct virtio_user_hw, and when necessary, invokes APIs provided
by device emulation later to start/stop the device.

  ----------------------
  | ------------------ |
  | | virtio driver  | |----> (virtio_user_ethdev.c)
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate | |
  | |                | |
  | | vhost adapter  | |
  | ------------------ |
  ----------------------
            |
            |
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/Makefile             |   1 +
 drivers/net/virtio/virtio_pci.h         |   1 +
 drivers/net/virtio/virtio_user_ethdev.c | 218 ++++++++++++++++++++++++++++++++
 3 files changed, 220 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user_ethdev.c

diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index 68068bd..d913df0 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -60,6 +60,7 @@ SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_kernel.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_dev.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user_ethdev.c
 endif
 
 # this lib depends upon:
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index a76daf7..d10d013 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -260,6 +260,7 @@ struct virtio_hw {
 	struct virtio_pci_common_cfg *common_cfg;
 	struct virtio_net_config *dev_cfg;
 	const struct virtio_pci_ops *vtpci_ops;
+	void	    *virtio_user_dev;
 };
 
 /*
diff --git a/drivers/net/virtio/virtio_user_ethdev.c b/drivers/net/virtio/virtio_user_ethdev.c
new file mode 100644
index 0000000..0ea3f23
--- /dev/null
+++ b/drivers/net/virtio/virtio_user_ethdev.c
@@ -0,0 +1,218 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <sys/types.h>
+#include <unistd.h>
+
+#include "virtio_logs.h"
+#include "virtio_pci.h"
+#include "virtqueue.h"
+#include "virtio_user/virtio_user_dev.h"
+
+#define virtio_user_get_dev(hw) \
+	((struct virtio_user_dev *)(hw)->virtio_user_dev);
+
+static void
+virtio_user_read_dev_config(struct virtio_hw *hw, uint64_t offset,
+		     void *dst, int length)
+{
+	int i;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (offset == offsetof(struct virtio_net_config, mac) &&
+	    length == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			((uint8_t *)dst)[i] = dev->mac_addr[i];
+		return;
+	}
+
+	if (offset == offsetof(struct virtio_net_config, status))
+		*(uint16_t *)dst = dev->status;
+
+	if (offset == offsetof(struct virtio_net_config, max_virtqueue_pairs))
+		*(uint16_t *)dst = dev->max_queue_pairs;
+}
+
+static void
+virtio_user_write_dev_config(struct virtio_hw *hw, uint64_t offset,
+		      const void *src, int length)
+{
+	int i;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if ((offset == offsetof(struct virtio_net_config, mac)) &&
+	    (length == ETHER_ADDR_LEN))
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			dev->mac_addr[i] = ((const uint8_t *)src)[i];
+	else
+		PMD_DRV_LOG(ERR, "not supported offset=%" PRIu64 ", len=%d\n",
+			    offset, length);
+}
+
+static void
+virtio_user_set_status(struct virtio_hw *hw, uint8_t status)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (status & VIRTIO_CONFIG_STATUS_DRIVER_OK)
+		virtio_user_start_device(dev);
+	dev->status = status;
+}
+
+static void
+virtio_user_reset(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	virtio_user_stop_device(dev);
+}
+
+static uint8_t
+virtio_user_get_status(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	return dev->status;
+}
+
+static uint64_t
+virtio_user_get_features(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	return dev->features;
+}
+
+static void
+virtio_user_set_features(struct virtio_hw *hw, uint64_t features)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	dev->features = features;
+}
+
+static uint8_t
+virtio_user_get_isr(struct virtio_hw *hw __rte_unused)
+{
+	/* When config interrupt happens, driver calls this function to query
+	 * what kinds of change happen. Interrupt mode not supported for now.
+	 */
+	return 0;
+}
+
+static uint16_t
+virtio_user_set_config_irq(struct virtio_hw *hw __rte_unused,
+		    uint16_t vec __rte_unused)
+{
+	return VIRTIO_MSI_NO_VECTOR;
+}
+
+/* This function is to get the queue size, aka, number of descs, of a specified
+ * queue. Different with the VHOST_USER_GET_QUEUE_NUM, which is used to get the
+ * max supported queues.
+ */
+static uint16_t
+virtio_user_get_queue_num(struct virtio_hw *hw, uint16_t queue_id __rte_unused)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	/* Currently, each queue has same queue size */
+	return dev->queue_size;
+}
+
+static int
+virtio_user_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+	uint16_t queue_idx = vq->vq_queue_index;
+	uint64_t desc_addr, avail_addr, used_addr;
+
+	desc_addr = (uint64_t)vq->mz->addr;
+	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
+	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
+							 ring[vq->vq_nentries]),
+				   VIRTIO_PCI_VRING_ALIGN);
+
+	dev->vrings[queue_idx].num = vq->vq_nentries;
+	dev->vrings[queue_idx].desc = (void *)desc_addr;
+	dev->vrings[queue_idx].avail = (void *)avail_addr;
+	dev->vrings[queue_idx].used = (void *)used_addr;
+
+	return 0;
+}
+
+static void
+virtio_user_del_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	/* For legacy devices, write 0 to VIRTIO_PCI_QUEUE_PFN port, QEMU
+	 * correspondingly stops the ioeventfds, and reset the status of
+	 * the device.
+	 * For modern devices, set queue desc, avail, used in PCI bar to 0,
+	 * not see any more behavior in QEMU.
+	 *
+	 * Here we just care about what information to deliver to vhost-user
+	 * or vhost-kernel. So we just close ioeventfd for now.
+	 */
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	close(dev->callfds[vq->vq_queue_index]);
+	close(dev->kickfds[vq->vq_queue_index]);
+}
+
+static void
+virtio_user_notify_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	uint64_t buf = 1;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (write(dev->kickfds[vq->vq_queue_index], &buf, sizeof(buf)) < 0)
+		PMD_DRV_LOG(ERR, "failed to kick backend: %s\n",
+			    strerror(errno));
+}
+
+static const struct virtio_pci_ops virtio_user_ops = {
+	.read_dev_cfg	= virtio_user_read_dev_config,
+	.write_dev_cfg	= virtio_user_write_dev_config,
+	.reset		= virtio_user_reset,
+	.get_status	= virtio_user_get_status,
+	.set_status	= virtio_user_set_status,
+	.get_features	= virtio_user_get_features,
+	.set_features	= virtio_user_set_features,
+	.get_isr	= virtio_user_get_isr,
+	.set_config_irq	= virtio_user_set_config_irq,
+	.get_queue_num	= virtio_user_get_queue_num,
+	.setup_queue	= virtio_user_setup_queue,
+	.del_queue	= virtio_user_del_queue,
+	.notify_queue	= virtio_user_notify_queue,
+};
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v6 7/7] virtio-user: add a new vdev named virtio-user
  2016-06-02  9:54 ` [PATCH v6 0/7] virtio support for container Jianfeng Tan
                     ` (5 preceding siblings ...)
  2016-06-02  9:54   ` [PATCH v6 6/7] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
@ 2016-06-02  9:54   ` Jianfeng Tan
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-02  9:54 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Add a new virtual device named vhost-user, which can be used just like
eth_ring, eth_null, etc. To reuse the code of original virtio, we do
some adjustment in virtio_ethdev.c, such as remove key _static_ of
eth_virtio_dev_init() so that it can be reused in virtual device; and
we add some check to make sure it will not crash.

Configured parameters include:
  - queues (optional, 1 by default), number of queue pairs, multi-queue
    not supported for now.
  - cq (optional, 0 by default), not supported for now.
  - mac (optional), random value will be given if not specified.
  - queue_size (optional, 256 by default), size of virtqueues.
  - path (madatory), path of vhost, depends on the file type, vhost
    user if the given path points to a unix socket; vhost-net if the
    given path points to a char device.
  - ifname (optional), specify the name of backend tap device; only
    valid when backend is vhost-net.

When enable CONFIG_RTE_VIRTIO_VDEV (enabled by default), the compiled
library can be used in both VM and container environment.

Examples:
path_vhost=/dev/vhost-net # use vhost-net as a backend
path_vhost=<path_to_vhost_user> # use vhost-user as a backend

sudo ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
    --socket-mem 0,1024 --no-pci --file-prefix=l2fwd \
    --vdev=virtio-user0,mac=00:01:02:03:04:05,path=$path_vhost -- -p 0x1

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 doc/guides/rel_notes/release_16_07.rst  |  11 ++
 doc/guides/sample_app_ug/vhost.rst      |  17 +++
 drivers/net/virtio/virtio_ethdev.c      |  19 ++-
 drivers/net/virtio/virtio_ethdev.h      |   2 +
 drivers/net/virtio/virtio_user_ethdev.c | 218 ++++++++++++++++++++++++++++++++
 5 files changed, 260 insertions(+), 7 deletions(-)

diff --git a/doc/guides/rel_notes/release_16_07.rst b/doc/guides/rel_notes/release_16_07.rst
index f6d543c..78787ca 100644
--- a/doc/guides/rel_notes/release_16_07.rst
+++ b/doc/guides/rel_notes/release_16_07.rst
@@ -34,6 +34,17 @@ This section should contain new features added in this release. Sample format:
 
   Refer to the previous release notes for examples.
 
+* **Virtio support for containers.**
+
+  Add a new virtual device, named virtio-user, to support virtio for containers.
+
+  Known limitations:
+
+  * Control queue and multi-queue are not supported yet.
+  * Cannot work with --huge-unlink.
+  * Cannot work with --no-huge.
+  * Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8) hugepages.
+  * Root privilege is a must for sorting hugepages by physical address.
 
 Resolved Issues
 ---------------
diff --git a/doc/guides/sample_app_ug/vhost.rst b/doc/guides/sample_app_ug/vhost.rst
index 5f81802..a93e54d 100644
--- a/doc/guides/sample_app_ug/vhost.rst
+++ b/doc/guides/sample_app_ug/vhost.rst
@@ -833,3 +833,20 @@ For example:
 The above message indicates that device 0 has been registered with MAC address cc:bb:bb:bb:bb:bb and VLAN tag 1000.
 Any packets received on the NIC with these values is placed on the devices receive queue.
 When a virtio-net device transmits packets, the VLAN tag is added to the packet by the DPDK vhost sample code.
+
+Running virtio-user with vhost-switch
+-------------------------------------
+
+We can also use virtio-user with vhost-switch now.
+Virtio-user is a virtual device that can be run in a application (container) parallelly with vhost in the same OS,
+aka, there is no need to start a VM. We just run it with a different --file-prefix to avoid startup failure.
+
+.. code-block:: console
+
+    cd ${RTE_SDK}/x86_64-native-linuxapp-gcc/app
+    ./testpmd -c 0x3 -n 4 --socket-mem 1024 --no-pci --file-prefix=virtio-user-testpmd \
+    --vdev=virtio-user0,mac=00:01:02:03:04:05,path=$path_vhost \
+    -- -i --txqflags=0xf01 --disable-hw-vlan
+
+There is no difference on the vhost side.
+Pleae note that there are some limitations (see release note for more information) in the usage of virtio-user.
diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 90f1a28..e1d5f0b 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -59,7 +59,6 @@
 #include "virtqueue.h"
 #include "virtio_rxtx.h"
 
-static int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
 static int eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev);
 static int  virtio_dev_configure(struct rte_eth_dev *dev);
 static int  virtio_dev_start(struct rte_eth_dev *dev);
@@ -1042,7 +1041,7 @@ rx_func_get(struct rte_eth_dev *eth_dev)
  * This function is based on probe() function in virtio_pci.c
  * It returns 0 on success.
  */
-static int
+int
 eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 {
 	struct virtio_hw *hw = eth_dev->data->dev_private;
@@ -1073,9 +1072,11 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	pci_dev = eth_dev->pci_dev;
 
-	ret = vtpci_init(pci_dev, hw, &dev_flags);
-	if (ret)
-		return ret;
+	if (pci_dev) {
+		ret = vtpci_init(pci_dev, hw, &dev_flags);
+		if (ret)
+			return ret;
+	}
 
 	/* Reset the device although not necessary at startup */
 	vtpci_reset(hw);
@@ -1167,7 +1168,8 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	PMD_INIT_LOG(DEBUG, "hw->max_rx_queues=%d   hw->max_tx_queues=%d",
 			hw->max_rx_queues, hw->max_tx_queues);
-	PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
+	if (pci_dev)
+		PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
 			eth_dev->data->port_id, pci_dev->id.vendor_id,
 			pci_dev->id.device_id);
 
@@ -1446,7 +1448,10 @@ virtio_dev_info_get(struct rte_eth_dev *dev, struct rte_eth_dev_info *dev_info)
 {
 	struct virtio_hw *hw = dev->data->dev_private;
 
-	dev_info->driver_name = dev->driver->pci_drv.name;
+	if (dev->pci_dev)
+		dev_info->driver_name = dev->driver->pci_drv.name;
+	else
+		dev_info->driver_name = "virtio-user PMD";
 	dev_info->max_rx_queues = (uint16_t)hw->max_rx_queues;
 	dev_info->max_tx_queues = (uint16_t)hw->max_tx_queues;
 	dev_info->min_rx_bufsize = VIRTIO_MIN_RX_BUFSIZE;
diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
index 66423a0..284afaa 100644
--- a/drivers/net/virtio/virtio_ethdev.h
+++ b/drivers/net/virtio/virtio_ethdev.h
@@ -113,6 +113,8 @@ uint16_t virtio_recv_pkts_vec(void *rx_queue, struct rte_mbuf **rx_pkts,
 uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 		uint16_t nb_pkts);
 
+int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
+
 /*
  * The VIRTIO_NET_F_GUEST_TSO[46] features permit the host to send us
  * frames larger than 1514 bytes. We do not yet support software LRO
diff --git a/drivers/net/virtio/virtio_user_ethdev.c b/drivers/net/virtio/virtio_user_ethdev.c
index 0ea3f23..11c1238 100644
--- a/drivers/net/virtio/virtio_user_ethdev.c
+++ b/drivers/net/virtio/virtio_user_ethdev.c
@@ -35,6 +35,10 @@
 #include <sys/types.h>
 #include <unistd.h>
 
+#include <rte_malloc.h>
+#include <rte_kvargs.h>
+
+#include "virtio_ethdev.h"
 #include "virtio_logs.h"
 #include "virtio_pci.h"
 #include "virtqueue.h"
@@ -216,3 +220,217 @@ static const struct virtio_pci_ops virtio_user_ops = {
 	.del_queue	= virtio_user_del_queue,
 	.notify_queue	= virtio_user_notify_queue,
 };
+
+static const char *valid_args[] = {
+#define VIRTIO_USER_ARG_QUEUES_NUM     "queues"
+	VIRTIO_USER_ARG_QUEUES_NUM,
+#define VIRTIO_USER_ARG_CQ_NUM         "cq"
+	VIRTIO_USER_ARG_CQ_NUM,
+#define VIRTIO_USER_ARG_MAC            "mac"
+	VIRTIO_USER_ARG_MAC,
+#define VIRTIO_USER_ARG_PATH           "path"
+	VIRTIO_USER_ARG_PATH,
+#define VIRTIO_USER_ARG_QUEUE_SIZE     "queue_size"
+	VIRTIO_USER_ARG_QUEUE_SIZE,
+#define VIRTIO_USER_ARG_IFNAME         "ifname"
+	VIRTIO_USER_ARG_IFNAME,
+	NULL
+};
+
+#define VIRTIO_USER_DEF_CQ_EN	0
+#define VIRTIO_USER_DEF_Q_NUM	1
+#define VIRTIO_USER_DEF_Q_SZ	256
+
+static int
+get_string_arg(const char *key __rte_unused,
+	       const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(char **)extra_args = strdup(value);
+
+	return 0;
+}
+
+static int
+get_integer_arg(const char *key __rte_unused,
+		const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(uint64_t *)extra_args = strtoull(value, NULL, 0);
+
+	return 0;
+}
+
+static struct rte_eth_dev *
+virtio_user_eth_dev_alloc(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct rte_eth_dev_data *data;
+	struct virtio_hw *hw;
+	struct virtio_user_dev *dev;
+
+	eth_dev = rte_eth_dev_allocate(name, RTE_ETH_DEV_VIRTUAL);
+	if (!eth_dev) {
+		PMD_INIT_LOG(ERR, "cannot alloc rte_eth_dev");
+		return NULL;
+	}
+
+	data = eth_dev->data;
+
+	hw = rte_zmalloc(NULL, sizeof(*hw), 0);
+	if (!hw) {
+		PMD_INIT_LOG(ERR, "malloc virtio_hw failed");
+		rte_eth_dev_release_port(eth_dev);
+		return NULL;
+	}
+
+	dev = rte_zmalloc(NULL, sizeof(*dev), 0);
+	if (!dev) {
+		PMD_INIT_LOG(ERR, "malloc virtio_user_dev failed");
+		rte_eth_dev_release_port(eth_dev);
+		rte_free(hw);
+		return NULL;
+	}
+
+	hw->vtpci_ops = &virtio_user_ops;
+	hw->use_msix = 0;
+	hw->modern   = 0;
+	hw->virtio_user_dev = dev;
+	data->dev_private = hw;
+	data->numa_node = SOCKET_ID_ANY;
+	data->kdrv = RTE_KDRV_NONE;
+	data->dev_flags = RTE_ETH_DEV_DETACHABLE;
+	eth_dev->pci_dev = NULL;
+	eth_dev->driver = NULL;
+	return eth_dev;
+}
+
+/* Dev initialization routine. Invoked once for each virtio vdev at
+ * EAL init time, see rte_eal_dev_init().
+ * Returns 0 on success.
+ */
+static int
+virtio_user_pmd_devinit(const char *name, const char *params)
+{
+	struct rte_kvargs *kvlist;
+	struct rte_eth_dev *eth_dev;
+	struct virtio_hw *hw;
+	uint64_t queues = VIRTIO_USER_DEF_Q_NUM;
+	uint64_t cq = VIRTIO_USER_DEF_CQ_EN;
+	uint64_t queue_size = VIRTIO_USER_DEF_Q_SZ;
+	char *path = NULL;
+	char *mac_addr = NULL;
+	char *ifname = NULL;
+	int ret = -1;
+
+	if (!params || params[0] == '\0') {
+		PMD_INIT_LOG(ERR, "arg %s is mandatory for virtio-user",
+			  VIRTIO_USER_ARG_QUEUE_SIZE);
+		goto end;
+	}
+
+	kvlist = rte_kvargs_parse(params, valid_args);
+	if (!kvlist) {
+		PMD_INIT_LOG(ERR, "error when parsing param");
+		goto end;
+	}
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_PATH) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_PATH,
+				   &get_string_arg, &path);
+	else {
+		PMD_INIT_LOG(ERR, "arg %s is mandatory for virtio-user\n",
+			  VIRTIO_USER_ARG_QUEUE_SIZE);
+		goto end;
+	}
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_MAC) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_MAC,
+				   &get_string_arg, &mac_addr);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_IFNAME) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_IFNAME,
+				   &get_string_arg, &ifname);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_QUEUE_SIZE) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_QUEUE_SIZE,
+				   &get_integer_arg, &queue_size);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_QUEUES_NUM) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_QUEUES_NUM,
+				   &get_integer_arg, &queues);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_CQ_NUM) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_CQ_NUM,
+				   &get_integer_arg, &cq);
+
+	eth_dev = virtio_user_eth_dev_alloc(name);
+	if (!eth_dev) {
+		PMD_INIT_LOG(ERR, "virtio-user fails to alloc device");
+		goto end;
+	}
+
+	hw = eth_dev->data->dev_private;
+	if (virtio_user_dev_init(hw->virtio_user_dev, path, queues, cq,
+				 queue_size, mac_addr, ifname) < 0)
+		goto end;
+
+	/* previously called by rte_eal_pci_probe() for physical dev */
+	if (eth_virtio_dev_init(eth_dev) < 0) {
+		PMD_INIT_LOG(ERR, "eth_virtio_dev_init fails");
+		goto end;
+	}
+	ret = 0;
+
+end:
+	if (path)
+		free(path);
+	if (mac_addr)
+		free(mac_addr);
+	if (ifname)
+		free(ifname);
+	return ret;
+}
+
+/** Called by rte_eth_dev_detach() */
+static int
+virtio_user_pmd_devuninit(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct virtio_hw *hw;
+	struct virtio_user_dev *dev;
+
+	if (!name)
+		return -EINVAL;
+
+	PMD_DRV_LOG(INFO, "Un-Initializing %s\n", name);
+	eth_dev = rte_eth_dev_allocated(name);
+	if (!eth_dev)
+		return -ENODEV;
+
+	/* make sure the device is stopped, queues freed */
+	rte_eth_dev_close(eth_dev->data->port_id);
+
+	hw = eth_dev->data->dev_private;
+	dev = hw->virtio_user_dev;
+	virtio_user_dev_uninit(dev);
+
+	rte_free(eth_dev->data->dev_private);
+	rte_free(eth_dev->data);
+	rte_eth_dev_release_port(eth_dev);
+
+	return 0;
+}
+
+static struct rte_driver virtio_user_driver = {
+	.name   = "virtio-user",
+	.type   = PMD_VDEV,
+	.init   = virtio_user_pmd_devinit,
+	.uninit = virtio_user_pmd_devuninit,
+};
+
+PMD_REGISTER_DRIVER(virtio_user_driver);
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* Re: [PATCH v6 6/7] virtio-user: add new virtual pci driver for virtio
  2016-06-02  9:54   ` [PATCH v6 6/7] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
@ 2016-06-06  8:01     ` Yuanhan Liu
  2016-06-06  8:31       ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Yuanhan Liu @ 2016-06-06  8:01 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

On Thu, Jun 02, 2016 at 09:54:36AM +0000, Jianfeng Tan wrote:
> +
> +	desc_addr = (uint64_t)vq->mz->addr;
> +	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
> +	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
> +							 ring[vq->vq_nentries]),
> +				   VIRTIO_PCI_VRING_ALIGN);
> +
> +	dev->vrings[queue_idx].num = vq->vq_nentries;
> +	dev->vrings[queue_idx].desc = (void *)desc_addr;
> +	dev->vrings[queue_idx].avail = (void *)avail_addr;
> +	dev->vrings[queue_idx].used = (void *)used_addr;

That would break 32 bit build. please also do more build and function
test, with and without CONFIG_RTE_VIRTIO_VDEV enabled, to make sure
we will not break anything. I'm sure you will meet build error without
that option enabled.

BTW, let's be consistent with using VIRTIO_USER_DEV instead of VDEV
or VIRTIO_VDEV?

Another thing that might be a bit late to ask is that how about
removing the vhost-net support? I mean, it's DPDK; if user stick
to using DPDK virtio-user, he will stick to using DPDK vhost-user
as well, but not the vhost-net. So, let's keep it being simple
first. And if there is really a need for vhost-net, we can add it
back later, easily. Makes sense?

I also would suggest you do a rebase based on my latest tree.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v6 6/7] virtio-user: add new virtual pci driver for virtio
  2016-06-06  8:01     ` Yuanhan Liu
@ 2016-06-06  8:31       ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-06-06  8:31 UTC (permalink / raw)
  To: Yuanhan Liu
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

Hi Yuanhan,


On 6/6/2016 4:01 PM, Yuanhan Liu wrote:
> On Thu, Jun 02, 2016 at 09:54:36AM +0000, Jianfeng Tan wrote:
>> +
>> +	desc_addr = (uint64_t)vq->mz->addr;
>> +	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
>> +	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
>> +							 ring[vq->vq_nentries]),
>> +				   VIRTIO_PCI_VRING_ALIGN);
>> +
>> +	dev->vrings[queue_idx].num = vq->vq_nentries;
>> +	dev->vrings[queue_idx].desc = (void *)desc_addr;
>> +	dev->vrings[queue_idx].avail = (void *)avail_addr;
>> +	dev->vrings[queue_idx].used = (void *)used_addr;
> That would break 32 bit build. please also do more build and function
> test, with and without CONFIG_RTE_VIRTIO_VDEV enabled, to make sure
> we will not break anything. I'm sure you will meet build error without
> that option enabled.

Yes, thanks for pointing this out.

>
> BTW, let's be consistent with using VIRTIO_USER_DEV instead of VDEV
> or VIRTIO_VDEV?

OK.

>
> Another thing that might be a bit late to ask is that how about
> removing the vhost-net support? I mean, it's DPDK; if user stick
> to using DPDK virtio-user, he will stick to using DPDK vhost-user
> as well, but not the vhost-net. So, let's keep it being simple
> first. And if there is really a need for vhost-net, we can add it
> back later, easily. Makes sense?

Yes, it makes sense, because from an initial test, I see low 
performance. Or anyone who are willing to use it can comment?

Thanks,
Jianfeng
>
> I also would suggest you do a rebase based on my latest tree.

No problem.

Thanks,
Jianfeng

>
> 	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v7 0/6] virtio support for container
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
                   ` (11 preceding siblings ...)
  2016-06-02  9:54 ` [PATCH v6 0/7] virtio support for container Jianfeng Tan
@ 2016-06-12  0:35 ` Jianfeng Tan
  2016-06-12  0:35   ` [PATCH v7 1/6] virtio: hide phys addr check inside pci ops Jianfeng Tan
                     ` (5 more replies)
  2016-06-13  6:38 ` [PATCH v8 0/6] virtio support for container Jianfeng Tan
  2016-06-15  9:03 ` [PATCH v9 " Jianfeng Tan
  14 siblings, 6 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-12  0:35 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

v7:
 - CONFIG_RTE_VIRTIO_VDEV -> CONFIG_RTE_VIRTIO_USER; and corresondingly,
   RTE_VIRTIO_VDEV -> RTE_VIRTIO_USER.
 - uint64_t -> uintptr_t, so that it can be compiled on 32-bit platform.
 - Rebase on latest dpdk-next-virtio branch.
 - Abandon abstracting related code into vring_hdr_desc_init(), instead,
   just move it behind setup_queue().

v6:
 - Move driver related code into from driver/net/virtio/virtio-user/ to
   driver/net/virtio/ directory, inside virtio_user_ethdev.c.
 - Rename vdev to virtio_user in comments and code.
 - Merge code, which lies in virtio_user_pci.c, into virtio_user_ethdev.c.
 - Add some comments at virtio-user special handling at virtio_dev_ethdev.c.
 - Merge document update into the 7nd commit where virtio-user is added.
 - Add usage with vhost-switch in vhost.rst.

v5:
 - Rename struct virtio_user_hw to struct virtio_user_dev.
 - Rename "vdev_private" to "virtio_user_dev".
 - Move special handling into virtio_ethdev.c from queue_setup().
 - Add vring in virtio_user_dev (remove rte_eth_dev_data), so that
   device does not depend on driver's data structure (rte_eth_dev_data).
 - Remove update on doc/guides/nics/overview.rst, because virtio-user has
   exact feature set with virtio.
 - Change "unsigned long int" to "uint64_t", "unsigned" to "uint32_t".
 - Remove unnecessary cast in vdev_read_dev_config().
 - Add functions in virtio_user_dev.c with prefix of "virtio_user_".
 - Rebase on virtio-next-virtio.

v4:
 - Avoid using dev_type, instead use (eth_dev->pci_device is NULL) to
   judge if it's virtual device or physical device.
 - Change the added device name to virtio-user.
 - Split into vhost_user.c, vhost_kernel.c, vhost.c, virtio_user_pci.c,
   virtio_user_dev.c.
 - Move virtio-user specific data from struct virtio_hw into struct
   virtio_user_hw.
 - Add support to send reset_owner message.
 - Change del_queue implementation. (This need more check)
 - Remove rte_panic(), and superseded with log.
 - Add reset_owner into virtio_pci_ops.reset.
 - Merge parameter "rx" and "tx" to "queues" to emliminate confusion.
 - Move get_features to after set_owner.
 - Redefine path in virtio_user_hw from char * to char [].

v3:
 - Remove --single-file option; do no change at EAL memory.
 - Remove the added API rte_eal_get_backfile_info(), instead we check all
   opened files with HUGEFILE_FMT to find hugepage files owned by DPDK.
 - Accordingly, add more restrictions at "Known issue" section.
 - Rename parameter from queue_num to queue_size for confusion.
 - Rename vhost_embedded.c to rte_eth_virtio_vdev.c.
 - Move code related to the newly added vdev to rte_eth_virtio_vdev.c, to
   reuse eth_virtio_dev_init(), remove its static declaration.
 - Implement dev_uninit() for rte_eth_dev_detach().
 - WARN -> ERR, in vhost_embedded.c
 - Add more commit message for clarify the model.

v2:
 - Rebase on the patchset of virtio 1.0 support.
 - Fix cannot create non-hugepage memory.
 - Fix wrong size of memory region when "single-file" is used.
 - Fix setting of offset in virtqueue to use virtual address.
 - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
 - Add mac option to specify the mac address of this virtual device.
 - Update doc.

This patchset is to provide high performance networking interface (virtio)
for container-based DPDK applications. The way of starting DPDK apps in
containers with ownership of NIC devices exclusively is beyond the scope.
The basic idea here is to present a new virtual device (named virtio-user),
which can be discovered and initialized by DPDK. To minimize the change,
we reuse already-existing virtio PMD code (driver/net/virtio/).

Background: Previously, we usually use a virtio device in the context of
QEMU/VM as below pic shows. Virtio nic is emulated in QEMU, and usually
presented in VM as a PCI device.

  ------------------
  |  virtio driver |  ----->  VM
  ------------------
        |
        | ----------> (over PCI bus or MMIO or Channel I/O)
        |
  ------------------
  | device emulate |
  |                |  ----->  QEMU
  | vhost adapter  |
  ------------------
        |
        | ----------> (vhost-user protocol or vhost-net ioctls)
        |
  ------------------
  | vhost backend  |
  ------------------
 
Compared to QEMU/VM case, virtio support for contaner requires to embedded
device framework inside the virtio PMD. So this converged driver actually
plays three roles:
  - virtio driver to drive this new kind of virtual device;
  - device emulation to present this virtual device and reponse to the
    virtio driver, which is originally by QEMU;
  - and the role to communicate with vhost backend, which is also
    originally by QEMU.

The code layout and functionality of each module:
 
  ----------------------
  | ------------------ |
  | | virtio driver  | |----> (virtio_user_ethdev.c)
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate |-|----> (virtio_user_dev.c)
  | |                | |
  | | vhost adapter  |-|----> (vhost_user.c, vhost_kernel.c, vhost.c)
  | ------------------ |
  ----------------------
         |
         | -------------- --> (vhost-user protocol or vhost-net ioctls)
         |
   ------------------
   | vhost backend  |
   ------------------

How to share memory? In VM's case, qemu always shares all physical layout
to backend. But it's not feasible for a container, as a process, to share
all virtual memory regions to backend. So only specified virtual memory
regions (with type of shared) are sent to backend. It's a limitation that
only addresses in these areas can be used to transmit or receive packets.

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").

How to use?

a. Apply this patchset.

b. To compile container apps:
$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc

c. To build a docker image using Dockerfile below.
$: cat ./Dockerfile
FROM ubuntu:latest
WORKDIR /usr/src/dpdk
COPY . /usr/src/dpdk
ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
$: docker build -t dpdk-app-l2fwd .

d. Used with vhost-user
$: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
	--socket-mem 1024,1024 -- -p 0x1 --stats 1
$: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=virtio-user0,path=/var/run/usvhost -- -p 0x1

f. Used with vhost-net
$: modprobe vhost
$: modprobe vhost-net
$: docker run -i -t --privileged \
	-v /dev/vhost-net:/dev/vhost-net \
	-v /dev/net/tun:/dev/net/tun \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=virtio-user0,path=/dev/vhost-net -- -p 0x1

By the way, it's not necessary to run in a container.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>




Jianfeng Tan (6):
  virtio: hide phys addr check inside pci ops
  virtio: enable use virtual address to fill desc
  virtio-user: add vhost adapter layer
  virtio-user: add device emulation layer APIs
  virtio-user: add new virtual pci driver for virtio
  virtio-user: add a new vdev named virtio-user

 config/common_linuxapp                           |   1 +
 doc/guides/rel_notes/release_16_07.rst           |  11 +
 doc/guides/sample_app_ug/vhost.rst               |  17 +
 drivers/net/virtio/Makefile                      |   8 +
 drivers/net/virtio/virtio_ethdev.c               |  77 ++--
 drivers/net/virtio/virtio_ethdev.h               |   2 +
 drivers/net/virtio/virtio_pci.c                  |  30 +-
 drivers/net/virtio/virtio_pci.h                  |   3 +-
 drivers/net/virtio/virtio_rxtx.c                 |   5 +-
 drivers/net/virtio/virtio_rxtx_simple.c          |  13 +-
 drivers/net/virtio/virtio_user/vhost.c           | 105 ++++++
 drivers/net/virtio/virtio_user/vhost.h           | 222 ++++++++++++
 drivers/net/virtio/virtio_user/vhost_kernel.c    | 254 +++++++++++++
 drivers/net/virtio/virtio_user/vhost_user.c      | 378 ++++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 263 ++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.h |  64 ++++
 drivers/net/virtio/virtio_user_ethdev.c          | 436 +++++++++++++++++++++++
 drivers/net/virtio/virtqueue.h                   |  10 +
 18 files changed, 1857 insertions(+), 42 deletions(-)
 create mode 100644 drivers/net/virtio/virtio_user/vhost.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost.h
 create mode 100644 drivers/net/virtio/virtio_user/vhost_kernel.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h
 create mode 100644 drivers/net/virtio/virtio_user_ethdev.c

-- 
2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v7 1/6] virtio: hide phys addr check inside pci ops
  2016-06-12  0:35 ` [PATCH v7 0/6] virtio support for container Jianfeng Tan
@ 2016-06-12  0:35   ` Jianfeng Tan
  2016-06-12  0:35   ` [PATCH v7 2/6] virtio: enable use virtual address to fill desc Jianfeng Tan
                     ` (4 subsequent siblings)
  5 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-12  0:35 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is to move phys addr check from virtio_dev_queue_setup
to pci ops. To makt that happen, make sure virtio_ops.setup_queue
return the result if we pass through the check.

Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Acked-by: Yuanhan Liu <yuanhan.liu@linux.intel.com>
---
 drivers/net/virtio/virtio_ethdev.c | 17 ++++++-----------
 drivers/net/virtio/virtio_pci.c    | 30 ++++++++++++++++++++++++++++--
 drivers/net/virtio/virtio_pci.h    |  2 +-
 3 files changed, 35 insertions(+), 14 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index a995520..841949b 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -385,16 +385,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		}
 	}
 
-	/*
-	 * Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
-	 * and only accepts 32 bit page frame number.
-	 * Check if the allocated physical memory exceeds 16TB.
-	 */
-	if ((mz->phys_addr + vq->vq_ring_size - 1) >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
-		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
-		ret = -ENOMEM;
-		goto fail_q_alloc;
-	}
 	memset(mz->addr, 0, sizeof(mz->len));
 
 	vq->vq_ring_mem = mz->phys_addr;
@@ -479,7 +469,12 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		*pvq = cvq;
 	}
 
-	hw->vtpci_ops->setup_queue(hw, vq);
+	if (hw->vtpci_ops->setup_queue(hw, vq) < 0) {
+		PMD_INIT_LOG(ERR, "setup_queue failed");
+		virtio_dev_queue_release(vq);
+		return -EINVAL;
+	}
+
 	vq->configured = 1;
 	return 0;
 
diff --git a/drivers/net/virtio/virtio_pci.c b/drivers/net/virtio/virtio_pci.c
index d0f2428..8d0c983 100644
--- a/drivers/net/virtio/virtio_pci.c
+++ b/drivers/net/virtio/virtio_pci.c
@@ -55,6 +55,22 @@
  */
 #define VIRTIO_PCI_CONFIG(hw) (((hw)->use_msix) ? 24 : 20)
 
+static inline int
+check_vq_phys_addr_ok(struct virtqueue *vq)
+{
+	/* Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
+	 * and only accepts 32 bit page frame number.
+	 * Check if the allocated physical memory exceeds 16TB.
+	 */
+	if ((vq->vq_ring_mem + vq->vq_ring_size - 1) >>
+			(VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
+		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
+		return 0;
+	}
+
+	return 1;
+}
+
 static void
 legacy_read_dev_config(struct virtio_hw *hw, size_t offset,
 		       void *dst, int length)
@@ -143,15 +159,20 @@ legacy_get_queue_num(struct virtio_hw *hw, uint16_t queue_id)
 	return dst;
 }
 
-static void
+static int
 legacy_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 {
 	uint32_t src;
 
+	if (!check_vq_phys_addr_ok(vq))
+		return -1;
+
 	rte_eal_pci_ioport_write(&hw->io, &vq->vq_queue_index, 2,
 			 VIRTIO_PCI_QUEUE_SEL);
 	src = vq->vq_ring_mem >> VIRTIO_PCI_QUEUE_ADDR_SHIFT;
 	rte_eal_pci_ioport_write(&hw->io, &src, 4, VIRTIO_PCI_QUEUE_PFN);
+
+	return 0;
 }
 
 static void
@@ -367,12 +388,15 @@ modern_get_queue_num(struct virtio_hw *hw, uint16_t queue_id)
 	return io_read16(&hw->common_cfg->queue_size);
 }
 
-static void
+static int
 modern_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 {
 	uint64_t desc_addr, avail_addr, used_addr;
 	uint16_t notify_off;
 
+	if (!check_vq_phys_addr_ok(vq))
+		return -1;
+
 	desc_addr = vq->vq_ring_mem;
 	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
 	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
@@ -400,6 +424,8 @@ modern_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 	PMD_INIT_LOG(DEBUG, "\t used_addr: %" PRIx64, used_addr);
 	PMD_INIT_LOG(DEBUG, "\t notify addr: %p (notify offset: %u)",
 		vq->notify_addr, notify_off);
+
+	return 0;
 }
 
 static void
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index f20468a..6c7f8d7 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -235,7 +235,7 @@ struct virtio_pci_ops {
 	uint16_t (*set_config_irq)(struct virtio_hw *hw, uint16_t vec);
 
 	uint16_t (*get_queue_num)(struct virtio_hw *hw, uint16_t queue_id);
-	void (*setup_queue)(struct virtio_hw *hw, struct virtqueue *vq);
+	int (*setup_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 	void (*del_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 	void (*notify_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 };
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v7 2/6] virtio: enable use virtual address to fill desc
  2016-06-12  0:35 ` [PATCH v7 0/6] virtio support for container Jianfeng Tan
  2016-06-12  0:35   ` [PATCH v7 1/6] virtio: hide phys addr check inside pci ops Jianfeng Tan
@ 2016-06-12  0:35   ` Jianfeng Tan
  2016-06-12  0:35   ` [PATCH v7 3/6] virtio-user: add vhost adapter layer Jianfeng Tan
                     ` (3 subsequent siblings)
  5 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-12  0:35 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is related to how to calculate relative address for vhost
backend.

The principle is that: based on one or multiple shared memory regions,
vhost maintains a reference system with the frontend start address,
backend start address, and length for each segment, so that each
frontend address (GPA, Guest Physical Address) can be translated into
vhost-recognizable backend address. To make the address translation
efficient, we need to maintain as few regions as possible. In the case
of VM, GPA is always locally continuous. But for some other case, like
virtio-user, we use virtual address here.

It basically means:
  a. when set_base_addr, VA address is used;
  b. when preparing RX's descriptors, VA address is used;
  c. when transmitting packets, VA is filled in TX's descriptors;
  d. in TX and CQ's header, VA is used.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/virtio_ethdev.c      | 43 +++++++++++++++++++++++----------
 drivers/net/virtio/virtio_rxtx.c        |  5 ++--
 drivers/net/virtio/virtio_rxtx_simple.c | 13 +++++-----
 drivers/net/virtio/virtqueue.h          | 10 ++++++++
 4 files changed, 49 insertions(+), 22 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 841949b..9ccce79 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -431,9 +431,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		rxvq->mz = mz;
 		*pvq = rxvq;
 	} else if (queue_type == VTNET_TQ) {
-		struct virtio_tx_region *txr;
-		unsigned int i;
-
 		txvq = (struct virtnet_tx *)RTE_PTR_ADD(vq, sz_vq);
 		txvq->vq = vq;
 		txvq->port_id = dev->data->port_id;
@@ -442,6 +439,36 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		txvq->virtio_net_hdr_mz = hdr_mz;
 		txvq->virtio_net_hdr_mem = hdr_mz->phys_addr;
 
+		*pvq = txvq;
+	} else if (queue_type == VTNET_CQ) {
+		cvq = (struct virtnet_ctl *)RTE_PTR_ADD(vq, sz_vq);
+		cvq->vq = vq;
+		cvq->mz = mz;
+		cvq->virtio_net_hdr_mz = hdr_mz;
+		cvq->virtio_net_hdr_mem = hdr_mz->phys_addr;
+		memset(cvq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
+		*pvq = cvq;
+	}
+
+	/* For virtio-user case (that is when dev->pci_dev is NULL), we use
+	 * virtual address. And we need properly set _offset_, please see
+	 * MBUF_DATA_DMA_ADDR in virtqueue.h for more information.
+	 */
+	if (dev->pci_dev)
+		vq->offset = offsetof(struct rte_mbuf, buf_physaddr);
+	else {
+		vq->vq_ring_mem = (uintptr_t)mz->addr;
+		vq->offset = offsetof(struct rte_mbuf, buf_addr);
+		if (queue_type == VTNET_TQ)
+			txvq->virtio_net_hdr_mem = (uintptr_t)hdr_mz->addr;
+		else if (queue_type == VTNET_CQ)
+			cvq->virtio_net_hdr_mem = (uintptr_t)hdr_mz->addr;
+	}
+
+	if (queue_type == VTNET_TQ) {
+		struct virtio_tx_region *txr;
+		unsigned int i;
+
 		txr = hdr_mz->addr;
 		memset(txr, 0, vq_size * sizeof(*txr));
 		for (i = 0; i < vq_size; i++) {
@@ -457,16 +484,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 			start_dp->len = hw->vtnet_hdr_size;
 			start_dp->flags = VRING_DESC_F_NEXT;
 		}
-
-		*pvq = txvq;
-	} else if (queue_type == VTNET_CQ) {
-		cvq = (struct virtnet_ctl *)RTE_PTR_ADD(vq, sz_vq);
-		cvq->vq = vq;
-		cvq->mz = mz;
-		cvq->virtio_net_hdr_mz = hdr_mz;
-		cvq->virtio_net_hdr_mem = hdr_mz->phys_addr;
-		memset(cvq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
-		*pvq = cvq;
 	}
 
 	if (hw->vtpci_ops->setup_queue(hw, vq) < 0) {
diff --git a/drivers/net/virtio/virtio_rxtx.c b/drivers/net/virtio/virtio_rxtx.c
index f371423..2e7205b 100644
--- a/drivers/net/virtio/virtio_rxtx.c
+++ b/drivers/net/virtio/virtio_rxtx.c
@@ -193,8 +193,7 @@ virtqueue_enqueue_recv_refill(struct virtqueue *vq, struct rte_mbuf *cookie)
 
 	start_dp = vq->vq_ring.desc;
 	start_dp[idx].addr =
-		(uint64_t)(cookie->buf_physaddr + RTE_PKTMBUF_HEADROOM
-		- hw->vtnet_hdr_size);
+		MBUF_DATA_DMA_ADDR(cookie, vq->offset) - hw->vtnet_hdr_size;
 	start_dp[idx].len =
 		cookie->buf_len - RTE_PKTMBUF_HEADROOM + hw->vtnet_hdr_size;
 	start_dp[idx].flags =  VRING_DESC_F_WRITE;
@@ -266,7 +265,7 @@ virtqueue_enqueue_xmit(struct virtnet_tx *txvq, struct rte_mbuf *cookie,
 	}
 
 	do {
-		start_dp[idx].addr  = rte_mbuf_data_dma_addr(cookie);
+		start_dp[idx].addr  = MBUF_DATA_DMA_ADDR(cookie, vq->offset);
 		start_dp[idx].len   = cookie->data_len;
 		start_dp[idx].flags = cookie->next ? VRING_DESC_F_NEXT : 0;
 		idx = start_dp[idx].next;
diff --git a/drivers/net/virtio/virtio_rxtx_simple.c b/drivers/net/virtio/virtio_rxtx_simple.c
index fdd655d..7b50119 100644
--- a/drivers/net/virtio/virtio_rxtx_simple.c
+++ b/drivers/net/virtio/virtio_rxtx_simple.c
@@ -80,8 +80,8 @@ virtqueue_enqueue_recv_refill_simple(struct virtqueue *vq,
 	vq->sw_ring[desc_idx] = cookie;
 
 	start_dp = vq->vq_ring.desc;
-	start_dp[desc_idx].addr = (uint64_t)((uintptr_t)cookie->buf_physaddr +
-		RTE_PKTMBUF_HEADROOM - vq->hw->vtnet_hdr_size);
+	start_dp[desc_idx].addr = MBUF_DATA_DMA_ADDR(cookie, vq->offset) -
+				  vq->hw->vtnet_hdr_size;
 	start_dp[desc_idx].len = cookie->buf_len -
 		RTE_PKTMBUF_HEADROOM + vq->hw->vtnet_hdr_size;
 
@@ -120,8 +120,8 @@ virtio_rxq_rearm_vec(struct virtnet_rx *rxvq)
 		*(uint64_t *)p = rxvq->mbuf_initializer;
 
 		start_dp[i].addr =
-			(uint64_t)((uintptr_t)sw_ring[i]->buf_physaddr +
-			RTE_PKTMBUF_HEADROOM - vq->hw->vtnet_hdr_size);
+			MBUF_DATA_DMA_ADDR(sw_ring[i], vq->offset) -
+			vq->hw->vtnet_hdr_size;
 		start_dp[i].len = sw_ring[i]->buf_len -
 			RTE_PKTMBUF_HEADROOM + vq->hw->vtnet_hdr_size;
 	}
@@ -369,7 +369,7 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 			vq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 		for (i = 0; i < nb_tail; i++) {
 			start_dp[desc_idx].addr =
-				rte_mbuf_data_dma_addr(*tx_pkts);
+				MBUF_DATA_DMA_ADDR(*tx_pkts, vq->offset);
 			start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 			tx_pkts++;
 			desc_idx++;
@@ -380,7 +380,8 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 	for (i = 0; i < nb_commit; i++)
 		vq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 	for (i = 0; i < nb_commit; i++) {
-		start_dp[desc_idx].addr = rte_mbuf_data_dma_addr(*tx_pkts);
+		start_dp[desc_idx].addr =
+			MBUF_DATA_DMA_ADDR(*tx_pkts, vq->offset);
 		start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 		tx_pkts++;
 		desc_idx++;
diff --git a/drivers/net/virtio/virtqueue.h b/drivers/net/virtio/virtqueue.h
index 17c1ea1..455aaaf 100644
--- a/drivers/net/virtio/virtqueue.h
+++ b/drivers/net/virtio/virtqueue.h
@@ -66,6 +66,14 @@ struct rte_mbuf;
 
 #define VIRTQUEUE_MAX_NAME_SZ 32
 
+#ifdef RTE_VIRTIO_USER
+#define MBUF_DATA_DMA_ADDR(mb, offset) \
+	((uint64_t)((uintptr_t)(*(void **)((uintptr_t)mb + offset)) \
+			+ (mb)->data_off))
+#else /* RTE_VIRTIO_USER */
+#define MBUF_DATA_DMA_ADDR(mb, offset) rte_mbuf_data_dma_addr(mb)
+#endif /* RTE_VIRTIO_USER */
+
 #define VTNET_SQ_RQ_QUEUE_IDX 0
 #define VTNET_SQ_TQ_QUEUE_IDX 1
 #define VTNET_SQ_CQ_QUEUE_IDX 2
@@ -175,6 +183,7 @@ struct virtqueue {
 	unsigned int vq_ring_size;
 
 	phys_addr_t vq_ring_mem; /**< physical address of vring */
+				/**< use virtual address for virtio-user. */
 
 	/**
 	 * Head of the free chain in the descriptor table. If
@@ -184,6 +193,7 @@ struct virtqueue {
 	uint16_t  vq_desc_head_idx;
 	uint16_t  vq_desc_tail_idx;
 	uint16_t  vq_queue_index;   /**< PCI queue index */
+	uint16_t  offset; /**< relative offset to obtain addr in mbuf */
 	uint16_t  *notify_addr;
 	int configured;
 	struct rte_mbuf **sw_ring;  /**< RX software ring. */
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v7 3/6] virtio-user: add vhost adapter layer
  2016-06-12  0:35 ` [PATCH v7 0/6] virtio support for container Jianfeng Tan
  2016-06-12  0:35   ` [PATCH v7 1/6] virtio: hide phys addr check inside pci ops Jianfeng Tan
  2016-06-12  0:35   ` [PATCH v7 2/6] virtio: enable use virtual address to fill desc Jianfeng Tan
@ 2016-06-12  0:35   ` Jianfeng Tan
  2016-06-12  0:35   ` [PATCH v7 4/6] virtio-user: add device emulation layer APIs Jianfeng Tan
                     ` (2 subsequent siblings)
  5 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-12  0:35 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is to provide vhost adapter layer implementations. Instead
of relying on a hypervisor to translate between device emulation and
vhost backend, here we directly talk with vhost backend through the
vhost file. Depending on the type of vhost file,
  - vhost-user is used if the given path points to a unix socket;
  - vhost-kernel is used if the given path points to a char device.

Here three main APIs are provided to upper layer (device emulation):
  - vhost_user_setup(), to set up env to talk to a vhost user backend;
  - vhost_kernel_setup(), to set up env to talk to a vhost kernel backend.
  - vhost_call(), to provide a unified interface to communicate with
    vhost backend.

  ----------------------
  | ------------------ |
  | | virtio driver  | |
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate | |
  | |                | |
  | | vhost adapter  |-|----> (vhost_user.c, vhost_kernel.c, vhost.c)
  | ------------------ |
  ----------------------
            |
            | -------------- --> (vhost-user protocol or vhost-net ioctls)
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 config/common_linuxapp                        |   1 +
 drivers/net/virtio/Makefile                   |   6 +
 drivers/net/virtio/virtio_user/vhost.c        | 105 +++++++
 drivers/net/virtio/virtio_user/vhost.h        | 222 +++++++++++++++
 drivers/net/virtio/virtio_user/vhost_kernel.c | 254 +++++++++++++++++
 drivers/net/virtio/virtio_user/vhost_user.c   | 378 ++++++++++++++++++++++++++
 6 files changed, 966 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/vhost.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost.h
 create mode 100644 drivers/net/virtio/virtio_user/vhost_kernel.c
 create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c

diff --git a/config/common_linuxapp b/config/common_linuxapp
index 7e698e2..2483dfa 100644
--- a/config/common_linuxapp
+++ b/config/common_linuxapp
@@ -43,3 +43,4 @@ CONFIG_RTE_LIBRTE_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_AF_PACKET=y
 CONFIG_RTE_LIBRTE_POWER=y
+CONFIG_RTE_VIRTIO_USER=y
diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index ef84f60..892c2ae 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -55,6 +55,12 @@ ifeq ($(findstring RTE_MACHINE_CPUFLAG_SSSE3,$(CFLAGS)),RTE_MACHINE_CPUFLAG_SSSE
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx_simple.c
 endif
 
+ifeq ($(CONFIG_RTE_VIRTIO_USER),y)
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_kernel.c
+endif
+
 # this lib depends upon:
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_eal lib/librte_ether
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_mempool lib/librte_mbuf
diff --git a/drivers/net/virtio/virtio_user/vhost.c b/drivers/net/virtio/virtio_user/vhost.c
new file mode 100644
index 0000000..1944a97
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost.c
@@ -0,0 +1,105 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include "vhost.h"
+
+static const char * const vhost_msg_strings[] = {
+	[VHOST_MSG_SET_OWNER] = "VHOST_MSG_SET_OWNER",
+	[VHOST_MSG_RESET_OWNER] = "VHOST_MSG_RESET_OWNER",
+	[VHOST_MSG_SET_FEATURES] = "VHOST_MSG_SET_FEATURES",
+	[VHOST_MSG_GET_FEATURES] = "VHOST_MSG_GET_FEATURES",
+	[VHOST_MSG_SET_VRING_CALL] = "VHOST_MSG_SET_VRING_CALL",
+	[VHOST_MSG_SET_VRING_NUM] = "VHOST_MSG_SET_VRING_NUM",
+	[VHOST_MSG_SET_VRING_BASE] = "VHOST_MSG_SET_VRING_BASE",
+	[VHOST_MSG_GET_VRING_BASE] = "VHOST_MSG_GET_VRING_BASE",
+	[VHOST_MSG_SET_VRING_ADDR] = "VHOST_MSG_SET_VRING_ADDR",
+	[VHOST_MSG_SET_VRING_KICK] = "VHOST_MSG_SET_VRING_KICK",
+	[VHOST_MSG_SET_MEM_TABLE] = "VHOST_MSG_SET_MEM_TABLE",
+	NULL,
+};
+
+static uint64_t vhost_req_map[][2] = {
+	[VHOST_MSG_SET_OWNER] = {
+		VHOST_SET_OWNER, VHOST_USER_SET_OWNER
+	},
+	[VHOST_MSG_RESET_OWNER] = {
+		VHOST_RESET_OWNER, VHOST_USER_RESET_OWNER
+	},
+	[VHOST_MSG_SET_FEATURES] = {
+		VHOST_SET_FEATURES, VHOST_USER_SET_FEATURES
+	},
+	[VHOST_MSG_GET_FEATURES] = {
+		VHOST_GET_FEATURES, VHOST_USER_GET_FEATURES
+	},
+	[VHOST_MSG_SET_VRING_CALL] = {
+		VHOST_SET_VRING_CALL, VHOST_USER_SET_VRING_CALL
+	},
+	[VHOST_MSG_SET_VRING_NUM] = {
+		VHOST_SET_VRING_NUM, VHOST_USER_SET_VRING_NUM
+	},
+	[VHOST_MSG_SET_VRING_BASE] = {
+		VHOST_SET_VRING_BASE, VHOST_USER_SET_VRING_BASE
+	},
+	[VHOST_MSG_GET_VRING_BASE] = {
+		VHOST_GET_VRING_BASE, VHOST_USER_GET_VRING_BASE
+	},
+	[VHOST_MSG_SET_VRING_ADDR] = {
+		VHOST_SET_VRING_ADDR, VHOST_USER_SET_VRING_ADDR
+	},
+	[VHOST_MSG_SET_VRING_KICK] = {
+		VHOST_SET_VRING_KICK, VHOST_USER_SET_VRING_KICK
+	},
+	[VHOST_MSG_SET_MEM_TABLE] = {
+		VHOST_SET_MEM_TABLE, VHOST_USER_SET_MEM_TABLE
+	},
+};
+
+int
+vhost_call(int vhostfd, int type, uint64_t req, void *arg)
+{
+	uint64_t req_new;
+	int ret;
+
+	PMD_DRV_LOG(INFO, "%s", vhost_msg_strings[req]);
+
+	req_new = vhost_req_map[req][type];
+	if (type == VHOST_USER)
+		ret = vhost_user_sock(vhostfd, req_new, arg);
+	else
+		ret = vhost_kernel_ioctl(vhostfd, req_new, arg);
+
+	if (ret < 0)
+		PMD_DRV_LOG(ERR, "vhost_call %s failed: %s\n",
+			    vhost_msg_strings[req], strerror(errno));
+	return ret;
+}
diff --git a/drivers/net/virtio/virtio_user/vhost.h b/drivers/net/virtio/virtio_user/vhost.h
new file mode 100644
index 0000000..6bac1e8
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost.h
@@ -0,0 +1,222 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VHOST_NET_USER_H
+#define _VHOST_NET_USER_H
+
+#include <stdint.h>
+#include <linux/types.h>
+#include <linux/ioctl.h>
+
+#include "../virtio_pci.h"
+#include "../virtio_logs.h"
+#include "../virtqueue.h"
+
+#define VHOST_MEMORY_MAX_NREGIONS 8
+
+struct vhost_vring_state {
+	unsigned int index;
+	unsigned int num;
+};
+
+struct vhost_vring_file {
+	unsigned int index;
+	int fd;
+};
+
+struct vhost_vring_addr {
+	unsigned int index;
+	/* Option flags. */
+	unsigned int flags;
+	/* Flag values: */
+	/* Whether log address is valid. If set enables logging. */
+#define VHOST_VRING_F_LOG 0
+
+	/* Start of array of descriptors (virtually contiguous) */
+	uint64_t desc_user_addr;
+	/* Used structure address. Must be 32 bit aligned */
+	uint64_t used_user_addr;
+	/* Available structure address. Must be 16 bit aligned */
+	uint64_t avail_user_addr;
+	/* Logging support. */
+	/* Log writes to used structure, at offset calculated from specified
+	 * address. Address must be 32 bit aligned.
+	 */
+	uint64_t log_guest_addr;
+};
+
+enum vhost_user_request {
+	VHOST_USER_NONE = 0,
+	VHOST_USER_GET_FEATURES = 1,
+	VHOST_USER_SET_FEATURES = 2,
+	VHOST_USER_SET_OWNER = 3,
+	VHOST_USER_RESET_OWNER = 4,
+	VHOST_USER_SET_MEM_TABLE = 5,
+	VHOST_USER_SET_LOG_BASE = 6,
+	VHOST_USER_SET_LOG_FD = 7,
+	VHOST_USER_SET_VRING_NUM = 8,
+	VHOST_USER_SET_VRING_ADDR = 9,
+	VHOST_USER_SET_VRING_BASE = 10,
+	VHOST_USER_GET_VRING_BASE = 11,
+	VHOST_USER_SET_VRING_KICK = 12,
+	VHOST_USER_SET_VRING_CALL = 13,
+	VHOST_USER_SET_VRING_ERR = 14,
+	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
+	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
+	VHOST_USER_GET_QUEUE_NUM = 17,
+	VHOST_USER_SET_VRING_ENABLE = 18,
+	VHOST_USER_MAX
+};
+
+struct vhost_memory_region {
+	uint64_t guest_phys_addr;
+	uint64_t memory_size; /* bytes */
+	uint64_t userspace_addr;
+	uint64_t mmap_offset;
+};
+
+struct vhost_memory_kernel {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[0];
+};
+
+struct vhost_memory {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[VHOST_MEMORY_MAX_NREGIONS];
+};
+
+struct vhost_user_msg {
+	enum vhost_user_request request;
+
+#define VHOST_USER_VERSION_MASK     0x3
+#define VHOST_USER_REPLY_MASK       (0x1 << 2)
+	uint32_t flags;
+	uint32_t size; /* the following payload size */
+	union {
+#define VHOST_USER_VRING_IDX_MASK   0xff
+#define VHOST_USER_VRING_NOFD_MASK  (0x1 << 8)
+		uint64_t u64;
+		struct vhost_vring_state state;
+		struct vhost_vring_addr addr;
+		struct vhost_memory memory;
+	} payload;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+} __attribute((packed));
+
+#define VHOST_USER_HDR_SIZE offsetof(struct vhost_user_msg, payload.u64)
+#define VHOST_USER_PAYLOAD_SIZE \
+	(sizeof(struct vhost_user_msg) - VHOST_USER_HDR_SIZE)
+
+/* The version of the protocol we support */
+#define VHOST_USER_VERSION    0x1
+
+/* ioctls */
+
+#define VHOST_VIRTIO 0xAF
+
+#define VHOST_GET_FEATURES _IOR(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_FEATURES _IOW(VHOST_VIRTIO, 0x00, __u64)
+#define VHOST_SET_OWNER _IO(VHOST_VIRTIO, 0x01)
+#define VHOST_RESET_OWNER _IO(VHOST_VIRTIO, 0x02)
+#define VHOST_SET_MEM_TABLE _IOW(VHOST_VIRTIO, 0x03, struct vhost_memory_kernel)
+#define VHOST_SET_LOG_BASE _IOW(VHOST_VIRTIO, 0x04, __u64)
+#define VHOST_SET_LOG_FD _IOW(VHOST_VIRTIO, 0x07, int)
+#define VHOST_SET_VRING_NUM _IOW(VHOST_VIRTIO, 0x10, struct vhost_vring_state)
+#define VHOST_SET_VRING_ADDR _IOW(VHOST_VIRTIO, 0x11, struct vhost_vring_addr)
+#define VHOST_SET_VRING_BASE _IOW(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_GET_VRING_BASE _IOWR(VHOST_VIRTIO, 0x12, struct vhost_vring_state)
+#define VHOST_SET_VRING_KICK _IOW(VHOST_VIRTIO, 0x20, struct vhost_vring_file)
+#define VHOST_SET_VRING_CALL _IOW(VHOST_VIRTIO, 0x21, struct vhost_vring_file)
+#define VHOST_SET_VRING_ERR _IOW(VHOST_VIRTIO, 0x22, struct vhost_vring_file)
+#define VHOST_NET_SET_BACKEND _IOW(VHOST_VIRTIO, 0x30, struct vhost_vring_file)
+
+/*****************************************************************************/
+
+/* Ioctl defines */
+#define TUNSETIFF     _IOW('T', 202, int)
+#define TUNGETFEATURES _IOR('T', 207, unsigned int)
+#define TUNSETOFFLOAD  _IOW('T', 208, unsigned int)
+#define TUNGETIFF      _IOR('T', 210, unsigned int)
+#define TUNSETSNDBUF   _IOW('T', 212, int)
+#define TUNGETVNETHDRSZ _IOR('T', 215, int)
+#define TUNSETVNETHDRSZ _IOW('T', 216, int)
+#define TUNSETQUEUE  _IOW('T', 217, int)
+#define TUNSETVNETLE _IOW('T', 220, int)
+#define TUNSETVNETBE _IOW('T', 222, int)
+
+/* TUNSETIFF ifr flags */
+#define IFF_TAP          0x0002
+#define IFF_NO_PI        0x1000
+#define IFF_ONE_QUEUE    0x2000
+#define IFF_VNET_HDR     0x4000
+#define IFF_MULTI_QUEUE  0x0100
+#define IFF_ATTACH_QUEUE 0x0200
+#define IFF_DETACH_QUEUE 0x0400
+
+/* Features for GSO (TUNSETOFFLOAD). */
+#define TUN_F_CSUM	0x01	/* You can hand me unchecksummed packets. */
+#define TUN_F_TSO4	0x02	/* I can handle TSO for IPv4 packets */
+#define TUN_F_TSO6	0x04	/* I can handle TSO for IPv6 packets */
+#define TUN_F_TSO_ECN	0x08	/* I can handle TSO with ECN bits. */
+#define TUN_F_UFO	0x10	/* I can handle UFO packets */
+
+enum {
+	VHOST_MSG_SET_OWNER,
+	VHOST_MSG_RESET_OWNER,
+	VHOST_MSG_SET_FEATURES,
+	VHOST_MSG_GET_FEATURES,
+	VHOST_MSG_SET_VRING_CALL,
+	VHOST_MSG_SET_VRING_NUM,
+	VHOST_MSG_SET_VRING_BASE,
+	VHOST_MSG_GET_VRING_BASE,
+	VHOST_MSG_SET_VRING_ADDR,
+	VHOST_MSG_SET_VRING_KICK,
+	VHOST_MSG_SET_MEM_TABLE,
+	VHOST_MSG_MAX,
+};
+
+#define VHOST_KERNEL	0
+#define VHOST_USER	1
+
+int vhost_user_sock(int vhostfd, uint64_t req, void *arg);
+int vhost_user_setup(const char *path);
+
+int vhost_kernel_ioctl(int vhostfd, uint64_t req, void *arg);
+int vhost_kernel_setup(const char *path, const char *ifname, int *p_tapfd);
+int vhost_kernel_post(int vhostfd, int tapfd, uint64_t features, uint32_t nvqs);
+
+int vhost_call(int vhostfd, int type, uint64_t req, void *arg);
+
+#endif
diff --git a/drivers/net/virtio/virtio_user/vhost_kernel.c b/drivers/net/virtio/virtio_user/vhost_kernel.c
new file mode 100644
index 0000000..7604f04
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost_kernel.c
@@ -0,0 +1,254 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <sys/ioctl.h>
+#include <net/if.h>
+#include <string.h>
+#include <errno.h>
+
+#include <rte_memory.h>
+#include <rte_eal_memconfig.h>
+
+#include "vhost.h"
+
+#define TUN_DEF_SNDBUF	(1ull << 20)
+#define PATH_NET_TUN	"/dev/net/tun"
+
+/** Merge those virtually adjacent memsegs into one region.
+ */
+static void
+prepare_vhost_memory_kernel(struct vhost_memory_kernel **p_vm)
+{
+	uint32_t i, j, k = 0;
+	struct rte_memseg *seg;
+	struct vhost_memory_region *mr;
+	struct vhost_memory_kernel *vm;
+
+	vm = malloc(sizeof(struct vhost_memory_kernel) +
+		    RTE_MAX_MEMSEG * sizeof(struct vhost_memory_region));
+
+	for (i = 0; i < RTE_MAX_MEMSEG; ++i) {
+		seg = &rte_eal_get_configuration()->mem_config->memseg[i];
+		if (!seg->addr)
+			break;
+
+		int new_region = 1;
+
+		for (j = 0; j < k; ++j) {
+			mr = &vm->regions[j];
+
+			if (mr->userspace_addr + mr->memory_size ==
+			    (uintptr_t)seg->addr) {
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+
+			if ((uintptr_t)seg->addr + seg->len ==
+			    mr->userspace_addr) {
+				mr->guest_phys_addr = (uintptr_t)seg->addr;
+				mr->userspace_addr = (uintptr_t)seg->addr;
+				mr->memory_size += seg->len;
+				new_region = 0;
+				break;
+			}
+		}
+
+		if (new_region == 0)
+			continue;
+
+		mr = &vm->regions[k++];
+		mr->guest_phys_addr = (uintptr_t)seg->addr; /* use vaddr here! */
+		mr->userspace_addr = (uintptr_t)seg->addr;
+		mr->memory_size = seg->len;
+		mr->mmap_offset = 0;
+	}
+
+	vm->nregions = k;
+	vm->padding = 0;
+	*p_vm = vm;
+}
+
+int
+vhost_kernel_ioctl(int vhostfd, uint64_t req, void *arg)
+{
+	int ret;
+	struct vhost_memory_kernel *vm = NULL;
+
+	if (req == VHOST_SET_MEM_TABLE) {
+		prepare_vhost_memory_kernel(&vm);
+		arg = (void *)vm;
+	}
+
+	ret = ioctl(vhostfd, req, arg);
+
+	if (req == VHOST_SET_MEM_TABLE)
+		free(vm);
+
+	return ret;
+}
+
+/**
+ * Set up environment to talk with a vhost kernel backend.
+ * @param path
+ *   - The path to vhost net (kernel) character file.
+ *
+ * @param ifname
+ *   - Specify the tap device name if any, or NULL.
+ *
+ * @param p_tapfd
+ *   - Pointer to store the fd of tap device.
+ *
+ * @return
+ *   - (-1) if fail to set up;
+ *   - (>=0) if successful, and it is the fd to vhostfd.
+ */
+int
+vhost_kernel_setup(const char *path, const char *ifname, int *p_tapfd)
+{
+	int vhostfd, tapfd;
+	int len = sizeof(struct virtio_net_hdr_mrg_rxbuf);
+	int req_mq = 0;
+	int sndbuf = TUN_DEF_SNDBUF;
+	unsigned int features;
+	struct ifreq ifr;
+
+	/* TODO:
+	 * 1. get/set offload capability, tap_probe_has_ufo, tap_fd_set_offload
+	 * 2. verify we can get/set vnet_hdr_len, tap_probe_vnet_hdr_len
+	 * 3. get number of memory regions from vhost module parameter
+	 * max_mem_regions, supported in newer version linux kernel
+	 */
+
+	tapfd = open(PATH_NET_TUN, O_RDWR);
+	if (tapfd < 0) {
+		PMD_DRV_LOG(ERR, "fail to open %s: %s",
+			    PATH_NET_TUN, strerror(errno));
+		return -1;
+	}
+
+	memset(&ifr, 0, sizeof(ifr));
+	ifr.ifr_flags = IFF_TAP | IFF_NO_PI;
+
+	if (ioctl(tapfd, TUNGETFEATURES, &features) == -1) {
+		PMD_DRV_LOG(ERR, "TUNGETFEATURES failed: %s", strerror(errno));
+		goto error;
+	}
+
+	if (features & IFF_ONE_QUEUE)
+		ifr.ifr_flags |= IFF_ONE_QUEUE;
+
+	if (features & IFF_VNET_HDR)
+		ifr.ifr_flags |= IFF_VNET_HDR;
+	else {
+		PMD_DRV_LOG(ERR, "vnet_hdr not supported by kernel");
+		goto error;
+	}
+
+	if (req_mq) {
+		if (features & IFF_MULTI_QUEUE)
+			ifr.ifr_flags |= IFF_MULTI_QUEUE;
+		else {
+			PMD_DRV_LOG(ERR, "multiqueue not supported by kernel");
+			goto error;
+		}
+	}
+
+	if (ifname)
+		strncpy(ifr.ifr_name, ifname, IFNAMSIZ);
+	else
+		strncpy(ifr.ifr_name, "tap%d", IFNAMSIZ);
+	if (ioctl(tapfd, TUNSETIFF, (void *)&ifr) == -1) {
+		PMD_DRV_LOG(ERR, "TUNSETIFF failed: %s", strerror(errno));
+		goto error;
+	}
+	fcntl(tapfd, F_SETFL, O_NONBLOCK);
+
+	if (ioctl(tapfd, TUNSETVNETHDRSZ, &len) < 0) {
+		PMD_DRV_LOG(ERR, "TUNSETVNETHDRSZ failed: %s", strerror(errno));
+		goto error;
+	}
+
+	if (ioctl(tapfd, TUNSETSNDBUF, &sndbuf) < 0) {
+		PMD_DRV_LOG(ERR, "TUNSETSNDBUF failed: %s", strerror(errno));
+		goto error;
+	}
+
+	vhostfd = open(path, O_RDWR);
+	if (vhostfd < 0) {
+		PMD_DRV_LOG(ERR, "fail to open %s, %s", path, strerror(errno));
+		goto error;
+	}
+
+	*p_tapfd = tapfd;
+	return vhostfd;
+
+error:
+	close(tapfd);
+	return -1;
+}
+
+int
+vhost_kernel_post(int vhostfd, int tapfd, uint64_t features, uint32_t nvqs)
+{
+	struct vhost_vring_file f;
+	uint16_t hdr_size;
+	int ret;
+
+	if ((features & (1ULL << VIRTIO_NET_F_MRG_RXBUF)) ||
+	    (features & (1ULL << VIRTIO_F_VERSION_1)))
+		hdr_size = sizeof(struct virtio_net_hdr_mrg_rxbuf);
+	else
+		hdr_size = sizeof(struct virtio_net_hdr);
+
+	if (ioctl(tapfd, TUNSETVNETHDRSZ, &hdr_size) == -1) {
+		PMD_DRV_LOG(ERR, "TUNSETVNETHDRSZ fails, %s", strerror(errno));
+		return -1;
+	}
+
+	f.fd = tapfd;
+	for (f.index = 0; f.index < nvqs; ++f.index) {
+		ret = vhost_kernel_ioctl(vhostfd, VHOST_NET_SET_BACKEND, &f);
+		if (ret < 0) {
+			PMD_DRV_LOG(ERR, "VHOST_NET_SET_BACKEND fails, %s",
+				    strerror(errno));
+			return -1;
+		}
+	}
+
+	return 0;
+}
diff --git a/drivers/net/virtio/virtio_user/vhost_user.c b/drivers/net/virtio/virtio_user/vhost_user.c
new file mode 100644
index 0000000..b0a84cf
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost_user.c
@@ -0,0 +1,378 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <unistd.h>
+#include <fcntl.h>
+#include <sys/un.h>
+#include <string.h>
+#include <errno.h>
+
+#include "vhost.h"
+
+static int
+vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
+{
+	int r;
+	struct msghdr msgh;
+	struct iovec iov;
+	size_t fd_size = fd_num * sizeof(int);
+	char control[CMSG_SPACE(fd_size)];
+	struct cmsghdr *cmsg;
+
+	memset(&msgh, 0, sizeof(msgh));
+	memset(control, 0, sizeof(control));
+
+	iov.iov_base = (uint8_t *)buf;
+	iov.iov_len = len;
+
+	msgh.msg_iov = &iov;
+	msgh.msg_iovlen = 1;
+	msgh.msg_control = control;
+	msgh.msg_controllen = sizeof(control);
+
+	cmsg = CMSG_FIRSTHDR(&msgh);
+	cmsg->cmsg_len = CMSG_LEN(fd_size);
+	cmsg->cmsg_level = SOL_SOCKET;
+	cmsg->cmsg_type = SCM_RIGHTS;
+	memcpy(CMSG_DATA(cmsg), fds, fd_size);
+
+	do {
+		r = sendmsg(fd, &msgh, 0);
+	} while (r < 0 && errno == EINTR);
+
+	return r;
+}
+
+static int
+vhost_user_read(int fd, struct vhost_user_msg *msg)
+{
+	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
+	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
+
+	ret = recv(fd, (void *)msg, sz_hdr, 0);
+	if (ret < sz_hdr) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
+			    ret, sz_hdr);
+		goto fail;
+	}
+
+	/* validate msg flags */
+	if (msg->flags != (valid_flags)) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg: flags %x instead of %x.",
+			    msg->flags, valid_flags);
+		goto fail;
+	}
+
+	sz_payload = msg->size;
+	if (sz_payload) {
+		ret = recv(fd, (void *)((char *)msg + sz_hdr), sz_payload, 0);
+		if (ret < sz_payload) {
+			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
+				    ret, msg->size);
+			goto fail;
+		}
+	}
+
+	return 0;
+
+fail:
+	return -1;
+}
+
+struct hugepage_file_info {
+	uint64_t addr;            /**< virtual addr */
+	size_t   size;            /**< the file size */
+	char     path[PATH_MAX];  /**< path to backing file */
+};
+
+/* Two possible options:
+ * 1. Match HUGEPAGE_INFO_FMT to find the file storing struct hugepage_file
+ * array. This is simple but cannot be used in secondary process because
+ * secondary process will close and munmap that file.
+ * 2. Match HUGEFILE_FMT to find hugepage files directly.
+ *
+ * We choose option 2.
+ */
+static int
+get_hugepage_file_info(struct hugepage_file_info huges[], int max)
+{
+	int idx;
+	FILE *f;
+	char buf[BUFSIZ], *tmp, *tail;
+	char *str_underline, *str_start;
+	int huge_index;
+	uint64_t v_start, v_end;
+
+	f = fopen("/proc/self/maps", "r");
+	if (!f) {
+		PMD_DRV_LOG(ERR, "cannot open /proc/self/maps");
+		return -1;
+	}
+
+	idx = 0;
+	while (fgets(buf, sizeof(buf), f) != NULL) {
+		if (sscanf(buf, "%" PRIx64 "-%" PRIx64, &v_start, &v_end) < 2) {
+			PMD_DRV_LOG(ERR, "Failed to parse address");
+			goto error;
+		}
+
+		tmp = strchr(buf, ' ') + 1; /** skip address */
+		tmp = strchr(tmp, ' ') + 1; /** skip perm */
+		tmp = strchr(tmp, ' ') + 1; /** skip offset */
+		tmp = strchr(tmp, ' ') + 1; /** skip dev */
+		tmp = strchr(tmp, ' ') + 1; /** skip inode */
+		while (*tmp == ' ')         /** skip spaces */
+			tmp++;
+		tail = strrchr(tmp, '\n');  /** remove newline if exists */
+		if (tail)
+			*tail = '\0';
+
+		/* Match HUGEFILE_FMT, aka "%s/%smap_%d",
+		 * which is defined in eal_filesystem.h
+		 */
+		str_underline = strrchr(tmp, '_');
+		if (!str_underline)
+			continue;
+
+		str_start = str_underline - strlen("map");
+		if (str_start < tmp)
+			continue;
+
+		if (sscanf(str_start, "map_%d", &huge_index) != 1)
+			continue;
+
+		if (idx >= max) {
+			PMD_DRV_LOG(ERR, "Exceed maximum of %d", max);
+			goto error;
+		}
+		huges[idx].addr = v_start;
+		huges[idx].size = v_end - v_start;
+		strcpy(huges[idx].path, tmp);
+		idx++;
+	}
+
+	fclose(f);
+	return idx;
+
+error:
+	fclose(f);
+	return -1;
+}
+
+static int
+prepare_vhost_memory_user(struct vhost_user_msg *msg, int fds[])
+{
+	int i, num;
+	struct hugepage_file_info huges[VHOST_MEMORY_MAX_NREGIONS];
+	struct vhost_memory_region *mr;
+
+	num = get_hugepage_file_info(huges, VHOST_MEMORY_MAX_NREGIONS);
+	if (num < 0) {
+		PMD_INIT_LOG(ERR, "Failed to prepare memory for vhost-user");
+		return -1;
+	}
+
+	for (i = 0; i < num; ++i) {
+		mr = &msg->payload.memory.regions[i];
+		mr->guest_phys_addr = huges[i].addr; /* use vaddr! */
+		mr->userspace_addr = huges[i].addr;
+		mr->memory_size = huges[i].size;
+		mr->mmap_offset = 0;
+		fds[i] = open(huges[i].path, O_RDWR);
+	}
+
+	msg->payload.memory.nregions = num;
+	msg->payload.memory.padding = 0;
+
+	return 0;
+}
+
+static struct vhost_user_msg m;
+
+int
+vhost_user_sock(int vhostfd, uint64_t req, void *arg)
+{
+	struct vhost_user_msg msg;
+	struct vhost_vring_file *file = 0;
+	int need_reply = 0;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+	int fd_num = 0;
+	int i, len;
+
+	msg.request = req;
+	msg.flags = VHOST_USER_VERSION;
+	msg.size = 0;
+
+	switch (req) {
+	case VHOST_USER_GET_FEATURES:
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_FEATURES:
+	case VHOST_USER_SET_LOG_BASE:
+		msg.payload.u64 = *((__u64 *)arg);
+		msg.size = sizeof(m.payload.u64);
+		break;
+
+	case VHOST_USER_SET_OWNER:
+	case VHOST_USER_RESET_OWNER:
+		break;
+
+	case VHOST_USER_SET_MEM_TABLE:
+		if (prepare_vhost_memory_user(&msg, fds) < 0)
+			return -1;
+		fd_num = msg.payload.memory.nregions;
+		msg.size = sizeof(m.payload.memory.nregions);
+		msg.size += sizeof(m.payload.memory.padding);
+		msg.size += fd_num * sizeof(struct vhost_memory_region);
+		break;
+
+	case VHOST_USER_SET_LOG_FD:
+		fds[fd_num++] = *((int *)arg);
+		break;
+
+	case VHOST_USER_SET_VRING_NUM:
+	case VHOST_USER_SET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		break;
+
+	case VHOST_USER_GET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_VRING_ADDR:
+		memcpy(&msg.payload.addr, arg, sizeof(msg.payload.addr));
+		msg.size = sizeof(m.payload.addr);
+		break;
+
+	case VHOST_USER_SET_VRING_KICK:
+	case VHOST_USER_SET_VRING_CALL:
+	case VHOST_USER_SET_VRING_ERR:
+		file = arg;
+		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
+		msg.size = sizeof(m.payload.u64);
+		if (file->fd > 0)
+			fds[fd_num++] = file->fd;
+		else
+			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
+		break;
+
+	default:
+		PMD_DRV_LOG(ERR, "trying to send unhandled msg type");
+		return -1;
+	}
+
+	len = VHOST_USER_HDR_SIZE + msg.size;
+	if (vhost_user_write(vhostfd, &msg, len, fds, fd_num) < 0)
+		return 0;
+
+	if (req == VHOST_USER_SET_MEM_TABLE)
+		for (i = 0; i < fd_num; ++i)
+			close(fds[i]);
+
+	if (need_reply) {
+		if (vhost_user_read(vhostfd, &msg) < 0)
+			return -1;
+
+		if (req != msg.request) {
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+
+		switch (req) {
+		case VHOST_USER_GET_FEATURES:
+			if (msg.size != sizeof(m.payload.u64)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			*((__u64 *)arg) = msg.payload.u64;
+			break;
+		case VHOST_USER_GET_VRING_BASE:
+			if (msg.size != sizeof(m.payload.state)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size.");
+				return -1;
+			}
+			memcpy(arg, &msg.payload.state,
+			       sizeof(struct vhost_vring_state));
+			break;
+		default:
+			PMD_DRV_LOG(ERR, "Received unexpected msg type.");
+			return -1;
+		}
+	}
+
+	return 0;
+}
+
+/**
+ * Set up environment to talk with a vhost user backend.
+ * @param path
+ *   - The path to vhost user unix socket file.
+ *
+ * @return
+ *   - (-1) if fail to set up;
+ *   - (>=0) if successful, and it is the fd to vhostfd.
+ */
+int
+vhost_user_setup(const char *path)
+{
+	int fd;
+	int flag;
+	struct sockaddr_un un;
+
+	fd = socket(AF_UNIX, SOCK_STREAM, 0);
+	if (fd < 0) {
+		PMD_DRV_LOG(ERR, "socket() error, %s", strerror(errno));
+		return -1;
+	}
+
+	flag = fcntl(fd, F_GETFD);
+	fcntl(fd, F_SETFD, flag | FD_CLOEXEC);
+
+	memset(&un, 0, sizeof(un));
+	un.sun_family = AF_UNIX;
+	snprintf(un.sun_path, sizeof(un.sun_path), "%s", path);
+	if (connect(fd, (struct sockaddr *)&un, sizeof(un)) < 0) {
+		PMD_DRV_LOG(ERR, "connect error, %s", strerror(errno));
+		close(fd);
+		return -1;
+	}
+
+	return fd;
+}
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v7 4/6] virtio-user: add device emulation layer APIs
  2016-06-12  0:35 ` [PATCH v7 0/6] virtio support for container Jianfeng Tan
                     ` (2 preceding siblings ...)
  2016-06-12  0:35   ` [PATCH v7 3/6] virtio-user: add vhost adapter layer Jianfeng Tan
@ 2016-06-12  0:35   ` Jianfeng Tan
  2016-06-12  0:35   ` [PATCH v7 5/6] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
  2016-06-12  0:35   ` [PATCH v7 6/6] virtio-user: add a new vdev named virtio-user Jianfeng Tan
  5 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-12  0:35 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Two device emulation layer APIs are added for virtio driver to call:
  - virtio_user_start_device()
  - virtio_user_stop_device()
  - virtio_user_dev_init()
  - virtio_user_dev_uninit()

These APIs will get called by virtio driver, and they call vhost adapter
layer APIs to implement the functionality. Besides, this patch defines
a struct named virtio_user_dev to help manage the data stands for this
kind of virtual device.

  ----------------------
  | ------------------ |
  | | virtio driver  | |
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate |-|----> (virtio_user_dev.c, virtio_user_dev.h)
  | |                | |
  | | vhost adapter  | |
  | ------------------ |
  ----------------------
            |
            |
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/Makefile                      |   1 +
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 263 +++++++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.h |  64 ++++++
 3 files changed, 328 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h

diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index 892c2ae..88a634a 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -59,6 +59,7 @@ ifeq ($(CONFIG_RTE_VIRTIO_USER),y)
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_kernel.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_dev.c
 endif
 
 # this lib depends upon:
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.c b/drivers/net/virtio/virtio_user/virtio_user_dev.c
new file mode 100644
index 0000000..b4e53a8
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.c
@@ -0,0 +1,263 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <stdio.h>
+#include <sys/stat.h>
+#include <fcntl.h>
+#include <string.h>
+#include <errno.h>
+#include <sys/mman.h>
+#include <unistd.h>
+#include <sys/eventfd.h>
+
+#include "vhost.h"
+#include "virtio_user_dev.h"
+#include "../virtio_ethdev.h"
+
+static int
+virtio_user_kick_queue(struct virtio_user_dev *dev, uint32_t queue_sel)
+{
+	int callfd, kickfd;
+	struct vhost_vring_file file;
+	struct vhost_vring_state state;
+	struct vring *vring = &dev->vrings[queue_sel];
+	struct vhost_vring_addr addr = {
+		.index = queue_sel,
+		.desc_user_addr = (uint64_t)(uintptr_t)vring->desc,
+		.avail_user_addr = (uint64_t)(uintptr_t)vring->avail,
+		.used_user_addr = (uint64_t)(uintptr_t)vring->used,
+		.log_guest_addr = 0,
+		.flags = 0, /* disable log */
+	};
+
+	/* May use invalid flag, but some backend leverages kickfd and callfd as
+	 * criteria to judge if dev is alive. so finally we use real event_fd.
+	 */
+	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (callfd < 0) {
+		PMD_DRV_LOG(ERR, "callfd error, %s\n", strerror(errno));
+		return -1;
+	}
+	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (kickfd < 0) {
+		close(callfd);
+		PMD_DRV_LOG(ERR, "kickfd error, %s\n", strerror(errno));
+		return -1;
+	}
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL come
+	 * firstly because vhost depends on this msg to allocate virtqueue
+	 * pair.
+	 */
+	file.index = queue_sel;
+	file.fd = callfd;
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_CALL, &file);
+	dev->callfds[queue_sel] = callfd;
+
+	state.index = queue_sel;
+	state.num = vring->num;
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_NUM, &state);
+
+	state.num = 0; /* no reservation */
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_BASE, &state);
+
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_ADDR, &addr);
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_KICK comes
+	 * lastly because vhost depends on this msg to judge if
+	 * virtio is ready.
+	 */
+	file.fd = kickfd;
+	vhost_call(dev->vhostfd, dev->type, VHOST_MSG_SET_VRING_KICK, &file);
+	dev->kickfds[queue_sel] = kickfd;
+
+	return 0;
+}
+
+int
+virtio_user_start_device(struct virtio_user_dev *dev)
+{
+	uint64_t features;
+	uint32_t i, queue_sel;
+	int ret;
+
+	/* construct memory region inside each implementation */
+	ret = vhost_call(dev->vhostfd, dev->type,
+			 VHOST_MSG_SET_MEM_TABLE, NULL);
+	if (ret < 0)
+		goto error;
+
+	for (i = 0; i < dev->queue_pairs; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
+		if (virtio_user_kick_queue(dev, queue_sel) < 0) {
+			PMD_DRV_LOG(INFO, "kick rx vq fails: %u", i);
+			goto error;
+		}
+	}
+	for (i = 0; i < dev->queue_pairs; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
+		if (virtio_user_kick_queue(dev, queue_sel) < 0) {
+			PMD_DRV_LOG(INFO, "kick tx vq fails: %u", i);
+			goto error;
+		}
+	}
+
+	/* After setup all virtqueues, we need to set_features so that
+	 * these features can be set into each virtqueue in vhost side.
+	 * And before that, make sure VIRTIO_NET_F_MAC is stripped.
+	 */
+	features = dev->features;
+	features &= ~(1ull << VIRTIO_NET_F_MAC);
+	ret = vhost_call(dev->vhostfd, dev->type,
+			 VHOST_MSG_SET_FEATURES, &features);
+	if (ret < 0)
+		goto error;
+	PMD_DRV_LOG(INFO, "set features: %" PRIx64, features);
+
+	if (dev->type == VHOST_USER)
+		return 0;
+
+	/* It is tricky here, but so far we have to do that */
+	ret = vhost_kernel_post(dev->vhostfd, dev->tapfd, features,
+				dev->queue_pairs * 2);
+	if (ret == 0)
+		return 0;
+error:
+	/* TODO: free resource here or caller to check */
+	return -1;
+}
+
+int virtio_user_stop_device(struct virtio_user_dev *dev)
+{
+	return vhost_call(dev->vhostfd, dev->type, VHOST_MSG_RESET_OWNER, NULL);
+}
+
+static inline void
+parse_mac(struct virtio_user_dev *dev, const char *mac)
+{
+	int i, r;
+	uint32_t tmp[ETHER_ADDR_LEN];
+
+	if (!mac)
+		return;
+
+	r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
+			&tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
+	if (r == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			dev->mac_addr[i] = (uint8_t)tmp[i];
+		dev->mac_specified = 1;
+	} else {
+		/* ignore the wrong mac, use random mac */
+		PMD_DRV_LOG(ERR, "wrong format of mac: %s", mac);
+	}
+}
+
+int
+virtio_user_dev_init(struct virtio_user_dev *dev, char *path, int queues,
+		     int cq, int queue_size, const char *mac, char *ifname)
+{
+	struct stat s;
+
+	strncpy(dev->path, path, PATH_MAX);
+	dev->max_queue_pairs = queues;
+	dev->queue_pairs = 1; /* mq disabled by default */
+	dev->queue_size = queue_size;
+	dev->mac_specified = 0;
+	parse_mac(dev, mac);
+	dev->vhostfd = -1;
+	dev->tapfd = -1;
+	/* TODO: cq */
+	RTE_SET_USED(cq);
+
+	if (stat(dev->path, &s) < 0) {
+		PMD_INIT_LOG(ERR, "stat: %s failed, %s", dev->path,
+			     strerror(errno));
+		return -1;
+	}
+
+	switch (s.st_mode & S_IFMT) {
+	case S_IFCHR:
+		dev->type = VHOST_KERNEL;
+		dev->vhostfd = vhost_kernel_setup(dev->path, ifname,
+						  &dev->tapfd);
+		break;
+	case S_IFSOCK:
+		dev->type = VHOST_USER;
+		dev->vhostfd = vhost_user_setup(dev->path);
+		break;
+	default:
+		PMD_INIT_LOG(ERR, "unknown file type of %s", dev->path);
+		return -1;
+	}
+	if (dev->vhostfd < 0) {
+		PMD_INIT_LOG(ERR, "backend set up fails");
+		return -1;
+	}
+	if (vhost_call(dev->vhostfd, dev->type,
+			VHOST_MSG_SET_OWNER, NULL) < 0) {
+		PMD_INIT_LOG(ERR, "set_owner fails: %s", strerror(errno));
+		return -1;
+	}
+
+	if (vhost_call(dev->vhostfd, dev->type,
+			VHOST_MSG_GET_FEATURES, &dev->features) < 0) {
+		PMD_INIT_LOG(ERR, "get_features failed: %s", strerror(errno));
+		return -1;
+	}
+	if (dev->mac_specified)
+		dev->features |= (1ull << VIRTIO_NET_F_MAC);
+	/* disable it until we support CQ */
+	dev->features &= ~(1ull << VIRTIO_NET_F_CTRL_VQ);
+	dev->features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
+
+	return 0;
+
+}
+
+void
+virtio_user_dev_uninit(struct virtio_user_dev *dev)
+{
+	uint32_t i;
+
+	if (dev->type == VHOST_KERNEL)
+		close(dev->tapfd);
+
+	for (i = 0; i < dev->queue_pairs * 2; ++i) {
+		close(dev->callfds[i]);
+		close(dev->kickfds[i]);
+	}
+
+	close(dev->vhostfd);
+}
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.h b/drivers/net/virtio/virtio_user/virtio_user_dev.h
new file mode 100644
index 0000000..9987b70
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.h
@@ -0,0 +1,64 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VIRTIO_USER_DEV_H
+#define _VIRTIO_USER_DEV_H
+
+#include <limits.h>
+#include "../virtio_pci.h"
+#include "../virtio_ring.h"
+
+struct virtio_user_dev {
+	int		type; /* VHOST_KERNEL or VHOST_USER */
+	int		vhostfd;
+	int		tapfd; /* only used in vhost kernel */
+	int		callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		mac_specified;
+	uint32_t	max_queue_pairs;
+	uint32_t	queue_pairs;
+	uint32_t	queue_size;
+	uint64_t	features;
+	uint8_t		status;
+	uint8_t		mac_addr[ETHER_ADDR_LEN];
+	char		path[PATH_MAX];
+	struct vring	vrings[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+};
+
+int virtio_user_start_device(struct virtio_user_dev *dev);
+int virtio_user_stop_device(struct virtio_user_dev *dev);
+int virtio_user_dev_init(struct virtio_user_dev *dev, char *path, int queues,
+			 int cq, int queue_size, const char *mac, char *ifname);
+void virtio_user_dev_uninit(struct virtio_user_dev *dev);
+
+#endif
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v7 5/6] virtio-user: add new virtual pci driver for virtio
  2016-06-12  0:35 ` [PATCH v7 0/6] virtio support for container Jianfeng Tan
                     ` (3 preceding siblings ...)
  2016-06-12  0:35   ` [PATCH v7 4/6] virtio-user: add device emulation layer APIs Jianfeng Tan
@ 2016-06-12  0:35   ` Jianfeng Tan
  2016-06-12  0:35   ` [PATCH v7 6/6] virtio-user: add a new vdev named virtio-user Jianfeng Tan
  5 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-12  0:35 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch implements another new instance of struct virtio_pci_ops to
drive the virtio-user virtual device. Instead of rd/wr ioport or PCI
configuration space, this virtual pci driver will rd/wr the virtual
device struct virtio_user_hw, and when necessary, invokes APIs provided
by device emulation later to start/stop the device.

  ----------------------
  | ------------------ |
  | | virtio driver  | |----> (virtio_user_ethdev.c)
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate | |
  | |                | |
  | | vhost adapter  | |
  | ------------------ |
  ----------------------
            |
            |
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/Makefile             |   1 +
 drivers/net/virtio/virtio_pci.h         |   1 +
 drivers/net/virtio/virtio_user_ethdev.c | 218 ++++++++++++++++++++++++++++++++
 3 files changed, 220 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user_ethdev.c

diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index 88a634a..459260b 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -60,6 +60,7 @@ SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_kernel.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_dev.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user_ethdev.c
 endif
 
 # this lib depends upon:
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index 6c7f8d7..dd7693f 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -261,6 +261,7 @@ struct virtio_hw {
 	struct virtio_pci_common_cfg *common_cfg;
 	struct virtio_net_config *dev_cfg;
 	const struct virtio_pci_ops *vtpci_ops;
+	void	    *virtio_user_dev;
 };
 
 /*
diff --git a/drivers/net/virtio/virtio_user_ethdev.c b/drivers/net/virtio/virtio_user_ethdev.c
new file mode 100644
index 0000000..e390242
--- /dev/null
+++ b/drivers/net/virtio/virtio_user_ethdev.c
@@ -0,0 +1,218 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <sys/types.h>
+#include <unistd.h>
+
+#include "virtio_logs.h"
+#include "virtio_pci.h"
+#include "virtqueue.h"
+#include "virtio_user/virtio_user_dev.h"
+
+#define virtio_user_get_dev(hw) \
+	((struct virtio_user_dev *)(hw)->virtio_user_dev);
+
+static void
+virtio_user_read_dev_config(struct virtio_hw *hw, uint64_t offset,
+		     void *dst, int length)
+{
+	int i;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (offset == offsetof(struct virtio_net_config, mac) &&
+	    length == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			((uint8_t *)dst)[i] = dev->mac_addr[i];
+		return;
+	}
+
+	if (offset == offsetof(struct virtio_net_config, status))
+		*(uint16_t *)dst = dev->status;
+
+	if (offset == offsetof(struct virtio_net_config, max_virtqueue_pairs))
+		*(uint16_t *)dst = dev->max_queue_pairs;
+}
+
+static void
+virtio_user_write_dev_config(struct virtio_hw *hw, uint64_t offset,
+		      const void *src, int length)
+{
+	int i;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if ((offset == offsetof(struct virtio_net_config, mac)) &&
+	    (length == ETHER_ADDR_LEN))
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			dev->mac_addr[i] = ((const uint8_t *)src)[i];
+	else
+		PMD_DRV_LOG(ERR, "not supported offset=%" PRIu64 ", len=%d\n",
+			    offset, length);
+}
+
+static void
+virtio_user_set_status(struct virtio_hw *hw, uint8_t status)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (status & VIRTIO_CONFIG_STATUS_DRIVER_OK)
+		virtio_user_start_device(dev);
+	dev->status = status;
+}
+
+static void
+virtio_user_reset(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	virtio_user_stop_device(dev);
+}
+
+static uint8_t
+virtio_user_get_status(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	return dev->status;
+}
+
+static uint64_t
+virtio_user_get_features(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	return dev->features;
+}
+
+static void
+virtio_user_set_features(struct virtio_hw *hw, uint64_t features)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	dev->features = features;
+}
+
+static uint8_t
+virtio_user_get_isr(struct virtio_hw *hw __rte_unused)
+{
+	/* When config interrupt happens, driver calls this function to query
+	 * what kinds of change happen. Interrupt mode not supported for now.
+	 */
+	return 0;
+}
+
+static uint16_t
+virtio_user_set_config_irq(struct virtio_hw *hw __rte_unused,
+		    uint16_t vec __rte_unused)
+{
+	return VIRTIO_MSI_NO_VECTOR;
+}
+
+/* This function is to get the queue size, aka, number of descs, of a specified
+ * queue. Different with the VHOST_USER_GET_QUEUE_NUM, which is used to get the
+ * max supported queues.
+ */
+static uint16_t
+virtio_user_get_queue_num(struct virtio_hw *hw, uint16_t queue_id __rte_unused)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	/* Currently, each queue has same queue size */
+	return dev->queue_size;
+}
+
+static int
+virtio_user_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+	uint16_t queue_idx = vq->vq_queue_index;
+	uint64_t desc_addr, avail_addr, used_addr;
+
+	desc_addr = (uintptr_t)vq->vq_ring_virt_mem;
+	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
+	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
+							 ring[vq->vq_nentries]),
+				   VIRTIO_PCI_VRING_ALIGN);
+
+	dev->vrings[queue_idx].num = vq->vq_nentries;
+	dev->vrings[queue_idx].desc = (void *)desc_addr;
+	dev->vrings[queue_idx].avail = (void *)avail_addr;
+	dev->vrings[queue_idx].used = (void *)used_addr;
+
+	return 0;
+}
+
+static void
+virtio_user_del_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	/* For legacy devices, write 0 to VIRTIO_PCI_QUEUE_PFN port, QEMU
+	 * correspondingly stops the ioeventfds, and reset the status of
+	 * the device.
+	 * For modern devices, set queue desc, avail, used in PCI bar to 0,
+	 * not see any more behavior in QEMU.
+	 *
+	 * Here we just care about what information to deliver to vhost-user
+	 * or vhost-kernel. So we just close ioeventfd for now.
+	 */
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	close(dev->callfds[vq->vq_queue_index]);
+	close(dev->kickfds[vq->vq_queue_index]);
+}
+
+static void
+virtio_user_notify_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	uint64_t buf = 1;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (write(dev->kickfds[vq->vq_queue_index], &buf, sizeof(buf)) < 0)
+		PMD_DRV_LOG(ERR, "failed to kick backend: %s\n",
+			    strerror(errno));
+}
+
+static const struct virtio_pci_ops virtio_user_ops = {
+	.read_dev_cfg	= virtio_user_read_dev_config,
+	.write_dev_cfg	= virtio_user_write_dev_config,
+	.reset		= virtio_user_reset,
+	.get_status	= virtio_user_get_status,
+	.set_status	= virtio_user_set_status,
+	.get_features	= virtio_user_get_features,
+	.set_features	= virtio_user_set_features,
+	.get_isr	= virtio_user_get_isr,
+	.set_config_irq	= virtio_user_set_config_irq,
+	.get_queue_num	= virtio_user_get_queue_num,
+	.setup_queue	= virtio_user_setup_queue,
+	.del_queue	= virtio_user_del_queue,
+	.notify_queue	= virtio_user_notify_queue,
+};
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v7 6/6] virtio-user: add a new vdev named virtio-user
  2016-06-12  0:35 ` [PATCH v7 0/6] virtio support for container Jianfeng Tan
                     ` (4 preceding siblings ...)
  2016-06-12  0:35   ` [PATCH v7 5/6] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
@ 2016-06-12  0:35   ` Jianfeng Tan
  5 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-12  0:35 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Add a new virtual device named vhost-user, which can be used just like
eth_ring, eth_null, etc. To reuse the code of original virtio, we do
some adjustment in virtio_ethdev.c, such as remove key _static_ of
eth_virtio_dev_init() so that it can be reused in virtual device; and
we add some check to make sure it will not crash.

Configured parameters include:
  - queues (optional, 1 by default), number of queue pairs, multi-queue
    not supported for now.
  - cq (optional, 0 by default), not supported for now.
  - mac (optional), random value will be given if not specified.
  - queue_size (optional, 256 by default), size of virtqueues.
  - path (madatory), path of vhost, depends on the file type, vhost
    user if the given path points to a unix socket; vhost-net if the
    given path points to a char device.
  - ifname (optional), specify the name of backend tap device; only
    valid when backend is vhost-net.

When enable CONFIG_RTE_VIRTIO_USER (enabled by default), the compiled
library can be used in both VM and container environment.

Examples:
path_vhost=/dev/vhost-net # use vhost-net as a backend
path_vhost=<path_to_vhost_user> # use vhost-user as a backend

sudo ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
    --socket-mem 0,1024 --no-pci --file-prefix=l2fwd \
    --vdev=virtio-user0,mac=00:01:02:03:04:05,path=$path_vhost -- -p 0x1

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 doc/guides/rel_notes/release_16_07.rst  |  11 ++
 doc/guides/sample_app_ug/vhost.rst      |  17 +++
 drivers/net/virtio/virtio_ethdev.c      |  19 ++-
 drivers/net/virtio/virtio_ethdev.h      |   2 +
 drivers/net/virtio/virtio_user_ethdev.c | 228 +++++++++++++++++++++++++++++++-
 5 files changed, 265 insertions(+), 12 deletions(-)

diff --git a/doc/guides/rel_notes/release_16_07.rst b/doc/guides/rel_notes/release_16_07.rst
index 30e78d4..4ecca7e 100644
--- a/doc/guides/rel_notes/release_16_07.rst
+++ b/doc/guides/rel_notes/release_16_07.rst
@@ -47,6 +47,17 @@ New Features
   * Dropped specific Xen Dom0 code.
   * Dropped specific anonymous mempool code in testpmd.
 
+* **Virtio support for containers.**
+
+  Add a new virtual device, named virtio-user, to support virtio for containers.
+
+  Known limitations:
+
+  * Control queue and multi-queue are not supported yet.
+  * Cannot work with --huge-unlink.
+  * Cannot work with --no-huge.
+  * Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8) hugepages.
+  * Root privilege is a must for sorting hugepages by physical address.
 
 Resolved Issues
 ---------------
diff --git a/doc/guides/sample_app_ug/vhost.rst b/doc/guides/sample_app_ug/vhost.rst
index 5f81802..a93e54d 100644
--- a/doc/guides/sample_app_ug/vhost.rst
+++ b/doc/guides/sample_app_ug/vhost.rst
@@ -833,3 +833,20 @@ For example:
 The above message indicates that device 0 has been registered with MAC address cc:bb:bb:bb:bb:bb and VLAN tag 1000.
 Any packets received on the NIC with these values is placed on the devices receive queue.
 When a virtio-net device transmits packets, the VLAN tag is added to the packet by the DPDK vhost sample code.
+
+Running virtio-user with vhost-switch
+-------------------------------------
+
+We can also use virtio-user with vhost-switch now.
+Virtio-user is a virtual device that can be run in a application (container) parallelly with vhost in the same OS,
+aka, there is no need to start a VM. We just run it with a different --file-prefix to avoid startup failure.
+
+.. code-block:: console
+
+    cd ${RTE_SDK}/x86_64-native-linuxapp-gcc/app
+    ./testpmd -c 0x3 -n 4 --socket-mem 1024 --no-pci --file-prefix=virtio-user-testpmd \
+    --vdev=virtio-user0,mac=00:01:02:03:04:05,path=$path_vhost \
+    -- -i --txqflags=0xf01 --disable-hw-vlan
+
+There is no difference on the vhost side.
+Pleae note that there are some limitations (see release note for more information) in the usage of virtio-user.
diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 9ccce79..4523ceb 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -59,7 +59,6 @@
 #include "virtqueue.h"
 #include "virtio_rxtx.h"
 
-static int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
 static int eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev);
 static int  virtio_dev_configure(struct rte_eth_dev *dev);
 static int  virtio_dev_start(struct rte_eth_dev *dev);
@@ -1081,7 +1080,7 @@ rx_func_get(struct rte_eth_dev *eth_dev)
  * This function is based on probe() function in virtio_pci.c
  * It returns 0 on success.
  */
-static int
+int
 eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 {
 	struct virtio_hw *hw = eth_dev->data->dev_private;
@@ -1112,9 +1111,11 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	pci_dev = eth_dev->pci_dev;
 
-	ret = vtpci_init(pci_dev, hw, &dev_flags);
-	if (ret)
-		return ret;
+	if (pci_dev) {
+		ret = vtpci_init(pci_dev, hw, &dev_flags);
+		if (ret)
+			return ret;
+	}
 
 	/* Reset the device although not necessary at startup */
 	vtpci_reset(hw);
@@ -1206,7 +1207,8 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	PMD_INIT_LOG(DEBUG, "hw->max_rx_queues=%d   hw->max_tx_queues=%d",
 			hw->max_rx_queues, hw->max_tx_queues);
-	PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
+	if (pci_dev)
+		PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
 			eth_dev->data->port_id, pci_dev->id.vendor_id,
 			pci_dev->id.device_id);
 
@@ -1494,7 +1496,10 @@ virtio_dev_info_get(struct rte_eth_dev *dev, struct rte_eth_dev_info *dev_info)
 {
 	struct virtio_hw *hw = dev->data->dev_private;
 
-	dev_info->driver_name = dev->driver->pci_drv.name;
+	if (dev->pci_dev)
+		dev_info->driver_name = dev->driver->pci_drv.name;
+	else
+		dev_info->driver_name = "virtio-user PMD";
 	dev_info->max_rx_queues = (uint16_t)hw->max_rx_queues;
 	dev_info->max_tx_queues = (uint16_t)hw->max_tx_queues;
 	dev_info->min_rx_bufsize = VIRTIO_MIN_RX_BUFSIZE;
diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
index 7e77259..2ecec6e 100644
--- a/drivers/net/virtio/virtio_ethdev.h
+++ b/drivers/net/virtio/virtio_ethdev.h
@@ -113,6 +113,8 @@ uint16_t virtio_recv_pkts_vec(void *rx_queue, struct rte_mbuf **rx_pkts,
 uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 		uint16_t nb_pkts);
 
+int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
+
 /*
  * The VIRTIO_NET_F_GUEST_TSO[46] features permit the host to send us
  * frames larger than 1514 bytes. We do not yet support software LRO
diff --git a/drivers/net/virtio/virtio_user_ethdev.c b/drivers/net/virtio/virtio_user_ethdev.c
index e390242..e1e77f0 100644
--- a/drivers/net/virtio/virtio_user_ethdev.c
+++ b/drivers/net/virtio/virtio_user_ethdev.c
@@ -35,6 +35,10 @@
 #include <sys/types.h>
 #include <unistd.h>
 
+#include <rte_malloc.h>
+#include <rte_kvargs.h>
+
+#include "virtio_ethdev.h"
 #include "virtio_logs.h"
 #include "virtio_pci.h"
 #include "virtqueue.h"
@@ -44,7 +48,7 @@
 	((struct virtio_user_dev *)(hw)->virtio_user_dev);
 
 static void
-virtio_user_read_dev_config(struct virtio_hw *hw, uint64_t offset,
+virtio_user_read_dev_config(struct virtio_hw *hw, size_t offset,
 		     void *dst, int length)
 {
 	int i;
@@ -65,7 +69,7 @@ virtio_user_read_dev_config(struct virtio_hw *hw, uint64_t offset,
 }
 
 static void
-virtio_user_write_dev_config(struct virtio_hw *hw, uint64_t offset,
+virtio_user_write_dev_config(struct virtio_hw *hw, size_t offset,
 		      const void *src, int length)
 {
 	int i;
@@ -165,9 +169,9 @@ virtio_user_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 				   VIRTIO_PCI_VRING_ALIGN);
 
 	dev->vrings[queue_idx].num = vq->vq_nentries;
-	dev->vrings[queue_idx].desc = (void *)desc_addr;
-	dev->vrings[queue_idx].avail = (void *)avail_addr;
-	dev->vrings[queue_idx].used = (void *)used_addr;
+	dev->vrings[queue_idx].desc = (void *)(uintptr_t)desc_addr;
+	dev->vrings[queue_idx].avail = (void *)(uintptr_t)avail_addr;
+	dev->vrings[queue_idx].used = (void *)(uintptr_t)used_addr;
 
 	return 0;
 }
@@ -216,3 +220,217 @@ static const struct virtio_pci_ops virtio_user_ops = {
 	.del_queue	= virtio_user_del_queue,
 	.notify_queue	= virtio_user_notify_queue,
 };
+
+static const char *valid_args[] = {
+#define VIRTIO_USER_ARG_QUEUES_NUM     "queues"
+	VIRTIO_USER_ARG_QUEUES_NUM,
+#define VIRTIO_USER_ARG_CQ_NUM         "cq"
+	VIRTIO_USER_ARG_CQ_NUM,
+#define VIRTIO_USER_ARG_MAC            "mac"
+	VIRTIO_USER_ARG_MAC,
+#define VIRTIO_USER_ARG_PATH           "path"
+	VIRTIO_USER_ARG_PATH,
+#define VIRTIO_USER_ARG_QUEUE_SIZE     "queue_size"
+	VIRTIO_USER_ARG_QUEUE_SIZE,
+#define VIRTIO_USER_ARG_IFNAME         "ifname"
+	VIRTIO_USER_ARG_IFNAME,
+	NULL
+};
+
+#define VIRTIO_USER_DEF_CQ_EN	0
+#define VIRTIO_USER_DEF_Q_NUM	1
+#define VIRTIO_USER_DEF_Q_SZ	256
+
+static int
+get_string_arg(const char *key __rte_unused,
+	       const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(char **)extra_args = strdup(value);
+
+	return 0;
+}
+
+static int
+get_integer_arg(const char *key __rte_unused,
+		const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(uint64_t *)extra_args = strtoull(value, NULL, 0);
+
+	return 0;
+}
+
+static struct rte_eth_dev *
+virtio_user_eth_dev_alloc(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct rte_eth_dev_data *data;
+	struct virtio_hw *hw;
+	struct virtio_user_dev *dev;
+
+	eth_dev = rte_eth_dev_allocate(name, RTE_ETH_DEV_VIRTUAL);
+	if (!eth_dev) {
+		PMD_INIT_LOG(ERR, "cannot alloc rte_eth_dev");
+		return NULL;
+	}
+
+	data = eth_dev->data;
+
+	hw = rte_zmalloc(NULL, sizeof(*hw), 0);
+	if (!hw) {
+		PMD_INIT_LOG(ERR, "malloc virtio_hw failed");
+		rte_eth_dev_release_port(eth_dev);
+		return NULL;
+	}
+
+	dev = rte_zmalloc(NULL, sizeof(*dev), 0);
+	if (!dev) {
+		PMD_INIT_LOG(ERR, "malloc virtio_user_dev failed");
+		rte_eth_dev_release_port(eth_dev);
+		rte_free(hw);
+		return NULL;
+	}
+
+	hw->vtpci_ops = &virtio_user_ops;
+	hw->use_msix = 0;
+	hw->modern   = 0;
+	hw->virtio_user_dev = dev;
+	data->dev_private = hw;
+	data->numa_node = SOCKET_ID_ANY;
+	data->kdrv = RTE_KDRV_NONE;
+	data->dev_flags = RTE_ETH_DEV_DETACHABLE;
+	eth_dev->pci_dev = NULL;
+	eth_dev->driver = NULL;
+	return eth_dev;
+}
+
+/* Dev initialization routine. Invoked once for each virtio vdev at
+ * EAL init time, see rte_eal_dev_init().
+ * Returns 0 on success.
+ */
+static int
+virtio_user_pmd_devinit(const char *name, const char *params)
+{
+	struct rte_kvargs *kvlist;
+	struct rte_eth_dev *eth_dev;
+	struct virtio_hw *hw;
+	uint64_t queues = VIRTIO_USER_DEF_Q_NUM;
+	uint64_t cq = VIRTIO_USER_DEF_CQ_EN;
+	uint64_t queue_size = VIRTIO_USER_DEF_Q_SZ;
+	char *path = NULL;
+	char *mac_addr = NULL;
+	char *ifname = NULL;
+	int ret = -1;
+
+	if (!params || params[0] == '\0') {
+		PMD_INIT_LOG(ERR, "arg %s is mandatory for virtio-user",
+			  VIRTIO_USER_ARG_QUEUE_SIZE);
+		goto end;
+	}
+
+	kvlist = rte_kvargs_parse(params, valid_args);
+	if (!kvlist) {
+		PMD_INIT_LOG(ERR, "error when parsing param");
+		goto end;
+	}
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_PATH) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_PATH,
+				   &get_string_arg, &path);
+	else {
+		PMD_INIT_LOG(ERR, "arg %s is mandatory for virtio-user\n",
+			  VIRTIO_USER_ARG_QUEUE_SIZE);
+		goto end;
+	}
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_MAC) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_MAC,
+				   &get_string_arg, &mac_addr);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_IFNAME) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_IFNAME,
+				   &get_string_arg, &ifname);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_QUEUE_SIZE) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_QUEUE_SIZE,
+				   &get_integer_arg, &queue_size);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_QUEUES_NUM) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_QUEUES_NUM,
+				   &get_integer_arg, &queues);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_CQ_NUM) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_CQ_NUM,
+				   &get_integer_arg, &cq);
+
+	eth_dev = virtio_user_eth_dev_alloc(name);
+	if (!eth_dev) {
+		PMD_INIT_LOG(ERR, "virtio-user fails to alloc device");
+		goto end;
+	}
+
+	hw = eth_dev->data->dev_private;
+	if (virtio_user_dev_init(hw->virtio_user_dev, path, queues, cq,
+				 queue_size, mac_addr, ifname) < 0)
+		goto end;
+
+	/* previously called by rte_eal_pci_probe() for physical dev */
+	if (eth_virtio_dev_init(eth_dev) < 0) {
+		PMD_INIT_LOG(ERR, "eth_virtio_dev_init fails");
+		goto end;
+	}
+	ret = 0;
+
+end:
+	if (path)
+		free(path);
+	if (mac_addr)
+		free(mac_addr);
+	if (ifname)
+		free(ifname);
+	return ret;
+}
+
+/** Called by rte_eth_dev_detach() */
+static int
+virtio_user_pmd_devuninit(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct virtio_hw *hw;
+	struct virtio_user_dev *dev;
+
+	if (!name)
+		return -EINVAL;
+
+	PMD_DRV_LOG(INFO, "Un-Initializing %s\n", name);
+	eth_dev = rte_eth_dev_allocated(name);
+	if (!eth_dev)
+		return -ENODEV;
+
+	/* make sure the device is stopped, queues freed */
+	rte_eth_dev_close(eth_dev->data->port_id);
+
+	hw = eth_dev->data->dev_private;
+	dev = hw->virtio_user_dev;
+	virtio_user_dev_uninit(dev);
+
+	rte_free(eth_dev->data->dev_private);
+	rte_free(eth_dev->data);
+	rte_eth_dev_release_port(eth_dev);
+
+	return 0;
+}
+
+static struct rte_driver virtio_user_driver = {
+	.name   = "virtio-user",
+	.type   = PMD_VDEV,
+	.init   = virtio_user_pmd_devinit,
+	.uninit = virtio_user_pmd_devuninit,
+};
+
+PMD_REGISTER_DRIVER(virtio_user_driver);
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v8 0/6] virtio support for container
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
                   ` (12 preceding siblings ...)
  2016-06-12  0:35 ` [PATCH v7 0/6] virtio support for container Jianfeng Tan
@ 2016-06-13  6:38 ` Jianfeng Tan
  2016-06-13  6:38   ` [PATCH v8 1/6] virtio: hide phys addr check inside pci ops Jianfeng Tan
                     ` (6 more replies)
  2016-06-15  9:03 ` [PATCH v9 " Jianfeng Tan
  14 siblings, 7 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-13  6:38 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

v8:
 - Change to use max_queue_pairs instead of queue_pairs to initialize
   and deinitialize queues.
 - Remove vhost-kernel support.

v7:
 - CONFIG_RTE_VIRTIO_VDEV -> CONFIG_RTE_VIRTIO_USER; and corresondingly,
   RTE_VIRTIO_VDEV -> RTE_VIRTIO_USER.
 - uint64_t -> uintptr_t, so that it can be compiled on 32-bit platform.
 - Rebase on latest dpdk-next-virtio branch.
 - Abandon abstracting related code into vring_hdr_desc_init(), instead,
   just move it behind setup_queue().

v6:
 - Move driver related code into from driver/net/virtio/virtio-user/ to
   driver/net/virtio/ directory, inside virtio_user_ethdev.c.
 - Rename vdev to virtio_user in comments and code.
 - Merge code, which lies in virtio_user_pci.c, into virtio_user_ethdev.c.
 - Add some comments at virtio-user special handling at virtio_dev_ethdev.c.
 - Merge document update into the 7nd commit where virtio-user is added.
 - Add usage with vhost-switch in vhost.rst.

v5:
 - Rename struct virtio_user_hw to struct virtio_user_dev.
 - Rename "vdev_private" to "virtio_user_dev".
 - Move special handling into virtio_ethdev.c from queue_setup().
 - Add vring in virtio_user_dev (remove rte_eth_dev_data), so that
   device does not depend on driver's data structure (rte_eth_dev_data).
 - Remove update on doc/guides/nics/overview.rst, because virtio-user has
   exact feature set with virtio.
 - Change "unsigned long int" to "uint64_t", "unsigned" to "uint32_t".
 - Remove unnecessary cast in vdev_read_dev_config().
 - Add functions in virtio_user_dev.c with prefix of "virtio_user_".
 - Rebase on virtio-next-virtio.

v4:
 - Avoid using dev_type, instead use (eth_dev->pci_device is NULL) to
   judge if it's virtual device or physical device.
 - Change the added device name to virtio-user.
 - Split into vhost_user.c, vhost_kernel.c, vhost.c, virtio_user_pci.c,
   virtio_user_dev.c.
 - Move virtio-user specific data from struct virtio_hw into struct
   virtio_user_hw.
 - Add support to send reset_owner message.
 - Change del_queue implementation. (This need more check)
 - Remove rte_panic(), and superseded with log.
 - Add reset_owner into virtio_pci_ops.reset.
 - Merge parameter "rx" and "tx" to "queues" to emliminate confusion.
 - Move get_features to after set_owner.
 - Redefine path in virtio_user_hw from char * to char [].

v3:
 - Remove --single-file option; do no change at EAL memory.
 - Remove the added API rte_eal_get_backfile_info(), instead we check all
   opened files with HUGEFILE_FMT to find hugepage files owned by DPDK.
 - Accordingly, add more restrictions at "Known issue" section.
 - Rename parameter from queue_num to queue_size for confusion.
 - Rename vhost_embedded.c to rte_eth_virtio_vdev.c.
 - Move code related to the newly added vdev to rte_eth_virtio_vdev.c, to
   reuse eth_virtio_dev_init(), remove its static declaration.
 - Implement dev_uninit() for rte_eth_dev_detach().
 - WARN -> ERR, in vhost_embedded.c
 - Add more commit message for clarify the model.

v2:
 - Rebase on the patchset of virtio 1.0 support.
 - Fix cannot create non-hugepage memory.
 - Fix wrong size of memory region when "single-file" is used.
 - Fix setting of offset in virtqueue to use virtual address.
 - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
 - Add mac option to specify the mac address of this virtual device.
 - Update doc.

This patchset is to provide high performance networking interface (virtio)
for container-based DPDK applications. The way of starting DPDK apps in
containers with ownership of NIC devices exclusively is beyond the scope.
The basic idea here is to present a new virtual device (named virtio-user),
which can be discovered and initialized by DPDK. To minimize the change,
we reuse already-existing virtio PMD code (driver/net/virtio/).

Background: Previously, we usually use a virtio device in the context of
QEMU/VM as below pic shows. Virtio nic is emulated in QEMU, and usually
presented in VM as a PCI device.

  ------------------
  |  virtio driver |  ----->  VM
  ------------------
        |
        | ----------> (over PCI bus or MMIO or Channel I/O)
        |
  ------------------
  | device emulate |
  |                |  ----->  QEMU
  | vhost adapter  |
  ------------------
        |
        | ----------> (vhost-user protocol or vhost-net ioctls)
        |
  ------------------
  | vhost backend  |
  ------------------
 
Compared to QEMU/VM case, virtio support for contaner requires to embedded
device framework inside the virtio PMD. So this converged driver actually
plays three roles:
  - virtio driver to drive this new kind of virtual device;
  - device emulation to present this virtual device and reponse to the
    virtio driver, which is originally by QEMU;
  - and the role to communicate with vhost backend, which is also
    originally by QEMU.

The code layout and functionality of each module:
 
  ----------------------
  | ------------------ |
  | | virtio driver  | |----> (virtio_user_ethdev.c)
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate |-|----> (virtio_user_dev.c)
  | |                | |
  | | vhost adapter  |-|----> (vhost_user.c, vhost_kernel.c, vhost.c)
  | ------------------ |
  ----------------------
         |
         | -------------- --> (vhost-user protocol)
         |
   ------------------
   | vhost backend  |
   ------------------

How to share memory? In VM's case, qemu always shares all physical layout
to backend. But it's not feasible for a container, as a process, to share
all virtual memory regions to backend. So only specified virtual memory
regions (with type of shared) are sent to backend. It's a limitation that
only addresses in these areas can be used to transmit or receive packets.

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").
 - Cannot work with vhost kernel.

How to use?

a. Apply this patchset.

b. To compile container apps:
$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc

c. To build a docker image using Dockerfile below.
$: cat ./Dockerfile
FROM ubuntu:latest
WORKDIR /usr/src/dpdk
COPY . /usr/src/dpdk
ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
$: docker build -t dpdk-app-l2fwd .

d. Used with vhost-user
$: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
	--socket-mem 1024,1024 -- -p 0x1 --stats 1
$: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=virtio-user0,path=/var/run/usvhost -- -p 0x1

By the way, it's not necessary to run in a container.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>


Jianfeng Tan (6):
  virtio: hide phys addr check inside pci ops
  virtio: enable use virtual address to fill desc
  virtio-user: add vhost user adapter layer
  virtio-user: add device emulation layer APIs
  virtio-user: add new virtual pci driver for virtio
  virtio-user: add a new vdev named virtio-user

 config/common_linuxapp                           |   1 +
 doc/guides/rel_notes/release_16_07.rst           |  12 +
 doc/guides/sample_app_ug/vhost.rst               |  17 +
 drivers/net/virtio/Makefile                      |   6 +
 drivers/net/virtio/virtio_ethdev.c               |  77 ++--
 drivers/net/virtio/virtio_ethdev.h               |   2 +
 drivers/net/virtio/virtio_pci.c                  |  30 +-
 drivers/net/virtio/virtio_pci.h                  |   3 +-
 drivers/net/virtio/virtio_rxtx.c                 |   5 +-
 drivers/net/virtio/virtio_rxtx_simple.c          |  13 +-
 drivers/net/virtio/virtio_user/vhost.h           | 141 ++++++++
 drivers/net/virtio/virtio_user/vhost_user.c      | 404 +++++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 227 ++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.h |  62 ++++
 drivers/net/virtio/virtio_user_ethdev.c          | 427 +++++++++++++++++++++++
 drivers/net/virtio/virtqueue.h                   |  10 +
 16 files changed, 1395 insertions(+), 42 deletions(-)
 create mode 100644 drivers/net/virtio/virtio_user/vhost.h
 create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h
 create mode 100644 drivers/net/virtio/virtio_user_ethdev.c

-- 
2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v8 1/6] virtio: hide phys addr check inside pci ops
  2016-06-13  6:38 ` [PATCH v8 0/6] virtio support for container Jianfeng Tan
@ 2016-06-13  6:38   ` Jianfeng Tan
  2016-06-13  6:38   ` [PATCH v8 2/6] virtio: enable use virtual address to fill desc Jianfeng Tan
                     ` (5 subsequent siblings)
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-13  6:38 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is to move phys addr check from virtio_dev_queue_setup
to pci ops. To makt that happen, make sure virtio_ops.setup_queue
return the result if we pass through the check.

Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Acked-by: Yuanhan Liu <yuanhan.liu@linux.intel.com>
---
 drivers/net/virtio/virtio_ethdev.c | 17 ++++++-----------
 drivers/net/virtio/virtio_pci.c    | 30 ++++++++++++++++++++++++++++--
 drivers/net/virtio/virtio_pci.h    |  2 +-
 3 files changed, 35 insertions(+), 14 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index a995520..841949b 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -385,16 +385,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		}
 	}
 
-	/*
-	 * Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
-	 * and only accepts 32 bit page frame number.
-	 * Check if the allocated physical memory exceeds 16TB.
-	 */
-	if ((mz->phys_addr + vq->vq_ring_size - 1) >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
-		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
-		ret = -ENOMEM;
-		goto fail_q_alloc;
-	}
 	memset(mz->addr, 0, sizeof(mz->len));
 
 	vq->vq_ring_mem = mz->phys_addr;
@@ -479,7 +469,12 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		*pvq = cvq;
 	}
 
-	hw->vtpci_ops->setup_queue(hw, vq);
+	if (hw->vtpci_ops->setup_queue(hw, vq) < 0) {
+		PMD_INIT_LOG(ERR, "setup_queue failed");
+		virtio_dev_queue_release(vq);
+		return -EINVAL;
+	}
+
 	vq->configured = 1;
 	return 0;
 
diff --git a/drivers/net/virtio/virtio_pci.c b/drivers/net/virtio/virtio_pci.c
index d0f2428..8d0c983 100644
--- a/drivers/net/virtio/virtio_pci.c
+++ b/drivers/net/virtio/virtio_pci.c
@@ -55,6 +55,22 @@
  */
 #define VIRTIO_PCI_CONFIG(hw) (((hw)->use_msix) ? 24 : 20)
 
+static inline int
+check_vq_phys_addr_ok(struct virtqueue *vq)
+{
+	/* Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
+	 * and only accepts 32 bit page frame number.
+	 * Check if the allocated physical memory exceeds 16TB.
+	 */
+	if ((vq->vq_ring_mem + vq->vq_ring_size - 1) >>
+			(VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
+		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
+		return 0;
+	}
+
+	return 1;
+}
+
 static void
 legacy_read_dev_config(struct virtio_hw *hw, size_t offset,
 		       void *dst, int length)
@@ -143,15 +159,20 @@ legacy_get_queue_num(struct virtio_hw *hw, uint16_t queue_id)
 	return dst;
 }
 
-static void
+static int
 legacy_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 {
 	uint32_t src;
 
+	if (!check_vq_phys_addr_ok(vq))
+		return -1;
+
 	rte_eal_pci_ioport_write(&hw->io, &vq->vq_queue_index, 2,
 			 VIRTIO_PCI_QUEUE_SEL);
 	src = vq->vq_ring_mem >> VIRTIO_PCI_QUEUE_ADDR_SHIFT;
 	rte_eal_pci_ioport_write(&hw->io, &src, 4, VIRTIO_PCI_QUEUE_PFN);
+
+	return 0;
 }
 
 static void
@@ -367,12 +388,15 @@ modern_get_queue_num(struct virtio_hw *hw, uint16_t queue_id)
 	return io_read16(&hw->common_cfg->queue_size);
 }
 
-static void
+static int
 modern_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 {
 	uint64_t desc_addr, avail_addr, used_addr;
 	uint16_t notify_off;
 
+	if (!check_vq_phys_addr_ok(vq))
+		return -1;
+
 	desc_addr = vq->vq_ring_mem;
 	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
 	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
@@ -400,6 +424,8 @@ modern_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 	PMD_INIT_LOG(DEBUG, "\t used_addr: %" PRIx64, used_addr);
 	PMD_INIT_LOG(DEBUG, "\t notify addr: %p (notify offset: %u)",
 		vq->notify_addr, notify_off);
+
+	return 0;
 }
 
 static void
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index f20468a..6c7f8d7 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -235,7 +235,7 @@ struct virtio_pci_ops {
 	uint16_t (*set_config_irq)(struct virtio_hw *hw, uint16_t vec);
 
 	uint16_t (*get_queue_num)(struct virtio_hw *hw, uint16_t queue_id);
-	void (*setup_queue)(struct virtio_hw *hw, struct virtqueue *vq);
+	int (*setup_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 	void (*del_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 	void (*notify_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 };
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v8 2/6] virtio: enable use virtual address to fill desc
  2016-06-13  6:38 ` [PATCH v8 0/6] virtio support for container Jianfeng Tan
  2016-06-13  6:38   ` [PATCH v8 1/6] virtio: hide phys addr check inside pci ops Jianfeng Tan
@ 2016-06-13  6:38   ` Jianfeng Tan
  2016-06-13  6:39   ` [PATCH v8 3/6] virtio-user: add vhost user adapter layer Jianfeng Tan
                     ` (4 subsequent siblings)
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-13  6:38 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is related to how to calculate relative address for vhost
backend.

The principle is that: based on one or multiple shared memory regions,
vhost maintains a reference system with the frontend start address,
backend start address, and length for each segment, so that each
frontend address (GPA, Guest Physical Address) can be translated into
vhost-recognizable backend address. To make the address translation
efficient, we need to maintain as few regions as possible. In the case
of VM, GPA is always locally continuous. But for some other case, like
virtio-user, we use virtual address here.

It basically means:
  a. when set_base_addr, VA address is used;
  b. when preparing RX's descriptors, VA address is used;
  c. when transmitting packets, VA is filled in TX's descriptors;
  d. in TX and CQ's header, VA is used.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/virtio_ethdev.c      | 43 +++++++++++++++++++++++----------
 drivers/net/virtio/virtio_rxtx.c        |  5 ++--
 drivers/net/virtio/virtio_rxtx_simple.c | 13 +++++-----
 drivers/net/virtio/virtqueue.h          | 10 ++++++++
 4 files changed, 49 insertions(+), 22 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 841949b..9ccce79 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -431,9 +431,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		rxvq->mz = mz;
 		*pvq = rxvq;
 	} else if (queue_type == VTNET_TQ) {
-		struct virtio_tx_region *txr;
-		unsigned int i;
-
 		txvq = (struct virtnet_tx *)RTE_PTR_ADD(vq, sz_vq);
 		txvq->vq = vq;
 		txvq->port_id = dev->data->port_id;
@@ -442,6 +439,36 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		txvq->virtio_net_hdr_mz = hdr_mz;
 		txvq->virtio_net_hdr_mem = hdr_mz->phys_addr;
 
+		*pvq = txvq;
+	} else if (queue_type == VTNET_CQ) {
+		cvq = (struct virtnet_ctl *)RTE_PTR_ADD(vq, sz_vq);
+		cvq->vq = vq;
+		cvq->mz = mz;
+		cvq->virtio_net_hdr_mz = hdr_mz;
+		cvq->virtio_net_hdr_mem = hdr_mz->phys_addr;
+		memset(cvq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
+		*pvq = cvq;
+	}
+
+	/* For virtio-user case (that is when dev->pci_dev is NULL), we use
+	 * virtual address. And we need properly set _offset_, please see
+	 * MBUF_DATA_DMA_ADDR in virtqueue.h for more information.
+	 */
+	if (dev->pci_dev)
+		vq->offset = offsetof(struct rte_mbuf, buf_physaddr);
+	else {
+		vq->vq_ring_mem = (uintptr_t)mz->addr;
+		vq->offset = offsetof(struct rte_mbuf, buf_addr);
+		if (queue_type == VTNET_TQ)
+			txvq->virtio_net_hdr_mem = (uintptr_t)hdr_mz->addr;
+		else if (queue_type == VTNET_CQ)
+			cvq->virtio_net_hdr_mem = (uintptr_t)hdr_mz->addr;
+	}
+
+	if (queue_type == VTNET_TQ) {
+		struct virtio_tx_region *txr;
+		unsigned int i;
+
 		txr = hdr_mz->addr;
 		memset(txr, 0, vq_size * sizeof(*txr));
 		for (i = 0; i < vq_size; i++) {
@@ -457,16 +484,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 			start_dp->len = hw->vtnet_hdr_size;
 			start_dp->flags = VRING_DESC_F_NEXT;
 		}
-
-		*pvq = txvq;
-	} else if (queue_type == VTNET_CQ) {
-		cvq = (struct virtnet_ctl *)RTE_PTR_ADD(vq, sz_vq);
-		cvq->vq = vq;
-		cvq->mz = mz;
-		cvq->virtio_net_hdr_mz = hdr_mz;
-		cvq->virtio_net_hdr_mem = hdr_mz->phys_addr;
-		memset(cvq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
-		*pvq = cvq;
 	}
 
 	if (hw->vtpci_ops->setup_queue(hw, vq) < 0) {
diff --git a/drivers/net/virtio/virtio_rxtx.c b/drivers/net/virtio/virtio_rxtx.c
index 28266d2..b96d0cb 100644
--- a/drivers/net/virtio/virtio_rxtx.c
+++ b/drivers/net/virtio/virtio_rxtx.c
@@ -193,8 +193,7 @@ virtqueue_enqueue_recv_refill(struct virtqueue *vq, struct rte_mbuf *cookie)
 
 	start_dp = vq->vq_ring.desc;
 	start_dp[idx].addr =
-		(uint64_t)(cookie->buf_physaddr + RTE_PKTMBUF_HEADROOM
-		- hw->vtnet_hdr_size);
+		MBUF_DATA_DMA_ADDR(cookie, vq->offset) - hw->vtnet_hdr_size;
 	start_dp[idx].len =
 		cookie->buf_len - RTE_PKTMBUF_HEADROOM + hw->vtnet_hdr_size;
 	start_dp[idx].flags =  VRING_DESC_F_WRITE;
@@ -266,7 +265,7 @@ virtqueue_enqueue_xmit(struct virtnet_tx *txvq, struct rte_mbuf *cookie,
 	}
 
 	do {
-		start_dp[idx].addr  = rte_mbuf_data_dma_addr(cookie);
+		start_dp[idx].addr  = MBUF_DATA_DMA_ADDR(cookie, vq->offset);
 		start_dp[idx].len   = cookie->data_len;
 		start_dp[idx].flags = cookie->next ? VRING_DESC_F_NEXT : 0;
 		idx = start_dp[idx].next;
diff --git a/drivers/net/virtio/virtio_rxtx_simple.c b/drivers/net/virtio/virtio_rxtx_simple.c
index fdd655d..7b50119 100644
--- a/drivers/net/virtio/virtio_rxtx_simple.c
+++ b/drivers/net/virtio/virtio_rxtx_simple.c
@@ -80,8 +80,8 @@ virtqueue_enqueue_recv_refill_simple(struct virtqueue *vq,
 	vq->sw_ring[desc_idx] = cookie;
 
 	start_dp = vq->vq_ring.desc;
-	start_dp[desc_idx].addr = (uint64_t)((uintptr_t)cookie->buf_physaddr +
-		RTE_PKTMBUF_HEADROOM - vq->hw->vtnet_hdr_size);
+	start_dp[desc_idx].addr = MBUF_DATA_DMA_ADDR(cookie, vq->offset) -
+				  vq->hw->vtnet_hdr_size;
 	start_dp[desc_idx].len = cookie->buf_len -
 		RTE_PKTMBUF_HEADROOM + vq->hw->vtnet_hdr_size;
 
@@ -120,8 +120,8 @@ virtio_rxq_rearm_vec(struct virtnet_rx *rxvq)
 		*(uint64_t *)p = rxvq->mbuf_initializer;
 
 		start_dp[i].addr =
-			(uint64_t)((uintptr_t)sw_ring[i]->buf_physaddr +
-			RTE_PKTMBUF_HEADROOM - vq->hw->vtnet_hdr_size);
+			MBUF_DATA_DMA_ADDR(sw_ring[i], vq->offset) -
+			vq->hw->vtnet_hdr_size;
 		start_dp[i].len = sw_ring[i]->buf_len -
 			RTE_PKTMBUF_HEADROOM + vq->hw->vtnet_hdr_size;
 	}
@@ -369,7 +369,7 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 			vq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 		for (i = 0; i < nb_tail; i++) {
 			start_dp[desc_idx].addr =
-				rte_mbuf_data_dma_addr(*tx_pkts);
+				MBUF_DATA_DMA_ADDR(*tx_pkts, vq->offset);
 			start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 			tx_pkts++;
 			desc_idx++;
@@ -380,7 +380,8 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 	for (i = 0; i < nb_commit; i++)
 		vq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 	for (i = 0; i < nb_commit; i++) {
-		start_dp[desc_idx].addr = rte_mbuf_data_dma_addr(*tx_pkts);
+		start_dp[desc_idx].addr =
+			MBUF_DATA_DMA_ADDR(*tx_pkts, vq->offset);
 		start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 		tx_pkts++;
 		desc_idx++;
diff --git a/drivers/net/virtio/virtqueue.h b/drivers/net/virtio/virtqueue.h
index 17c1ea1..455aaaf 100644
--- a/drivers/net/virtio/virtqueue.h
+++ b/drivers/net/virtio/virtqueue.h
@@ -66,6 +66,14 @@ struct rte_mbuf;
 
 #define VIRTQUEUE_MAX_NAME_SZ 32
 
+#ifdef RTE_VIRTIO_USER
+#define MBUF_DATA_DMA_ADDR(mb, offset) \
+	((uint64_t)((uintptr_t)(*(void **)((uintptr_t)mb + offset)) \
+			+ (mb)->data_off))
+#else /* RTE_VIRTIO_USER */
+#define MBUF_DATA_DMA_ADDR(mb, offset) rte_mbuf_data_dma_addr(mb)
+#endif /* RTE_VIRTIO_USER */
+
 #define VTNET_SQ_RQ_QUEUE_IDX 0
 #define VTNET_SQ_TQ_QUEUE_IDX 1
 #define VTNET_SQ_CQ_QUEUE_IDX 2
@@ -175,6 +183,7 @@ struct virtqueue {
 	unsigned int vq_ring_size;
 
 	phys_addr_t vq_ring_mem; /**< physical address of vring */
+				/**< use virtual address for virtio-user. */
 
 	/**
 	 * Head of the free chain in the descriptor table. If
@@ -184,6 +193,7 @@ struct virtqueue {
 	uint16_t  vq_desc_head_idx;
 	uint16_t  vq_desc_tail_idx;
 	uint16_t  vq_queue_index;   /**< PCI queue index */
+	uint16_t offset; /**< relative offset to obtain addr in mbuf */
 	uint16_t  *notify_addr;
 	int configured;
 	struct rte_mbuf **sw_ring;  /**< RX software ring. */
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v8 3/6] virtio-user: add vhost user adapter layer
  2016-06-13  6:38 ` [PATCH v8 0/6] virtio support for container Jianfeng Tan
  2016-06-13  6:38   ` [PATCH v8 1/6] virtio: hide phys addr check inside pci ops Jianfeng Tan
  2016-06-13  6:38   ` [PATCH v8 2/6] virtio: enable use virtual address to fill desc Jianfeng Tan
@ 2016-06-13  6:39   ` Jianfeng Tan
  2016-06-13  6:39   ` [PATCH v8 4/6] virtio-user: add device emulation layer APIs Jianfeng Tan
                     ` (3 subsequent siblings)
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-13  6:39 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is to provide vhost adapter layer implementations. Instead
of relying on a hypervisor to translate between device emulation and
vhost backend, here we directly talk with vhost backend through the
vhost file.

Here three main APIs are provided to upper layer (device emulation):
  - vhost_user_setup(), to set up vhost user backend;
  - vhost_user_sock(), to talk with vhost user backend.

  ----------------------
  | ------------------ |
  | | virtio driver  | |
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate | |
  | |                | |
  | | vhost adapter  |-|----> (vhost_user.c)
  | ------------------ |
  ----------------------
            |
            | -------------- --> (vhost-user protocol)
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 config/common_linuxapp                      |   1 +
 drivers/net/virtio/Makefile                 |   4 +
 drivers/net/virtio/virtio_user/vhost.h      | 141 ++++++++++
 drivers/net/virtio/virtio_user/vhost_user.c | 404 ++++++++++++++++++++++++++++
 4 files changed, 550 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/vhost.h
 create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c

diff --git a/config/common_linuxapp b/config/common_linuxapp
index 7e698e2..2483dfa 100644
--- a/config/common_linuxapp
+++ b/config/common_linuxapp
@@ -43,3 +43,4 @@ CONFIG_RTE_LIBRTE_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_AF_PACKET=y
 CONFIG_RTE_LIBRTE_POWER=y
+CONFIG_RTE_VIRTIO_USER=y
diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index ef84f60..c2ed0fa 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -55,6 +55,10 @@ ifeq ($(findstring RTE_MACHINE_CPUFLAG_SSSE3,$(CFLAGS)),RTE_MACHINE_CPUFLAG_SSSE
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx_simple.c
 endif
 
+ifeq ($(CONFIG_RTE_VIRTIO_USER),y)
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
+endif
+
 # this lib depends upon:
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_eal lib/librte_ether
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_mempool lib/librte_mbuf
diff --git a/drivers/net/virtio/virtio_user/vhost.h b/drivers/net/virtio/virtio_user/vhost.h
new file mode 100644
index 0000000..4e04ede
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost.h
@@ -0,0 +1,141 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VHOST_NET_USER_H
+#define _VHOST_NET_USER_H
+
+#include <stdint.h>
+#include <linux/types.h>
+#include <linux/ioctl.h>
+
+#include "../virtio_pci.h"
+#include "../virtio_logs.h"
+#include "../virtqueue.h"
+
+#define VHOST_MEMORY_MAX_NREGIONS 8
+
+struct vhost_vring_state {
+	unsigned int index;
+	unsigned int num;
+};
+
+struct vhost_vring_file {
+	unsigned int index;
+	int fd;
+};
+
+struct vhost_vring_addr {
+	unsigned int index;
+	/* Option flags. */
+	unsigned int flags;
+	/* Flag values: */
+	/* Whether log address is valid. If set enables logging. */
+#define VHOST_VRING_F_LOG 0
+
+	/* Start of array of descriptors (virtually contiguous) */
+	uint64_t desc_user_addr;
+	/* Used structure address. Must be 32 bit aligned */
+	uint64_t used_user_addr;
+	/* Available structure address. Must be 16 bit aligned */
+	uint64_t avail_user_addr;
+	/* Logging support. */
+	/* Log writes to used structure, at offset calculated from specified
+	 * address. Address must be 32 bit aligned.
+	 */
+	uint64_t log_guest_addr;
+};
+
+enum vhost_user_request {
+	VHOST_USER_NONE = 0,
+	VHOST_USER_GET_FEATURES = 1,
+	VHOST_USER_SET_FEATURES = 2,
+	VHOST_USER_SET_OWNER = 3,
+	VHOST_USER_RESET_OWNER = 4,
+	VHOST_USER_SET_MEM_TABLE = 5,
+	VHOST_USER_SET_LOG_BASE = 6,
+	VHOST_USER_SET_LOG_FD = 7,
+	VHOST_USER_SET_VRING_NUM = 8,
+	VHOST_USER_SET_VRING_ADDR = 9,
+	VHOST_USER_SET_VRING_BASE = 10,
+	VHOST_USER_GET_VRING_BASE = 11,
+	VHOST_USER_SET_VRING_KICK = 12,
+	VHOST_USER_SET_VRING_CALL = 13,
+	VHOST_USER_SET_VRING_ERR = 14,
+	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
+	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
+	VHOST_USER_GET_QUEUE_NUM = 17,
+	VHOST_USER_SET_VRING_ENABLE = 18,
+	VHOST_USER_MAX
+};
+
+struct vhost_memory_region {
+	uint64_t guest_phys_addr;
+	uint64_t memory_size; /* bytes */
+	uint64_t userspace_addr;
+	uint64_t mmap_offset;
+};
+
+struct vhost_memory {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[VHOST_MEMORY_MAX_NREGIONS];
+};
+
+struct vhost_user_msg {
+	enum vhost_user_request request;
+
+#define VHOST_USER_VERSION_MASK     0x3
+#define VHOST_USER_REPLY_MASK       (0x1 << 2)
+	uint32_t flags;
+	uint32_t size; /* the following payload size */
+	union {
+#define VHOST_USER_VRING_IDX_MASK   0xff
+#define VHOST_USER_VRING_NOFD_MASK  (0x1 << 8)
+		uint64_t u64;
+		struct vhost_vring_state state;
+		struct vhost_vring_addr addr;
+		struct vhost_memory memory;
+	} payload;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+} __attribute((packed));
+
+#define VHOST_USER_HDR_SIZE offsetof(struct vhost_user_msg, payload.u64)
+#define VHOST_USER_PAYLOAD_SIZE \
+	(sizeof(struct vhost_user_msg) - VHOST_USER_HDR_SIZE)
+
+/* The version of the protocol we support */
+#define VHOST_USER_VERSION    0x1
+
+int vhost_user_sock(int vhostfd, uint64_t req, void *arg);
+int vhost_user_setup(const char *path);
+#endif
diff --git a/drivers/net/virtio/virtio_user/vhost_user.c b/drivers/net/virtio/virtio_user/vhost_user.c
new file mode 100644
index 0000000..8e5c0a9
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost_user.c
@@ -0,0 +1,404 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <unistd.h>
+#include <fcntl.h>
+#include <sys/un.h>
+#include <string.h>
+#include <errno.h>
+
+#include "vhost.h"
+
+static int
+vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
+{
+	int r;
+	struct msghdr msgh;
+	struct iovec iov;
+	size_t fd_size = fd_num * sizeof(int);
+	char control[CMSG_SPACE(fd_size)];
+	struct cmsghdr *cmsg;
+
+	memset(&msgh, 0, sizeof(msgh));
+	memset(control, 0, sizeof(control));
+
+	iov.iov_base = (uint8_t *)buf;
+	iov.iov_len = len;
+
+	msgh.msg_iov = &iov;
+	msgh.msg_iovlen = 1;
+	msgh.msg_control = control;
+	msgh.msg_controllen = sizeof(control);
+
+	cmsg = CMSG_FIRSTHDR(&msgh);
+	cmsg->cmsg_len = CMSG_LEN(fd_size);
+	cmsg->cmsg_level = SOL_SOCKET;
+	cmsg->cmsg_type = SCM_RIGHTS;
+	memcpy(CMSG_DATA(cmsg), fds, fd_size);
+
+	do {
+		r = sendmsg(fd, &msgh, 0);
+	} while (r < 0 && errno == EINTR);
+
+	return r;
+}
+
+static int
+vhost_user_read(int fd, struct vhost_user_msg *msg)
+{
+	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
+	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
+
+	ret = recv(fd, (void *)msg, sz_hdr, 0);
+	if (ret < sz_hdr) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
+			    ret, sz_hdr);
+		goto fail;
+	}
+
+	/* validate msg flags */
+	if (msg->flags != (valid_flags)) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg: flags %x instead of %x.",
+			    msg->flags, valid_flags);
+		goto fail;
+	}
+
+	sz_payload = msg->size;
+	if (sz_payload) {
+		ret = recv(fd, (void *)((char *)msg + sz_hdr), sz_payload, 0);
+		if (ret < sz_payload) {
+			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
+				    ret, msg->size);
+			goto fail;
+		}
+	}
+
+	return 0;
+
+fail:
+	return -1;
+}
+
+struct hugepage_file_info {
+	uint64_t addr;            /**< virtual addr */
+	size_t   size;            /**< the file size */
+	char     path[PATH_MAX];  /**< path to backing file */
+};
+
+/* Two possible options:
+ * 1. Match HUGEPAGE_INFO_FMT to find the file storing struct hugepage_file
+ * array. This is simple but cannot be used in secondary process because
+ * secondary process will close and munmap that file.
+ * 2. Match HUGEFILE_FMT to find hugepage files directly.
+ *
+ * We choose option 2.
+ */
+static int
+get_hugepage_file_info(struct hugepage_file_info huges[], int max)
+{
+	int idx;
+	FILE *f;
+	char buf[BUFSIZ], *tmp, *tail;
+	char *str_underline, *str_start;
+	int huge_index;
+	uint64_t v_start, v_end;
+
+	f = fopen("/proc/self/maps", "r");
+	if (!f) {
+		PMD_DRV_LOG(ERR, "cannot open /proc/self/maps");
+		return -1;
+	}
+
+	idx = 0;
+	while (fgets(buf, sizeof(buf), f) != NULL) {
+		if (sscanf(buf, "%" PRIx64 "-%" PRIx64, &v_start, &v_end) < 2) {
+			PMD_DRV_LOG(ERR, "Failed to parse address");
+			goto error;
+		}
+
+		tmp = strchr(buf, ' ') + 1; /** skip address */
+		tmp = strchr(tmp, ' ') + 1; /** skip perm */
+		tmp = strchr(tmp, ' ') + 1; /** skip offset */
+		tmp = strchr(tmp, ' ') + 1; /** skip dev */
+		tmp = strchr(tmp, ' ') + 1; /** skip inode */
+		while (*tmp == ' ')         /** skip spaces */
+			tmp++;
+		tail = strrchr(tmp, '\n');  /** remove newline if exists */
+		if (tail)
+			*tail = '\0';
+
+		/* Match HUGEFILE_FMT, aka "%s/%smap_%d",
+		 * which is defined in eal_filesystem.h
+		 */
+		str_underline = strrchr(tmp, '_');
+		if (!str_underline)
+			continue;
+
+		str_start = str_underline - strlen("map");
+		if (str_start < tmp)
+			continue;
+
+		if (sscanf(str_start, "map_%d", &huge_index) != 1)
+			continue;
+
+		if (idx >= max) {
+			PMD_DRV_LOG(ERR, "Exceed maximum of %d", max);
+			goto error;
+		}
+		huges[idx].addr = v_start;
+		huges[idx].size = v_end - v_start;
+		strcpy(huges[idx].path, tmp);
+		idx++;
+	}
+
+	fclose(f);
+	return idx;
+
+error:
+	fclose(f);
+	return -1;
+}
+
+static int
+prepare_vhost_memory_user(struct vhost_user_msg *msg, int fds[])
+{
+	int i, num;
+	struct hugepage_file_info huges[VHOST_MEMORY_MAX_NREGIONS];
+	struct vhost_memory_region *mr;
+
+	num = get_hugepage_file_info(huges, VHOST_MEMORY_MAX_NREGIONS);
+	if (num < 0) {
+		PMD_INIT_LOG(ERR, "Failed to prepare memory for vhost-user");
+		return -1;
+	}
+
+	for (i = 0; i < num; ++i) {
+		mr = &msg->payload.memory.regions[i];
+		mr->guest_phys_addr = huges[i].addr; /* use vaddr! */
+		mr->userspace_addr = huges[i].addr;
+		mr->memory_size = huges[i].size;
+		mr->mmap_offset = 0;
+		fds[i] = open(huges[i].path, O_RDWR);
+	}
+
+	msg->payload.memory.nregions = num;
+	msg->payload.memory.padding = 0;
+
+	return 0;
+}
+
+static struct vhost_user_msg m;
+
+static const char * const vhost_msg_strings[] = {
+	[VHOST_USER_SET_OWNER] = "VHOST_USER_SET_OWNER",
+	[VHOST_USER_RESET_OWNER] = "VHOST_USER_RESET_OWNER",
+	[VHOST_USER_SET_FEATURES] = "VHOST_USER_SET_FEATURES",
+	[VHOST_USER_GET_FEATURES] = "VHOST_USER_GET_FEATURES",
+	[VHOST_USER_SET_VRING_CALL] = "VHOST_USER_SET_VRING_CALL",
+	[VHOST_USER_SET_VRING_NUM] = "VHOST_USER_SET_VRING_NUM",
+	[VHOST_USER_SET_VRING_BASE] = "VHOST_USER_SET_VRING_BASE",
+	[VHOST_USER_GET_VRING_BASE] = "VHOST_USER_GET_VRING_BASE",
+	[VHOST_USER_SET_VRING_ADDR] = "VHOST_USER_SET_VRING_ADDR",
+	[VHOST_USER_SET_VRING_KICK] = "VHOST_USER_SET_VRING_KICK",
+	[VHOST_USER_SET_MEM_TABLE] = "VHOST_USER_SET_MEM_TABLE",
+	NULL,
+};
+
+int
+vhost_user_sock(int vhostfd, uint64_t req, void *arg)
+{
+	struct vhost_user_msg msg;
+	struct vhost_vring_file *file = 0;
+	int need_reply = 0;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+	int fd_num = 0;
+	int i, len;
+
+	RTE_SET_USED(m);
+	RTE_SET_USED(vhost_msg_strings);
+
+	PMD_DRV_LOG(INFO, "%s", vhost_msg_strings[req]);
+
+	msg.request = req;
+	msg.flags = VHOST_USER_VERSION;
+	msg.size = 0;
+
+	switch (req) {
+	case VHOST_USER_GET_FEATURES:
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_FEATURES:
+	case VHOST_USER_SET_LOG_BASE:
+		msg.payload.u64 = *((__u64 *)arg);
+		msg.size = sizeof(m.payload.u64);
+		break;
+
+	case VHOST_USER_SET_OWNER:
+	case VHOST_USER_RESET_OWNER:
+		break;
+
+	case VHOST_USER_SET_MEM_TABLE:
+		if (prepare_vhost_memory_user(&msg, fds) < 0)
+			return -1;
+		fd_num = msg.payload.memory.nregions;
+		msg.size = sizeof(m.payload.memory.nregions);
+		msg.size += sizeof(m.payload.memory.padding);
+		msg.size += fd_num * sizeof(struct vhost_memory_region);
+		break;
+
+	case VHOST_USER_SET_LOG_FD:
+		fds[fd_num++] = *((int *)arg);
+		break;
+
+	case VHOST_USER_SET_VRING_NUM:
+	case VHOST_USER_SET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		break;
+
+	case VHOST_USER_GET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_VRING_ADDR:
+		memcpy(&msg.payload.addr, arg, sizeof(msg.payload.addr));
+		msg.size = sizeof(m.payload.addr);
+		break;
+
+	case VHOST_USER_SET_VRING_KICK:
+	case VHOST_USER_SET_VRING_CALL:
+	case VHOST_USER_SET_VRING_ERR:
+		file = arg;
+		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
+		msg.size = sizeof(m.payload.u64);
+		if (file->fd > 0)
+			fds[fd_num++] = file->fd;
+		else
+			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
+		break;
+
+	default:
+		PMD_DRV_LOG(ERR, "trying to send unhandled msg type");
+		return -1;
+	}
+
+	len = VHOST_USER_HDR_SIZE + msg.size;
+	if (vhost_user_write(vhostfd, &msg, len, fds, fd_num) < 0) {
+		PMD_DRV_LOG(ERR, "%s failed: %s",
+			    vhost_msg_strings[req], strerror(errno));
+		return -1;
+	}
+
+	if (req == VHOST_USER_SET_MEM_TABLE)
+		for (i = 0; i < fd_num; ++i)
+			close(fds[i]);
+
+	if (need_reply) {
+		if (vhost_user_read(vhostfd, &msg) < 0) {
+			PMD_DRV_LOG(ERR, "Received msg failed",
+				    strerror(errno));
+			return -1;
+		}
+
+		if (req != msg.request) {
+			PMD_DRV_LOG(ERR, "Received unexpected msg type");
+			return -1;
+		}
+
+		switch (req) {
+		case VHOST_USER_GET_FEATURES:
+			if (msg.size != sizeof(m.payload.u64)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size");
+				return -1;
+			}
+			*((__u64 *)arg) = msg.payload.u64;
+			break;
+		case VHOST_USER_GET_VRING_BASE:
+			if (msg.size != sizeof(m.payload.state)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size");
+				return -1;
+			}
+			memcpy(arg, &msg.payload.state,
+			       sizeof(struct vhost_vring_state));
+			break;
+		default:
+			PMD_DRV_LOG(ERR, "Received unexpected msg type");
+			return -1;
+		}
+	}
+
+	return 0;
+}
+
+/**
+ * Set up environment to talk with a vhost user backend.
+ * @param path
+ *   - The path to vhost user unix socket file.
+ *
+ * @return
+ *   - (-1) if fail to set up;
+ *   - (>=0) if successful, and it is the fd to vhostfd.
+ */
+int
+vhost_user_setup(const char *path)
+{
+	int fd;
+	int flag;
+	struct sockaddr_un un;
+
+	fd = socket(AF_UNIX, SOCK_STREAM, 0);
+	if (fd < 0) {
+		PMD_DRV_LOG(ERR, "socket() error, %s", strerror(errno));
+		return -1;
+	}
+
+	flag = fcntl(fd, F_GETFD);
+	fcntl(fd, F_SETFD, flag | FD_CLOEXEC);
+
+	memset(&un, 0, sizeof(un));
+	un.sun_family = AF_UNIX;
+	snprintf(un.sun_path, sizeof(un.sun_path), "%s", path);
+	if (connect(fd, (struct sockaddr *)&un, sizeof(un)) < 0) {
+		PMD_DRV_LOG(ERR, "connect error, %s", strerror(errno));
+		close(fd);
+		return -1;
+	}
+
+	return fd;
+}
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v8 4/6] virtio-user: add device emulation layer APIs
  2016-06-13  6:38 ` [PATCH v8 0/6] virtio support for container Jianfeng Tan
                     ` (2 preceding siblings ...)
  2016-06-13  6:39   ` [PATCH v8 3/6] virtio-user: add vhost user adapter layer Jianfeng Tan
@ 2016-06-13  6:39   ` Jianfeng Tan
  2016-06-13  6:39   ` [PATCH v8 5/6] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
                     ` (2 subsequent siblings)
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-13  6:39 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Two device emulation layer APIs are added for virtio driver to call:
  - virtio_user_start_device()
  - virtio_user_stop_device()
  - virtio_user_dev_init()
  - virtio_user_dev_uninit()

These APIs will get called by virtio driver, and they call vhost adapter
layer APIs to implement the functionality. Besides, this patch defines
a struct named virtio_user_dev to help manage the data stands for this
kind of virtual device.

  ----------------------
  | ------------------ |
  | | virtio driver  | |
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate |-|----> (virtio_user_dev.c, virtio_user_dev.h)
  | |                | |
  | | vhost adapter  | |
  | ------------------ |
  ----------------------
            |
            |
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/Makefile                      |   1 +
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 227 +++++++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.h |  62 +++++++
 3 files changed, 290 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h

diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index c2ed0fa..d37b83d 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -57,6 +57,7 @@ endif
 
 ifeq ($(CONFIG_RTE_VIRTIO_USER),y)
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_dev.c
 endif
 
 # this lib depends upon:
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.c b/drivers/net/virtio/virtio_user/virtio_user_dev.c
new file mode 100644
index 0000000..93cb758
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.c
@@ -0,0 +1,227 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <stdio.h>
+#include <fcntl.h>
+#include <string.h>
+#include <errno.h>
+#include <sys/mman.h>
+#include <unistd.h>
+#include <sys/eventfd.h>
+
+#include "vhost.h"
+#include "virtio_user_dev.h"
+#include "../virtio_ethdev.h"
+
+static int
+virtio_user_kick_queue(struct virtio_user_dev *dev, uint32_t queue_sel)
+{
+	int callfd, kickfd;
+	struct vhost_vring_file file;
+	struct vhost_vring_state state;
+	struct vring *vring = &dev->vrings[queue_sel];
+	struct vhost_vring_addr addr = {
+		.index = queue_sel,
+		.desc_user_addr = (uint64_t)(uintptr_t)vring->desc,
+		.avail_user_addr = (uint64_t)(uintptr_t)vring->avail,
+		.used_user_addr = (uint64_t)(uintptr_t)vring->used,
+		.log_guest_addr = 0,
+		.flags = 0, /* disable log */
+	};
+
+	/* May use invalid flag, but some backend leverages kickfd and callfd as
+	 * criteria to judge if dev is alive. so finally we use real event_fd.
+	 */
+	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (callfd < 0) {
+		PMD_DRV_LOG(ERR, "callfd error, %s\n", strerror(errno));
+		return -1;
+	}
+	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (kickfd < 0) {
+		close(callfd);
+		PMD_DRV_LOG(ERR, "kickfd error, %s\n", strerror(errno));
+		return -1;
+	}
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL come
+	 * firstly because vhost depends on this msg to allocate virtqueue
+	 * pair.
+	 */
+	file.index = queue_sel;
+	file.fd = callfd;
+	vhost_user_sock(dev->vhostfd, VHOST_USER_SET_VRING_CALL, &file);
+	dev->callfds[queue_sel] = callfd;
+
+	state.index = queue_sel;
+	state.num = vring->num;
+	vhost_user_sock(dev->vhostfd, VHOST_USER_SET_VRING_NUM, &state);
+
+	state.num = 0; /* no reservation */
+	vhost_user_sock(dev->vhostfd, VHOST_USER_SET_VRING_BASE, &state);
+
+	vhost_user_sock(dev->vhostfd, VHOST_USER_SET_VRING_ADDR, &addr);
+
+	/* Of all per virtqueue MSGs, make sure VHOST_USER_SET_VRING_KICK comes
+	 * lastly because vhost depends on this msg to judge if
+	 * virtio is ready.
+	 */
+	file.fd = kickfd;
+	vhost_user_sock(dev->vhostfd, VHOST_USER_SET_VRING_KICK, &file);
+	dev->kickfds[queue_sel] = kickfd;
+
+	return 0;
+}
+
+int
+virtio_user_start_device(struct virtio_user_dev *dev)
+{
+	uint64_t features;
+	uint32_t i, queue_sel;
+	int ret;
+
+	/* construct memory region inside each implementation */
+	ret = vhost_user_sock(dev->vhostfd, VHOST_USER_SET_MEM_TABLE, NULL);
+	if (ret < 0)
+		goto error;
+
+	for (i = 0; i < dev->max_queue_pairs; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
+		if (virtio_user_kick_queue(dev, queue_sel) < 0) {
+			PMD_DRV_LOG(INFO, "kick rx vq fails: %u", i);
+			goto error;
+		}
+	}
+	for (i = 0; i < dev->max_queue_pairs; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
+		if (virtio_user_kick_queue(dev, queue_sel) < 0) {
+			PMD_DRV_LOG(INFO, "kick tx vq fails: %u", i);
+			goto error;
+		}
+	}
+
+	/* After setup all virtqueues, we need to set_features so that
+	 * these features can be set into each virtqueue in vhost side.
+	 * And before that, make sure VIRTIO_NET_F_MAC is stripped.
+	 */
+	features = dev->features;
+	features &= ~(1ull << VIRTIO_NET_F_MAC);
+	ret = vhost_user_sock(dev->vhostfd, VHOST_USER_SET_FEATURES, &features);
+	if (ret < 0)
+		goto error;
+	PMD_DRV_LOG(INFO, "set features: %" PRIx64, features);
+
+	return 0;
+error:
+	/* TODO: free resource here or caller to check */
+	return -1;
+}
+
+int virtio_user_stop_device(struct virtio_user_dev *dev)
+{
+	return vhost_user_sock(dev->vhostfd, VHOST_USER_RESET_OWNER, NULL);
+}
+
+static inline void
+parse_mac(struct virtio_user_dev *dev, const char *mac)
+{
+	int i, r;
+	uint32_t tmp[ETHER_ADDR_LEN];
+
+	if (!mac)
+		return;
+
+	r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
+			&tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
+	if (r == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			dev->mac_addr[i] = (uint8_t)tmp[i];
+		dev->mac_specified = 1;
+	} else {
+		/* ignore the wrong mac, use random mac */
+		PMD_DRV_LOG(ERR, "wrong format of mac: %s", mac);
+	}
+}
+
+int
+virtio_user_dev_init(struct virtio_user_dev *dev, char *path, int queues,
+		     int cq, int queue_size, const char *mac)
+{
+	strncpy(dev->path, path, PATH_MAX);
+	dev->max_queue_pairs = queues;
+	dev->queue_pairs = 1; /* mq disabled by default */
+	dev->queue_size = queue_size;
+	dev->mac_specified = 0;
+	parse_mac(dev, mac);
+	dev->vhostfd = -1;
+	/* TODO: cq */
+	RTE_SET_USED(cq);
+
+	dev->vhostfd = vhost_user_setup(dev->path);
+	if (dev->vhostfd < 0) {
+		PMD_INIT_LOG(ERR, "backend set up fails");
+		return -1;
+	}
+	if (vhost_user_sock(dev->vhostfd, VHOST_USER_SET_OWNER, NULL) < 0) {
+		PMD_INIT_LOG(ERR, "set_owner fails: %s", strerror(errno));
+		return -1;
+	}
+
+	if (vhost_user_sock(dev->vhostfd, VHOST_USER_GET_FEATURES,
+			    &dev->features) < 0) {
+		PMD_INIT_LOG(ERR, "get_features failed: %s", strerror(errno));
+		return -1;
+	}
+	if (dev->mac_specified)
+		dev->features |= (1ull << VIRTIO_NET_F_MAC);
+	/* disable it until we support CQ */
+	dev->features &= ~(1ull << VIRTIO_NET_F_CTRL_VQ);
+	dev->features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
+
+	return 0;
+
+}
+
+void
+virtio_user_dev_uninit(struct virtio_user_dev *dev)
+{
+	uint32_t i;
+
+	for (i = 0; i < dev->max_queue_pairs * 2; ++i) {
+		close(dev->callfds[i]);
+		close(dev->kickfds[i]);
+	}
+
+	close(dev->vhostfd);
+}
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.h b/drivers/net/virtio/virtio_user/virtio_user_dev.h
new file mode 100644
index 0000000..68bee37
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.h
@@ -0,0 +1,62 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VIRTIO_USER_DEV_H
+#define _VIRTIO_USER_DEV_H
+
+#include <limits.h>
+#include "../virtio_pci.h"
+#include "../virtio_ring.h"
+
+struct virtio_user_dev {
+	int		vhostfd;
+	int		callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		mac_specified;
+	uint32_t	max_queue_pairs;
+	uint32_t	queue_pairs;
+	uint32_t	queue_size;
+	uint64_t	features;
+	uint8_t		status;
+	uint8_t		mac_addr[ETHER_ADDR_LEN];
+	char		path[PATH_MAX];
+	struct vring	vrings[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+};
+
+int virtio_user_start_device(struct virtio_user_dev *dev);
+int virtio_user_stop_device(struct virtio_user_dev *dev);
+int virtio_user_dev_init(struct virtio_user_dev *dev, char *path, int queues,
+			 int cq, int queue_size, const char *mac);
+void virtio_user_dev_uninit(struct virtio_user_dev *dev);
+
+#endif
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v8 5/6] virtio-user: add new virtual pci driver for virtio
  2016-06-13  6:38 ` [PATCH v8 0/6] virtio support for container Jianfeng Tan
                     ` (3 preceding siblings ...)
  2016-06-13  6:39   ` [PATCH v8 4/6] virtio-user: add device emulation layer APIs Jianfeng Tan
@ 2016-06-13  6:39   ` Jianfeng Tan
  2016-06-13  6:39   ` [PATCH v8 6/6] virtio-user: add a new vdev named virtio-user Jianfeng Tan
  2016-06-14  8:34   ` [PATCH v8 0/6] virtio support for container Yuanhan Liu
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-13  6:39 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch implements another new instance of struct virtio_pci_ops to
drive the virtio-user virtual device. Instead of rd/wr ioport or PCI
configuration space, this virtual pci driver will rd/wr the virtual
device struct virtio_user_hw, and when necessary, invokes APIs provided
by device emulation later to start/stop the device.

  ----------------------
  | ------------------ |
  | | virtio driver  | |----> (virtio_user_ethdev.c)
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate | |
  | |                | |
  | | vhost adapter  | |
  | ------------------ |
  ----------------------
            |
            |
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/Makefile             |   1 +
 drivers/net/virtio/virtio_pci.h         |   1 +
 drivers/net/virtio/virtio_user_ethdev.c | 218 ++++++++++++++++++++++++++++++++
 3 files changed, 220 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user_ethdev.c

diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index d37b83d..43de46c 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -58,6 +58,7 @@ endif
 ifeq ($(CONFIG_RTE_VIRTIO_USER),y)
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_dev.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user_ethdev.c
 endif
 
 # this lib depends upon:
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index 6c7f8d7..dd7693f 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -261,6 +261,7 @@ struct virtio_hw {
 	struct virtio_pci_common_cfg *common_cfg;
 	struct virtio_net_config *dev_cfg;
 	const struct virtio_pci_ops *vtpci_ops;
+	void	    *virtio_user_dev;
 };
 
 /*
diff --git a/drivers/net/virtio/virtio_user_ethdev.c b/drivers/net/virtio/virtio_user_ethdev.c
new file mode 100644
index 0000000..7814514
--- /dev/null
+++ b/drivers/net/virtio/virtio_user_ethdev.c
@@ -0,0 +1,218 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <sys/types.h>
+#include <unistd.h>
+
+#include "virtio_logs.h"
+#include "virtio_pci.h"
+#include "virtqueue.h"
+#include "virtio_user/virtio_user_dev.h"
+
+#define virtio_user_get_dev(hw) \
+	((struct virtio_user_dev *)(hw)->virtio_user_dev);
+
+static void
+virtio_user_read_dev_config(struct virtio_hw *hw, size_t offset,
+		     void *dst, int length)
+{
+	int i;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (offset == offsetof(struct virtio_net_config, mac) &&
+	    length == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			((uint8_t *)dst)[i] = dev->mac_addr[i];
+		return;
+	}
+
+	if (offset == offsetof(struct virtio_net_config, status))
+		*(uint16_t *)dst = dev->status;
+
+	if (offset == offsetof(struct virtio_net_config, max_virtqueue_pairs))
+		*(uint16_t *)dst = dev->max_queue_pairs;
+}
+
+static void
+virtio_user_write_dev_config(struct virtio_hw *hw, size_t offset,
+		      const void *src, int length)
+{
+	int i;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if ((offset == offsetof(struct virtio_net_config, mac)) &&
+	    (length == ETHER_ADDR_LEN))
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			dev->mac_addr[i] = ((const uint8_t *)src)[i];
+	else
+		PMD_DRV_LOG(ERR, "not supported offset=%" PRIu64 ", len=%d\n",
+			    offset, length);
+}
+
+static void
+virtio_user_set_status(struct virtio_hw *hw, uint8_t status)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (status & VIRTIO_CONFIG_STATUS_DRIVER_OK)
+		virtio_user_start_device(dev);
+	dev->status = status;
+}
+
+static void
+virtio_user_reset(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	virtio_user_stop_device(dev);
+}
+
+static uint8_t
+virtio_user_get_status(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	return dev->status;
+}
+
+static uint64_t
+virtio_user_get_features(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	return dev->features;
+}
+
+static void
+virtio_user_set_features(struct virtio_hw *hw, uint64_t features)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	dev->features = features;
+}
+
+static uint8_t
+virtio_user_get_isr(struct virtio_hw *hw __rte_unused)
+{
+	/* When config interrupt happens, driver calls this function to query
+	 * what kinds of change happen. Interrupt mode not supported for now.
+	 */
+	return 0;
+}
+
+static uint16_t
+virtio_user_set_config_irq(struct virtio_hw *hw __rte_unused,
+		    uint16_t vec __rte_unused)
+{
+	return VIRTIO_MSI_NO_VECTOR;
+}
+
+/* This function is to get the queue size, aka, number of descs, of a specified
+ * queue. Different with the VHOST_USER_GET_QUEUE_NUM, which is used to get the
+ * max supported queues.
+ */
+static uint16_t
+virtio_user_get_queue_num(struct virtio_hw *hw, uint16_t queue_id __rte_unused)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	/* Currently, each queue has same queue size */
+	return dev->queue_size;
+}
+
+static int
+virtio_user_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+	uint16_t queue_idx = vq->vq_queue_index;
+	uint64_t desc_addr, avail_addr, used_addr;
+
+	desc_addr = (uintptr_t)vq->vq_ring_virt_mem;
+	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
+	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
+							 ring[vq->vq_nentries]),
+				   VIRTIO_PCI_VRING_ALIGN);
+
+	dev->vrings[queue_idx].num = vq->vq_nentries;
+	dev->vrings[queue_idx].desc = (void *)(uintptr_t)desc_addr;
+	dev->vrings[queue_idx].avail = (void *)(uintptr_t)avail_addr;
+	dev->vrings[queue_idx].used = (void *)(uintptr_t)used_addr;
+
+	return 0;
+}
+
+static void
+virtio_user_del_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	/* For legacy devices, write 0 to VIRTIO_PCI_QUEUE_PFN port, QEMU
+	 * correspondingly stops the ioeventfds, and reset the status of
+	 * the device.
+	 * For modern devices, set queue desc, avail, used in PCI bar to 0,
+	 * not see any more behavior in QEMU.
+	 *
+	 * Here we just care about what information to deliver to vhost-user
+	 * or vhost-kernel. So we just close ioeventfd for now.
+	 */
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	close(dev->callfds[vq->vq_queue_index]);
+	close(dev->kickfds[vq->vq_queue_index]);
+}
+
+static void
+virtio_user_notify_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	uint64_t buf = 1;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (write(dev->kickfds[vq->vq_queue_index], &buf, sizeof(buf)) < 0)
+		PMD_DRV_LOG(ERR, "failed to kick backend: %s\n",
+			    strerror(errno));
+}
+
+static const struct virtio_pci_ops virtio_user_ops = {
+	.read_dev_cfg	= virtio_user_read_dev_config,
+	.write_dev_cfg	= virtio_user_write_dev_config,
+	.reset		= virtio_user_reset,
+	.get_status	= virtio_user_get_status,
+	.set_status	= virtio_user_set_status,
+	.get_features	= virtio_user_get_features,
+	.set_features	= virtio_user_set_features,
+	.get_isr	= virtio_user_get_isr,
+	.set_config_irq	= virtio_user_set_config_irq,
+	.get_queue_num	= virtio_user_get_queue_num,
+	.setup_queue	= virtio_user_setup_queue,
+	.del_queue	= virtio_user_del_queue,
+	.notify_queue	= virtio_user_notify_queue,
+};
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v8 6/6] virtio-user: add a new vdev named virtio-user
  2016-06-13  6:38 ` [PATCH v8 0/6] virtio support for container Jianfeng Tan
                     ` (4 preceding siblings ...)
  2016-06-13  6:39   ` [PATCH v8 5/6] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
@ 2016-06-13  6:39   ` Jianfeng Tan
  2016-06-14  8:34   ` [PATCH v8 0/6] virtio support for container Yuanhan Liu
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-13  6:39 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Add a new virtual device named vhost-user, which can be used just like
eth_ring, eth_null, etc. To reuse the code of original virtio, we do
some adjustment in virtio_ethdev.c, such as remove key _static_ of
eth_virtio_dev_init() so that it can be reused in virtual device; and
we add some check to make sure it will not crash.

Configured parameters include:
  - queues (optional, 1 by default), number of queue pairs, multi-queue
    not supported for now.
  - cq (optional, 0 by default), not supported for now.
  - mac (optional), random value will be given if not specified.
  - queue_size (optional, 256 by default), size of virtqueues.
  - path (madatory), path of vhost user.

When enable CONFIG_RTE_VIRTIO_USER (enabled by default), the compiled
library can be used in both VM and container environment.

Examples:
path_vhost=<path_to_vhost_user> # use vhost-user as a backend

sudo ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
    --socket-mem 0,1024 --no-pci --file-prefix=l2fwd \
    --vdev=virtio-user0,mac=00:01:02:03:04:05,path=$path_vhost -- -p 0x1

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").
 - Cannot work with vhost-net backend.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 doc/guides/rel_notes/release_16_07.rst  |  12 ++
 doc/guides/sample_app_ug/vhost.rst      |  17 +++
 drivers/net/virtio/virtio_ethdev.c      |  19 +--
 drivers/net/virtio/virtio_ethdev.h      |   2 +
 drivers/net/virtio/virtio_user_ethdev.c | 209 ++++++++++++++++++++++++++++++++
 5 files changed, 252 insertions(+), 7 deletions(-)

diff --git a/doc/guides/rel_notes/release_16_07.rst b/doc/guides/rel_notes/release_16_07.rst
index 30e78d4..90bdcd4 100644
--- a/doc/guides/rel_notes/release_16_07.rst
+++ b/doc/guides/rel_notes/release_16_07.rst
@@ -47,6 +47,18 @@ New Features
   * Dropped specific Xen Dom0 code.
   * Dropped specific anonymous mempool code in testpmd.
 
+* **Virtio support for containers.**
+
+  Add a new virtual device, named virtio-user, to support virtio for containers.
+
+  Known limitations:
+
+  * Control queue and multi-queue are not supported yet.
+  * Cannot work with --huge-unlink.
+  * Cannot work with --no-huge.
+  * Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8) hugepages.
+  * Root privilege is a must for sorting hugepages by physical address.
+  * Can only be used with vhost user backend.
 
 Resolved Issues
 ---------------
diff --git a/doc/guides/sample_app_ug/vhost.rst b/doc/guides/sample_app_ug/vhost.rst
index 5f81802..a93e54d 100644
--- a/doc/guides/sample_app_ug/vhost.rst
+++ b/doc/guides/sample_app_ug/vhost.rst
@@ -833,3 +833,20 @@ For example:
 The above message indicates that device 0 has been registered with MAC address cc:bb:bb:bb:bb:bb and VLAN tag 1000.
 Any packets received on the NIC with these values is placed on the devices receive queue.
 When a virtio-net device transmits packets, the VLAN tag is added to the packet by the DPDK vhost sample code.
+
+Running virtio-user with vhost-switch
+-------------------------------------
+
+We can also use virtio-user with vhost-switch now.
+Virtio-user is a virtual device that can be run in a application (container) parallelly with vhost in the same OS,
+aka, there is no need to start a VM. We just run it with a different --file-prefix to avoid startup failure.
+
+.. code-block:: console
+
+    cd ${RTE_SDK}/x86_64-native-linuxapp-gcc/app
+    ./testpmd -c 0x3 -n 4 --socket-mem 1024 --no-pci --file-prefix=virtio-user-testpmd \
+    --vdev=virtio-user0,mac=00:01:02:03:04:05,path=$path_vhost \
+    -- -i --txqflags=0xf01 --disable-hw-vlan
+
+There is no difference on the vhost side.
+Pleae note that there are some limitations (see release note for more information) in the usage of virtio-user.
diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 9ccce79..4523ceb 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -59,7 +59,6 @@
 #include "virtqueue.h"
 #include "virtio_rxtx.h"
 
-static int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
 static int eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev);
 static int  virtio_dev_configure(struct rte_eth_dev *dev);
 static int  virtio_dev_start(struct rte_eth_dev *dev);
@@ -1081,7 +1080,7 @@ rx_func_get(struct rte_eth_dev *eth_dev)
  * This function is based on probe() function in virtio_pci.c
  * It returns 0 on success.
  */
-static int
+int
 eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 {
 	struct virtio_hw *hw = eth_dev->data->dev_private;
@@ -1112,9 +1111,11 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	pci_dev = eth_dev->pci_dev;
 
-	ret = vtpci_init(pci_dev, hw, &dev_flags);
-	if (ret)
-		return ret;
+	if (pci_dev) {
+		ret = vtpci_init(pci_dev, hw, &dev_flags);
+		if (ret)
+			return ret;
+	}
 
 	/* Reset the device although not necessary at startup */
 	vtpci_reset(hw);
@@ -1206,7 +1207,8 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	PMD_INIT_LOG(DEBUG, "hw->max_rx_queues=%d   hw->max_tx_queues=%d",
 			hw->max_rx_queues, hw->max_tx_queues);
-	PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
+	if (pci_dev)
+		PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
 			eth_dev->data->port_id, pci_dev->id.vendor_id,
 			pci_dev->id.device_id);
 
@@ -1494,7 +1496,10 @@ virtio_dev_info_get(struct rte_eth_dev *dev, struct rte_eth_dev_info *dev_info)
 {
 	struct virtio_hw *hw = dev->data->dev_private;
 
-	dev_info->driver_name = dev->driver->pci_drv.name;
+	if (dev->pci_dev)
+		dev_info->driver_name = dev->driver->pci_drv.name;
+	else
+		dev_info->driver_name = "virtio-user PMD";
 	dev_info->max_rx_queues = (uint16_t)hw->max_rx_queues;
 	dev_info->max_tx_queues = (uint16_t)hw->max_tx_queues;
 	dev_info->min_rx_bufsize = VIRTIO_MIN_RX_BUFSIZE;
diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
index 7e77259..2ecec6e 100644
--- a/drivers/net/virtio/virtio_ethdev.h
+++ b/drivers/net/virtio/virtio_ethdev.h
@@ -113,6 +113,8 @@ uint16_t virtio_recv_pkts_vec(void *rx_queue, struct rte_mbuf **rx_pkts,
 uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 		uint16_t nb_pkts);
 
+int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
+
 /*
  * The VIRTIO_NET_F_GUEST_TSO[46] features permit the host to send us
  * frames larger than 1514 bytes. We do not yet support software LRO
diff --git a/drivers/net/virtio/virtio_user_ethdev.c b/drivers/net/virtio/virtio_user_ethdev.c
index 7814514..8f401a3 100644
--- a/drivers/net/virtio/virtio_user_ethdev.c
+++ b/drivers/net/virtio/virtio_user_ethdev.c
@@ -35,6 +35,10 @@
 #include <sys/types.h>
 #include <unistd.h>
 
+#include <rte_malloc.h>
+#include <rte_kvargs.h>
+
+#include "virtio_ethdev.h"
 #include "virtio_logs.h"
 #include "virtio_pci.h"
 #include "virtqueue.h"
@@ -216,3 +220,208 @@ static const struct virtio_pci_ops virtio_user_ops = {
 	.del_queue	= virtio_user_del_queue,
 	.notify_queue	= virtio_user_notify_queue,
 };
+
+static const char *valid_args[] = {
+#define VIRTIO_USER_ARG_QUEUES_NUM     "queues"
+	VIRTIO_USER_ARG_QUEUES_NUM,
+#define VIRTIO_USER_ARG_CQ_NUM         "cq"
+	VIRTIO_USER_ARG_CQ_NUM,
+#define VIRTIO_USER_ARG_MAC            "mac"
+	VIRTIO_USER_ARG_MAC,
+#define VIRTIO_USER_ARG_PATH           "path"
+	VIRTIO_USER_ARG_PATH,
+#define VIRTIO_USER_ARG_QUEUE_SIZE     "queue_size"
+	VIRTIO_USER_ARG_QUEUE_SIZE,
+	NULL
+};
+
+#define VIRTIO_USER_DEF_CQ_EN	0
+#define VIRTIO_USER_DEF_Q_NUM	1
+#define VIRTIO_USER_DEF_Q_SZ	256
+
+static int
+get_string_arg(const char *key __rte_unused,
+	       const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(char **)extra_args = strdup(value);
+
+	return 0;
+}
+
+static int
+get_integer_arg(const char *key __rte_unused,
+		const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(uint64_t *)extra_args = strtoull(value, NULL, 0);
+
+	return 0;
+}
+
+static struct rte_eth_dev *
+virtio_user_eth_dev_alloc(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct rte_eth_dev_data *data;
+	struct virtio_hw *hw;
+	struct virtio_user_dev *dev;
+
+	eth_dev = rte_eth_dev_allocate(name, RTE_ETH_DEV_VIRTUAL);
+	if (!eth_dev) {
+		PMD_INIT_LOG(ERR, "cannot alloc rte_eth_dev");
+		return NULL;
+	}
+
+	data = eth_dev->data;
+
+	hw = rte_zmalloc(NULL, sizeof(*hw), 0);
+	if (!hw) {
+		PMD_INIT_LOG(ERR, "malloc virtio_hw failed");
+		rte_eth_dev_release_port(eth_dev);
+		return NULL;
+	}
+
+	dev = rte_zmalloc(NULL, sizeof(*dev), 0);
+	if (!dev) {
+		PMD_INIT_LOG(ERR, "malloc virtio_user_dev failed");
+		rte_eth_dev_release_port(eth_dev);
+		rte_free(hw);
+		return NULL;
+	}
+
+	hw->vtpci_ops = &virtio_user_ops;
+	hw->use_msix = 0;
+	hw->modern   = 0;
+	hw->virtio_user_dev = dev;
+	data->dev_private = hw;
+	data->numa_node = SOCKET_ID_ANY;
+	data->kdrv = RTE_KDRV_NONE;
+	data->dev_flags = RTE_ETH_DEV_DETACHABLE;
+	eth_dev->pci_dev = NULL;
+	eth_dev->driver = NULL;
+	return eth_dev;
+}
+
+/* Dev initialization routine. Invoked once for each virtio vdev at
+ * EAL init time, see rte_eal_dev_init().
+ * Returns 0 on success.
+ */
+static int
+virtio_user_pmd_devinit(const char *name, const char *params)
+{
+	struct rte_kvargs *kvlist;
+	struct rte_eth_dev *eth_dev;
+	struct virtio_hw *hw;
+	uint64_t queues = VIRTIO_USER_DEF_Q_NUM;
+	uint64_t cq = VIRTIO_USER_DEF_CQ_EN;
+	uint64_t queue_size = VIRTIO_USER_DEF_Q_SZ;
+	char *path = NULL;
+	char *mac_addr = NULL;
+	int ret = -1;
+
+	if (!params || params[0] == '\0') {
+		PMD_INIT_LOG(ERR, "arg %s is mandatory for virtio-user",
+			  VIRTIO_USER_ARG_QUEUE_SIZE);
+		goto end;
+	}
+
+	kvlist = rte_kvargs_parse(params, valid_args);
+	if (!kvlist) {
+		PMD_INIT_LOG(ERR, "error when parsing param");
+		goto end;
+	}
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_PATH) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_PATH,
+				   &get_string_arg, &path);
+	else {
+		PMD_INIT_LOG(ERR, "arg %s is mandatory for virtio-user\n",
+			  VIRTIO_USER_ARG_QUEUE_SIZE);
+		goto end;
+	}
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_MAC) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_MAC,
+				   &get_string_arg, &mac_addr);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_QUEUE_SIZE) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_QUEUE_SIZE,
+				   &get_integer_arg, &queue_size);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_QUEUES_NUM) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_QUEUES_NUM,
+				   &get_integer_arg, &queues);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_CQ_NUM) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_CQ_NUM,
+				   &get_integer_arg, &cq);
+
+	eth_dev = virtio_user_eth_dev_alloc(name);
+	if (!eth_dev) {
+		PMD_INIT_LOG(ERR, "virtio-user fails to alloc device");
+		goto end;
+	}
+
+	hw = eth_dev->data->dev_private;
+	if (virtio_user_dev_init(hw->virtio_user_dev, path, queues, cq,
+				 queue_size, mac_addr) < 0)
+		goto end;
+
+	/* previously called by rte_eal_pci_probe() for physical dev */
+	if (eth_virtio_dev_init(eth_dev) < 0) {
+		PMD_INIT_LOG(ERR, "eth_virtio_dev_init fails");
+		goto end;
+	}
+	ret = 0;
+
+end:
+	if (path)
+		free(path);
+	if (mac_addr)
+		free(mac_addr);
+	return ret;
+}
+
+/** Called by rte_eth_dev_detach() */
+static int
+virtio_user_pmd_devuninit(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct virtio_hw *hw;
+	struct virtio_user_dev *dev;
+
+	if (!name)
+		return -EINVAL;
+
+	PMD_DRV_LOG(INFO, "Un-Initializing %s\n", name);
+	eth_dev = rte_eth_dev_allocated(name);
+	if (!eth_dev)
+		return -ENODEV;
+
+	/* make sure the device is stopped, queues freed */
+	rte_eth_dev_close(eth_dev->data->port_id);
+
+	hw = eth_dev->data->dev_private;
+	dev = hw->virtio_user_dev;
+	virtio_user_dev_uninit(dev);
+
+	rte_free(eth_dev->data->dev_private);
+	rte_free(eth_dev->data);
+	rte_eth_dev_release_port(eth_dev);
+
+	return 0;
+}
+
+static struct rte_driver virtio_user_driver = {
+	.name   = "virtio-user",
+	.type   = PMD_VDEV,
+	.init   = virtio_user_pmd_devinit,
+	.uninit = virtio_user_pmd_devuninit,
+};
+
+PMD_REGISTER_DRIVER(virtio_user_driver);
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* Re: [PATCH v8 0/6] virtio support for container
  2016-06-13  6:38 ` [PATCH v8 0/6] virtio support for container Jianfeng Tan
                     ` (5 preceding siblings ...)
  2016-06-13  6:39   ` [PATCH v8 6/6] virtio-user: add a new vdev named virtio-user Jianfeng Tan
@ 2016-06-14  8:34   ` Yuanhan Liu
  6 siblings, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-06-14  8:34 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

Series Acked-by: Yuanhan Liu <yuanhan.liu@linux.intel.com>

	--yliu

On Mon, Jun 13, 2016 at 06:38:57AM +0000, Jianfeng Tan wrote:
> v8:
>  - Change to use max_queue_pairs instead of queue_pairs to initialize
>    and deinitialize queues.
>  - Remove vhost-kernel support.
> 
> v7:
>  - CONFIG_RTE_VIRTIO_VDEV -> CONFIG_RTE_VIRTIO_USER; and corresondingly,
>    RTE_VIRTIO_VDEV -> RTE_VIRTIO_USER.
>  - uint64_t -> uintptr_t, so that it can be compiled on 32-bit platform.
>  - Rebase on latest dpdk-next-virtio branch.
>  - Abandon abstracting related code into vring_hdr_desc_init(), instead,
>    just move it behind setup_queue().
> 
> v6:
>  - Move driver related code into from driver/net/virtio/virtio-user/ to
>    driver/net/virtio/ directory, inside virtio_user_ethdev.c.
>  - Rename vdev to virtio_user in comments and code.
>  - Merge code, which lies in virtio_user_pci.c, into virtio_user_ethdev.c.
>  - Add some comments at virtio-user special handling at virtio_dev_ethdev.c.
>  - Merge document update into the 7nd commit where virtio-user is added.
>  - Add usage with vhost-switch in vhost.rst.
> 
> v5:
>  - Rename struct virtio_user_hw to struct virtio_user_dev.
>  - Rename "vdev_private" to "virtio_user_dev".
>  - Move special handling into virtio_ethdev.c from queue_setup().
>  - Add vring in virtio_user_dev (remove rte_eth_dev_data), so that
>    device does not depend on driver's data structure (rte_eth_dev_data).
>  - Remove update on doc/guides/nics/overview.rst, because virtio-user has
>    exact feature set with virtio.
>  - Change "unsigned long int" to "uint64_t", "unsigned" to "uint32_t".
>  - Remove unnecessary cast in vdev_read_dev_config().
>  - Add functions in virtio_user_dev.c with prefix of "virtio_user_".
>  - Rebase on virtio-next-virtio.
> 
> v4:
>  - Avoid using dev_type, instead use (eth_dev->pci_device is NULL) to
>    judge if it's virtual device or physical device.
>  - Change the added device name to virtio-user.
>  - Split into vhost_user.c, vhost_kernel.c, vhost.c, virtio_user_pci.c,
>    virtio_user_dev.c.
>  - Move virtio-user specific data from struct virtio_hw into struct
>    virtio_user_hw.
>  - Add support to send reset_owner message.
>  - Change del_queue implementation. (This need more check)
>  - Remove rte_panic(), and superseded with log.
>  - Add reset_owner into virtio_pci_ops.reset.
>  - Merge parameter "rx" and "tx" to "queues" to emliminate confusion.
>  - Move get_features to after set_owner.
>  - Redefine path in virtio_user_hw from char * to char [].
> 
> v3:
>  - Remove --single-file option; do no change at EAL memory.
>  - Remove the added API rte_eal_get_backfile_info(), instead we check all
>    opened files with HUGEFILE_FMT to find hugepage files owned by DPDK.
>  - Accordingly, add more restrictions at "Known issue" section.
>  - Rename parameter from queue_num to queue_size for confusion.
>  - Rename vhost_embedded.c to rte_eth_virtio_vdev.c.
>  - Move code related to the newly added vdev to rte_eth_virtio_vdev.c, to
>    reuse eth_virtio_dev_init(), remove its static declaration.
>  - Implement dev_uninit() for rte_eth_dev_detach().
>  - WARN -> ERR, in vhost_embedded.c
>  - Add more commit message for clarify the model.
> 
> v2:
>  - Rebase on the patchset of virtio 1.0 support.
>  - Fix cannot create non-hugepage memory.
>  - Fix wrong size of memory region when "single-file" is used.
>  - Fix setting of offset in virtqueue to use virtual address.
>  - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
>  - Add mac option to specify the mac address of this virtual device.
>  - Update doc.
> 
> This patchset is to provide high performance networking interface (virtio)
> for container-based DPDK applications. The way of starting DPDK apps in
> containers with ownership of NIC devices exclusively is beyond the scope.
> The basic idea here is to present a new virtual device (named virtio-user),
> which can be discovered and initialized by DPDK. To minimize the change,
> we reuse already-existing virtio PMD code (driver/net/virtio/).
> 
> Background: Previously, we usually use a virtio device in the context of
> QEMU/VM as below pic shows. Virtio nic is emulated in QEMU, and usually
> presented in VM as a PCI device.
> 
>   ------------------
>   |  virtio driver |  ----->  VM
>   ------------------
>         |
>         | ----------> (over PCI bus or MMIO or Channel I/O)
>         |
>   ------------------
>   | device emulate |
>   |                |  ----->  QEMU
>   | vhost adapter  |
>   ------------------
>         |
>         | ----------> (vhost-user protocol or vhost-net ioctls)
>         |
>   ------------------
>   | vhost backend  |
>   ------------------
>  
> Compared to QEMU/VM case, virtio support for contaner requires to embedded
> device framework inside the virtio PMD. So this converged driver actually
> plays three roles:
>   - virtio driver to drive this new kind of virtual device;
>   - device emulation to present this virtual device and reponse to the
>     virtio driver, which is originally by QEMU;
>   - and the role to communicate with vhost backend, which is also
>     originally by QEMU.
> 
> The code layout and functionality of each module:
>  
>   ----------------------
>   | ------------------ |
>   | | virtio driver  | |----> (virtio_user_ethdev.c)
>   | ------------------ |
>   |         |          |
>   | ------------------ | ------>  virtio-user PMD
>   | | device emulate |-|----> (virtio_user_dev.c)
>   | |                | |
>   | | vhost adapter  |-|----> (vhost_user.c, vhost_kernel.c, vhost.c)
>   | ------------------ |
>   ----------------------
>          |
>          | -------------- --> (vhost-user protocol)
>          |
>    ------------------
>    | vhost backend  |
>    ------------------
> 
> How to share memory? In VM's case, qemu always shares all physical layout
> to backend. But it's not feasible for a container, as a process, to share
> all virtual memory regions to backend. So only specified virtual memory
> regions (with type of shared) are sent to backend. It's a limitation that
> only addresses in these areas can be used to transmit or receive packets.
> 
> Known issues:
>  - Control queue and multi-queue are not supported yet.
>  - Cannot work with --huge-unlink.
>  - Cannot work with no-huge.
>  - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
>    hugepages.
>  - Root privilege is a must (mainly becase of sorting hugepages according
>    to physical address).
>  - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").
>  - Cannot work with vhost kernel.
> 
> How to use?
> 
> a. Apply this patchset.
> 
> b. To compile container apps:
> $: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> $: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
> 
> c. To build a docker image using Dockerfile below.
> $: cat ./Dockerfile
> FROM ubuntu:latest
> WORKDIR /usr/src/dpdk
> COPY . /usr/src/dpdk
> ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
> $: docker build -t dpdk-app-l2fwd .
> 
> d. Used with vhost-user
> $: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
> 	--socket-mem 1024,1024 -- -p 0x1 --stats 1
> $: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
> 	-v /dev/hugepages:/dev/hugepages \
> 	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
> 	--vdev=virtio-user0,path=/var/run/usvhost -- -p 0x1
> 
> By the way, it's not necessary to run in a container.
> 
> Signed-off-by: Huawei Xie <huawei.xie@intel.com>
> Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
> 
> 
> Jianfeng Tan (6):
>   virtio: hide phys addr check inside pci ops
>   virtio: enable use virtual address to fill desc
>   virtio-user: add vhost user adapter layer
>   virtio-user: add device emulation layer APIs
>   virtio-user: add new virtual pci driver for virtio
>   virtio-user: add a new vdev named virtio-user
> 
>  config/common_linuxapp                           |   1 +
>  doc/guides/rel_notes/release_16_07.rst           |  12 +
>  doc/guides/sample_app_ug/vhost.rst               |  17 +
>  drivers/net/virtio/Makefile                      |   6 +
>  drivers/net/virtio/virtio_ethdev.c               |  77 ++--
>  drivers/net/virtio/virtio_ethdev.h               |   2 +
>  drivers/net/virtio/virtio_pci.c                  |  30 +-
>  drivers/net/virtio/virtio_pci.h                  |   3 +-
>  drivers/net/virtio/virtio_rxtx.c                 |   5 +-
>  drivers/net/virtio/virtio_rxtx_simple.c          |  13 +-
>  drivers/net/virtio/virtio_user/vhost.h           | 141 ++++++++
>  drivers/net/virtio/virtio_user/vhost_user.c      | 404 +++++++++++++++++++++
>  drivers/net/virtio/virtio_user/virtio_user_dev.c | 227 ++++++++++++
>  drivers/net/virtio/virtio_user/virtio_user_dev.h |  62 ++++
>  drivers/net/virtio/virtio_user_ethdev.c          | 427 +++++++++++++++++++++++
>  drivers/net/virtio/virtqueue.h                   |  10 +
>  16 files changed, 1395 insertions(+), 42 deletions(-)
>  create mode 100644 drivers/net/virtio/virtio_user/vhost.h
>  create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c
>  create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
>  create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h
>  create mode 100644 drivers/net/virtio/virtio_user_ethdev.c
> 
> -- 
> 2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v9 0/6] virtio support for container
  2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
                   ` (13 preceding siblings ...)
  2016-06-13  6:38 ` [PATCH v8 0/6] virtio support for container Jianfeng Tan
@ 2016-06-15  9:03 ` Jianfeng Tan
  2016-06-15  9:03   ` [PATCH v9 1/6] virtio: hide phys addr check inside PCI ops Jianfeng Tan
                     ` (6 more replies)
  14 siblings, 7 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-15  9:03 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

v9:
 - Squash a patch in mq support of virtio-user to use virtual address
   in control queue inside this series.
 - A regression bug fix, missed "%s" when printing some message.
 - Run check-git-log.sh, change pci to PCI.

v8:
 - Change to use max_queue_pairs instead of queue_pairs to initialize
   and deinitialize queues.
 - Remove vhost-kernel support.

v7:
 - CONFIG_RTE_VIRTIO_VDEV -> CONFIG_RTE_VIRTIO_USER; and corresondingly,
   RTE_VIRTIO_VDEV -> RTE_VIRTIO_USER.
 - uint64_t -> uintptr_t, so that it can be compiled on 32-bit platform.
 - Rebase on latest dpdk-next-virtio branch.
 - Abandon abstracting related code into vring_hdr_desc_init(), instead,
   just move it behind setup_queue().

v6:
 - Move driver related code into from driver/net/virtio/virtio-user/ to
   driver/net/virtio/ directory, inside virtio_user_ethdev.c.
 - Rename vdev to virtio_user in comments and code.
 - Merge code, which lies in virtio_user_pci.c, into virtio_user_ethdev.c.
 - Add some comments at virtio-user special handling at virtio_dev_ethdev.c.
 - Merge document update into the 7nd commit where virtio-user is added.
 - Add usage with vhost-switch in vhost.rst.

v5:
 - Rename struct virtio_user_hw to struct virtio_user_dev.
 - Rename "vdev_private" to "virtio_user_dev".
 - Move special handling into virtio_ethdev.c from queue_setup().
 - Add vring in virtio_user_dev (remove rte_eth_dev_data), so that
   device does not depend on driver's data structure (rte_eth_dev_data).
 - Remove update on doc/guides/nics/overview.rst, because virtio-user has
   exact feature set with virtio.
 - Change "unsigned long int" to "uint64_t", "unsigned" to "uint32_t".
 - Remove unnecessary cast in vdev_read_dev_config().
 - Add functions in virtio_user_dev.c with prefix of "virtio_user_".
 - Rebase on virtio-next-virtio.

v4:
 - Avoid using dev_type, instead use (eth_dev->pci_device is NULL) to
   judge if it's virtual device or physical device.
 - Change the added device name to virtio-user.
 - Split into vhost_user.c, vhost_kernel.c, vhost.c, virtio_user_pci.c,
   virtio_user_dev.c.
 - Move virtio-user specific data from struct virtio_hw into struct
   virtio_user_hw.
 - Add support to send reset_owner message.
 - Change del_queue implementation. (This need more check)
 - Remove rte_panic(), and superseded with log.
 - Add reset_owner into virtio_pci_ops.reset.
 - Merge parameter "rx" and "tx" to "queues" to emliminate confusion.
 - Move get_features to after set_owner.
 - Redefine path in virtio_user_hw from char * to char [].

v3:
 - Remove --single-file option; do no change at EAL memory.
 - Remove the added API rte_eal_get_backfile_info(), instead we check all
   opened files with HUGEFILE_FMT to find hugepage files owned by DPDK.
 - Accordingly, add more restrictions at "Known issue" section.
 - Rename parameter from queue_num to queue_size for confusion.
 - Rename vhost_embedded.c to rte_eth_virtio_vdev.c.
 - Move code related to the newly added vdev to rte_eth_virtio_vdev.c, to
   reuse eth_virtio_dev_init(), remove its static declaration.
 - Implement dev_uninit() for rte_eth_dev_detach().
 - WARN -> ERR, in vhost_embedded.c
 - Add more commit message for clarify the model.

v2:
 - Rebase on the patchset of virtio 1.0 support.
 - Fix cannot create non-hugepage memory.
 - Fix wrong size of memory region when "single-file" is used.
 - Fix setting of offset in virtqueue to use virtual address.
 - Fix setting TUNSETVNETHDRSZ in vhost-user's branch.
 - Add mac option to specify the mac address of this virtual device.
 - Update doc.

This patchset is to provide high performance networking interface (virtio)
for container-based DPDK applications. The way of starting DPDK apps in
containers with ownership of NIC devices exclusively is beyond the scope.
The basic idea here is to present a new virtual device (named virtio-user),
which can be discovered and initialized by DPDK. To minimize the change,
we reuse already-existing virtio PMD code (driver/net/virtio/).

Background: Previously, we usually use a virtio device in the context of
QEMU/VM as below pic shows. Virtio nic is emulated in QEMU, and usually
presented in VM as a PCI device.

  ------------------
  |  virtio driver |  ----->  VM
  ------------------
        |
        | ----------> (over PCI bus or MMIO or Channel I/O)
        |
  ------------------
  | device emulate |
  |                |  ----->  QEMU
  | vhost adapter  |
  ------------------
        |
        | ----------> (vhost-user protocol or vhost-net ioctls)
        |
  ------------------
  | vhost backend  |
  ------------------
 
Compared to QEMU/VM case, virtio support for contaner requires to embedded
device framework inside the virtio PMD. So this converged driver actually
plays three roles:
  - virtio driver to drive this new kind of virtual device;
  - device emulation to present this virtual device and reponse to the
    virtio driver, which is originally by QEMU;
  - and the role to communicate with vhost backend, which is also
    originally by QEMU.

The code layout and functionality of each module:
 
  ----------------------
  | ------------------ |
  | | virtio driver  | |----> (virtio_user_ethdev.c)
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate |-|----> (virtio_user_dev.c)
  | |                | |
  | | vhost adapter  |-|----> (vhost_user.c, vhost_kernel.c, vhost.c)
  | ------------------ |
  ----------------------
         |
         | -------------- --> (vhost-user protocol)
         |
   ------------------
   | vhost backend  |
   ------------------

How to share memory? In VM's case, qemu always shares all physical layout
to backend. But it's not feasible for a container, as a process, to share
all virtual memory regions to backend. So only specified virtual memory
regions (with type of shared) are sent to backend. It's a limitation that
only addresses in these areas can be used to transmit or receive packets.

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").
 - Cannot work with vhost kernel.

How to use?

a. Apply this patchset.

b. To compile container apps:
$: make config RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make install RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/l2fwd RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc
$: make -C examples/vhost RTE_SDK=`pwd` T=x86_64-native-linuxapp-gcc

c. To build a docker image using Dockerfile below.
$: cat ./Dockerfile
FROM ubuntu:latest
WORKDIR /usr/src/dpdk
COPY . /usr/src/dpdk
ENV PATH "$PATH:/usr/src/dpdk/examples/l2fwd/build/"
$: docker build -t dpdk-app-l2fwd .

d. Used with vhost-user
$: ./examples/vhost/build/vhost-switch -c 3 -n 4 \
	--socket-mem 1024,1024 -- -p 0x1 --stats 1
$: docker run -i -t -v <path_to_vhost_unix_socket>:/var/run/usvhost \
	-v /dev/hugepages:/dev/hugepages \
	dpdk-app-l2fwd l2fwd -c 0x4 -n 4 -m 1024 --no-pci \
	--vdev=virtio-user0,path=/var/run/usvhost -- -p 0x1

By the way, it's not necessary to run in a container.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Yuanhan Liu <yuanhan.liu@linux.intel.com>

Jianfeng Tan (6):
  virtio: hide phys addr check inside PCI ops
  virtio: enable use virtual address to fill desc
  virtio-user: add vhost user adapter layer
  virtio-user: add device emulation layer APIs
  virtio-user: add new virtual PCI driver for virtio
  virtio-user: add a new vdev named virtio-user

 config/common_linuxapp                           |   1 +
 doc/guides/rel_notes/release_16_07.rst           |  12 +
 doc/guides/sample_app_ug/vhost.rst               |  17 +
 drivers/net/virtio/Makefile                      |   6 +
 drivers/net/virtio/virtio_ethdev.c               |  83 +++--
 drivers/net/virtio/virtio_ethdev.h               |   2 +
 drivers/net/virtio/virtio_pci.c                  |  30 +-
 drivers/net/virtio/virtio_pci.h                  |   3 +-
 drivers/net/virtio/virtio_rxtx.c                 |   5 +-
 drivers/net/virtio/virtio_rxtx_simple.c          |  13 +-
 drivers/net/virtio/virtio_user/vhost.h           | 141 ++++++++
 drivers/net/virtio/virtio_user/vhost_user.c      | 404 +++++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 227 ++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.h |  62 ++++
 drivers/net/virtio/virtio_user_ethdev.c          | 427 +++++++++++++++++++++++
 drivers/net/virtio/virtqueue.h                   |  10 +
 16 files changed, 1398 insertions(+), 45 deletions(-)
 create mode 100644 drivers/net/virtio/virtio_user/vhost.h
 create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h
 create mode 100644 drivers/net/virtio/virtio_user_ethdev.c

-- 
2.1.4

^ permalink raw reply	[flat|nested] 196+ messages in thread

* [PATCH v9 1/6] virtio: hide phys addr check inside PCI ops
  2016-06-15  9:03 ` [PATCH v9 " Jianfeng Tan
@ 2016-06-15  9:03   ` Jianfeng Tan
  2016-06-15  9:03   ` [PATCH v9 2/6] virtio: enable use virtual address to fill desc Jianfeng Tan
                     ` (5 subsequent siblings)
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-15  9:03 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is to move phys addr check from virtio_dev_queue_setup
to pci ops. To makt that happen, make sure virtio_ops.setup_queue
return the result if we pass through the check.

Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Acked-by: Yuanhan Liu <yuanhan.liu@linux.intel.com>
---
 drivers/net/virtio/virtio_ethdev.c | 17 ++++++-----------
 drivers/net/virtio/virtio_pci.c    | 30 ++++++++++++++++++++++++++++--
 drivers/net/virtio/virtio_pci.h    |  2 +-
 3 files changed, 35 insertions(+), 14 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index ad0f5a6..53faa46 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -388,16 +388,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		}
 	}
 
-	/*
-	 * Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
-	 * and only accepts 32 bit page frame number.
-	 * Check if the allocated physical memory exceeds 16TB.
-	 */
-	if ((mz->phys_addr + vq->vq_ring_size - 1) >> (VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
-		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
-		ret = -ENOMEM;
-		goto fail_q_alloc;
-	}
 	memset(mz->addr, 0, sizeof(mz->len));
 
 	vq->vq_ring_mem = mz->phys_addr;
@@ -482,7 +472,12 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		*pvq = cvq;
 	}
 
-	hw->vtpci_ops->setup_queue(hw, vq);
+	if (hw->vtpci_ops->setup_queue(hw, vq) < 0) {
+		PMD_INIT_LOG(ERR, "setup_queue failed");
+		virtio_dev_queue_release(vq);
+		return -EINVAL;
+	}
+
 	vq->configured = 1;
 	return 0;
 
diff --git a/drivers/net/virtio/virtio_pci.c b/drivers/net/virtio/virtio_pci.c
index 60ec4da..02f605d 100644
--- a/drivers/net/virtio/virtio_pci.c
+++ b/drivers/net/virtio/virtio_pci.c
@@ -55,6 +55,22 @@
  */
 #define VIRTIO_PCI_CONFIG(hw) (((hw)->use_msix) ? 24 : 20)
 
+static inline int
+check_vq_phys_addr_ok(struct virtqueue *vq)
+{
+	/* Virtio PCI device VIRTIO_PCI_QUEUE_PF register is 32bit,
+	 * and only accepts 32 bit page frame number.
+	 * Check if the allocated physical memory exceeds 16TB.
+	 */
+	if ((vq->vq_ring_mem + vq->vq_ring_size - 1) >>
+			(VIRTIO_PCI_QUEUE_ADDR_SHIFT + 32)) {
+		PMD_INIT_LOG(ERR, "vring address shouldn't be above 16TB!");
+		return 0;
+	}
+
+	return 1;
+}
+
 static void
 legacy_read_dev_config(struct virtio_hw *hw, size_t offset,
 		       void *dst, int length)
@@ -143,15 +159,20 @@ legacy_get_queue_num(struct virtio_hw *hw, uint16_t queue_id)
 	return dst;
 }
 
-static void
+static int
 legacy_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 {
 	uint32_t src;
 
+	if (!check_vq_phys_addr_ok(vq))
+		return -1;
+
 	rte_eal_pci_ioport_write(&hw->io, &vq->vq_queue_index, 2,
 			 VIRTIO_PCI_QUEUE_SEL);
 	src = vq->vq_ring_mem >> VIRTIO_PCI_QUEUE_ADDR_SHIFT;
 	rte_eal_pci_ioport_write(&hw->io, &src, 4, VIRTIO_PCI_QUEUE_PFN);
+
+	return 0;
 }
 
 static void
@@ -367,12 +388,15 @@ modern_get_queue_num(struct virtio_hw *hw, uint16_t queue_id)
 	return io_read16(&hw->common_cfg->queue_size);
 }
 
-static void
+static int
 modern_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 {
 	uint64_t desc_addr, avail_addr, used_addr;
 	uint16_t notify_off;
 
+	if (!check_vq_phys_addr_ok(vq))
+		return -1;
+
 	desc_addr = vq->vq_ring_mem;
 	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
 	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
@@ -400,6 +424,8 @@ modern_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
 	PMD_INIT_LOG(DEBUG, "\t used_addr: %" PRIx64, used_addr);
 	PMD_INIT_LOG(DEBUG, "\t notify addr: %p (notify offset: %u)",
 		vq->notify_addr, notify_off);
+
+	return 0;
 }
 
 static void
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index f20468a..6c7f8d7 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -235,7 +235,7 @@ struct virtio_pci_ops {
 	uint16_t (*set_config_irq)(struct virtio_hw *hw, uint16_t vec);
 
 	uint16_t (*get_queue_num)(struct virtio_hw *hw, uint16_t queue_id);
-	void (*setup_queue)(struct virtio_hw *hw, struct virtqueue *vq);
+	int (*setup_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 	void (*del_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 	void (*notify_queue)(struct virtio_hw *hw, struct virtqueue *vq);
 };
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v9 2/6] virtio: enable use virtual address to fill desc
  2016-06-15  9:03 ` [PATCH v9 " Jianfeng Tan
  2016-06-15  9:03   ` [PATCH v9 1/6] virtio: hide phys addr check inside PCI ops Jianfeng Tan
@ 2016-06-15  9:03   ` Jianfeng Tan
  2016-06-15  9:03   ` [PATCH v9 3/6] virtio-user: add vhost user adapter layer Jianfeng Tan
                     ` (4 subsequent siblings)
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-15  9:03 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is related to how to calculate relative address for vhost
backend.

The principle is that: based on one or multiple shared memory regions,
vhost maintains a reference system with the frontend start address,
backend start address, and length for each segment, so that each
frontend address (GPA, Guest Physical Address) can be translated into
vhost-recognizable backend address. To make the address translation
efficient, we need to maintain as few regions as possible. In the case
of VM, GPA is always locally continuous. But for some other case, like
virtio-user, we use virtual address here.

It basically means:
  a. when set_base_addr, VA address is used;
  b. when preparing RX's descriptors, VA address is used;
  c. when transmitting packets, VA is filled in TX's descriptors;
  d. in TX and CQ's header, VA is used.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
Acked-by: Yuanhan Liu <yuanhan.liu@linux.intel.com>
---
 drivers/net/virtio/virtio_ethdev.c      | 49 ++++++++++++++++++++++-----------
 drivers/net/virtio/virtio_rxtx.c        |  5 ++--
 drivers/net/virtio/virtio_rxtx_simple.c | 13 +++++----
 drivers/net/virtio/virtqueue.h          | 10 +++++++
 4 files changed, 52 insertions(+), 25 deletions(-)

diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 53faa46..5fd9f51 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -188,14 +188,14 @@ virtio_send_command(struct virtnet_ctl *cvq, struct virtio_pmd_ctrl *ctrl,
 	 * One RX packet for ACK.
 	 */
 	vq->vq_ring.desc[head].flags = VRING_DESC_F_NEXT;
-	vq->vq_ring.desc[head].addr = cvq->virtio_net_hdr_mz->phys_addr;
+	vq->vq_ring.desc[head].addr = cvq->virtio_net_hdr_mem;
 	vq->vq_ring.desc[head].len = sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_free_cnt--;
 	i = vq->vq_ring.desc[head].next;
 
 	for (k = 0; k < pkt_num; k++) {
 		vq->vq_ring.desc[i].flags = VRING_DESC_F_NEXT;
-		vq->vq_ring.desc[i].addr = cvq->virtio_net_hdr_mz->phys_addr
+		vq->vq_ring.desc[i].addr = cvq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr)
 			+ sizeof(ctrl->status) + sizeof(uint8_t)*sum;
 		vq->vq_ring.desc[i].len = dlen[k];
@@ -205,7 +205,7 @@ virtio_send_command(struct virtnet_ctl *cvq, struct virtio_pmd_ctrl *ctrl,
 	}
 
 	vq->vq_ring.desc[i].flags = VRING_DESC_F_WRITE;
-	vq->vq_ring.desc[i].addr = cvq->virtio_net_hdr_mz->phys_addr
+	vq->vq_ring.desc[i].addr = cvq->virtio_net_hdr_mem
 			+ sizeof(struct virtio_net_ctrl_hdr);
 	vq->vq_ring.desc[i].len = sizeof(ctrl->status);
 	vq->vq_free_cnt--;
@@ -434,9 +434,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		rxvq->mz = mz;
 		*pvq = rxvq;
 	} else if (queue_type == VTNET_TQ) {
-		struct virtio_tx_region *txr;
-		unsigned int i;
-
 		txvq = (struct virtnet_tx *)RTE_PTR_ADD(vq, sz_vq);
 		txvq->vq = vq;
 		txvq->port_id = dev->data->port_id;
@@ -445,6 +442,36 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 		txvq->virtio_net_hdr_mz = hdr_mz;
 		txvq->virtio_net_hdr_mem = hdr_mz->phys_addr;
 
+		*pvq = txvq;
+	} else if (queue_type == VTNET_CQ) {
+		cvq = (struct virtnet_ctl *)RTE_PTR_ADD(vq, sz_vq);
+		cvq->vq = vq;
+		cvq->mz = mz;
+		cvq->virtio_net_hdr_mz = hdr_mz;
+		cvq->virtio_net_hdr_mem = hdr_mz->phys_addr;
+		memset(cvq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
+		*pvq = cvq;
+	}
+
+	/* For virtio-user case (that is when dev->pci_dev is NULL), we use
+	 * virtual address. And we need properly set _offset_, please see
+	 * MBUF_DATA_DMA_ADDR in virtqueue.h for more information.
+	 */
+	if (dev->pci_dev)
+		vq->offset = offsetof(struct rte_mbuf, buf_physaddr);
+	else {
+		vq->vq_ring_mem = (uintptr_t)mz->addr;
+		vq->offset = offsetof(struct rte_mbuf, buf_addr);
+		if (queue_type == VTNET_TQ)
+			txvq->virtio_net_hdr_mem = (uintptr_t)hdr_mz->addr;
+		else if (queue_type == VTNET_CQ)
+			cvq->virtio_net_hdr_mem = (uintptr_t)hdr_mz->addr;
+	}
+
+	if (queue_type == VTNET_TQ) {
+		struct virtio_tx_region *txr;
+		unsigned int i;
+
 		txr = hdr_mz->addr;
 		memset(txr, 0, vq_size * sizeof(*txr));
 		for (i = 0; i < vq_size; i++) {
@@ -460,16 +487,6 @@ int virtio_dev_queue_setup(struct rte_eth_dev *dev,
 			start_dp->len = hw->vtnet_hdr_size;
 			start_dp->flags = VRING_DESC_F_NEXT;
 		}
-
-		*pvq = txvq;
-	} else if (queue_type == VTNET_CQ) {
-		cvq = (struct virtnet_ctl *)RTE_PTR_ADD(vq, sz_vq);
-		cvq->vq = vq;
-		cvq->mz = mz;
-		cvq->virtio_net_hdr_mz = hdr_mz;
-		cvq->virtio_net_hdr_mem = hdr_mz->phys_addr;
-		memset(cvq->virtio_net_hdr_mz->addr, 0, PAGE_SIZE);
-		*pvq = cvq;
 	}
 
 	if (hw->vtpci_ops->setup_queue(hw, vq) < 0) {
diff --git a/drivers/net/virtio/virtio_rxtx.c b/drivers/net/virtio/virtio_rxtx.c
index 3c69110..a27208e 100644
--- a/drivers/net/virtio/virtio_rxtx.c
+++ b/drivers/net/virtio/virtio_rxtx.c
@@ -193,8 +193,7 @@ virtqueue_enqueue_recv_refill(struct virtqueue *vq, struct rte_mbuf *cookie)
 
 	start_dp = vq->vq_ring.desc;
 	start_dp[idx].addr =
-		(uint64_t)(cookie->buf_physaddr + RTE_PKTMBUF_HEADROOM
-		- hw->vtnet_hdr_size);
+		MBUF_DATA_DMA_ADDR(cookie, vq->offset) - hw->vtnet_hdr_size;
 	start_dp[idx].len =
 		cookie->buf_len - RTE_PKTMBUF_HEADROOM + hw->vtnet_hdr_size;
 	start_dp[idx].flags =  VRING_DESC_F_WRITE;
@@ -266,7 +265,7 @@ virtqueue_enqueue_xmit(struct virtnet_tx *txvq, struct rte_mbuf *cookie,
 	}
 
 	do {
-		start_dp[idx].addr  = rte_mbuf_data_dma_addr(cookie);
+		start_dp[idx].addr  = MBUF_DATA_DMA_ADDR(cookie, vq->offset);
 		start_dp[idx].len   = cookie->data_len;
 		start_dp[idx].flags = cookie->next ? VRING_DESC_F_NEXT : 0;
 		idx = start_dp[idx].next;
diff --git a/drivers/net/virtio/virtio_rxtx_simple.c b/drivers/net/virtio/virtio_rxtx_simple.c
index fdd655d..7b50119 100644
--- a/drivers/net/virtio/virtio_rxtx_simple.c
+++ b/drivers/net/virtio/virtio_rxtx_simple.c
@@ -80,8 +80,8 @@ virtqueue_enqueue_recv_refill_simple(struct virtqueue *vq,
 	vq->sw_ring[desc_idx] = cookie;
 
 	start_dp = vq->vq_ring.desc;
-	start_dp[desc_idx].addr = (uint64_t)((uintptr_t)cookie->buf_physaddr +
-		RTE_PKTMBUF_HEADROOM - vq->hw->vtnet_hdr_size);
+	start_dp[desc_idx].addr = MBUF_DATA_DMA_ADDR(cookie, vq->offset) -
+				  vq->hw->vtnet_hdr_size;
 	start_dp[desc_idx].len = cookie->buf_len -
 		RTE_PKTMBUF_HEADROOM + vq->hw->vtnet_hdr_size;
 
@@ -120,8 +120,8 @@ virtio_rxq_rearm_vec(struct virtnet_rx *rxvq)
 		*(uint64_t *)p = rxvq->mbuf_initializer;
 
 		start_dp[i].addr =
-			(uint64_t)((uintptr_t)sw_ring[i]->buf_physaddr +
-			RTE_PKTMBUF_HEADROOM - vq->hw->vtnet_hdr_size);
+			MBUF_DATA_DMA_ADDR(sw_ring[i], vq->offset) -
+			vq->hw->vtnet_hdr_size;
 		start_dp[i].len = sw_ring[i]->buf_len -
 			RTE_PKTMBUF_HEADROOM + vq->hw->vtnet_hdr_size;
 	}
@@ -369,7 +369,7 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 			vq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 		for (i = 0; i < nb_tail; i++) {
 			start_dp[desc_idx].addr =
-				rte_mbuf_data_dma_addr(*tx_pkts);
+				MBUF_DATA_DMA_ADDR(*tx_pkts, vq->offset);
 			start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 			tx_pkts++;
 			desc_idx++;
@@ -380,7 +380,8 @@ virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 	for (i = 0; i < nb_commit; i++)
 		vq->vq_descx[desc_idx + i].cookie = tx_pkts[i];
 	for (i = 0; i < nb_commit; i++) {
-		start_dp[desc_idx].addr = rte_mbuf_data_dma_addr(*tx_pkts);
+		start_dp[desc_idx].addr =
+			MBUF_DATA_DMA_ADDR(*tx_pkts, vq->offset);
 		start_dp[desc_idx].len = (*tx_pkts)->pkt_len;
 		tx_pkts++;
 		desc_idx++;
diff --git a/drivers/net/virtio/virtqueue.h b/drivers/net/virtio/virtqueue.h
index 17c1ea1..455aaaf 100644
--- a/drivers/net/virtio/virtqueue.h
+++ b/drivers/net/virtio/virtqueue.h
@@ -66,6 +66,14 @@ struct rte_mbuf;
 
 #define VIRTQUEUE_MAX_NAME_SZ 32
 
+#ifdef RTE_VIRTIO_USER
+#define MBUF_DATA_DMA_ADDR(mb, offset) \
+	((uint64_t)((uintptr_t)(*(void **)((uintptr_t)mb + offset)) \
+			+ (mb)->data_off))
+#else /* RTE_VIRTIO_USER */
+#define MBUF_DATA_DMA_ADDR(mb, offset) rte_mbuf_data_dma_addr(mb)
+#endif /* RTE_VIRTIO_USER */
+
 #define VTNET_SQ_RQ_QUEUE_IDX 0
 #define VTNET_SQ_TQ_QUEUE_IDX 1
 #define VTNET_SQ_CQ_QUEUE_IDX 2
@@ -175,6 +183,7 @@ struct virtqueue {
 	unsigned int vq_ring_size;
 
 	phys_addr_t vq_ring_mem; /**< physical address of vring */
+				/**< use virtual address for virtio-user. */
 
 	/**
 	 * Head of the free chain in the descriptor table. If
@@ -184,6 +193,7 @@ struct virtqueue {
 	uint16_t  vq_desc_head_idx;
 	uint16_t  vq_desc_tail_idx;
 	uint16_t  vq_queue_index;   /**< PCI queue index */
+	uint16_t offset; /**< relative offset to obtain addr in mbuf */
 	uint16_t  *notify_addr;
 	int configured;
 	struct rte_mbuf **sw_ring;  /**< RX software ring. */
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v9 3/6] virtio-user: add vhost user adapter layer
  2016-06-15  9:03 ` [PATCH v9 " Jianfeng Tan
  2016-06-15  9:03   ` [PATCH v9 1/6] virtio: hide phys addr check inside PCI ops Jianfeng Tan
  2016-06-15  9:03   ` [PATCH v9 2/6] virtio: enable use virtual address to fill desc Jianfeng Tan
@ 2016-06-15  9:03   ` Jianfeng Tan
  2016-06-23  9:01     ` Ferruh Yigit
  2016-06-15  9:03   ` [PATCH v9 4/6] virtio-user: add device emulation layer APIs Jianfeng Tan
                     ` (3 subsequent siblings)
  6 siblings, 1 reply; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-15  9:03 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch is to provide vhost adapter layer implementations. Instead
of relying on a hypervisor to translate between device emulation and
vhost backend, here we directly talk with vhost backend through the
vhost file.

Here three main APIs are provided to upper layer (device emulation):
  - vhost_user_setup(), to set up vhost user backend;
  - vhost_user_sock(), to talk with vhost user backend.

  ----------------------
  | ------------------ |
  | | virtio driver  | |
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate | |
  | |                | |
  | | vhost adapter  |-|----> (vhost_user.c)
  | ------------------ |
  ----------------------
            |
            | -------------- --> (vhost-user protocol)
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
Acked-by: Yuanhan Liu <yuanhan.liu@linux.intel.com>
---
 config/common_linuxapp                      |   1 +
 drivers/net/virtio/Makefile                 |   4 +
 drivers/net/virtio/virtio_user/vhost.h      | 141 ++++++++++
 drivers/net/virtio/virtio_user/vhost_user.c | 404 ++++++++++++++++++++++++++++
 4 files changed, 550 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/vhost.h
 create mode 100644 drivers/net/virtio/virtio_user/vhost_user.c

diff --git a/config/common_linuxapp b/config/common_linuxapp
index 7e698e2..2483dfa 100644
--- a/config/common_linuxapp
+++ b/config/common_linuxapp
@@ -43,3 +43,4 @@ CONFIG_RTE_LIBRTE_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_VHOST=y
 CONFIG_RTE_LIBRTE_PMD_AF_PACKET=y
 CONFIG_RTE_LIBRTE_POWER=y
+CONFIG_RTE_VIRTIO_USER=y
diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index ef84f60..c2ed0fa 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -55,6 +55,10 @@ ifeq ($(findstring RTE_MACHINE_CPUFLAG_SSSE3,$(CFLAGS)),RTE_MACHINE_CPUFLAG_SSSE
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_rxtx_simple.c
 endif
 
+ifeq ($(CONFIG_RTE_VIRTIO_USER),y)
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
+endif
+
 # this lib depends upon:
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_eal lib/librte_ether
 DEPDIRS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += lib/librte_mempool lib/librte_mbuf
diff --git a/drivers/net/virtio/virtio_user/vhost.h b/drivers/net/virtio/virtio_user/vhost.h
new file mode 100644
index 0000000..4e04ede
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost.h
@@ -0,0 +1,141 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VHOST_NET_USER_H
+#define _VHOST_NET_USER_H
+
+#include <stdint.h>
+#include <linux/types.h>
+#include <linux/ioctl.h>
+
+#include "../virtio_pci.h"
+#include "../virtio_logs.h"
+#include "../virtqueue.h"
+
+#define VHOST_MEMORY_MAX_NREGIONS 8
+
+struct vhost_vring_state {
+	unsigned int index;
+	unsigned int num;
+};
+
+struct vhost_vring_file {
+	unsigned int index;
+	int fd;
+};
+
+struct vhost_vring_addr {
+	unsigned int index;
+	/* Option flags. */
+	unsigned int flags;
+	/* Flag values: */
+	/* Whether log address is valid. If set enables logging. */
+#define VHOST_VRING_F_LOG 0
+
+	/* Start of array of descriptors (virtually contiguous) */
+	uint64_t desc_user_addr;
+	/* Used structure address. Must be 32 bit aligned */
+	uint64_t used_user_addr;
+	/* Available structure address. Must be 16 bit aligned */
+	uint64_t avail_user_addr;
+	/* Logging support. */
+	/* Log writes to used structure, at offset calculated from specified
+	 * address. Address must be 32 bit aligned.
+	 */
+	uint64_t log_guest_addr;
+};
+
+enum vhost_user_request {
+	VHOST_USER_NONE = 0,
+	VHOST_USER_GET_FEATURES = 1,
+	VHOST_USER_SET_FEATURES = 2,
+	VHOST_USER_SET_OWNER = 3,
+	VHOST_USER_RESET_OWNER = 4,
+	VHOST_USER_SET_MEM_TABLE = 5,
+	VHOST_USER_SET_LOG_BASE = 6,
+	VHOST_USER_SET_LOG_FD = 7,
+	VHOST_USER_SET_VRING_NUM = 8,
+	VHOST_USER_SET_VRING_ADDR = 9,
+	VHOST_USER_SET_VRING_BASE = 10,
+	VHOST_USER_GET_VRING_BASE = 11,
+	VHOST_USER_SET_VRING_KICK = 12,
+	VHOST_USER_SET_VRING_CALL = 13,
+	VHOST_USER_SET_VRING_ERR = 14,
+	VHOST_USER_GET_PROTOCOL_FEATURES = 15,
+	VHOST_USER_SET_PROTOCOL_FEATURES = 16,
+	VHOST_USER_GET_QUEUE_NUM = 17,
+	VHOST_USER_SET_VRING_ENABLE = 18,
+	VHOST_USER_MAX
+};
+
+struct vhost_memory_region {
+	uint64_t guest_phys_addr;
+	uint64_t memory_size; /* bytes */
+	uint64_t userspace_addr;
+	uint64_t mmap_offset;
+};
+
+struct vhost_memory {
+	uint32_t nregions;
+	uint32_t padding;
+	struct vhost_memory_region regions[VHOST_MEMORY_MAX_NREGIONS];
+};
+
+struct vhost_user_msg {
+	enum vhost_user_request request;
+
+#define VHOST_USER_VERSION_MASK     0x3
+#define VHOST_USER_REPLY_MASK       (0x1 << 2)
+	uint32_t flags;
+	uint32_t size; /* the following payload size */
+	union {
+#define VHOST_USER_VRING_IDX_MASK   0xff
+#define VHOST_USER_VRING_NOFD_MASK  (0x1 << 8)
+		uint64_t u64;
+		struct vhost_vring_state state;
+		struct vhost_vring_addr addr;
+		struct vhost_memory memory;
+	} payload;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+} __attribute((packed));
+
+#define VHOST_USER_HDR_SIZE offsetof(struct vhost_user_msg, payload.u64)
+#define VHOST_USER_PAYLOAD_SIZE \
+	(sizeof(struct vhost_user_msg) - VHOST_USER_HDR_SIZE)
+
+/* The version of the protocol we support */
+#define VHOST_USER_VERSION    0x1
+
+int vhost_user_sock(int vhostfd, uint64_t req, void *arg);
+int vhost_user_setup(const char *path);
+#endif
diff --git a/drivers/net/virtio/virtio_user/vhost_user.c b/drivers/net/virtio/virtio_user/vhost_user.c
new file mode 100644
index 0000000..47bbf74
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/vhost_user.c
@@ -0,0 +1,404 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <unistd.h>
+#include <fcntl.h>
+#include <sys/un.h>
+#include <string.h>
+#include <errno.h>
+
+#include "vhost.h"
+
+static int
+vhost_user_write(int fd, void *buf, int len, int *fds, int fd_num)
+{
+	int r;
+	struct msghdr msgh;
+	struct iovec iov;
+	size_t fd_size = fd_num * sizeof(int);
+	char control[CMSG_SPACE(fd_size)];
+	struct cmsghdr *cmsg;
+
+	memset(&msgh, 0, sizeof(msgh));
+	memset(control, 0, sizeof(control));
+
+	iov.iov_base = (uint8_t *)buf;
+	iov.iov_len = len;
+
+	msgh.msg_iov = &iov;
+	msgh.msg_iovlen = 1;
+	msgh.msg_control = control;
+	msgh.msg_controllen = sizeof(control);
+
+	cmsg = CMSG_FIRSTHDR(&msgh);
+	cmsg->cmsg_len = CMSG_LEN(fd_size);
+	cmsg->cmsg_level = SOL_SOCKET;
+	cmsg->cmsg_type = SCM_RIGHTS;
+	memcpy(CMSG_DATA(cmsg), fds, fd_size);
+
+	do {
+		r = sendmsg(fd, &msgh, 0);
+	} while (r < 0 && errno == EINTR);
+
+	return r;
+}
+
+static int
+vhost_user_read(int fd, struct vhost_user_msg *msg)
+{
+	uint32_t valid_flags = VHOST_USER_REPLY_MASK | VHOST_USER_VERSION;
+	int ret, sz_hdr = VHOST_USER_HDR_SIZE, sz_payload;
+
+	ret = recv(fd, (void *)msg, sz_hdr, 0);
+	if (ret < sz_hdr) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg hdr: %d instead of %d.",
+			    ret, sz_hdr);
+		goto fail;
+	}
+
+	/* validate msg flags */
+	if (msg->flags != (valid_flags)) {
+		PMD_DRV_LOG(ERR, "Failed to recv msg: flags %x instead of %x.",
+			    msg->flags, valid_flags);
+		goto fail;
+	}
+
+	sz_payload = msg->size;
+	if (sz_payload) {
+		ret = recv(fd, (void *)((char *)msg + sz_hdr), sz_payload, 0);
+		if (ret < sz_payload) {
+			PMD_DRV_LOG(ERR, "Failed to recv msg payload: %d instead of %d.",
+				    ret, msg->size);
+			goto fail;
+		}
+	}
+
+	return 0;
+
+fail:
+	return -1;
+}
+
+struct hugepage_file_info {
+	uint64_t addr;            /**< virtual addr */
+	size_t   size;            /**< the file size */
+	char     path[PATH_MAX];  /**< path to backing file */
+};
+
+/* Two possible options:
+ * 1. Match HUGEPAGE_INFO_FMT to find the file storing struct hugepage_file
+ * array. This is simple but cannot be used in secondary process because
+ * secondary process will close and munmap that file.
+ * 2. Match HUGEFILE_FMT to find hugepage files directly.
+ *
+ * We choose option 2.
+ */
+static int
+get_hugepage_file_info(struct hugepage_file_info huges[], int max)
+{
+	int idx;
+	FILE *f;
+	char buf[BUFSIZ], *tmp, *tail;
+	char *str_underline, *str_start;
+	int huge_index;
+	uint64_t v_start, v_end;
+
+	f = fopen("/proc/self/maps", "r");
+	if (!f) {
+		PMD_DRV_LOG(ERR, "cannot open /proc/self/maps");
+		return -1;
+	}
+
+	idx = 0;
+	while (fgets(buf, sizeof(buf), f) != NULL) {
+		if (sscanf(buf, "%" PRIx64 "-%" PRIx64, &v_start, &v_end) < 2) {
+			PMD_DRV_LOG(ERR, "Failed to parse address");
+			goto error;
+		}
+
+		tmp = strchr(buf, ' ') + 1; /** skip address */
+		tmp = strchr(tmp, ' ') + 1; /** skip perm */
+		tmp = strchr(tmp, ' ') + 1; /** skip offset */
+		tmp = strchr(tmp, ' ') + 1; /** skip dev */
+		tmp = strchr(tmp, ' ') + 1; /** skip inode */
+		while (*tmp == ' ')         /** skip spaces */
+			tmp++;
+		tail = strrchr(tmp, '\n');  /** remove newline if exists */
+		if (tail)
+			*tail = '\0';
+
+		/* Match HUGEFILE_FMT, aka "%s/%smap_%d",
+		 * which is defined in eal_filesystem.h
+		 */
+		str_underline = strrchr(tmp, '_');
+		if (!str_underline)
+			continue;
+
+		str_start = str_underline - strlen("map");
+		if (str_start < tmp)
+			continue;
+
+		if (sscanf(str_start, "map_%d", &huge_index) != 1)
+			continue;
+
+		if (idx >= max) {
+			PMD_DRV_LOG(ERR, "Exceed maximum of %d", max);
+			goto error;
+		}
+		huges[idx].addr = v_start;
+		huges[idx].size = v_end - v_start;
+		strcpy(huges[idx].path, tmp);
+		idx++;
+	}
+
+	fclose(f);
+	return idx;
+
+error:
+	fclose(f);
+	return -1;
+}
+
+static int
+prepare_vhost_memory_user(struct vhost_user_msg *msg, int fds[])
+{
+	int i, num;
+	struct hugepage_file_info huges[VHOST_MEMORY_MAX_NREGIONS];
+	struct vhost_memory_region *mr;
+
+	num = get_hugepage_file_info(huges, VHOST_MEMORY_MAX_NREGIONS);
+	if (num < 0) {
+		PMD_INIT_LOG(ERR, "Failed to prepare memory for vhost-user");
+		return -1;
+	}
+
+	for (i = 0; i < num; ++i) {
+		mr = &msg->payload.memory.regions[i];
+		mr->guest_phys_addr = huges[i].addr; /* use vaddr! */
+		mr->userspace_addr = huges[i].addr;
+		mr->memory_size = huges[i].size;
+		mr->mmap_offset = 0;
+		fds[i] = open(huges[i].path, O_RDWR);
+	}
+
+	msg->payload.memory.nregions = num;
+	msg->payload.memory.padding = 0;
+
+	return 0;
+}
+
+static struct vhost_user_msg m;
+
+static const char * const vhost_msg_strings[] = {
+	[VHOST_USER_SET_OWNER] = "VHOST_USER_SET_OWNER",
+	[VHOST_USER_RESET_OWNER] = "VHOST_USER_RESET_OWNER",
+	[VHOST_USER_SET_FEATURES] = "VHOST_USER_SET_FEATURES",
+	[VHOST_USER_GET_FEATURES] = "VHOST_USER_GET_FEATURES",
+	[VHOST_USER_SET_VRING_CALL] = "VHOST_USER_SET_VRING_CALL",
+	[VHOST_USER_SET_VRING_NUM] = "VHOST_USER_SET_VRING_NUM",
+	[VHOST_USER_SET_VRING_BASE] = "VHOST_USER_SET_VRING_BASE",
+	[VHOST_USER_GET_VRING_BASE] = "VHOST_USER_GET_VRING_BASE",
+	[VHOST_USER_SET_VRING_ADDR] = "VHOST_USER_SET_VRING_ADDR",
+	[VHOST_USER_SET_VRING_KICK] = "VHOST_USER_SET_VRING_KICK",
+	[VHOST_USER_SET_MEM_TABLE] = "VHOST_USER_SET_MEM_TABLE",
+	NULL,
+};
+
+int
+vhost_user_sock(int vhostfd, uint64_t req, void *arg)
+{
+	struct vhost_user_msg msg;
+	struct vhost_vring_file *file = 0;
+	int need_reply = 0;
+	int fds[VHOST_MEMORY_MAX_NREGIONS];
+	int fd_num = 0;
+	int i, len;
+
+	RTE_SET_USED(m);
+	RTE_SET_USED(vhost_msg_strings);
+
+	PMD_DRV_LOG(INFO, "%s", vhost_msg_strings[req]);
+
+	msg.request = req;
+	msg.flags = VHOST_USER_VERSION;
+	msg.size = 0;
+
+	switch (req) {
+	case VHOST_USER_GET_FEATURES:
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_FEATURES:
+	case VHOST_USER_SET_LOG_BASE:
+		msg.payload.u64 = *((__u64 *)arg);
+		msg.size = sizeof(m.payload.u64);
+		break;
+
+	case VHOST_USER_SET_OWNER:
+	case VHOST_USER_RESET_OWNER:
+		break;
+
+	case VHOST_USER_SET_MEM_TABLE:
+		if (prepare_vhost_memory_user(&msg, fds) < 0)
+			return -1;
+		fd_num = msg.payload.memory.nregions;
+		msg.size = sizeof(m.payload.memory.nregions);
+		msg.size += sizeof(m.payload.memory.padding);
+		msg.size += fd_num * sizeof(struct vhost_memory_region);
+		break;
+
+	case VHOST_USER_SET_LOG_FD:
+		fds[fd_num++] = *((int *)arg);
+		break;
+
+	case VHOST_USER_SET_VRING_NUM:
+	case VHOST_USER_SET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		break;
+
+	case VHOST_USER_GET_VRING_BASE:
+		memcpy(&msg.payload.state, arg, sizeof(msg.payload.state));
+		msg.size = sizeof(m.payload.state);
+		need_reply = 1;
+		break;
+
+	case VHOST_USER_SET_VRING_ADDR:
+		memcpy(&msg.payload.addr, arg, sizeof(msg.payload.addr));
+		msg.size = sizeof(m.payload.addr);
+		break;
+
+	case VHOST_USER_SET_VRING_KICK:
+	case VHOST_USER_SET_VRING_CALL:
+	case VHOST_USER_SET_VRING_ERR:
+		file = arg;
+		msg.payload.u64 = file->index & VHOST_USER_VRING_IDX_MASK;
+		msg.size = sizeof(m.payload.u64);
+		if (file->fd > 0)
+			fds[fd_num++] = file->fd;
+		else
+			msg.payload.u64 |= VHOST_USER_VRING_NOFD_MASK;
+		break;
+
+	default:
+		PMD_DRV_LOG(ERR, "trying to send unhandled msg type");
+		return -1;
+	}
+
+	len = VHOST_USER_HDR_SIZE + msg.size;
+	if (vhost_user_write(vhostfd, &msg, len, fds, fd_num) < 0) {
+		PMD_DRV_LOG(ERR, "%s failed: %s",
+			    vhost_msg_strings[req], strerror(errno));
+		return -1;
+	}
+
+	if (req == VHOST_USER_SET_MEM_TABLE)
+		for (i = 0; i < fd_num; ++i)
+			close(fds[i]);
+
+	if (need_reply) {
+		if (vhost_user_read(vhostfd, &msg) < 0) {
+			PMD_DRV_LOG(ERR, "Received msg failed: %s",
+				    strerror(errno));
+			return -1;
+		}
+
+		if (req != msg.request) {
+			PMD_DRV_LOG(ERR, "Received unexpected msg type");
+			return -1;
+		}
+
+		switch (req) {
+		case VHOST_USER_GET_FEATURES:
+			if (msg.size != sizeof(m.payload.u64)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size");
+				return -1;
+			}
+			*((__u64 *)arg) = msg.payload.u64;
+			break;
+		case VHOST_USER_GET_VRING_BASE:
+			if (msg.size != sizeof(m.payload.state)) {
+				PMD_DRV_LOG(ERR, "Received bad msg size");
+				return -1;
+			}
+			memcpy(arg, &msg.payload.state,
+			       sizeof(struct vhost_vring_state));
+			break;
+		default:
+			PMD_DRV_LOG(ERR, "Received unexpected msg type");
+			return -1;
+		}
+	}
+
+	return 0;
+}
+
+/**
+ * Set up environment to talk with a vhost user backend.
+ * @param path
+ *   - The path to vhost user unix socket file.
+ *
+ * @return
+ *   - (-1) if fail to set up;
+ *   - (>=0) if successful, and it is the fd to vhostfd.
+ */
+int
+vhost_user_setup(const char *path)
+{
+	int fd;
+	int flag;
+	struct sockaddr_un un;
+
+	fd = socket(AF_UNIX, SOCK_STREAM, 0);
+	if (fd < 0) {
+		PMD_DRV_LOG(ERR, "socket() error, %s", strerror(errno));
+		return -1;
+	}
+
+	flag = fcntl(fd, F_GETFD);
+	fcntl(fd, F_SETFD, flag | FD_CLOEXEC);
+
+	memset(&un, 0, sizeof(un));
+	un.sun_family = AF_UNIX;
+	snprintf(un.sun_path, sizeof(un.sun_path), "%s", path);
+	if (connect(fd, (struct sockaddr *)&un, sizeof(un)) < 0) {
+		PMD_DRV_LOG(ERR, "connect error, %s", strerror(errno));
+		close(fd);
+		return -1;
+	}
+
+	return fd;
+}
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v9 4/6] virtio-user: add device emulation layer APIs
  2016-06-15  9:03 ` [PATCH v9 " Jianfeng Tan
                     ` (2 preceding siblings ...)
  2016-06-15  9:03   ` [PATCH v9 3/6] virtio-user: add vhost user adapter layer Jianfeng Tan
@ 2016-06-15  9:03   ` Jianfeng Tan
  2016-06-15  9:03   ` [PATCH v9 5/6] virtio-user: add new virtual PCI driver for virtio Jianfeng Tan
                     ` (2 subsequent siblings)
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-15  9:03 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Two device emulation layer APIs are added for virtio driver to call:
  - virtio_user_start_device()
  - virtio_user_stop_device()
  - virtio_user_dev_init()
  - virtio_user_dev_uninit()

These APIs will get called by virtio driver, and they call vhost adapter
layer APIs to implement the functionality. Besides, this patch defines
a struct named virtio_user_dev to help manage the data stands for this
kind of virtual device.

  ----------------------
  | ------------------ |
  | | virtio driver  | |
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate |-|----> (virtio_user_dev.c, virtio_user_dev.h)
  | |                | |
  | | vhost adapter  | |
  | ------------------ |
  ----------------------
            |
            |
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
---
 drivers/net/virtio/Makefile                      |   1 +
 drivers/net/virtio/virtio_user/virtio_user_dev.c | 227 +++++++++++++++++++++++
 drivers/net/virtio/virtio_user/virtio_user_dev.h |  62 +++++++
 3 files changed, 290 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.c
 create mode 100644 drivers/net/virtio/virtio_user/virtio_user_dev.h

diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index c2ed0fa..d37b83d 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -57,6 +57,7 @@ endif
 
 ifeq ($(CONFIG_RTE_VIRTIO_USER),y)
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_dev.c
 endif
 
 # this lib depends upon:
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.c b/drivers/net/virtio/virtio_user/virtio_user_dev.c
new file mode 100644
index 0000000..93cb758
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.c
@@ -0,0 +1,227 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <stdio.h>
+#include <fcntl.h>
+#include <string.h>
+#include <errno.h>
+#include <sys/mman.h>
+#include <unistd.h>
+#include <sys/eventfd.h>
+
+#include "vhost.h"
+#include "virtio_user_dev.h"
+#include "../virtio_ethdev.h"
+
+static int
+virtio_user_kick_queue(struct virtio_user_dev *dev, uint32_t queue_sel)
+{
+	int callfd, kickfd;
+	struct vhost_vring_file file;
+	struct vhost_vring_state state;
+	struct vring *vring = &dev->vrings[queue_sel];
+	struct vhost_vring_addr addr = {
+		.index = queue_sel,
+		.desc_user_addr = (uint64_t)(uintptr_t)vring->desc,
+		.avail_user_addr = (uint64_t)(uintptr_t)vring->avail,
+		.used_user_addr = (uint64_t)(uintptr_t)vring->used,
+		.log_guest_addr = 0,
+		.flags = 0, /* disable log */
+	};
+
+	/* May use invalid flag, but some backend leverages kickfd and callfd as
+	 * criteria to judge if dev is alive. so finally we use real event_fd.
+	 */
+	callfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (callfd < 0) {
+		PMD_DRV_LOG(ERR, "callfd error, %s\n", strerror(errno));
+		return -1;
+	}
+	kickfd = eventfd(0, O_CLOEXEC | O_NONBLOCK);
+	if (kickfd < 0) {
+		close(callfd);
+		PMD_DRV_LOG(ERR, "kickfd error, %s\n", strerror(errno));
+		return -1;
+	}
+
+	/* Of all per virtqueue MSGs, make sure VHOST_SET_VRING_CALL come
+	 * firstly because vhost depends on this msg to allocate virtqueue
+	 * pair.
+	 */
+	file.index = queue_sel;
+	file.fd = callfd;
+	vhost_user_sock(dev->vhostfd, VHOST_USER_SET_VRING_CALL, &file);
+	dev->callfds[queue_sel] = callfd;
+
+	state.index = queue_sel;
+	state.num = vring->num;
+	vhost_user_sock(dev->vhostfd, VHOST_USER_SET_VRING_NUM, &state);
+
+	state.num = 0; /* no reservation */
+	vhost_user_sock(dev->vhostfd, VHOST_USER_SET_VRING_BASE, &state);
+
+	vhost_user_sock(dev->vhostfd, VHOST_USER_SET_VRING_ADDR, &addr);
+
+	/* Of all per virtqueue MSGs, make sure VHOST_USER_SET_VRING_KICK comes
+	 * lastly because vhost depends on this msg to judge if
+	 * virtio is ready.
+	 */
+	file.fd = kickfd;
+	vhost_user_sock(dev->vhostfd, VHOST_USER_SET_VRING_KICK, &file);
+	dev->kickfds[queue_sel] = kickfd;
+
+	return 0;
+}
+
+int
+virtio_user_start_device(struct virtio_user_dev *dev)
+{
+	uint64_t features;
+	uint32_t i, queue_sel;
+	int ret;
+
+	/* construct memory region inside each implementation */
+	ret = vhost_user_sock(dev->vhostfd, VHOST_USER_SET_MEM_TABLE, NULL);
+	if (ret < 0)
+		goto error;
+
+	for (i = 0; i < dev->max_queue_pairs; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_RQ_QUEUE_IDX;
+		if (virtio_user_kick_queue(dev, queue_sel) < 0) {
+			PMD_DRV_LOG(INFO, "kick rx vq fails: %u", i);
+			goto error;
+		}
+	}
+	for (i = 0; i < dev->max_queue_pairs; ++i) {
+		queue_sel = 2 * i + VTNET_SQ_TQ_QUEUE_IDX;
+		if (virtio_user_kick_queue(dev, queue_sel) < 0) {
+			PMD_DRV_LOG(INFO, "kick tx vq fails: %u", i);
+			goto error;
+		}
+	}
+
+	/* After setup all virtqueues, we need to set_features so that
+	 * these features can be set into each virtqueue in vhost side.
+	 * And before that, make sure VIRTIO_NET_F_MAC is stripped.
+	 */
+	features = dev->features;
+	features &= ~(1ull << VIRTIO_NET_F_MAC);
+	ret = vhost_user_sock(dev->vhostfd, VHOST_USER_SET_FEATURES, &features);
+	if (ret < 0)
+		goto error;
+	PMD_DRV_LOG(INFO, "set features: %" PRIx64, features);
+
+	return 0;
+error:
+	/* TODO: free resource here or caller to check */
+	return -1;
+}
+
+int virtio_user_stop_device(struct virtio_user_dev *dev)
+{
+	return vhost_user_sock(dev->vhostfd, VHOST_USER_RESET_OWNER, NULL);
+}
+
+static inline void
+parse_mac(struct virtio_user_dev *dev, const char *mac)
+{
+	int i, r;
+	uint32_t tmp[ETHER_ADDR_LEN];
+
+	if (!mac)
+		return;
+
+	r = sscanf(mac, "%x:%x:%x:%x:%x:%x", &tmp[0],
+			&tmp[1], &tmp[2], &tmp[3], &tmp[4], &tmp[5]);
+	if (r == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			dev->mac_addr[i] = (uint8_t)tmp[i];
+		dev->mac_specified = 1;
+	} else {
+		/* ignore the wrong mac, use random mac */
+		PMD_DRV_LOG(ERR, "wrong format of mac: %s", mac);
+	}
+}
+
+int
+virtio_user_dev_init(struct virtio_user_dev *dev, char *path, int queues,
+		     int cq, int queue_size, const char *mac)
+{
+	strncpy(dev->path, path, PATH_MAX);
+	dev->max_queue_pairs = queues;
+	dev->queue_pairs = 1; /* mq disabled by default */
+	dev->queue_size = queue_size;
+	dev->mac_specified = 0;
+	parse_mac(dev, mac);
+	dev->vhostfd = -1;
+	/* TODO: cq */
+	RTE_SET_USED(cq);
+
+	dev->vhostfd = vhost_user_setup(dev->path);
+	if (dev->vhostfd < 0) {
+		PMD_INIT_LOG(ERR, "backend set up fails");
+		return -1;
+	}
+	if (vhost_user_sock(dev->vhostfd, VHOST_USER_SET_OWNER, NULL) < 0) {
+		PMD_INIT_LOG(ERR, "set_owner fails: %s", strerror(errno));
+		return -1;
+	}
+
+	if (vhost_user_sock(dev->vhostfd, VHOST_USER_GET_FEATURES,
+			    &dev->features) < 0) {
+		PMD_INIT_LOG(ERR, "get_features failed: %s", strerror(errno));
+		return -1;
+	}
+	if (dev->mac_specified)
+		dev->features |= (1ull << VIRTIO_NET_F_MAC);
+	/* disable it until we support CQ */
+	dev->features &= ~(1ull << VIRTIO_NET_F_CTRL_VQ);
+	dev->features &= ~(1ull << VIRTIO_NET_F_CTRL_RX);
+
+	return 0;
+
+}
+
+void
+virtio_user_dev_uninit(struct virtio_user_dev *dev)
+{
+	uint32_t i;
+
+	for (i = 0; i < dev->max_queue_pairs * 2; ++i) {
+		close(dev->callfds[i]);
+		close(dev->kickfds[i]);
+	}
+
+	close(dev->vhostfd);
+}
diff --git a/drivers/net/virtio/virtio_user/virtio_user_dev.h b/drivers/net/virtio/virtio_user/virtio_user_dev.h
new file mode 100644
index 0000000..68bee37
--- /dev/null
+++ b/drivers/net/virtio/virtio_user/virtio_user_dev.h
@@ -0,0 +1,62 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#ifndef _VIRTIO_USER_DEV_H
+#define _VIRTIO_USER_DEV_H
+
+#include <limits.h>
+#include "../virtio_pci.h"
+#include "../virtio_ring.h"
+
+struct virtio_user_dev {
+	int		vhostfd;
+	int		callfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		kickfds[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+	int		mac_specified;
+	uint32_t	max_queue_pairs;
+	uint32_t	queue_pairs;
+	uint32_t	queue_size;
+	uint64_t	features;
+	uint8_t		status;
+	uint8_t		mac_addr[ETHER_ADDR_LEN];
+	char		path[PATH_MAX];
+	struct vring	vrings[VIRTIO_MAX_VIRTQUEUES * 2 + 1];
+};
+
+int virtio_user_start_device(struct virtio_user_dev *dev);
+int virtio_user_stop_device(struct virtio_user_dev *dev);
+int virtio_user_dev_init(struct virtio_user_dev *dev, char *path, int queues,
+			 int cq, int queue_size, const char *mac);
+void virtio_user_dev_uninit(struct virtio_user_dev *dev);
+
+#endif
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v9 5/6] virtio-user: add new virtual PCI driver for virtio
  2016-06-15  9:03 ` [PATCH v9 " Jianfeng Tan
                     ` (3 preceding siblings ...)
  2016-06-15  9:03   ` [PATCH v9 4/6] virtio-user: add device emulation layer APIs Jianfeng Tan
@ 2016-06-15  9:03   ` Jianfeng Tan
  2016-06-15  9:03   ` [PATCH v9 6/6] virtio-user: add a new vdev named virtio-user Jianfeng Tan
  2016-06-15  9:54   ` [PATCH v9 0/6] virtio support for container Yuanhan Liu
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-15  9:03 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

This patch implements another new instance of struct virtio_pci_ops to
drive the virtio-user virtual device. Instead of rd/wr ioport or PCI
configuration space, this virtual pci driver will rd/wr the virtual
device struct virtio_user_hw, and when necessary, invokes APIs provided
by device emulation later to start/stop the device.

  ----------------------
  | ------------------ |
  | | virtio driver  | |----> (virtio_user_ethdev.c)
  | ------------------ |
  |         |          |
  | ------------------ | ------>  virtio-user PMD
  | | device emulate | |
  | |                | |
  | | vhost adapter  | |
  | ------------------ |
  ----------------------
            |
            |
            |
   ------------------
   | vhost backend  |
   ------------------

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
Acked-by: Yuanhan Liu <yuanhan.liu@linux.intel.com>
---
 drivers/net/virtio/Makefile             |   1 +
 drivers/net/virtio/virtio_pci.h         |   1 +
 drivers/net/virtio/virtio_user_ethdev.c | 218 ++++++++++++++++++++++++++++++++
 3 files changed, 220 insertions(+)
 create mode 100644 drivers/net/virtio/virtio_user_ethdev.c

diff --git a/drivers/net/virtio/Makefile b/drivers/net/virtio/Makefile
index d37b83d..43de46c 100644
--- a/drivers/net/virtio/Makefile
+++ b/drivers/net/virtio/Makefile
@@ -58,6 +58,7 @@ endif
 ifeq ($(CONFIG_RTE_VIRTIO_USER),y)
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/vhost_user.c
 SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user/virtio_user_dev.c
+SRCS-$(CONFIG_RTE_LIBRTE_VIRTIO_PMD) += virtio_user_ethdev.c
 endif
 
 # this lib depends upon:
diff --git a/drivers/net/virtio/virtio_pci.h b/drivers/net/virtio/virtio_pci.h
index 6c7f8d7..dd7693f 100644
--- a/drivers/net/virtio/virtio_pci.h
+++ b/drivers/net/virtio/virtio_pci.h
@@ -261,6 +261,7 @@ struct virtio_hw {
 	struct virtio_pci_common_cfg *common_cfg;
 	struct virtio_net_config *dev_cfg;
 	const struct virtio_pci_ops *vtpci_ops;
+	void	    *virtio_user_dev;
 };
 
 /*
diff --git a/drivers/net/virtio/virtio_user_ethdev.c b/drivers/net/virtio/virtio_user_ethdev.c
new file mode 100644
index 0000000..7814514
--- /dev/null
+++ b/drivers/net/virtio/virtio_user_ethdev.c
@@ -0,0 +1,218 @@
+/*-
+ *   BSD LICENSE
+ *
+ *   Copyright(c) 2010-2016 Intel Corporation. All rights reserved.
+ *   All rights reserved.
+ *
+ *   Redistribution and use in source and binary forms, with or without
+ *   modification, are permitted provided that the following conditions
+ *   are met:
+ *
+ *     * Redistributions of source code must retain the above copyright
+ *       notice, this list of conditions and the following disclaimer.
+ *     * Redistributions in binary form must reproduce the above copyright
+ *       notice, this list of conditions and the following disclaimer in
+ *       the documentation and/or other materials provided with the
+ *       distribution.
+ *     * Neither the name of Intel Corporation nor the names of its
+ *       contributors may be used to endorse or promote products derived
+ *       from this software without specific prior written permission.
+ *
+ *   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ *   "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+ *   LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ *   A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+ *   OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ *   SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ *   LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ *   DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ *   THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ *   (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+ *   OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+#include <stdint.h>
+#include <sys/types.h>
+#include <unistd.h>
+
+#include "virtio_logs.h"
+#include "virtio_pci.h"
+#include "virtqueue.h"
+#include "virtio_user/virtio_user_dev.h"
+
+#define virtio_user_get_dev(hw) \
+	((struct virtio_user_dev *)(hw)->virtio_user_dev);
+
+static void
+virtio_user_read_dev_config(struct virtio_hw *hw, size_t offset,
+		     void *dst, int length)
+{
+	int i;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (offset == offsetof(struct virtio_net_config, mac) &&
+	    length == ETHER_ADDR_LEN) {
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			((uint8_t *)dst)[i] = dev->mac_addr[i];
+		return;
+	}
+
+	if (offset == offsetof(struct virtio_net_config, status))
+		*(uint16_t *)dst = dev->status;
+
+	if (offset == offsetof(struct virtio_net_config, max_virtqueue_pairs))
+		*(uint16_t *)dst = dev->max_queue_pairs;
+}
+
+static void
+virtio_user_write_dev_config(struct virtio_hw *hw, size_t offset,
+		      const void *src, int length)
+{
+	int i;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if ((offset == offsetof(struct virtio_net_config, mac)) &&
+	    (length == ETHER_ADDR_LEN))
+		for (i = 0; i < ETHER_ADDR_LEN; ++i)
+			dev->mac_addr[i] = ((const uint8_t *)src)[i];
+	else
+		PMD_DRV_LOG(ERR, "not supported offset=%" PRIu64 ", len=%d\n",
+			    offset, length);
+}
+
+static void
+virtio_user_set_status(struct virtio_hw *hw, uint8_t status)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (status & VIRTIO_CONFIG_STATUS_DRIVER_OK)
+		virtio_user_start_device(dev);
+	dev->status = status;
+}
+
+static void
+virtio_user_reset(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	virtio_user_stop_device(dev);
+}
+
+static uint8_t
+virtio_user_get_status(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	return dev->status;
+}
+
+static uint64_t
+virtio_user_get_features(struct virtio_hw *hw)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	return dev->features;
+}
+
+static void
+virtio_user_set_features(struct virtio_hw *hw, uint64_t features)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	dev->features = features;
+}
+
+static uint8_t
+virtio_user_get_isr(struct virtio_hw *hw __rte_unused)
+{
+	/* When config interrupt happens, driver calls this function to query
+	 * what kinds of change happen. Interrupt mode not supported for now.
+	 */
+	return 0;
+}
+
+static uint16_t
+virtio_user_set_config_irq(struct virtio_hw *hw __rte_unused,
+		    uint16_t vec __rte_unused)
+{
+	return VIRTIO_MSI_NO_VECTOR;
+}
+
+/* This function is to get the queue size, aka, number of descs, of a specified
+ * queue. Different with the VHOST_USER_GET_QUEUE_NUM, which is used to get the
+ * max supported queues.
+ */
+static uint16_t
+virtio_user_get_queue_num(struct virtio_hw *hw, uint16_t queue_id __rte_unused)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	/* Currently, each queue has same queue size */
+	return dev->queue_size;
+}
+
+static int
+virtio_user_setup_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+	uint16_t queue_idx = vq->vq_queue_index;
+	uint64_t desc_addr, avail_addr, used_addr;
+
+	desc_addr = (uintptr_t)vq->vq_ring_virt_mem;
+	avail_addr = desc_addr + vq->vq_nentries * sizeof(struct vring_desc);
+	used_addr = RTE_ALIGN_CEIL(avail_addr + offsetof(struct vring_avail,
+							 ring[vq->vq_nentries]),
+				   VIRTIO_PCI_VRING_ALIGN);
+
+	dev->vrings[queue_idx].num = vq->vq_nentries;
+	dev->vrings[queue_idx].desc = (void *)(uintptr_t)desc_addr;
+	dev->vrings[queue_idx].avail = (void *)(uintptr_t)avail_addr;
+	dev->vrings[queue_idx].used = (void *)(uintptr_t)used_addr;
+
+	return 0;
+}
+
+static void
+virtio_user_del_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	/* For legacy devices, write 0 to VIRTIO_PCI_QUEUE_PFN port, QEMU
+	 * correspondingly stops the ioeventfds, and reset the status of
+	 * the device.
+	 * For modern devices, set queue desc, avail, used in PCI bar to 0,
+	 * not see any more behavior in QEMU.
+	 *
+	 * Here we just care about what information to deliver to vhost-user
+	 * or vhost-kernel. So we just close ioeventfd for now.
+	 */
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	close(dev->callfds[vq->vq_queue_index]);
+	close(dev->kickfds[vq->vq_queue_index]);
+}
+
+static void
+virtio_user_notify_queue(struct virtio_hw *hw, struct virtqueue *vq)
+{
+	uint64_t buf = 1;
+	struct virtio_user_dev *dev = virtio_user_get_dev(hw);
+
+	if (write(dev->kickfds[vq->vq_queue_index], &buf, sizeof(buf)) < 0)
+		PMD_DRV_LOG(ERR, "failed to kick backend: %s\n",
+			    strerror(errno));
+}
+
+static const struct virtio_pci_ops virtio_user_ops = {
+	.read_dev_cfg	= virtio_user_read_dev_config,
+	.write_dev_cfg	= virtio_user_write_dev_config,
+	.reset		= virtio_user_reset,
+	.get_status	= virtio_user_get_status,
+	.set_status	= virtio_user_set_status,
+	.get_features	= virtio_user_get_features,
+	.set_features	= virtio_user_set_features,
+	.get_isr	= virtio_user_get_isr,
+	.set_config_irq	= virtio_user_set_config_irq,
+	.get_queue_num	= virtio_user_get_queue_num,
+	.setup_queue	= virtio_user_setup_queue,
+	.del_queue	= virtio_user_del_queue,
+	.notify_queue	= virtio_user_notify_queue,
+};
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* [PATCH v9 6/6] virtio-user: add a new vdev named virtio-user
  2016-06-15  9:03 ` [PATCH v9 " Jianfeng Tan
                     ` (4 preceding siblings ...)
  2016-06-15  9:03   ` [PATCH v9 5/6] virtio-user: add new virtual PCI driver for virtio Jianfeng Tan
@ 2016-06-15  9:03   ` Jianfeng Tan
  2016-06-15  9:54   ` [PATCH v9 0/6] virtio support for container Yuanhan Liu
  6 siblings, 0 replies; 196+ messages in thread
From: Jianfeng Tan @ 2016-06-15  9:03 UTC (permalink / raw)
  To: dev
  Cc: Jianfeng Tan, Huawei Xie, rich.lane, yuanhan.liu, mst,
	nakajima.yoshihiro, p.fedin, ann.zhuangyanying, mukawa, nhorman

Add a new virtual device named vhost-user, which can be used just like
eth_ring, eth_null, etc. To reuse the code of original virtio, we do
some adjustment in virtio_ethdev.c, such as remove key _static_ of
eth_virtio_dev_init() so that it can be reused in virtual device; and
we add some check to make sure it will not crash.

Configured parameters include:
  - queues (optional, 1 by default), number of queue pairs, multi-queue
    not supported for now.
  - cq (optional, 0 by default), not supported for now.
  - mac (optional), random value will be given if not specified.
  - queue_size (optional, 256 by default), size of virtqueues.
  - path (madatory), path of vhost user.

When enable CONFIG_RTE_VIRTIO_USER (enabled by default), the compiled
library can be used in both VM and container environment.

Examples:
path_vhost=<path_to_vhost_user> # use vhost-user as a backend

sudo ./examples/l2fwd/build/l2fwd -c 0x100000 -n 4 \
    --socket-mem 0,1024 --no-pci --file-prefix=l2fwd \
    --vdev=virtio-user0,mac=00:01:02:03:04:05,path=$path_vhost -- -p 0x1

Known issues:
 - Control queue and multi-queue are not supported yet.
 - Cannot work with --huge-unlink.
 - Cannot work with no-huge.
 - Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8)
   hugepages.
 - Root privilege is a must (mainly becase of sorting hugepages according
   to physical address).
 - Applications should not use file name like HUGEFILE_FMT ("%smap_%d").
 - Cannot work with vhost-net backend.

Signed-off-by: Huawei Xie <huawei.xie@intel.com>
Signed-off-by: Jianfeng Tan <jianfeng.tan@intel.com>
Acked-by: Neil Horman <nhorman@tuxdriver.com>
Acked-by: Yuanhan Liu <yuanhan.liu@linux.intel.com>
---
 doc/guides/rel_notes/release_16_07.rst  |  12 ++
 doc/guides/sample_app_ug/vhost.rst      |  17 +++
 drivers/net/virtio/virtio_ethdev.c      |  19 +--
 drivers/net/virtio/virtio_ethdev.h      |   2 +
 drivers/net/virtio/virtio_user_ethdev.c | 209 ++++++++++++++++++++++++++++++++
 5 files changed, 252 insertions(+), 7 deletions(-)

diff --git a/doc/guides/rel_notes/release_16_07.rst b/doc/guides/rel_notes/release_16_07.rst
index 13df729..681235a 100644
--- a/doc/guides/rel_notes/release_16_07.rst
+++ b/doc/guides/rel_notes/release_16_07.rst
@@ -68,6 +68,18 @@ New Features
 
   It can be turned off if flag ``RTE_VHOST_USER_NO_RECONNECT`` is set.
 
+* **Virtio support for containers.**
+
+  Add a new virtual device, named virtio-user, to support virtio for containers.
+
+  Known limitations:
+
+  * Control queue and multi-queue are not supported yet.
+  * Cannot work with --huge-unlink.
+  * Cannot work with --no-huge.
+  * Cannot work when there are more than VHOST_MEMORY_MAX_NREGIONS(8) hugepages.
+  * Root privilege is a must for sorting hugepages by physical address.
+  * Can only be used with vhost user backend.
 
 Resolved Issues
 ---------------
diff --git a/doc/guides/sample_app_ug/vhost.rst b/doc/guides/sample_app_ug/vhost.rst
index 5f81802..a93e54d 100644
--- a/doc/guides/sample_app_ug/vhost.rst
+++ b/doc/guides/sample_app_ug/vhost.rst
@@ -833,3 +833,20 @@ For example:
 The above message indicates that device 0 has been registered with MAC address cc:bb:bb:bb:bb:bb and VLAN tag 1000.
 Any packets received on the NIC with these values is placed on the devices receive queue.
 When a virtio-net device transmits packets, the VLAN tag is added to the packet by the DPDK vhost sample code.
+
+Running virtio-user with vhost-switch
+-------------------------------------
+
+We can also use virtio-user with vhost-switch now.
+Virtio-user is a virtual device that can be run in a application (container) parallelly with vhost in the same OS,
+aka, there is no need to start a VM. We just run it with a different --file-prefix to avoid startup failure.
+
+.. code-block:: console
+
+    cd ${RTE_SDK}/x86_64-native-linuxapp-gcc/app
+    ./testpmd -c 0x3 -n 4 --socket-mem 1024 --no-pci --file-prefix=virtio-user-testpmd \
+    --vdev=virtio-user0,mac=00:01:02:03:04:05,path=$path_vhost \
+    -- -i --txqflags=0xf01 --disable-hw-vlan
+
+There is no difference on the vhost side.
+Pleae note that there are some limitations (see release note for more information) in the usage of virtio-user.
diff --git a/drivers/net/virtio/virtio_ethdev.c b/drivers/net/virtio/virtio_ethdev.c
index 5fd9f51..026b8a1 100644
--- a/drivers/net/virtio/virtio_ethdev.c
+++ b/drivers/net/virtio/virtio_ethdev.c
@@ -59,7 +59,6 @@
 #include "virtqueue.h"
 #include "virtio_rxtx.h"
 
-static int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
 static int eth_virtio_dev_uninit(struct rte_eth_dev *eth_dev);
 static int  virtio_dev_configure(struct rte_eth_dev *dev);
 static int  virtio_dev_start(struct rte_eth_dev *dev);
@@ -1084,7 +1083,7 @@ rx_func_get(struct rte_eth_dev *eth_dev)
  * This function is based on probe() function in virtio_pci.c
  * It returns 0 on success.
  */
-static int
+int
 eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 {
 	struct virtio_hw *hw = eth_dev->data->dev_private;
@@ -1115,9 +1114,11 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	pci_dev = eth_dev->pci_dev;
 
-	ret = vtpci_init(pci_dev, hw, &dev_flags);
-	if (ret)
-		return ret;
+	if (pci_dev) {
+		ret = vtpci_init(pci_dev, hw, &dev_flags);
+		if (ret)
+			return ret;
+	}
 
 	/* Reset the device although not necessary at startup */
 	vtpci_reset(hw);
@@ -1209,7 +1210,8 @@ eth_virtio_dev_init(struct rte_eth_dev *eth_dev)
 
 	PMD_INIT_LOG(DEBUG, "hw->max_rx_queues=%d   hw->max_tx_queues=%d",
 			hw->max_rx_queues, hw->max_tx_queues);
-	PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
+	if (pci_dev)
+		PMD_INIT_LOG(DEBUG, "port %d vendorID=0x%x deviceID=0x%x",
 			eth_dev->data->port_id, pci_dev->id.vendor_id,
 			pci_dev->id.device_id);
 
@@ -1497,7 +1499,10 @@ virtio_dev_info_get(struct rte_eth_dev *dev, struct rte_eth_dev_info *dev_info)
 {
 	struct virtio_hw *hw = dev->data->dev_private;
 
-	dev_info->driver_name = dev->driver->pci_drv.name;
+	if (dev->pci_dev)
+		dev_info->driver_name = dev->driver->pci_drv.name;
+	else
+		dev_info->driver_name = "virtio-user PMD";
 	dev_info->max_rx_queues = (uint16_t)hw->max_rx_queues;
 	dev_info->max_tx_queues = (uint16_t)hw->max_tx_queues;
 	dev_info->min_rx_bufsize = VIRTIO_MIN_RX_BUFSIZE;
diff --git a/drivers/net/virtio/virtio_ethdev.h b/drivers/net/virtio/virtio_ethdev.h
index 7e77259..2ecec6e 100644
--- a/drivers/net/virtio/virtio_ethdev.h
+++ b/drivers/net/virtio/virtio_ethdev.h
@@ -113,6 +113,8 @@ uint16_t virtio_recv_pkts_vec(void *rx_queue, struct rte_mbuf **rx_pkts,
 uint16_t virtio_xmit_pkts_simple(void *tx_queue, struct rte_mbuf **tx_pkts,
 		uint16_t nb_pkts);
 
+int eth_virtio_dev_init(struct rte_eth_dev *eth_dev);
+
 /*
  * The VIRTIO_NET_F_GUEST_TSO[46] features permit the host to send us
  * frames larger than 1514 bytes. We do not yet support software LRO
diff --git a/drivers/net/virtio/virtio_user_ethdev.c b/drivers/net/virtio/virtio_user_ethdev.c
index 7814514..8f401a3 100644
--- a/drivers/net/virtio/virtio_user_ethdev.c
+++ b/drivers/net/virtio/virtio_user_ethdev.c
@@ -35,6 +35,10 @@
 #include <sys/types.h>
 #include <unistd.h>
 
+#include <rte_malloc.h>
+#include <rte_kvargs.h>
+
+#include "virtio_ethdev.h"
 #include "virtio_logs.h"
 #include "virtio_pci.h"
 #include "virtqueue.h"
@@ -216,3 +220,208 @@ static const struct virtio_pci_ops virtio_user_ops = {
 	.del_queue	= virtio_user_del_queue,
 	.notify_queue	= virtio_user_notify_queue,
 };
+
+static const char *valid_args[] = {
+#define VIRTIO_USER_ARG_QUEUES_NUM     "queues"
+	VIRTIO_USER_ARG_QUEUES_NUM,
+#define VIRTIO_USER_ARG_CQ_NUM         "cq"
+	VIRTIO_USER_ARG_CQ_NUM,
+#define VIRTIO_USER_ARG_MAC            "mac"
+	VIRTIO_USER_ARG_MAC,
+#define VIRTIO_USER_ARG_PATH           "path"
+	VIRTIO_USER_ARG_PATH,
+#define VIRTIO_USER_ARG_QUEUE_SIZE     "queue_size"
+	VIRTIO_USER_ARG_QUEUE_SIZE,
+	NULL
+};
+
+#define VIRTIO_USER_DEF_CQ_EN	0
+#define VIRTIO_USER_DEF_Q_NUM	1
+#define VIRTIO_USER_DEF_Q_SZ	256
+
+static int
+get_string_arg(const char *key __rte_unused,
+	       const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(char **)extra_args = strdup(value);
+
+	return 0;
+}
+
+static int
+get_integer_arg(const char *key __rte_unused,
+		const char *value, void *extra_args)
+{
+	if (!value || !extra_args)
+		return -EINVAL;
+
+	*(uint64_t *)extra_args = strtoull(value, NULL, 0);
+
+	return 0;
+}
+
+static struct rte_eth_dev *
+virtio_user_eth_dev_alloc(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct rte_eth_dev_data *data;
+	struct virtio_hw *hw;
+	struct virtio_user_dev *dev;
+
+	eth_dev = rte_eth_dev_allocate(name, RTE_ETH_DEV_VIRTUAL);
+	if (!eth_dev) {
+		PMD_INIT_LOG(ERR, "cannot alloc rte_eth_dev");
+		return NULL;
+	}
+
+	data = eth_dev->data;
+
+	hw = rte_zmalloc(NULL, sizeof(*hw), 0);
+	if (!hw) {
+		PMD_INIT_LOG(ERR, "malloc virtio_hw failed");
+		rte_eth_dev_release_port(eth_dev);
+		return NULL;
+	}
+
+	dev = rte_zmalloc(NULL, sizeof(*dev), 0);
+	if (!dev) {
+		PMD_INIT_LOG(ERR, "malloc virtio_user_dev failed");
+		rte_eth_dev_release_port(eth_dev);
+		rte_free(hw);
+		return NULL;
+	}
+
+	hw->vtpci_ops = &virtio_user_ops;
+	hw->use_msix = 0;
+	hw->modern   = 0;
+	hw->virtio_user_dev = dev;
+	data->dev_private = hw;
+	data->numa_node = SOCKET_ID_ANY;
+	data->kdrv = RTE_KDRV_NONE;
+	data->dev_flags = RTE_ETH_DEV_DETACHABLE;
+	eth_dev->pci_dev = NULL;
+	eth_dev->driver = NULL;
+	return eth_dev;
+}
+
+/* Dev initialization routine. Invoked once for each virtio vdev at
+ * EAL init time, see rte_eal_dev_init().
+ * Returns 0 on success.
+ */
+static int
+virtio_user_pmd_devinit(const char *name, const char *params)
+{
+	struct rte_kvargs *kvlist;
+	struct rte_eth_dev *eth_dev;
+	struct virtio_hw *hw;
+	uint64_t queues = VIRTIO_USER_DEF_Q_NUM;
+	uint64_t cq = VIRTIO_USER_DEF_CQ_EN;
+	uint64_t queue_size = VIRTIO_USER_DEF_Q_SZ;
+	char *path = NULL;
+	char *mac_addr = NULL;
+	int ret = -1;
+
+	if (!params || params[0] == '\0') {
+		PMD_INIT_LOG(ERR, "arg %s is mandatory for virtio-user",
+			  VIRTIO_USER_ARG_QUEUE_SIZE);
+		goto end;
+	}
+
+	kvlist = rte_kvargs_parse(params, valid_args);
+	if (!kvlist) {
+		PMD_INIT_LOG(ERR, "error when parsing param");
+		goto end;
+	}
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_PATH) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_PATH,
+				   &get_string_arg, &path);
+	else {
+		PMD_INIT_LOG(ERR, "arg %s is mandatory for virtio-user\n",
+			  VIRTIO_USER_ARG_QUEUE_SIZE);
+		goto end;
+	}
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_MAC) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_MAC,
+				   &get_string_arg, &mac_addr);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_QUEUE_SIZE) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_QUEUE_SIZE,
+				   &get_integer_arg, &queue_size);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_QUEUES_NUM) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_QUEUES_NUM,
+				   &get_integer_arg, &queues);
+
+	if (rte_kvargs_count(kvlist, VIRTIO_USER_ARG_CQ_NUM) == 1)
+		rte_kvargs_process(kvlist, VIRTIO_USER_ARG_CQ_NUM,
+				   &get_integer_arg, &cq);
+
+	eth_dev = virtio_user_eth_dev_alloc(name);
+	if (!eth_dev) {
+		PMD_INIT_LOG(ERR, "virtio-user fails to alloc device");
+		goto end;
+	}
+
+	hw = eth_dev->data->dev_private;
+	if (virtio_user_dev_init(hw->virtio_user_dev, path, queues, cq,
+				 queue_size, mac_addr) < 0)
+		goto end;
+
+	/* previously called by rte_eal_pci_probe() for physical dev */
+	if (eth_virtio_dev_init(eth_dev) < 0) {
+		PMD_INIT_LOG(ERR, "eth_virtio_dev_init fails");
+		goto end;
+	}
+	ret = 0;
+
+end:
+	if (path)
+		free(path);
+	if (mac_addr)
+		free(mac_addr);
+	return ret;
+}
+
+/** Called by rte_eth_dev_detach() */
+static int
+virtio_user_pmd_devuninit(const char *name)
+{
+	struct rte_eth_dev *eth_dev;
+	struct virtio_hw *hw;
+	struct virtio_user_dev *dev;
+
+	if (!name)
+		return -EINVAL;
+
+	PMD_DRV_LOG(INFO, "Un-Initializing %s\n", name);
+	eth_dev = rte_eth_dev_allocated(name);
+	if (!eth_dev)
+		return -ENODEV;
+
+	/* make sure the device is stopped, queues freed */
+	rte_eth_dev_close(eth_dev->data->port_id);
+
+	hw = eth_dev->data->dev_private;
+	dev = hw->virtio_user_dev;
+	virtio_user_dev_uninit(dev);
+
+	rte_free(eth_dev->data->dev_private);
+	rte_free(eth_dev->data);
+	rte_eth_dev_release_port(eth_dev);
+
+	return 0;
+}
+
+static struct rte_driver virtio_user_driver = {
+	.name   = "virtio-user",
+	.type   = PMD_VDEV,
+	.init   = virtio_user_pmd_devinit,
+	.uninit = virtio_user_pmd_devuninit,
+};
+
+PMD_REGISTER_DRIVER(virtio_user_driver);
-- 
2.1.4

^ permalink raw reply related	[flat|nested] 196+ messages in thread

* Re: [PATCH v9 0/6] virtio support for container
  2016-06-15  9:03 ` [PATCH v9 " Jianfeng Tan
                     ` (5 preceding siblings ...)
  2016-06-15  9:03   ` [PATCH v9 6/6] virtio-user: add a new vdev named virtio-user Jianfeng Tan
@ 2016-06-15  9:54   ` Yuanhan Liu
  6 siblings, 0 replies; 196+ messages in thread
From: Yuanhan Liu @ 2016-06-15  9:54 UTC (permalink / raw)
  To: Jianfeng Tan
  Cc: dev, Huawei Xie, rich.lane, mst, nakajima.yoshihiro, p.fedin,
	ann.zhuangyanying, mukawa, nhorman

Applied to dpdk-next-virtio.

Thanks.

	--yliu

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v9 3/6] virtio-user: add vhost user adapter layer
  2016-06-15  9:03   ` [PATCH v9 3/6] virtio-user: add vhost user adapter layer Jianfeng Tan
@ 2016-06-23  9:01     ` Ferruh Yigit
  2016-06-24  1:18       ` Tan, Jianfeng
  0 siblings, 1 reply; 196+ messages in thread
From: Ferruh Yigit @ 2016-06-23  9:01 UTC (permalink / raw)
  To: Jianfeng Tan, dev
  Cc: Huawei Xie, rich.lane, yuanhan.liu, mst, nakajima.yoshihiro,
	p.fedin, ann.zhuangyanying, mukawa, nhorman

On 6/15/2016 10:03 AM, Jianfeng Tan wrote:

...

> 
> diff --git a/config/common_linuxapp b/config/common_linuxapp
> index 7e698e2..2483dfa 100644
> --- a/config/common_linuxapp
> +++ b/config/common_linuxapp
> @@ -43,3 +43,4 @@ CONFIG_RTE_LIBRTE_VHOST=y
>  CONFIG_RTE_LIBRTE_PMD_VHOST=y
>  CONFIG_RTE_LIBRTE_PMD_AF_PACKET=y
>  CONFIG_RTE_LIBRTE_POWER=y
> +CONFIG_RTE_VIRTIO_USER=y

With current config structure, all configuration parameters put into
common_base with a default value, and overwritten in environment file if
required, VIRTIO_USER is missing in common_base.
You can check CONFIG_RTE_LIBRTE_VHOST as sample.

^ permalink raw reply	[flat|nested] 196+ messages in thread

* Re: [PATCH v9 3/6] virtio-user: add vhost user adapter layer
  2016-06-23  9:01     ` Ferruh Yigit
@ 2016-06-24  1:18       ` Tan, Jianfeng
  0 siblings, 0 replies; 196+ messages in thread
From: Tan, Jianfeng @ 2016-06-24  1:18 UTC (permalink / raw)
  To: Yigit, Ferruh, dev
  Cc: Xie, Huawei, rich.lane, yuanhan.liu, mst, nakajima.yoshihiro,
	p.fedin, ann.zhuangyanying, mukawa, nhorman

Hi Yigit,

> -----Original Message-----
> From: Yigit, Ferruh
> Sent: Thursday, June 23, 2016 5:02 PM
> To: Tan, Jianfeng; dev@dpdk.org
> Cc: Xie, Huawei; rich.lane@bigswitch.com; yuanhan.liu@linux.intel.com;
> mst@redhat.com; nakajima.yoshihiro@lab.ntt.co.jp; p.fedin@samsung.com;
> ann.zhuangyanying@huawei.com; mukawa@igel.co.jp;
> nhorman@tuxdriver.com
> Subject: Re: [dpdk-dev] [PATCH v9 3/6] virtio-user: add vhost user adapter
> layer
> 
> On 6/15/2016 10:03 AM, Jianfeng Tan wrote:
> 
> ...
> 
> >
> > diff --git a/config/common_linuxapp b/config/common_linuxapp
> > index 7e698e2..2483dfa 100644
> > --- a/config/common_linuxapp
> > +++ b/config/common_linuxapp
> > @@ -43,3 +43,4 @@ CONFIG_RTE_LIBRTE_VHOST=y
> >  CONFIG_RTE_LIBRTE_PMD_VHOST=y
> >  CONFIG_RTE_LIBRTE_PMD_AF_PACKET=y
> >  CONFIG_RTE_LIBRTE_POWER=y
> > +CONFIG_RTE_VIRTIO_USER=y
> 
> With current config structure, all configuration parameters put into
> common_base with a default value, and overwritten in environment file if
> required, VIRTIO_USER is missing in common_base.
> You can check CONFIG_RTE_LIBRTE_VHOST as sample.

Oh yes, I need to add CONFIG_RTE_VIRTIO_USER=n in common_base. I'll send a patch to fix this.

Thank you for suggestion!

Thanks,
Jianfeng

^ permalink raw reply	[flat|nested] 196+ messages in thread

end of thread, other threads:[~2016-06-24  1:20 UTC | newest]

Thread overview: 196+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2015-11-05 18:31 [RFC 0/5] virtio support for container Jianfeng Tan
2015-11-05 18:31 ` [RFC 1/5] virtio/container: add handler for ioport rd/wr Jianfeng Tan
2015-11-05 18:31 ` [RFC 2/5] virtio/container: add a new virtual device named eth_cvio Jianfeng Tan
2015-11-05 18:31 ` [RFC 3/5] virtio/container: unify desc->addr assignment Jianfeng Tan
2015-11-05 18:31 ` [RFC 4/5] virtio/container: adjust memory initialization process Jianfeng Tan
2015-11-06 16:21   ` Ananyev, Konstantin
2015-11-08 11:18     ` Tan, Jianfeng
2015-11-09 13:32       ` Ananyev, Konstantin
2015-11-09 14:13         ` Tan, Jianfeng
2015-11-05 18:31 ` [RFC 5/5] vhost/container: change mode of vhost listening socket Jianfeng Tan
2015-11-09  3:54   ` Yuanhan Liu
2015-11-09  5:15     ` Tan, Jianfeng
2015-11-09  5:40       ` Yuanhan Liu
2015-11-09  5:46         ` Tan, Jianfeng
2015-11-24  3:53 ` [RFC 0/5] virtio support for container Zhuangyanying
2015-11-24  6:19   ` Tan, Jianfeng
2016-01-10 11:42 ` [PATCH 0/4] " Jianfeng Tan
2016-01-10 11:42   ` [PATCH 1/4] mem: add --single-file to create single mem-backed file Jianfeng Tan
2016-01-21  1:57     ` Xie, Huawei
2016-01-10 11:43   ` [PATCH 2/4] mem: add API to obstain memory-backed file info Jianfeng Tan
2016-01-11 11:43     ` Pavel Fedin
2016-01-11 20:26     ` Rich Lane
2016-01-12  9:12       ` Tan, Jianfeng
2016-01-12 10:04         ` Pavel Fedin
2016-01-12 10:48           ` Tan, Jianfeng
2016-01-12 11:00             ` Pavel Fedin
2016-01-12 11:07               ` Sergio Gonzalez Monroy
2016-01-12 11:37                 ` Pavel Fedin
2016-01-12 12:12                   ` Sergio Gonzalez Monroy
2016-01-12 13:57                     ` Pavel Fedin
2016-01-12 14:13                       ` Sergio Gonzalez Monroy
2016-01-12 11:44                 ` Sergio Gonzalez Monroy
2016-01-12 11:22             ` Pavel Fedin
2016-01-12 11:33               ` Sergio Gonzalez Monroy
2016-01-12 12:01                 ` Pavel Fedin
2016-01-12 13:39                   ` Sergio Gonzalez Monroy
2016-01-10 11:43   ` [PATCH 3/4] virtio/vdev: add ways to interact with vhost Jianfeng Tan
2016-01-11 10:42     ` Pavel Fedin
2016-01-11 14:02     ` Pavel Fedin
2016-01-21  2:18     ` Xie, Huawei
2016-01-10 11:43   ` [PATCH 4/4] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
2016-01-12  7:45     ` Pavel Fedin
2016-01-12  7:59       ` Yuanhan Liu
2016-01-12  8:39       ` Tan, Jianfeng
2016-01-12  9:15         ` Tan, Jianfeng
2016-01-27  3:10     ` Qiu, Michael
2016-01-11 14:21   ` [PATCH 0/4] virtio support for container Pavel Fedin
2016-01-11 15:53     ` Tan, Jianfeng
2016-01-12  7:38       ` Pavel Fedin
2016-01-12  8:14         ` Rich Lane
2016-01-12  8:39           ` Pavel Fedin
2016-01-12  8:51             ` Tan, Jianfeng
2016-01-12 10:48               ` Pavel Fedin
2016-01-12 14:45                 ` Amit Tomer
2016-01-12 14:50                   ` Pavel Fedin
2016-01-12 14:58                     ` Amit Tomer
2016-01-12 14:53                   ` Tan, Jianfeng
2016-01-12 15:11                     ` Amit Tomer
2016-01-12 16:18                       ` Tan, Jianfeng
2016-01-13 15:00                         ` Amit Tomer
2016-01-13 18:41                           ` Tan, Jianfeng
2016-01-14  9:34                             ` Amit Tomer
2016-01-14 11:41                               ` Tan, Jianfeng
2016-01-14 12:03                                 ` Amit Tomer
2016-01-15  6:39                                   ` Tan, Jianfeng
2016-01-20 15:19                                     ` Amit Tomer
2016-01-22  6:04                                       ` Tan, Jianfeng
2016-01-12  5:36   ` Tetsuya Mukawa
2016-01-12  5:46     ` Tan, Jianfeng
2016-01-12  6:01       ` Tetsuya Mukawa
2016-01-12  6:14         ` Yuanhan Liu
2016-01-12  6:26           ` Tetsuya Mukawa
2016-01-12  6:29             ` Yuanhan Liu
2016-01-20  3:48     ` Xie, Huawei
2016-01-26  6:02   ` Qiu, Michael
2016-01-26  6:09     ` Tan, Jianfeng
2016-02-05 11:20 ` [PATCH v2 0/5] " Jianfeng Tan
2016-02-05 11:20   ` [PATCH v2 1/5] mem: add --single-file to create single mem-backed file Jianfeng Tan
2016-03-07 13:13     ` Yuanhan Liu
2016-03-08  1:55       ` Tan, Jianfeng
2016-03-08  2:44         ` Yuanhan Liu
2016-03-09 14:44           ` Tan, Jianfeng
2016-03-08  8:49       ` Panu Matilainen
2016-03-08  9:04         ` Yuanhan Liu
2016-03-08 10:30           ` Thomas Monjalon
2016-03-08 10:57             ` Burakov, Anatoly
2016-03-14 13:53             ` Traynor, Kevin
2016-03-14 14:45               ` Thomas Monjalon
2016-03-14 18:21                 ` Traynor, Kevin
2016-02-05 11:20   ` [PATCH v2 2/5] mem: add API to obtain memory-backed file info Jianfeng Tan
2016-03-07 13:22     ` Yuanhan Liu
2016-03-08  2:31       ` Tan, Jianfeng
2016-03-08  2:53         ` Yuanhan Liu
2016-02-05 11:20   ` [PATCH v2 3/5] virtio/vdev: add embeded device emulation Jianfeng Tan
2016-02-07 10:47     ` Michael S. Tsirkin
2016-02-08  6:59     ` Tetsuya Mukawa
2016-02-16  2:47       ` Tan, Jianfeng
2016-02-16  2:40     ` Tan, Jianfeng
2016-02-05 11:20   ` [PATCH v2 4/5] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
2016-02-05 11:20   ` [PATCH v2 5/5] docs: add release note for virtio for container Jianfeng Tan
2016-03-23 19:17   ` [PATCH v2 0/5] virtio support " Neil Horman
2016-03-24  3:10     ` Tan, Jianfeng
2016-03-24 13:45       ` Neil Horman
2016-03-25  1:25         ` Tan, Jianfeng
2016-03-25 11:06           ` Neil Horman
2016-04-13 16:14   ` Thomas Monjalon
2016-04-14  6:08     ` Tan, Jianfeng
2016-04-21  2:56 ` [PATCH v3 0/2] " Jianfeng Tan
2016-04-21  2:56   ` [PATCH v3 1/2] virtio/vdev: add embeded device emulation Jianfeng Tan
2016-04-21 22:01     ` Yuanhan Liu
2016-04-22 10:12       ` Tan, Jianfeng
2016-04-22 10:17         ` Thomas Monjalon
2016-04-22 17:27         ` Yuanhan Liu
2016-04-21  2:56   ` [PATCH v3 2/2] virtio/vdev: add a new vdev named eth_cvio Jianfeng Tan
2016-04-21  8:51     ` David Marchand
2016-04-22  5:15       ` Tan, Jianfeng
2016-04-22  7:36         ` David Marchand
2016-04-22 10:25           ` Tan, Jianfeng
2016-04-21 10:05     ` Thomas Monjalon
2016-04-22  7:26       ` Tan, Jianfeng
2016-04-22  8:30         ` Thomas Monjalon
2016-04-21 22:14     ` Yuanhan Liu
2016-04-22 10:12       ` Tan, Jianfeng
2016-04-29  1:18 ` [PATCH v4 0/8] virtio support for container Jianfeng Tan
2016-04-29  1:18   ` [PATCH v4 1/8] virtio: hide phys addr check inside pci ops Jianfeng Tan
2016-05-11 23:05     ` Yuanhan Liu
2016-04-29  1:18   ` [PATCH v4 2/8] virtio: abstract vring hdr desc init as a method Jianfeng Tan
2016-04-29  1:18   ` [PATCH v4 3/8] virtio: enable use virtual address to fill desc Jianfeng Tan
2016-04-29  1:18   ` [PATCH v4 4/8] virtio-user: add vhost adapter layer Jianfeng Tan
2016-04-29  1:18   ` [PATCH v4 5/8] virtio-user: add device emulation layer APIs Jianfeng Tan
2016-05-12  1:05     ` Yuanhan Liu
2016-04-29  1:18   ` [PATCH v4 6/8] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
2016-05-12  2:12     ` Yuanhan Liu
2016-05-12  7:08       ` Tan, Jianfeng
2016-05-12 16:40         ` Yuanhan Liu
2016-05-13  1:54           ` Tan, Jianfeng
2016-05-13  4:45             ` Yuanhan Liu
2016-05-16  1:48               ` Tan, Jianfeng
2016-05-16  2:51                 ` Yuanhan Liu
2016-05-12 17:02         ` Michael S. Tsirkin
2016-05-13  2:00           ` Tan, Jianfeng
2016-04-29  1:18   ` [PATCH v4 7/8] virtio-user: add a new virtual device named virtio-user Jianfeng Tan
2016-04-29  1:18   ` [PATCH v4 8/8] doc: update doc for virtio-user Jianfeng Tan
2016-04-29  1:35   ` [PATCH v4 0/8] virtio support for container Tan, Jianfeng
2016-05-30 10:55 ` [PATCH v5 " Jianfeng Tan
2016-05-30 10:55   ` [PATCH v5 1/8] virtio: hide phys addr check inside pci ops Jianfeng Tan
2016-05-30 10:55   ` [PATCH v5 2/8] virtio: clean up virtio_dev_queue_setup Jianfeng Tan
2016-06-01  7:38     ` Yuanhan Liu
2016-06-01  7:44       ` Tan, Jianfeng
2016-06-01  7:58         ` Yuanhan Liu
2016-05-30 10:55   ` [PATCH v5 3/8] virtio: enable use virtual address to fill desc Jianfeng Tan
2016-06-01  8:03     ` Yuanhan Liu
2016-05-30 10:55   ` [PATCH v5 4/8] virtio-user: add vhost adapter layer Jianfeng Tan
2016-05-30 10:55   ` [PATCH v5 5/8] virtio-user: add device emulation layer APIs Jianfeng Tan
2016-05-30 10:55   ` [PATCH v5 6/8] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
2016-06-01  8:21     ` Yuanhan Liu
2016-05-30 10:55   ` [PATCH v5 7/8] virtio-user: add a new vdev named virtio-user Jianfeng Tan
2016-06-01  8:26     ` Yuanhan Liu
2016-06-02  1:27       ` Tan, Jianfeng
2016-05-30 10:55   ` [PATCH v5 8/8] doc: update doc for virtio-user Jianfeng Tan
2016-06-01  8:30     ` Yuanhan Liu
2016-06-02  9:54 ` [PATCH v6 0/7] virtio support for container Jianfeng Tan
2016-06-02  9:54   ` [PATCH v6 1/7] virtio: hide phys addr check inside pci ops Jianfeng Tan
2016-06-02  9:54   ` [PATCH v6 2/7] virtio: clean up virtio_dev_queue_setup Jianfeng Tan
2016-06-02  9:54   ` [PATCH v6 3/7] virtio: enable use virtual address to fill desc Jianfeng Tan
2016-06-02  9:54   ` [PATCH v6 4/7] virtio-user: add vhost adapter layer Jianfeng Tan
2016-06-02  9:54   ` [PATCH v6 5/7] virtio-user: add device emulation layer APIs Jianfeng Tan
2016-06-02  9:54   ` [PATCH v6 6/7] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
2016-06-06  8:01     ` Yuanhan Liu
2016-06-06  8:31       ` Tan, Jianfeng
2016-06-02  9:54   ` [PATCH v6 7/7] virtio-user: add a new vdev named virtio-user Jianfeng Tan
2016-06-12  0:35 ` [PATCH v7 0/6] virtio support for container Jianfeng Tan
2016-06-12  0:35   ` [PATCH v7 1/6] virtio: hide phys addr check inside pci ops Jianfeng Tan
2016-06-12  0:35   ` [PATCH v7 2/6] virtio: enable use virtual address to fill desc Jianfeng Tan
2016-06-12  0:35   ` [PATCH v7 3/6] virtio-user: add vhost adapter layer Jianfeng Tan
2016-06-12  0:35   ` [PATCH v7 4/6] virtio-user: add device emulation layer APIs Jianfeng Tan
2016-06-12  0:35   ` [PATCH v7 5/6] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
2016-06-12  0:35   ` [PATCH v7 6/6] virtio-user: add a new vdev named virtio-user Jianfeng Tan
2016-06-13  6:38 ` [PATCH v8 0/6] virtio support for container Jianfeng Tan
2016-06-13  6:38   ` [PATCH v8 1/6] virtio: hide phys addr check inside pci ops Jianfeng Tan
2016-06-13  6:38   ` [PATCH v8 2/6] virtio: enable use virtual address to fill desc Jianfeng Tan
2016-06-13  6:39   ` [PATCH v8 3/6] virtio-user: add vhost user adapter layer Jianfeng Tan
2016-06-13  6:39   ` [PATCH v8 4/6] virtio-user: add device emulation layer APIs Jianfeng Tan
2016-06-13  6:39   ` [PATCH v8 5/6] virtio-user: add new virtual pci driver for virtio Jianfeng Tan
2016-06-13  6:39   ` [PATCH v8 6/6] virtio-user: add a new vdev named virtio-user Jianfeng Tan
2016-06-14  8:34   ` [PATCH v8 0/6] virtio support for container Yuanhan Liu
2016-06-15  9:03 ` [PATCH v9 " Jianfeng Tan
2016-06-15  9:03   ` [PATCH v9 1/6] virtio: hide phys addr check inside PCI ops Jianfeng Tan
2016-06-15  9:03   ` [PATCH v9 2/6] virtio: enable use virtual address to fill desc Jianfeng Tan
2016-06-15  9:03   ` [PATCH v9 3/6] virtio-user: add vhost user adapter layer Jianfeng Tan
2016-06-23  9:01     ` Ferruh Yigit
2016-06-24  1:18       ` Tan, Jianfeng
2016-06-15  9:03   ` [PATCH v9 4/6] virtio-user: add device emulation layer APIs Jianfeng Tan
2016-06-15  9:03   ` [PATCH v9 5/6] virtio-user: add new virtual PCI driver for virtio Jianfeng Tan
2016-06-15  9:03   ` [PATCH v9 6/6] virtio-user: add a new vdev named virtio-user Jianfeng Tan
2016-06-15  9:54   ` [PATCH v9 0/6] virtio support for container Yuanhan Liu

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.