virtio: Implement the 9P device

Implement a new virtio_9p device to be used for sharing directories with
the VM.

BUG=chromium:703939
TEST=mount inside a VM and run `bonnie++ -r 256`
Append the shared directory to the crosvm command line:
--shared-dir /path/to/dir:test_9p
Then mount in the guest:
mkdir /tmp/9p
mount -t 9p -o trans=virtio test_9p /tmp/9p -oversion=9p2000.L
Or for a 9p root:
run --shared-dir /mnt/vm_root:/dev/root -p 'root=/dev/root ro rootflags=ro,trans=virtio,version=9p2000.L,cache=loose rootfstype=9p' vmlinux.bin

CQ-DEPEND=CL:1065170

Change-Id: I41fc21306ab5fa318a271f172d7057b767b29f31
Signed-off-by: Chirantan Ekbote <chirantan@chromium.org>
Reviewed-on: https://chromium-review.googlesource.com/1065173
Commit-Ready: ChromeOS CL Exonerator Bot <chromiumos-cl-exonerator@appspot.gserviceaccount.com>
Reviewed-by: Dylan Reid <dgreid@chromium.org>
This commit is contained in:
Chirantan Ekbote 2018-04-16 19:32:04 -07:00 committed by chrome-bot
parent a79073ad7d
commit ebd56813e1
9 changed files with 663 additions and 1 deletions

1
Cargo.lock generated
View file

