diff --git a/bin/propolis-server/src/lib/initializer.rs b/bin/propolis-server/src/lib/initializer.rs index e55263494..eff35e9fb 100644 --- a/bin/propolis-server/src/lib/initializer.rs +++ b/bin/propolis-server/src/lib/initializer.rs @@ -12,8 +12,8 @@ use std::time::{SystemTime, UNIX_EPOCH}; use crate::serial::Serial; use crate::spec::{self, Spec, StorageBackend, StorageDevice}; use crate::stats::{ - track_network_interface_kstats, track_vcpu_kstats, VirtualDiskProducer, - VirtualMachine, + track_network_interface_kstats, track_vcpu_kstats, BlockMetrics, + VirtualDisk, VirtualMachine, }; use crate::vm::{ BlockBackendMap, CrucibleBackendMap, DeviceMap, NetworkInterfaceIds, @@ -658,7 +658,8 @@ impl MachineInitializer<'_> { let vioblk = virtio::PciVirtioBlock::new(0x100); self.devices.insert(device_id.clone(), vioblk.clone()); - block::attach(vioblk.clone(), backend).unwrap(); + block::attach(&vioblk.block_attach, backend.attachment()) + .unwrap(); chipset.pci_attach(bdf, vioblk.clone()); vioblk } @@ -678,7 +679,8 @@ impl MachineInitializer<'_> { self.log.new(slog::o!("component" => component)), ); self.devices.insert(device_id.clone(), nvme.clone()); - block::attach(nvme.clone(), backend).unwrap(); + block::attach(&nvme.block_attach, backend.attachment()) + .unwrap(); chipset.pci_attach(bdf, nvme.clone()); nvme } @@ -720,14 +722,20 @@ impl MachineInitializer<'_> { }; if let Some(registry) = &self.producer_registry { - let stats = VirtualDiskProducer::new( - block_size, - self.properties.id, - volume_id, - &self.properties.metadata, + let block_metrics = BlockMetrics::new( + VirtualDisk { + attached_instance_id: self.properties.id, + block_size, + disk_id: volume_id, + project_id: self.properties.metadata.project_id, + silo_id: self.properties.metadata.silo_id, + }, + block_dev.attachment().max_queues(), ); - if let Err(e) = registry.register_producer(stats.clone()) { + if let Err(e) = + registry.register_producer(block_metrics.producer()) + { slog::error!( self.log, "Could not register virtual disk producer, \ @@ -739,12 +747,7 @@ impl MachineInitializer<'_> { continue; }; - // Set the on-completion callback for the block device, to - // update stats. - let callback = move |op, result, duration| { - stats.on_completion(op, result, duration); - }; - block_dev.on_completion(Box::new(callback)); + block_dev.attachment().set_metric_consumer(block_metrics); }; } } diff --git a/bin/propolis-server/src/lib/stats/mod.rs b/bin/propolis-server/src/lib/stats/mod.rs index f4c9033f4..fb24e1bc7 100644 --- a/bin/propolis-server/src/lib/stats/mod.rs +++ b/bin/propolis-server/src/lib/stats/mod.rs @@ -28,7 +28,7 @@ mod virtual_machine; #[cfg(all(not(test), target_os = "illumos"))] use self::network_interface::InstanceNetworkInterfaces; pub(crate) use self::pvpanic::PvpanicProducer; -pub(crate) use self::virtual_disk::VirtualDiskProducer; +pub(crate) use self::virtual_disk::{BlockMetrics, VirtualDisk}; pub(crate) use self::virtual_machine::VirtualMachine; /// Interval on which we ask `oximeter` to poll us for metric data. diff --git a/bin/propolis-server/src/lib/stats/virtual_disk.rs b/bin/propolis-server/src/lib/stats/virtual_disk.rs index d37a28f25..4a39cfde0 100644 --- a/bin/propolis-server/src/lib/stats/virtual_disk.rs +++ b/bin/propolis-server/src/lib/stats/virtual_disk.rs @@ -7,6 +7,7 @@ //! Types for tracking statistics about virtual disks. use std::{ + num::NonZeroUsize, sync::{Arc, Mutex}, time::Duration, }; @@ -22,12 +23,11 @@ use oximeter::{ MetricsError, Producer, Sample, }; use propolis::block::{self, Operation}; -use propolis_api_types::InstanceMetadata; -use uuid::Uuid; +pub use self::virtual_disk::VirtualDisk; use self::virtual_disk::{ BytesRead, BytesWritten, FailedFlushes, FailedReads, FailedWrites, Flushes, - IoLatency, IoSize, Reads, VirtualDisk, Writes, + IoLatency, IoSize, Reads, Writes, }; /// Type for tracking virtual disk stats. @@ -67,12 +67,8 @@ struct VirtualDiskStats { impl VirtualDiskStats { /// Update the tracked statistics with the result of an I/O completion. - fn on_completion( - &mut self, - op: block::Operation, - result: block::Result, - duration: Duration, - ) { + fn on_completion(&mut self, sample: BlockSample) { + let BlockSample { op, result, duration } = sample; match op { Operation::Read(_, len) => { self.on_read_completion(result, len, duration) @@ -190,6 +186,12 @@ const LATENCY_POWERS: (u16, u16) = (3, 10); /// We use 512B as the minimum since that is the minimum supported block size. const SIZE_POWERS: (u16, u16) = (9, 30); +/// Maximum number of samples to buffer for a given device queue before +/// consolidating them into the totals for the device. +/// +/// This value was arbitrarily chosen +const MAX_BUFFERED_SAMPLES: usize = 512; + /// A [`Producer`] that emits statistics about virtual disks. /// /// This type is shared between the block devie that handles guest I/Os, and the @@ -199,32 +201,19 @@ const SIZE_POWERS: (u16, u16) = (9, 30); /// /// As oximeter polls us, the producer server also collects these updated /// statistics. -#[derive(Clone, Debug)] -pub struct VirtualDiskProducer { - // Shareable inner type actually managing the stats. - inner: Arc>, -} -impl VirtualDiskProducer { - /// Create a producer to track a virtual disk. - pub fn new( - block_size: u32, - instance_id: Uuid, - disk_id: Uuid, - metadata: &InstanceMetadata, - ) -> Self { - let disk = VirtualDisk { - attached_instance_id: instance_id, - block_size, - disk_id, - project_id: metadata.project_id, - silo_id: metadata.silo_id, - }; +#[derive(Debug)] +pub(crate) struct BlockMetrics { + sample_buffer: Vec>>, + stats: Mutex, +} +impl BlockMetrics { + pub fn new(disk: VirtualDisk, max_queues: NonZeroUsize) -> Arc { let now = Utc::now(); let datum = Cumulative::with_start_time(now, 0); let latency_histogram = Self::latency_histogram(); let size_histogram = Self::size_histogram(); - let inner = VirtualDiskStats { + let stats = VirtualDiskStats { disk, reads: Reads { datum }, bytes_read: BytesRead { datum }, @@ -274,17 +263,14 @@ impl VirtualDiskProducer { }, ], }; - Self { inner: Arc::new(Mutex::new(inner)) } + let mut sample_buffer = Vec::with_capacity(max_queues.get()); + sample_buffer.resize_with(max_queues.get(), Default::default); + + Arc::new(Self { sample_buffer, stats: Mutex::new(stats) }) } - /// A callback that updates statistics with the result of a completed I/O. - pub fn on_completion( - &self, - op: block::Operation, - result: block::Result, - duration: Duration, - ) { - self.inner.lock().unwrap().on_completion(op, result, duration); + pub(crate) fn producer(self: &Arc) -> VirtualDiskProducer { + VirtualDiskProducer(self.clone()) } /// Construct a histogram for tracking I/O latencies. @@ -307,60 +293,115 @@ impl VirtualDiskProducer { // Safety: This only fails if the bins are not valid. Histogram::new(&bins).unwrap() } + + fn consolidate_one(&self, idx: usize) { + let mut stats = self.stats.lock().unwrap(); + let mut buf = self.sample_buffer.get(idx).unwrap().lock().unwrap(); + for sample in buf.drain(..) { + stats.on_completion(sample); + } + } + fn consolidate_all(&self) { + let mut stats = self.stats.lock().unwrap(); + for buf in self.sample_buffer.iter() { + let mut buf = buf.lock().unwrap(); + for sample in buf.drain(..) { + stats.on_completion(sample); + } + } + } +} + +#[derive(Debug)] +struct BlockSample { + op: block::Operation, + result: block::Result, + duration: Duration, +} + +#[derive(Clone, Debug)] +pub struct VirtualDiskProducer(Arc); + +impl block::MetricConsumer for BlockMetrics { + fn request_completed( + &self, + queue_id: block::QueueId, + op: Operation, + result: block::Result, + _time_queued: Duration, + time_processed: Duration, + ) { + let idx = usize::from(queue_id); + let buf = self + .sample_buffer + .get(idx) + .expect("queue ID should be within maximum"); + let mut guard = buf.lock().unwrap(); + guard.push(BlockSample { op, result, duration: time_processed }); + + // Do not let an unbounded number of samples accumulate + if guard.len() > MAX_BUFFERED_SAMPLES { + drop(guard); + self.consolidate_one(idx); + } + } } impl Producer for VirtualDiskProducer { fn produce( &mut self, ) -> Result>, MetricsError> { + // Consolidate any buffer samples first + self.0.consolidate_all(); + // 5 scalar samples (reads, writes, flushes, bytes read / written) // 3 scalars broken out by failure kind // 2 histograms broken out by I/O kind const N_SAMPLES: usize = 5 + 3 * N_FAILURE_KINDS + 2 * N_IO_KINDS; let mut out = Vec::with_capacity(N_SAMPLES); - let inner = self.inner.lock().unwrap(); + let stats = self.0.stats.lock().unwrap(); // Read statistics. - out.push(Sample::new(&inner.disk, &inner.reads)?); - out.push(Sample::new(&inner.disk, &inner.bytes_read)?); - for failed in inner.failed_reads.iter() { - out.push(Sample::new(&inner.disk, failed)?); + out.push(Sample::new(&stats.disk, &stats.reads)?); + out.push(Sample::new(&stats.disk, &stats.bytes_read)?); + for failed in stats.failed_reads.iter() { + out.push(Sample::new(&stats.disk, failed)?); } // Write statistics. - out.push(Sample::new(&inner.disk, &inner.writes)?); - out.push(Sample::new(&inner.disk, &inner.bytes_written)?); - for failed in inner.failed_writes.iter() { - out.push(Sample::new(&inner.disk, failed)?); + out.push(Sample::new(&stats.disk, &stats.writes)?); + out.push(Sample::new(&stats.disk, &stats.bytes_written)?); + for failed in stats.failed_writes.iter() { + out.push(Sample::new(&stats.disk, failed)?); } // Flushes - out.push(Sample::new(&inner.disk, &inner.flushes)?); - for failed in inner.failed_flushes.iter() { - out.push(Sample::new(&inner.disk, failed)?); + out.push(Sample::new(&stats.disk, &stats.flushes)?); + for failed in stats.failed_flushes.iter() { + out.push(Sample::new(&stats.disk, failed)?); } // Histograms for latency and size. - for hist in inner.io_latency.iter() { - out.push(Sample::new(&inner.disk, hist)?); + for hist in stats.io_latency.iter() { + out.push(Sample::new(&stats.disk, hist)?); } - for hist in inner.io_size.iter() { - out.push(Sample::new(&inner.disk, hist)?); + for hist in stats.io_size.iter() { + out.push(Sample::new(&stats.disk, hist)?); } - drop(inner); + drop(stats); Ok(Box::new(out.into_iter())) } } #[cfg(test)] mod test { - use super::VirtualDiskProducer; + use super::BlockMetrics; use super::LATENCY_POWERS; use super::SIZE_POWERS; #[test] fn test_latency_histogram() { - let hist = VirtualDiskProducer::latency_histogram(); + let hist = BlockMetrics::latency_histogram(); println!("{:#?}", hist.iter().map(|bin| bin.range).collect::>()); // The math here is a bit silly, but we end up with 9 bins in each // "interior" power of 10, plus one more bin on the right and left for @@ -373,7 +414,7 @@ mod test { #[test] fn test_size_histogram() { - let hist = VirtualDiskProducer::size_histogram(); + let hist = BlockMetrics::size_histogram(); println!("{:#?}", hist.iter().map(|bin| bin.range).collect::>()); // 1 extra left bin for [0, 512), and 1 because the range is inclusive. assert_eq!( diff --git a/bin/propolis-server/src/lib/vm/objects.rs b/bin/propolis-server/src/lib/vm/objects.rs index b35e3a4d0..953085be9 100644 --- a/bin/propolis-server/src/lib/vm/objects.rs +++ b/bin/propolis-server/src/lib/vm/objects.rs @@ -18,7 +18,7 @@ use propolis::{ Machine, }; use propolis_api_types::instance_spec::SpecKey; -use slog::{error, info}; +use slog::info; use tokio::sync::{RwLock, RwLockReadGuard, RwLockWriteGuard}; use crate::{serial::Serial, spec::Spec, vcpu_tasks::VcpuTaskController}; @@ -383,11 +383,7 @@ impl VmObjectsLocked { for (id, backend) in self.block_backends.iter() { info!(self.log, "stopping and detaching block backend {}", id); backend.stop().await; - if let Err(err) = backend.detach() { - error!(self.log, "error detaching block backend"; - "id" => %id, - "error" => ?err); - } + backend.attachment().detach(); } } diff --git a/bin/propolis-standalone/src/main.rs b/bin/propolis-standalone/src/main.rs index 7e5cbbefe..507284faf 100644 --- a/bin/propolis-standalone/src/main.rs +++ b/bin/propolis-standalone/src/main.rs @@ -234,7 +234,7 @@ impl Inventory { fn destroy(&mut self) { // Detach all block backends from their devices for backend in self.block.values() { - let _ = backend.attachment().detach(); + backend.attachment().detach(); } // Drop all refs in the hopes that things can clean up after themselves @@ -321,7 +321,7 @@ impl Instance { state: State, guard: &MutexGuard, first_boot: bool, - log: &slog::Logger, + _log: &slog::Logger, ) { for (name, device) in guard.inventory.devs.iter() { match state { @@ -368,16 +368,9 @@ impl Instance { } State::Halt => { tokio::runtime::Handle::current().block_on(async { - for (name, be) in guard.inventory.block.iter() { + for (_name, be) in guard.inventory.block.iter() { be.stop().await; - if let Err(err) = be.detach() { - slog::error!( - log, - "Error during detach of block backend {}: {:?}", - name, - err - ); - } + be.attachment().detach(); } }); } @@ -1213,7 +1206,8 @@ fn setup_instance( .register_instance(&vioblk, &bdf.to_string()); guard.inventory.register_block(&backend, name); - block::attach(vioblk.clone(), backend).unwrap(); + block::attach(&vioblk.block_attach, backend.attachment()) + .unwrap(); chipset_pci_attach(bdf, vioblk); } "pci-virtio-viona" => { @@ -1272,7 +1266,8 @@ fn setup_instance( guard.inventory.register_instance(&nvme, &bdf.to_string()); guard.inventory.register_block(&backend, name); - block::attach(nvme.clone(), backend).unwrap(); + block::attach(&nvme.block_attach, backend.attachment()) + .unwrap(); chipset_pci_attach(bdf, nvme); } qemu::pvpanic::DEVICE_NAME => { diff --git a/lib/propolis/Cargo.toml b/lib/propolis/Cargo.toml index 88a04558a..5ac6f0e96 100644 --- a/lib/propolis/Cargo.toml +++ b/lib/propolis/Cargo.toml @@ -3,7 +3,7 @@ name = "propolis" version = "0.1.0" license = "MPL-2.0" edition = "2021" -rust-version = "1.70" +rust-version = "1.83" [dependencies] libc.workspace = true diff --git a/lib/propolis/src/attachment.rs b/lib/propolis/src/attachment.rs deleted file mode 100644 index 5ccf9d2df..000000000 --- a/lib/propolis/src/attachment.rs +++ /dev/null @@ -1,279 +0,0 @@ -// This Source Code Form is subject to the terms of the Mozilla Public -// License, v. 2.0. If a copy of the MPL was not distributed with this -// file, You can obtain one at https://mozilla.org/MPL/2.0/. - -use std::sync::atomic::{AtomicBool, Ordering}; -use std::sync::{Arc, Mutex}; - -/// Frontend (emulated device) attachment point -pub struct FrontAttachment(Mutex>); -impl FrontAttachment { - pub fn new() -> Self { - Self(Mutex::new(AttachPoint::default())) - } - pub fn access( - &self, - when_attached: impl FnOnce(&T, &AttachState) -> R, - ) -> Option { - self.0.lock().unwrap().access_fallback(when_attached, |_state| ()) - } - pub fn access_fallback( - &self, - when_attached: impl FnOnce(&T, &AttachState) -> R, - fallback: impl FnOnce(&AttachState) -> (), - ) -> Option { - self.0.lock().unwrap().access_fallback(when_attached, fallback) - } -} - -/// Backend (resource provider) attachment point -pub struct BackAttachment(Mutex>); -impl BackAttachment { - pub fn new() -> Self { - Self(Mutex::new(AttachPoint::default())) - } - pub fn access( - &self, - when_attached: impl FnOnce(&T, &AttachState) -> R, - ) -> Option { - self.0.lock().unwrap().access_fallback(when_attached, |_state| ()) - } - pub fn access_fallback( - &self, - when_attached: impl FnOnce(&T, &AttachState) -> R, - fallback: impl FnOnce(&AttachState) -> (), - ) -> Option { - self.0.lock().unwrap().access_fallback(when_attached, fallback) - } -} - -enum AttachPoint { - Attached(Arc<(AttachState, T)>), - Detached(AttachState), -} -impl Default for AttachPoint { - fn default() -> Self { - Self::Detached(AttachState::new()) - } -} -impl AttachPoint { - fn access_fallback( - &self, - when_attached: impl FnOnce(&T, &AttachState) -> R, - fallback: impl FnOnce(&AttachState) -> (), - ) -> Option { - match self { - AttachPoint::Attached(data) => { - Some(when_attached(&data.1, &data.0)) - } - AttachPoint::Detached(state) => { - fallback(state); - None - } - } - } -} - -/// State of an Attachment ([FrontAttachment] or [BackAttachment]) -#[derive(Clone)] -pub struct AttachState(Arc); -impl AttachState { - /// Is this Attachment attached to a corresponding peer? - /// - /// Because the [AttachState] can be cloned to outlive its initial access - /// via [FrontAttachment::access] or [BackAttachment::access], the ability - /// to check attachedness remains relevant. - pub fn is_attached(&self) -> bool { - self.0.is_attached.load(Ordering::Acquire) - } - /// Is the Attachment frontend paused? - pub fn is_paused(&self) -> bool { - self.0.is_paused.load(Ordering::Acquire) - } - /// Is the Attachmend backend stopped (from processing additional requests) - pub fn is_stopped(&self) -> bool { - self.0.is_stopped.load(Ordering::Acquire) - } - /// Notify waiters of event (either attachment state change, or of - /// attachment-specific events, such as new pending requests) - pub fn notify(&self) { - self.0.notify.notify_waiters() - } - /// Wait for notification relating to Attachment - pub fn notified(&self) -> tokio::sync::futures::Notified<'_> { - self.0.notify.notified() - } - /// Mark Attachment frontend as paused - pub fn pause(&self) { - if !self.0.is_paused.swap(true, Ordering::Release) { - self.0.notify.notify_waiters(); - } - } - /// Mark Attachment frontend as resumed (no longer paused) - pub fn resume(&self) { - if self.0.is_paused.swap(false, Ordering::Release) { - self.0.notify.notify_waiters(); - } - } - /// Mark Attachment backend as stopped - pub fn stop(&self) { - if !self.0.is_stopped.swap(true, Ordering::Release) { - self.0.notify.notify_waiters(); - } - } - /// Mark Attachment backend as started (no longer stopped) - pub fn start(&self) { - if self.0.is_stopped.swap(false, Ordering::Release) { - self.0.notify.notify_waiters(); - } - } - - fn new() -> Self { - Self(Arc::new(AttachStateInner::default())) - } - fn merge(front: &Self, back: &Self) -> Self { - // Because `is_paused` state is a property of the frontend (suspending - // emission of requests to be processed by backend) and `is_stopped` - // state is a property of the backend (stopping workers, so none are - // attempting to process requests from frontend), we use those bits of - // state from their respective "owners". - Self(Arc::new(AttachStateInner { - is_attached: AtomicBool::new(false), - is_paused: AtomicBool::new(front.is_paused()), - is_stopped: AtomicBool::new(back.is_stopped()), - ..Default::default() - })) - } - fn split(&self) -> (Self, Self) { - ( - Self(Arc::new(AttachStateInner { - is_paused: AtomicBool::new(self.is_paused()), - ..Default::default() - })), - Self(Arc::new(AttachStateInner { - is_stopped: AtomicBool::new(self.is_stopped()), - ..Default::default() - })), - ) - } - - fn attach(&self) { - let prev = self.0.is_attached.swap(true, Ordering::Release); - self.0.notify.notify_waiters(); - assert!(!prev, "AttachState was not 'detached' during attach()"); - } - fn detach(&self) { - let prev = self.0.is_attached.swap(false, Ordering::Release); - self.0.notify.notify_waiters(); - assert!(prev, "AttachState was not 'attached' during detach()"); - } -} -struct AttachStateInner { - is_attached: AtomicBool, - is_paused: AtomicBool, - is_stopped: AtomicBool, - notify: tokio::sync::Notify, -} -impl Default for AttachStateInner { - fn default() -> Self { - Self { - is_attached: AtomicBool::new(false), - is_paused: AtomicBool::new(false), - is_stopped: AtomicBool::new(true), - notify: tokio::sync::Notify::new(), - } - } -} - -/// Errors possible during attempted [attach] call -#[derive(thiserror::Error, Debug)] -pub enum AttachError { - #[error("frontend is already attached")] - FrontAttached, - #[error("backend is already attached")] - BackAttached, -} - -/// Attempt to attach `front_attach` and `back_attach`. -/// -/// If neither are already attached to another instance, they'll be attached -/// together and the `on_attach` handler run against a reference of the provided -/// `data` once the attachment has been made. -pub fn attach( - data: T, - front_attach: &FrontAttachment, - back_attach: &BackAttachment, - on_attach: impl FnOnce(&T), -) -> Result<(), AttachError> { - let mut front_lock = front_attach.0.lock().unwrap(); - let mut back_lock = back_attach.0.lock().unwrap(); - - let att_state = match (&*front_lock, &*back_lock) { - ( - AttachPoint::Detached(front_state), - AttachPoint::Detached(back_state), - ) => Ok(AttachState::merge(front_state, back_state)), - (AttachPoint::Attached(..), _) => Err(AttachError::FrontAttached), - (_, AttachPoint::Attached(..)) => Err(AttachError::BackAttached), - }?; - - let att_data = Arc::new((att_state, data)); - *front_lock = AttachPoint::Attached(att_data.clone()); - *back_lock = AttachPoint::Attached(att_data.clone()); - - att_data.0.attach(); - on_attach(&att_data.1); - - Ok(()) -} - -/// Errors possible during attempted [detach] call -#[derive(thiserror::Error, Debug)] -pub enum DetachError { - #[error("frontend is not attached")] - FrontNotAttached, - #[error("backend is not attached")] - BackNotAttached, - #[error("frontend and backend are not attached to each other")] - AttachmentMismatch, - #[error("detach handler failed")] - DetachFailed, -} - -/// Attempt to detach `front_attach` from `back_attach`. -/// -/// If those attachment points are indeed attached to each other, the -/// `on_detach` handler will be run, and if it does not yield an error, the -/// detach operation will complete successfully. -pub fn detach( - front_attach: &FrontAttachment, - back_attach: &BackAttachment, - on_detach: impl FnOnce(&T, &AttachState) -> Result<(), ()>, -) -> Result<(), DetachError> { - let mut front_lock = front_attach.0.lock().unwrap(); - let mut back_lock = back_attach.0.lock().unwrap(); - - let data = match (&*front_lock, &*back_lock) { - ( - AttachPoint::Attached(front_data), - AttachPoint::Attached(back_data), - ) => { - if !Arc::ptr_eq(front_data, back_data) { - Err(DetachError::AttachmentMismatch) - } else { - Ok(front_data.clone()) - } - } - (AttachPoint::Detached(..), _) => Err(DetachError::FrontNotAttached), - (_, AttachPoint::Detached(..)) => Err(DetachError::BackNotAttached), - }?; - - on_detach(&data.1, &data.0).map_err(|_| DetachError::DetachFailed)?; - data.0.detach(); - let (front_state, back_state) = data.0.split(); - - *front_lock = AttachPoint::Detached(front_state); - *back_lock = AttachPoint::Detached(back_state); - - Ok(()) -} diff --git a/lib/propolis/src/block/attachment.rs b/lib/propolis/src/block/attachment.rs index 2e521049f..b8f515fd5 100644 --- a/lib/propolis/src/block/attachment.rs +++ b/lib/propolis/src/block/attachment.rs @@ -2,308 +2,1218 @@ // License, v. 2.0. If a copy of the MPL was not distributed with this // file, You can obtain one at https://mozilla.org/MPL/2.0/. +//! Block "attachments" provide the plumbing between emulated devices and the +//! backends which execute the IO requests from said devices. +//! +//! Each emulated block device will contain a [DeviceAttachment] to which it +//! will associate one or more [DeviceQueue] instances. The queue(s) is the +//! source of [super::Request]s, which are to be processed by an attached +//! backend. +//! +//! Block backends will each contain a [BackendAttachment] which they will +//! request worker contexts from ([SyncWorkerCtx] or [AsyncWorkerCtx]). It is +//! through the worker context that the backend will fetch [super::Request]s +//! from the associated device in order to process them. + +use std::collections::BTreeMap; use std::future::Future; +use std::marker::PhantomPinned; +use std::num::NonZeroUsize; use std::pin::Pin; -use std::sync::{Arc, Condvar, Mutex}; +use std::sync::atomic::{AtomicU32, AtomicUsize, Ordering}; +use std::sync::{Arc, Condvar, Mutex, MutexGuard, Weak}; use std::task::{Context, Poll}; -use super::{Backend, CacheMode, Device, DeviceInfo, Request}; +use super::minder::{NoneInFlight, QueueMinder}; +use super::{ + devq_id, probes, DeviceId, DeviceInfo, DeviceQueue, DeviceRequest, + MetricConsumer, QueueId, WorkerId, +}; use crate::accessors::MemAccessor; -use crate::attachment; +use futures::stream::FuturesUnordered; +use futures::Stream; use pin_project_lite::pin_project; +use strum::IntoStaticStr; +use thiserror::Error; use tokio::sync::futures::Notified; +use tokio::sync::Notify; -struct BlockData { - device: Arc, - backend: Arc, - acc_mem: MemAccessor, - lock: Mutex<()>, - cv: Condvar, +/// Static for generating unique block [DeviceId]s with a process +static NEXT_DEVICE_ID: AtomicU32 = AtomicU32::new(0); + +pub const MAX_WORKERS: NonZeroUsize = NonZeroUsize::new(64).unwrap(); + +pub type ReqCountHint = Option; + +#[derive(Default)] +struct QueueSlotState { + minder: Option>, } -impl BlockData { - fn new(device: Arc, backend: Arc) -> Arc { - let acc_mem = device.accessor_mem(); - Arc::new(Self { - device, - backend, - acc_mem, - lock: Mutex::new(()), - cv: Condvar::new(), - }) + +struct QueueSlot { + state: Mutex, + workers: Mutex>>, + notify_count: AtomicUsize, + queue_id: QueueId, +} +impl QueueSlot { + fn new(queue_id: QueueId) -> Self { + Self { + state: Mutex::new(Default::default()), + workers: Mutex::new(None), + notify_count: AtomicUsize::new(0), + queue_id, + } } + fn request_notify(&self, hint: ReqCountHint) { + let existing = self.notify_count.load(Ordering::Acquire); + if existing < MAX_WORKERS.get() { + // wake everyone if we weren't given a hint + let count = MAX_WORKERS.min(hint.unwrap_or(MAX_WORKERS)); + let _ = self.notify_count.fetch_add(count.get(), Ordering::Release); + } + } + fn take_notifications(&self) -> Option<(NonZeroUsize, Bitmap)> { + let state = self.state.lock().unwrap(); + let minder = state.minder.as_ref()?; + + if self.notify_count.load(Ordering::Relaxed) == 0 { + return None; + } + let wake_wids = minder.take_notifications()?; - /// Notify any [blocked](Self::block_for_req()) tasks of a state change. - /// This could be a change to the device, to the backend, or simply new - /// request(s) available. - fn notify(&self) { - let _guard = self.lock.lock().unwrap(); - self.cv.notify_all(); + let pending = self.notify_count.swap(0, Ordering::Acquire); + let count = + NonZeroUsize::new(pending).expect("notify count is non-zero"); + Some((count, wake_wids)) + } + fn flush_notifications(&self) { + let guard = self.workers.lock().unwrap(); + let Some(workers) = guard.as_ref() else { + return; + }; + if let Some((count, wake_wids)) = self.take_notifications() { + workers.wake(wake_wids, Some(count), Some(self.queue_id)); + } } +} - fn next_req( - &self, - att_state: &attachment::AttachState, - ) -> Result { - check_state(att_state)?; - match self.device.next() { - Some(req) => Ok(req), - None => { - check_state(att_state)?; - Err(ReqError::NonePending) +#[derive(Default, Copy, Clone)] +struct QueueColState { + associated_qids: Versioned, + paused: bool, +} +impl QueueColState { + fn queue_associate(&mut self, qid: QueueId) -> Versioned { + self.associated_qids.update().set(qid.into()); + self.associated_qids + } + fn queue_dissociate(&mut self, qid: QueueId) -> Versioned { + self.associated_qids.update().unset(qid.into()); + self.associated_qids + } +} +struct QueueCollection { + queues: Vec, + state: Mutex, + pub devid: DeviceId, +} +impl QueueCollection { + fn new(max_queues: NonZeroUsize, devid: DeviceId) -> Arc { + let count = max_queues.get(); + assert!(count <= MAX_WORKERS.get()); + let queues = + (0..count).map(|n| QueueSlot::new(QueueId::from(n))).collect(); + + Arc::new(Self { queues, devid, state: Default::default() }) + } + fn attach(&self, workers: &Arc) { + for slot in self.queues.iter() { + let old = slot.workers.lock().unwrap().replace(workers.clone()); + assert!(old.is_none(), "workers ref should not have been attached"); + } + } + fn detach(&self) { + for slot in self.queues.iter() { + let old = slot.workers.lock().unwrap().take(); + assert!(old.is_some(), "workers ref should have been attached"); + } + } + fn slot(&self, queue_id: QueueId) -> &QueueSlot { + self.queues.get(usize::from(queue_id)).expect("queue id within range") + } + fn notify(&self, queue_id: QueueId, hint: ReqCountHint) { + let slot = self.slot(queue_id); + slot.request_notify(hint); + slot.flush_notifications(); + } + fn set_metric_consumer(&self, consumer: Arc) { + for queue in self.queues.iter() { + if let Some(minder) = queue.state.lock().unwrap().minder.as_mut() { + minder.set_metric_consumer(consumer.clone()); } } } + fn associated_qids(&self) -> Versioned { + self.state.lock().unwrap().associated_qids + } + fn pause(&self) { + let mut state = self.state.lock().unwrap(); + assert!(!state.paused); + + state.paused = true; + for slot in self.queues.iter() { + if let Some(minder) = slot.state.lock().unwrap().minder.as_ref() { + minder.pause(); + } + } + } + fn resume(&self) { + let mut state = self.state.lock().unwrap(); + assert!(state.paused); + + state.paused = false; + for slot in self.queues.iter() { + let state = slot.state.lock().unwrap(); + let Some(minder) = state.minder.as_ref() else { + continue; + }; + minder.resume(); + drop(state); + + slot.flush_notifications(); + } + } + fn none_processing(&self) -> NoneProcessing { + let minders = self + .queues + .iter() + .filter_map(|slot| { + let state = slot.state.lock().unwrap(); + state.minder.as_ref().map(Arc::clone) + }) + .collect::>(); + NoneProcessing { + minders: MinderRefs { values: minders, _pinned: PhantomPinned }, + unordered: FuturesUnordered::new(), + loaded: false, + } + } - fn block_for_req( + fn next_req( &self, - att_state: &attachment::AttachState, - ) -> Option { + queue_select: QueueId, + wid: WorkerId, + ) -> Option { + let idx: usize = queue_select.into(); + let slot = self.queues.get(idx)?; + + let guard = slot.state.lock().unwrap(); + let minder = guard.minder.as_ref()?; + let result = minder.next_req(wid); + + probes::block_poll!(|| { + ( + devq_id(self.devid, slot.queue_id), + wid as u64, /* fix */ + result.is_some() as u8, + ) + }); + result + } + + fn next_req_any( + &self, + cursor: &mut PollCursor, + wid: WorkerId, + ) -> Option { + let idx = usize::from(cursor.0 .0); + assert!(idx < self.queues.len()); + let (front, back) = self.queues.split_at(idx); + let queues = back.iter().chain(front.iter()); + + let (hit_qid, dreq) = queues + .filter_map(|slot| { + let guard = slot.state.lock().unwrap(); + let minder = guard.minder.as_ref()?; + let result = minder.next_req(wid); + + probes::block_poll!(|| { + ( + devq_id(self.devid, slot.queue_id), + wid as u64, /* fix */ + result.is_some() as u8, + ) + }); + + Some((slot.queue_id, result?)) + }) + .next()?; + + // Which slot should the caller start with next time? + cursor.0 = hit_qid.next(self.queues.len()); + + Some(dreq) + } +} + +struct MinderRefs { + values: Vec>, + _pinned: PhantomPinned, +} +pin_project! { + pub struct NoneProcessing { + #[pin] + minders: MinderRefs, + #[pin] + unordered: FuturesUnordered>, + loaded: bool, + } + impl PinnedDrop for NoneProcessing { + fn drop(this: Pin<&mut Self>) { + let mut this = this.project(); + + // Ensure that all references into `minders` held by NoneInFlight + // futures are dropped before the `minders` contents themselves. + this.unordered.clear(); + } + } +} +impl Future for NoneProcessing { + type Output = (); + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let mut this = self.project(); + if !*this.loaded { + for minder in this.minders.values.iter().map(Arc::as_ref) { + // # SAFETY + // + // With the Vec> pinned (and barred via marker + // from Unpin), it should not be possible to remove them for the + // lifetime of this future. With that promised to us, we can + // extend the lifetime of the QueueMinder references long enough + // to run the NoneInFlight futures. + // + // The contents of `minders` will remain pinned and untouched + // until NoneProcessing is dropped. At that point, any + // lingering references held by the FuturesUnordered will be + // explicitly released in PinnedDrop::drop(), ensuring they do + // not outlive MinderRefs. + let extended: &'static QueueMinder = + unsafe { std::mem::transmute(minder) }; + + this.unordered.push(extended.none_in_flight()); + } + *this.loaded = true; + } loop { - match self.next_req(att_state) { - Ok(req) => { - return Some(req); + match Stream::poll_next(this.unordered.as_mut(), cx) { + Poll::Ready(None) => { + return Poll::Ready(()); + } + Poll::Ready(Some(_)) => { + continue; + } + Poll::Pending => { + return Poll::Pending; } - Err(err) => match err { - ReqError::NonePending | ReqError::Paused => { - // Double-check for attachment-related "error" - // conditions under protection of the lock before - // finally blocking. - let guard = self.lock.lock().unwrap(); - if !att_state.is_attached() || att_state.is_stopped() { - return None; - } - - let _guard = self.cv.wait(guard).unwrap(); - continue; - } - _ => { - return None; - } - }, } } } } -fn check_state(state: &attachment::AttachState) -> Result<(), ReqError> { - if !state.is_attached() { - Err(ReqError::Detached) - } else if state.is_stopped() { - Err(ReqError::Stopped) - } else if state.is_paused() { - Err(ReqError::Paused) - } else { +/// A pair of weak references to inner state of a device and backend which are +/// attached to one another. +/// +/// Attachment and detachment requires taking locks in both the device and +/// backend, and such lock is done in that specific order to avoid deadlock. +#[derive(Clone)] +struct AttachPair { + dev_attach: Weak, + backend_attach: Weak, +} +impl AttachPair { + fn attach( + dev: &DeviceAttachment, + be: &BackendAttachment, + ) -> Result<(), AttachError> { + let mut dev_att_state = dev.0.att_state.lock().unwrap(); + let mut be_att_state = be.0.att_state.lock().unwrap(); + + if dev_att_state.is_some() { + return Err(AttachError::DeviceAttached); + } + if be_att_state.is_some() { + return Err(AttachError::BackendAttached); + } + + // TODO: name the accessor child? + let be_acc_mem = dev.0.acc_mem.child(None); + be.0.workers.attach(&be_acc_mem, &dev.0.queues); + dev.0.queues.attach(&be.0.workers); + + let shared = AttachPair { + dev_attach: Arc::downgrade(&dev.0), + backend_attach: Arc::downgrade(&be.0), + }; + *dev_att_state = Some(shared.clone()); + *be_att_state = Some((shared, be_acc_mem)); + + drop(dev_att_state); + drop(be_att_state); + + let dev_state = dev.0.dev_state.lock().unwrap(); + if let Some(on_attach) = dev_state.on_attach.as_ref() { + on_attach(be.info()) + } + Ok(()) } + + fn detach(self) { + let (Some(dev), Some(be)) = + (self.dev_attach.upgrade(), self.backend_attach.upgrade()) + else { + // If the drop handler has run for the device or backend, resulting + // in its Weak pointer being unable to upgrade, then a detach is + // already in progress, and we can let that run to completion. + return; + }; + + let mut dev_state = dev.att_state.lock().unwrap(); + let mut be_state = be.att_state.lock().unwrap(); + match (dev_state.as_ref(), be_state.as_ref()) { + (Some(ds), Some((bs, _))) if self.eq(ds) && self.eq(bs) => { + // Device and backend agree about mutual attachment + } + _ => return, + } + *dev_state = None; + *be_state = None; + + // TODO: ensure workers have no in-flight requests + be.workers.detach(); + dev.queues.detach(); + } +} +impl PartialEq for AttachPair { + fn eq(&self, other: &Self) -> bool { + self.dev_attach.ptr_eq(&other.dev_attach) + && self.backend_attach.ptr_eq(&other.backend_attach) + } +} + +pub type OnAttachFn = Box; + +#[derive(Default)] +struct DeviceState { + on_attach: Option, } -pub struct DeviceAttachment(attachment::FrontAttachment>); +struct DeviceAttachInner { + att_state: Mutex>, + dev_state: Mutex, + queues: Arc, + acc_mem: MemAccessor, +} + +/// Main "attachment point" for a block device. +pub struct DeviceAttachment(Arc); impl DeviceAttachment { - pub fn new() -> Self { - Self(attachment::FrontAttachment::new()) + /// Create a [DeviceAttachment] for a given device. The maximum number of + /// queues which the device will ever expose is set via `max_queues`. DMA + /// done by attached backend workers will be through the provided `acc_mem`. + pub fn new(max_queues: NonZeroUsize, acc_mem: MemAccessor) -> Self { + let devid = NEXT_DEVICE_ID.fetch_add(1, Ordering::Relaxed); + let queues = QueueCollection::new(max_queues, devid); + Self(Arc::new(DeviceAttachInner { + att_state: Mutex::new(None), + dev_state: Mutex::new(DeviceState::default()), + queues, + acc_mem, + })) } - /// Query [DeviceInfo] from associated backend (if attached) - pub fn info(&self) -> Option { - self.0.access(|data, _state| data.backend.info()) + /// If a backend is attached to this device, notify it that the queue + /// associations for this device have changed. + fn queues_update_assoc(&self, queues_associated: Versioned) { + let guard = self.0.att_state.lock().unwrap(); + if let Some(att_state) = guard.as_ref() { + if let Some(backend) = Weak::upgrade(&att_state.backend_attach) { + drop(guard); + backend.workers.update_queue_associations(queues_associated); + } + } } - /// Set cache mode on associated backend + /// Associate a [DeviceQueue] with this device. /// - /// # Warning + /// Once associated, any attached backend will process requests emitted from + /// that queue. /// - /// This is currently unimplemented! - pub fn set_cache_mode(&self, _mode: CacheMode) { - todo!("wire up cache mode toggling") - } + /// # Panics + /// + /// If `queue_id` is >= the max queues specified for this device, or if + /// an existing queue is associated with that ID. + pub fn queue_associate( + &self, + queue_id: QueueId, + queue: Arc, + ) { + let minder = QueueMinder::new(queue, self.0.queues.devid, queue_id); - /// Notify attached backend of (new) pending requests - pub fn notify(&self) { - self.0.access(|data, state| { - if !state.is_paused() { - state.notify(); - data.notify(); - } - }); + let mut state = self.0.queues.state.lock().unwrap(); + let slot = self.0.queues.slot(queue_id); + let mut slot_state = slot.state.lock().unwrap(); + assert!( + slot_state.minder.is_none(), + "queue slot should not be occupied" + ); + + if state.paused { + // Propagate any pause state of the device into any newly + // associating queues while in such a pause. + minder.pause(); + } + slot_state.minder = Some(minder); + drop(slot_state); + + let associated = state.queue_associate(queue_id); + drop(state); + + self.queues_update_assoc(associated); } - /// Pause request processing for this device. + /// Dissociate a [DeviceQueue] from this device + /// + /// After dissociation, any attached backend will cease processing requests + /// from that queue. /// - /// Backend (if attached) will not be able to retrieving any requests from - /// this device while paused. The completions for any requests in flight, - /// however, will be able to flow through. + /// # Panics + /// + /// if `queue_id` is >= the max queues specified for this device, or if + /// there is not queue associated with that ID. + pub fn queue_dissociate(&self, queue_id: QueueId) { + let mut state = self.0.queues.state.lock().unwrap(); + let slot = self.0.queues.slot(queue_id); + let mut slot_state = slot.state.lock().unwrap(); + + let _minder = + slot_state.minder.take().expect("queue slot should be occupied"); + drop(slot_state); + + let associated = state.queue_dissociate(queue_id); + drop(state); + + self.queues_update_assoc(associated); + } + + /// Notify attached backend (if any) that `queue_id` may have new IO + /// requests to process. If the number of available requests is known, it + /// can be communicated via `hint` in order to optimize worker waking. + pub fn notify(&self, queue_id: QueueId, hint: ReqCountHint) { + self.0.queues.notify(queue_id, hint); + } + + pub fn device_id(&self) -> DeviceId { + self.0.queues.devid + } + /// Get the maximum queues configured for this device. + pub fn max_queues(&self) -> NonZeroUsize { + NonZeroUsize::new(self.0.queues.queues.len()) + .expect("non-zero queue count") + } + /// Get the [DeviceInfo] of the attached backend (if any) + pub fn info(&self) -> Option { + let state = self.0.att_state.lock().unwrap(); + let backend = Weak::upgrade(&state.as_ref()?.backend_attach)?; + Some(backend.info) + } + + /// Detach the device from attached backend (if any) + pub fn detach(&self) { + let guard = self.0.att_state.lock().unwrap(); + if let Some(att_state) = guard.as_ref().map(Clone::clone) { + drop(guard); + att_state.detach(); + } + } + + /// Pause the device, preventing workers from an attached backend (if any) + /// from fetching new IO requests to process. Outstanding requests will + /// proceed as normal. pub fn pause(&self) { - self.0.access_fallback( - |data, state| { - state.pause(); - data.notify(); - }, - |state| state.pause(), - ); + self.0.queues.pause(); } - /// Clear the paused state on this device, allowing the backend (if - /// attached) to retrieve requests once again. + /// Resume the device, allowing workers from an attached backend (if any) to + /// once again fetch new IO requests to process. pub fn resume(&self) { - self.0.access_fallback( - |data, state| { - state.resume(); - data.notify(); - }, - |state| state.resume(), - ); + self.0.queues.resume(); } - /// Attempt to detach device from backend - pub fn detach(&self) -> Result<(), attachment::DetachError> { - if let Some((dev, backend)) = self - .0 - .access(|data, _state| (data.device.clone(), data.backend.clone())) - { - detach(dev, backend) + /// Emit a [Future] which will resolve when there are no request being + /// actively processed by an attached backend. + pub fn none_processing(&self) -> NoneProcessing { + self.0.queues.none_processing() + } + + /// Set the [MetricConsumer] to be informed of all request completions + /// processed by this device. + pub fn set_metric_consumer(&self, consumer: Arc) { + self.0.queues.set_metric_consumer(consumer); + } + + /// Register a function to be called when this device becomes attached to a + /// backend. Intended for tasks such as querying the [DeviceInfo] for + /// presentation to the guest. + pub fn on_attach(&self, cb: OnAttachFn) { + self.0.dev_state.lock().unwrap().on_attach = Some(cb); + } +} +impl Drop for DeviceAttachment { + fn drop(&mut self) { + self.detach(); + } +} + +#[derive(Copy, Clone, Default)] +struct PollCursor(QueueId); +impl PollCursor { + /// Suggest that the worker using this cursor poll a specific queue next + fn suggest(&mut self, queue_id: QueueId) { + self.0 = queue_id; + } +} + +#[derive(Default)] +struct WorkerState { + /// Has the worker associated with this slot indicated that it is active? + active_type: Option, + /// Has the absence of work caused this worker to sleep? + sleeping_on: Option, + + assign_strat: Versioned, + assign_poll: PollAssignment, + + cursor: PollCursor, + + queues: Option>, +} + +pub(crate) struct WorkerSlot { + state: Mutex, + acc_mem: MemAccessor, + cv: Condvar, + notify: Notify, + id: WorkerId, +} +impl WorkerSlot { + fn new(id: WorkerId) -> Self { + Self { + state: Mutex::new(Default::default()), + acc_mem: MemAccessor::new_orphan(), + cv: Condvar::new(), + notify: Notify::new(), + id, + } + } + fn block_for_req(&self) -> Option { + let mut state = self.state.lock().unwrap(); + assert!(state.active_type.is_some()); + + loop { + let devid = match self.next_req(&mut state) { + PollResult::Ok(device_request) => { + return Some(device_request); + } + PollResult::Detached | PollResult::Halted => { + return None; + } + PollResult::WaitFor(devid) => devid, + }; + + state.sleeping_on = Some(devid); + probes::block_sleep!(|| { (devid, self.id as u64) }); + state = self.cv.wait(state).unwrap(); + probes::block_wake!(|| { (devid, self.id as u64) }); + state.sleeping_on = None; + } + } + + fn next_req(&self, state: &mut MutexGuard) -> PollResult { + assert!(state.active_type.is_some()); + + let Some(queues) = state.queues.as_ref() else { + return PollResult::Detached; + }; + let devid = queues.devid; + let result = match state.assign_poll { + PollAssignment::Halt => { + return PollResult::Halted; + } + PollAssignment::Idle => None, + PollAssignment::Fixed(queue_id) => { + queues.next_req(queue_id, self.id) + } + PollAssignment::Any => { + // Copy cursor since split borrows confuses borrowck + let mut cursor = state.cursor; + let result = queues.next_req_any(&mut cursor, self.id); + state.cursor = cursor; + result + } + }; + match result { + Some(req) => PollResult::Ok(req), + None => PollResult::WaitFor(devid), + } + } + + fn async_start_sleep( + &self, + mut state: MutexGuard, + devid: DeviceId, + ) { + state.sleeping_on = Some(devid); + probes::block_sleep!(|| { (devid, self.id as u64) }); + } + + fn async_stop_sleep(&self) { + let mut state = self.state.lock().unwrap(); + if let Some(devid) = state.sleeping_on.take() { + probes::block_wake!(|| { (devid, self.id as u64) }); + } + } + + fn wait_for_req(&self) -> WaitForReq<'_> { + WaitForReq::new(self) + } + + fn update_assignment(&self, assign: &Assignment) { + let mut state = self.state.lock().unwrap(); + if state.assign_strat.newer_than(&assign.strategy) { + // We already have a newer assignment + return; + } + state.assign_strat = assign.strategy; + if assign.should_halt { + state.assign_poll = PollAssignment::Halt; } else { - Err(attachment::DetachError::FrontNotAttached) + state.assign_poll = + if let Some(poll_assign) = assign.poll_assignments.as_ref() { + *poll_assign.get(&self.id).unwrap_or(&PollAssignment::Any) + } else { + PollAssignment::Idle + }; } + self.wake(Some(state), None); + } + + fn wake( + &self, + state: Option>, + qid_hint: Option, + ) -> bool { + let mut state = state.unwrap_or_else(|| self.state.lock().unwrap()); + if let Some(wtype) = state.active_type { + if state.sleeping_on.is_some() { + if let Some(qid) = qid_hint { + state.cursor.suggest(qid); + } + match wtype { + WorkerType::Sync => self.cv.notify_one(), + WorkerType::Async => self.notify.notify_one(), + } + return true; + } + } + + false } } -/// Reason why next request is unavailable from associated device -pub enum ReqError { - /// No request is pending from the device - NonePending, - /// Processing of requests from the device is paused - Paused, - /// Backend is not attached to any device - Detached, - /// Backend is stopping workers - Stopped, +/// Device queue worker is assigned to poll +#[derive(Clone, Copy, Default)] +enum PollAssignment { + /// End polling immediately since backend is halted + Halt, + /// Poll no queue(s) as worker is in idle state + Idle, + /// Fixed queue specified by [QueueId] + Fixed(QueueId), + /// Poll any queue(s) + #[default] + Any, } -pub struct BackendAttachment(attachment::BackAttachment>); -impl BackendAttachment { - pub fn new() -> Self { - Self(attachment::BackAttachment::new()) +#[derive(Default)] +struct WorkerColState { + backend_running: bool, + + strategy: Versioned, + + workers_active: Bitmap, + + associated_qids: Versioned, + + device_id: Option, +} +impl WorkerColState { + fn set_worker_state(&mut self, wid: WorkerId, is_active: bool) { + if is_active { + self.workers_active.set(wid); + } else { + self.workers_active.unset(wid); + } } + /// Based on active workers and queues, pick a suitable dispatch strategy + /// and perform any worker->queue assignments (if applicable to the newly + /// selected strategy). + fn generate_assignments(&mut self) -> Assignment { + // Pick a (potentially) new strategy in the face of updated state + self.strategy.replace(if self.backend_running { + Strategy::choose( + self.workers_active.count(), + self.associated_qids.get().count(), + ) + } else { + Strategy::Idle + }); - /// Attempt to retrieve the next [Request] from the attached (if any) - /// device. - /// - /// Will return an error if: - /// - /// - No device is attached - /// - The device is paused - /// - The backend is stopped - /// - No requests are queued in the device - pub fn next_req(&self) -> Result { - match self.0.access(|data, state| data.next_req(state)) { - None => Err(ReqError::Detached), - Some(Err(e)) => Err(e), - Some(Ok(r)) => Ok(r), + if !self.backend_running { + return Assignment { + strategy: self.strategy, + poll_assignments: None, + should_halt: true, + }; + } + let poll_assignments = match self.strategy.get() { + Strategy::Idle => None, + Strategy::Single => { + assert_eq!(self.associated_qids.get().count(), 1); + let single_queue: QueueId = + self.associated_qids.get().iter().next().unwrap().into(); + + Some( + self.workers_active + .iter() + .map(|wid| (wid, PollAssignment::Fixed(single_queue))) + .collect(), + ) + } + Strategy::Static => { + let worker_count = self.workers_active.count(); + let queue_count = self.associated_qids.get().count(); + assert!( + worker_count >= queue_count, + "workers should >= queues when {:?} is chosen", + self.strategy.get() + ); + let per_queue = worker_count / queue_count; + let mut queue_loop = self.associated_qids.get().looping_iter(); + + let mut workers = self.workers_active.iter(); + + let mut assigned: BTreeMap = workers + .by_ref() + .take(per_queue * queue_count) + .map(|wid| { + ( + wid, + PollAssignment::Fixed(queue_loop.next().expect( + "looping queue iter should emit results", + ).into()), + ) + }) + .collect(); + // Remaining workers will be idled + assigned.extend(workers.map(|wid| (wid, PollAssignment::Idle))); + + Some(assigned) + } + Strategy::FreeForAll => Some( + self.workers_active + .iter() + .map(|wid| (wid, PollAssignment::Any)) + .collect(), + ), + }; + Assignment { + strategy: self.strategy, + poll_assignments, + should_halt: false, + } + } +} + +#[derive(Default, Copy, Clone, PartialEq, Eq, Debug, IntoStaticStr)] +pub enum Strategy { + /// An explicitly stopped backend or lack of workers and/or queues means + /// there is no dispatching to do + #[default] + Idle, + + /// All workers servicing single queue + Single, + + /// Workers are statically assigned to queues in an even distribution. + Static, + + /// Workers will round-robin through all queues, attempting to pick up + /// requests from any they can. + FreeForAll, +} +impl Strategy { + pub fn choose(worker_count: usize, queue_count: usize) -> Self { + if worker_count == 0 || queue_count == 0 { + return Strategy::Idle; + } + if queue_count == 1 { + return Strategy::Single; + } + if worker_count >= queue_count { + return Strategy::Static; } + // Unfortunate, but better than leaving requests to linger in a queue + // which lacks any assigned workers + Strategy::FreeForAll + } +} + +struct Assignment { + strategy: Versioned, + poll_assignments: Option>, + should_halt: bool, +} + +pub(crate) struct WorkerCollection { + workers: Vec, + state: Mutex, +} +impl WorkerCollection { + fn new(max_workers: NonZeroUsize) -> Arc { + let max_workers = max_workers.get(); + assert!(max_workers <= MAX_WORKERS.get()); + let workers: Vec<_> = (0..max_workers) + .map(|id| WorkerSlot::new(WorkerId::from(id))) + .collect(); + Arc::new(Self { workers, state: Default::default() }) } + fn set_active(&self, id: WorkerId, new_type: Option) -> bool { + if let Some(slot) = self.workers.get(id) { + let refresh_guard = { + let mut wstate = slot.state.lock().unwrap(); + if wstate.active_type.is_some() != new_type.is_some() { + let mut cstate = self.state.lock().unwrap(); + cstate.set_worker_state(id, new_type.is_some()); + wstate.active_type = new_type; + Some(cstate) + } else { + None + } + }; - /// Block (synchronously) in order to retrieve the next [Request] from the - /// device. Will return [None] if no device is attached, or the backend - /// is stopped, otherwise it will block until a request is available. - pub fn block_for_req(&self) -> Option { - match self.0.access(|data, state| { - // If we do not immediately get a request, clone the BackendData - // so we can do our blocking through that, instead of repeated calls - // into BackAttachment::access() - data.next_req(state).map_err(|_e| (data.clone(), state.clone())) - })? { - Ok(req) => Some(req), - Err((be, state)) => be.block_for_req(&state), + if let Some(guard) = refresh_guard { + self.assignments_refresh(guard); + return true; + } } + false } + fn assignments_refresh(&self, mut state: MutexGuard) { + let assign = state.generate_assignments(); + let devid = state.device_id.unwrap_or(u32::MAX); + drop(state); - /// Get a [Waiter], through which one can asynchronously poll for new - /// requests via [`Waiter::for_req()`]. Will return [None] if no device - /// is attached, or the backend is stopped. - pub fn waiter(&self) -> Option { - self.0.access(|data, state| Waiter(data.clone(), state.clone())) + super::probes::block_strategy!(|| { + let assign_name: &'static str = assign.strategy.get().into(); + let generation = assign.strategy.generation() as u64; + (devid, assign_name, generation) + }); + for slot in self.workers.iter() { + slot.update_assignment(&assign); + } + } + fn slot(&self, id: WorkerId) -> &WorkerSlot { + self.workers.get(id).expect("valid worker id for slot") } + fn attach(&self, parent_mem: &MemAccessor, queues: &Arc) { + for (idx, slot) in self.workers.iter().enumerate() { + parent_mem.adopt(&slot.acc_mem, Some(format!("worker-{idx}"))); + let mut state = slot.state.lock().unwrap(); + let old = state.queues.replace(queues.clone()); + assert!(old.is_none(), "worker slot not already attached"); + } - /// Run provided function against [MemAccessor] for this backend. - /// - /// Intended to provide caller with means of creating/associated child - /// accessors. - pub fn accessor_mem( + let mut state = self.state.lock().unwrap(); + state.device_id = Some(queues.devid); + state.associated_qids = queues.associated_qids(); + } + fn detach(&self) { + for slot in self.workers.iter() { + let mut state = slot.state.lock().unwrap(); + let old = state.queues.take(); + assert!(old.is_some(), "worker slot should have been attached"); + } + let mut state = self.state.lock().unwrap(); + state.strategy.replace(Strategy::Idle); + // With no device attached, the queues information should be cleared + state.associated_qids = Versioned::default(); + state.device_id = None; + } + fn wake( &self, - f: impl FnOnce(&MemAccessor) -> R, - ) -> Option { - self.0.access(|data, _state| f(&data.acc_mem)) + wake_wids: Bitmap, + limit: Option, + qid_hint: Option, + ) { + let _num_woke = wake_wids + .iter() + .take(limit.unwrap_or(MAX_WORKERS).get()) + .filter_map(|wid| { + let slot = self.workers.get(wid)?; + slot.wake(None, qid_hint).then_some(()) + }) + .count(); } + fn update_queue_associations(&self, queues_associated: Versioned) { + let mut state = self.state.lock().unwrap(); + state.associated_qids.replace_if_newer(&queues_associated); + self.assignments_refresh(state); + } + fn start(&self) { + let mut state = self.state.lock().unwrap(); + state.backend_running = true; + self.assignments_refresh(state); + } + fn stop(&self) { + let mut state = self.state.lock().unwrap(); + state.backend_running = false; + self.assignments_refresh(state); + } +} - /// Assert stopped state on this Attachment - pub fn stop(&self) { - self.0.access_fallback( - |data, state| { - state.stop(); - data.notify(); - }, - |state| state.stop(), +#[derive(Copy, Clone)] +pub enum WorkerType { + Sync, + Async, +} + +pub struct InactiveWorkerCtx { + workers: Arc, + id: WorkerId, +} +impl InactiveWorkerCtx { + /// Activate this worker for synchronous operation. + /// + /// Returns [None] if there is already an active worker in the slot + /// associated with this [WorkerId]. + pub fn activate_sync(self) -> Option { + if self.workers.set_active(self.id, Some(WorkerType::Sync)) { + Some(SyncWorkerCtx(self.into())) + } else { + None + } + } + + /// Activate this worker for asynchronous operation. + /// + /// Returns [None] if there is already an active worker in the slot + /// associated with this [WorkerId]. + pub fn activate_async(self) -> Option { + if self.workers.set_active(self.id, Some(WorkerType::Async)) { + Some(AsyncWorkerCtx(self.into())) + } else { + None + } + } +} + +/// Worker context for synchronous (blocking) request processing. +/// +/// Note: When the context is dropped, the slot for this [WorkerId] will become +/// vacant, and available to be activated again. +pub struct SyncWorkerCtx(WorkerCtxInner); +impl SyncWorkerCtx { + /// Block (synchronously) in order to retrieve the next + /// [request](DeviceRequest) from the device. Will return [None] if no + /// device is attached, or the backend is stopped, otherwise it will block + /// until a request is available. + pub fn block_for_req(&self) -> Option { + self.0.workers.slot(self.0.id).block_for_req() + } + /// Get the [MemAccessor] required to do DMA for request processing + pub fn acc_mem(&self) -> &MemAccessor { + self.0.acc_mem() + } +} + +/// Worker context for asynchronous request processing +/// +/// Note: When the context is dropped, the slot for this [WorkerId] will become +/// vacant, and available to be activated again. +pub struct AsyncWorkerCtx(WorkerCtxInner); +impl AsyncWorkerCtx { + /// Get a [Future] which will wait for a [request](DeviceRequest) to be made + /// available from an attached device. + pub fn wait_for_req(&self) -> WaitForReq<'_> { + self.0.workers.slot(self.0.id).wait_for_req() + } + /// Get the [MemAccessor] required to do DMA for request processing + pub fn acc_mem(&self) -> &MemAccessor { + self.0.acc_mem() + } +} + +struct WorkerCtxInner { + workers: Arc, + id: WorkerId, +} +impl From for WorkerCtxInner { + fn from(value: InactiveWorkerCtx) -> Self { + let InactiveWorkerCtx { workers, id } = value; + WorkerCtxInner { workers, id } + } +} +impl WorkerCtxInner { + fn acc_mem(&self) -> &MemAccessor { + &self.workers.slot(self.id).acc_mem + } +} +impl Drop for WorkerCtxInner { + /// Deactivate the worker when it is dropped + fn drop(&mut self) { + assert!( + self.workers.set_active(self.id, None), + "active worker is valid during deactivation" ); } +} + +struct BackendAttachInner { + att_state: Mutex>, + workers: Arc, + info: DeviceInfo, +} + +/// Main "attachment point" for a block backend. +pub struct BackendAttachment(Arc); +impl BackendAttachment { + pub fn new(max_workers: NonZeroUsize, info: DeviceInfo) -> Self { + Self(Arc::new(BackendAttachInner { + att_state: Mutex::new(None), + workers: WorkerCollection::new(max_workers), + info, + })) + } + /// Get an (inactive) [context](InactiveWorkerCtx) for a given [WorkerId]. + pub fn worker(&self, id: WorkerId) -> InactiveWorkerCtx { + assert!(id < self.0.workers.workers.len()); + InactiveWorkerCtx { workers: self.0.workers.clone(), id } + } - /// Clear stopped state from this Attachment + pub fn max_workers(&self) -> NonZeroUsize { + NonZeroUsize::new(self.0.workers.workers.len()) + .expect("WorkerCollection correctly initialized") + } + + pub fn info(&self) -> DeviceInfo { + self.0.info + } + + /// Permit workers to pull requests from the attached device (if any) for + /// processing. pub fn start(&self) { - self.0.access_fallback( - |data, state| { - state.start(); - data.notify(); - }, - |state| state.start(), - ); + self.0.workers.start() } - /// Attempt to detach backend from device - pub fn detach(&self) -> Result<(), attachment::DetachError> { - if let Some((dev, backend)) = self - .0 - .access(|data, _state| (data.device.clone(), data.backend.clone())) + /// Remove access to pull requests from the attached device (if any) from + /// workers, causing them to halt processing once they have completed any + /// in-flight work. + pub fn stop(&self) { + self.0.workers.stop() + } + + /// Detach this backend from the device (if any) + pub fn detach(&self) { + let guard = self.0.att_state.lock().unwrap(); + if let Some(att_state) = + guard.as_ref().map(|(att_state, _)| att_state.clone()) { - detach(dev, backend) - } else { - Err(attachment::DetachError::BackNotAttached) + drop(guard); + att_state.detach(); } } } - -pub struct Waiter(Arc, attachment::AttachState); -impl Waiter { - /// Wait (via a [Future]) to retrieve the next [Request] from the device. - pub fn for_req(&self) -> WaitForReq<'_> { - WaitForReq { be: &self.0, att_state: &self.1, wait: self.1.notified() } +impl Drop for BackendAttachment { + fn drop(&mut self) { + self.detach() } } +/// Attach a [device](DeviceAttachment) to a [backend](BackendAttachment). +pub fn attach( + device: &DeviceAttachment, + backend: &BackendAttachment, +) -> Result<(), AttachError> { + AttachPair::attach(device, backend) +} + pin_project! { /// [Future] returned from [`Waiter::for_req()`] pub struct WaitForReq<'a> { - be: &'a BlockData, - att_state: &'a attachment::AttachState, + slot: &'a WorkerSlot, + sleeping_on: Option, #[pin] wait: Notified<'a> } + + impl PinnedDrop for WaitForReq<'_> { + fn drop(this: Pin<&mut Self>) { + let this = this.project(); + if let Some(_) = this.sleeping_on.take() { + this.slot.async_stop_sleep(); + } + } + } +} + +impl WaitForReq<'_> { + fn new<'a>(slot: &'a WorkerSlot) -> WaitForReq<'a> { + let wait = slot.notify.notified(); + WaitForReq { slot, sleeping_on: None, wait } + } } -impl Future for WaitForReq<'_> { - type Output = Option; +impl Future for WaitForReq<'_> { + type Output = Option; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let mut this = self.project(); + + if let Some(_) = this.sleeping_on.take() { + this.slot.async_stop_sleep(); + } + loop { - match this.be.next_req(this.att_state) { - Ok(req) => return Poll::Ready(Some(req)), - Err(ReqError::NonePending) | Err(ReqError::Paused) => { + let mut state = this.slot.state.lock().unwrap(); + match this.slot.next_req(&mut state) { + PollResult::Ok(dreq) => { + return Poll::Ready(Some(dreq)); + } + PollResult::WaitFor(devid) => { + // Record that this worker is going to sleep + *this.sleeping_on = Some(devid); + this.slot.async_start_sleep(state, devid); + if let Poll::Ready(_) = Notified::poll(this.wait.as_mut(), cx) { // The `Notified` future is fused, so we must "refresh" // prior to any subsequent attempts to poll it after it // emits `Ready` - this.wait.set(this.att_state.notified()); + this.wait.set(this.slot.notify.notified()); // Take another lap if woken by the notifier to check // for a pending request @@ -311,8 +1221,7 @@ impl Future for WaitForReq<'_> { } return Poll::Pending; } - Err(_) => { - // Let the consumer know that they should bail + PollResult::Detached | PollResult::Halted => { return Poll::Ready(None); } } @@ -320,40 +1229,136 @@ impl Future for WaitForReq<'_> { } } -/// Attempt to attach a block [Device] to a block [Backend]. -pub fn attach( - device: Arc, - backend: Arc, -) -> Result<(), attachment::AttachError> { - attachment::attach( - BlockData::new(device.clone(), backend.clone()), - &device.attachment().0, - &backend.attachment().0, - |data| { - data.device.on_attach(data.backend.info()); - }, - ) -} - -fn detach( - device: Arc, - backend: Arc, -) -> Result<(), attachment::DetachError> { - attachment::detach( - &device.attachment().0, - &backend.attachment().0, - |data, state| { - if !state.is_stopped() { - // Demand that backend be stopped before detaching. - // - // Assuming it is otherwise well behaved, such a check should - // ensure that it is not holding on any unprocessed requests - // from the device. - Err(()) - } else { - data.notify(); - Ok(()) - } - }, - ) +enum PollResult { + /// Accepted request from device queue + Ok(DeviceRequest), + /// Worker has been idled, likely due to empty device queue(s) + WaitFor(DeviceId), + /// Backend is not attached to any device + Detached, + /// Backend is halting workers + Halted, +} + +#[derive(Error, Debug)] +pub enum AttachError { + #[error("backend already attached")] + BackendAttached, + #[error("device already attached")] + DeviceAttached, +} + +/// Resource versioned with a generation number +#[derive(Copy, Clone)] +struct Versioned { + generation: usize, + item: T, +} +impl Versioned { + fn new(item: T) -> Self { + Self { generation: 0, item } + } + /// Is this resource newer than `compare`? + fn newer_than(&self, compare: &Self) -> bool { + self.generation > compare.generation + } + /// Get mutable reference to resource while incrementing its generation + fn update(&mut self) -> &mut T { + self.generation += 1; + &mut self.item + } + /// Replace contained resource and increment the generation + fn replace(&mut self, item: T) { + *self.update() = item; + } + fn replace_if_newer(&mut self, compare: &Self) { + if compare.newer_than(self) { + *self = *compare; + } + } + fn get(&self) -> T { + self.item + } + fn generation(&self) -> usize { + self.generation + } +} +impl Default for Versioned { + fn default() -> Self { + Self::new(T::default()) + } +} + +/// Simple bitmap which facilitates iterator over bits which are asserted +#[derive(Copy, Clone, Default)] +pub(crate) struct Bitmap(u64); +impl Bitmap { + const TOP_BIT: usize = u64::BITS as usize; + + pub const ALL: Self = Self(u64::MAX); + + pub fn set(&mut self, idx: usize) { + assert!(idx < Self::TOP_BIT); + self.0 |= 1u64 << idx; + } + pub fn unset(&mut self, idx: usize) { + assert!(idx < Self::TOP_BIT); + self.0 &= !(1u64 << idx); + } + pub fn lowest_set(&self) -> Option { + if self.0.count_ones() == 0 { + None + } else { + Some(self.0.trailing_zeros() as usize) + } + } + pub fn count(&self) -> usize { + self.0.count_ones() as usize + } + pub fn is_empty(&self) -> bool { + self.count() == 0 + } + pub fn take(&mut self) -> Self { + Self(std::mem::replace(&mut self.0, 0)) + } + /// Get iterator which emits indices of bits which are set in this map. + pub fn iter(&self) -> BitIter { + BitIter(*self) + } + /// Get iterator which emits indices of bits which are set in this map. + /// It will infinitely loop back to the first bit whenever the last bit is + /// reached. + pub fn looping_iter(&self) -> LoopIter { + LoopIter { orig: *self, cur: *self } + } +} + +pub struct BitIter(Bitmap); +impl Iterator for BitIter { + type Item = usize; + + fn next(&mut self) -> Option { + let idx = self.0.lowest_set()?; + self.0.unset(idx); + Some(idx) + } +} +pub struct LoopIter { + cur: Bitmap, + orig: Bitmap, +} +impl Iterator for LoopIter { + type Item = usize; + + fn next(&mut self) -> Option { + if self.orig.count() == 0 { + return None; + } + if self.cur.count() == 0 { + self.cur = self.orig; + } + let idx = self.cur.lowest_set().unwrap(); + self.cur.unset(idx); + Some(idx) + } } diff --git a/lib/propolis/src/block/backend.rs b/lib/propolis/src/block/backend.rs deleted file mode 100644 index 6f861975a..000000000 --- a/lib/propolis/src/block/backend.rs +++ /dev/null @@ -1,232 +0,0 @@ -// This Source Code Form is subject to the terms of the Mozilla Public -// License, v. 2.0. If a copy of the MPL was not distributed with this -// file, You can obtain one at https://mozilla.org/MPL/2.0/. - -//! Mechanisms required to implement a block backend - -use std::future::Future; -use std::pin::Pin; -use std::sync::{Arc, Condvar, Mutex, Weak}; -use std::task::{Context, Poll}; - -use crate::accessors::MemAccessor; -use crate::block::{device, Device, Request}; - -use pin_project_lite::pin_project; -use tokio::sync::{futures::Notified, Notify}; - -/// Reason why next request is unavailable from associated device -pub enum ReqError { - /// No request is pending from the device - NonePending, - /// Processing of requests from the device is paused - Paused, - /// Backend is not attached to any device - Detached, - /// Backend is halting workers - Halted, -} - -pub(super) struct AttachState { - sibling: Weak>>, - device: Arc, - acc_mem: MemAccessor, - dev_is_paused: bool, - backend_is_halted: bool, -} -impl AttachState { - fn next_req(&self) -> Result { - if self.backend_is_halted { - // The backend being halted is the most pressing status to consider, - // so it must be checked first - Err(ReqError::Halted) - } else if self.dev_is_paused { - // Do not allow the backend to pull any requests while the device is - // in the paused state - Err(ReqError::Paused) - } else { - self.device.next().ok_or(ReqError::NonePending) - } - } - pub(super) fn new( - dev_attach: &device::Attachment, - device: &Arc, - ) -> Self { - Self { - sibling: Arc::downgrade(&dev_attach.0), - device: device.clone(), - acc_mem: device.accessor_mem(), - dev_is_paused: false, - backend_is_halted: false, - } - } - pub(super) fn set_paused(&mut self, is_paused: bool) { - self.dev_is_paused = is_paused; - } - pub(super) fn same_as_sibling( - &self, - other: &Arc>>, - ) -> bool { - self.sibling.ptr_eq(&Arc::downgrade(other)) - } -} -pub(super) struct AttachInner { - pub(super) state: Mutex>, - req_notifier: Notify, - cv: Condvar, -} -impl AttachInner { - fn new() -> Self { - Self { - state: Mutex::new(None), - req_notifier: Notify::new(), - cv: Condvar::new(), - } - } -} - -/// State held by the backend about the attached (if any) device -pub struct Attachment(pub(super) Arc); -impl Attachment { - pub fn new() -> Self { - Attachment(Arc::new(AttachInner::new())) - } - - /// Attempt to retrieve the next [`Request`] from the attached (if any) - /// device. - /// - /// Will return an error if: - /// - /// - No device is attached - /// - The device is paused - /// - The backend is halted - /// - No requests are queued in the device - pub fn next_req(&self) -> Result { - let guard = self.0.state.lock().unwrap(); - let inner = guard.as_ref().ok_or(ReqError::Detached)?; - inner.next_req() - } - - /// Block (synchronously) in order to retrieve the next [`Request`] from the - /// device. Will return [`None`] if no device is attached, or the backend - /// is halted, otherwise it will block until a request is available. - pub fn block_for_req(&self) -> Option { - let mut guard = self.0.state.lock().unwrap(); - loop { - // bail if not attached - let inner = guard.as_ref()?; - if inner.backend_is_halted { - return None; - } - - if let Ok(req) = inner.next_req() { - return Some(req); - } - - guard = self.0.cv.wait(guard).unwrap(); - } - } - - /// Wait (via a [`Future`]) to retrieve the next [`Request`] from the - /// device. Will return [`None`] if no device is attached, or the backend - /// is halted. - pub fn wait_for_req(&self) -> WaitForReq { - WaitForReq { attachment: self, wait: self.0.req_notifier.notified() } - } - - /// Run provided function against [`MemAccessor`] for this backend. - /// - /// Intended to provide caller with means of creating/associated child - /// accessors. - pub fn accessor_mem( - &self, - f: impl FnOnce(Option<&MemAccessor>) -> R, - ) -> R { - match self.0.state.lock().unwrap().as_ref() { - Some(inner) => f(Some(&inner.acc_mem)), - None => f(None), - } - } - - /// Assert halted state on Attachment - pub fn halt(&self) { - if let Some(state) = self.0.state.lock().unwrap().as_mut() { - state.backend_is_halted = true; - } - self.notify(); - } - - /// Clear halted state from Attachment - pub fn start(&self) { - if let Some(state) = self.0.state.lock().unwrap().as_mut() { - state.backend_is_halted = false; - } - self.notify(); - } - - /// Detach from the associated (if any) device. - pub fn detach(&self) -> Option<()> { - // lock ordering demands we approach this from the device side - let be_lock = self.0.state.lock().unwrap(); - let be_inner = be_lock.as_ref()?; - let dev_inner = be_inner.sibling.upgrade()?; - drop(be_lock); - - device::AttachInner::detach(&dev_inner) - } - - /// Notify any [blocked](Self::block_for_req()) or - /// [waiting](Self::wait_for_req()) tasks of a state change. This could be - /// a change to the device, to the backend, or simply new request(s) - /// available. - pub fn notify(&self) { - // TODO: avoid thundering herd? - self.0.req_notifier.notify_waiters(); - let _guard = self.0.state.lock().unwrap(); - self.0.cv.notify_all(); - } -} - -pin_project! { - /// Future returned from [`Attachment::wait_for_req()`] - /// - /// This future is not fused, so it can be repeatedly polled for additional - /// [`Request`]s as they become available. - pub struct WaitForReq<'a> { - attachment: &'a Attachment, - #[pin] - wait: Notified<'a>, - } -} -impl Future for WaitForReq<'_> { - type Output = Option; - - fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let mut this = self.project(); - loop { - match this.attachment.next_req() { - Ok(req) => return Poll::Ready(Some(req)), - Err(ReqError::Detached) | Err(ReqError::Halted) => { - // Let the consumer know that they should bail - return Poll::Ready(None); - } - Err(ReqError::NonePending) | Err(ReqError::Paused) => { - if let Poll::Ready(_) = - Notified::poll(this.wait.as_mut(), cx) - { - // The `Notified` future is fused, so we must "refresh" - // prior to any subsequent attempts to poll it after it - // emits `Ready` - this.wait - .set(this.attachment.0.req_notifier.notified()); - - // Take another lap if woken by the notifier to check - // for a pending request - continue; - } - return Poll::Pending; - } - } - } - } -} diff --git a/lib/propolis/src/block/crucible.rs b/lib/propolis/src/block/crucible.rs index 2b41adbb0..9541fb417 100644 --- a/lib/propolis/src/block/crucible.rs +++ b/lib/propolis/src/block/crucible.rs @@ -5,11 +5,11 @@ //! Implement a virtual block device backed by Crucible use std::io; +use std::num::NonZeroUsize; use std::ops::Deref; use std::sync::Arc; -use crate::accessors::MemAccessor; -use crate::block::{self, DeviceInfo}; +use crate::block; use crate::tasks::TaskGroup; use crate::vmm::MemCtx; @@ -25,60 +25,133 @@ use uuid::Uuid; pub use nexus_client::Client as NexusClient; +// TODO: Make this a runtime tunable? +const WORKER_COUNT: NonZeroUsize = NonZeroUsize::new(8).unwrap(); + pub struct CrucibleBackend { + block_attach: block::BackendAttachment, state: Arc, workers: TaskGroup, } struct WorkerState { - attachment: block::BackendAttachment, volume: Volume, info: block::DeviceInfo, skip_flush: bool, } impl WorkerState { - async fn process_loop(&self, acc_mem: MemAccessor) { - let waiter = match self.attachment.waiter() { - None => { - return; - } - Some(w) => w, - }; + async fn process_loop(&self, wctx: block::AsyncWorkerCtx) { // Start with a read buffer of a single block // It will be resized larger (and remain so) if subsequent read // operations required additional space. let mut readbuf = Buffer::new(1, self.info.block_size as usize); loop { - let req = match waiter.for_req().await { - Some(r) => r, - None => { - // bail - break; - } + let Some(dreq) = wctx.wait_for_req().await else { + break; }; - let res = if let Some(memctx) = acc_mem.access() { - match process_request( + + let Some(memctx) = wctx.acc_mem().access() else { + dreq.complete(block::Result::Failure); + continue; + }; + + let res = match self + .process_request( self.volume.deref(), - &self.info, - self.skip_flush, - &req, + dreq.req(), &mut readbuf, &memctx, ) .await - { - Ok(_) => block::Result::Success, - Err(e) => { - let mapped = block::Result::from(e); - assert!(mapped.is_err()); - mapped - } + { + Ok(_) => block::Result::Success, + Err(e) => { + let mapped = block::Result::from(e); + assert!(mapped.is_err()); + mapped } - } else { - block::Result::Failure }; - req.complete(res); + + dreq.complete(res); } } + + async fn process_request( + &self, + block: &(dyn BlockIO + Send + Sync), + req: &block::Request, + readbuf: &mut Buffer, + mem: &MemCtx, + ) -> Result<(), Error> { + let block_size = self.info.block_size as usize; + + match req.op { + block::Operation::Read(off, len) => { + let (off_blocks, len_blocks) = + block_offset_count(off, len, block_size)?; + + let maps = + req.mappings(mem).ok_or_else(|| Error::BadGuestRegion)?; + + // Perform one large read from crucible, and write from data into + // mappings + readbuf.reset(len_blocks, block_size); + let _ = block.read(off_blocks, readbuf).await?; + + let mut nwritten = 0; + for mapping in maps { + nwritten += mapping.write_bytes( + &readbuf[nwritten..(nwritten + mapping.len())], + )?; + } + + if nwritten != len { + return Err(Error::CopyError(nwritten, len)); + } + } + block::Operation::Write(off, len) => { + if self.info.read_only { + return Err(Error::ReadOnly); + } + + let (off_blocks, _len_blocks) = + block_offset_count(off, len, block_size)?; + + // Read from all the mappings into vec, and perform one large write + // to crucible + let maps = + req.mappings(mem).ok_or_else(|| Error::BadGuestRegion)?; + let mut data = crucible::BytesMut::with_capacity(len); + let mut nread = 0; + for mapping in maps { + let n = mapping.read_bytes_uninit( + &mut data.spare_capacity_mut()[..mapping.len()], + )?; + // `read_bytes` returns the number of bytes written, so we can + // expand our initialized area by this amount. + unsafe { + data.set_len(data.len() + n); + } + nread += n; + } + if nread != len { + return Err(Error::CopyError(nread, len)); + } + + let _ = block.write(off_blocks, data).await?; + } + block::Operation::Flush => { + if !self.skip_flush { + // Send flush to crucible + let _ = block.flush(None).await?; + } + } + block::Operation::Discard(..) => { + // Crucible does not support discard operations for now + return Err(Error::Unsupported); + } + } + Ok(()) + } } impl CrucibleBackend { @@ -141,16 +214,18 @@ impl CrucibleBackend { let total_size = volume.total_size().await?; let sectors = total_size / block_size; + let info = block::DeviceInfo { + block_size: block_size as u32, + total_size: sectors, + read_only: opts.read_only.unwrap_or(false), + supports_discard: false, + }; + Ok(Arc::new(Self { + block_attach: block::BackendAttachment::new(WORKER_COUNT, info), state: Arc::new(WorkerState { - attachment: block::BackendAttachment::new(), volume, - info: block::DeviceInfo { - block_size: block_size as u32, - total_size: sectors, - read_only: opts.read_only.unwrap_or(false), - supports_discard: false, - }, + info, skip_flush: opts.skip_flush.unwrap_or(false), }), workers: TaskGroup::new(), @@ -191,16 +266,18 @@ impl CrucibleBackend { .await .map_err(|e| std::io::Error::from(e))?; + let info = block::DeviceInfo { + block_size: block_size as u32, + total_size: size / block_size, + read_only: opts.read_only.unwrap_or(false), + supports_discard: false, + }; + Ok(Arc::new(CrucibleBackend { + block_attach: block::BackendAttachment::new(WORKER_COUNT, info), state: Arc::new(WorkerState { - attachment: block::BackendAttachment::new(), volume: builder.into(), - info: block::DeviceInfo { - block_size: block_size as u32, - total_size: size / block_size, - read_only: opts.read_only.unwrap_or(false), - supports_discard: false, - }, + info, skip_flush: opts.skip_flush.unwrap_or(false), }), workers: TaskGroup::new(), @@ -266,20 +343,17 @@ impl CrucibleBackend { } fn spawn_workers(&self) { - // TODO: make this tunable? - let worker_count = 8; - self.workers.extend((0..worker_count).map(|n| { + let max_workers = self.block_attach.max_workers().get(); + self.workers.extend((0..max_workers).map(|n| { let worker_state = self.state.clone(); - let worker_acc = self - .state - .attachment - .accessor_mem(|acc_mem| { - acc_mem.child(Some(format!("crucible worker {n}"))) - }) - .expect("backend is attached"); - tokio::spawn( - async move { worker_state.process_loop(worker_acc).await }, - ) + let wctx = self.block_attach.worker(n); + + tokio::spawn(async move { + let Some(wctx) = wctx.activate_async() else { + return; + }; + worker_state.process_loop(wctx).await + }) })) } @@ -291,19 +365,16 @@ impl CrucibleBackend { #[async_trait::async_trait] impl block::Backend for CrucibleBackend { fn attachment(&self) -> &block::BackendAttachment { - &self.state.attachment - } - fn info(&self) -> DeviceInfo { - self.state.info + &self.block_attach } async fn start(&self) -> anyhow::Result<()> { self.state.volume.activate().await?; - self.state.attachment.start(); + self.block_attach.start(); self.spawn_workers(); Ok(()) } async fn stop(&self) -> () { - self.state.attachment.stop(); + self.block_attach.stop(); self.workers.join_all().await; } } @@ -355,85 +426,6 @@ fn block_offset_count( } } -async fn process_request( - block: &(dyn BlockIO + Send + Sync), - info: &block::DeviceInfo, - skip_flush: bool, - req: &block::Request, - readbuf: &mut Buffer, - mem: &MemCtx, -) -> Result<(), Error> { - let block_size = info.block_size as usize; - - match req.oper() { - block::Operation::Read(off, len) => { - let (off_blocks, len_blocks) = - block_offset_count(off, len, block_size)?; - - let maps = - req.mappings(mem).ok_or_else(|| Error::BadGuestRegion)?; - - // Perform one large read from crucible, and write from data into - // mappings - readbuf.reset(len_blocks, block_size); - let _ = block.read(off_blocks, readbuf).await?; - - let mut nwritten = 0; - for mapping in maps { - nwritten += mapping.write_bytes( - &readbuf[nwritten..(nwritten + mapping.len())], - )?; - } - - if nwritten != len { - return Err(Error::CopyError(nwritten, len)); - } - } - block::Operation::Write(off, len) => { - if info.read_only { - return Err(Error::ReadOnly); - } - - let (off_blocks, _len_blocks) = - block_offset_count(off, len, block_size)?; - - // Read from all the mappings into vec, and perform one large write - // to crucible - let maps = - req.mappings(mem).ok_or_else(|| Error::BadGuestRegion)?; - let mut data = crucible::BytesMut::with_capacity(len); - let mut nread = 0; - for mapping in maps { - let n = mapping.read_bytes_uninit( - &mut data.spare_capacity_mut()[..mapping.len()], - )?; - // `read_bytes` returns the number of bytes written, so we can - // expand our initialized area by this amount. - unsafe { - data.set_len(data.len() + n); - } - nread += n; - } - if nread != len { - return Err(Error::CopyError(nread, len)); - } - - let _ = block.write(off_blocks, data).await?; - } - block::Operation::Flush => { - if !skip_flush { - // Send flush to crucible - let _ = block.flush(None).await?; - } - } - block::Operation::Discard(..) => { - // Crucible does not support discard operations for now - return Err(Error::Unsupported); - } - } - Ok(()) -} - #[cfg(test)] mod test { use super::block_offset_count; diff --git a/lib/propolis/src/block/file.rs b/lib/propolis/src/block/file.rs index f1c835a72..85f76f805 100644 --- a/lib/propolis/src/block/file.rs +++ b/lib/propolis/src/block/file.rs @@ -9,21 +9,20 @@ use std::os::unix::io::AsRawFd; use std::path::Path; use std::sync::{Arc, Mutex}; -use crate::accessors::MemAccessor; -use crate::block::{self, DeviceInfo}; +use crate::block::{self, SyncWorkerCtx, WorkerId}; use crate::tasks::ThreadGroup; use crate::vmm::{MappingExt, MemCtx}; use anyhow::Context; pub struct FileBackend { - state: Arc, + state: Arc, + block_attach: block::BackendAttachment, worker_count: NonZeroUsize, workers: ThreadGroup, } -struct WorkerState { - attachment: block::BackendAttachment, +struct SharedState { fp: File, /// Write-Cache-Enable state (if supported) of the underlying device @@ -37,7 +36,7 @@ struct WceState { initial: bool, current: bool, } -impl WorkerState { +impl SharedState { fn new( fp: File, info: block::DeviceInfo, @@ -45,8 +44,7 @@ impl WorkerState { wce_state: Option, discard_mech: Option, ) -> Arc { - let state = WorkerState { - attachment: block::BackendAttachment::new(), + let state = SharedState { fp, wce_state: Mutex::new(wce_state), discard_mech, @@ -60,29 +58,27 @@ impl WorkerState { Arc::new(state) } - fn processing_loop(&self, acc_mem: MemAccessor) { - while let Some(req) = self.attachment.block_for_req() { - if self.info.read_only && req.oper().is_write() { - req.complete(block::Result::ReadOnly); + fn processing_loop(&self, wctx: SyncWorkerCtx) { + while let Some(dreq) = wctx.block_for_req() { + let req = dreq.req(); + if self.info.read_only && req.op.is_write() { + dreq.complete(block::Result::ReadOnly); continue; } - if self.discard_mech.is_none() && req.oper().is_discard() { - req.complete(block::Result::Unsupported); + if self.discard_mech.is_none() && req.op.is_discard() { + dreq.complete(block::Result::Unsupported); continue; } - let mem = match acc_mem.access() { - Some(m) => m, - None => { - req.complete(block::Result::Failure); - continue; - } + let Some(mem) = wctx.acc_mem().access() else { + dreq.complete(block::Result::Failure); + continue; }; let res = match self.process_request(&req, &mem) { Ok(_) => block::Result::Success, Err(_) => block::Result::Failure, }; - req.complete(res); + dreq.complete(res); } } @@ -91,7 +87,7 @@ impl WorkerState { req: &block::Request, mem: &MemCtx, ) -> std::result::Result<(), &'static str> { - match req.oper() { + match req.op { block::Operation::Read(off, len) => { let maps = req.mappings(mem).ok_or("mapping unavailable")?; @@ -145,7 +141,7 @@ impl WorkerState { } } } -impl Drop for WorkerState { +impl Drop for SharedState { fn drop(&mut self) { // Attempt to return WCE state on the device to how it was when we // initially opened it. @@ -198,14 +194,16 @@ impl FileBackend { } else { None }; + let block_attach = block::BackendAttachment::new(worker_count, info); Ok(Arc::new(Self { - state: WorkerState::new( + state: SharedState::new( fp, info, skip_flush, wce_state, disk_info.discard_mech, ), + block_attach, worker_count, workers: ThreadGroup::new(), })) @@ -213,17 +211,16 @@ impl FileBackend { fn spawn_workers(&self) -> std::io::Result<()> { let spawn_results = (0..self.worker_count.get()) .map(|n| { - let worker_state = self.state.clone(); - let worker_acc = self - .state - .attachment - .accessor_mem(|mem| mem.child(Some(format!("worker {n}")))) - .expect("backend is attached"); + let shared_state = self.state.clone(); + let wctx = self.block_attach.worker(n as WorkerId); std::thread::Builder::new() .name(format!("file worker {n}")) .spawn(move || { - worker_state.processing_loop(worker_acc); + let wctx = wctx + .activate_sync() + .expect("worker slot is uncontended"); + shared_state.processing_loop(wctx); }) }) .collect::>(); @@ -235,17 +232,13 @@ impl FileBackend { #[async_trait::async_trait] impl block::Backend for FileBackend { fn attachment(&self) -> &block::BackendAttachment { - &self.state.attachment - } - - fn info(&self) -> DeviceInfo { - self.state.info + &self.block_attach } async fn start(&self) -> anyhow::Result<()> { - self.state.attachment.start(); + self.block_attach.start(); if let Err(e) = self.spawn_workers() { - self.state.attachment.stop(); + self.block_attach.stop(); self.workers.block_until_joined(); Err(e).context("failure while spawning workers") } else { @@ -254,7 +247,7 @@ impl block::Backend for FileBackend { } async fn stop(&self) -> () { - self.state.attachment.stop(); + self.block_attach.stop(); self.workers.block_until_joined(); } } diff --git a/lib/propolis/src/block/in_memory.rs b/lib/propolis/src/block/in_memory.rs index b205f9527..d6efdc543 100644 --- a/lib/propolis/src/block/in_memory.rs +++ b/lib/propolis/src/block/in_memory.rs @@ -6,7 +6,6 @@ use std::io::{Error, ErrorKind, Result}; use std::num::NonZeroUsize; use std::sync::{Arc, Mutex}; -use crate::accessors::MemAccessor; use crate::block; use crate::common::Lifecycle; use crate::migrate::{ @@ -19,41 +18,39 @@ use crate::vmm::{MemCtx, SubMapping}; use anyhow::Context; pub struct InMemoryBackend { - state: Arc, + shared_state: Arc, + block_attach: block::BackendAttachment, - worker_count: NonZeroUsize, workers: ThreadGroup, } -struct WorkingState { - attachment: block::BackendAttachment, +struct SharedState { bytes: Mutex>, info: block::DeviceInfo, } -impl WorkingState { - fn processing_loop(&self, acc_mem: MemAccessor) { - while let Some(req) = self.attachment.block_for_req() { - if self.info.read_only && req.oper().is_write() { - req.complete(block::Result::ReadOnly); +impl SharedState { + fn processing_loop(&self, wctx: block::SyncWorkerCtx) { + while let Some(dreq) = wctx.block_for_req() { + let req = dreq.req(); + if self.info.read_only && req.op.is_write() { + dreq.complete(block::Result::ReadOnly); continue; } if req.op.is_discard() { // Punt on discard support - req.complete(block::Result::Unsupported); + dreq.complete(block::Result::Unsupported); continue; } - let mem = match acc_mem.access() { - Some(m) => m, - None => { - req.complete(block::Result::Failure); - continue; - } - }; - let res = match self.process_request(&req, &mem) { - Ok(_) => block::Result::Success, - Err(_) => block::Result::Failure, + let res = match wctx + .acc_mem() + .access() + .and_then(|mem| self.process_request(&req, &mem).ok()) + { + Some(_) => block::Result::Success, + None => block::Result::Failure, }; - req.complete(res); + + dreq.complete(res); } } @@ -62,7 +59,7 @@ impl WorkingState { req: &block::Request, mem: &MemCtx, ) -> Result<()> { - match req.oper() { + match req.op { block::Operation::Read(off, len) => { let maps = req.mappings(mem).ok_or_else(|| { Error::new(ErrorKind::Other, "bad guest region") @@ -119,34 +116,34 @@ impl InMemoryBackend { )); } + let info = block::DeviceInfo { + block_size, + total_size: len as u64 / u64::from(block_size), + read_only: opts.read_only.unwrap_or(false), + supports_discard: false, + }; + let bytes = Mutex::new(bytes); + let block_attach = block::BackendAttachment::new(worker_count, info); + Ok(Arc::new(Self { - state: Arc::new(WorkingState { - attachment: block::BackendAttachment::new(), - bytes: Mutex::new(bytes), - info: block::DeviceInfo { - block_size, - total_size: len as u64 / u64::from(block_size), - read_only: opts.read_only.unwrap_or(false), - supports_discard: false, - }, - }), - worker_count, + shared_state: Arc::new(SharedState { bytes, info }), + block_attach, + workers: ThreadGroup::new(), })) } fn spawn_workers(&self) -> Result<()> { - let spawn_results = (0..self.worker_count.get()).map(|n| { - let worker_state = self.state.clone(); - let worker_acc = self - .state - .attachment - .accessor_mem(|mem| mem.child(Some(format!("worker {n}")))) - .expect("backend is attached"); - + let count = self.block_attach.max_workers().get(); + let spawn_results = (0..count).map(|n| { + let shared_state = self.shared_state.clone(); + let wctx = self.block_attach.worker(n); std::thread::Builder::new() .name(format!("in-memory worker {n}")) .spawn(move || { - worker_state.processing_loop(worker_acc); + let wctx = wctx + .activate_sync() + .expect("worker slot is uncontended"); + shared_state.processing_loop(wctx); }) }); @@ -157,17 +154,13 @@ impl InMemoryBackend { #[async_trait::async_trait] impl block::Backend for InMemoryBackend { fn attachment(&self) -> &block::BackendAttachment { - &self.state.attachment - } - - fn info(&self) -> block::DeviceInfo { - self.state.info + &self.block_attach } async fn start(&self) -> anyhow::Result<()> { - self.state.attachment.start(); + self.block_attach.start(); if let Err(e) = self.spawn_workers() { - self.state.attachment.stop(); + self.block_attach.stop(); self.workers.block_until_joined(); Err(e).context("failure while spawning workers") } else { @@ -176,7 +169,7 @@ impl block::Backend for InMemoryBackend { } async fn stop(&self) -> () { - self.state.attachment.stop(); + self.block_attach.stop(); self.workers.block_until_joined(); } } @@ -262,7 +255,7 @@ impl MigrateSingle for InMemoryBackend { &self, _ctx: &MigrateCtx, ) -> std::result::Result { - let bytes = self.state.bytes.lock().unwrap(); + let bytes = self.shared_state.bytes.lock().unwrap(); Ok(migrate::InMemoryBlockBackendV1 { bytes: bytes.clone() }.into()) } @@ -272,7 +265,7 @@ impl MigrateSingle for InMemoryBackend { _ctx: &MigrateCtx, ) -> std::result::Result<(), MigrateStateError> { let data: migrate::InMemoryBlockBackendV1 = offer.parse()?; - let mut guard = self.state.bytes.lock().unwrap(); + let mut guard = self.shared_state.bytes.lock().unwrap(); if guard.len() != data.bytes.len() { return Err(MigrateStateError::ImportFailed(format!( "imported in-memory block backend data has length {}, \ diff --git a/lib/propolis/src/block/mem_async.rs b/lib/propolis/src/block/mem_async.rs index 1d0f3c53e..c43c073b2 100644 --- a/lib/propolis/src/block/mem_async.rs +++ b/lib/propolis/src/block/mem_async.rs @@ -7,7 +7,6 @@ use std::num::NonZeroUsize; use std::ptr::NonNull; use std::sync::Arc; -use crate::accessors::MemAccessor; use crate::block; use crate::tasks::TaskGroup; use crate::vmm::MemCtx; @@ -18,42 +17,37 @@ use crate::vmm::MemCtx; /// this backend can be used for measuring how other parts of the emulation /// stack perform. pub struct MemAsyncBackend { - work_state: Arc, + shared_state: Arc, + block_attach: block::BackendAttachment, - worker_count: NonZeroUsize, workers: TaskGroup, } -struct WorkingState { - attachment: block::BackendAttachment, +struct SharedState { seg: MmapSeg, info: block::DeviceInfo, } -impl WorkingState { - async fn processing_loop(&self, acc_mem: MemAccessor) { - let waiter = match self.attachment.waiter() { - None => { - // Backend was detached - return; - } - Some(w) => w, - }; - while let Some(req) = waiter.for_req().await { - if self.info.read_only && req.oper().is_write() { - req.complete(block::Result::ReadOnly); +impl SharedState { + async fn processing_loop(&self, wctx: block::AsyncWorkerCtx) { + while let Some(dreq) = wctx.wait_for_req().await { + let req = dreq.req(); + if self.info.read_only && req.op.is_write() { + dreq.complete(block::Result::ReadOnly); continue; } - if req.oper().is_discard() { - req.complete(block::Result::Unsupported); + if req.op.is_discard() { + dreq.complete(block::Result::Unsupported); continue; } - let res = match acc_mem + + let res = match wctx + .acc_mem() .access() .and_then(|mem| self.process_request(&req, &mem).ok()) { Some(_) => block::Result::Success, None => block::Result::Failure, }; - req.complete(res); + dreq.complete(res); } } @@ -63,40 +57,34 @@ impl WorkingState { mem: &MemCtx, ) -> std::result::Result<(), &'static str> { let seg = &self.seg; - match req.oper() { + match req.op { block::Operation::Read(off, _len) => { - let maps = req.mappings(mem).ok_or("bad mapping")?; - - let mut nread = 0; - for map in maps { - unsafe { - let len = map.len(); - let read_ptr = map - .raw_writable() - .ok_or("expected writable mapping")?; - if !seg.read(off + nread, read_ptr, len) { - return Err("failed mem read"); + req.regions + .iter() + .try_fold(0usize, |nread, region| { + let map = mem.writable_region(region)?; + unsafe { + let read_ptr = map.raw_writable()?; + let len = map.len(); + seg.read(off + nread, read_ptr, len) + .then_some(nread + len) } - nread += len; - }; - } + }) + .ok_or("read failure")?; } block::Operation::Write(off, _len) => { - let maps = req.mappings(mem).ok_or("bad mapping")?; - - let mut nwritten = 0; - for map in maps { - unsafe { - let len = map.len(); - let write_ptr = map - .raw_readable() - .ok_or("expected readable mapping")?; - if !seg.write(off + nwritten, write_ptr, len) { - return Err("failed mem write"); + req.regions + .iter() + .try_fold(0usize, |nwritten, region| { + let map = mem.readable_region(region)?; + unsafe { + let write_ptr = map.raw_readable()?; + let len = map.len(); + seg.write(off + nwritten, write_ptr, len) + .then_some(nwritten + len) } - nwritten += len; - }; - } + }) + .ok_or("write failure")?; } block::Operation::Flush => { // nothing to do @@ -130,42 +118,53 @@ impl MemAsyncBackend { )); } + let info = block::DeviceInfo { + block_size, + total_size: size / u64::from(block_size), + read_only: opts.read_only.unwrap_or(false), + supports_discard: false, + }; let seg = MmapSeg::new(size as usize)?; + let block_attach = block::BackendAttachment::new(worker_count, info); Ok(Arc::new(Self { - work_state: Arc::new(WorkingState { - attachment: block::BackendAttachment::new(), - info: block::DeviceInfo { - block_size, - total_size: size / u64::from(block_size), - read_only: opts.read_only.unwrap_or(false), - supports_discard: false, - }, - seg, - }), - - worker_count, + shared_state: Arc::new(SharedState { info, seg }), + block_attach, + workers: TaskGroup::new(), })) } fn spawn_workers(&self) { - self.workers.extend((0..self.worker_count.get()).map(|n| { - let worker_state = self.work_state.clone(); - let worker_acc = self - .work_state - .attachment - .accessor_mem(|acc_mem| { - acc_mem.child(Some(format!("worker {n}"))) - }) - .expect("backend is attached"); + let count = self.block_attach.max_workers().get(); + self.workers.extend((0..count).map(|n| { + let shared_state = self.shared_state.clone(); + let wctx = self.block_attach.worker(n); tokio::spawn(async move { - worker_state.processing_loop(worker_acc).await + let wctx = + wctx.activate_async().expect("worker slot is uncontended"); + shared_state.processing_loop(wctx).await }) })) } } +#[async_trait::async_trait] +impl block::Backend for MemAsyncBackend { + fn attachment(&self) -> &block::BackendAttachment { + &self.block_attach + } + async fn start(&self) -> anyhow::Result<()> { + self.block_attach.start(); + self.spawn_workers(); + Ok(()) + } + async fn stop(&self) -> () { + self.block_attach.stop(); + self.workers.join_all().await; + } +} + struct MmapSeg(NonNull, usize); impl MmapSeg { fn new(size: usize) -> Result { @@ -212,25 +211,3 @@ impl Drop for MmapSeg { // Safety: The consumer is allowed to make their own pointer mistakes unsafe impl Send for MmapSeg {} unsafe impl Sync for MmapSeg {} - -#[async_trait::async_trait] -impl block::Backend for MemAsyncBackend { - fn info(&self) -> block::DeviceInfo { - self.work_state.info - } - - fn attachment(&self) -> &block::BackendAttachment { - &self.work_state.attachment - } - - async fn start(&self) -> anyhow::Result<()> { - self.work_state.attachment.start(); - self.spawn_workers(); - Ok(()) - } - - async fn stop(&self) -> () { - self.work_state.attachment.stop(); - self.workers.join_all().await; - } -} diff --git a/lib/propolis/src/block/minder.rs b/lib/propolis/src/block/minder.rs new file mode 100644 index 000000000..602f94920 --- /dev/null +++ b/lib/propolis/src/block/minder.rs @@ -0,0 +1,406 @@ +// This Source Code Form is subject to the terms of the Mozilla Public +// License, v. 2.0. If a copy of the MPL was not distributed with this +// file, You can obtain one at https://mozilla.org/MPL/2.0/. + +//! Mechanisms required to implement a block device + +use std::any::Any; +use std::borrow::Borrow; +use std::collections::BTreeMap; +use std::future::Future; +use std::pin::Pin; +use std::sync::{Arc, Mutex, Weak}; +use std::task::{Context, Poll}; +use std::time::Instant; + +use pin_project_lite::pin_project; +use tokio::sync::futures::Notified; +use tokio::sync::Notify; + +use crate::block::attachment::Bitmap; +use crate::block::{self, devq_id, probes, Operation, Request}; +use crate::block::{DeviceId, MetricConsumer, QueueId, WorkerId}; + +/// Each emulated block device will have one or more [DeviceQueue]s which can be +/// polled through [next_req()](DeviceQueue::next_req()) to emit IO requests. +/// The completions for those requests are then processed through +/// [complete()](DeviceQueue::complete()) calls. +pub trait DeviceQueue: Send + Sync + 'static { + /// Requests emitted from a [DeviceQueue] may require some associated state + /// in order to communicate their completion to the guest. The `Token` type + /// represents that state. + type Token: Send + Sync + 'static; + + /// Get the next [Request] (if any) from this queue. Supporting data + /// included with the request consists of the necessary [Self::Token] as + /// well an optional [queued-time](Instant). + fn next_req(&self) -> Option<(Request, Self::Token, Option)>; + + /// Emit a completion for a processed request, identified by its + /// [token](Self::Token). + fn complete( + &self, + op: block::Operation, + result: block::Result, + token: Self::Token, + ); +} + +/// A wrapper for an IO [Request] bearing necessary tracking information to +/// issue its completion back to the [queue](DeviceQueue) from which it came. +/// +/// A panic will occur a `DeviceRequest` instance is dropped without calling +/// [complete()](DeviceRequest::complete()). +pub struct DeviceRequest { + req: Request, + id: ReqId, + source: Weak, + _nodrop: NoDropDevReq, +} +impl DeviceRequest { + fn new(id: ReqId, req: Request, source: Weak) -> Self { + Self { req, id, source, _nodrop: NoDropDevReq } + } + + /// Get the underlying block [Request] + pub fn req(&self) -> &Request { + &self.req + } + + /// Issue a completion for this [Request]. + pub fn complete(self, result: super::Result) { + let DeviceRequest { id, source, _nodrop, .. } = self; + std::mem::forget(_nodrop); + + if let Some(src) = source.upgrade() { + src.complete(id, result); + } + } +} + +/// Marker struct to ensure that [DeviceRequest] consumers call +/// [complete()](DeviceRequest::complete()), rather than silently dropping it. +struct NoDropDevReq; +impl Drop for NoDropDevReq { + fn drop(&mut self) { + panic!("DeviceRequest should be complete()-ed before drop"); + } +} + +/// Closure to permit [QueueMinder] to type-erase the calling of +/// [DeviceQueue::next_req()]. +type NextReqFn = Box< + dyn Fn() -> Option<( + block::Request, + Box, + Option, + )> + Send + + Sync, +>; + +/// Closure to permit [QueueMinder] to type-erase the calling of +/// [DeviceQueue::complete()]. +type CompleteReqFn = Box< + dyn Fn(Operation, block::Result, Box) + Send + Sync, +>; + +struct QmEntry { + token: Box, + op: Operation, + when_queued: Instant, + when_started: Instant, +} + +struct QmInner { + next_id: ReqId, + /// Map of [WorkerId]s which we emitted [None] to via + /// [QueueMinder::next_req()] and which are likely candidates to notify when + /// this queue has new entries. + notify_workers: Bitmap, + paused: bool, + in_flight: BTreeMap, + metric_consumer: Option>, + /// Number of [Request] completions which are currently being processed by + /// the device. This is tracked only for requests which are the last entry + /// removed from `in_flight`, as a means providing accurate results from + /// [NoneInFlight]. + processing_last: usize, +} +impl Default for QmInner { + fn default() -> Self { + Self { + next_id: ReqId::START, + notify_workers: Bitmap::default(), + paused: false, + processing_last: 0, + in_flight: BTreeMap::new(), + metric_consumer: None, + } + } +} + +pub(super) struct QueueMinder { + pub queue_id: QueueId, + pub device_id: DeviceId, + state: Mutex, + self_ref: Weak, + notify: Notify, + /// Type-erased wrapper function for [DeviceQueue::next_req()] + next_req_fn: NextReqFn, + /// Type-erased wrapper function for [DeviceQueue::complete()] + complete_req_fn: CompleteReqFn, +} + +impl QueueMinder { + pub fn new( + queue: Arc, + device_id: DeviceId, + queue_id: QueueId, + ) -> Arc { + let next_req_queue = queue.clone(); + let next_req_fn: NextReqFn = Box::new(move || { + let (req, token, when_queued) = next_req_queue.next_req()?; + Some(( + req, + Box::new(token) as Box, + when_queued, + )) + }); + + let complete_req_fn: CompleteReqFn = + Box::new(move |op, result, token| { + let token = token + .downcast::() + .expect("token type unchanged"); + let token = *token; + queue.complete(op, result, token); + }); + + Arc::new_cyclic(|self_ref| Self { + queue_id, + device_id, + state: Mutex::new(QmInner::default()), + self_ref: self_ref.clone(), + notify: Notify::new(), + next_req_fn, + complete_req_fn, + }) + } + + /// Attempt to fetch the next IO request from this queue for a worker. + /// + /// If no requests are available, that worker (specified by `wid`) will be + /// recorded so it can be notified if/when the guest notifies this queue + /// that more requests are available. + pub fn next_req(&self, wid: WorkerId) -> Option { + let mut state = self.state.lock().unwrap(); + if state.paused || !state.notify_workers.is_empty() { + state.notify_workers.set(wid); + return None; + } + if let Some((req, token, when_queued)) = (self.next_req_fn)() { + let id = state.next_id; + state.next_id.advance(); + + let devqid = devq_id(self.device_id, self.queue_id); + match req.op { + Operation::Read(off, len) => { + probes::block_begin_read!(|| { + (devqid, id, off as u64, len as u64) + }); + } + Operation::Write(off, len) => { + probes::block_begin_write!(|| { + (devqid, id, off as u64, len as u64) + }); + } + Operation::Flush => { + probes::block_begin_flush!(|| { (devqid, id) }); + } + Operation::Discard(off, len) => { + probes::block_begin_discard!(|| { + (devqid, id, off as u64, len as u64) + }); + } + } + let when_started = Instant::now(); + let old = state.in_flight.insert( + id, + QmEntry { + token, + op: req.op, + when_queued: when_queued.unwrap_or(when_started), + when_started, + }, + ); + assert!(old.is_none(), "request IDs should not overlap"); + + Some(DeviceRequest::new(id, req, self.self_ref.clone())) + } else { + state.notify_workers.set(wid); + None + } + } + + /// Process a completion for an in-flight IO request on this queue. + pub fn complete(&self, id: ReqId, result: block::Result) { + let mut state = self.state.lock().unwrap(); + let ent = + state.in_flight.remove(&id).expect("state for request not lost"); + let metric_consumer = state.metric_consumer.as_ref().map(Arc::clone); + let is_last_req = state.in_flight.is_empty(); + if is_last_req { + state.processing_last += 1; + } + drop(state); + + let when_done = Instant::now(); + let time_queued = ent.when_started.duration_since(ent.when_queued); + let time_processed = when_done.duration_since(ent.when_started); + + let ns_queued = time_queued.as_nanos() as u64; + let ns_processed = time_processed.as_nanos() as u64; + let rescode = result as u8; + let devqid = devq_id(self.device_id, self.queue_id); + match ent.op { + Operation::Read(..) => { + probes::block_complete_read!(|| { + (devqid, id, rescode, ns_processed, ns_queued) + }); + } + Operation::Write(..) => { + probes::block_complete_write!(|| { + (devqid, id, rescode, ns_processed, ns_queued) + }); + } + Operation::Flush => { + probes::block_complete_flush!(|| { + (devqid, id, rescode, ns_processed, ns_queued) + }); + } + Operation::Discard(..) => { + probes::block_complete_discard!(|| { + (devqid, id, rescode, ns_processed, ns_queued) + }); + } + } + + (self.complete_req_fn)(ent.op, result, ent.token); + + probes::block_completion_sent!(|| { + (devqid, id, when_done.elapsed().as_nanos() as u64) + }); + + // Report the completion to the metrics consumer, if one exists + if let Some(consumer) = metric_consumer { + consumer.request_completed( + self.queue_id, + ent.op, + result, + time_queued, + time_processed, + ); + } + + // We must track how many completions are being processed by the device, + // since they are done outside the state lock, in order to present a + // reliably accurate accurate signal of when the device has no more + // in-flight requests. + if is_last_req { + let mut state = self.state.lock().unwrap(); + state.processing_last -= 1; + if state.in_flight.is_empty() && state.processing_last == 0 { + self.notify.notify_waiters(); + } + } + } + + /// Get a bitmap of the workers which should be notified that this queue may + /// now have requests available. + pub(crate) fn take_notifications(&self) -> Option { + let mut state = self.state.lock().unwrap(); + if state.paused { + state.notify_workers = Bitmap::ALL; + None + } else { + Some(state.notify_workers.take()) + } + } + + /// Associate a [MetricConsumer] with this queue. + /// + /// It will be notified about each IO completion as they occur. + pub(crate) fn set_metric_consumer( + &self, + consumer: Arc, + ) { + self.state.lock().unwrap().metric_consumer = Some(consumer); + } + + pub(crate) fn pause(&self) { + let mut state = self.state.lock().unwrap(); + state.paused = true; + self.notify.notify_waiters(); + } + + pub(crate) fn resume(&self) { + let mut state = self.state.lock().unwrap(); + state.paused = false; + self.notify.notify_waiters(); + } + + pub(crate) fn none_in_flight(&self) -> NoneInFlight<'_> { + NoneInFlight { minder: self, wait: self.notify.notified() } + } +} + +pin_project! { + /// A [Future] which resolves to [Ready](Poll::Ready) when there are no + /// requests being processed by an attached backend. + pub(crate) struct NoneInFlight<'a> { + minder: &'a QueueMinder, + #[pin] + wait: Notified<'a> + } +} +impl Future for NoneInFlight<'_> { + type Output = (); + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let mut this = self.project(); + + loop { + let state = this.minder.state.lock().unwrap(); + if state.in_flight.is_empty() && state.processing_last == 0 { + return Poll::Ready(()); + } + if let Poll::Ready(_) = Notified::poll(this.wait.as_mut(), cx) { + // Refresh fused future from Notify + this.wait.set(this.minder.notify.notified()); + } else { + return Poll::Pending; + } + } + } +} + +/// Unique ID assigned to a given block [Request]. +#[derive(Copy, Clone, PartialEq, PartialOrd, Eq, Ord)] +pub struct ReqId(u64); +impl ReqId { + const START: Self = ReqId(0); + + fn advance(&mut self) { + self.0 += 1; + } +} +impl Borrow for ReqId { + fn borrow(&self) -> &u64 { + &self.0 + } +} +impl From for u64 { + fn from(value: ReqId) -> Self { + value.0 + } +} diff --git a/lib/propolis/src/block/mod.rs b/lib/propolis/src/block/mod.rs index 0436f9431..104c7589e 100644 --- a/lib/propolis/src/block/mod.rs +++ b/lib/propolis/src/block/mod.rs @@ -4,16 +4,13 @@ //! Implements an interface to virtualized block devices. -use std::borrow::Borrow; +use std::time::Duration; -use crate::accessors::MemAccessor; -use crate::attachment::DetachError; use crate::common::*; use crate::vmm::{MemCtx, SubMapping}; mod file; pub use file::FileBackend; -use tracking::CompletionCallback; #[cfg(feature = "crucible")] mod crucible; @@ -27,9 +24,13 @@ mod mem_async; pub use mem_async::MemAsyncBackend; pub mod attachment; -pub mod tracking; +pub mod minder; -pub use attachment::{attach, BackendAttachment, DeviceAttachment}; +pub use attachment::{ + attach, AsyncWorkerCtx, AttachError, BackendAttachment, DeviceAttachment, + SyncWorkerCtx, +}; +pub use minder::{DeviceQueue, DeviceRequest}; pub type ByteOffset = usize; pub type ByteLen = usize; @@ -40,13 +41,13 @@ pub const DEFAULT_BLOCK_SIZE: u32 = 512; #[usdt::provider(provider = "propolis")] mod probes { - fn block_begin_read(dev_id: u64, req_id: u64, offset: u64, len: u64) {} - fn block_begin_write(dev_id: u64, req_id: u64, offset: u64, len: u64) {} - fn block_begin_flush(dev_id: u64, req_id: u64) {} - fn block_begin_discard(dev_id: u64, req_id: u64, offset: u64, len: u64) {} + fn block_begin_read(devq_id: u64, req_id: u64, offset: u64, len: u64) {} + fn block_begin_write(devq_id: u64, req_id: u64, offset: u64, len: u64) {} + fn block_begin_flush(devq_id: u64, req_id: u64) {} + fn block_begin_discard(devq_id: u64, req_id: u64, offset: u64, len: u64) {} fn block_complete_read( - dev_id: u64, + devq_id: u64, req_id: u64, result: u8, proc_ns: u64, @@ -54,7 +55,7 @@ mod probes { ) { } fn block_complete_write( - dev_id: u64, + devq_id: u64, req_id: u64, result: u8, proc_ns: u64, @@ -62,7 +63,7 @@ mod probes { ) { } fn block_complete_flush( - dev_id: u64, + devq_id: u64, req_id: u64, result: u8, proc_ns: u64, @@ -70,13 +71,21 @@ mod probes { ) { } fn block_complete_discard( - dev_id: u64, + devq_id: u64, req_id: u64, result: u8, proc_ns: u64, queue_ns: u64, ) { } + + fn block_completion_sent(devq_id: u64, req_id: u64, complete_ns: u64) {} + + fn block_poll(devq_id: u64, worker_id: u64, emit_req: u8) {} + fn block_sleep(dev_id: u32, worker_id: u64) {} + fn block_wake(dev_id: u32, worker_id: u64) {} + fn block_notify(devq_id: u64) {} + fn block_strategy(dev_id: u32, strat: String, generation: u64) {} } /// Type of operations which may be issued to a virtual block device. @@ -124,20 +133,68 @@ impl Result { } } +#[derive(Copy, Clone, Eq, PartialEq, Debug, Default)] +pub struct QueueId(u8); +impl QueueId { + /// Arbitrary limit for per-device queues. + /// Sized to match [attachment::Bitmap] capacity + pub const MAX_QUEUES: usize = 64; + + pub const MAX: Self = Self(Self::MAX_QUEUES as u8); + + /// Get the next sequential QueueId, wrapping around at a maximum + fn next(self, max: usize) -> Self { + let max: u8 = max.try_into().expect("max should be in-range"); + assert!(max != 0 && max <= Self::MAX.0); + + let next = self.0.wrapping_add(1); + if next >= max { + Self(0) + } else { + Self(next) + } + } +} +impl From for QueueId { + fn from(value: usize) -> Self { + assert!(value < Self::MAX_QUEUES); + Self(value as u8) + } +} +impl From for usize { + fn from(value: QueueId) -> Self { + value.0 as usize + } +} +impl From for QueueId { + fn from(value: u16) -> Self { + assert!(value < (Self::MAX_QUEUES as u16)); + Self(value as u8) + } +} +impl From for u16 { + fn from(value: QueueId) -> Self { + value.0 as u16 + } +} + +pub type DeviceId = u32; +pub type WorkerId = usize; + +/// Combine device and queue IDs into single u64 for probes +pub(crate) fn devq_id(dev: DeviceId, queue: QueueId) -> u64 { + ((dev as u64) << 8) | (queue.0 as u64) +} + /// Block device operation request +#[derive(Clone)] pub struct Request { /// The type of operation requested by the block device - op: Operation, + pub op: Operation, /// A list of regions of guest memory to read/write into as part of the I/O /// request - regions: Vec, - - /// Store [`tracking::TrackingMarker`] when this request is tracked by a - /// [`tracking::Tracking`] for that device. It is through this marker that - /// the result of the block request is communicated back to the device - /// emulation for processing. - marker: Option, + pub regions: Vec, } impl Request { pub fn new_read( @@ -145,7 +202,7 @@ impl Request { len: ByteLen, regions: Vec, ) -> Self { - Self { op: Operation::Read(off, len), regions, marker: None } + Self { op: Operation::Read(off, len), regions } } pub fn new_write( @@ -153,27 +210,17 @@ impl Request { len: ByteLen, regions: Vec, ) -> Self { - Self { op: Operation::Write(off, len), regions, marker: None } + Self { op: Operation::Write(off, len), regions } } pub fn new_flush() -> Self { let op = Operation::Flush; - Self { op, regions: Vec::new(), marker: None } + Self { op, regions: Vec::new() } } pub fn new_discard(off: ByteOffset, len: ByteLen) -> Self { let op = Operation::Discard(off, len); - Self { op, regions: Vec::new(), marker: None } - } - - /// Type of operation being issued. - pub fn oper(&self) -> Operation { - self.op - } - - /// Guest memory regions underlying the request - pub fn regions(&self) -> &[GuestRegion] { - &self.regions[..] + Self { op, regions: Vec::new() } } pub fn mappings<'a>(&self, mem: &'a MemCtx) -> Option>> { @@ -187,20 +234,6 @@ impl Request { Operation::Flush | Operation::Discard(..) => None, } } - - /// Indicate disposition of completed request - pub fn complete(mut self, res: Result) { - if let Some(marker) = self.marker.take() { - marker.complete(res); - } - } -} -impl Drop for Request { - fn drop(&mut self) { - if self.marker.is_some() { - panic!("request dropped prior to completion"); - } - } } /// Metadata regarding a virtualized block device. @@ -239,23 +272,6 @@ pub struct BackendOpts { pub trait Device: Send + Sync + 'static { /// Access to the [DeviceAttachment] representing this device. fn attachment(&self) -> &DeviceAttachment; - - /// Retrieve the next request (if any) - fn next(&self) -> Option; - - /// Complete processing of result - fn complete(&self, res: Result, id: ReqId); - - /// Attach a callback to be run on completion of I/Os. - /// - /// Returns whether there was a previously-registered callback. - fn on_completion(&self, _cb: Box) -> bool; - - /// Get an accessor to guest memory via the underlying device - fn accessor_mem(&self) -> MemAccessor; - - /// Optional on-attach handler to update device state with new `DeviceInfo` - fn on_attach(&self, _info: DeviceInfo) {} } /// Top-level trait for block backends which will attach to [Device]s in order @@ -265,9 +281,6 @@ pub trait Backend: Send + Sync + 'static { /// Access to the [BackendAttachment] representing this backend. fn attachment(&self) -> &BackendAttachment; - /// Query [DeviceInfo] from the backend - fn info(&self) -> DeviceInfo; - /// Start attempting to process [Request]s from [Device] (if attached) /// /// Spawning of any tasks required to do such request processing can be done @@ -282,6 +295,7 @@ pub trait Backend: Send + Sync + 'static { /// by this routine. In this case the caller may not call [`Self::stop()`] /// prior to dropping the backend. This routine is, however, guaranteed to /// be called before the VM's vCPUs are started. + /// async fn start(&self) -> anyhow::Result<()>; /// Stop attempting to process new [Request]s from [Device] (if attached) @@ -297,41 +311,19 @@ pub trait Backend: Send + Sync + 'static { /// events; instead, their corresponding devices will stop issuing new /// requests when they are told to pause (and will only report they are /// fully paused when all their in-flight requests have completed). - async fn stop(&self) -> (); - - /// Attempt to detach from associated [Device] - /// - /// Any attached backend should be [stopped](Backend::stop()) and detached - /// prior to its references being dropped. An attached [Backend]/[Device] - /// pair holds mutual references and thus will not be reaped if all other - /// external references are dropped. - fn detach(&self) -> std::result::Result<(), DetachError> { - self.attachment().detach() - } + async fn stop(&self); } -pub enum CacheMode { - Synchronous, - WriteBack, -} - -/// Unique ID assigned (by [`tracking::Tracking`] to a given block [`Request`]. -#[derive(Copy, Clone, PartialEq, PartialOrd, Eq, Ord)] -pub struct ReqId(u64); -impl ReqId { - const START: Self = ReqId(1); - - fn advance(&mut self) { - self.0 += 1; - } -} -impl Borrow for ReqId { - fn borrow(&self) -> &u64 { - &self.0 - } -} -impl From for u64 { - fn from(value: ReqId) -> Self { - value.0 - } +/// Consumer of per-[Request] metrics +pub trait MetricConsumer: Send + Sync + 'static { + /// Called upon the completion of each block [Request] when a MetricConsumer + /// has been set for a given [DeviceAttachment]. + fn request_completed( + &self, + queue_id: QueueId, + op: Operation, + result: Result, + time_queued: Duration, + time_processed: Duration, + ); } diff --git a/lib/propolis/src/block/tracking.rs b/lib/propolis/src/block/tracking.rs deleted file mode 100644 index 4468b3f54..000000000 --- a/lib/propolis/src/block/tracking.rs +++ /dev/null @@ -1,281 +0,0 @@ -// This Source Code Form is subject to the terms of the Mozilla Public -// License, v. 2.0. If a copy of the MPL was not distributed with this -// file, You can obtain one at https://mozilla.org/MPL/2.0/. - -//! Mechanisms required to implement a block device - -use std::collections::BTreeMap; -use std::future::Future; -use std::pin::Pin; -use std::sync::atomic::{AtomicU64, Ordering}; -use std::sync::{Arc, Mutex, Weak}; -use std::task::{Context, Poll, Waker}; -use std::time::{Duration, Instant}; - -use crate::block::{self, probes, Device, Operation, ReqId, Request}; - -static NEXT_DEVICE_ID: AtomicU64 = AtomicU64::new(1); - -/// A function that is called when a block operation completes. -pub trait CompletionCallback: - Fn(Operation, block::Result, Duration) + Send + Sync + 'static -{ -} - -impl CompletionCallback for T where - T: Fn(Operation, block::Result, Duration) + Send + Sync + 'static -{ -} - -/// Tracking structure for outstanding block [`Request`]s. -/// -/// As requests are emitted to the associated backend, the corresponding data -/// required to indicate a completion to the guest will be stored here. The -/// Request is tagged with a unique [`ReqId`] which is used to retrieve said -/// completion data, as well as track the time used to process the request. -/// -/// Although use of [`Tracking`] is not required by the block abstraction, it is -/// here where the general USDT probes are attached. A device which eschews its -/// use will be missing calls into those probes. -/// -/// Each [`Tracking`] also allows one optional callback that it will call -/// whenever an I/O is completed. This can be set via -/// [`Tracking::set_completion_callback()`]. -pub struct Tracking { - inner: Mutex>, - wait: Arc>, -} -struct TrackingInner { - device_id: u64, - next_id: ReqId, - dev: Weak, - outstanding: BTreeMap>, - on_completion: Option>, -} -struct TrackingEntry { - op: Operation, - payload: T, - /// When this request was submitted to the backend to be processed - time_submitted: Instant, -} - -/// Track device-specific data for outstanding block [Request]s. -impl Tracking { - /// Create a new block tracking object. - /// - /// NOTE: This does not set the completion callback, use - /// [`Self::set_completion_callback()`] to do so. - pub fn new(dev: Weak) -> Self { - let device_id = NEXT_DEVICE_ID.fetch_add(1, Ordering::Relaxed); - Self { - inner: Mutex::new(TrackingInner { - device_id, - next_id: ReqId::START, - dev, - outstanding: BTreeMap::new(), - on_completion: None, - }), - wait: Arc::new(Mutex::new(TrackingWait::new())), - } - } - - /// Set or overwrite the completion callback. - /// - /// Returns true if there was a previous callback. - pub fn set_completion_callback( - &self, - cb: Box, - ) -> bool { - self.inner.lock().unwrap().on_completion.replace(cb).is_some() - } - - /// Record tracking in an [`Request`] prior to passing it to the associated - /// [`block::Backend`]. The request will be assigned a unique [`ReqId`] - /// which can be used to a later call to [`Tracking::complete()`] to - /// retrieve the `payload` data required to communicate its completion. - /// - pub fn track(&self, mut req: Request, payload: T) -> Request { - let now = Instant::now(); - let mut guard = self.inner.lock().unwrap(); - let began_empty = guard.outstanding.is_empty(); - let id = guard.next_id; - guard.next_id.advance(); - let marker = TrackingMarker { - id, - dev: guard.dev.upgrade().expect("device still exists"), - }; - guard.outstanding.insert( - marker.id, - TrackingEntry { op: req.op, payload, time_submitted: now }, - ); - - let old = req.marker.replace(marker); - assert!(old.is_none(), "request should be tracked only once"); - - if began_empty { - self.wait.lock().unwrap().clear_empty() - } - let devid = guard.device_id; - match req.op { - Operation::Read(off, len) => { - probes::block_begin_read!(|| { - (devid, id, off as u64, len as u64) - }); - } - Operation::Write(off, len) => { - probes::block_begin_write!(|| { - (devid, id, off as u64, len as u64) - }); - } - Operation::Flush => { - probes::block_begin_flush!(|| { (devid, id) }); - } - Operation::Discard(off, len) => { - probes::block_begin_discard!(|| { - (devid, id, off as u64, len as u64) - }); - } - } - - req - } - - /// Indicate the completion of a pending [`Request`], retrieving the - /// associated payload data. The [`block::Result`] argument is used to - /// communicate the result through the generic block USDT probe. - pub fn complete(&self, id: ReqId, res: block::Result) -> (Operation, T) { - let now = Instant::now(); - let mut guard = self.inner.lock().unwrap(); - let entry = guard - .outstanding - .remove(&id) - .expect("tracked request should be present"); - - let devid = guard.device_id; - let elapsed = now.duration_since(entry.time_submitted); - let proc_ns = elapsed.as_nanos() as u64; - // TODO: calculate queued time - let queue_ns = 0; - let rescode = res as u8; - match entry.op { - Operation::Read(..) => { - probes::block_complete_read!(|| { - (devid, id, rescode, proc_ns, queue_ns) - }); - } - Operation::Write(..) => { - probes::block_complete_write!(|| { - (devid, id, rescode, proc_ns, queue_ns) - }); - } - Operation::Flush => { - probes::block_complete_flush!(|| { - (devid, id, rescode, proc_ns, queue_ns) - }); - } - Operation::Discard(..) => { - probes::block_complete_discard!(|| { - (devid, id, rescode, proc_ns, queue_ns) - }); - } - } - - if let Some(cb) = guard.on_completion.as_ref() { - cb(entry.op, res, elapsed); - } - - if guard.outstanding.is_empty() { - self.wait.lock().unwrap().set_empty(); - } - - (entry.op, entry.payload) - } - - /// Query if there are any tracked requests outstanding - pub fn any_outstanding(&self) -> bool { - let guard = self.inner.lock().unwrap(); - !guard.outstanding.is_empty() - } - - /// Emit a [`Future`] which will resolve when there are no tracked - /// request outstanding in this structure. - pub fn none_outstanding(&self) -> NoneOutstanding { - NoneOutstanding::new(self.wait.clone()) - } -} - -/// Record keeping for [`NoneOutstanding`] futures emitted by [`Tracking`] -struct TrackingWait { - empty: bool, - gen: usize, - wake: Vec, -} -impl TrackingWait { - fn new() -> Self { - Self { empty: true, gen: 1, wake: Vec::new() } - } - fn set_empty(&mut self) { - self.empty = true; - self.gen += 1; - - for waker in self.wake.drain(..) { - waker.wake() - } - } - fn clear_empty(&mut self) { - self.empty = false; - } -} - -/// Future which will complete when the referenced [`Tracking`] has no more -/// requests outstanding. -pub struct NoneOutstanding { - wait: Arc>, - gen: usize, -} -impl NoneOutstanding { - fn new(wait: Arc>) -> Self { - Self { wait, gen: 0 } - } -} -impl Future for NoneOutstanding { - type Output = (); - - fn poll( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll { - let mut wguard = self.wait.lock().unwrap(); - if wguard.empty { - Poll::Ready(()) - } else { - // Add us to the waker list if the TrackingWait generation is newer - // than this future. That list is cleaned up by draining its - // entries when waking them (forcing interested futures to re-add - // themselves if need be). - // - // NoneOutstanding instances which are dropped before this occurs - // will "leak" their entry in waker list insofar as it will not be - // cleaned up until the next time the Tracking structure becomes - // empty. This is considered an acceptable trade-off for - // simplicity. - if wguard.gen > self.gen { - wguard.wake.push(cx.waker().clone()); - let gen = wguard.gen; - drop(wguard); - self.gen = gen; - } - Poll::Pending - } - } -} - -pub(super) struct TrackingMarker { - id: ReqId, - dev: Arc, -} -impl TrackingMarker { - pub(super) fn complete(self, res: block::Result) { - self.dev.complete(res, self.id); - } -} diff --git a/lib/propolis/src/hw/nvme/admin.rs b/lib/propolis/src/hw/nvme/admin.rs index 76a17e94d..576acc246 100644 --- a/lib/propolis/src/hw/nvme/admin.rs +++ b/lib/propolis/src/hw/nvme/admin.rs @@ -8,8 +8,10 @@ use crate::common::{GuestAddr, GuestRegion, PAGE_SIZE}; use crate::vmm::MemCtx; use super::bits::*; -use super::queue::{QueueId, ADMIN_QUEUE_ID}; -use super::{cmds, NvmeCtrl, NvmeError, MAX_NUM_IO_QUEUES, MAX_NUM_QUEUES}; +use super::queue::{sqid_to_block_qid, QueueId, ADMIN_QUEUE_ID}; +use super::{ + cmds, NvmeCtrl, NvmeError, PciNvme, MAX_NUM_IO_QUEUES, MAX_NUM_QUEUES, +}; #[usdt::provider(provider = "propolis")] mod probes { @@ -44,6 +46,7 @@ impl NvmeCtrl { pub(super) fn acmd_create_io_cq( &mut self, cmd: &cmds::CreateIOCQCmd, + nvme: &PciNvme, mem: &MemCtx, ) -> cmds::Completion { // If the host hasn't specified an IOCQES, fail this request @@ -68,10 +71,14 @@ impl NvmeCtrl { // Finally, create the Completion Queue match self.create_cq( - cmd.qid, + super::queue::CreateParams { + id: cmd.qid, + base: GuestAddr(cmd.prp), + size: cmd.qsize, + }, + false, cmd.intr_vector, - GuestAddr(cmd.prp), - cmd.qsize, + nvme, mem, ) { Ok(_) => cmds::Completion::success(), @@ -93,6 +100,7 @@ impl NvmeCtrl { pub(super) fn acmd_create_io_sq( &mut self, cmd: &cmds::CreateIOSQCmd, + nvme: &PciNvme, mem: &MemCtx, ) -> cmds::Completion { // If the host hasn't specified an IOSQES, fail this request @@ -110,13 +118,20 @@ impl NvmeCtrl { // Finally, create the Submission Queue match self.create_sq( - cmd.qid, + super::queue::CreateParams { + id: cmd.qid, + base: GuestAddr(cmd.prp), + size: cmd.qsize, + }, + false, cmd.cqid, - GuestAddr(cmd.prp), - cmd.qsize, + nvme, mem, ) { - Ok(_) => cmds::Completion::success(), + Ok(sq) => { + self.io_sq_post_create(nvme, sq); + cmds::Completion::success() + } Err(NvmeError::InvalidCompQueue(_)) => { cmds::Completion::specific_err( StatusCodeType::CmdSpecific, @@ -141,6 +156,7 @@ impl NvmeCtrl { pub(super) fn acmd_delete_io_cq( &mut self, cqid: QueueId, + nvme: &PciNvme, ) -> cmds::Completion { // Not allowed to delete the Admin Completion Queue if cqid == ADMIN_QUEUE_ID { @@ -153,7 +169,7 @@ impl NvmeCtrl { // Remove the CQ from our list of active CQs. // At this point, all associated SQs should've been deleted // otherwise we'll return an error. - match self.delete_cq(cqid) { + match self.delete_cq(cqid, nvme) { Ok(()) => cmds::Completion::success(), Err(NvmeError::InvalidCompQueue(_)) => { cmds::Completion::specific_err( @@ -177,6 +193,7 @@ impl NvmeCtrl { pub(super) fn acmd_delete_io_sq( &mut self, sqid: QueueId, + nvme: &PciNvme, ) -> cmds::Completion { // Not allowed to delete the Admin Submission Queue if sqid == ADMIN_QUEUE_ID { @@ -194,8 +211,12 @@ impl NvmeCtrl { // Note: The NVMe 1.0e spec says "The command causes all commands // submitted to the indicated Submission Queue that are still in // progress to be aborted." - match self.delete_sq(sqid) { - Ok(()) => cmds::Completion::success(), + match self.delete_sq(sqid, nvme) { + Ok(()) => { + nvme.block_attach.queue_dissociate(sqid_to_block_qid(sqid)); + // TODO: wait until requests are done? + cmds::Completion::success() + } Err(NvmeError::InvalidSubQueue(_)) => { cmds::Completion::specific_err( StatusCodeType::CmdSpecific, diff --git a/lib/propolis/src/hw/nvme/mod.rs b/lib/propolis/src/hw/nvme/mod.rs index ef21266a2..019fbc549 100644 --- a/lib/propolis/src/hw/nvme/mod.rs +++ b/lib/propolis/src/hw/nvme/mod.rs @@ -4,6 +4,8 @@ use std::convert::TryInto; use std::mem::size_of; +use std::num::NonZeroUsize; +use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::{Arc, Mutex, MutexGuard, Weak}; use crate::accessors::Guard; @@ -27,7 +29,7 @@ mod queue; mod requests; use bits::*; -use queue::{CompQueue, Permit, QueueId, SubQueue}; +use queue::{CompQueue, QueueId, SubQueue}; #[usdt::provider(provider = "propolis")] mod probes { @@ -172,41 +174,64 @@ impl NvmeCtrl { /// Creates the admin completion and submission queues. /// /// Admin queues are always created with `cqid`/`sqid` `0`. - fn create_admin_queues(&mut self, mem: &MemCtx) -> Result<(), NvmeError> { + fn create_admin_queues( + &mut self, + nvme: &PciNvme, + mem: &MemCtx, + ) -> Result<(), NvmeError> { // Admin CQ uses interrupt vector 0 (See NVMe 1.0e Section 3.1.9 ACQ) self.create_cq( - queue::ADMIN_QUEUE_ID, + queue::CreateParams { + id: queue::ADMIN_QUEUE_ID, + base: GuestAddr(self.ctrl.admin_cq_base), + // Convert from 0's based + size: u32::from(self.ctrl.aqa.acqs()) + 1, + }, + true, 0, - GuestAddr(self.ctrl.admin_cq_base), - // Convert from 0's based - u32::from(self.ctrl.aqa.acqs()) + 1, + nvme, mem, )?; self.create_sq( + queue::CreateParams { + id: queue::ADMIN_QUEUE_ID, + base: GuestAddr(self.ctrl.admin_sq_base), + // Convert from 0's based + size: u32::from(self.ctrl.aqa.asqs()) + 1, + }, + true, queue::ADMIN_QUEUE_ID, - queue::ADMIN_QUEUE_ID, - GuestAddr(self.ctrl.admin_sq_base), - // Convert from 0's based - u32::from(self.ctrl.aqa.asqs()) + 1, + nvme, mem, )?; Ok(()) } - /// Creates and stores a new completion queue ([`CompQueue`]) for the controller. + /// Creates a new [completion queue](CompQueue) for the controller. /// - /// The specified `cqid` must not already be in use by another completion queue. + /// The CQ ID must not already be in use. For the admin queue, it must be + /// 0, while for IO queues, it must _not_ be 0. This is explicitly enforced + /// through the `is_admin` argument. fn create_cq( &mut self, - cqid: QueueId, + params: queue::CreateParams, + is_admin: bool, iv: u16, - base: GuestAddr, - size: u32, + nvme: &PciNvme, mem: &MemCtx, ) -> Result, NvmeError> { + let cqid = params.id; if (cqid as usize) >= MAX_NUM_QUEUES { return Err(NvmeError::InvalidCompQueue(cqid)); } + if is_admin { + // Creating admin queue(s) with wrong ID is programmer error + assert_eq!(cqid, 0); + } else if cqid == 0 { + // Guest requests to create an IO queue with the ID belonging to the + // admin queue is explicitly disallowed. + return Err(NvmeError::InvalidCompQueue(cqid)); + } if self.cqs[cqid as usize].is_some() { return Err(NvmeError::CompQueueAlreadyExists(cqid)); } @@ -215,37 +240,64 @@ impl NvmeCtrl { .as_ref() .ok_or(NvmeError::MsixHdlUnavailable)? .clone(); - let cq = Arc::new(CompQueue::new(cqid, iv, size, base, msix_hdl, mem)?); + let cq = Arc::new(CompQueue::new(params, iv, msix_hdl, mem)?); self.cqs[cqid as usize] = Some(cq.clone()); + nvme.queues.set_cq_slot(cqid, Some(cq.clone())); Ok(cq) } - /// Creates and stores a new submission queue ([`SubQueue`]) for the controller. + /// Creates a new [submission queue](SubQueue) for the controller. /// - /// The specified `sqid` must not already be in use by another submission queue. - /// The corresponding completion queue specified (`cqid`) must already exist. + /// The SQ ID must not already be in use. For the admin queue, it must be + /// 0, while for IO queues, it must _not_ be 0. This is explicitly enforced + /// through the `is_admin` argument. The `cqid` to which this SQ will be + /// associated must correspond to an existing CQ. fn create_sq( &mut self, - sqid: QueueId, + params: queue::CreateParams, + is_admin: bool, cqid: QueueId, - base: GuestAddr, - size: u32, + nvme: &PciNvme, mem: &MemCtx, ) -> Result, NvmeError> { + let sqid = params.id; if (sqid as usize) >= MAX_NUM_QUEUES { return Err(NvmeError::InvalidSubQueue(sqid)); } + if is_admin { + // Creating admin queue(s) with wrong ID is programmer error + assert_eq!(sqid, 0); + // So too is associating an admin SQ to an IO CQ + assert_eq!(cqid, 0); + } else { + if sqid == 0 { + // Guest requests to create an IO queue with the ID belonging to + // the admin queue is not allowed. + return Err(NvmeError::InvalidSubQueue(cqid)); + } + if cqid == 0 { + // Guest requests to associate the to-be-created IO SQ with an + // admin CQ is not allowed. + return Err(NvmeError::InvalidCompQueue(cqid)); + } + } if self.sqs[sqid as usize].is_some() { return Err(NvmeError::SubQueueAlreadyExists(sqid)); } let cq = self.get_cq(cqid)?; - let sq = SubQueue::new(sqid, cq, size, base, mem)?; + let sq = SubQueue::new(params, cq, mem)?; self.sqs[sqid as usize] = Some(sq.clone()); + nvme.queues.set_sq_slot(sqid, Some(sq.clone())); Ok(sq) } - /// Removes the [`CompQueue`] which corresponds to the given completion queue id (`cqid`). - fn delete_cq(&mut self, cqid: QueueId) -> Result<(), NvmeError> { + /// Removes the [`CompQueue`] which corresponds to the given completion + /// queue id (`cqid`). + fn delete_cq( + &mut self, + cqid: QueueId, + nvme: &PciNvme, + ) -> Result<(), NvmeError> { if (cqid as usize) >= MAX_NUM_QUEUES || self.cqs[cqid as usize].is_none() { @@ -260,6 +312,7 @@ impl NvmeCtrl { // Remove it from the authoritative list of CQs self.cqs[cqid as usize] = None; + nvme.queues.set_cq_slot(cqid, None); Ok(()) } @@ -268,7 +321,11 @@ impl NvmeCtrl { /// **NOTE:** This only removes the SQ from our list of active SQ and there may still be /// in-flight IO requests for this SQ. But after this call, we'll no longer /// answer any new doorbell requests for this SQ. - fn delete_sq(&mut self, sqid: QueueId) -> Result<(), NvmeError> { + fn delete_sq( + &mut self, + sqid: QueueId, + nvme: &PciNvme, + ) -> Result<(), NvmeError> { if (sqid as usize) >= MAX_NUM_QUEUES || self.sqs[sqid as usize].is_none() { @@ -277,6 +334,7 @@ impl NvmeCtrl { // Remove it from the authoritative list of SQs self.sqs[sqid as usize] = None; + nvme.queues.set_sq_slot(sqid, None); Ok(()) } @@ -306,6 +364,20 @@ impl NvmeCtrl { self.get_sq(queue::ADMIN_QUEUE_ID) } + /// Perform necessary setup tasks after an IO SQ has been created. + fn io_sq_post_create(&self, nvme: &PciNvme, sq: Arc) { + let sqid = sq.id(); + assert!(sqid != 0, "attempting IO SQ setup on admin SQ"); + sq.update_params(self.transfer_params()); + nvme.block_attach.queue_associate( + queue::sqid_to_block_qid(sqid), + requests::NvmeBlockQueue::new( + sq, + nvme.pci_state.acc_mem.child(Some(format!("SubQueue-{sqid}"))), + ), + ); + } + /// Configure Controller fn configure(&mut self, cc: Configuration) -> Result<(), NvmeError> { let mut inner = || { @@ -362,9 +434,13 @@ impl NvmeCtrl { } /// Get the controller in a state ready to process requests - fn enable(&mut self, mem: &MemCtx) -> Result<(), NvmeError> { + fn enable( + &mut self, + nvme: &PciNvme, + mem: &MemCtx, + ) -> Result<(), NvmeError> { // Create the Admin Queues - self.create_admin_queues(mem)?; + self.create_admin_queues(nvme, mem)?; Ok(()) } @@ -380,15 +456,29 @@ impl NvmeCtrl { /// The controller shall ensure that there is no data loss for commands /// that have had corresponding completion queue entries posted to an I/O /// Completion Queue prior to the reset operation. - fn reset(&mut self) { + fn reset(&mut self, nvme: &PciNvme) { + // Immediately mark the controller as disabled to prevent any inbound + // doorbells from being accepted on the queues we are about to delete. + nvme.is_enabled.store(false, Ordering::Release); + // Remove our references to the Qs which should be the only strong refs // at this point. Any in-flight I/O commands will just implicitly be // aborted once they try to issue their completions. - for sq in &mut self.sqs { - *sq = None; + for (sqid, state_slot) in self.sqs.iter_mut().enumerate() { + if let Some(_sq) = state_slot.take() { + let sqid = sqid as QueueId; + if sqid != queue::ADMIN_QUEUE_ID { + // TODO: cancel any existing requests? + nvme.block_attach + .queue_dissociate(queue::sqid_to_block_qid(sqid)); + } + nvme.queues.set_sq_slot(sqid, None); + } } - for cq in &mut self.cqs { - *cq = None; + for (cqid, state_slot) in self.cqs.iter_mut().enumerate() { + if let Some(_cq) = state_slot.take() { + nvme.queues.set_cq_slot(cqid as QueueId, None); + } } // Clear the CC & CSTS registers @@ -400,23 +490,16 @@ impl NvmeCtrl { // and thus don't need to do anything on reset } - /// Convert some number of logical blocks to bytes with the currently active LBA data size - fn nlb_to_size(&self, b: usize) -> usize { - b << (self.ns_ident.lbaf[(self.ns_ident.flbas & 0xF) as usize]).lbads - } - - /// Check a given request transfer size against any MDTS configured on the - /// controller. - /// - /// Returns `true` if the size is acceptable for the MDTS. - fn valid_for_mdts(&self, sz: u64) -> bool { - match self.ctrl_ident.mdts { - 0 => true, - mdts => { - let limit = (self.ctrl.cap.mpsmin_sz() as u64) << mdts; - sz <= limit - } - } + /// Calculate parameters for Submission Queue data transfer, derived from + /// the LBA configuration as well as MDTS + fn transfer_params(&self) -> queue::TransferParams { + let lba_data_size = 1u64 + << (self.ns_ident.lbaf[(self.ns_ident.flbas & 0xF) as usize]).lbads; + let max_data_tranfser_size = match self.ctrl_ident.mdts { + 0 => u64::MAX, + mdts => (self.ctrl.cap.mpsmin_sz() as u64) << mdts, + }; + queue::TransferParams { lba_data_size, max_data_tranfser_size } } fn update_block_info(&mut self, info: block::DeviceInfo) { @@ -429,6 +512,13 @@ impl NvmeCtrl { ..self.ns_ident }; self.ns_ident.lbaf[0].lbads = info.block_size.trailing_zeros() as u8; + + // Communicate new parameters to SQs + let params = self.transfer_params(); + self.sqs + .iter() + .filter_map(Option::as_ref) + .for_each(|sq| sq.update_params(params)); } fn export(&self) -> migrate::NvmeCtrlV1 { @@ -449,8 +539,15 @@ impl NvmeCtrl { fn import( &mut self, state: migrate::NvmeCtrlV1, + nvme: &PciNvme, mem: &MemCtx, ) -> Result<(), MigrateStateError> { + // TODO: verify that controller state is consistent with SQ/CQs defined + // in the payload + + // If any queues exist, clear them out first through a reset. + self.reset(nvme); + // TODO: bitstruct doesn't have a validation routine? self.ctrl.cap.0 = state.cap; self.ctrl.cc.0 = state.cc; @@ -460,43 +557,161 @@ impl NvmeCtrl { self.ctrl.admin_cq_base = state.acq_base; self.ctrl.admin_sq_base = state.asq_base; - for cq in state.cqs { - self.create_cq(cq.id, cq.iv, GuestAddr(cq.base), cq.size, mem) - .map_err(|e| { - MigrateStateError::ImportFailed(format!( - "NVMe: failed to create CQ: {}", - e - )) - })? - .import(cq)?; + for cqi in state.cqs { + let is_admin_queue = cqi.id == 0; + self.create_cq( + queue::CreateParams { + id: cqi.id, + base: GuestAddr(cqi.base), + size: cqi.size, + }, + is_admin_queue, + cqi.iv, + nvme, + mem, + ) + .map_err(|e| { + MigrateStateError::ImportFailed(format!( + "NVMe: failed to create CQ: {}", + e + )) + })? + .import(cqi)?; } - for sq in state.sqs { - self.create_sq(sq.id, sq.cq_id, GuestAddr(sq.base), sq.size, mem) + for sqi in state.sqs { + let is_admin_queue = sqi.id == 0; + let sq = self + .create_sq( + queue::CreateParams { + id: sqi.id, + base: GuestAddr(sqi.base), + size: sqi.size, + }, + is_admin_queue, + sqi.cq_id, + nvme, + mem, + ) .map_err(|e| { MigrateStateError::ImportFailed(format!( "NVMe: failed to create SQ: {}", e )) - })? - .import(sq)?; + })?; + sq.import(sqi)?; + + if !is_admin_queue { + self.io_sq_post_create(nvme, sq); + } } Ok(()) } } +#[derive(Default)] +struct NvmeQueues { + sqs: [Mutex>>; MAX_NUM_QUEUES], + cqs: [Mutex>>; MAX_NUM_QUEUES], +} +impl NvmeQueues { + /// Replace the contents of a [SubQueue] slot. + fn set_sq_slot(&self, sqid: QueueId, queue: Option>) { + let replace_some = queue.is_some(); + + let old = std::mem::replace( + &mut *self + .sqs + .get(sqid as usize) + .expect("sqid should be valid") + .lock() + .unwrap(), + queue, + ); + + // We should either be filling an empty slot with a new SQ (during queue + // creation) or vacating a populated slot (during queue deletion). + // + // Swapping an existing SQ for a differing one in a single step would be + // an unexpected operation. + if replace_some { + assert!(old.is_none(), "SQ slot should be empty"); + } else { + assert!(old.is_some(), "SQ slot should be occupied"); + } + } + + /// Replace the contents of a [CompQueue] slot. + fn set_cq_slot(&self, cqid: QueueId, queue: Option>) { + let replace_some = queue.is_some(); + + let old = std::mem::replace( + &mut *self + .cqs + .get(cqid as usize) + .expect("cqid should be valid") + .lock() + .unwrap(), + queue, + ); + + // Same justification in set_sq_slot() above applies to CQs as well + if replace_some { + assert!(old.is_none(), "CQ slot should be empty"); + } else { + assert!(old.is_some(), "CQ slot should be occupied"); + } + } + + /// Get the slot guard for a given `sqid`, but only if that slot is already + /// occupied by a [SubQueue]. + /// + /// (A returned `Some(guard)` implies `guard.is_some()`) + fn get_sq( + &self, + sqid: QueueId, + ) -> Option>>> { + let guard = self.sqs.get(sqid as usize)?.lock().unwrap(); + guard.is_some().then_some(guard) + } + + /// Get the slot guard for a given `cqid`, but only if that slot is already + /// occupied by a [CompQueue]. + /// + /// (A returned `Some(guard)` implies `guard.is_some()`) + fn get_cq( + &self, + cqid: QueueId, + ) -> Option>>> { + let guard = self.cqs.get(cqid as usize)?.lock().unwrap(); + guard.is_some().then_some(guard) + } +} + /// NVMe over PCIe pub struct PciNvme { /// NVMe Controller state: Mutex, + /// Duplicate of the controller-enabled state, but not requiring locking + /// [NvmeCtrl] to read. It is used to gate per-queue doorbell accesses + /// without stacking them up behind one central lock. + is_enabled: AtomicBool, + + /// Access to NVMe Submission and Completion queues. + /// + /// These are protected with per-slot (queue ID) locks, so actions taken on + /// a single queue will not contend with others. The queue references + /// contained within are kept in sync with those housed in the [NvmeCtrl] + /// state. + queues: NvmeQueues, + /// PCI device state pci_state: pci::DeviceState, - block_attach: block::DeviceAttachment, - - block_tracking: block::tracking::Tracking, + /// Block attachment point + pub block_attach: block::DeviceAttachment, /// Logger resource log: slog::Logger, @@ -605,14 +820,27 @@ impl PciNvme { .add_cap_msix(pci::BarN::BAR4, NVME_MSIX_COUNT) .finish(); - Arc::new_cyclic(|weak| PciNvme { - state: Mutex::new(state), - pci_state, - block_attach: block::DeviceAttachment::new(), - block_tracking: block::tracking::Tracking::new( - weak.clone() as Weak - ), - log, + let block_attach = block::DeviceAttachment::new( + NonZeroUsize::new(MAX_NUM_IO_QUEUES).unwrap(), + pci_state.acc_mem.child(Some("block backend".to_string())), + ); + + Arc::new_cyclic(move |self_weak: &Weak| { + let this = self_weak.clone(); + block_attach.on_attach(Box::new(move |info| { + if let Some(this) = Weak::upgrade(&this) { + this.state.lock().unwrap().update_block_info(info); + } + })); + + PciNvme { + state: Mutex::new(state), + is_enabled: AtomicBool::new(false), + pci_state, + queues: NvmeQueues::default(), + block_attach, + log, + } }) } @@ -633,7 +861,7 @@ impl PciNvme { let mem = mem.ok_or(NvmeError::MemoryInaccessible)?; // Get the controller ready to service requests - if let Err(e) = state.enable(&mem) { + if let Err(e) = state.enable(self, &mem) { // Couldn't enable controller, set Controller Fail Status state.ctrl.csts.set_cfs(true); return Err(e); @@ -642,12 +870,13 @@ impl PciNvme { // Set CC.EN=1 and CSTS.RDY=1 state.ctrl.cc.set_enabled(true); state.ctrl.csts.set_ready(true); + self.is_enabled.store(true, Ordering::Release); } } else if !new.enabled() && cur.enabled() { slog::debug!(self.log, "Disabling controller"); // Reset controller state which will set CC.EN=0 and CSTS.RDY=0 - state.reset(); + state.reset(self); } let shutdown = new.shn() != ShutdownNotification::None; @@ -807,7 +1036,7 @@ impl PciNvme { // We may have skipped pulling entries off the admin sq // due to no available completion entry permit, so just // kick it here again in case. - if admin_cq.kick() { + if admin_cq.kick().is_some() { let admin_sq = state.get_admin_sq()?; self.process_admin_queue(state, admin_sq)?; } @@ -822,61 +1051,111 @@ impl PciNvme { // See NVMe 1.0e Section 3.1.10 & 3.1.11 // // But note that we only support CAP.DSTRD = 0 + // + // NOTE: Normally the `wo.offset()` would be relative to the + // beginning of the RegMap-ed register, but writes to the + // doorbells have a special fast path via PciNvme::bar_write(). let off = wo.offset() - 0x1000; + let is_cq = (off >> 2) & 0b1 == 0b1; let qid = if is_cq { (off - 4) >> 3 } else { off >> 3 }; - // Queue IDs should be 16-bit and we know `off <= CONTROLLER_REG_SZ (0x4000)` + // Queue IDs should be 16-bit and we know + // `off <= CONTROLLER_REG_SZ (0x4000)` let qid = qid.try_into().unwrap(); - let state = self.state.lock().unwrap(); - if !state.ctrl.cc.enabled() { - slog::warn!( - self.log, - "Doorbell write while controller is disabled" - ); - return Err(if is_cq { - NvmeError::InvalidCompQueue(qid) - } else { - NvmeError::InvalidSubQueue(qid) - }); - } - // 32-bit register but ignore reserved top 16-bits let val = wo.read_u32() as u16; + probes::nvme_doorbell!(|| ( off as u64, qid, u8::from(is_cq), val )); - if is_cq { - // Completion Queue y Head Doorbell - let cq = state.get_cq(qid)?; - cq.notify_head(val)?; - } else { - // Submission Queue y Tail Doorbell - let sq = state.get_sq(qid)?; - sq.notify_tail(val)?; - } - - // Poke block backend to service new requests - // - // This is done for CQs in addition to SQs, since we may have - // skipped pulling entries off some SQ due to CQs having no - // available entry slots. - // - // The state lock must be released before issuing the - // notification to avoid deadlocking with the block backend - // attempting to fetch new requests. - drop(state); - self.block_attach.notify(); + self.ring_doorbell(qid, is_cq, val)?; } } Ok(()) } + /// Perform the actual work of a doorbell ring + fn ring_doorbell( + &self, + qid: QueueId, + is_cq: bool, + val: u16, + ) -> Result<(), NvmeError> { + if !self.is_enabled.load(Ordering::Acquire) { + slog::debug!( + self.log, + "Doorbell write while controller is disabled" + ); + return Err(if is_cq { + NvmeError::InvalidCompQueue(qid) + } else { + NvmeError::InvalidSubQueue(qid) + }); + } + + let to_notify = if is_cq { + // Completion Queue y Head Doorbell + let guard = self + .queues + .get_cq(qid) + .ok_or(NvmeError::InvalidCompQueue(qid))?; + let cq = guard.as_ref().unwrap(); + + cq.notify_head(val)?; + + // If this CQ was previously full, SQs may have become + // corked while trying to get permits. Notify them that + // there may now be capacity. + let to_notify = cq.kick(); + + // Query the number of entries to notify the block layer about on + // these now-uncorked SQIDs. + // + // Do this without holding on to the lock for the CQ slot. + drop(guard); + to_notify.map(|notify_sqids| { + notify_sqids + .into_iter() + .filter_map(|sqid| { + let sq_guard = self.queues.get_sq(sqid)?; + let sq = sq_guard.as_ref().unwrap(); + Some((sqid, sq.num_occupied())) + }) + .collect::>() + }) + } else { + // Submission Queue y Tail Doorbell + let guard = self + .queues + .get_sq(qid) + .ok_or(NvmeError::InvalidSubQueue(qid))?; + let sq = guard.as_ref().unwrap(); + + let num_occupied = sq.notify_tail(val)?; + Some(vec![(qid, num_occupied)]) + }; + + // Flush any notifications to SQs which were the result of the + // doorbell, now that we've dropped all involved locks. + if let Some(sqids) = to_notify { + for (sqid, occupied) in + sqids.into_iter().filter(|(id, _)| *id != queue::ADMIN_QUEUE_ID) + { + self.block_attach.notify( + queue::sqid_to_block_qid(sqid), + NonZeroUsize::new(occupied as usize), + ); + } + } + Ok(()) + } + /// Process any new entries in the Admin Submission Queue fn process_admin_queue( &self, @@ -908,10 +1187,10 @@ impl PciNvme { let comp = match cmd { AdminCmd::Abort(cmd) => state.acmd_abort(&cmd), AdminCmd::CreateIOCompQ(cmd) => { - state.acmd_create_io_cq(&cmd, &mem) + state.acmd_create_io_cq(&cmd, self, &mem) } AdminCmd::CreateIOSubQ(cmd) => { - state.acmd_create_io_sq(&cmd, &mem) + state.acmd_create_io_sq(&cmd, self, &mem) } AdminCmd::GetLogPage(cmd) => { state.acmd_get_log_page(&cmd, &mem) @@ -919,8 +1198,12 @@ impl PciNvme { AdminCmd::Identify(cmd) => state.acmd_identify(&cmd, &mem), AdminCmd::GetFeatures(cmd) => state.acmd_get_features(&cmd), AdminCmd::SetFeatures(cmd) => state.acmd_set_features(&cmd), - AdminCmd::DeleteIOCompQ(cqid) => state.acmd_delete_io_cq(cqid), - AdminCmd::DeleteIOSubQ(sqid) => state.acmd_delete_io_sq(sqid), + AdminCmd::DeleteIOCompQ(cqid) => { + state.acmd_delete_io_cq(cqid, self) + } + AdminCmd::DeleteIOSubQ(sqid) => { + state.acmd_delete_io_sq(sqid, self) + } AdminCmd::AsyncEventReq => { // async event requests do not appear to be an optional // feature but are not yet supported. The only @@ -970,12 +1253,12 @@ impl pci::Device for PciNvme { } }; - if rwo.offset() >= CONTROLLER_REGS.1 { + if rwo.offset() >= CONTROLLER_REGS.db_offset { // This is an I/O DoorBell op, so skip RegMaps's process f(&CtrlrReg::IOQueueDoorBells, rwo); } else { // Otherwise deal with every other register as normal - CONTROLLER_REGS.0.process(&mut rwo, f) + CONTROLLER_REGS.map.process(&mut rwo, f) } } @@ -1015,7 +1298,7 @@ impl MigrateMulti for PciNvme { let input: migrate::NvmeCtrlV1 = offer.take()?; let mut ctrl = self.state.lock().unwrap(); - ctrl.import(input, ctx.mem)?; + ctrl.import(input, self, ctx.mem)?; drop(ctrl); MigrateMulti::import(&self.pci_state, offer, ctx)?; @@ -1031,7 +1314,7 @@ impl Lifecycle for PciNvme { fn reset(&self) { let mut ctrl = self.state.lock().unwrap(); - ctrl.reset(); + ctrl.reset(self); self.pci_state.reset(self); } @@ -1044,7 +1327,7 @@ impl Lifecycle for PciNvme { } fn paused(&self) -> BoxFuture<'static, ()> { - Box::pin(self.block_tracking.none_outstanding()) + Box::pin(self.block_attach.none_processing()) } fn migrate(&self) -> Migrator<'_> { @@ -1149,8 +1432,12 @@ enum CtrlrReg { /// See NVMe 1.0e Section 2.1.10 Offset 10h: MLBAR (BAR0) - Memory Register Base Address, lower 32 bits const CONTROLLER_REG_SZ: usize = 0x4000; +struct CtrlRegs { + map: RegMap, + db_offset: usize, +} lazy_static! { - static ref CONTROLLER_REGS: (RegMap, usize) = { + static ref CONTROLLER_REGS: CtrlRegs = { let mut layout = [ (CtrlrReg::CtrlrCaps, 8), (CtrlrReg::Version, 4), @@ -1185,10 +1472,13 @@ lazy_static! { .map(|&(_, sz)| sz) .sum(); - (RegMap::create_packed( - CONTROLLER_REG_SZ, - &layout, - Some(CtrlrReg::Reserved), - ), db_offset) + CtrlRegs { + map: RegMap::create_packed( + CONTROLLER_REG_SZ, + &layout, + Some(CtrlrReg::Reserved), + ), + db_offset, + } }; } diff --git a/lib/propolis/src/hw/nvme/queue.rs b/lib/propolis/src/hw/nvme/queue.rs index 20ee87962..701be4b9e 100644 --- a/lib/propolis/src/hw/nvme/queue.rs +++ b/lib/propolis/src/hw/nvme/queue.rs @@ -2,13 +2,15 @@ // License, v. 2.0. If a copy of the MPL was not distributed with this // file, You can obtain one at https://mozilla.org/MPL/2.0/. -use std::collections::HashMap; -use std::fmt::Debug; +use std::collections::{HashMap, HashSet}; +use std::fmt::{self, Debug}; use std::mem::size_of; +use std::sync::atomic::{AtomicU16, Ordering}; use std::sync::{Arc, Mutex, MutexGuard, Weak}; use super::bits::{CompletionQueueEntry, SubmissionQueueEntry}; use super::cmds::Completion; +use crate::block; use crate::common::*; use crate::hw::pci; use crate::migrate::MigrateStateError; @@ -65,17 +67,17 @@ struct CompQueueState { /// See NVMe 1.0e Section 4.5 Completion Queue Entry - Phase Tag (P) phase: bool, - /// Whether the CQ should kick its SQs due to no permits being available - /// previously. - /// - /// One may only pop something off the SQ if there's at least one space - /// available in the corresponding CQ. If there isn't, we set the kick flag. - kick: bool, + /// Collection of Submission Queues (IDs) which issue their completions to + /// this CQ, and which became "corked": unable to acquire a permit while + /// attempting to process a pending entry. + corked: HashSet, } /// Submission Queue State #[derive(Debug)] -struct SubQueueState(); +struct SubQueueState { + params: TransferParams, +} /// Helper for manipulating Completion/Submission Queues /// @@ -182,18 +184,19 @@ impl QueueGuard<'_, QS> { || (state.head == 0 && state.tail == (*self.size - 1) as u16) } - fn head(&self) -> u16 { - self.state.head + /// How many queue entries are currently occupied? + fn num_occupied(&self) -> u16 { + wrap_sub(*self.size, self.state.tail, self.state.head) } - /// Helper method to calculate a positive offset for a given index, wrapping at - /// the size of the queue. + /// Helper method to calculate a positive offset for a given index, wrapping + /// at the size of the queue. fn idx_add(&self, idx: u16, off: u16) -> u16 { wrap_add(*self.size, idx, off) } - /// Helper method to calculate a negative offset for a given index, wrapping at - /// the size of the queue. + /// Helper method to calculate a negative offset for a given index, wrapping + /// at the size of the queue. fn idx_sub(&self, idx: u16, off: u16) -> u16 { wrap_sub(*self.size, idx, off) } @@ -246,13 +249,13 @@ impl QueueGuard<'_, CompQueueState> { Ok(()) } - fn take_avail(&mut self) -> bool { + fn take_avail(&mut self, sq: &Arc) -> bool { if let Some(avail) = self.state.inner.avail.checked_sub(1) { self.state.inner.avail = avail; true } else { // Make sure we kick the SQs when we have space available again - self.state.inner.kick = true; + self.record_corked(sq); false } } @@ -269,8 +272,18 @@ impl QueueGuard<'_, CompQueueState> { } } - fn kick(&mut self) -> bool { - std::mem::replace(&mut self.state.inner.kick, false) + /// Record an SQ as being corked on this CQ due to lack of permit capacity. + fn record_corked(&mut self, sq: &Arc) { + self.state.inner.corked.insert(sq.id); + } + + /// Get list of SQ IDs which were corked on this CQ + fn kick(&mut self) -> Option> { + if !self.state.inner.corked.is_empty() { + Some(self.state.inner.corked.drain().collect()) + } else { + None + } } } @@ -285,7 +298,7 @@ impl CompQueueState { // since the host side (VM) will create all the Completion Queue // entries with the phase initially zeroed out. phase: true, - kick: false, + corked: HashSet::new(), }, ) } @@ -294,7 +307,7 @@ impl CompQueueState { impl SubQueueState { /// Create a new `QueueState` for a Submission Queue fn new(size: u32) -> QueueState { - QueueState::new(size, SubQueueState()) + QueueState::new(size, SubQueueState { params: Default::default() }) } } impl QueueGuard<'_, SubQueueState> { @@ -309,12 +322,13 @@ impl QueueGuard<'_, SubQueueState> { /// If the queue is empty this method returns [`None`]. /// Otherwise, this method returns the current Head entry pointer and then /// increments the Head entry pointer by 1 (wrapping if necessary). - fn pop_head(&mut self) -> Option { + fn pop_head(&mut self, last_head: &AtomicU16) -> Option { if self.is_empty() { return None; } else { let old_head = self.state.head; self.state.head = self.idx_add(old_head, 1); + last_head.store(self.state.head, Ordering::Release); Some(old_head) } } @@ -367,6 +381,14 @@ pub enum QueueUpdateError { TooManyEntries, } +/// Basic parameters for Submission & Completion Queue creation +#[derive(Copy, Clone)] +pub struct CreateParams { + pub id: QueueId, + pub base: GuestAddr, + pub size: u32, +} + /// Type for manipulating Submission Queues. #[derive(Debug)] pub struct SubQueue { @@ -379,6 +401,10 @@ pub struct SubQueue { /// Queue state such as the size and current head/tail entry pointers. state: QueueState, + /// Duplicate of head pointer value from inside [SubQueueState], kept in + /// sync for lockless access during [SubQueue::annotate_completion()] calls. + cur_head: AtomicU16, + /// The [`GuestAddr`] at which the Queue is mapped. base: GuestAddr, } @@ -395,16 +421,21 @@ impl SubQueue { /// Create a Submission Queue object backed by the guest memory at the /// given base address. pub fn new( - id: QueueId, + params: CreateParams, cq: Arc, - size: u32, - base: GuestAddr, mem: &MemCtx, ) -> Result, QueueCreateErr> { - use std::collections::hash_map::Entry; + let CreateParams { id, base, size } = params; Self::validate(id, base, size, mem)?; - let sq = - Arc::new(Self { id, cq, state: SubQueueState::new(size), base }); + let sq = Arc::new(Self { + id, + cq, + state: SubQueueState::new(size), + cur_head: AtomicU16::new(0), + base, + }); + + use std::collections::hash_map::Entry; // Associate this SQ with the given CQ let mut cq_sqs = sq.cq.sqs.lock().unwrap(); match cq_sqs.entry(id) { @@ -420,9 +451,15 @@ impl SubQueue { } /// Attempt to move the Tail entry pointer forward to the given index. - pub fn notify_tail(&self, idx: u16) -> Result<(), QueueUpdateError> { + pub fn notify_tail(&self, idx: u16) -> Result { let mut state = self.state.lock(); - state.push_tail_to(idx) + state.push_tail_to(idx)?; + + Ok(state.num_occupied()) + } + + pub fn num_occupied(&self) -> u16 { + self.state.lock().num_occupied() } /// Returns the next entry off of the Queue or [`None`] if it is empty. @@ -433,7 +470,7 @@ impl SubQueue { // Attempt to reserve an entry on the Completion Queue let permit = self.cq.reserve_entry(&self)?; let mut state = self.state.lock(); - if let Some(idx) = state.pop_head() { + if let Some(idx) = state.pop_head(&self.cur_head) { let addr = self.base.offset::(idx as usize); let ent = mem.read::(addr); // XXX: handle a guest addr that becomes unmapped later @@ -453,11 +490,17 @@ impl SubQueue { self.id } + pub(super) fn update_params(&self, params: TransferParams) { + self.state.lock().state.inner.params = params; + } + pub(super) fn params(&self) -> TransferParams { + self.state.lock().state.inner.params + } + /// Annotate a CQE with data (ID and head index) from this SQ fn annotate_completion(&self, cqe: &mut CompletionQueueEntry) { - let state = self.state.lock(); cqe.sqid = self.id; - cqe.sqhd = state.head(); + cqe.sqhd = self.cur_head.load(Ordering::Acquire); } /// Validates whether the given parameters may be used to create a @@ -542,13 +585,12 @@ impl CompQueue { /// Creates a Completion Queue object backed by the guest memory at the /// given base address. pub fn new( - id: QueueId, + params: CreateParams, iv: u16, - size: u32, - base: GuestAddr, hdl: pci::MsixHdl, mem: &MemCtx, ) -> Result { + let CreateParams { id, base, size } = params; Self::validate(id, base, size, mem)?; Ok(Self { id, @@ -562,7 +604,8 @@ impl CompQueue { /// Attempt to move the Head entry pointer forward to the given index. pub fn notify_head(&self, idx: u16) -> Result<(), QueueUpdateError> { - self.state.lock().pop_head_to(idx) + let mut state = self.state.lock(); + state.pop_head_to(idx) } /// Fires an interrupt to the guest with the associated interrupt vector @@ -574,11 +617,11 @@ impl CompQueue { } } - /// Returns whether the SQs should be kicked due to no permits being + /// Returns whether the SQIDs should be kicked due to no permits being /// available previously. /// /// If the value was true, it will also get reset to false. - pub fn kick(&self) -> bool { + pub fn kick(&self) -> Option> { self.state.lock().kick() } @@ -596,7 +639,7 @@ impl CompQueue { sq: &Arc, ) -> Option { let mut state = self.state.lock(); - if state.take_avail() { + if state.take_avail(sq) { Some(ProtoPermit::new(self, sq)) } else { // No more spots available. @@ -605,14 +648,8 @@ impl CompQueue { } /// Add a new entry to the Completion Queue while consuming a `Permit`. - fn push( - &self, - comp: Completion, - permit: Permit, - sq: &SubQueue, - mem: &MemCtx, - ) { - let mut cqe = CompletionQueueEntry::new(comp, permit.cid); + fn push(&self, comp: Completion, cid: u16, sq: &SubQueue, mem: &MemCtx) { + let mut cqe = CompletionQueueEntry::new(comp, cid); sq.annotate_completion(&mut cqe); let mut guard = self.state.lock(); @@ -733,7 +770,7 @@ impl ProtoPermit { sq: self.sq, sqid: self.sqid, cid, - completed: false, + _nodrop: NoDropPermit, } } @@ -751,7 +788,6 @@ impl ProtoPermit { /// A permit reserving capacity to push a [CompletionQueueEntry] into a /// Completion Queue for a command submitted to the device. -#[derive(Debug)] pub struct Permit { /// The corresponding Completion Queue for which we have a permit. cq: Weak, @@ -766,30 +802,28 @@ pub struct Permit { /// Completion Queue Entry. cid: u16, - /// Track that `complete()` was actually called - completed: bool, + /// Marker to ensure holder calls [Permit::complete()]. + _nodrop: NoDropPermit, } impl Permit { /// Consume the permit by placing an entry into the Completion Queue. - pub fn complete(mut self, comp: Completion, mem: Option<&MemCtx>) { - assert!(!self.completed); - self.completed = true; + pub fn complete(self, comp: Completion, mem: Option<&MemCtx>) { + let Permit { cq, sq, cid, _nodrop, .. } = self; + std::mem::forget(_nodrop); - let cq = match self.cq.upgrade() { + let cq = match cq.upgrade() { Some(cq) => cq, None => { // The CQ has since been deleted so no way to complete this // request nor to return the permit. - debug_assert!(self.sq.upgrade().is_none()); + debug_assert!(sq.upgrade().is_none()); return; } }; - if let (Some(sq), Some(mem)) = (self.sq.upgrade(), mem) { - cq.push(comp, self, &sq, mem); - - // TODO: should this be done here? + if let (Some(sq), Some(mem)) = (sq.upgrade(), mem) { + cq.push(comp, cid, &sq, mem); cq.fire_interrupt(); } else { // The SQ has since been deleted (so the request has already @@ -820,10 +854,12 @@ impl Permit { /// only to be used for excercising the Submission and Completion Queues in /// unit tests. #[cfg(test)] - fn test_complete(mut self, sq: &SubQueue, mem: &MemCtx) { - self.completed = true; - if let Some(cq) = self.cq.upgrade() { - cq.push(Completion::success(), self, sq, mem); + fn test_complete(self, sq: &SubQueue, mem: &MemCtx) { + let Permit { cq, cid, _nodrop, .. } = self; + std::mem::forget(_nodrop); + + if let Some(cq) = cq.upgrade() { + cq.push(Completion::success(), cid, sq, mem); } } @@ -831,20 +867,44 @@ impl Permit { /// drive them through to completion. Allow them to bypass the /// ensure-this-permit-is-completed check in [`Drop`]. #[cfg(test)] - fn ignore(mut self) -> Self { - self.completed = true; - self + fn ignore(self) { + let Permit { _nodrop, .. } = self; + std::mem::forget(_nodrop); + } +} +impl Debug for Permit { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("Permit") + .field("sq", &self.sq) + .field("cq", &self.cq) + .field("sqid", &self.sqid) + .field("cid", &self.cid) + .finish() } } -impl Drop for Permit { + +/// Marker struct to ensure that [Permit] consumers call +/// [complete()](Permit::complete()), rather than silently dropping it. +struct NoDropPermit; +impl Drop for NoDropPermit { fn drop(&mut self) { - assert!( - self.completed, - "permit was dropped without calling complete()" - ); + panic!("Permit should be complete()-ed before drop"); } } +/// Convert IO SQID to block-layer [block::QueueId] +pub fn sqid_to_block_qid(sqid: super::QueueId) -> block::QueueId { + // With the admin SQ occupying ID 0, the corresponding block-layer queue IDs + // are offset by 1 + sqid.checked_sub(1).expect("IO SQID is non-zero").into() +} + +#[derive(Copy, Clone, Debug, Default)] +pub struct TransferParams { + pub lba_data_size: u64, + pub max_data_tranfser_size: u64, +} + pub(super) mod migrate { use serde::{Deserialize, Serialize}; @@ -893,48 +953,74 @@ mod test { let hdl = pci::MsixHdl::new_test(); let read_base = GuestAddr(0); let write_base = GuestAddr(1024 * 1024); + let tmpl = + CreateParams { id: ADMIN_QUEUE_ID, base: write_base, size: 0 }; let acc_mem = machine.acc_mem.child(None); let mem = acc_mem.access().unwrap(); // Admin queues must be less than 4K let cq = CompQueue::new( - ADMIN_QUEUE_ID, + CreateParams { id: ADMIN_QUEUE_ID, size: 1024, ..tmpl }, 0, - 1024, - write_base, hdl.clone(), &mem, ); assert!(matches!(cq, Ok(_))); let cq = CompQueue::new( - ADMIN_QUEUE_ID, + CreateParams { id: ADMIN_QUEUE_ID, size: 5 * 1024, ..tmpl }, 0, - 5 * 1024, - write_base, hdl.clone(), &mem, ); assert!(matches!(cq, Err(QueueCreateErr::InvalidSize))); // I/O queues must be less than 64K - let cq = CompQueue::new(1, 0, 1024, write_base, hdl.clone(), &mem); + let cq = CompQueue::new( + CreateParams { id: 1, size: 1024, ..tmpl }, + 0, + hdl.clone(), + &mem, + ); assert!(matches!(cq, Ok(_))); - let cq = CompQueue::new(1, 0, 65 * 1024, write_base, hdl.clone(), &mem); + let cq = CompQueue::new( + CreateParams { id: 1, size: 65 * 1024, ..tmpl }, + 0, + hdl.clone(), + &mem, + ); assert!(matches!(cq, Err(QueueCreateErr::InvalidSize))); // Neither must be less than 2 - let cq = - CompQueue::new(ADMIN_QUEUE_ID, 0, 1, write_base, hdl.clone(), &mem); + let cq = CompQueue::new( + CreateParams { id: ADMIN_QUEUE_ID, size: 1, ..tmpl }, + 0, + hdl.clone(), + &mem, + ); assert!(matches!(cq, Err(QueueCreateErr::InvalidSize))); - let cq = CompQueue::new(1, 0, 1, write_base, hdl.clone(), &mem); + let cq = CompQueue::new( + CreateParams { id: 1, size: 1, ..tmpl }, + 0, + hdl.clone(), + &mem, + ); assert!(matches!(cq, Err(QueueCreateErr::InvalidSize))); // Completion Queue's must be mapped to writable memory - let cq = - CompQueue::new(ADMIN_QUEUE_ID, 0, 2, read_base, hdl.clone(), &mem); + let cq = CompQueue::new( + CreateParams { id: ADMIN_QUEUE_ID, base: read_base, size: 2 }, + 0, + hdl.clone(), + &mem, + ); assert!(matches!(cq, Err(QueueCreateErr::InvalidBaseAddr))); - let cq = CompQueue::new(1, 0, 2, read_base, hdl, &mem); + let cq = CompQueue::new( + CreateParams { id: 1, base: read_base, size: 2 }, + 0, + hdl, + &mem, + ); assert!(matches!(cq, Err(QueueCreateErr::InvalidBaseAddr))); Ok(()) @@ -953,48 +1039,71 @@ mod test { // Create corresponding CQs let admin_cq = Arc::new( CompQueue::new( - ADMIN_QUEUE_ID, + CreateParams { + id: ADMIN_QUEUE_ID, + base: write_base, + size: 1024, + }, 0, - 1024, - write_base, hdl.clone(), &mem, ) .unwrap(), ); let io_cq = Arc::new( - CompQueue::new(1, 0, 1024, write_base, hdl, &mem).unwrap(), + CompQueue::new( + CreateParams { id: 1, base: write_base, size: 1024 }, + 0, + hdl, + &mem, + ) + .unwrap(), ); // Admin queues must be less than 4K let sq = SubQueue::new( - ADMIN_QUEUE_ID, + CreateParams { id: ADMIN_QUEUE_ID, base: read_base, size: 1024 }, admin_cq.clone(), - 1024, - read_base, &mem, ); assert!(matches!(sq, Ok(_))); let sq = SubQueue::new( - ADMIN_QUEUE_ID, + CreateParams { + id: ADMIN_QUEUE_ID, + base: read_base, + size: 5 * 1024, + }, admin_cq.clone(), - 5 * 1024, - read_base, &mem, ); assert!(matches!(sq, Err(QueueCreateErr::InvalidSize))); // I/O queues must be less than 64K - let sq = SubQueue::new(1, io_cq.clone(), 1024, read_base, &mem); + let sq = SubQueue::new( + CreateParams { id: 1, base: read_base, size: 1024 }, + io_cq.clone(), + &mem, + ); assert!(matches!(sq, Ok(_))); - let sq = SubQueue::new(1, io_cq, 65 * 1024, read_base, &mem); + let sq = SubQueue::new( + CreateParams { id: 1, base: read_base, size: 65 * 1024 }, + io_cq, + &mem, + ); assert!(matches!(sq, Err(QueueCreateErr::InvalidSize))); // Neither must be less than 2 - let sq = - SubQueue::new(ADMIN_QUEUE_ID, admin_cq.clone(), 1, read_base, &mem); + let sq = SubQueue::new( + CreateParams { id: ADMIN_QUEUE_ID, base: read_base, size: 1 }, + admin_cq.clone(), + &mem, + ); assert!(matches!(sq, Err(QueueCreateErr::InvalidSize))); - let sq = SubQueue::new(1, admin_cq, 1, read_base, &mem); + let sq = SubQueue::new( + CreateParams { id: 1, base: read_base, size: 1 }, + admin_cq, + &mem, + ); assert!(matches!(sq, Err(QueueCreateErr::InvalidSize))); // Completion Queue's must be mapped to readable memory @@ -1028,10 +1137,23 @@ mod test { let mem = acc_mem.access().unwrap(); // Create our queues - let cq = - Arc::new(CompQueue::new(1, 0, 4, write_base, hdl, &mem).unwrap()); - let sq = - Arc::new(SubQueue::new(1, cq.clone(), 4, read_base, &mem).unwrap()); + let cq = Arc::new( + CompQueue::new( + CreateParams { id: 1, base: write_base, size: 4 }, + 0, + hdl, + &mem, + ) + .unwrap(), + ); + let sq = Arc::new( + SubQueue::new( + CreateParams { id: 1, base: read_base, size: 4 }, + cq.clone(), + &mem, + ) + .unwrap(), + ); // Replicate guest VM notifying us things were pushed to the SQ let mut sq_tail = 0; @@ -1095,10 +1217,23 @@ mod test { // Create our queues // Purposely make the CQ smaller to test kicks - let cq = - Arc::new(CompQueue::new(1, 0, 2, write_base, hdl, &mem).unwrap()); - let sq = - Arc::new(SubQueue::new(1, cq.clone(), 4, read_base, &mem).unwrap()); + let cq = Arc::new( + CompQueue::new( + CreateParams { id: 1, base: write_base, size: 2 }, + 0, + hdl, + &mem, + ) + .unwrap(), + ); + let sq = Arc::new( + SubQueue::new( + CreateParams { id: 1, base: read_base, size: 4 }, + cq.clone(), + &mem, + ) + .unwrap(), + ); // Replicate guest VM notifying us things were pushed to the SQ let mut sq_tail = 0; @@ -1123,7 +1258,7 @@ mod test { assert!(matches!(cq.notify_head(1), Ok(_))); // Kick should've been set in the failed pop - assert!(cq.kick()); + assert!(cq.kick().is_some()); // We should have one more space now and should be able to pop 1 more assert!(matches!( @@ -1154,10 +1289,22 @@ mod test { // space available in the CQ. let mut rng = rand::rng(); let sq_size = rng.random_range(512..2048); - let cq = - Arc::new(CompQueue::new(1, 0, 4, write_base, hdl, &mem).unwrap()); + let cq = Arc::new( + CompQueue::new( + CreateParams { id: 1, base: write_base, size: 4 }, + 0, + hdl, + &mem, + ) + .unwrap(), + ); let sq = Arc::new( - SubQueue::new(1, cq.clone(), sq_size, read_base, &mem).unwrap(), + SubQueue::new( + CreateParams { id: 1, base: read_base, size: sq_size }, + cq.clone(), + &mem, + ) + .unwrap(), ); // We'll be generating a random number of submissions @@ -1188,7 +1335,7 @@ mod test { doorbell_cq.notify_head(cq_head), Ok(_) )); - if doorbell_cq.kick() { + if doorbell_cq.kick().is_some() { assert!(workers_tx.send(()).is_ok()); } } diff --git a/lib/propolis/src/hw/nvme/requests.rs b/lib/propolis/src/hw/nvme/requests.rs index 09d341d8e..4a05afa4c 100644 --- a/lib/propolis/src/hw/nvme/requests.rs +++ b/lib/propolis/src/hw/nvme/requests.rs @@ -2,17 +2,14 @@ // License, v. 2.0. If a copy of the MPL was not distributed with this // file, You can obtain one at https://mozilla.org/MPL/2.0/. -use crate::{ - accessors::MemAccessor, - block::{ - self, tracking::CompletionCallback, Operation, Request, - Result as BlockResult, - }, - hw::nvme::{bits, cmds::Completion}, - vmm::mem::MemCtx, -}; +use std::sync::Arc; +use std::time::Instant; use super::{cmds::NvmCmd, queue::Permit, PciNvme}; +use crate::accessors::MemAccessor; +use crate::block::{self, Operation, Request}; +use crate::hw::nvme::{bits, cmds::Completion, queue::SubQueue}; +use crate::vmm::mem::MemCtx; #[usdt::provider(provider = "propolis")] mod probes { @@ -39,130 +36,108 @@ impl block::Device for PciNvme { fn attachment(&self) -> &block::DeviceAttachment { &self.block_attach } +} - fn on_attach(&self, info: block::DeviceInfo) { - self.state.lock().unwrap().update_block_info(info); - } - - fn next(&self) -> Option { - let (req, permit) = self.next_req()?; - Some(self.block_tracking.track(req, permit)) +pub(super) struct NvmeBlockQueue { + sq: Arc, + acc_mem: MemAccessor, +} +impl NvmeBlockQueue { + pub(super) fn new(sq: Arc, acc_mem: MemAccessor) -> Arc { + Arc::new(Self { sq, acc_mem }) } +} +impl block::DeviceQueue for NvmeBlockQueue { + type Token = Permit; + + /// Pop an available I/O request off of the Submission Queue for hand-off to + /// the underlying block backend + fn next_req(&self) -> Option<(Request, Self::Token, Option)> { + let sq = &self.sq; + let mem = self.acc_mem.access()?; + let params = self.sq.params(); + + while let Some((sub, permit, idx)) = sq.pop(&mem) { + let qid = sq.id(); + probes::nvme_raw_cmd!(|| { + ( + qid, + u64::from(sub.cdw0) | (u64::from(sub.nsid) << 32), + sub.prp1, + sub.prp2, + (u64::from(sub.cdw10) | (u64::from(sub.cdw11) << 32)), + ) + }); + let cid = sub.cid(); + let cmd = NvmCmd::parse(sub); + + fn fail_mdts(permit: Permit, mem: &MemCtx) { + permit.complete( + Completion::generic_err(bits::STS_INVAL_FIELD).dnr(), + Some(&mem), + ); + } - fn complete(&self, res: BlockResult, id: block::ReqId) { - let (op, permit) = self.block_tracking.complete(id, res); - self.complete_req(op, res, permit); - } + match cmd { + Ok(NvmCmd::Write(cmd)) => { + let off = params.lba_data_size * cmd.slba; + let size = params.lba_data_size * (cmd.nlb as u64); - fn on_completion(&self, cb: Box) -> bool { - self.block_tracking.set_completion_callback(cb) - } + if size > params.max_data_tranfser_size { + fail_mdts(permit, &mem); + continue; + } - fn accessor_mem(&self) -> MemAccessor { - self.pci_state.acc_mem.child(Some("block backend".to_string())) - } -} + probes::nvme_write_enqueue!(|| (qid, idx, cid, off, size)); -impl PciNvme { - /// Pop an available I/O request off of a Submission Queue to begin - /// processing by the underlying Block Device. - fn next_req(&self) -> Option<(Request, Permit)> { - let state = self.state.lock().unwrap(); - - let mem = self.mem_access()?; - - // Go through all the queues (skip admin as we just want I/O queues) - // looking for a request to service - for sq in state.sqs.iter().skip(1).flatten() { - while let Some((sub, permit, idx)) = sq.pop(&mem) { - let qid = sq.id(); - probes::nvme_raw_cmd!(|| { - ( - qid, - u64::from(sub.cdw0) | (u64::from(sub.nsid) << 32), - sub.prp1, - sub.prp2, - (u64::from(sub.cdw10) | (u64::from(sub.cdw11) << 32)), - ) - }); - let cid = sub.cid(); - let cmd = NvmCmd::parse(sub); - - fn fail_mdts(permit: Permit, mem: &MemCtx) { - permit.complete( - Completion::generic_err(bits::STS_INVAL_FIELD).dnr(), - Some(&mem), - ); + let bufs = cmd.data(size, &mem).collect(); + let req = + Request::new_write(off as usize, size as usize, bufs); + return Some((req, permit, None)); } + Ok(NvmCmd::Read(cmd)) => { + let off = params.lba_data_size * cmd.slba; + let size = params.lba_data_size * (cmd.nlb as u64); - match cmd { - Ok(NvmCmd::Write(cmd)) => { - let off = state.nlb_to_size(cmd.slba as usize) as u64; - let size = state.nlb_to_size(cmd.nlb as usize) as u64; - - if !state.valid_for_mdts(size) { - fail_mdts(permit, &mem); - continue; - } - - probes::nvme_write_enqueue!(|| ( - qid, idx, cid, off, size - )); - - let bufs = cmd.data(size, &mem).collect(); - let req = Request::new_write( - off as usize, - size as usize, - bufs, - ); - return Some((req, permit)); - } - Ok(NvmCmd::Read(cmd)) => { - let off = state.nlb_to_size(cmd.slba as usize) as u64; - let size = state.nlb_to_size(cmd.nlb as usize) as u64; - - if !state.valid_for_mdts(size) { - fail_mdts(permit, &mem); - continue; - } - - probes::nvme_read_enqueue!(|| ( - qid, idx, cid, off, size - )); - - let bufs = cmd.data(size, &mem).collect(); - let req = Request::new_read( - off as usize, - size as usize, - bufs, - ); - return Some((req, permit)); - } - Ok(NvmCmd::Flush) => { - probes::nvme_flush_enqueue!(|| (qid, idx, cid)); - let req = Request::new_flush(); - return Some((req, permit)); - } - Ok(NvmCmd::Unknown(_)) | Err(_) => { - // For any other unrecognized or malformed command, - // just immediately complete it with an error - let comp = - Completion::generic_err(bits::STS_INTERNAL_ERR); - permit.complete(comp, Some(&mem)); + if size > params.max_data_tranfser_size { + fail_mdts(permit, &mem); + continue; } + + probes::nvme_read_enqueue!(|| (qid, idx, cid, off, size)); + + let bufs = cmd.data(size, &mem).collect(); + let req = + Request::new_read(off as usize, size as usize, bufs); + return Some((req, permit, None)); + } + Ok(NvmCmd::Flush) => { + probes::nvme_flush_enqueue!(|| (qid, idx, cid)); + let req = Request::new_flush(); + return Some((req, permit, None)); + } + Ok(NvmCmd::Unknown(_)) | Err(_) => { + // For any other unrecognized or malformed command, + // just immediately complete it with an error + let comp = Completion::generic_err(bits::STS_INTERNAL_ERR); + permit.complete(comp, Some(&mem)); } } } - None } /// Place the operation result (success or failure) onto the corresponding /// Completion Queue. - fn complete_req(&self, op: Operation, res: BlockResult, permit: Permit) { + fn complete( + &self, + op: block::Operation, + result: block::Result, + permit: Self::Token, + ) { let qid = permit.sqid(); let cid = permit.cid(); - let resnum = res as u8; + let resnum = result as u8; match op { Operation::Read(..) => { probes::nvme_read_complete!(|| (qid, cid, resnum)); @@ -178,7 +153,9 @@ impl PciNvme { } } - let guard = self.mem_access(); - permit.complete(Completion::from(res), guard.as_deref()); + permit.complete( + Completion::from(result), + self.acc_mem.access().as_deref(), + ); } } diff --git a/lib/propolis/src/hw/qemu/ramfb.rs b/lib/propolis/src/hw/qemu/ramfb.rs index f8e782e05..d765a7fe2 100644 --- a/lib/propolis/src/hw/qemu/ramfb.rs +++ b/lib/propolis/src/hw/qemu/ramfb.rs @@ -46,7 +46,7 @@ lazy_static! { } #[derive(Default, Debug)] -#[repr(packed)] +#[repr(C, packed)] struct Config { addr: u64, fourcc: u32, diff --git a/lib/propolis/src/hw/virtio/block.rs b/lib/propolis/src/hw/virtio/block.rs index e2d36944d..311ffb07a 100644 --- a/lib/propolis/src/hw/virtio/block.rs +++ b/lib/propolis/src/hw/virtio/block.rs @@ -2,11 +2,12 @@ // License, v. 2.0. If a copy of the MPL was not distributed with this // file, You can obtain one at https://mozilla.org/MPL/2.0/. -use std::sync::{Arc, Weak}; +use std::num::NonZeroUsize; +use std::sync::Arc; +use std::time::Instant; use crate::accessors::MemAccessor; use crate::block; -use crate::block::tracking::CompletionCallback; use crate::common::*; use crate::hw::pci; use crate::migrate::*; @@ -27,19 +28,10 @@ const SECTOR_SZ: usize = 512; /// Arbitrary limit to sectors permitted per discard request const MAX_DISCARD_SECTORS: u32 = ((1024 * 1024) / SECTOR_SZ) as u32; -struct CompletionPayload { - /// ID of original request. - rid: u16, - /// VirtIO chain in which we indicate the result. - chain: Chain, -} - pub struct PciVirtioBlock { virtio_state: PciVirtioState, pci_state: pci::DeviceState, - - block_attach: block::DeviceAttachment, - block_tracking: block::tracking::Tracking, + pub block_attach: block::DeviceAttachment, } impl PciVirtioBlock { pub fn new(queue_size: u16) -> Arc { @@ -59,14 +51,17 @@ impl PciVirtioBlock { VIRTIO_BLK_CFG_SIZE, ); - Arc::new_cyclic(|weak| Self { - pci_state, - virtio_state, - block_attach: block::DeviceAttachment::new(), - block_tracking: block::tracking::Tracking::new( - weak.clone() as Weak - ), - }) + let block_attach = block::DeviceAttachment::new( + NonZeroUsize::new(1).unwrap(), + pci_state.acc_mem.child(Some("block backend".to_string())), + ); + let bvq = BlockVq::new( + virtio_state.queues.get(0).unwrap().clone(), + pci_state.acc_mem.child(Some("block queue".to_string())), + ); + block_attach.queue_associate(0usize.into(), bvq); + + Arc::new(Self { pci_state, virtio_state, block_attach }) } fn block_cfg_read(&self, id: &BlockReg, ro: &mut ReadOp) { @@ -112,10 +107,29 @@ impl PciVirtioBlock { } } } +} - fn next_req(&self) -> Option { - let vq = &self.virtio_state.queues[0]; - let mem = self.pci_state.acc_mem.access()?; +struct CompletionToken { + /// ID of original request. + rid: u16, + /// VirtIO chain in which we indicate the result. + chain: Chain, +} + +struct BlockVq(Arc, MemAccessor); +impl BlockVq { + fn new(vq: Arc, acc_mem: MemAccessor) -> Arc { + Arc::new(Self(vq, acc_mem)) + } +} +impl block::DeviceQueue for BlockVq { + type Token = CompletionToken; + + fn next_req( + &self, + ) -> Option<(block::Request, Self::Token, Option)> { + let vq = &self.0; + let mem = self.1.access()?; let mut chain = Chain::with_capacity(4); // Pop a request off the queue if there's one available. @@ -140,9 +154,10 @@ impl PciVirtioBlock { probes::vioblk_read_enqueue!(|| ( rid, off as u64, sz as u64 )); - Ok(self.block_tracking.track( + Ok(( block::Request::new_read(off, sz, regions), - CompletionPayload { rid, chain }, + CompletionToken { rid, chain }, + None, )) } else { Err(chain) @@ -157,9 +172,10 @@ impl PciVirtioBlock { probes::vioblk_write_enqueue!(|| ( rid, off as u64, sz as u64 )); - Ok(self.block_tracking.track( + Ok(( block::Request::new_write(off, sz, regions), - CompletionPayload { rid, chain }, + CompletionToken { rid, chain }, + None, )) } else { Err(chain) @@ -167,9 +183,10 @@ impl PciVirtioBlock { } VIRTIO_BLK_T_FLUSH => { probes::vioblk_flush_enqueue!(|| rid); - Ok(self.block_tracking.track( + Ok(( block::Request::new_flush(), - CompletionPayload { rid, chain }, + CompletionToken { rid, chain }, + None, )) } VIRTIO_BLK_T_DISCARD => { @@ -182,9 +199,10 @@ impl PciVirtioBlock { probes::vioblk_discard_enqueue!(|| ( rid, off as u64, sz as u64, )); - Ok(self.block_tracking.track( + Ok(( block::Request::new_discard(off, sz), - CompletionPayload { rid, chain }, + CompletionToken { rid, chain }, + None, )) } } @@ -205,16 +223,15 @@ impl PciVirtioBlock { } } - fn complete_req( + fn complete( &self, - rid: u16, op: block::Operation, - res: block::Result, - chain: &mut Chain, + result: block::Result, + mut token: Self::Token, ) { - let vq = self.virtio_state.queues.get(0).expect("vq must exist"); - if let Some(mem) = vq.acc_mem.access() { - let resnum = match res { + let CompletionToken { rid, ref mut chain } = token; + if let Some(mem) = self.1.access() { + let resnum = match result { block::Result::Success => VIRTIO_BLK_S_OK, block::Result::Failure => VIRTIO_BLK_S_IOERR, block::Result::ReadOnly => VIRTIO_BLK_S_IOERR, @@ -235,7 +252,7 @@ impl PciVirtioBlock { } } chain.write(&resnum, &mem); - vq.push_used(chain, &mem); + self.0.push_used(chain, &mem); } } } @@ -269,7 +286,8 @@ impl VirtioDevice for PciVirtioBlock { } fn queue_notify(&self, _vq: &Arc) { - self.block_attach.notify() + // TODO: provide proper hint + self.block_attach.notify(0usize.into(), None); } } impl PciVirtio for PciVirtioBlock { @@ -284,24 +302,6 @@ impl block::Device for PciVirtioBlock { fn attachment(&self) -> &block::DeviceAttachment { &self.block_attach } - - fn next(&self) -> Option { - self.next_req() - } - - fn complete(&self, res: block::Result, id: block::ReqId) { - let (op, mut payload) = self.block_tracking.complete(id, res); - let CompletionPayload { rid, ref mut chain } = payload; - self.complete_req(rid, op, res, chain); - } - - fn on_completion(&self, cb: Box) -> bool { - self.block_tracking.set_completion_callback(cb) - } - - fn accessor_mem(&self) -> MemAccessor { - self.pci_state.acc_mem.child(Some("block backend".to_string())) - } } impl Lifecycle for PciVirtioBlock { fn type_name(&self) -> &'static str { @@ -311,13 +311,13 @@ impl Lifecycle for PciVirtioBlock { self.virtio_state.reset(self); } fn pause(&self) { - self.block_attach.pause(); + self.block_attach.pause() } fn resume(&self) { self.block_attach.resume(); } fn paused(&self) -> BoxFuture<'static, ()> { - Box::pin(self.block_tracking.none_outstanding()) + Box::pin(self.block_attach.none_processing()) } fn migrate(&self) -> Migrator<'_> { Migrator::Multi(self) diff --git a/lib/propolis/src/lib.rs b/lib/propolis/src/lib.rs index 4a153b977..c608a816c 100644 --- a/lib/propolis/src/lib.rs +++ b/lib/propolis/src/lib.rs @@ -16,7 +16,6 @@ extern crate bitflags; pub mod accessors; pub mod api_version; -pub mod attachment; pub mod block; pub mod chardev; pub mod common;