summary refs log tree commit diff
path: root/devices/src/virtio/fs
diff options
context:
space:
mode:
authorChirantan Ekbote <chirantan@chromium.org>2019-07-17 10:30:28 +0900
committerCommit Bot <commit-bot@chromium.org>2019-11-06 18:31:07 +0000
commit152117f3d383cd562ea6df33cb4126746a499f81 (patch)
tree1acf8ce95b987073b0ee7d3c9620bb8d7a8cb1b8 /devices/src/virtio/fs
parent7b4822aa382da58b6a797fa617bb833a8edd6abc (diff)
downloadcrosvm-152117f3d383cd562ea6df33cb4126746a499f81.tar
crosvm-152117f3d383cd562ea6df33cb4126746a499f81.tar.gz
crosvm-152117f3d383cd562ea6df33cb4126746a499f81.tar.bz2
crosvm-152117f3d383cd562ea6df33cb4126746a499f81.tar.lz
crosvm-152117f3d383cd562ea6df33cb4126746a499f81.tar.xz
crosvm-152117f3d383cd562ea6df33cb4126746a499f81.tar.zst
crosvm-152117f3d383cd562ea6df33cb4126746a499f81.zip
devices: Add virtio-fs device
Add a new virtio-fs device for sharing files between the host and the
guest.  This change adds all the device infrastructure necessary for the
driver probe to succeed but doesn't currently handle the actual fuse
protocol.  Additionally, shared memory support is not currently
implemented.  The device is not hooked up to the command line.

Testing this device requires a kernel with the virtio-fs patches.  To
test with a standard crostini setup, use
https://user.git.corp.google.com/chirantan/virtiofs/+/refs/heads/chromeos-5.1
which is the 5.1 kernel with the virtio-fs and chromium-specific
virtio-{gpu,wl} patches applied.

BUG=b:136128319
TEST=`tast run vm.VirtioFs`