@ -133,6 +133,7 @@ dependencies = [
"libc 0.2.40 (registry+https://github.com/rust-lang/crates.io-index)",
"net_sys 0.1.0",
"net_util 0.1.0",
"p9 0.1.0",
"resources 0.1.0",
"sys_util 0.1.0",
"vhost 0.1.0",

View file

@ -17,6 +17,7 @@ libc = "*"
io_jail = { path = "../io_jail" }
net_sys = { path = "../net_sys" }
net_util = { path = "../net_util" }
p9 = { path = "../p9" }
resources = { path = "../resources" }
sys_util = { path = "../sys_util" }
vhost = { path = "../vhost" }

View file

@ -10,6 +10,7 @@ extern crate io_jail;
extern crate libc;
extern crate net_sys;
extern crate net_util;
extern crate p9;
extern crate resources;
#[macro_use]
extern crate sys_util;

View file

@ -12,6 +12,7 @@ mod rng;
mod net;
#[cfg(feature = "gpu")]
mod gpu;
mod p9;
mod wl;
pub mod vhost;
@ -24,6 +25,7 @@ pub use self::rng::*;
pub use self::net::*;
#[cfg(feature = "gpu")]
pub use self::gpu::*;
pub use self::p9::*;
pub use self::wl::*;
const DEVICE_ACKNOWLEDGE: u32 = 0x01;
@ -39,6 +41,7 @@ const TYPE_RNG: u32 = 4;
const TYPE_BALLOON: u32 = 5;
#[allow(dead_code)]
const TYPE_GPU: u32 = 16;
const TYPE_9P: u32 = 9;
const TYPE_VSOCK: u32 = 19;
const TYPE_WL: u32 = 30;

468
devices/src/virtio/p9.rs Normal file
View file

@ -0,0 +1,468 @@
// Copyright 2018 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 file.
use std::cmp::min;
use std::error;
use std::fmt;
use std::io::{self, Read, Write};
use std::iter::Peekable;
use std::mem;
use std::os::unix::io::RawFd;
use std::path::{Path, PathBuf};
use std::result;
use std::sync::Arc;
use std::sync::atomic::{AtomicUsize, Ordering};
use std::thread;
use p9;
use sys_util::{Error as SysError, EventFd, GuestAddress, GuestMemory, PollContext, PollToken};
use virtio_sys::vhost::VIRTIO_F_VERSION_1;
use super::{DescriptorChain, Queue, TYPE_9P, VirtioDevice, INTERRUPT_STATUS_USED_RING};
const QUEUE_SIZE: u16 = 128;
const QUEUE_SIZES: &'static [u16] = &[QUEUE_SIZE];
// The only virtio_9p feature.
const VIRTIO_9P_MOUNT_TAG: u8 = 0;
/// Errors that occur during operation of a virtio 9P device.
#[derive(Debug)]
pub enum P9Error {
/// The tag for the 9P device was too large to fit in the config space.
TagTooLong(usize),
/// The root directory for the 9P server is not absolute.
RootNotAbsolute(PathBuf),
/// Creating PollContext failed.
CreatePollContext(SysError),
/// Error while polling for events.
PollError(SysError),
/// Error while reading from the virtio queue's EventFd.
ReadQueueEventFd(SysError),
/// A request is missing readable descriptors.
NoReadableDescriptors,
/// A request is missing writable descriptors.
NoWritableDescriptors,
/// A descriptor contained an invalid guest address range.
InvalidGuestAddress(GuestAddress, u32),
/// Failed to signal the virio used queue.
SignalUsedQueue(SysError),
/// An internal I/O error occurred.
Internal(io::Error),
}
impl error::Error for P9Error {
fn description(&self) -> &str {
"An error occurred in the virtio 9P device"
}
}
impl fmt::Display for P9Error {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
&P9Error::TagTooLong(len) => write!(
f,
"P9 device tag is too long: len = {}, max = {}",
len,
::std::u16::MAX
),
&P9Error::RootNotAbsolute(ref buf) => write!(
f,
"P9 root directory is not absolute: root = {}",
buf.display()
),
&P9Error::CreatePollContext(ref err) => {
write!(f, "failed to create PollContext: {:?}", err)
}
&P9Error::PollError(ref err) => write!(f, "failed to poll events: {:?}", err),
&P9Error::ReadQueueEventFd(ref err) => {
write!(f, "failed to read from virtio queue EventFd: {:?}", err)
}
&P9Error::NoReadableDescriptors => {
write!(f, "request does not have any readable descriptors")
}
&P9Error::NoWritableDescriptors => {
write!(f, "request does not have any writable descriptors")
}
&P9Error::InvalidGuestAddress(addr, len) => write!(
f,
"descriptor contained invalid guest address range: address = {:?}, len = {}",
addr, len
),
&P9Error::SignalUsedQueue(ref err) => {
write!(f, "failed to signal used queue: {:?}", err)
}
&P9Error::Internal(ref err) => write!(f, "P9 internal server error: {}", err),
}
}
}
pub type P9Result<T> = result::Result<T, P9Error>;
struct Reader<'a, I>
where
I: Iterator<Item = DescriptorChain<'a>>,
{
mem: &'a GuestMemory,
offset: u32,
iter: Peekable<I>,
}
impl<'a, I> Read for Reader<'a, I>
where
I: Iterator<Item = DescriptorChain<'a>>,
{
fn read(&mut self, buf: &mut [u8]) -> io::Result<usize> {
let needs_advance = if let Some(current) = self.iter.peek() {
self.offset >= current.len
} else {
false
};
if needs_advance {
self.offset = 0;
self.iter.next();
}
if let Some(current) = self.iter.peek() {
debug_assert!(current.is_read_only());
let addr = current
.addr
.checked_add(self.offset as u64)
.ok_or_else(|| {
io::Error::new(
io::ErrorKind::InvalidData,
P9Error::InvalidGuestAddress(current.addr, current.len),
)
})?;
let len = min(buf.len(), (current.len - self.offset) as usize);
let count = self.mem
.read_slice_at_addr(&mut buf[..len], addr)
.map_err(|e| io::Error::new(io::ErrorKind::Other, e))?;
// |count| has to fit into a u32 because it must be less than or equal to
// |current.len|, which does fit into a u32.
self.offset += count as u32;
Ok(count)
} else {
// Nothing left to read.
Ok(0)
}
}
}
struct Writer<'a, I>
where
I: Iterator<Item = DescriptorChain<'a>>,
{
mem: &'a GuestMemory,
bytes_written: u32,
offset: u32,
iter: Peekable<I>,
}
impl<'a, I> Write for Writer<'a, I>
where
I: Iterator<Item = DescriptorChain<'a>>,
{
fn write(&mut self, buf: &[u8]) -> io::Result<usize> {
let needs_advance = if let Some(current) = self.iter.peek() {
self.offset >= current.len
} else {
false
};
if needs_advance {
self.offset = 0;
self.iter.next();
}
if let Some(current) = self.iter.peek() {
debug_assert!(current.is_write_only());
let addr = current
.addr
.checked_add(self.offset as u64)
.ok_or_else(|| {
io::Error::new(
io::ErrorKind::InvalidData,
P9Error::InvalidGuestAddress(current.addr, current.len),
)
})?;
let len = min(buf.len(), (current.len - self.offset) as usize);
let count = self.mem
.write_slice_at_addr(&buf[..len], addr)
.map_err(|e| io::Error::new(io::ErrorKind::Other, e))?;
// |count| has to fit into a u32 because it must be less than or equal to
// |current.len|, which does fit into a u32.
self.offset += count as u32;
self.bytes_written += count as u32;
Ok(count)
} else {
// No more room in the descriptor chain.
Ok(0)
}
}
fn flush(&mut self) -> io::Result<()> {
// Nothing to flush since the writes go straight into the buffer.
Ok(())
}
}
struct Worker {
mem: GuestMemory,
queue: Queue,
server: p9::Server,
irq_status: Arc<AtomicUsize>,
irq_evt: EventFd,
}
impl Worker {
fn signal_used_queue(&self) -> P9Result<()> {
self.irq_status
.fetch_or(INTERRUPT_STATUS_USED_RING as usize, Ordering::SeqCst);
self.irq_evt.write(1).map_err(P9Error::SignalUsedQueue)
}
fn process_queue(&mut self) -> P9Result<()> {
let mut used_desc_heads = [(0, 0); QUEUE_SIZE as usize];
let mut used_count = 0;
for avail_desc in self.queue.iter(&self.mem) {
let mut reader = Reader {
mem: &self.mem,
offset: 0,
iter: avail_desc
.clone()
.into_iter()
.readable()
.peekable(),
};
let mut writer = Writer {
mem: &self.mem,
bytes_written: 0,
offset: 0,
iter: avail_desc
.clone()
.into_iter()
.writable()
.peekable(),
};
self.server
.handle_message(&mut reader, &mut writer)
.map_err(P9Error::Internal)?;
used_desc_heads[used_count] = (avail_desc.index, writer.bytes_written);
used_count += 1;
}
for &(idx, count) in &used_desc_heads[..used_count] {
self.queue.add_used(&self.mem, idx, count);
}
self.signal_used_queue()?;
Ok(())
}
fn run(&mut self, queue_evt: EventFd, kill_evt: EventFd) -> P9Result<()> {
#[derive(PollToken)]
enum Token {
// A request is ready on the queue.
QueueReady,
// The parent thread requested an exit.
Kill,
}
let poll_ctx: PollContext<Token> = PollContext::new()
.and_then(|pc| pc.add(&queue_evt, Token::QueueReady).and(Ok(pc)))
.and_then(|pc| pc.add(&kill_evt, Token::Kill).and(Ok(pc)))
.map_err(P9Error::CreatePollContext)?;
loop {
let events = poll_ctx.wait().map_err(P9Error::PollError)?;
for event in events.iter_readable() {
match event.token() {
Token::QueueReady => {
queue_evt.read().map_err(P9Error::ReadQueueEventFd)?;
self.process_queue()?;
}
Token::Kill => return Ok(()),
}
}
}
}
}
/// Virtio device for sharing specific directories on the host system with the guest VM.
pub struct P9 {
config: Vec<u8>,
server: Option<p9::Server>,
kill_evt: Option<EventFd>,
avail_features: u64,
acked_features: u64,
worker: Option<thread::JoinHandle<P9Result<()>>>,
}
impl P9 {
pub fn new<P: AsRef<Path>>(root: P, tag: &str) -> P9Result<P9> {
if tag.len() > ::std::u16::MAX as usize {
return Err(P9Error::TagTooLong(tag.len()));
}
if !root.as_ref().is_absolute() {
return Err(P9Error::RootNotAbsolute(root.as_ref().into()));
}
let len = tag.len() as u16;
let mut cfg = Vec::with_capacity(tag.len() + mem::size_of::<u16>());
cfg.push(len as u8);
cfg.push((len >> 8) as u8);
cfg.write_all(tag.as_bytes()).map_err(P9Error::Internal)?;
Ok(P9 {
config: cfg,
server: Some(p9::Server::new(root)),
kill_evt: None,
avail_features: 1 << VIRTIO_9P_MOUNT_TAG | 1 << VIRTIO_F_VERSION_1,
acked_features: 0,
worker: None,
})
}
}
impl VirtioDevice for P9 {
fn keep_fds(&self) -> Vec<RawFd> {
Vec::new()
}
fn device_type(&self) -> u32 {
TYPE_9P
}
fn queue_max_sizes(&self) -> &[u16] {
QUEUE_SIZES
}
fn features(&self, page: u32) -> u32 {
match page {
0 => self.avail_features as u32,
1 => (self.avail_features >> 32) as u32,
_ => {
warn!("virtio_9p got request for features page: {}", page);
0u32
}
}
}
fn ack_features(&mut self, page: u32, value: u32) {
let mut v = match page {
0 => value as u64,
1 => (value as u64) << 32,
_ => {
warn!("virtio_9p device cannot ack unknown feature page: {}", page);
0u64
}
};
// Check if the guest is ACK'ing a feature that we didn't claim to have.
let unrequested_features = v & !self.avail_features;
if unrequested_features != 0 {
warn!(
"virtio_9p got unknown feature ack: {:x}, page = {}, value = {:x}",
v, page, value
);
// Don't count these features as acked.
v &= !unrequested_features;
}
self.acked_features |= v;
}
fn read_config(&self, offset: u64, data: &mut [u8]) {
if offset >= self.config.len() as u64 {
// Nothing to see here.
return;
}
// The config length cannot be more than ::std::u16::MAX + mem::size_of::<u16>(), which
// is significantly smaller than ::std::usize::MAX on the architectures we care about so
// if we reach this point then we know that `offset` will fit into a usize.
let offset = offset as usize;
let len = min(data.len(), self.config.len() - offset);
data[..len].copy_from_slice(&self.config[offset..offset + len])
}
fn activate(
&mut self,
guest_mem: GuestMemory,
interrupt_evt: EventFd,
status: Arc<AtomicUsize>,
mut queues: Vec<Queue>,
mut queue_evts: Vec<EventFd>,
) {
if queues.len() != 1 || queue_evts.len() != 1 {
return;
}
let (self_kill_evt, kill_evt) = match EventFd::new().and_then(|e| Ok((e.try_clone()?, e))) {
Ok(v) => v,
Err(e) => {
error!("failed creating kill EventFd pair: {:?}", e);
return;
}
};
self.kill_evt = Some(self_kill_evt);
if let Some(server) = self.server.take() {
let worker_result = thread::Builder::new().name("virtio_9p".to_string()).spawn(
move || {
let mut worker = Worker {
mem: guest_mem,
queue: queues.remove(0),
server: server,
irq_status: status,
irq_evt: interrupt_evt,
};
worker.run(queue_evts.remove(0), kill_evt)
},
);
match worker_result {
Ok(worker) => self.worker = Some(worker),
Err(e) => error!("failed to spawn virtio_9p worker: {:?}", e),
}
}
}
}
impl Drop for P9 {
fn drop(&mut self) {
if let Some(kill_evt) = self.kill_evt.take() {
if let Err(e) = kill_evt.write(1) {
error!("failed to kill virtio_9p worker thread: {:?}", e);
return;
}
// Only wait on the child thread if we were able to send it a kill event.
if let Some(worker) = self.worker.take() {
match worker.join() {
Ok(r) => {
if let Err(e) = r {
error!("virtio_9p worker thread exited with error: {}", e)
}
}
Err(e) => error!("virtio_9p worker thread panicked: {:?}", e),
}
}
}
}
}

