summary refs log tree commit diff
path: root/devices
diff options
context:
space:
mode:
Diffstat (limited to 'devices')
-rw-r--r--devices/Cargo.toml8
-rw-r--r--devices/src/virtio/mod.rs2
-rw-r--r--devices/src/virtio/vhost_user/cloud_hypervisor/mod.rs1
-rw-r--r--devices/src/virtio/vhost_user/cloud_hypervisor/net_util.rs162
-rw-r--r--devices/src/virtio/vhost_user/handler.rs123
-rw-r--r--devices/src/virtio/vhost_user/mod.rs82
-rw-r--r--devices/src/virtio/vhost_user/net.rs226
-rw-r--r--devices/src/virtio/vhost_user/vu_common_ctrl.rs115
8 files changed, 719 insertions, 0 deletions
diff --git a/devices/Cargo.toml b/devices/Cargo.toml
index 18a203a..8d1c860 100644
--- a/devices/Cargo.toml
+++ b/devices/Cargo.toml
@@ -16,6 +16,7 @@ bit_field = { path = "../bit_field" }
 bitflags = "1"
 data_model = { path = "../data_model" }
 disk = { path = "../disk" }
+epoll = "= 4.1.0"
 enumn = { path = "../enumn" }
 gpu_buffer = { path = "../gpu_buffer", optional = true }
 gpu_display = { path = "../gpu_display", optional = true }
@@ -40,6 +41,13 @@ vfio_sys = { path = "../vfio_sys" }
 vhost = { path = "../vhost" }
 virtio_sys = { path = "../virtio_sys" }
 vm_control = { path = "../vm_control" }
+vmm-sys-util = { git = "https://github.com/rust-vmm/vmm-sys-util" }
+vhost_rs = { path = "../vhost_rs", features = ["vhost-user-master"] }
+virtio-bindings = { path = "../virtio-bindings" }
+
+[dependencies.vm-memory]
+git = "https://github.com/rust-vmm/vm-memory"
+features = ["backend-mmap"]
 
 [dev-dependencies]
 tempfile = { path = "../tempfile" }
diff --git a/devices/src/virtio/mod.rs b/devices/src/virtio/mod.rs
index ca3511f..4aa366c 100644
--- a/devices/src/virtio/mod.rs
+++ b/devices/src/virtio/mod.rs
@@ -23,6 +23,8 @@ mod virtio_pci_common_config;
 mod virtio_pci_device;
 mod wl;
 
+pub mod vhost_user;
+
 pub mod resource_bridge;
 pub mod vhost;
 
