mirror of
https://github.com/asterinas/asterinas.git
synced 2025-06-19 04:26:39 +00:00
Modify the virtio-blk to async
This commit is contained in:
committed by
Tate, Hongliang Tian
parent
2616335755
commit
b7131e721c
@ -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 {
|
||||
|
@ -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 {
|
||||
|
Reference in New Issue
Block a user