View file

@ -0,0 +1,54 @@
# Copyright 2018 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 file.
write: 1
recv: 1
read: 1
epoll_wait: 1
pread64: 1
pwrite64: 1
lstat64: 1
stat64: 1
close: 1
prctl: arg0 == PR_SET_NAME
open: 1
fstat64: 1
# ioctl(fd, FIOCLEX, 0) is equivalent to fcntl(fd, F_SETFD, FD_CLOEXEC).
ioctl: arg1 == FIOCLEX
getdents64: 1
fdatasync: 1
fsync: 1
# Disallow mmap with PROT_EXEC set. The syntax here doesn't allow bit
# negation, thus the manually negated mask constant.
mmap2: arg2 in 0xfffffffb
mprotect: arg2 in 0xfffffffb
sigaltstack: 1
munmap: 1
mkdir: 1
rmdir: 1
epoll_ctl: 1
rename: 1
writev: 1
link: 1
unlink: 1
restart_syscall: 1
exit: 1
rt_sigreturn: 1
epoll_create1: 1
sched_getaffinity: 1
dup: 1
# Disallow clone's other than new threads.
clone: arg0 & 0x00010000
set_robust_list: 1
exit_group: 1
socket: arg0 == AF_UNIX
futex: 1
eventfd2: 1
mremap: 1
# Allow MADV_DONTDUMP and MADV_DONTNEED only.
madvise: arg2 == 0x00000010 || arg2 == 0x00000004
utimensat: 1
ftruncate64: 1
fchown: arg1 == 0xffffffff && arg2 == 0xffffffff
statfs64: 1

