virtio-devices: vhost_user: net: Move control queue back

We thought we could move the control queue to the backend as it was
making some good sense. Unfortunately, doing so was a wrong design
decision as it broke the compatibility with OVS-DPDK backend.

This is why this commit moves the control queue back to the VMM side,
meaning an additional thread is being run for handling the communication
with the guest.

Signed-off-by: Sebastien Boeuf <sebastien.boeuf@intel.com>
This commit is contained in:
Sebastien Boeuf 2021-05-25 17:14:53 +02:00 committed by Rob Bradford
parent 6c245f6cf1
commit e9cc23ea94
4 changed files with 188 additions and 34 deletions

View File

@ -122,12 +122,12 @@ pub fn build_net_config_space(
mut config: &mut VirtioNetConfig,
mac: MacAddr,
num_queues: usize,
mut avail_features: &mut u64,
avail_features: &mut u64,
) {
config.mac.copy_from_slice(mac.get_bytes());
*avail_features |= 1 << VIRTIO_NET_F_MAC;
build_net_config_space_with_mq(&mut config, num_queues, &mut avail_features);
build_net_config_space_with_mq(&mut config, num_queues, avail_features);
}
pub fn build_net_config_space_with_mq(

View File

@ -22,6 +22,7 @@ pub enum Thread {
VirtioPmem,
VirtioRng,
VirtioVhostFs,
VirtioVhostNetCtl,
VirtioVsock,
VirtioWatchdog,
}
@ -332,6 +333,26 @@ fn virtio_vhost_fs_thread_rules() -> Vec<SyscallRuleSet> {
]
}
fn virtio_vhost_net_ctl_thread_rules() -> Vec<SyscallRuleSet> {
vec![
allow_syscall(libc::SYS_brk),
allow_syscall(libc::SYS_close),
allow_syscall(libc::SYS_dup),
allow_syscall(libc::SYS_epoll_create1),
allow_syscall(libc::SYS_epoll_ctl),
allow_syscall(libc::SYS_epoll_pwait),
#[cfg(target_arch = "x86_64")]
allow_syscall(libc::SYS_epoll_wait),
allow_syscall(libc::SYS_exit),
allow_syscall(libc::SYS_futex),
allow_syscall(libc::SYS_munmap),
allow_syscall(libc::SYS_madvise),
allow_syscall(libc::SYS_read),
allow_syscall(libc::SYS_sigaltstack),
allow_syscall(libc::SYS_write),
]
}
fn create_vsock_ioctl_seccomp_rule() -> Vec<SeccompRule> {
or![and![Cond::new(1, ArgLen::DWORD, Eq, FIONBIO,).unwrap()],]
}
@ -399,6 +420,7 @@ fn get_seccomp_filter_trap(thread_type: Thread) -> Result<SeccompFilter, Error>
Thread::VirtioPmem => virtio_pmem_thread_rules(),
Thread::VirtioRng => virtio_rng_thread_rules(),
Thread::VirtioVhostFs => virtio_vhost_fs_thread_rules(),
Thread::VirtioVhostNetCtl => virtio_vhost_net_ctl_thread_rules(),
Thread::VirtioVsock => virtio_vsock_thread_rules(),
Thread::VirtioWatchdog => virtio_watchdog_thread_rules(),
};
@ -418,6 +440,7 @@ fn get_seccomp_filter_log(thread_type: Thread) -> Result<SeccompFilter, Error> {
Thread::VirtioPmem => virtio_pmem_thread_rules(),
Thread::VirtioRng => virtio_rng_thread_rules(),
Thread::VirtioVhostFs => virtio_vhost_fs_thread_rules(),
Thread::VirtioVhostNetCtl => virtio_vhost_net_ctl_thread_rules(),
Thread::VirtioVsock => virtio_vsock_thread_rules(),
Thread::VirtioWatchdog => virtio_watchdog_thread_rules(),
};

View File