Change-Id: I09dcefafaf0d2a7e13d54df11384dfcee3b85ba6
Reviewed-on: https://chromium-review.googlesource.com/c/chromiumos/platform/crosvm/+/1705654
Tested-by: kokoro <noreply+kokoro@google.com>
Commit-Queue: Daniel Verkamp <dverkamp@chromium.org>
Reviewed-by: Daniel Verkamp <dverkamp@chromium.org>
Reviewed-by: Stephen Barber <smbarber@chromium.org>
Diffstat (limited to 'devices/src/virtio/fs')
-rw-r--r--devices/src/virtio/fs/mod.rs286
-rw-r--r--devices/src/virtio/fs/passthrough.rs152
-rw-r--r--devices/src/virtio/fs/worker.rs96
3 files changed, 486 insertions, 48 deletions
diff --git a/devices/src/virtio/fs/mod.rs b/devices/src/virtio/fs/mod.rs
new file mode 100644
index 0000000..ff71dc1
--- /dev/null
+++ b/devices/src/virtio/fs/mod.rs
@@ -0,0 +1,286 @@
+// 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::ffi::FromBytesWithNulError;
+use std::fmt;
+use std::io;
+use std::mem;
+use std::os::unix::io::RawFd;
+use std::sync::atomic::AtomicUsize;
+use std::sync::Arc;
+use std::thread;
+
+use data_model::{DataInit, Le32};
+use sync::Mutex;
+use sys_util::{error, warn, Error as SysError, EventFd, GuestMemory};
+
+use crate::pci::MsixConfig;
+use crate::virtio::{
+    copy_config, DescriptorError, Interrupt, Queue, VirtioDevice, TYPE_FS, VIRTIO_F_VERSION_1,
+};
+
+mod filesystem;
+#[allow(dead_code)]
+mod fuse;
+mod multikey;
+pub mod passthrough;
+mod server;
+mod worker;
+
+use passthrough::PassthroughFs;
+use server::Server;
+use worker::Worker;
+
+// The fs device does not have a fixed number of queues.
+const QUEUE_SIZE: u16 = 1024;
+
+/// The maximum allowable length of the tag used to identify a specific virtio-fs device.
+pub const FS_MAX_TAG_LEN: usize = 36;
+
+/// kernel/include/uapi/linux/virtio_fs.h
+#[repr(C, packed)]
+#[derive(Clone, Copy)]
+struct Config {
+    /// Filesystem name (UTF-8, not NUL-terminated, padded with NULs)
+    tag: [u8; FS_MAX_TAG_LEN],
+    /// Number of request queues
+    num_queues: Le32,
+}
+
+// Safe because all members are plain old data and any value is valid.
+unsafe impl DataInit for Config {}
+
+/// Errors that may occur during the creation or operation of an Fs device.
+#[derive(Debug)]
+pub enum Error {
+    /// The tag for the Fs device was too long to fit in the config space.
+    TagTooLong(usize),
+    /// Failed to create the file system.
+    CreateFs(io::Error),
+    /// 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,
+    /// Failed to signal the virio used queue.
+    SignalUsedQueue(SysError),
+    /// Failed to decode protocol messages.
+    DecodeMessage(io::Error),
+    /// Failed to encode protocol messages.
+    EncodeMessage(io::Error),
+    /// One or more parameters are missing.
+    MissingParameter,
+    /// A C string parameter is invalid.
+    InvalidCString(FromBytesWithNulError),
+    /// The `len` field of the header is too small.
+    InvalidHeaderLength,
+    /// A `DescriptorChain` contains invalid data.
+    InvalidDescriptorChain(DescriptorError),
+    /// The `size` field of the `SetxattrIn` message does not match the length
+    /// of the decoded value.
+    InvalidXattrSize((u32, usize)),
+}
+
+impl ::std::error::Error for Error {}
+
+impl fmt::Display for Error {
+    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
+        use Error::*;
+        match self {
+            TagTooLong(len) => write!(
+                f,
+                "Fs device tag is too long: len = {}, max = {}",
+                len, FS_MAX_TAG_LEN
+            ),
+            CreateFs(err) => write!(f, "failed to create file system: {}", err),
+            CreatePollContext(err) => write!(f, "failed to create PollContext: {}", err),
+            PollError(err) => write!(f, "failed to poll events: {}", err),
+            ReadQueueEventFd(err) => write!(f, "failed to read from virtio queue EventFd: {}", err),
+            NoReadableDescriptors => write!(f, "request does not have any readable descriptors"),
+            NoWritableDescriptors => write!(f, "request does not have any writable descriptors"),
+            SignalUsedQueue(err) => write!(f, "failed to signal used queue: {}", err),
+            DecodeMessage(err) => write!(f, "failed to decode fuse message: {}", err),
+            EncodeMessage(err) => write!(f, "failed to encode fuse message: {}", err),
+            MissingParameter => write!(f, "one or more parameters are missing"),
+            InvalidHeaderLength => write!(f, "the `len` field of the header is too small"),
+            InvalidCString(err) => write!(f, "a c string parameter is invalid: {}", err),
+            InvalidDescriptorChain(err) => write!(f, "DescriptorChain is invalid: {}", err),
+            InvalidXattrSize((size, len)) => write!(
+                f,
+                "The `size` field of the `SetxattrIn` message does not match the length of the\
+                 decoded value: size = {}, value.len() = {}",
+                size, len
+            ),
+        }
+    }
+}
+
+pub type Result<T> = ::std::result::Result<T, Error>;
+
+pub struct Fs {
+    cfg: Config,
+    fs: Option<PassthroughFs>,
+    queue_sizes: Box<[u16]>,
+    avail_features: u64,
+    acked_features: u64,
+    workers: Vec<(EventFd, thread::JoinHandle<Result<()>>)>,
+}
+
+impl Fs {
+    pub fn new(tag: &str, num_workers: usize, fs_cfg: passthrough::Config) -> Result<Fs> {
+        if tag.len() > FS_MAX_TAG_LEN {
+            return Err(Error::TagTooLong(tag.len()));
+        }
+
+        let mut cfg_tag = [0u8; FS_MAX_TAG_LEN];
+        cfg_tag[..tag.len()].copy_from_slice(tag.as_bytes());
+
+        let cfg = Config {
+            tag: cfg_tag,
+            num_queues: Le32::from(num_workers as u32),
+        };
+
+        let fs = PassthroughFs::new(fs_cfg).map_err(Error::CreateFs)?;
+
+        // There is always a high priority queue in addition to the request queues.
+        let num_queues = num_workers + 1;
+
+        Ok(Fs {
+            cfg,
+            fs: Some(fs),
+            queue_sizes: vec![QUEUE_SIZE; num_queues].into_boxed_slice(),
+            avail_features: 1 << VIRTIO_F_VERSION_1,
+            acked_features: 0,
+            workers: Vec::with_capacity(num_workers + 1),
+        })
+    }
+
+    fn stop_workers(&mut self) {
+        for (kill_evt, handle) in mem::replace(&mut self.workers, Vec::new()) {
+            if let Err(e) = kill_evt.write(1) {
+                error!("failed to kill virtio-fs worker thread: {}", e);
+                continue;
+            }
+
+            // Only wait on the child thread if we were able to send it a kill event.
+            match handle.join() {
+                Ok(r) => {
+                    if let Err(e) = r {
+                        error!("virtio-fs worker thread exited with error: {}", e)
+                    }
+                }
+                Err(e) => error!("virtio-fs worker thread panicked: {:?}", e),
+            }
+        }
+    }
+}
+
+impl VirtioDevice for Fs {
+    fn keep_fds(&self) -> Vec<RawFd> {
+        self.fs
+            .as_ref()
+            .map(PassthroughFs::keep_fds)
+            .unwrap_or_else(Vec::new)
+    }
+
+    fn device_type(&self) -> u32 {
+        TYPE_FS
+    }
+
+    fn msix_vectors(&self) -> u16 {
+        self.queue_sizes.len() as u16
+    }
+
+    fn queue_max_sizes(&self) -> &[u16] {
+        &self.queue_sizes
+    }
+
+    fn features(&self) -> u64 {
+        self.avail_features
+    }
+
+    fn ack_features(&mut self, mut v: u64) {
+        // 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_fs got unknown feature ack: {:x}", v);
+
+            // Don't count these features as acked.
+            v &= !unrequested_features;
+        }
+        self.acked_features |= v;
+    }
+
+    fn read_config(&self, offset: u64, data: &mut [u8]) {
+        copy_config(data, 0, self.cfg.as_slice(), offset)
+    }
+
+    fn activate(
+        &mut self,
+        guest_mem: GuestMemory,
+        interrupt_evt: EventFd,
+        interrupt_resample_evt: EventFd,
+        msix_config: Option<Arc<Mutex<MsixConfig>>>,
+        status: Arc<AtomicUsize>,
+        queues: Vec<Queue>,
+        queue_evts: Vec<EventFd>,
+    ) {
+        if queues.len() != self.queue_sizes.len() || queue_evts.len() != self.queue_sizes.len() {
+            return;
+        }
+
+        let fs = self.fs.take().expect("missing file system implementation");
+
+        let server = Arc::new(Server::new(fs));
+        let irq = Arc::new(Interrupt::new(
+            status,
+            interrupt_evt,
+            interrupt_resample_evt,
+            msix_config,
+        ));
+
+        for (idx, (queue, evt)) in queues.into_iter().zip(queue_evts.into_iter()).enumerate() {
+            let (self_kill_evt, kill_evt) =
+                match EventFd::new().and_then(|e| Ok((e.try_clone()?, e))) {
+                    Ok(v) => v,
+                    Err(e) => {
+                        error!("fs: failed creating kill EventFd pair: {}", e);
+                        self.stop_workers();
+                        return;
+                    }
+                };
+
+            let mem = guest_mem.clone();
+            let server = server.clone();
+            let irq = irq.clone();
+
+            let worker_result = thread::Builder::new()
+                .name(format!("virtio-fs worker {}", idx))
+                .spawn(move || {
+                    let mut worker = Worker::new(mem, queue, server, irq);
+                    worker.run(evt, kill_evt)
+                });
+
+            match worker_result {
+                Ok(worker) => self.workers.push((self_kill_evt, worker)),
+                Err(e) => {
+                    error!("fs: failed to spawn virtio_fs worker: {}", e);
+                    self.stop_workers();
+                    return;
+                }
+            }
+        }
+    }
+}
+
+impl Drop for Fs {
+    fn drop(&mut self) {
+        self.stop_workers()
+    }
+}
diff --git a/devices/src/virtio/fs/passthrough.rs b/devices/src/virtio/fs/passthrough.rs
index 73e7e8b..4ca7ee6 100644
--- a/devices/src/virtio/fs/passthrough.rs
+++ b/devices/src/virtio/fs/passthrough.rs
@@ -63,38 +63,6 @@ struct LinuxDirent64 {
 }
 unsafe impl DataInit for LinuxDirent64 {}
 