View file

@ -0,0 +1,54 @@
# Copyright 2018 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 file.
write: 1
writev: 1
recvfrom: 1
epoll_wait: 1
read: 1
pwrite64: 1
stat: 1
lstat: 1
close: 1
open: 1
fstat: 1
# ioctl(fd, FIOCLEX, 0) is equivalent to fcntl(fd, F_SETFD, FD_CLOEXEC).
ioctl: arg1 == FIOCLEX
link: 1
unlink: 1
rename: 1
pread64: 1
getdents: 1
# Disallow mmap with PROT_EXEC set. The syntax here doesn't allow bit
# negation, thus the manually negated mask constant.
mmap: arg2 in 0xfffffffb
mprotect: arg2 in 0xfffffffb
munmap: 1
mkdir: 1
sigaltstack: 1
epoll_ctl: 1
mremap: 1
rmdir: 1
fsync: 1
fdatasync: 1
restart_syscall: 1
exit: 1
rt_sigreturn: 1
epoll_create1: 1
prctl: arg0 == PR_SET_NAME
eventfd2: 1
sched_getaffinity: 1
dup: 1
getpid: 1
# Disallow clone's other than new threads.
clone: arg0 & 0x00010000
set_robust_list: 1
exit_group: 1
# Allow MADV_DONTDUMP and MADV_DONTNEED only.
madvise: arg2 == 0x00000010 || arg2 == 0x00000004
futex: 1
utimensat: 1
ftruncate: 1
fchown: arg1 == 0xffffffff && arg2 == 0xffffffff
statfs: 1

