Refactor the block layer by introducing BioSegmentPool

This commit is contained in:
Shaowei Song
2024-11-20 13:51:22 +00:00
committed by Tate, Hongliang Tian
parent d37da228ab
commit ecad132ec9
13 changed files with 608 additions and 348 deletions

View File

@ -15,5 +15,6 @@ int-to-c-enum = { path = "../../libs/int-to-c-enum" }
component = { path = "../../libs/comp-sys/component" }
log = "0.4"
static_assertions = "1.1.0"
bitvec = { version = "1.0.1", default-features = false, features = ["alloc"] }
[features]

View File

@ -1,15 +1,20 @@
// SPDX-License-Identifier: MPL-2.0
use align_ext::AlignExt;
use aster_util::segment_slice::SegmentSlice;
use bitvec::array::BitArray;
use int_to_c_enum::TryFromInt;
use ostd::{
mm::{Frame, Infallible, VmReader, VmWriter},
sync::WaitQueue,
mm::{
DmaDirection, DmaStream, DmaStreamSlice, FrameAllocOptions, Infallible, Segment, VmIo,
VmReader, VmWriter,
},
sync::{SpinLock, WaitQueue},
Error,
};
use spin::Once;
use super::{id::Sid, BlockDevice};
use crate::prelude::*;
use crate::{prelude::*, BLOCK_SIZE, SECTOR_SIZE};
/// The unit for block I/O.
///
@ -361,88 +366,349 @@ pub enum BioStatus {
IoError = 5,
}
/// `BioSegment` is a smallest memory unit in block I/O.
///
/// It is a contiguous memory region that contains multiple sectors.
/// `BioSegment` is the basic memory unit of a block I/O request.
#[derive(Debug, Clone)]
pub struct BioSegment {
/// The contiguous pages on which this segment resides.
pages: SegmentSlice,
/// The starting offset (in bytes) within the first page.
/// The offset should always be aligned to the sector size and
/// must not exceed the size of a single page.
offset: AlignedUsize<SECTOR_SIZE>,
/// The total length (in bytes).
/// The length can span multiple pages and should be aligned to
/// the sector size.
len: AlignedUsize<SECTOR_SIZE>,
inner: Arc<BioSegmentInner>,
}
const SECTOR_SIZE: u16 = super::SECTOR_SIZE as u16;
/// The inner part of `BioSegment`.
// TODO: Decouple `BioSegmentInner` with DMA-related buffers.
#[derive(Debug, Clone)]
struct BioSegmentInner {
/// Internal DMA slice.
dma_slice: DmaStreamSlice<DmaStream>,
/// Whether the segment is allocated from the pool.
from_pool: bool,
}
/// The direction of a bio request.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub enum BioDirection {
/// Read from the backed block device.
FromDevice,
/// Write to the backed block device.
ToDevice,
}
impl<'a> BioSegment {
/// Constructs a new `BioSegment` from `Segment`.
pub fn from_segment(segment: SegmentSlice, offset: usize, len: usize) -> Self {
assert!(offset + len <= segment.nbytes());
/// Allocates a new `BioSegment` with the wanted blocks count and
/// the bio direction.
pub fn alloc(nblocks: usize, direction: BioDirection) -> Self {
Self::alloc_inner(nblocks, 0, nblocks * BLOCK_SIZE, direction)
}
/// The inner function that do the real segment allocation.
///
/// Support two extended parameters:
/// 1. `offset_within_first_block`: the offset (in bytes) within the first block.
/// 2. `len`: the exact length (in bytes) of the wanted segment. (May
/// less than `nblocks * BLOCK_SIZE`)
///
/// # Panics
///
/// If the `offset_within_first_block` or `len` is not sector aligned,
/// this method will panic.
pub(super) fn alloc_inner(
nblocks: usize,
offset_within_first_block: usize,
len: usize,
direction: BioDirection,
) -> Self {
assert!(
is_sector_aligned(offset_within_first_block)
&& offset_within_first_block < BLOCK_SIZE
&& is_sector_aligned(len)
&& offset_within_first_block + len <= nblocks * BLOCK_SIZE
);
// The target segment is whether from the pool or newly-allocated
let bio_segment_inner = target_pool(direction)
.and_then(|pool| pool.alloc(nblocks, offset_within_first_block, len))
.unwrap_or_else(|| {
let segment = FrameAllocOptions::new(nblocks)
.uninit(true)
.alloc_contiguous()
.unwrap();
let dma_stream = DmaStream::map(segment, direction.into(), false).unwrap();
BioSegmentInner {
dma_slice: DmaStreamSlice::new(dma_stream, offset_within_first_block, len),
from_pool: false,
}
});
Self {
pages: segment.range(frame_range(&(offset..offset + len))),
offset: AlignedUsize::<SECTOR_SIZE>::new(offset % super::BLOCK_SIZE).unwrap(),
len: AlignedUsize::<SECTOR_SIZE>::new(len).unwrap(),
inner: Arc::new(bio_segment_inner),
}
}
/// Constructs a new `BioSegment` from `Frame`.
pub fn from_frame(frame: Frame, offset: usize, len: usize) -> Self {
assert!(offset + len <= super::BLOCK_SIZE);
/// Constructs a new `BioSegment` with a given `Segment` and the bio direction.
pub fn new_from_segment(segment: Segment, direction: BioDirection) -> Self {
let len = segment.nbytes();
let dma_stream = DmaStream::map(segment, direction.into(), false).unwrap();
Self {
pages: SegmentSlice::from(frame),
offset: AlignedUsize::<SECTOR_SIZE>::new(offset).unwrap(),
len: AlignedUsize::<SECTOR_SIZE>::new(len).unwrap(),
inner: Arc::new(BioSegmentInner {
dma_slice: DmaStreamSlice::new(dma_stream, 0, len),
from_pool: false,
}),
}
}
/// Returns the number of sectors.
pub fn nsectors(&self) -> Sid {
Sid::from_offset(self.len.value())
}
/// Returns the number of bytes.
pub fn nbytes(&self) -> usize {
self.len.value()
self.inner.dma_slice.nbytes()
}
/// Returns the offset (in bytes) within the first page.
pub fn offset(&self) -> usize {
self.offset.value()
/// Returns the number of sectors.
pub fn nsectors(&self) -> Sid {
Sid::from_offset(self.nbytes())
}
/// Returns the contiguous pages on which this segment resides.
pub fn pages(&self) -> &SegmentSlice {
&self.pages
/// Returns the number of blocks.
pub fn nblocks(&self) -> usize {
self.nbytes().align_up(BLOCK_SIZE) / BLOCK_SIZE
}
/// Returns the offset (in bytes) within the first block.
pub fn offset_within_first_block(&self) -> usize {
self.inner.dma_slice.offset() % BLOCK_SIZE
}
/// Returns the inner DMA slice.
pub fn inner_dma_slice(&self) -> &DmaStreamSlice<DmaStream> {
&self.inner.dma_slice
}
/// Returns the inner VM segment.
#[cfg(ktest)]
pub fn inner_segment(&self) -> &Segment {
self.inner.dma_slice.stream().vm_segment()
}
/// Returns a reader to read data from it.
pub fn reader(&'a self) -> VmReader<'a, Infallible> {
self.pages
.reader()
.skip(self.offset.value())
.limit(self.len.value())
pub fn reader(&'a self) -> Result<VmReader<'a, Infallible>, Error> {
self.inner.dma_slice.reader()
}
/// Returns a writer to write data into it.
pub fn writer(&'a self) -> VmWriter<'a, Infallible> {
self.pages
.writer()
.skip(self.offset.value())
.limit(self.len.value())
pub fn writer(&'a self) -> Result<VmWriter<'a, Infallible>, Error> {
self.inner.dma_slice.writer()
}
}
fn frame_range(byte_range: &Range<usize>) -> Range<usize> {
let start = byte_range.start.align_down(super::BLOCK_SIZE);
let end = byte_range.end.align_up(super::BLOCK_SIZE);
(start / super::BLOCK_SIZE)..(end / super::BLOCK_SIZE)
impl VmIo for BioSegment {
fn read(&self, offset: usize, writer: &mut VmWriter) -> Result<(), Error> {
self.inner.dma_slice.read(offset, writer)
}
fn write(&self, offset: usize, reader: &mut VmReader) -> Result<(), Error> {
self.inner.dma_slice.write(offset, reader)
}
}
// The timing for free the segment to the pool.
impl Drop for BioSegmentInner {
fn drop(&mut self) {
if !self.from_pool {
return;
}
if let Some(pool) = target_pool(self.direction()) {
pool.free(self);
}
}
}
impl BioSegmentInner {
/// Returns the bio direction.
fn direction(&self) -> BioDirection {
match self.dma_slice.stream().direction() {
DmaDirection::FromDevice => BioDirection::FromDevice,
DmaDirection::ToDevice => BioDirection::ToDevice,
_ => unreachable!(),
}
}
}
/// A pool of managing segments for block I/O requests.
///
/// Inside the pool, it's a large chunk of `DmaStream` which
/// contains the mapped segment. The allocation/free is done by slicing
/// the `DmaStream`.
// TODO: Use a more advanced allocation algorithm to replace the naive one to improve efficiency.
struct BioSegmentPool {
pool: DmaStream,
total_blocks: usize,
direction: BioDirection,
manager: SpinLock<PoolSlotManager>,
}
/// Manages the free slots in the pool.
struct PoolSlotManager {
/// A bit array to manage the occupied slots in the pool (Bit
/// value 1 represents "occupied"; 0 represents "free").
/// The total size is currently determined by `POOL_DEFAULT_NBLOCKS`.
occupied: BitArray<[u8; POOL_DEFAULT_NBLOCKS.div_ceil(8)]>,
/// The first index of all free slots in the pool.
min_free: usize,
}
impl BioSegmentPool {
/// Creates a new pool given the bio direction. The total number of
/// managed blocks is currently set to `POOL_DEFAULT_NBLOCKS`.
///
/// The new pool will be allocated and mapped for later allocation.
pub fn new(direction: BioDirection) -> Self {
let total_blocks = POOL_DEFAULT_NBLOCKS;
let pool = {
let segment = FrameAllocOptions::new(total_blocks)
.uninit(true)
.alloc_contiguous()
.unwrap();
DmaStream::map(segment, direction.into(), false).unwrap()
};
let manager = SpinLock::new(PoolSlotManager {
occupied: BitArray::ZERO,
min_free: 0,
});
Self {
pool,
total_blocks,
direction,
manager,
}
}
/// Allocates a bio segment with the given count `nblocks`
/// from the pool.
///
/// Support two extended parameters:
/// 1. `offset_within_first_block`: the offset (in bytes) within the first block.
/// 2. `len`: the exact length (in bytes) of the wanted segment. (May
/// less than `nblocks * BLOCK_SIZE`)
///
/// If there is no enough space in the pool, this method
/// will return `None`.
///
/// # Panics
///
/// If the `offset_within_first_block` exceeds the block size, or the `len`
/// exceeds the total length, this method will panic.
pub fn alloc(
&self,
nblocks: usize,
offset_within_first_block: usize,
len: usize,
) -> Option<BioSegmentInner> {
assert!(
offset_within_first_block < BLOCK_SIZE
&& offset_within_first_block + len <= nblocks * BLOCK_SIZE
);
let mut manager = self.manager.lock();
if nblocks > self.total_blocks - manager.min_free {
return None;
}
// Find the free range
let (start, end) = {
let mut start = manager.min_free;
let mut end = start;
while end < self.total_blocks && end - start < nblocks {
if manager.occupied[end] {
start = end + 1;
end = start;
} else {
end += 1;
}
}
if end - start < nblocks {
return None;
}
(start, end)
};
manager.occupied[start..end].fill(true);
manager.min_free = manager.occupied[end..]
.iter()
.position(|i| !i)
.map(|pos| end + pos)
.unwrap_or(self.total_blocks);
let dma_slice = DmaStreamSlice::new(
self.pool.clone(),
start * BLOCK_SIZE + offset_within_first_block,
len,
);
let bio_segment = BioSegmentInner {
dma_slice,
from_pool: true,
};
Some(bio_segment)
}
/// Returns an allocated bio segment to the pool,
/// free the space. This method is not public and should only
/// be called automatically by `BioSegmentInner::drop()`.
///
/// # Panics
///
/// If the target bio segment is not allocated from the pool
/// or not the same direction, this method will panic.
fn free(&self, bio_segment: &BioSegmentInner) {
assert!(bio_segment.from_pool && bio_segment.direction() == self.direction);
let (start, end) = {
let dma_slice = &bio_segment.dma_slice;
let start = dma_slice.offset().align_down(BLOCK_SIZE) / BLOCK_SIZE;
let end = (dma_slice.offset() + dma_slice.nbytes()).align_up(BLOCK_SIZE) / BLOCK_SIZE;
if end <= start || end > self.total_blocks {
return;
}
(start, end)
};
let mut manager = self.manager.lock();
debug_assert!(manager.occupied[start..end].iter().all(|i| *i));
manager.occupied[start..end].fill(false);
if start < manager.min_free {
manager.min_free = start;
}
}
}
/// A pool of segments for read bio requests only.
static BIO_SEGMENT_RPOOL: Once<Arc<BioSegmentPool>> = Once::new();
/// A pool of segments for write bio requests only.
static BIO_SEGMENT_WPOOL: Once<Arc<BioSegmentPool>> = Once::new();
/// The default number of blocks in each pool. (16MB each for now)
const POOL_DEFAULT_NBLOCKS: usize = 4096;
/// Initializes the bio segment pool.
pub fn bio_segment_pool_init() {
BIO_SEGMENT_RPOOL.call_once(|| Arc::new(BioSegmentPool::new(BioDirection::FromDevice)));
BIO_SEGMENT_WPOOL.call_once(|| Arc::new(BioSegmentPool::new(BioDirection::ToDevice)));
}
/// Gets the target pool with the given `direction`.
fn target_pool(direction: BioDirection) -> Option<&'static Arc<BioSegmentPool>> {
match direction {
BioDirection::FromDevice => BIO_SEGMENT_RPOOL.get(),
BioDirection::ToDevice => BIO_SEGMENT_WPOOL.get(),
}
}
impl From<BioDirection> for DmaDirection {
fn from(direction: BioDirection) -> Self {
match direction {
BioDirection::FromDevice => DmaDirection::FromDevice,
BioDirection::ToDevice => DmaDirection::ToDevice,
}
}
}
/// Checks if the given offset is aligned to sector.
pub fn is_sector_aligned(offset: usize) -> bool {
offset % SECTOR_SIZE == 0
}
/// An aligned unsigned integer number.

View File

@ -1,16 +1,16 @@
// SPDX-License-Identifier: MPL-2.0
use aster_util::segment_slice::SegmentSlice;
use ostd::mm::{
FallibleVmRead, FallibleVmWrite, Frame, FrameAllocOptions, VmIo, VmReader, VmWriter,
};
use ostd::mm::{VmIo, VmReader, VmWriter};
use super::{
bio::{Bio, BioEnqueueError, BioSegment, BioStatus, BioType, BioWaiter, SubmittedBio},
id::{Bid, Sid},
BlockDevice, BLOCK_SIZE, SECTOR_SIZE,
BlockDevice, BLOCK_SIZE,
};
use crate::{
bio::{is_sector_aligned, BioDirection},
prelude::*,
};
use crate::prelude::*;
/// Implements several commonly used APIs for the block device to conveniently
/// read and write block(s).
@ -20,9 +20,14 @@ impl dyn BlockDevice {
pub fn read_blocks(
&self,
bid: Bid,
segment: &SegmentSlice,
bio_segment: BioSegment,
) -> Result<BioStatus, BioEnqueueError> {
let bio = create_bio_from_segment(BioType::Read, bid, segment);
let bio = Bio::new(
BioType::Read,
Sid::from(bid),
vec![bio_segment],
Some(general_complete_fn),
);
let status = bio.submit_and_wait(self)?;
Ok(status)
}
@ -31,22 +36,14 @@ impl dyn BlockDevice {
pub fn read_blocks_async(
&self,
bid: Bid,
segment: &SegmentSlice,
bio_segment: BioSegment,
) -> Result<BioWaiter, BioEnqueueError> {
let bio = create_bio_from_segment(BioType::Read, bid, segment);
bio.submit(self)
}
/// Synchronously reads one block indicated by the `bid`.
pub fn read_block(&self, bid: Bid, frame: &Frame) -> Result<BioStatus, BioEnqueueError> {
let bio = create_bio_from_frame(BioType::Read, bid, frame);
let status = bio.submit_and_wait(self)?;
Ok(status)
}
/// Asynchronously reads one block indicated by the `bid`.
pub fn read_block_async(&self, bid: Bid, frame: &Frame) -> Result<BioWaiter, BioEnqueueError> {
let bio = create_bio_from_frame(BioType::Read, bid, frame);
let bio = Bio::new(
BioType::Read,
Sid::from(bid),
vec![bio_segment],
Some(general_complete_fn),
);
bio.submit(self)
}
@ -54,9 +51,14 @@ impl dyn BlockDevice {
pub fn write_blocks(
&self,
bid: Bid,
segment: &SegmentSlice,
bio_segment: BioSegment,
) -> Result<BioStatus, BioEnqueueError> {
let bio = create_bio_from_segment(BioType::Write, bid, segment);
let bio = Bio::new(
BioType::Write,
Sid::from(bid),
vec![bio_segment],
Some(general_complete_fn),
);
let status = bio.submit_and_wait(self)?;
Ok(status)
}
@ -65,22 +67,14 @@ impl dyn BlockDevice {
pub fn write_blocks_async(
&self,
bid: Bid,
segment: &SegmentSlice,
bio_segment: BioSegment,
) -> Result<BioWaiter, BioEnqueueError> {
let bio = create_bio_from_segment(BioType::Write, bid, segment);
bio.submit(self)
}
/// Synchronously writes one block indicated by the `bid`.
pub fn write_block(&self, bid: Bid, frame: &Frame) -> Result<BioStatus, BioEnqueueError> {
let bio = create_bio_from_frame(BioType::Write, bid, frame);
let status = bio.submit_and_wait(self)?;
Ok(status)
}
/// Asynchronously writes one block indicated by the `bid`.
pub fn write_block_async(&self, bid: Bid, frame: &Frame) -> Result<BioWaiter, BioEnqueueError> {
let bio = create_bio_from_frame(BioType::Write, bid, frame);
let bio = Bio::new(
BioType::Write,
Sid::from(bid),
vec![bio_segment],
Some(general_complete_fn),
);
bio.submit(self)
}
@ -101,7 +95,7 @@ impl VmIo for dyn BlockDevice {
/// Reads consecutive bytes of several sectors in size.
fn read(&self, offset: usize, writer: &mut VmWriter) -> ostd::Result<()> {
let read_len = writer.avail();
if offset % SECTOR_SIZE != 0 || read_len % SECTOR_SIZE != 0 {
if !is_sector_aligned(offset) || !is_sector_aligned(read_len) {
return Err(ostd::Error::InvalidArgs);
}
if read_len == 0 {
@ -112,20 +106,21 @@ impl VmIo for dyn BlockDevice {
let num_blocks = {
let first = Bid::from_offset(offset).to_raw();
let last = Bid::from_offset(offset + read_len - 1).to_raw();
last - first + 1
(last - first + 1) as usize
};
let segment = FrameAllocOptions::new(num_blocks as usize)
.uninit(true)
.alloc_contiguous()?;
let bio_segment =
BioSegment::from_segment(segment.into(), offset % BLOCK_SIZE, read_len);
let bio_segment = BioSegment::alloc_inner(
num_blocks,
offset % BLOCK_SIZE,
read_len,
BioDirection::FromDevice,
);
(
Bio::new(
BioType::Read,
Sid::from_offset(offset),
vec![bio_segment.clone()],
None,
Some(general_complete_fn),
),
bio_segment,
)
@ -133,13 +128,7 @@ impl VmIo for dyn BlockDevice {
let status = bio.submit_and_wait(self)?;
match status {
BioStatus::Complete => {
let _ = bio_segment
.reader()
.read_fallible(writer)
.map_err(|(e, _)| e)?;
Ok(())
}
BioStatus::Complete => bio_segment.read(0, writer),
_ => Err(ostd::Error::IoError),
}
}
@ -147,7 +136,7 @@ impl VmIo for dyn BlockDevice {
/// Writes consecutive bytes of several sectors in size.
fn write(&self, offset: usize, reader: &mut VmReader) -> ostd::Result<()> {
let write_len = reader.remain();
if offset % SECTOR_SIZE != 0 || write_len % SECTOR_SIZE != 0 {
if !is_sector_aligned(offset) || !is_sector_aligned(write_len) {
return Err(ostd::Error::InvalidArgs);
}
if write_len == 0 {
@ -158,23 +147,21 @@ impl VmIo for dyn BlockDevice {
let num_blocks = {
let first = Bid::from_offset(offset).to_raw();
let last = Bid::from_offset(offset + write_len - 1).to_raw();
last - first + 1
(last - first + 1) as usize
};
let segment = FrameAllocOptions::new(num_blocks as usize)
.uninit(true)
.alloc_contiguous()?;
segment.write(offset % BLOCK_SIZE, reader)?;
let len = segment
.writer()
.skip(offset % BLOCK_SIZE)
.write_fallible(reader)
.map_err(|(e, _)| e)?;
let bio_segment = BioSegment::from_segment(segment.into(), offset % BLOCK_SIZE, len);
let bio_segment = BioSegment::alloc_inner(
num_blocks,
offset % BLOCK_SIZE,
write_len,
BioDirection::ToDevice,
);
bio_segment.write(0, reader)?;
Bio::new(
BioType::Write,
Sid::from_offset(offset),
vec![bio_segment],
None,
Some(general_complete_fn),
)
};
@ -189,28 +176,27 @@ impl VmIo for dyn BlockDevice {
impl dyn BlockDevice {
/// Asynchronously writes consecutive bytes of several sectors in size.
pub fn write_bytes_async(&self, offset: usize, buf: &[u8]) -> ostd::Result<BioWaiter> {
if offset % SECTOR_SIZE != 0 || buf.len() % SECTOR_SIZE != 0 {
let write_len = buf.len();
if !is_sector_aligned(offset) || !is_sector_aligned(write_len) {
return Err(ostd::Error::InvalidArgs);
}
if buf.is_empty() {
if write_len == 0 {
return Ok(BioWaiter::new());
}
let bio = {
let num_blocks = {
let first = Bid::from_offset(offset).to_raw();
let last = Bid::from_offset(offset + buf.len() - 1).to_raw();
last - first + 1
let last = Bid::from_offset(offset + write_len - 1).to_raw();
(last - first + 1) as usize
};
let segment = FrameAllocOptions::new(num_blocks as usize)
.uninit(true)
.alloc_contiguous()?;
segment.write_bytes(offset % BLOCK_SIZE, buf)?;
let len = segment
.writer()
.skip(offset % BLOCK_SIZE)
.write(&mut buf.into());
let bio_segment = BioSegment::from_segment(segment.into(), offset % BLOCK_SIZE, len);
let bio_segment = BioSegment::alloc_inner(
num_blocks,
offset % BLOCK_SIZE,
write_len,
BioDirection::ToDevice,
);
bio_segment.write(0, &mut VmReader::from(buf).to_fallible())?;
Bio::new(
BioType::Write,
Sid::from_offset(offset),
@ -224,28 +210,6 @@ impl dyn BlockDevice {
}
}
// TODO: Maybe we should have a builder for `Bio`.
fn create_bio_from_segment(type_: BioType, bid: Bid, segment: &SegmentSlice) -> Bio {
let bio_segment = BioSegment::from_segment(segment.clone(), 0, segment.nbytes());
Bio::new(
type_,
Sid::from(bid),
vec![bio_segment],
Some(general_complete_fn),
)
}
// TODO: Maybe we should have a builder for `Bio`.
fn create_bio_from_frame(type_: BioType, bid: Bid, frame: &Frame) -> Bio {
let bio_segment = BioSegment::from_frame(frame.clone(), 0, BLOCK_SIZE);
Bio::new(
type_,
Sid::from(bid),
vec![bio_segment],
Some(general_complete_fn),
)
}
fn general_complete_fn(bio: &SubmittedBio) {
match bio.status() {
BioStatus::Complete => (),

View File

@ -11,7 +11,7 @@ use alloc::{
use core::{fmt::Debug, hint::spin_loop, mem::size_of};
use aster_block::{
bio::{BioEnqueueError, BioStatus, BioType, SubmittedBio},
bio::{bio_segment_pool_init, BioEnqueueError, BioStatus, BioType, SubmittedBio},
request_queue::{BioRequest, BioRequestSingleQueue},
BlockDeviceMeta,
};
@ -63,6 +63,8 @@ impl BlockDevice {
});
aster_block::register_device(device_id, block_device);
bio_segment_pool_init();
Ok(())
}
@ -215,11 +217,14 @@ impl DeviceInner {
// 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();
});
.bio_request
.bios()
.flat_map(|bio| {
bio.segments()
.iter()
.map(|segment| segment.inner_dma_slice())
})
.for_each(|dma_slice| dma_slice.sync().unwrap());
}
// Completes the bio request
@ -301,11 +306,10 @@ impl DeviceInner {
/// 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.disable_irq().lock().alloc().unwrap();
let req_slice = {
let req_slice = DmaStreamSlice::new(&self.block_requests, id * REQ_SIZE, REQ_SIZE);
let req_slice =
DmaStreamSlice::new(self.block_requests.clone(), id * REQ_SIZE, REQ_SIZE);
let req = BlockReq {
type_: ReqType::In as _,
reserved: 0,
@ -317,18 +321,21 @@ impl DeviceInner {
};
let resp_slice = {
let resp_slice = DmaStreamSlice::new(&self.block_responses, id * RESP_SIZE, RESP_SIZE);
let resp_slice =
DmaStreamSlice::new(self.block_responses.clone(), id * RESP_SIZE, RESP_SIZE);
resp_slice.write_val(0, &BlockResp::default()).unwrap();
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<_>> = Vec::with_capacity(dma_streams.len() + 1);
outputs.extend(dma_slices.iter());
let mut outputs: Vec<&DmaStreamSlice<_>> =
Vec::with_capacity(bio_request.num_segments() + 1);
let dma_slices_iter = bio_request.bios().flat_map(|bio| {
bio.segments()
.iter()
.map(|segment| segment.inner_dma_slice())
});
outputs.extend(dma_slices_iter);
outputs.push(&resp_slice);
outputs
};
@ -352,7 +359,7 @@ impl DeviceInner {
}
// Records the submitted request
let submitted_request = SubmittedRequest::new(id as u16, bio_request, dma_streams);
let submitted_request = SubmittedRequest::new(id as u16, bio_request);
self.submitted_requests
.disable_irq()
.lock()
@ -363,11 +370,10 @@ impl DeviceInner {
/// 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.disable_irq().lock().alloc().unwrap();
let req_slice = {
let req_slice = DmaStreamSlice::new(&self.block_requests, id * REQ_SIZE, REQ_SIZE);
let req_slice =
DmaStreamSlice::new(self.block_requests.clone(), id * REQ_SIZE, REQ_SIZE);
let req = BlockReq {
type_: ReqType::Out as _,
reserved: 0,
@ -379,19 +385,22 @@ impl DeviceInner {
};
let resp_slice = {
let resp_slice = DmaStreamSlice::new(&self.block_responses, id * RESP_SIZE, RESP_SIZE);
let resp_slice =
DmaStreamSlice::new(self.block_responses.clone(), id * RESP_SIZE, RESP_SIZE);
resp_slice.write_val(0, &BlockResp::default()).unwrap();
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<_>> = Vec::with_capacity(dma_streams.len() + 1);
let mut inputs: Vec<&DmaStreamSlice<_>> =
Vec::with_capacity(bio_request.num_segments() + 1);
inputs.push(&req_slice);
inputs.extend(dma_slices.iter());
let dma_slices_iter = bio_request.bios().flat_map(|bio| {
bio.segments()
.iter()
.map(|segment| segment.inner_dma_slice())
});
inputs.extend(dma_slices_iter);
inputs
};
@ -413,7 +422,7 @@ impl DeviceInner {
}
// Records the submitted request
let submitted_request = SubmittedRequest::new(id as u16, bio_request, dma_streams);
let submitted_request = SubmittedRequest::new(id as u16, bio_request);
self.submitted_requests
.disable_irq()
.lock()
@ -465,7 +474,7 @@ impl DeviceInner {
}
// Records the submitted request
let submitted_request = SubmittedRequest::new(id as u16, bio_request, Vec::new());
let submitted_request = SubmittedRequest::new(id as u16, bio_request);
self.submitted_requests
.disable_irq()
.lock()
@ -473,27 +482,6 @@ impl DeviceInner {
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().into(), dma_direction, false)
.unwrap();
(dma_stream, segment.offset(), segment.nbytes())
})
})
.collect()
}
}
/// A submitted bio request for callback.
@ -501,16 +489,11 @@ impl DeviceInner {
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,
}
pub fn new(id: u16, bio_request: BioRequest) -> Self {
Self { id, bio_request }
}
}