summary refs log tree commit diff
diff options
context:
space:
mode:
authorJakub Staron <jstaron@google.com>2019-04-24 10:55:25 -0700
committerCommit Bot <commit-bot@chromium.org>2019-06-05 07:28:54 +0000
commita3411eaac1ad3d51d5282d1462d6764ad539ce13 (patch)
tree7a9898280f1cd8c78dcdfac6893189ed42419d76
parent127453d7eccdb6a903d0855fabb8f0935be90882 (diff)
downloadcrosvm-a3411eaac1ad3d51d5282d1462d6764ad539ce13.tar
crosvm-a3411eaac1ad3d51d5282d1462d6764ad539ce13.tar.gz
crosvm-a3411eaac1ad3d51d5282d1462d6764ad539ce13.tar.bz2
crosvm-a3411eaac1ad3d51d5282d1462d6764ad539ce13.tar.lz
crosvm-a3411eaac1ad3d51d5282d1462d6764ad539ce13.tar.xz
crosvm-a3411eaac1ad3d51d5282d1462d6764ad539ce13.tar.zst
crosvm-a3411eaac1ad3d51d5282d1462d6764ad539ce13.zip
crosvm: virtio-pmem device
Adds support for virtio-pmem device as an alternative for virtio-blk.

Exposing disk image to guest as virtio-blk device results in both guest
and host independently caching the disk I/O. Using virtio-pmem device
allows to mount disk image as direct access (DAX) in the guest and thus
bypass the guest cache. This will reduce memory foodprint of the VMs.

BUG=None
TEST=cargo test
TEST=Boot patched termina kernel in crosvm; mount virtio-pmem device as
DAX and run xfstests.