@ -2,26 +2,30 @@
// SPDX-License-Identifier: Apache-2.0
use super::super::{
ActivateError, ActivateResult, Queue, VirtioCommon, VirtioDevice, VirtioDeviceType,
ActivateError, ActivateResult, EpollHelper, EpollHelperError, EpollHelperHandler, Queue,
VirtioCommon, VirtioDevice, VirtioDeviceType, EPOLL_HELPER_EVENT_LAST,
};
use super::vu_common_ctrl::*;
use super::{Error, Result, DEFAULT_VIRTIO_FEATURES};
use crate::VirtioInterrupt;
use net_util::{build_net_config_space, MacAddr, VirtioNetConfig};
use super::{Error, Result};
use crate::seccomp_filters::{get_seccomp_filter, Thread};
use crate::{VirtioInterrupt, VIRTIO_F_RING_EVENT_IDX, VIRTIO_F_VERSION_1};
use net_util::{build_net_config_space, CtrlQueue, MacAddr, VirtioNetConfig};
use seccomp::{SeccompAction, SeccompFilter};
use std::ops::Deref;
use std::os::unix::io::AsRawFd;
use std::os::unix::net::UnixListener;
use std::result;
use std::sync::atomic::AtomicBool;
use std::sync::{Arc, Barrier};
use std::thread;
use std::vec::Vec;
use vhost::vhost_user::message::{VhostUserProtocolFeatures, VhostUserVirtioFeatures};
use vhost::vhost_user::{Master, VhostUserMaster, VhostUserMasterReqHandler};
use vhost::VhostBackend;
use virtio_bindings::bindings::virtio_net::{
VIRTIO_NET_F_CSUM, VIRTIO_NET_F_CTRL_VQ, VIRTIO_NET_F_GUEST_CSUM, VIRTIO_NET_F_GUEST_ECN,
VIRTIO_NET_F_GUEST_TSO4, VIRTIO_NET_F_GUEST_TSO6, VIRTIO_NET_F_GUEST_UFO,
VIRTIO_NET_F_HOST_ECN, VIRTIO_NET_F_HOST_TSO4, VIRTIO_NET_F_HOST_TSO6, VIRTIO_NET_F_HOST_UFO,
VIRTIO_NET_F_MRG_RXBUF,
VIRTIO_NET_F_MAC, VIRTIO_NET_F_MRG_RXBUF,
};
use vm_memory::{
ByteValued, GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap, GuestRegionMmap,
@ -34,6 +38,58 @@ const DEFAULT_QUEUE_NUMBER: usize = 2;
struct SlaveReqHandler {}
impl VhostUserMasterReqHandler for SlaveReqHandler {}
/// Control queue
// Event available on the control queue.
const CTRL_QUEUE_EVENT: u16 = EPOLL_HELPER_EVENT_LAST + 1;
pub struct NetCtrlEpollHandler {
pub mem: GuestMemoryAtomic<GuestMemoryMmap>,
pub kill_evt: EventFd,
pub pause_evt: EventFd,
pub ctrl_q: CtrlQueue,
pub queue_evt: EventFd,
pub queue: Queue,
}
impl NetCtrlEpollHandler {
pub fn run_ctrl(
&mut self,
paused: Arc<AtomicBool>,
paused_sync: Arc<Barrier>,
) -> std::result::Result<(), EpollHelperError> {
let mut helper = EpollHelper::new(&self.kill_evt, &self.pause_evt)?;
helper.add_event(self.queue_evt.as_raw_fd(), CTRL_QUEUE_EVENT)?;
helper.run(paused, paused_sync, self)?;
Ok(())
}
}
impl EpollHelperHandler for NetCtrlEpollHandler {
fn handle_event(&mut self, _helper: &mut EpollHelper, event: &epoll::Event) -> bool {
let ev_type = event.data as u16;
match ev_type {
CTRL_QUEUE_EVENT => {
let mem = self.mem.memory();
if let Err(e) = self.queue_evt.read() {
error!("failed to get ctl queue event: {:?}", e);
return true;
}
if let Err(e) = self.ctrl_q.process(&mem, &mut self.queue) {
error!("failed to process ctrl queue: {:?}", e);
return true;
}
}
_ => {
error!("Unknown event for virtio-net");
return true;
}
}
false
}
}
pub struct Net {
common: VirtioCommon,
id: String,
@ -42,6 +98,8 @@ pub struct Net {
guest_memory: Option<GuestMemoryAtomic<GuestMemoryMmap>>,
acked_protocol_features: u64,
socket_path: Option<String>,
ctrl_queue_epoll_thread: Option<thread::JoinHandle<()>>,
seccomp_action: SeccompAction,
}
impl Net {
@ -51,6 +109,7 @@ impl Net {
mac_addr: MacAddr,
vu_cfg: VhostUserConfig,
server: bool,
seccomp_action: SeccompAction,
) -> Result<Net> {
let mut socket_path: Option<String> = None;
@ -69,14 +128,13 @@ impl Net {
| 1 << VIRTIO_NET_F_HOST_UFO
| 1 << VIRTIO_NET_F_MRG_RXBUF
| 1 << VIRTIO_NET_F_CTRL_VQ
| DEFAULT_VIRTIO_FEATURES;
| 1 << VIRTIO_F_RING_EVENT_IDX
| 1 << VIRTIO_F_VERSION_1
| VhostUserVirtioFeatures::PROTOCOL_FEATURES.bits();
let mut config = VirtioNetConfig::default();
build_net_config_space(&mut config, mac_addr, num_queues, &mut avail_features);
// Adding one potential queue for the control queue.
num_queues += 1;
let mut vhost_user_net = if server {
info!("Binding vhost-user-net listener...");
let listener = UnixListener::bind(&vu_cfg.socket).map_err(Error::BindSocket)?;
@ -95,25 +153,17 @@ impl Net {
| VhostUserProtocolFeatures::CONFIGURE_MEM_SLOTS
| VhostUserProtocolFeatures::REPLY_ACK;
let (acked_features, acked_protocol_features) = negotiate_features_vhost_user(
let (mut acked_features, acked_protocol_features) = negotiate_features_vhost_user(
&mut vhost_user_net,
avail_features,
avail_protocol_features,
)?;
// If the control queue feature has not been negotiated, let's decrease
// the number of queues.
if acked_features & (1 << VIRTIO_NET_F_CTRL_VQ) == 0 {
num_queues -= 1;
}
let backend_num_queues =
if acked_protocol_features & VhostUserProtocolFeatures::MQ.bits() != 0 {
vhost_user_net
.get_queue_num()
.map_err(Error::VhostUserGetQueueMaxNum)? as usize
} else if acked_features & (1 << VIRTIO_NET_F_CTRL_VQ) != 0 {
DEFAULT_QUEUE_NUMBER + 1
} else {
DEFAULT_QUEUE_NUMBER
};
@ -124,14 +174,16 @@ impl Net {
return Err(Error::BadQueueNum);
}
// 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..num_queues {
vhost_user_net
.set_vring_base(i, 0)
.map_err(Error::VhostUserSetVringBase)?;
// If the control queue feature has been negotiated, let's increase
// the number of queues.
if acked_features & (1 << VIRTIO_NET_F_CTRL_VQ) != 0 {
num_queues += 1;
}
// Make sure the virtio feature to set the MAC address is exposed to
// the guest, even if it hasn't been negotiated with the backend.
acked_features |= 1 << VIRTIO_NET_F_MAC;
Ok(Net {
id,
common: VirtioCommon {
@ -148,6 +200,8 @@ impl Net {
guest_memory: None,
acked_protocol_features,
socket_path,
ctrl_queue_epoll_thread: None,
seccomp_action,
})
}
}
@ -187,21 +241,88 @@ impl VirtioDevice for Net {
&mut self,
mem: GuestMemoryAtomic<GuestMemoryMmap>,
interrupt_cb: Arc<dyn VirtioInterrupt>,
queues: Vec<Queue>,
queue_evts: Vec<EventFd>,
mut queues: Vec<Queue>,
mut queue_evts: Vec<EventFd>,
) -> ActivateResult {
self.common.activate(&queues, &queue_evts, &interrupt_cb)?;
self.guest_memory = Some(mem.clone());
let num_queues = queues.len();
if self.common.feature_acked(VIRTIO_NET_F_CTRL_VQ.into()) && num_queues % 2 != 0 {
let cvq_queue = queues.remove(num_queues - 1);
let cvq_queue_evt = queue_evts.remove(num_queues - 1);
let kill_evt = self
.common
.kill_evt
.as_ref()
.unwrap()
.try_clone()
.map_err(|e| {
error!("failed to clone kill_evt eventfd: {}", e);
ActivateError::BadActivate
})?;
let pause_evt = self
.common
.pause_evt
.as_ref()
.unwrap()
.try_clone()
.map_err(|e| {
error!("failed to clone pause_evt eventfd: {}", e);
ActivateError::BadActivate
})?;
let mut ctrl_handler = NetCtrlEpollHandler {
mem: mem.clone(),
kill_evt,
pause_evt,
ctrl_q: CtrlQueue::new(Vec::new()),
queue: cvq_queue,
queue_evt: cvq_queue_evt,
};
let paused = self.common.paused.clone();
// Let's update the barrier as we need 1 for the control queue + 1
// for the main thread signalling the pause.
self.common.paused_sync = Some(Arc::new(Barrier::new(2)));
let paused_sync = self.common.paused_sync.clone();
// Retrieve seccomp filter for virtio_net_ctl thread
let virtio_vhost_net_ctl_seccomp_filter =
get_seccomp_filter(&self.seccomp_action, Thread::VirtioVhostNetCtl)
.map_err(ActivateError::CreateSeccompFilter)?;
thread::Builder::new()
.name(format!("{}_ctrl", self.id))
.spawn(move || {
if let Err(e) = SeccompFilter::apply(virtio_vhost_net_ctl_seccomp_filter) {
error!("Error applying seccomp filter: {:?}", e);
} else if let Err(e) = ctrl_handler.run_ctrl(paused, paused_sync.unwrap()) {
error!("Error running worker: {:?}", e);
}
})
.map(|thread| self.ctrl_queue_epoll_thread = Some(thread))
.map_err(|e| {
error!("failed to clone queue EventFd: {}", e);
ActivateError::BadActivate
})?;
}
// The backend acknowledged features must contain the protocol feature
// bit in case it was initially set but lost through the features
// negotiation with the guest. Additionally, it must not contain
// VIRTIO_NET_F_MAC since we don't expect the backend to handle it.
let backend_acked_features = self.common.acked_features & !(1 << VIRTIO_NET_F_MAC)
| (self.common.avail_features & VhostUserVirtioFeatures::PROTOCOL_FEATURES.bits());
setup_vhost_user(
&mut self.vhost_user_net,
&mem.memory(),
queues,
queue_evts,
&interrupt_cb,
self.common.acked_features
| (self.common.avail_features & VhostUserVirtioFeatures::PROTOCOL_FEATURES.bits()),
backend_acked_features,
)
.map_err(ActivateError::VhostUserNetSetup)?;
@ -262,7 +383,12 @@ impl Pausable for Net {
}
fn resume(&mut self) -> result::Result<(), MigratableError> {
self.common.resume()
self.common.resume()?;
if let Some(ctrl_queue_epoll_thread) = &self.ctrl_queue_epoll_thread {
ctrl_queue_epoll_thread.thread().unpark();
}
Ok(())
}
}

View File

@ -2039,8 +2039,13 @@ impl DeviceManager {
VhostMode::Server => true,
};
let vhost_user_net_device = Arc::new(Mutex::new(
match virtio_devices::vhost_user::Net::new(id.clone(), net_cfg.mac, vu_cfg, server)
{
match virtio_devices::vhost_user::Net::new(
id.clone(),
net_cfg.mac,
vu_cfg,
server,
self.seccomp_action.clone(),
) {
Ok(vun_device) => vun_device,
Err(e) => {
return Err(DeviceManagerError::CreateVhostUserNet(e));