View file

@ -74,6 +74,7 @@ pub enum Error {
NetDeviceNew(devices::virtio::NetError),
NoVarEmpty,
OpenKernel(PathBuf, io::Error),
P9DeviceNew(devices::virtio::P9Error),
PollContextAdd(sys_util::Error),
PollContextDelete(sys_util::Error),
QcowDeviceCreate(qcow::Error),
@ -83,6 +84,7 @@ pub enum Error {
RegisterBlock(MmioRegisterError),
RegisterGpu(MmioRegisterError),
RegisterNet(MmioRegisterError),
RegisterP9(MmioRegisterError),
RegisterRng(MmioRegisterError),
RegisterSignalHandler(sys_util::Error),
RegisterVsock(MmioRegisterError),
@ -139,6 +141,7 @@ impl fmt::Display for Error {
&Error::OpenKernel(ref p, ref e) => {
write!(f, "failed to open kernel image {:?}: {}", p, e)
}
&Error::P9DeviceNew(ref e) => write!(f, "failed to create 9p device: {}", e),
&Error::PollContextAdd(ref e) => write!(f, "failed to add fd to poll context: {:?}", e),
&Error::PollContextDelete(ref e) => {
write!(f, "failed to remove fd from poll context: {:?}", e)
@ -158,6 +161,7 @@ impl fmt::Display for Error {
&Error::RegisterBlock(ref e) => write!(f, "error registering block device: {:?}", e),
&Error::RegisterGpu(ref e) => write!(f, "error registering gpu device: {:?}", e),
&Error::RegisterNet(ref e) => write!(f, "error registering net device: {:?}", e),
&Error::RegisterP9(ref e) => write!(f, "error registering 9p device: {:?}", e),
&Error::RegisterRng(ref e) => write!(f, "error registering rng device: {:?}", e),
&Error::RegisterSignalHandler(ref e) => {
write!(f, "error registering signal handler: {:?}", e)
@ -602,6 +606,52 @@ fn setup_mmio_bus(cfg: &Config,
}
}
let chronos_user_group = CStr::from_bytes_with_nul(b"chronos\0").unwrap();
let chronos_uid = match get_user_id(&chronos_user_group) {
Ok(u) => u,
Err(e) => {
warn!("falling back to current user id for 9p: {:?}", e);
geteuid()
}
};
let chronos_gid = match get_group_id(&chronos_user_group) {
Ok(u) => u,
Err(e) => {
warn!("falling back to current group id for 9p: {:?}", e);
getegid()
}
};
for &(ref src, ref tag) in &cfg.shared_dirs {
let (jail, root) = if cfg.multiprocess {
let policy_path: PathBuf = cfg.seccomp_policy_dir.join("9p_device.policy");
let mut jail = create_base_minijail(empty_root_path, &policy_path)?;
// The shared directory becomes the root of the device's file system.
let root = Path::new("/");
jail.mount_bind(&src, root, true).unwrap();
// Set the uid/gid for the jailed process, and give a basic id map. This
// is required for the above bind mount to work.
jail.change_uid(chronos_uid);
jail.change_gid(chronos_gid);
jail.uidmap(&format!("{0} {0} 1", chronos_uid))
.map_err(Error::SettingUidMap)?;
jail.gidmap(&format!("{0} {0} 1", chronos_gid))
.map_err(Error::SettingGidMap)?;
(Some(jail), root)
} else {
// There's no bind mount so we tell the server to treat the source directory as the
// root. The double deref here converts |src| from a &PathBuf into a &Path.
(None, &**src)
};
let p9_box = Box::new(devices::virtio::P9::new(root, tag).map_err(Error::P9DeviceNew)?);
register_mmio(&mut bus, vm, p9_box, jail, resources, cmdline).map_err(Error::RegisterP9)?;
}
Ok(bus)
}

View file

@ -80,6 +80,7 @@ pub struct Config {
wayland_socket_path: Option<PathBuf>,
wayland_dmabuf: bool,
socket_path: Option<PathBuf>,
shared_dirs: Vec<(PathBuf, String)>,
multiprocess: bool,
seccomp_policy_dir: PathBuf,
cid: Option<u64>,
@ -105,6 +106,7 @@ impl Default for Config {
wayland_dmabuf: false,
socket_path: None,
multiprocess: !cfg!(feature = "default-no-sandbox"),
shared_dirs: Vec::new(),
seccomp_policy_dir: PathBuf::from(SECCOMP_POLICY_DIR),
cid: None,
gpu: false,
@ -318,6 +320,32 @@ fn set_argument(cfg: &mut Config, name: &str, value: Option<&str>) -> argument::
}
})?);
}
"shared-dir" => {
// Formatted as <src:tag>.
let param = value.unwrap();
let mut components = param.splitn(2, ':');
let src = PathBuf::from(components.next().ok_or_else(|| {
argument::Error::InvalidValue {
value: param.to_owned(),
expected: "missing source path for `shared-dir`",
}
})?);
let tag = components.next().ok_or_else(|| {
argument::Error::InvalidValue {
value: param.to_owned(),
expected: "missing tag for `shared-dir`",
}
})?.to_owned();
if !src.is_dir() {
return Err(argument::Error::InvalidValue {
value: param.to_owned(),
expected: "source path for `shared-dir` must be a directory",
});
}
cfg.shared_dirs.push((src, tag));
}
"seccomp-policy-dir" => {
// `value` is Some because we are in this match so it's safe to unwrap.
cfg.seccomp_policy_dir = PathBuf::from(value.unwrap());
@ -401,7 +429,9 @@ fn run_vm(args: std::env::Args) -> std::result::Result<(), ()> {
"Path to put the control socket. If PATH is a directory, a name will be generated."),
Argument::short_flag('u', "multiprocess", "Run each device in a child process(default)."),
Argument::flag("disable-sandbox", "Run all devices in one, non-sandboxed process."),
Argument::value("cid", "CID", "Context ID for virtual sockets"),
Argument::value("cid", "CID", "Context ID for virtual sockets."),
Argument::value("shared-dir", "PATH:TAG",
"Directory to be shared with a VM as a source:tag pair. Can be given more than once."),
Argument::value("seccomp-policy-dir", "PATH", "Path to seccomp .policy files."),
#[cfg(feature = "plugin")]
Argument::value("plugin", "PATH", "Absolute path to plugin process to run under crosvm."),