Change-Id: I935fc8fc7527f79e5169f07ec7927e4ea4fa6027
Reviewed-on: https://chromium-review.googlesource.com/c/chromiumos/platform/crosvm/+/1605517
Tested-by: kokoro <noreply+kokoro@google.com>
Reviewed-by: Zach Reizner <zachr@chromium.org>
Commit-Queue: Jakub StaroĊ„ <jstaron@google.com>
-rw-r--r--aarch64/src/lib.rs6
-rw-r--r--arch/src/lib.rs7
-rw-r--r--devices/src/virtio/mod.rs3
-rw-r--r--devices/src/virtio/pmem.rs367
-rw-r--r--resources/src/address_allocator.rs75
-rw-r--r--resources/src/lib.rs2
-rw-r--r--src/linux.rs106
-rw-r--r--src/main.rs18
-rw-r--r--sys_util/src/mmap.rs6
-rw-r--r--x86_64/src/lib.rs6
10 files changed, 576 insertions, 20 deletions
diff --git a/aarch64/src/lib.rs b/aarch64/src/lib.rs
index 4d226c0..d21d70d 100644
--- a/aarch64/src/lib.rs
+++ b/aarch64/src/lib.rs
@@ -200,6 +200,8 @@ impl arch::LinuxArch for AArch64 {
     where
         F: FnOnce(
             &GuestMemory,
+            &mut Vm,
+            &mut SystemAllocator,
             &EventFd,
         ) -> std::result::Result<Vec<(Box<dyn PciDevice>, Option<Minijail>)>, E>,
         E: StdError + 'static,
@@ -233,8 +235,8 @@ impl arch::LinuxArch for AArch64 {
 
         let exit_evt = EventFd::new().map_err(Error::CreateEventFd)?;
 
-        let pci_devices =
-            create_devices(&mem, &exit_evt).map_err(|e| Error::CreateDevices(Box::new(e)))?;
+        let pci_devices = create_devices(&mem, &mut vm, &mut resources, &exit_evt)
+            .map_err(|e| Error::CreateDevices(Box::new(e)))?;
         let (pci, pci_irqs, pid_debug_label_map) =
             arch::generate_pci_root(pci_devices, &mut mmio_bus, &mut resources, &mut vm)
                 .map_err(Error::CreatePciRoot)?;
diff --git a/arch/src/lib.rs b/arch/src/lib.rs
index 91e7ee4..71e4327 100644
--- a/arch/src/lib.rs
+++ b/arch/src/lib.rs
@@ -83,7 +83,12 @@ pub trait LinuxArch {
         create_devices: F,
     ) -> Result<RunnableLinuxVm, Self::Error>
     where
-        F: FnOnce(&GuestMemory, &EventFd) -> Result<Vec<(Box<dyn PciDevice>, Option<Minijail>)>, E>,
+        F: FnOnce(
+            &GuestMemory,
+            &mut Vm,
+            &mut SystemAllocator,
+            &EventFd,
+        ) -> Result<Vec<(Box<dyn PciDevice>, Option<Minijail>)>, E>,
         E: StdError + 'static;
 }
 
diff --git a/devices/src/virtio/mod.rs b/devices/src/virtio/mod.rs
index a9fe525..3531ccd 100644
--- a/devices/src/virtio/mod.rs
+++ b/devices/src/virtio/mod.rs
@@ -11,6 +11,7 @@ mod gpu;
 mod input;
 mod net;
 mod p9;
+mod pmem;
 mod queue;
 mod rng;
 #[cfg(feature = "tpm")]
@@ -30,6 +31,7 @@ pub use self::gpu::*;
 pub use self::input::*;
 pub use self::net::*;
 pub use self::p9::*;
+pub use self::pmem::*;
 pub use self::queue::*;
 pub use self::rng::*;
 #[cfg(feature = "tpm")]
@@ -54,6 +56,7 @@ const TYPE_GPU: u32 = 16;
 const TYPE_9P: u32 = 9;
 const TYPE_INPUT: u32 = 18;
 const TYPE_VSOCK: u32 = 19;
+const TYPE_PMEM: u32 = 27;
 // Additional types invented by crosvm
 const TYPE_WL: u32 = 30;
 #[cfg(feature = "tpm")]
diff --git a/devices/src/virtio/pmem.rs b/devices/src/virtio/pmem.rs
new file mode 100644
index 0000000..5a4fa32
--- /dev/null
+++ b/devices/src/virtio/pmem.rs
@@ -0,0 +1,367 @@
+// Copyright 2019 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;
+use std::fmt::{self, Display};
+use std::fs::File;
+use std::io::Write;
+use std::mem::{size_of, size_of_val};
+use std::os::unix::io::{AsRawFd, RawFd};
+use std::result;
+use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::Arc;
+use std::thread;
+
+use sys_util::Result as SysResult;
+use sys_util::{
+    error, EventFd, GuestAddress, GuestMemory, GuestMemoryError, PollContext, PollToken,
+};
+
+use data_model::{DataInit, Le32, Le64};
+
+use super::{
+    DescriptorChain, Queue, VirtioDevice, INTERRUPT_STATUS_USED_RING, TYPE_PMEM, VIRTIO_F_VERSION_1,
+};
+
+const QUEUE_SIZE: u16 = 256;
+const QUEUE_SIZES: &[u16] = &[QUEUE_SIZE];
+
+const VIRTIO_PMEM_REQ_TYPE_FLUSH: u32 = 0;
+const VIRTIO_PMEM_RESP_TYPE_OK: u32 = 0;
+const VIRTIO_PMEM_RESP_TYPE_EIO: u32 = 1;
+
+#[derive(Copy, Clone, Debug, Default)]
+#[repr(C)]
+struct virtio_pmem_config {
+    start_address: Le64,
+    size: Le64,
+}
+
+// Safe because it only has data and has no implicit padding.
+unsafe impl DataInit for virtio_pmem_config {}
+
+#[derive(Copy, Clone, Debug, Default)]
+#[repr(C)]
+struct virtio_pmem_resp {
+    status_code: Le32,
+}
+
+// Safe because it only has data and has no implicit padding.
+unsafe impl DataInit for virtio_pmem_resp {}
+
+#[derive(Copy, Clone, Debug, Default)]
+#[repr(C)]
+struct virtio_pmem_req {
+    type_: Le32,
+}
+
+// Safe because it only has data and has no implicit padding.
+unsafe impl DataInit for virtio_pmem_req {}
+
+#[derive(Debug)]
+enum ParseError {
+    /// Guest gave us bad memory addresses.
+    GuestMemory(GuestMemoryError),
+    /// Guest gave us a write only descriptor that protocol says to read from.
+    UnexpectedWriteOnlyDescriptor,
+    /// Guest gave us a read only descriptor that protocol says to write to.
+    UnexpectedReadOnlyDescriptor,
+    /// Guest gave us too few descriptors in a descriptor chain.
+    DescriptorChainTooShort,
+    /// Guest gave us a buffer that was too short to use.
+    BufferLengthTooSmall,
+    /// Guest sent us invalid request.
+    InvalidRequest,
+}
+
+impl Display for ParseError {
+    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
+        use self::ParseError::*;
+
+        match self {
+            BufferLengthTooSmall => write!(f, "buffer length too small"),
+            DescriptorChainTooShort => write!(f, "descriptor chain too short"),
+            GuestMemory(e) => write!(f, "bad guest memory address: {}", e),
+            InvalidRequest => write!(f, "invalid request"),
+            UnexpectedReadOnlyDescriptor => write!(f, "unexpected read-only descriptor"),
+            UnexpectedWriteOnlyDescriptor => write!(f, "unexpected write-only descriptor"),
+        }
+    }
+}
+
+enum Request {
+    Flush { status_address: GuestAddress },
+}
+
+impl Request {
+    fn parse(
+        avail_desc: &DescriptorChain,
+        memory: &GuestMemory,
+    ) -> result::Result<Request, ParseError> {
+        // The head contains the request type which MUST be readable.
+        if avail_desc.is_write_only() {
+            return Err(ParseError::UnexpectedWriteOnlyDescriptor);
+        }
+
+        if avail_desc.len as usize != size_of::<virtio_pmem_req>() {
+            return Err(ParseError::InvalidRequest);
+        }
+
+        let request: virtio_pmem_req = memory
+            .read_obj_from_addr(avail_desc.addr)
+            .map_err(ParseError::GuestMemory)?;
+
+        // Currently, there is only one virtio-pmem request, FLUSH.
+        if request.type_ != VIRTIO_PMEM_REQ_TYPE_FLUSH {
+            error!("unknown request type: {}", request.type_.to_native());
+            return Err(ParseError::InvalidRequest);
+        }
+
+        let status_desc = avail_desc
+            .next_descriptor()
+            .ok_or(ParseError::DescriptorChainTooShort)?;
+
+        // The status MUST always be writable
+        if status_desc.is_read_only() {
+            return Err(ParseError::UnexpectedReadOnlyDescriptor);
+        }
+
+        if (status_desc.len as usize) < size_of::<virtio_pmem_resp>() {
+            return Err(ParseError::BufferLengthTooSmall);
+        }
+
+        Ok(Request::Flush {
+            status_address: status_desc.addr,
+        })
+    }
+}
+
+struct Worker {
+    queue: Queue,
+    memory: GuestMemory,
+    disk_image: File,
+    interrupt_status: Arc<AtomicUsize>,
+    interrupt_event: EventFd,
+    interrupt_resample_event: EventFd,
+}
+
+impl Worker {
+    fn process_queue(&mut self) -> bool {
+        let mut needs_interrupt = false;
+        while let Some(avail_desc) = self.queue.pop(&self.memory) {
+            let len;
+            match Request::parse(&avail_desc, &self.memory) {
+                Ok(Request::Flush { status_address }) => {
+                    let status_code = match self.disk_image.sync_all() {
+                        Ok(()) => VIRTIO_PMEM_RESP_TYPE_OK,
+                        Err(e) => {
+                            error!("failed flushing disk image: {}", e);
+                            VIRTIO_PMEM_RESP_TYPE_EIO
+                        }
+                    };
+
+                    let response = virtio_pmem_resp {
+                        status_code: status_code.into(),
+                    };
+                    len = match self.memory.write_obj_at_addr(response, status_address) {
+                        Ok(_) => size_of::<virtio_pmem_resp>() as u32,
+                        Err(e) => {
+                            error!("bad guest memory address: {}", e);
+                            0
+                        }
+                    }
+                }
+                Err(e) => {
+                    error!("failed processing available descriptor chain: {}", e);
+                    len = 0;
+                }
+            }
+            self.queue.add_used(&self.memory, avail_desc.index, len);
+            needs_interrupt = true;
+        }
+
+        needs_interrupt
+    }
+
+    fn signal_used_queue(&self) {
+        self.interrupt_status
+            .fetch_or(INTERRUPT_STATUS_USED_RING as usize, Ordering::SeqCst);
+        self.interrupt_event.write(1).unwrap();
+    }
+
+    fn run(&mut self, queue_evt: EventFd, kill_evt: EventFd) {
+        #[derive(PollToken)]
+        enum Token {
+            QueueAvailable,
+            InterruptResample,
+            Kill,
+        }
+
+        let poll_ctx: PollContext<Token> = match PollContext::new()
+            .and_then(|pc| pc.add(&queue_evt, Token::QueueAvailable).and(Ok(pc)))
+            .and_then(|pc| {
+                pc.add(&self.interrupt_resample_event, Token::InterruptResample)
+                    .and(Ok(pc))
+            })
+            .and_then(|pc| pc.add(&kill_evt, Token::Kill).and(Ok(pc)))
+        {
+            Ok(pc) => pc,
+            Err(e) => {
+                error!("failed creating PollContext: {}", e);
+                return;
+            }
+        };
+
+        'poll: loop {
+            let events = match poll_ctx.wait() {
+                Ok(v) => v,
+                Err(e) => {
+                    error!("failed polling for events: {}", e);
+                    break;
+                }
+            };
+
+            let mut needs_interrupt = false;
+            for event in events.iter_readable() {
+                match event.token() {
+                    Token::QueueAvailable => {
+                        if let Err(e) = queue_evt.read() {
+                            error!("failed reading queue EventFd: {}", e);
+                            break 'poll;
+                        }
+                        needs_interrupt |= self.process_queue();
+                    }
+                    Token::InterruptResample => {
+                        let _ = self.interrupt_resample_event.read();
+                        if self.interrupt_status.load(Ordering::SeqCst) != 0 {
+                            self.interrupt_event.write(1).unwrap();
+                        }
+                    }
+                    Token::Kill => break 'poll,
+                }
+            }
+            if needs_interrupt {
+                self.signal_used_queue();
+            }
+        }
+    }
+}
+
+pub struct Pmem {
+    kill_event: Option<EventFd>,
+    disk_image: Option<File>,
+    mapping_address: GuestAddress,
+    mapping_size: u64,
+}
+
+impl Pmem {
+    pub fn new(
+        disk_image: File,
+        mapping_address: GuestAddress,
+        mapping_size: u64,
+    ) -> SysResult<Pmem> {
+        Ok(Pmem {
+            kill_event: None,
+            disk_image: Some(disk_image),
+            mapping_address,
+            mapping_size,
+        })
+    }
+}
+
+impl Drop for Pmem {
+    fn drop(&mut self) {
+        if let Some(kill_evt) = self.kill_event.take() {
+            // Ignore the result because there is nothing we can do about it.
+            let _ = kill_evt.write(1);
+        }
+    }
+}
+
+impl VirtioDevice for Pmem {
+    fn keep_fds(&self) -> Vec<RawFd> {
+        if let Some(disk_image) = &self.disk_image {
+            vec![disk_image.as_raw_fd()]
+        } else {
+            vec![]
+        }
+    }
+
+    fn device_type(&self) -> u32 {
+        TYPE_PMEM
+    }
+
+    fn queue_max_sizes(&self) -> &[u16] {
+        QUEUE_SIZES
+    }
+
+    fn features(&self) -> u64 {
+        1 << VIRTIO_F_VERSION_1
+    }
+
+    fn read_config(&self, offset: u64, mut data: &mut [u8]) {
+        let config = virtio_pmem_config {
+            start_address: Le64::from(self.mapping_address.offset()),
+            size: Le64::from(self.mapping_size as u64),
+        };
+        let config_len = size_of_val(&config) as u64;
+        if offset >= config_len {
+            return;
+        }
+
+        if let Some(end) = offset.checked_add(data.len() as u64) {
+            let offset = offset as usize;
+            let end = cmp::min(end, config_len) as usize;
+            // This write can't fail, offset and end are checked against config_len.
+            data.write_all(&config.as_slice()[offset..end]).unwrap();
+        }
+    }
+
+    fn activate(
+        &mut self,
+        memory: GuestMemory,
+        interrupt_event: EventFd,
+        interrupt_resample_event: EventFd,
+        status: Arc<AtomicUsize>,
+        mut queues: Vec<Queue>,
+        mut queue_events: Vec<EventFd>,
+    ) {
+        if queues.len() != 1 || queue_events.len() != 1 {
+            return;
+        }
+
+        let queue = queues.remove(0);
+        let queue_event = queue_events.remove(0);
+
+        if let Some(disk_image) = self.disk_image.take() {
+            let (self_kill_event, kill_event) =
+                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_event = Some(self_kill_event);
+
+            let worker_result = thread::Builder::new()
+                .name("virtio_pmem".to_string())
+                .spawn(move || {
+                    let mut worker = Worker {
+                        memory,
+                        disk_image,
+                        queue,
+                        interrupt_status: status,
+                        interrupt_event,
+                        interrupt_resample_event,
+                    };
+                    worker.run(queue_event, kill_event);
+                });
+            if let Err(e) = worker_result {
+                error!("failed to spawn virtio_pmem worker: {}", e);
+                return;
+            }
+        }
+    }
+}
diff --git a/resources/src/address_allocator.rs b/resources/src/address_allocator.rs
index 45927e0..11978ee 100644
--- a/resources/src/address_allocator.rs
+++ b/resources/src/address_allocator.rs
@@ -2,6 +2,7 @@
 // Use of this source code is governed by a BSD-style license that can be
 // found in the LICENSE file.
 