-/// The caching policy that the server should report to the client.
-pub enum CachePolicy {
-    /// The client should never cache file data and all I/O should be directly forwarded
-    /// to the server.
-    Never,
-
-    /// The client is free to choose when and how to cache file data.
-    Auto,
-
-    /// The client should always cache file data.
-    Always,
-}
-
-impl FromStr for CachePolicy {
-    type Err = &'static str;
-
-    fn from_str(s: &str) -> Result<Self, Self::Err> {
-        match s {
-            "never" | "Never" | "NEVER" => Ok(CachePolicy::Never),
-            "auto" | "Auto" | "AUTO" => Ok(CachePolicy::Auto),
-            "always" | "Always" | "ALWAYS" => Ok(CachePolicy::Always),
-            _ => Err("invalid cache policy"),
-        }
-    }
-}
-
-impl Default for CachePolicy {
-    fn default() -> Self {
-        CachePolicy::Auto
-    }
-}
-
 macro_rules! scoped_cred {
     ($name:ident, $ty:ty, $syscall_nr:expr) => {
         #[derive(Debug)]
@@ -185,6 +153,99 @@ fn stat(f: &File) -> io::Result<libc::stat64> {
     }
 }
 
+/// The caching policy that the file system should report to the FUSE client. By default the FUSE
+/// protocol uses close-to-open consistency. This means that any cached contents of the file are
+/// invalidated the next time that file is opened.
+#[derive(Debug, Clone)]
+pub enum CachePolicy {
+    /// The client should never cache file data and all I/O should be directly forwarded to the
+    /// server. This policy must be selected when file contents may change without the knowledge of
+    /// the FUSE client (i.e., the file system does not have exclusive access to the directory).
+    Never,
+
+    /// The client is free to choose when and how to cache file data. This is the default policy and
+    /// uses close-to-open consistency as described in the enum documentation.
+    Auto,
+
+    /// The client should always cache file data. This means that the FUSE client will not
+    /// invalidate any cached data that was returned by the file system the last time the file was
+    /// opened. This policy should only be selected when the file system has exclusive access to the
+    /// directory.
+    Always,
+}
+
+impl FromStr for CachePolicy {
+    type Err = &'static str;
+
+    fn from_str(s: &str) -> Result<Self, Self::Err> {
+        match s {
+            "never" | "Never" | "NEVER" => Ok(CachePolicy::Never),
+            "auto" | "Auto" | "AUTO" => Ok(CachePolicy::Auto),
+            "always" | "Always" | "ALWAYS" => Ok(CachePolicy::Always),
+            _ => Err("invalid cache policy"),
+        }
+    }
+}
+
+impl Default for CachePolicy {
+    fn default() -> Self {
+        CachePolicy::Auto
+    }
+}
+
+/// Options that configure the behavior of the file system.
+#[derive(Debug, Clone)]
+pub struct Config {
+    /// How long the FUSE client should consider directory entries to be valid. If the contents of a
+    /// directory can only be modified by the FUSE client (i.e., the file system has exclusive
+    /// access), then this should be a large value.
+    ///
+    /// The default value for this option is 5 seconds.
+    pub entry_timeout: Duration,
+
+    /// How long the FUSE client should consider file and directory attributes to be valid. If the
+    /// attributes of a file or directory can only be modified by the FUSE client (i.e., the file
+    /// system has exclusive access), then this should be set to a large value.
+    ///
+    /// The default value for this option is 5 seconds.
+    pub attr_timeout: Duration,
+
+    /// The caching policy the file system should use. See the documentation of `CachePolicy` for
+    /// more details.
+    pub cache_policy: CachePolicy,
+
+    /// Whether the file system should enabled writeback caching. This can improve performance as it
+    /// allows the FUSE client to cache and coalesce multiple writes before sending them to the file
+    /// system. However, enabling this option can increase the risk of data corruption if the file
+    /// contents can change without the knowledge of the FUSE client (i.e., the server does **NOT**
+    /// have exclusive access). Additionally, the file system should have read access to all files
+    /// in the directory it is serving as the FUSE client may send read requests even for files
+    /// opened with `O_WRONLY`.
+    ///
+    /// Therefore callers should only enable this option when they can guarantee that: 1) the file
+    /// system has exclusive access to the directory and 2) the file system has read permissions for
+    /// all files in that directory.
+    ///
+    /// The default value for this option is `false`.
+    pub writeback: bool,
+}
+
+impl Default for Config {
+    fn default() -> Self {
+        Config {
+            entry_timeout: Duration::from_secs(5),
+            attr_timeout: Duration::from_secs(5),
+            cache_policy: Default::default(),
+            writeback: false,
+        }
+    }
+}
+
+/// A file system that simply "passes through" all requests it receives to the underlying file
+/// system. To keep the implementation simple it servers the contents of its root directory. Users
+/// that wish to serve only a specific directory should set up the environment so that that
+/// directory ends up as the root of the file system process. One way to accomplish this is via a
+/// combination of mount namespaces and the pivot_root system call.
 pub struct PassthroughFs {
     // File descriptors for various points in the file system tree. These fds are always opened with
     // the `O_PATH` option so they cannot be used for reading or writing any data. See the
@@ -204,19 +265,15 @@ pub struct PassthroughFs {
     // to be serving doesn't have access to `/proc`.
     proc: File,
 
-    // Whether writeback caching is enabled for this directory. This can improve write performance
-    // as it allows the guest to complete write requests before the data has been flushed to this
-    // server. However, this also has the possibility of causing data corruption as the contents of
-    // a file may change on disk while they are still buffered in the guest. So this should only be
-    // enabled when the guest has exclusive access to the directory being shared.
+    // Whether writeback caching is enabled for this directory. This will only be true when
+    // `cfg.writeback` is true and `init` was called with `FsOptions::WRITEBACK_CACHE`.
     writeback: AtomicBool,
 
-    timeout: Duration,
-    cache_policy: CachePolicy,
+    cfg: Config,
 }
 
 impl PassthroughFs {
-    pub fn new(timeout: Duration, cache_policy: CachePolicy) -> io::Result<PassthroughFs> {
+    pub fn new(cfg: Config) -> io::Result<PassthroughFs> {
         // Safe because this is a constant value and a valid C string.
         let proc_cstr = unsafe { CStr::from_bytes_with_nul_unchecked(PROC_CSTR) };
 
@@ -245,8 +302,7 @@ impl PassthroughFs {
             proc,
 
             writeback: AtomicBool::new(false),
-            timeout,
-            cache_policy,
+            cfg,
         })
     }
 
@@ -365,8 +421,8 @@ impl PassthroughFs {
             inode,
             generation: 0,
             attr: st,
-            attr_timeout: self.timeout.clone(),
-            entry_timeout: self.timeout.clone(),
+            attr_timeout: self.cfg.attr_timeout.clone(),
+            entry_timeout: self.cfg.entry_timeout.clone(),
         })
     }
 
@@ -484,7 +540,7 @@ impl PassthroughFs {
         self.handles.write().unwrap().insert(handle, Arc::new(data));
 
         let mut opts = OpenOptions::empty();
-        match self.cache_policy {
+        match self.cfg.cache_policy {
             // We only set the direct I/O option on files.
             CachePolicy::Never => opts.set(
                 OpenOptions::DIRECT_IO,
@@ -523,7 +579,7 @@ impl PassthroughFs {
 
         let st = stat(&data.file)?;
 
-        Ok((st, self.timeout.clone()))
+        Ok((st, self.cfg.attr_timeout.clone()))
     }
 
     fn do_unlink(&self, parent: Inode, name: &CStr, flags: libc::c_int) -> io::Result<()> {
@@ -631,7 +687,7 @@ impl FileSystem for PassthroughFs {
         );
 
         let mut opts = FsOptions::DO_READDIRPLUS | FsOptions::READDIRPLUS_AUTO;
-        if capable.contains(FsOptions::WRITEBACK_CACHE) {
+        if self.cfg.writeback && capable.contains(FsOptions::WRITEBACK_CACHE) {
             opts |= FsOptions::WRITEBACK_CACHE;
             self.writeback.store(true, Ordering::Relaxed);
         }
@@ -841,7 +897,7 @@ impl FileSystem for PassthroughFs {
         self.handles.write().unwrap().insert(handle, Arc::new(data));
 
         let mut opts = OpenOptions::empty();
-        match self.cache_policy {
+        match self.cfg.cache_policy {
             CachePolicy::Never => opts |= OpenOptions::DIRECT_IO,
             CachePolicy::Always => opts |= OpenOptions::KEEP_CACHE,
             _ => {}
diff --git a/devices/src/virtio/fs/worker.rs b/devices/src/virtio/fs/worker.rs
new file mode 100644
index 0000000..ec0ec8a
--- /dev/null
+++ b/devices/src/virtio/fs/worker.rs
@@ -0,0 +1,96 @@
+// 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::sync::Arc;
+
+use sys_util::{error, EventFd, GuestMemory, PollContext, PollToken};
+
+use crate::virtio::fs::filesystem::FileSystem;
+use crate::virtio::fs::server::Server;
+use crate::virtio::fs::{Error, Result};
+use crate::virtio::{Interrupt, Queue, Reader, Writer};
+
+pub struct Worker<F: FileSystem + Sync> {
+    mem: GuestMemory,
+    queue: Queue,
+    server: Arc<Server<F>>,
+    irq: Arc<Interrupt>,
+}
+
+impl<F: FileSystem + Sync> Worker<F> {
+    pub fn new(
+        mem: GuestMemory,
+        queue: Queue,
+        server: Arc<Server<F>>,
+        irq: Arc<Interrupt>,
+    ) -> Worker<F> {
+        Worker {
+            mem,
+            queue,
+            server,
+            irq,
+        }
+    }
+
+    fn process_queue(&mut self) -> Result<()> {
+        let mut needs_interrupt = false;
+        while let Some(avail_desc) = self.queue.pop(&self.mem) {
+            let reader = Reader::new(&self.mem, avail_desc.clone())
+                .map_err(Error::InvalidDescriptorChain)?;
+            let writer = Writer::new(&self.mem, avail_desc.clone())
+                .map_err(Error::InvalidDescriptorChain)?;
+
+            let total = self.server.handle_message(reader, writer)?;
+
+            self.queue
+                .add_used(&self.mem, avail_desc.index, total as u32);
+
+            needs_interrupt = true;
+        }
+
+        if needs_interrupt {
+            self.irq.signal_used_queue(self.queue.vector);
+        }
+
+        Ok(())
+    }
+
+    pub fn run(&mut self, queue_evt: EventFd, kill_evt: EventFd) -> Result<()> {
+        #[derive(PollToken)]
+        enum Token {
+            // A request is ready on the queue.
+            QueueReady,
+            // Check if any interrupts need to be re-asserted.
+            InterruptResample,
+            // The parent thread requested an exit.
+            Kill,
+        }
+
+        let poll_ctx = PollContext::build_with(&[
+            (&queue_evt, Token::QueueReady),
+            (&kill_evt, Token::Kill),
+            (self.irq.get_resample_evt(), Token::InterruptResample),
+        ])
+        .map_err(Error::CreatePollContext)?;
+
+        loop {
+            let events = poll_ctx.wait().map_err(Error::PollError)?;
+            for event in events.iter_readable() {
+                match event.token() {
+                    Token::QueueReady => {
+                        queue_evt.read().map_err(Error::ReadQueueEventFd)?;
+                        if let Err(e) = self.process_queue() {
+                            error!("virtio-fs transport error: {}", e);
+                            return Err(e);
+                        }
+                    }
+                    Token::InterruptResample => {
+                        self.irq.interrupt_resample();
+                    }
+                    Token::Kill => return Ok(()),
+                }
+            }
+        }
+    }
+}