Modify the virtio-blk to async

This commit is contained in:
LI Qing 2024-04-10 16:50:49 +08:00 committed by Tate, Hongliang Tian
parent 2616335755
commit b7131e721c
8 changed files with 263 additions and 207 deletions

View File

@ -156,3 +156,16 @@ impl Drop for DisabledLocalIrqGuard {
}
}
}
/// FIXME: The reason we need to add this API is that currently IRQs
/// are enabled when the CPU enters the user space for the first time,
/// which is too late. During the OS initialization phase,
/// we need to get the block device working and mount the filesystems,
/// thus requiring the IRQs should be enabled as early as possible.
///
/// FIXME: this method may be unsound.
pub fn enable_local() {
if !crate::arch::irq::is_local_enabled() {
crate::arch::irq::enable_local();
}
}

View File

@ -7,7 +7,9 @@ pub use handler::in_interrupt_context;
pub use trapframe::TrapFrame;
pub(crate) use self::handler::call_irq_callback_functions;
pub use self::irq::{disable_local, DisabledLocalIrqGuard, IrqCallbackFunction, IrqLine};
pub use self::irq::{
disable_local, enable_local, DisabledLocalIrqGuard, IrqCallbackFunction, IrqLine,
};
pub(crate) fn init() {
unsafe {

View File

@ -74,10 +74,6 @@ mod test {
}
impl BlockDevice for ExfatMemoryDisk {
fn handle_irq(&self) {
info!("ExfatMemoryDisk handle irq");
}
fn enqueue(&self, bio: SubmittedBio) -> core::prelude::v1::Result<(), BioEnqueueError> {
let start_device_ofs = bio.sid_range().start.to_raw() as usize * SECTOR_SIZE;
let mut cur_device_ofs = start_device_ofs;

View File

@ -360,8 +360,8 @@ impl Ext2 {
block_group.sync_metadata()?;
}
let mut bio_waiter = BioWaiter::new();
// Writes back the main superblock and group descriptor table.
let mut bio_waiter = BioWaiter::new();
let raw_super_block = RawSuperBlock::from((*super_block).deref());
bio_waiter.concat(
self.block_device
@ -371,11 +371,16 @@ impl Ext2 {
super_block.group_descriptors_bid(0),
&self.group_descriptors_segment,
)?);
bio_waiter
.wait()
.ok_or_else(|| Error::with_message(Errno::EIO, "failed to sync main metadata"))?;
drop(bio_waiter);
// Writes back the backups of superblock and group descriptor table.
let mut raw_super_block_backup = raw_super_block;
for idx in 1..super_block.block_groups_count() {
if super_block.is_backup_group(idx as usize) {
let mut bio_waiter = BioWaiter::new();
raw_super_block_backup.block_group_idx = idx as u16;
bio_waiter.concat(self.block_device.write_bytes_async(
super_block.bid(idx as usize).to_offset(),
@ -385,14 +390,12 @@ impl Ext2 {
super_block.group_descriptors_bid(idx as usize),
&self.group_descriptors_segment,
)?);
bio_waiter.wait().ok_or_else(|| {
Error::with_message(Errno::EIO, "failed to sync backup metadata")
})?;
}
}
// Waits for the completion of all submitted bios.
bio_waiter
.wait()
.ok_or_else(|| Error::with_message(Errno::EIO, "failed to sync metadata of fs"))?;
// Reset to clean.
super_block.clear_dirty();
Ok(())

View File

@ -2,7 +2,11 @@
use lru::LruCache;
use super::{block_ptr::BID_SIZE, fs::Ext2, prelude::*};
use super::{
block_ptr::{Ext2Bid, BID_SIZE},
fs::Ext2,
prelude::*,
};
/// `IndirectBlockCache` is a caching structure that stores `IndirectBlock` objects for Ext2.
///
@ -11,7 +15,7 @@ use super::{block_ptr::BID_SIZE, fs::Ext2, prelude::*};
/// for new blocks.
#[derive(Debug)]
pub struct IndirectBlockCache {
cache: LruCache<u32, IndirectBlock>,
cache: LruCache<Ext2Bid, IndirectBlock>,
fs: Weak<Ext2>,
}
@ -32,7 +36,7 @@ impl IndirectBlockCache {
/// Retrieves a reference to an `IndirectBlock` by its `bid`.
///
/// If the block is not present in the cache, it will be loaded from the disk.
pub fn find(&mut self, bid: u32) -> Result<&IndirectBlock> {
pub fn find(&mut self, bid: Ext2Bid) -> Result<&IndirectBlock> {
self.try_shrink()?;
let fs = self.fs();
@ -49,7 +53,7 @@ impl IndirectBlockCache {
/// Retrieves a mutable reference to an `IndirectBlock` by its `bid`.
///
/// If the block is not present in the cache, it will be loaded from the disk.
pub fn find_mut(&mut self, bid: u32) -> Result<&mut IndirectBlock> {
pub fn find_mut(&mut self, bid: Ext2Bid) -> Result<&mut IndirectBlock> {
self.try_shrink()?;
let fs = self.fs();
@ -64,7 +68,7 @@ impl IndirectBlockCache {
}
/// Inserts or updates an `IndirectBlock` in the cache with the specified `bid`.
pub fn insert(&mut self, bid: u32, block: IndirectBlock) -> Result<()> {
pub fn insert(&mut self, bid: Ext2Bid, block: IndirectBlock) -> Result<()> {
self.try_shrink()?;
self.cache.put(bid, block);
Ok(())
@ -72,54 +76,43 @@ impl IndirectBlockCache {
/// Removes and returns the `IndirectBlock` corresponding to the `bid`
/// from the cache or `None` if does not exist.
pub fn remove(&mut self, bid: u32) -> Option<IndirectBlock> {
pub fn remove(&mut self, bid: Ext2Bid) -> Option<IndirectBlock> {
self.cache.pop(&bid)
}
/// Evicts all blocks from the cache, persisting any with a 'Dirty' state to the disk.
pub fn evict_all(&mut self) -> Result<()> {
let mut bio_waiter = BioWaiter::new();
loop {
let Some((bid, block)) = self.cache.pop_lru() else {
break;
};
if block.is_dirty() {
bio_waiter.concat(
self.fs()
.block_device()
.write_block(Bid::new(bid as _), &block.frame)?,
);
}
}
bio_waiter.wait().ok_or_else(|| {
Error::with_message(Errno::EIO, "failed to evict_all the indirect blocks")
})?;
Ok(())
let cache_size = self.cache.len();
self.evict(cache_size)
}
/// Attempts to shrink the cache size if it exceeds the maximum allowed cache size.
/// Attempts to evict some blocks from cache if it exceeds the maximum size.
fn try_shrink(&mut self) -> Result<()> {
if self.cache.len() < Self::MAX_SIZE {
return Ok(());
}
// TODO: How to determine the number of evictions each time?
//
// FIXME: When we set it to `Self::MAX_SIZE / 2` here,
// running the `/regression/ext2.sh` test may cause a deadlock issue.
let evict_num = 1;
self.evict(evict_num)
}
/// Evicts `num` blocks from cache.
fn evict(&mut self, num: usize) -> Result<()> {
let num = num.min(self.cache.len());
let mut bio_waiter = BioWaiter::new();
for _ in 0..(Self::MAX_SIZE / 2) {
for _ in 0..num {
let (bid, block) = self.cache.pop_lru().unwrap();
if block.is_dirty() {
bio_waiter.concat(
self.fs()
.block_device()
.write_block(Bid::new(bid as _), &block.frame)?,
);
bio_waiter.concat(self.fs().write_block_async(bid, &block.frame)?);
}
}
bio_waiter.wait().ok_or_else(|| {
Error::with_message(Errno::EIO, "failed to write back the indirect block")
Error::with_message(Errno::EIO, "failed to evict the indirect blocks")
})?;
Ok(())
@ -164,16 +157,16 @@ impl IndirectBlock {
}
/// Reads a bid at a specified `idx`.
pub fn read_bid(&self, idx: usize) -> Result<u32> {
pub fn read_bid(&self, idx: usize) -> Result<Ext2Bid> {
assert!(self.state != State::Uninit);
let bid: u32 = self.frame.read_val(idx * BID_SIZE)?;
let bid: Ext2Bid = self.frame.read_val(idx * BID_SIZE)?;
Ok(bid)
}
/// Writes a value of bid at a specified `idx`.
///
/// After a successful write operation, the block's state will be marked as dirty.
pub fn write_bid(&mut self, idx: usize, bid: &u32) -> Result<()> {
pub fn write_bid(&mut self, idx: usize, bid: &Ext2Bid) -> Result<()> {
assert!(self.state != State::Uninit);
self.frame.write_val(idx * BID_SIZE, bid)?;
self.state = State::Dirty;

View File

@ -80,6 +80,9 @@ fn init_thread() {
"[kernel] Spawn init thread, tid = {}",
current_thread!().tid()
);
// FIXME: Remove this if we move the step of mounting
// the filesystems to be done within the init process.
aster_frame::trap::enable_local();
net::lazy_init();
fs::lazy_init();
// driver::pci::virtio::block::block_device_test();

View File

@ -55,7 +55,6 @@ pub const SECTOR_SIZE: usize = 512;
pub trait BlockDevice: Send + Sync + Any + Debug {
/// Enqueues a new `SubmittedBio` to the block device.
fn enqueue(&self, bio: SubmittedBio) -> Result<(), BioEnqueueError>;
fn handle_irq(&self);
}
impl dyn BlockDevice {

View File

@ -1,11 +1,10 @@
// SPDX-License-Identifier: MPL-2.0
use alloc::{boxed::Box, string::String, sync::Arc, vec, vec::Vec};
use alloc::{boxed::Box, collections::BTreeMap, string::String, sync::Arc, vec, vec::Vec};
use core::{fmt::Debug, hint::spin_loop, mem::size_of};
use aster_block::{
bio::{BioEnqueueError, BioStatus, BioType, SubmittedBio},
id::Sid,
request_queue::{BioRequest, BioRequestSingleQueue},
};
use aster_frame::{
@ -14,7 +13,7 @@ use aster_frame::{
trap::TrapFrame,
vm::{DmaDirection, DmaStream, DmaStreamSlice, VmAllocOptions, VmIo},
};
use aster_util::safe_ptr::SafePtr;
use aster_util::{id_allocator::IdAlloc, safe_ptr::SafePtr};
use log::info;
use pod::Pod;
@ -30,7 +29,7 @@ use crate::{
#[derive(Debug)]
pub struct BlockDevice {
device: DeviceInner,
device: Arc<DeviceInner>,
/// The software staging queue.
queue: BioRequestSingleQueue,
}
@ -38,91 +37,30 @@ pub struct BlockDevice {
impl BlockDevice {
/// Creates a new VirtIO-Block driver and registers it.
pub(crate) fn init(transport: Box<dyn VirtioTransport>) -> Result<(), VirtioDeviceError> {
let block_device = {
let device = DeviceInner::init(transport)?;
Self {
device,
queue: BioRequestSingleQueue::new(),
}
};
let device = DeviceInner::init(transport)?;
let device_id = device.request_device_id();
let device_id = block_device.device.device_id.clone().unwrap();
aster_block::register_device(device_id, Arc::new(block_device));
let block_device = Arc::new(Self {
device,
queue: BioRequestSingleQueue::new(),
});
aster_block::register_device(device_id, block_device);
Ok(())
}
/// Dequeues a `BioRequest` from the software staging queue and
/// processes the request.
///
/// TODO: Current read and write operations are still synchronous
/// it needs to be modified to use the queue-based asynchronous programming pattern.
pub fn handle_requests(&self) {
let request = self.queue.dequeue();
info!("Handle Request: {:?}", request);
match request.type_() {
BioType::Read => self.do_read(&request),
BioType::Write => self.do_write(&request),
BioType::Read => self.device.read(request),
BioType::Write => self.device.write(request),
BioType::Flush | BioType::Discard => todo!(),
}
}
fn do_read(&self, request: &BioRequest) {
let start_sid = request.sid_range().start;
let dma_streams: Vec<(DmaStream, usize, usize)> = request
.bios()
.flat_map(|bio| {
bio.segments().iter().map(|segment| {
let dma_stream =
DmaStream::map(segment.pages().clone(), DmaDirection::FromDevice, false)
.unwrap();
(dma_stream, segment.offset(), segment.nbytes())
})
})
.collect();
let dma_slices: Vec<DmaStreamSlice> = dma_streams
.iter()
.map(|(stream, offset, len)| DmaStreamSlice::new(stream, *offset, *len))
.collect();
self.device.read(start_sid, &dma_slices);
dma_slices.iter().for_each(|dma_slice| {
dma_slice.sync().unwrap();
});
drop(dma_slices);
drop(dma_streams);
request.bios().for_each(|bio| {
bio.complete(BioStatus::Complete);
});
}
fn do_write(&self, request: &BioRequest) {
let start_sid = request.sid_range().start;
let dma_streams: Vec<(DmaStream, usize, usize)> = request
.bios()
.flat_map(|bio| {
bio.segments().iter().map(|segment| {
let dma_stream =
DmaStream::map(segment.pages().clone(), DmaDirection::ToDevice, false)
.unwrap();
(dma_stream, segment.offset(), segment.nbytes())
})
})
.collect();
let dma_slices: Vec<DmaStreamSlice> = dma_streams
.iter()
.map(|(stream, offset, len)| DmaStreamSlice::new(stream, *offset, *len))
.collect();
self.device.write(start_sid, &dma_slices);
drop(dma_slices);
drop(dma_streams);
request.bios().for_each(|bio| {
bio.complete(BioStatus::Complete);
});
}
/// Negotiate features for the device specified bits 0~23
pub(crate) fn negotiate_features(features: u64) -> u64 {
let feature = BlockFeatures::from_bits(features).unwrap();
@ -135,33 +73,31 @@ impl aster_block::BlockDevice for BlockDevice {
fn enqueue(&self, bio: SubmittedBio) -> Result<(), BioEnqueueError> {
self.queue.enqueue(bio)
}
fn handle_irq(&self) {
info!("Virtio block device handle irq");
}
}
#[derive(Debug)]
struct DeviceInner {
config: SafePtr<VirtioBlockConfig, IoMem>,
queue: SpinLock<VirtQueue>,
transport: Box<dyn VirtioTransport>,
transport: SpinLock<Box<dyn VirtioTransport>>,
block_requests: DmaStream,
block_responses: DmaStream,
id_allocator: SpinLock<Vec<u8>>,
device_id: Option<String>,
id_allocator: SpinLock<IdAlloc>,
submitted_requests: SpinLock<BTreeMap<u16, SubmittedRequest>>,
}
impl DeviceInner {
const QUEUE_SIZE: u16 = 64;
/// Creates and inits the device.
pub fn init(mut transport: Box<dyn VirtioTransport>) -> Result<Self, VirtioDeviceError> {
pub fn init(mut transport: Box<dyn VirtioTransport>) -> Result<Arc<Self>, VirtioDeviceError> {
let config = VirtioBlockConfig::new(transport.as_mut());
let num_queues = transport.num_queues();
if num_queues != 1 {
return Err(VirtioDeviceError::QueuesAmountDoNotMatch(num_queues, 1));
}
let queue = VirtQueue::new(0, 64, transport.as_mut()).expect("create virtqueue failed");
let queue = VirtQueue::new(0, Self::QUEUE_SIZE, transport.as_mut())
.expect("create virtqueue failed");
let block_requests = {
let vm_segment = VmAllocOptions::new(1)
.is_contiguous(true)
@ -169,6 +105,7 @@ impl DeviceInner {
.unwrap();
DmaStream::map(vm_segment, DmaDirection::Bidirectional, false).unwrap()
};
assert!(Self::QUEUE_SIZE as usize * REQ_SIZE <= block_requests.nbytes());
let block_responses = {
let vm_segment = VmAllocOptions::new(1)
.is_contiguous(true)
@ -176,48 +113,92 @@ impl DeviceInner {
.unwrap();
DmaStream::map(vm_segment, DmaDirection::Bidirectional, false).unwrap()
};
let mut device = Self {
assert!(Self::QUEUE_SIZE as usize * RESP_SIZE <= block_responses.nbytes());
let device = Arc::new(Self {
config,
queue: SpinLock::new(queue),
transport,
transport: SpinLock::new(transport),
block_requests,
block_responses,
id_allocator: SpinLock::new((0..64).collect()),
device_id: None,
id_allocator: SpinLock::new(IdAlloc::with_capacity(Self::QUEUE_SIZE as usize)),
submitted_requests: SpinLock::new(BTreeMap::new()),
});
let cloned_device = device.clone();
let handle_irq = move |_: &TrapFrame| {
cloned_device.handle_irq();
};
let device_id = device.get_id();
let cloned_device_id = device_id.clone();
let handle_block_device = move |_: &TrapFrame| {
aster_block::get_device(device_id.as_str())
.unwrap()
.handle_irq();
let cloned_device = device.clone();
let handle_config_change = move |_: &TrapFrame| {
cloned_device.handle_config_change();
};
device.device_id = Some(cloned_device_id);
device
.transport
.register_cfg_callback(Box::new(config_space_change))
.unwrap();
device
.transport
.register_queue_callback(0, Box::new(handle_block_device), false)
.unwrap();
fn config_space_change(_: &TrapFrame) {
info!("Virtio block device config space change");
{
let mut transport = device.transport.lock();
transport
.register_cfg_callback(Box::new(handle_config_change))
.unwrap();
transport
.register_queue_callback(0, Box::new(handle_irq), false)
.unwrap();
transport.finish_init();
}
device.transport.finish_init();
Ok(device)
}
/// Handles the irq issued from the device
fn handle_irq(&self) {
info!("Virtio block device handle irq");
loop {
// Pops the complete request
let complete_request = {
let mut queue = self.queue.lock_irq_disabled();
let Ok((token, _)) = queue.pop_used() else {
return;
};
self.submitted_requests.lock().remove(&token).unwrap()
};
// Handles the response
let id = complete_request.id as usize;
let resp_slice = DmaStreamSlice::new(&self.block_responses, id * RESP_SIZE, RESP_SIZE);
resp_slice.sync().unwrap();
let resp: BlockResp = resp_slice.read_val(0).unwrap();
self.id_allocator.lock().free(id);
match RespStatus::try_from(resp.status).unwrap() {
RespStatus::Ok => {}
// FIXME: Return an error instead of triggering a kernel panic
_ => panic!("io error in block device"),
};
// Synchronize DMA mapping if read from the device
if let BioType::Read = complete_request.bio_request.type_() {
complete_request
.dma_bufs
.iter()
.for_each(|(stream, offset, len)| {
stream.sync(*offset..*offset + *len).unwrap();
});
}
// Completes the bio request
complete_request.bio_request.bios().for_each(|bio| {
bio.complete(BioStatus::Complete);
});
}
}
fn handle_config_change(&self) {
info!("Virtio block device config space change");
}
// TODO: Most logic is the same as read and write, there should be a refactor.
// TODO: Should return an Err instead of panic if the device fails.
fn get_id(&self) -> String {
let id = self.id_allocator.lock().pop().unwrap() as usize;
fn request_device_id(&self) -> String {
let id = self.id_allocator.lock().alloc().unwrap();
let req_slice = {
let req_slice = DmaStreamSlice::new(&self.block_requests, id * REQ_SIZE, REQ_SIZE);
let req = BlockReq {
@ -251,14 +232,16 @@ impl DeviceInner {
let token = queue
.add_dma_buf(&[&req_slice], outputs.as_slice())
.expect("add queue failed");
queue.notify();
if queue.should_notify() {
queue.notify();
}
while !queue.can_pop() {
spin_loop();
}
queue.pop_used_with_token(token).expect("pop used failed");
resp_slice.sync().unwrap();
self.id_allocator.lock().push(id as u8);
self.id_allocator.lock().free(id);
let resp: BlockResp = resp_slice.read_val(0).unwrap();
match RespStatus::try_from(resp.status).unwrap() {
RespStatus::Ok => {}
@ -279,17 +262,17 @@ impl DeviceInner {
String::from_utf8(device_id).unwrap()
}
/// Reads data from the block device, this function is blocking.
pub fn read(&self, sector_id: Sid, bufs: &[DmaStreamSlice]) {
// FIXME: Handling cases without id.
let id = self.id_allocator.lock().pop().unwrap() as usize;
/// Reads data from the device, this function is non-blocking.
fn read(&self, bio_request: BioRequest) {
let dma_streams = Self::dma_stream_map(&bio_request);
let id = self.id_allocator.lock().alloc().unwrap();
let req_slice = {
let req_slice = DmaStreamSlice::new(&self.block_requests, id * REQ_SIZE, REQ_SIZE);
let req = BlockReq {
type_: ReqType::In as _,
reserved: 0,
sector: sector_id.to_raw(),
sector: bio_request.sid_range().start.to_raw(),
};
req_slice.write_val(0, &req).unwrap();
req_slice.sync().unwrap();
@ -302,42 +285,55 @@ impl DeviceInner {
resp_slice
};
let dma_slices: Vec<DmaStreamSlice> = dma_streams
.iter()
.map(|(stream, offset, len)| DmaStreamSlice::new(stream, *offset, *len))
.collect();
let outputs = {
let mut outputs: Vec<&DmaStreamSlice> = bufs.iter().collect();
let mut outputs: Vec<&DmaStreamSlice> = Vec::with_capacity(dma_streams.len() + 1);
outputs.extend(dma_slices.iter());
outputs.push(&resp_slice);
outputs
};
let mut queue = self.queue.lock_irq_disabled();
let token = queue
.add_dma_buf(&[&req_slice], outputs.as_slice())
.expect("add queue failed");
queue.notify();
while !queue.can_pop() {
spin_loop();
let num_used_descs = outputs.len() + 1;
// FIXME: Split the request if it is too big
if num_used_descs > Self::QUEUE_SIZE as usize {
panic!("The request size surpasses the queue size");
}
queue.pop_used_with_token(token).expect("pop used failed");
resp_slice.sync().unwrap();
let resp: BlockResp = resp_slice.read_val(0).unwrap();
self.id_allocator.lock().push(id as u8);
match RespStatus::try_from(resp.status).unwrap() {
RespStatus::Ok => {}
_ => panic!("io error in block device"),
};
loop {
let mut queue = self.queue.lock_irq_disabled();
if num_used_descs > queue.available_desc() {
continue;
}
let token = queue
.add_dma_buf(&[&req_slice], outputs.as_slice())
.expect("add queue failed");
if queue.should_notify() {
queue.notify();
}
// Records the submitted request
let submitted_request = SubmittedRequest::new(id as u16, bio_request, dma_streams);
self.submitted_requests
.lock()
.insert(token, submitted_request);
return;
}
}
/// Writes data to the block device, this function is blocking.
pub fn write(&self, sector_id: Sid, bufs: &[DmaStreamSlice]) {
// FIXME: Handling cases without id.
let id = self.id_allocator.lock().pop().unwrap() as usize;
/// Writes data to the device, this function is non-blocking.
fn write(&self, bio_request: BioRequest) {
let dma_streams = Self::dma_stream_map(&bio_request);
let id = self.id_allocator.lock().alloc().unwrap();
let req_slice = {
let req_slice = DmaStreamSlice::new(&self.block_requests, id * REQ_SIZE, REQ_SIZE);
let req = BlockReq {
type_: ReqType::Out as _,
reserved: 0,
sector: sector_id.to_raw(),
sector: bio_request.sid_range().start.to_raw(),
};
req_slice.write_val(0, &req).unwrap();
req_slice.sync().unwrap();
@ -350,32 +346,83 @@ impl DeviceInner {
resp_slice
};
let dma_slices: Vec<DmaStreamSlice> = dma_streams
.iter()
.map(|(stream, offset, len)| DmaStreamSlice::new(stream, *offset, *len))
.collect();
let inputs = {
let mut inputs: Vec<&DmaStreamSlice> = bufs.iter().collect();
inputs.insert(0, &req_slice);
let mut inputs: Vec<&DmaStreamSlice> = Vec::with_capacity(dma_streams.len() + 1);
inputs.push(&req_slice);
inputs.extend(dma_slices.iter());
inputs
};
let mut queue = self.queue.lock_irq_disabled();
let token = queue
.add_dma_buf(inputs.as_slice(), &[&resp_slice])
.expect("add queue failed");
queue.notify();
while !queue.can_pop() {
spin_loop();
let num_used_descs = inputs.len() + 1;
// FIXME: Split the request if it is too big
if num_used_descs > Self::QUEUE_SIZE as usize {
panic!("The request size surpasses the queue size");
}
queue.pop_used_with_token(token).expect("pop used failed");
loop {
let mut queue = self.queue.lock_irq_disabled();
if num_used_descs > queue.available_desc() {
continue;
}
let token = queue
.add_dma_buf(inputs.as_slice(), &[&resp_slice])
.expect("add queue failed");
if queue.should_notify() {
queue.notify();
}
resp_slice.sync().unwrap();
let resp: BlockResp = resp_slice.read_val(0).unwrap();
self.id_allocator.lock().push(id as u8);
match RespStatus::try_from(resp.status).unwrap() {
RespStatus::Ok => {}
_ => panic!("io error in block device:{:?}", resp.status),
// Records the submitted request
let submitted_request = SubmittedRequest::new(id as u16, bio_request, dma_streams);
self.submitted_requests
.lock()
.insert(token, submitted_request);
return;
}
}
/// Performs DMA mapping for the segments in bio request.
fn dma_stream_map(bio_request: &BioRequest) -> Vec<(DmaStream, usize, usize)> {
let dma_direction = match bio_request.type_() {
BioType::Read => DmaDirection::FromDevice,
BioType::Write => DmaDirection::ToDevice,
_ => todo!(),
};
bio_request
.bios()
.flat_map(|bio| {
bio.segments().iter().map(|segment| {
let dma_stream =
DmaStream::map(segment.pages().clone(), dma_direction, false).unwrap();
(dma_stream, segment.offset(), segment.nbytes())
})
})
.collect()
}
}
/// A submitted bio request for callback.
#[derive(Debug)]
struct SubmittedRequest {
id: u16,
bio_request: BioRequest,
dma_bufs: Vec<(DmaStream, usize, usize)>,
}
impl SubmittedRequest {
pub fn new(id: u16, bio_request: BioRequest, dma_bufs: Vec<(DmaStream, usize, usize)>) -> Self {
Self {
id,
bio_request,
dma_bufs,
}
}
}
/// VirtIOBlock request.
#[repr(C)]
#[derive(Debug, Copy, Clone, Pod)]
struct BlockReq {