+use std::cmp;
 use std::collections::HashMap;
 
 use crate::{Alloc, Error, Result};
@@ -63,18 +64,29 @@ impl AddressAllocator {
         })
     }
 
-    /// Allocates a range of addresses from the managed region with an optional tag.
-    /// Returns allocated_address. (allocated_address, size, tag) can be retrieved
-    /// through the `get` method.
-    pub fn allocate(&mut self, size: u64, alloc: Alloc, tag: String) -> Result<u64> {
+    /// Allocates a range of addresses from the managed region with an optional tag
+    /// and minimal alignment. Returns allocated_address. (allocated_address, size, tag)
+    /// can be retrieved through the `get` method.
+    pub fn allocate_with_align(
+        &mut self,
+        size: u64,
+        alloc: Alloc,
+        tag: String,
+        alignment: u64,
+    ) -> Result<u64> {
+        let alignment = cmp::max(self.alignment, alignment);
+
         if self.allocs.contains_key(&alloc) {
             return Err(Error::ExistingAlloc(alloc));
         }
         if size == 0 {
             return Err(Error::AllocSizeZero);
         }
-        let align_adjust = if self.next_addr % self.alignment != 0 {
-            self.alignment - (self.next_addr % self.alignment)
+        if !alignment.is_power_of_two() {
+            return Err(Error::BadAlignment);
+        }
+        let align_adjust = if self.next_addr % alignment != 0 {
+            alignment - (self.next_addr % alignment)
         } else {
             0
         };
@@ -95,6 +107,10 @@ impl AddressAllocator {
         Ok(addr)
     }
 
+    pub fn allocate(&mut self, size: u64, alloc: Alloc, tag: String) -> Result<u64> {
+        self.allocate_with_align(size, alloc, tag, self.alignment)
+    }
+
     /// Returns allocation associated with `alloc`, or None if no such allocation exists.
     pub fn get(&self, alloc: &Alloc) -> Option<&(u64, u64, String)> {
         self.allocs.get(alloc)
@@ -180,4 +196,51 @@ mod tests {
             Some(&(0x1000, 0x110, String::from("bar0")))
         );
     }
+
+    #[test]
+    fn allocate_with_alignment_allocator_alignment() {
+        let mut pool = AddressAllocator::new(0x1000, 0x10000, Some(0x100)).unwrap();
+        assert_eq!(
+            pool.allocate_with_align(0x110, Alloc::Anon(0), String::from("bar0"), 0x1),
+            Ok(0x1000)
+        );
+        assert_eq!(
+            pool.allocate_with_align(0x100, Alloc::Anon(1), String::from("bar1"), 0x1),
+            Ok(0x1200)
+        );
+    }
+
+    #[test]
+    fn allocate_with_alignment_custom_alignment() {
+        let mut pool = AddressAllocator::new(0x1000, 0x10000, Some(0x4)).unwrap();
+        assert_eq!(
+            pool.allocate_with_align(0x110, Alloc::Anon(0), String::from("bar0"), 0x100),
+            Ok(0x1000)
+        );
+        assert_eq!(
+            pool.allocate_with_align(0x100, Alloc::Anon(1), String::from("bar1"), 0x100),
+            Ok(0x1200)
+        );
+    }
+
+    #[test]
+    fn allocate_with_alignment_no_allocator_alignment() {
+        let mut pool = AddressAllocator::new(0x1000, 0x10000, None).unwrap();
+        assert_eq!(
+            pool.allocate_with_align(0x110, Alloc::Anon(0), String::from("bar0"), 0x100),
+            Ok(0x1000)
+        );
+        assert_eq!(
+            pool.allocate_with_align(0x100, Alloc::Anon(1), String::from("bar1"), 0x100),
+            Ok(0x1200)
+        );
+    }
+
+    #[test]
+    fn allocate_with_alignment_alignment_non_power_of_two() {
+        let mut pool = AddressAllocator::new(0x1000, 0x10000, None).unwrap();
+        assert!(pool
+            .allocate_with_align(0x110, Alloc::Anon(0), String::from("bar0"), 200)
+            .is_err());
+    }
 }
diff --git a/resources/src/lib.rs b/resources/src/lib.rs
index a49f998..c10608e 100644
--- a/resources/src/lib.rs
+++ b/resources/src/lib.rs
@@ -33,6 +33,8 @@ pub enum Alloc {
     PciBar { bus: u8, dev: u8, bar: u8 },
     /// GPU render node region.
     GpuRenderNode,
+    /// Pmem device region with associated device index.
+    PmemDevice(usize),
 }
 
 #[derive(Debug, Eq, PartialEq)]
diff --git a/src/linux.rs b/src/linux.rs
index 9f158ed..84bd699 100644
--- a/src/linux.rs
+++ b/src/linux.rs
@@ -4,6 +4,7 @@
 
 use std;
 use std::cmp::min;
+use std::convert::TryFrom;
 use std::error::Error as StdError;
 use std::ffi::CStr;
 use std::fmt::{self, Display};
@@ -32,18 +33,16 @@ use net_util::{Error as NetError, MacAddress, Tap};
 use qcow::{self, ImageType, QcowFile};
 use rand_ish::SimpleRng;
 use remain::sorted;
-#[cfg(feature = "gpu-forward")]
-use resources::Alloc;
+use resources::{Alloc, SystemAllocator};
 use sync::{Condvar, Mutex};
 use sys_util::net::{UnixSeqpacket, UnixSeqpacketListener, UnlinkUnixSeqpacketListener};
+
 use sys_util::{
     self, block_signal, clear_signal, drop_capabilities, error, flock, get_blocked_signals,
     get_group_id, get_user_id, getegid, geteuid, info, register_signal_handler, set_cpu_affinity,
-    validate_raw_fd, warn, EventFd, FlockOperation, GuestMemory, Killable, PollContext, PollToken,
-    SignalFd, Terminal, TimerFd, SIGRTMIN,
+    validate_raw_fd, warn, EventFd, FlockOperation, GuestAddress, GuestMemory, Killable,
+    MemoryMapping, PollContext, PollToken, Protection, SignalFd, Terminal, TimerFd, SIGRTMIN,
 };
-#[cfg(feature = "gpu-forward")]
-use sys_util::{GuestAddress, MemoryMapping, Protection};
 use vhost;
 use vm_control::{
     BalloonControlCommand, BalloonControlRequestSocket, BalloonControlResponseSocket,
@@ -70,7 +69,9 @@ type RenderNodeHost = ();
 #[derive(Debug)]
 pub enum Error {
     AddGpuDeviceMemory(sys_util::Error),
+    AddPmemDeviceMemory(sys_util::Error),
     AllocateGpuDeviceAddress,
+    AllocatePmemDeviceAddress(resources::Error),
     BalloonDeviceNew(virtio::BalloonError),
     BlockDeviceNew(sys_util::Error),
     BlockSignal(sys_util::signal::Error),
@@ -106,6 +107,8 @@ pub enum Error {
     OpenVinput(PathBuf, io::Error),
     P9DeviceNew(virtio::P9Error),
     PivotRootDoesntExist(&'static str),
+    PmemDeviceImageTooBig,
+    PmemDeviceNew(sys_util::Error),
     PollContextAdd(sys_util::Error),
     PollContextDelete(sys_util::Error),
     QcowDeviceCreate(qcow::Error),
@@ -121,6 +124,7 @@ pub enum Error {
     RegisterWayland(arch::DeviceRegistrationError),
     ReserveGpuMemory(sys_util::MmapError),
     ReserveMemory(sys_util::Error),
+    ReservePmemMemory(sys_util::MmapError),
     ResetTimerFd(sys_util::Error),
     RngDeviceNew(virtio::RngError),
     SettingGidMap(io_jail::Error),
@@ -143,7 +147,11 @@ impl Display for Error {
         #[sorted]
         match self {
             AddGpuDeviceMemory(e) => write!(f, "failed to add gpu device memory: {}", e),
+            AddPmemDeviceMemory(e) => write!(f, "failed to add pmem device memory: {}", e),
             AllocateGpuDeviceAddress => write!(f, "failed to allocate gpu device guest address"),
+            AllocatePmemDeviceAddress(e) => {
+                write!(f, "failed to allocate memory for pmem device: {}", e)
+            }
             BalloonDeviceNew(e) => write!(f, "failed to create balloon: {}", e),
             BlockDeviceNew(e) => write!(f, "failed to create block device: {}", e),
             BlockSignal(e) => write!(f, "failed to block signal: {}", e),
@@ -186,6 +194,10 @@ impl Display for Error {
             OpenVinput(p, e) => write!(f, "failed to open vinput device {}: {}", p.display(), e),
             P9DeviceNew(e) => write!(f, "failed to create 9p device: {}", e),
             PivotRootDoesntExist(p) => write!(f, "{} doesn't exist, can't jail devices.", p),
+            PmemDeviceImageTooBig => {
+                write!(f, "failed to create pmem device: pmem device image too big")
+            }
+            PmemDeviceNew(e) => write!(f, "failed to create pmem device: {}", e),
             PollContextAdd(e) => write!(f, "failed to add fd to poll context: {}", e),
             PollContextDelete(e) => write!(f, "failed to remove fd from poll context: {}", e),
             QcowDeviceCreate(e) => write!(f, "failed to read qcow formatted file {}", e),
@@ -209,6 +221,7 @@ impl Display for Error {
             RegisterWayland(e) => write!(f, "error registering wayland device: {}", e),
             ReserveGpuMemory(e) => write!(f, "failed to reserve gpu memory: {}", e),
             ReserveMemory(e) => write!(f, "failed to reserve memory: {}", e),
+            ReservePmemMemory(e) => write!(f, "failed to reserve pmem memory: {}", e),
             ResetTimerFd(e) => write!(f, "failed to reset timerfd: {}", e),
             RngDeviceNew(e) => write!(f, "failed to set up rng: {}", e),
             SettingGidMap(e) => write!(f, "error setting GID map: {}", e),
@@ -692,11 +705,76 @@ fn create_9p_device(cfg: &Config, chronos: Ids, src: &Path, tag: &str) -> Device
     })
 }
 
+fn create_pmem_device(
+    cfg: &Config,
+    vm: &mut Vm,
+    resources: &mut SystemAllocator,
+    disk: &DiskOption,
+    index: usize,
+) -> DeviceResult {
+    let fd = OpenOptions::new()
+        .read(true)
+        .write(!disk.read_only)
+        .open(&disk.path)
+        .map_err(Error::Disk)?;
+
+    let image_size = {
+        let metadata = std::fs::metadata(&disk.path).map_err(Error::Disk)?;
+        metadata.len()
+    };
+
+    let protection = {
+        if disk.read_only {
+            Protection::read()
+        } else {
+            Protection::read_write()
+        }
+    };
+
+    let memory_mapping = {
+        // Conversion from u64 to usize may fail on 32bit system.
+        let image_size = usize::try_from(image_size).map_err(|_| Error::PmemDeviceImageTooBig)?;
+
+        MemoryMapping::from_fd_offset_protection(&fd, image_size, 0, protection)
+            .map_err(Error::ReservePmemMemory)?
+    };
+
+    let mapping_address = resources
+        .device_allocator()
+        .allocate_with_align(
+            image_size,
+            Alloc::PmemDevice(index),
+            format!("pmem_disk_image_{}", index),
+            // Linux kernel requires pmem namespaces to be 128 MiB aligned.
+            128 * 1024 * 1024, /* 128 MiB */
+        )
+        .map_err(Error::AllocatePmemDeviceAddress)?;
+
+    vm.add_device_memory(
+        GuestAddress(mapping_address),
+        memory_mapping,
+        /* read_only = */ disk.read_only,
+        /* log_dirty_pages = */ false,
+    )
+    .map_err(Error::AddPmemDeviceMemory)?;
+
+    let dev = virtio::Pmem::new(fd, GuestAddress(mapping_address), image_size)
+        .map_err(Error::PmemDeviceNew)?;
+
+    Ok(VirtioDeviceStub {
+        dev: Box::new(dev) as Box<dyn VirtioDevice>,
+        /// TODO(jstaron) Create separate device policy for pmem_device.
+        jail: simple_jail(&cfg, "block_device.policy")?,
+    })
+}
+
 // gpu_device_socket is not used when GPU support is disabled.
 #[cfg_attr(not(feature = "gpu"), allow(unused_variables))]
 fn create_virtio_devices(
     cfg: &Config,
     mem: &GuestMemory,
+    vm: &mut Vm,
+    resources: &mut SystemAllocator,
     _exit_evt: &EventFd,
     wayland_device_socket: VmMemoryControlRequestSocket,
     gpu_device_socket: VmMemoryControlRequestSocket,
@@ -710,6 +788,10 @@ fn create_virtio_devices(
         devs.push(create_block_device(cfg, disk, disk_device_socket)?);
     }
 
+    for (index, pmem_disk) in cfg.pmem_devices.iter().enumerate() {
+        devs.push(create_pmem_device(cfg, vm, resources, pmem_disk, index)?);
+    }
+
     devs.push(create_rng_device(cfg)?);
 
     #[cfg(feature = "tpm")]
@@ -799,6 +881,8 @@ fn create_virtio_devices(
 fn create_devices(
     cfg: &Config,
     mem: &GuestMemory,
+    vm: &mut Vm,
+    resources: &mut SystemAllocator,
     exit_evt: &EventFd,
     wayland_device_socket: VmMemoryControlRequestSocket,
     gpu_device_socket: VmMemoryControlRequestSocket,
@@ -809,6 +893,8 @@ fn create_devices(
     let stubs = create_virtio_devices(
         &cfg,
         mem,
+        vm,
+        resources,
         exit_evt,
         wayland_device_socket,
         gpu_device_socket,
@@ -1210,11 +1296,13 @@ pub fn run_config(cfg: Config) -> Result<()> {
         components,
         cfg.split_irqchip,
         &cfg.serial_parameters,
-        |m, e| {
+        |mem, vm, sys_allocator, exit_evt| {
             create_devices(
                 &cfg,
-                m,
-                e,
+                mem,
+                vm,
+                sys_allocator,
+                exit_evt,
                 wayland_device_socket,
                 gpu_device_socket,
                 balloon_device_socket,
diff --git a/src/main.rs b/src/main.rs
index 530d119..d57174b 100644
--- a/src/main.rs
+++ b/src/main.rs
@@ -102,6 +102,7 @@ pub struct Config {
     plugin_mounts: Vec<BindMount>,
     plugin_gid_maps: Vec<GidMap>,
     disks: Vec<DiskOption>,
+    pmem_devices: Vec<DiskOption>,
     host_ip: Option<net::Ipv4Addr>,
     netmask: Option<net::Ipv4Addr>,
     mac_address: Option<net_util::MacAddress>,
@@ -143,6 +144,7 @@ impl Default for Config {
             plugin_mounts: Vec::new(),
             plugin_gid_maps: Vec::new(),
             disks: Vec::new(),
+            pmem_devices: Vec::new(),
             host_ip: None,
             netmask: None,
             mac_address: None,
@@ -440,6 +442,20 @@ fn set_argument(cfg: &mut Config, name: &str, value: Option<&str>) -> argument::
                 read_only: !name.starts_with("rw"),
             });
         }
+        "pmem-device" | "rw-pmem-device" => {
+            let disk_path = PathBuf::from(value.unwrap());
+            if !disk_path.exists() {
+                return Err(argument::Error::InvalidValue {
+                    value: value.unwrap().to_owned(),
+                    expected: "this disk path does not exist",
+                });
+            }
+
+            cfg.pmem_devices.push(DiskOption {
+                path: disk_path,
+                read_only: !name.starts_with("rw"),
+            });
+        }
         "host_ip" => {
             if cfg.host_ip.is_some() {
                 return Err(argument::Error::TooManyArguments(
@@ -805,6 +821,8 @@ fn run_vm(args: std::env::Args) -> std::result::Result<(), ()> {
           Argument::value("qcow", "PATH", "Path to a qcow2 disk image. (Deprecated; use --disk instead.)"),
           Argument::value("rwdisk", "PATH", "Path to a writable disk image."),
           Argument::value("rwqcow", "PATH", "Path to a writable qcow2 disk image. (Deprecated; use --rwdisk instead.)"),
+          Argument::value("rw-pmem-device", "PATH", "Path to a writable disk image."),
+          Argument::value("pmem-device", "PATH", "Path to a disk image."),
           Argument::value("host_ip",
                           "IP",
                           "IP address to assign to host tap interface."),
diff --git a/sys_util/src/mmap.rs b/sys_util/src/mmap.rs
index 9f573e4..3d1e577 100644
--- a/sys_util/src/mmap.rs
+++ b/sys_util/src/mmap.rs
@@ -82,6 +82,12 @@ impl Protection {
         Protection(libc::PROT_READ | libc::PROT_WRITE)
     }
 
+    /// Returns Protection allowing read access.
+    #[inline(always)]
+    pub fn read() -> Protection {
+        Protection(libc::PROT_READ)
+    }
+
     /// Set read events.
     #[inline(always)]
     pub fn set_read(self) -> Protection {
diff --git a/x86_64/src/lib.rs b/x86_64/src/lib.rs
index 3fe8d69..a047b8c 100644
--- a/x86_64/src/lib.rs
+++ b/x86_64/src/lib.rs
@@ -309,6 +309,8 @@ impl arch::LinuxArch for X8664arch {
     where
         F: FnOnce(
             &GuestMemory,
+            &mut Vm,
+            &mut SystemAllocator,
             &EventFd,
         ) -> std::result::Result<Vec<(Box<dyn PciDevice>, Option<Minijail>)>, E>,
         E: StdError + 'static,
@@ -348,8 +350,8 @@ impl arch::LinuxArch for X8664arch {
 
         let exit_evt = EventFd::new().map_err(Error::CreateEventFd)?;
 
-        let pci_devices =
-            create_devices(&mem, &exit_evt).map_err(|e| Error::CreateDevices(Box::new(e)))?;
+        let pci_devices = create_devices(&mem, &mut vm, &mut resources, &exit_evt)
+            .map_err(|e| Error::CreateDevices(Box::new(e)))?;
         let (pci, pci_irqs, pid_debug_label_map) =
             arch::generate_pci_root(pci_devices, &mut mmio_bus, &mut resources, &mut vm)
                 .map_err(Error::CreatePciRoot)?;