diff --git a/devices/src/virtio/vhost_user/cloud_hypervisor/mod.rs b/devices/src/virtio/vhost_user/cloud_hypervisor/mod.rs
new file mode 100644
index 0000000..24082d3
--- /dev/null
+++ b/devices/src/virtio/vhost_user/cloud_hypervisor/mod.rs
@@ -0,0 +1 @@
+pub mod net_util;
diff --git a/devices/src/virtio/vhost_user/cloud_hypervisor/net_util.rs b/devices/src/virtio/vhost_user/cloud_hypervisor/net_util.rs
new file mode 100644
index 0000000..69e70b9
--- /dev/null
+++ b/devices/src/virtio/vhost_user/cloud_hypervisor/net_util.rs
@@ -0,0 +1,162 @@
+// Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
+// SPDX-License-Identifier: Apache-2.0
+//
+// Portions Copyright 2017 The Chromium OS Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the THIRD-PARTY file.
+
+// use rand::Rng;
+use std::result::Result;
+
+// use serde::de::{Deserialize, Deserializer, Error};
+// use serde::ser::{Serialize, Serializer};
+
+pub const MAC_ADDR_LEN: usize = 6;
+
+#[derive(Clone, Copy, Debug, PartialEq)]
+pub struct MacAddr {
+    bytes: [u8; MAC_ADDR_LEN],
+}
+
+impl MacAddr {
+    // The error contains the str that failed to be parsed, for nicer error message generation.
+    pub fn parse_str<S>(s: &S) -> Result<MacAddr, &str>
+    where
+        S: AsRef<str> + ?Sized,
+    {
+        let v: Vec<&str> = s.as_ref().split(':').collect();
+        let mut bytes = [0u8; MAC_ADDR_LEN];
+
+        if v.len() != MAC_ADDR_LEN {
+            return Err(s.as_ref());
+        }
+
+        for i in 0..MAC_ADDR_LEN {
+            if v[i].len() != 2 {
+                return Err(s.as_ref());
+            }
+            bytes[i] = u8::from_str_radix(v[i], 16).map_err(|_| s.as_ref())?;
+        }
+
+        Ok(MacAddr { bytes })
+    }
+
+    // Does not check whether src.len() == MAC_ADDR_LEN.
+    #[inline]
+    pub fn from_bytes_unchecked(src: &[u8]) -> MacAddr {
+        // TODO: using something like std::mem::uninitialized could avoid the extra initialization,
+        // if this ever becomes a performance bottleneck.
+        let mut bytes = [0u8; MAC_ADDR_LEN];
+        bytes[..].copy_from_slice(&src[..]);
+
+        MacAddr { bytes }
+    }
+
+    // An error can only occur if the slice length is different from MAC_ADDR_LEN.
+    #[inline]
+    pub fn from_bytes(src: &[u8]) -> Result<MacAddr, ()> {
+        if src.len() != MAC_ADDR_LEN {
+            return Err(());
+        }
+        Ok(MacAddr::from_bytes_unchecked(src))
+    }
+
+    #[inline]
+    pub fn get_bytes(&self) -> &[u8] {
+        &self.bytes
+    }
+
+    pub fn to_string(self) -> String {
+        let b = &self.bytes;
+        format!(
+            "{:02x}:{:02x}:{:02x}:{:02x}:{:02x}:{:02x}",
+            b[0], b[1], b[2], b[3], b[4], b[5]
+        )
+    }
+
+    // pub fn local_random() -> MacAddr {
+    //     // Generate a fully random MAC
+    //     let mut random_bytes = rand::thread_rng().gen::<[u8; MAC_ADDR_LEN]>();
+
+    //     // Set the first byte to make the OUI a locally administered OUI
+    //     random_bytes[0] = 0x2e;
+
+    //     MacAddr {
+    //         bytes: random_bytes,
+    //     }
+    // }
+}
+
+// impl Serialize for MacAddr {
+//     fn serialize<S>(&self, serializer: S) -> Result<S::Ok, S::Error>
+//     where
+//         S: Serializer,
+//     {
+//         self.to_string().serialize(serializer)
+//     }
+// }
+
+// impl<'de> Deserialize<'de> for MacAddr {
+//     fn deserialize<D>(deserializer: D) -> Result<MacAddr, D::Error>
+//     where
+//         D: Deserializer<'de>,
+//     {
+//         let s = String::deserialize(deserializer)?;
+//         MacAddr::parse_str(&s).map_err(|_| D::Error::custom("The provided MAC address is invalid."))
+//     }
+// }
+
+#[cfg(test)]
+mod tests {
+    // extern crate serde_json;
+
+    use super::*;
+
+    #[test]
+    fn test_mac_addr() {
+        // too long
+        assert!(MacAddr::parse_str("aa:aa:aa:aa:aa:aa:aa").is_err());
+
+        // invalid hex
+        assert!(MacAddr::parse_str("aa:aa:aa:aa:aa:ax").is_err());
+
+        // single digit mac address component should be invalid
+        assert!(MacAddr::parse_str("aa:aa:aa:aa:aa:b").is_err());
+
+        // components with more than two digits should also be invalid
+        assert!(MacAddr::parse_str("aa:aa:aa:aa:aa:bbb").is_err());
+
+        let mac = MacAddr::parse_str("12:34:56:78:9a:BC").unwrap();
+
+        println!("parsed MAC address: {}", mac.to_string());
+
+        let bytes = mac.get_bytes();
+        assert_eq!(bytes, [0x12u8, 0x34, 0x56, 0x78, 0x9a, 0xbc]);
+    }
+
+    #[test]
+    fn test_from_bytes() {
+        let src1 = [0x01, 0x02, 0x03, 0x04, 0x05];
+        let src2 = [0x01, 0x02, 0x03, 0x04, 0x05, 0x06];
+        let src3 = [0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07];
+
+        assert!(MacAddr::from_bytes(&src1[..]).is_err());
+
+        let x = MacAddr::from_bytes(&src2[..]).unwrap();
+        assert_eq!(x.to_string(), String::from("01:02:03:04:05:06"));
+
+        assert!(MacAddr::from_bytes(&src3[..]).is_err());
+    }
+
+    // #[test]
+    // fn test_mac_addr_serialization_and_deserialization() {
+    //     let mac: MacAddr =
+    //         serde_json::from_str("\"12:34:56:78:9a:bc\"").expect("MacAddr deserialization failed.");
+
+    //     let bytes = mac.get_bytes();
+    //     assert_eq!(bytes, [0x12u8, 0x34, 0x56, 0x78, 0x9a, 0xbc]);
+
+    //     let s = serde_json::to_string(&mac).expect("MacAddr serialization failed.");
+    //     assert_eq!(s, "\"12:34:56:78:9a:bc\"");
+    // }
+}
diff --git a/devices/src/virtio/vhost_user/handler.rs b/devices/src/virtio/vhost_user/handler.rs
new file mode 100644
index 0000000..f8a68ff
--- /dev/null
+++ b/devices/src/virtio/vhost_user/handler.rs
@@ -0,0 +1,123 @@
+// Copyright (c) 2019 Intel Corporation. All rights reserved.
+// Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
+//
+// Copyright 2017 The Chromium OS Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE-BSD-3-Clause file.
+//
+// SPDX-License-Identifier: Apache-2.0 AND BSD-3-Clause
+
+use super::super::{Interrupt, Queue};
+use super::{Error, Result};
+use sys_util::error;
+use vmm_sys_util::eventfd::EventFd;
+
+use std::io;
+use std::os::unix::io::AsRawFd;
+use std::sync::Arc;
+
+/// Collection of common parameters required by vhost-user devices while
+/// call Epoll handler.
+///
+/// # Arguments
+/// * `interrupt_cb` interrupt for virtqueue change.
+/// * `kill_evt` - EventFd used to kill the vhost-user device.
+/// * `vu_interrupt_list` - virtqueue and EventFd to signal when buffer used.
+pub struct VhostUserEpollConfig {
+    pub interrupt: Interrupt,
+    pub kill_evt: EventFd,
+    pub vu_interrupt_list: Vec<(EventFd, Queue)>,
+}
+
+pub struct VhostUserEpollHandler {
+    pub vu_epoll_cfg: VhostUserEpollConfig,
+}
+
+impl VhostUserEpollHandler {
+    /// Construct a new event handler for vhost-user based devices.
+    ///
+    /// # Arguments
+    /// * `vu_epoll_cfg` - collection of common parameters for vhost-user devices
+    ///
+    /// # Return
+    /// * `VhostUserEpollHandler` - epoll handler for vhost-user based devices
+    pub fn new(vu_epoll_cfg: VhostUserEpollConfig) -> VhostUserEpollHandler {
+        VhostUserEpollHandler { vu_epoll_cfg }
+    }
+
+    fn signal_used_queue(&self, queue: &Queue) -> Result<()> {
+        self.vu_epoll_cfg.interrupt.signal_used_queue(queue.vector);
+        Ok(())
+    }
+
+    pub fn run(&mut self) -> Result<()> {
+        let epoll_fd = epoll::create(true).map_err(Error::EpollCreateFd)?;
+
+        for (index, vhost_user_interrupt) in self.vu_epoll_cfg.vu_interrupt_list.iter().enumerate()
+        {
+            epoll::ctl(
+                epoll_fd,
+                epoll::ControlOptions::EPOLL_CTL_ADD,
+                vhost_user_interrupt.0.as_raw_fd(),
+                epoll::Event::new(epoll::Events::EPOLLIN, index as u64),
+            )
+            .map_err(Error::EpollCtl)?;
+        }
+
+        let kill_evt_index = self.vu_epoll_cfg.vu_interrupt_list.len();
+
+        epoll::ctl(
+            epoll_fd,
+            epoll::ControlOptions::EPOLL_CTL_ADD,
+            self.vu_epoll_cfg.kill_evt.as_raw_fd(),
+            epoll::Event::new(epoll::Events::EPOLLIN, kill_evt_index as u64),
+        )
+        .map_err(Error::EpollCtl)?;
+
+        let mut events = vec![epoll::Event::new(epoll::Events::empty(), 0); kill_evt_index + 1];
+
+        'poll: loop {
+            let num_events = match epoll::wait(epoll_fd, -1, &mut events[..]) {
+                Ok(res) => res,
+                Err(e) => {
+                    if e.kind() == io::ErrorKind::Interrupted {
+                        // It's well defined from the epoll_wait() syscall
+                        // documentation that the epoll loop can be interrupted
+                        // before any of the requested events occurred or the
+                        // timeout expired. In both those cases, epoll_wait()
+                        // returns an error of type EINTR, but this should not
+                        // be considered as a regular error. Instead it is more
+                        // appropriate to retry, by calling into epoll_wait().
+                        continue;
+                    }
+                    return Err(Error::EpollWait(e));
+                }
+            };
+
+            for event in events.iter().take(num_events) {
+                let ev_type = event.data as usize;
+
+                match ev_type {
+                    x if x < kill_evt_index => {
+                        let vhost_user_interrupt = &self.vu_epoll_cfg.vu_interrupt_list[x].0;
+                        vhost_user_interrupt
+                            .read()
+                            .map_err(Error::FailedReadingQueue)?;
+                        let result =
+                            self.signal_used_queue(&self.vu_epoll_cfg.vu_interrupt_list[x].1);
+                        if let Err(_e) = result {
+                            error!("failed to signal used queue");
+                        }
+                    }
+                    x if kill_evt_index == x => {
+                        break 'poll;
+                    }
+                    _ => {
+                        error!("Unknown event for vhost-user-net");
+                    }
+                }
+            }
+        }
+        Ok(())
+    }
+}
diff --git a/devices/src/virtio/vhost_user/mod.rs b/devices/src/virtio/vhost_user/mod.rs
new file mode 100644
index 0000000..a4ebb6d
--- /dev/null
+++ b/devices/src/virtio/vhost_user/mod.rs
@@ -0,0 +1,82 @@
+pub mod cloud_hypervisor;
+
+mod handler;
+mod net;
+mod vu_common_ctrl;
+
+pub use net::Net;
+pub use vu_common_ctrl::VhostUserConfig;
+
+use std::io;
+use sys_util::guest_memory;
+
+// Copyright 2019 Intel Corporation. All Rights Reserved.
+// SPDX-License-Identifier: Apache-2.0
+use vhost_rs::Error as VhostError;
+use vm_memory::Error as MmapError;
+#[derive(Debug)]
+pub enum Error {
+    /// Invalid available address.
+    AvailAddress(guest_memory::Error),
+    /// Queue number  is not correct
+    BadQueueNum,
+    /// Creating kill eventfd failed.
+    CreateKillEventFd(io::Error),
+    /// Cloning kill eventfd failed.
+    CloneKillEventFd(io::Error),
+    /// Invalid descriptor table address.
+    DescriptorTableAddress(guest_memory::Error),
+    /// Create Epoll eventfd failed
+    EpollCreateFd(io::Error),
+    /// Epoll ctl error
+    EpollCtl(io::Error),
+    /// Epoll wait error
+    EpollWait(io::Error),
+    /// Read queue failed.
+    FailedReadingQueue(io::Error),
+    /// Signal used queue failed.
+    FailedSignalingUsedQueue(io::Error),
+    /// Failed to read vhost eventfd.
+    MemoryRegions(MmapError),
+    /// Failed to create master.
+    VhostUserCreateMaster(VhostError),
+    /// Failed to open vhost device.
+    VhostUserOpen(VhostError),
+    /// Get features failed.
+    VhostUserGetFeatures(VhostError),
+    /// Get protocol features failed.
+    VhostUserGetProtocolFeatures(VhostError),
+    /// Vhost-user Backend not support vhost-user protocol.
+    VhostUserProtocolNotSupport,
+    /// Set owner failed.
+    VhostUserSetOwner(VhostError),
+    /// Set features failed.
+    VhostUserSetFeatures(VhostError),
+    /// Set protocol features failed.
+    VhostUserSetProtocolFeatures(VhostError),
+    /// Set mem table failed.
+    VhostUserSetMemTable(VhostError),
+    /// Set vring num failed.
+    VhostUserSetVringNum(VhostError),
+    /// Set vring addr failed.
+    VhostUserSetVringAddr(VhostError),
+    /// Set vring base failed.
+    VhostUserSetVringBase(VhostError),
+    /// Set vring call failed.
+    VhostUserSetVringCall(VhostError),
+    /// Set vring kick failed.
+    VhostUserSetVringKick(VhostError),
+    /// Set vring enable failed.
+    VhostUserSetVringEnable(VhostError),
+    /// Vhost-user setup vring failed.
+    VhostUserSetupVringFailed,
+    /// Failed to create vhost eventfd.
+    VhostIrqCreate(std::io::Error),
+    /// Failed to read vhost eventfd.
+    VhostIrqRead(io::Error),
+    /// Failed to read vhost eventfd.
+    VhostUserMemoryRegion(MmapError),
+    /// Invalid used address.
+    UsedAddress(guest_memory::Error),
+}
+type Result<T> = std::result::Result<T, Error>;
diff --git a/devices/src/virtio/vhost_user/net.rs b/devices/src/virtio/vhost_user/net.rs
new file mode 100644
index 0000000..4d222d2
--- /dev/null
+++ b/devices/src/virtio/vhost_user/net.rs
@@ -0,0 +1,226 @@
+// Copyright 2019 Intel Corporation. All Rights Reserved.
+// SPDX-License-Identifier: Apache-2.0
+
+use libc;
+use libc::EFD_NONBLOCK;
+use std::cmp;
+use std::io::Write;
+use std::os::unix::prelude::*;
+use std::sync::atomic::AtomicUsize;
+use std::sync::{Arc, RwLock};
+use std::thread;
+use std::vec::Vec;
+use sync::Mutex;
+use sys_util::{error, warn, EventFd, GuestMemory};
+
+use super::cloud_hypervisor::net_util::{MacAddr, MAC_ADDR_LEN};
+
+use super::super::{Queue, VirtioDevice, TYPE_NET};
+use super::handler::*;
+use super::vu_common_ctrl::*;
+use super::{Error, Result};
+use crate::{pci::MsixConfig, virtio::Interrupt};
+use vhost_rs::vhost_user::message::VhostUserVirtioFeatures;
+use vhost_rs::vhost_user::{Master, VhostUserMaster};
+use vhost_rs::VhostBackend;
+use virtio_bindings::virtio_net;
+use virtio_bindings::virtio_ring;
+
+pub struct Net {
+    vhost_user_net: Master,
+    kill_evt: vmm_sys_util::eventfd::EventFd,
+    avail_features: u64,
+    acked_features: u64,
+    config_space: Vec<u8>,
+    queue_sizes: Vec<u16>,
+}
+
+impl<'a> Net {
+    /// Create a new vhost-user-net device
+    pub fn new(mac_addr: MacAddr, vu_cfg: VhostUserConfig<'a>) -> Result<Net> {
+        let mut vhost_user_net = Master::connect(vu_cfg.sock, vu_cfg.num_queues as u64)
+            .map_err(Error::VhostUserCreateMaster)?;
+
+        let kill_evt =
+            vmm_sys_util::eventfd::EventFd::new(EFD_NONBLOCK).map_err(Error::CreateKillEventFd)?;
+
+        // Filling device and vring features VMM supports.
+        let mut avail_features = 1 << virtio_net::VIRTIO_NET_F_GUEST_CSUM
+            | 1 << virtio_net::VIRTIO_NET_F_CSUM
+            | 1 << virtio_net::VIRTIO_NET_F_GUEST_TSO4
+            | 1 << virtio_net::VIRTIO_NET_F_GUEST_TSO6
+            | 1 << virtio_net::VIRTIO_NET_F_GUEST_ECN
+            | 1 << virtio_net::VIRTIO_NET_F_GUEST_UFO
+            | 1 << virtio_net::VIRTIO_NET_F_HOST_TSO4
+            | 1 << virtio_net::VIRTIO_NET_F_HOST_TSO6
+            | 1 << virtio_net::VIRTIO_NET_F_HOST_ECN
+            | 1 << virtio_net::VIRTIO_NET_F_HOST_UFO
+            | 1 << virtio_net::VIRTIO_NET_F_MRG_RXBUF
+            | 1 << virtio_net::VIRTIO_F_NOTIFY_ON_EMPTY
+            | 1 << virtio_net::VIRTIO_F_VERSION_1
+            | 1 << virtio_ring::VIRTIO_RING_F_EVENT_IDX
+            | VhostUserVirtioFeatures::PROTOCOL_FEATURES.bits();
+
+        vhost_user_net
+            .set_owner()
+            .map_err(Error::VhostUserSetOwner)?;
+
+        // Get features from backend, do negotiation to get a feature collection which
+        // both VMM and backend support.
+        let backend_features = vhost_user_net.get_features().unwrap();
+        avail_features &= backend_features;
+        // Set features back is required by the vhost crate mechanism, since the
+        // later vhost call will check if features is filled in master before execution.
+        vhost_user_net
+            .set_features(backend_features)
+            .map_err(Error::VhostUserSetFeatures)?;
+
+        let mut acked_features = 0;
+        if backend_features & VhostUserVirtioFeatures::PROTOCOL_FEATURES.bits() != 0 {
+            acked_features |= VhostUserVirtioFeatures::PROTOCOL_FEATURES.bits();
+            vhost_user_net
+                .get_protocol_features()
+                .map_err(Error::VhostUserGetProtocolFeatures)?;
+        } else {
+            return Err(Error::VhostUserProtocolNotSupport);
+        }
+
+        let mut config_space = Vec::with_capacity(MAC_ADDR_LEN);
+        unsafe { config_space.set_len(MAC_ADDR_LEN) }
+        config_space[..].copy_from_slice(mac_addr.get_bytes());
+        avail_features |= 1 << virtio_net::VIRTIO_NET_F_MAC;
+
+        // Send set_vring_base here, since it could tell backends, like OVS + DPDK,
+        // how many virt queues to be handled, which backend required to know at early stage.
+        for i in 0..vu_cfg.num_queues {
+            vhost_user_net
+                .set_vring_base(i, 0)
+                .map_err(Error::VhostUserSetVringBase)?;
+        }
+
+        Ok(Net {
+            vhost_user_net,
+            kill_evt,
+            avail_features,
+            acked_features,
+            config_space,
+            queue_sizes: vec![vu_cfg.queue_size; vu_cfg.num_queues],
+        })
+    }
+}
+
+impl Drop for Net {
+    fn drop(&mut self) {
+        if let Err(_e) = self.kill_evt.write(1) {
+            error!("failed to kill vhost-user-net with error {}", _e);
+        }
+    }
+}
+
+impl VirtioDevice for Net {
+    fn keep_fds(&self) -> Vec<RawFd> {
+        vec![self.kill_evt.as_raw_fd()]
+    }
+
+    fn device_type(&self) -> u32 {
+        TYPE_NET as u32
+    }
+
+    fn queue_max_sizes(&self) -> &[u16] {
+        &self.queue_sizes
+    }
+
+    fn features(&self) -> u64 {
+        self.avail_features
+    }
+
+    fn ack_features(&mut self, mut value: u64) {
+        // Check if the guest is ACK'ing a feature that we didn't claim to have.
+        let unrequested_features = value & !self.avail_features;
+        if unrequested_features != 0 {
+            warn!(
+                "Received acknowledge request for unknown feature: {:x}",
+                value
+            );
+            // Don't count these features as acked.
+            value &= !unrequested_features;
+        }
+        self.acked_features |= value;
+    }
+
+    fn read_config(&self, offset: u64, mut data: &mut [u8]) {
+        let config_len = self.config_space.len() as u64;
+        if offset >= config_len {
+            error!("Failed to read config space");
+            return;
+        }
+        if let Some(end) = offset.checked_add(data.len() as u64) {
+            // This write can't fail, offset and end are checked against config_len.
+            data.write_all(&self.config_space[offset as usize..cmp::min(end, config_len) as usize])
+                .unwrap();
+        }
+    }
+
+    fn write_config(&mut self, offset: u64, data: &[u8]) {
+        let data_len = data.len() as u64;
+        let config_len = self.config_space.len() as u64;
+        if offset + data_len > config_len {
+            error!("Failed to write config space");
+            return;
+        }
+        let (_, right) = self.config_space.split_at_mut(offset as usize);
+        right.copy_from_slice(&data[..]);
+    }
+
+    fn activate(
+        &mut self,
+        mem: GuestMemory,
+        interrupt_evt: sys_util::EventFd,
+        interrupt_resample_evt: sys_util::EventFd,
+        msix_config: Option<Arc<Mutex<MsixConfig>>>,
+        status: Arc<AtomicUsize>,
+        queues: Vec<Queue>,
+        queue_evts: Vec<EventFd>,
+    ) {
+        let handler_kill_evt = match self.kill_evt.try_clone() {
+            Ok(v) => v,
+            Err(e) => {
+                error!("failed creating kill EventFd pair: {}", e);
+                return;
+            }
+        };
+
+        let vu_interrupt_list = match setup_vhost_user(
+            &mut self.vhost_user_net,
+            &mem,
+            queues,
+            queue_evts,
+            self.acked_features,
+        ) {
+            Ok(v) => v,
+            Err(e) => {
+                error!("failed to set up vhost_user: {:?}", e);
+                return;
+            }
+        };
+
+        let vu_epoll_cfg = VhostUserEpollConfig {
+            interrupt: Interrupt::new(status, interrupt_evt, interrupt_resample_evt, msix_config),
+            kill_evt: handler_kill_evt,
+            vu_interrupt_list,
+        };
+
+        let _handler_result = thread::Builder::new()
+            .name("vhost_user_net".to_string())
+            .spawn(move || {
+                let mut handler = VhostUserEpollHandler::new(vu_epoll_cfg);
+                let result = handler.run();
+                if let Err(_e) = result {
+                    error!("net worker thread exited with error {:?}!", _e);
+                }
+            });
+        if let Err(_e) = _handler_result {
+            error!("vhost-user net thread create failed with error {:?}", _e);
+        }
+    }
+}
diff --git a/devices/src/virtio/vhost_user/vu_common_ctrl.rs b/devices/src/virtio/vhost_user/vu_common_ctrl.rs
new file mode 100644
index 0000000..38b7b4f
--- /dev/null
+++ b/devices/src/virtio/vhost_user/vu_common_ctrl.rs
@@ -0,0 +1,115 @@
+// Copyright 2019 Intel Corporation. All Rights Reserved.
+// SPDX-License-Identifier: Apache-2.0
+
+use libc;
+use libc::EFD_NONBLOCK;
+use std::os::unix::io::AsRawFd;
+use std::vec::Vec;
+use sys_util::{EventFd, GuestMemory};
+
+use vm_memory::{Address, Error as MmapError};
+
+use super::super::Queue;
+use super::{Error, Result};
+use vhost_rs::vhost_user::{Master, VhostUserMaster};
+use vhost_rs::{VhostBackend, VhostUserMemoryRegionInfo, VringConfigData};
+
+#[derive(Debug, Copy, Clone)]
+pub struct VhostUserConfig<'a> {
+    pub sock: &'a str,
+    pub num_queues: usize,
+    pub queue_size: u16,
+}
+
+pub fn setup_vhost_user_vring(
+    vu: &mut Master,
+    mem: &GuestMemory,
+    queues: Vec<Queue>,
+    queue_evts: Vec<EventFd>,
+) -> Result<Vec<(vmm_sys_util::eventfd::EventFd, Queue)>> {
+    let mut regions: Vec<VhostUserMemoryRegionInfo> = Vec::new();
+    mem.with_regions(
+        |_index,
+         region_guest_base,
+         region_mapping_size,
+         region_mapping_as_ptr,
+         region_memfd_offset| {
+            let vhost_user_net_reg = VhostUserMemoryRegionInfo {
+                guest_phys_addr: region_guest_base.0,
+                memory_size: region_mapping_size as u64,
+                userspace_addr: region_mapping_as_ptr as u64,
+                mmap_offset: region_memfd_offset as u64,
+                mmap_handle: mem.as_raw_fd(),
+            };
+
+            regions.push(vhost_user_net_reg);
+
+            Ok(())
+        },
+    )
+    .map_err(Error::VhostUserMemoryRegion)?;
+
+    vu.set_mem_table(regions.as_slice())
+        .map_err(Error::VhostUserSetMemTable)?;
+
+    let mut vu_interrupt_list = Vec::new();
+
+    for (queue_index, queue) in queues.into_iter().enumerate() {
+        vu.set_vring_num(queue_index, queue.max_size)
+            .map_err(Error::VhostUserSetVringNum)?;
+
+        let config_data = VringConfigData {
+            queue_max_size: queue.max_size,
+            queue_size: queue.actual_size(),
+            flags: 0u32,
+            desc_table_addr: mem
+                .get_host_address(queue.desc_table)
+                .map_err(Error::DescriptorTableAddress)? as u64,
+            used_ring_addr: mem
+                .get_host_address(queue.used_ring)
+                .map_err(Error::UsedAddress)? as u64,
+            avail_ring_addr: mem
+                .get_host_address(queue.avail_ring)
+                .map_err(Error::AvailAddress)? as u64,
+            log_addr: None,
+        };
+
+        vu.set_vring_addr(queue_index, &config_data)
+            .map_err(Error::VhostUserSetVringAddr)?;
+        vu.set_vring_base(queue_index, 0u16)
+            .map_err(Error::VhostUserSetVringBase)?;
+
+        let vhost_user_interrupt =
+            vmm_sys_util::eventfd::EventFd::new(EFD_NONBLOCK).map_err(Error::VhostIrqCreate)?;
+        vu.set_vring_call(queue_index, &vhost_user_interrupt)
+            .map_err(Error::VhostUserSetVringCall)?;
+        vu_interrupt_list.push((vhost_user_interrupt, queue));
+
+        vu.set_vring_kick(queue_index, &queue_evts[queue_index])
+            .map_err(Error::VhostUserSetVringKick)?;
+    }
+
+    Ok(vu_interrupt_list)
+}
+
+pub fn setup_vhost_user(
+    vu: &mut Master,
+    mem: &GuestMemory,
+    queues: Vec<Queue>,
+    queue_evts: Vec<EventFd>,
+    acked_features: u64,
+) -> Result<Vec<(vmm_sys_util::eventfd::EventFd, Queue)>> {
+    for i in 0..queues.len() {
+        vu.set_vring_enable(i, true)
+            .map_err(Error::VhostUserSetVringEnable)?;
+    }
+
+    let backend_features = vu.get_features().unwrap();
+    vu.set_features(acked_features & backend_features)
+        .map_err(Error::VhostUserSetFeatures)?;
+
+    match setup_vhost_user_vring(vu, mem, queues, queue_evts) {
+        Ok(vu_interrupt_list) => Ok(vu_interrupt_list),
+        Err(_) => Err(Error::VhostUserSetupVringFailed),
+    }
+}