mirror of
https://github.com/asterinas/asterinas.git
synced 2025-06-25 02:13:24 +00:00
Change block device trait
This commit is contained in:
committed by
Tate, Hongliang Tian
parent
5b274d6cbe
commit
1616f2d32c
@ -530,6 +530,11 @@ impl<'a> VmReader<'a> {
|
||||
unsafe { self.end.sub_ptr(self.cursor) }
|
||||
}
|
||||
|
||||
/// Returns the cursor pointer, which refers to the address of the next byte to read.
|
||||
pub const fn cursor(&self) -> *const u8 {
|
||||
self.cursor
|
||||
}
|
||||
|
||||
/// Returns if it has remaining data to read.
|
||||
pub const fn has_remain(&self) -> bool {
|
||||
self.remain() > 0
|
||||
@ -635,6 +640,11 @@ impl<'a> VmWriter<'a> {
|
||||
unsafe { self.end.sub_ptr(self.cursor) }
|
||||
}
|
||||
|
||||
/// Returns the cursor pointer, which refers to the address of the next byte to write.
|
||||
pub const fn cursor(&self) -> *mut u8 {
|
||||
self.cursor
|
||||
}
|
||||
|
||||
/// Returns if it has avaliable space to write.
|
||||
pub const fn has_avail(&self) -> bool {
|
||||
self.avail() > 0
|
||||
|
@ -13,6 +13,8 @@ use alloc::string::String;
|
||||
use alloc::sync::Arc;
|
||||
use alloc::vec::Vec;
|
||||
use aster_frame::sync::SpinLock;
|
||||
use aster_frame::vm::VmReader;
|
||||
use aster_frame::vm::VmWriter;
|
||||
use component::init_component;
|
||||
use component::ComponentInitError;
|
||||
use spin::Once;
|
||||
@ -20,8 +22,8 @@ use spin::Once;
|
||||
pub const BLK_SIZE: usize = 512;
|
||||
|
||||
pub trait BlockDevice: Send + Sync + Any + Debug {
|
||||
fn read_block(&self, block_id: usize, buf: &mut [u8]);
|
||||
fn write_block(&self, block_id: usize, buf: &[u8]);
|
||||
fn read_block(&self, block_id: usize, buf: &[VmWriter]);
|
||||
fn write_block(&self, block_id: usize, buf: &[VmReader]);
|
||||
fn handle_irq(&self);
|
||||
}
|
||||
|
||||
|
@ -1,13 +1,18 @@
|
||||
use core::hint::spin_loop;
|
||||
use core::{hint::spin_loop, mem::size_of};
|
||||
|
||||
use alloc::{boxed::Box, string::ToString, sync::Arc};
|
||||
use aster_frame::{io_mem::IoMem, sync::SpinLock, trap::TrapFrame};
|
||||
use alloc::{boxed::Box, string::ToString, sync::Arc, vec::Vec};
|
||||
use aster_frame::{
|
||||
io_mem::IoMem,
|
||||
sync::SpinLock,
|
||||
trap::TrapFrame,
|
||||
vm::{VmAllocOptions, VmFrame, VmIo, VmReader, VmWriter},
|
||||
};
|
||||
use aster_util::safe_ptr::SafePtr;
|
||||
use log::info;
|
||||
use pod::Pod;
|
||||
|
||||
use crate::{
|
||||
device::block::{BlkReq, BlkResp, ReqType, RespStatus, BLK_SIZE},
|
||||
device::block::{BlkReq, BlkResp, ReqType, RespStatus},
|
||||
device::VirtioDeviceError,
|
||||
queue::VirtQueue,
|
||||
transport::VirtioTransport,
|
||||
@ -20,28 +25,60 @@ pub struct BlockDevice {
|
||||
config: SafePtr<VirtioBlkConfig, IoMem>,
|
||||
queue: SpinLock<VirtQueue>,
|
||||
transport: Box<dyn VirtioTransport>,
|
||||
/// Block requests, we use VmFrame to store the requests so that
|
||||
/// it can pass to the `add_vm` function
|
||||
block_requests: VmFrame,
|
||||
/// Block responses, we use VmFrame to store the requests so that
|
||||
/// it can pass to the `add_vm` function
|
||||
block_responses: VmFrame,
|
||||
id_allocator: SpinLock<Vec<u8>>,
|
||||
}
|
||||
|
||||
impl BlockDevice {
|
||||
/// read data from block device, this function is blocking
|
||||
/// FIEME: replace slice with a more secure data structure to use dma mapping.
|
||||
pub fn read(&self, block_id: usize, buf: &mut [u8]) {
|
||||
assert_eq!(buf.len(), BLK_SIZE);
|
||||
pub fn read(&self, block_id: usize, buf: &[VmWriter]) {
|
||||
// FIXME: Handling cases without id.
|
||||
let id = self.id_allocator.lock().pop().unwrap() as usize;
|
||||
let req = BlkReq {
|
||||
type_: ReqType::In as _,
|
||||
reserved: 0,
|
||||
sector: block_id as u64,
|
||||
};
|
||||
let mut resp = BlkResp::default();
|
||||
let resp = BlkResp::default();
|
||||
self.block_requests
|
||||
.write_val(id * size_of::<BlkReq>(), &req)
|
||||
.unwrap();
|
||||
self.block_responses
|
||||
.write_val(id * size_of::<BlkResp>(), &resp)
|
||||
.unwrap();
|
||||
let req = self
|
||||
.block_requests
|
||||
.reader()
|
||||
.skip(id * size_of::<BlkReq>())
|
||||
.limit(size_of::<BlkReq>());
|
||||
let resp = self
|
||||
.block_responses
|
||||
.writer()
|
||||
.skip(id * size_of::<BlkResp>())
|
||||
.limit(size_of::<BlkResp>());
|
||||
|
||||
let mut outputs: Vec<&VmWriter<'_>> = buf.iter().collect();
|
||||
outputs.push(&resp);
|
||||
let mut queue = self.queue.lock_irq_disabled();
|
||||
let token = queue
|
||||
.add(&[req.as_bytes()], &[buf, resp.as_bytes_mut()])
|
||||
.add_vm(&[&req], outputs.as_slice())
|
||||
.expect("add queue failed");
|
||||
queue.notify();
|
||||
while !queue.can_pop() {
|
||||
spin_loop();
|
||||
}
|
||||
queue.pop_used_with_token(token).expect("pop used failed");
|
||||
let resp: BlkResp = self
|
||||
.block_responses
|
||||
.read_val(id * size_of::<BlkResp>())
|
||||
.unwrap();
|
||||
self.id_allocator.lock().push(id as u8);
|
||||
match RespStatus::try_from(resp.status).unwrap() {
|
||||
RespStatus::Ok => {}
|
||||
_ => panic!("io error in block device"),
|
||||
@ -49,27 +86,51 @@ impl BlockDevice {
|
||||
}
|
||||
/// write data to block device, this function is blocking
|
||||
/// FIEME: replace slice with a more secure data structure to use dma mapping.
|
||||
pub fn write(&self, block_id: usize, buf: &[u8]) {
|
||||
assert_eq!(buf.len(), BLK_SIZE);
|
||||
pub fn write(&self, block_id: usize, buf: &[VmReader]) {
|
||||
// FIXME: Handling cases without id.
|
||||
let id = self.id_allocator.lock().pop().unwrap() as usize;
|
||||
let req = BlkReq {
|
||||
type_: ReqType::Out as _,
|
||||
reserved: 0,
|
||||
sector: block_id as u64,
|
||||
};
|
||||
let mut resp = BlkResp::default();
|
||||
let resp = BlkResp::default();
|
||||
self.block_requests
|
||||
.write_val(id * size_of::<BlkReq>(), &req)
|
||||
.unwrap();
|
||||
self.block_responses
|
||||
.write_val(id * size_of::<BlkResp>(), &resp)
|
||||
.unwrap();
|
||||
let req = self
|
||||
.block_requests
|
||||
.reader()
|
||||
.skip(id * size_of::<BlkReq>())
|
||||
.limit(size_of::<BlkReq>());
|
||||
let resp = self
|
||||
.block_responses
|
||||
.writer()
|
||||
.skip(id * size_of::<BlkResp>())
|
||||
.limit(size_of::<BlkResp>());
|
||||
|
||||
let mut queue = self.queue.lock_irq_disabled();
|
||||
let mut inputs: Vec<&VmReader<'_>> = buf.iter().collect();
|
||||
inputs.insert(0, &req);
|
||||
let token = queue
|
||||
.add(&[req.as_bytes(), buf], &[resp.as_bytes_mut()])
|
||||
.add_vm(inputs.as_slice(), &[&resp])
|
||||
.expect("add queue failed");
|
||||
queue.notify();
|
||||
while !queue.can_pop() {
|
||||
spin_loop();
|
||||
}
|
||||
queue.pop_used_with_token(token).expect("pop used failed");
|
||||
let st = resp.status;
|
||||
match RespStatus::try_from(st).unwrap() {
|
||||
let resp: BlkResp = self
|
||||
.block_responses
|
||||
.read_val(id * size_of::<BlkResp>())
|
||||
.unwrap();
|
||||
self.id_allocator.lock().push(id as u8);
|
||||
match RespStatus::try_from(resp.status).unwrap() {
|
||||
RespStatus::Ok => {}
|
||||
_ => panic!("io error in block device:{:?}", st),
|
||||
_ => panic!("io error in block device:{:?}", resp.status),
|
||||
};
|
||||
}
|
||||
|
||||
@ -85,6 +146,9 @@ impl BlockDevice {
|
||||
config,
|
||||
queue: SpinLock::new(queue),
|
||||
transport,
|
||||
block_requests: VmAllocOptions::new(1).alloc_single().unwrap(),
|
||||
block_responses: VmAllocOptions::new(1).alloc_single().unwrap(),
|
||||
id_allocator: SpinLock::new((0..64).collect()),
|
||||
};
|
||||
|
||||
device
|
||||
@ -121,11 +185,11 @@ impl BlockDevice {
|
||||
}
|
||||
|
||||
impl aster_block::BlockDevice for BlockDevice {
|
||||
fn read_block(&self, block_id: usize, buf: &mut [u8]) {
|
||||
fn read_block(&self, block_id: usize, buf: &[VmWriter]) {
|
||||
self.read(block_id, buf);
|
||||
}
|
||||
|
||||
fn write_block(&self, block_id: usize, buf: &[u8]) {
|
||||
fn write_block(&self, block_id: usize, buf: &[VmReader]) {
|
||||
self.write(block_id, buf);
|
||||
}
|
||||
|
||||
|
@ -26,7 +26,7 @@ pub struct ConsoleDevice {
|
||||
impl AnyConsoleDevice for ConsoleDevice {
|
||||
fn send(&self, value: &[u8]) {
|
||||
let mut transmit_queue = self.transmit_queue.lock_irq_disabled();
|
||||
transmit_queue.add(&[value], &[]).unwrap();
|
||||
transmit_queue.add_buf(&[value], &[]).unwrap();
|
||||
if transmit_queue.should_notify() {
|
||||
transmit_queue.notify();
|
||||
}
|
||||
@ -45,7 +45,7 @@ impl AnyConsoleDevice for ConsoleDevice {
|
||||
|
||||
let mut recv_buffer = self.buffer.lock();
|
||||
buf.copy_from_slice(&recv_buffer.as_ref()[..len as usize]);
|
||||
receive_queue.add(&[], &[recv_buffer.as_mut()]).unwrap();
|
||||
receive_queue.add_buf(&[], &[recv_buffer.as_mut()]).unwrap();
|
||||
if receive_queue.should_notify() {
|
||||
receive_queue.notify();
|
||||
}
|
||||
@ -68,7 +68,7 @@ impl AnyConsoleDevice for ConsoleDevice {
|
||||
for callback in lock.iter() {
|
||||
callback.call((buffer,));
|
||||
}
|
||||
receive_queue.add(&[], &[recv_buffer.as_mut()]).unwrap();
|
||||
receive_queue.add_buf(&[], &[recv_buffer.as_mut()]).unwrap();
|
||||
if receive_queue.should_notify() {
|
||||
receive_queue.notify();
|
||||
}
|
||||
@ -114,7 +114,7 @@ impl ConsoleDevice {
|
||||
|
||||
let mut receive_queue = device.receive_queue.lock();
|
||||
receive_queue
|
||||
.add(&[], &[device.buffer.lock().as_mut()])
|
||||
.add_buf(&[], &[device.buffer.lock().as_mut()])
|
||||
.unwrap();
|
||||
if receive_queue.should_notify() {
|
||||
receive_queue.notify();
|
||||
|
@ -80,7 +80,7 @@ impl InputDevice {
|
||||
|
||||
for (i, event) in event_buf.as_mut().iter_mut().enumerate() {
|
||||
// FIEME: replace slice with a more secure data structure to use dma mapping.
|
||||
let token = event_queue.add(&[], &[event.as_bytes_mut()]);
|
||||
let token = event_queue.add_buf(&[], &[event.as_bytes_mut()]);
|
||||
match token {
|
||||
Ok(value) => {
|
||||
assert_eq!(value, i as u16);
|
||||
@ -146,7 +146,7 @@ impl InputDevice {
|
||||
let event = &mut self.event_buf.lock()[token as usize];
|
||||
// requeue
|
||||
// FIEME: replace slice with a more secure data structure to use dma mapping.
|
||||
if let Ok(new_token) = lock.add(&[], &[event.as_bytes_mut()]) {
|
||||
if let Ok(new_token) = lock.add_buf(&[], &[event.as_bytes_mut()]) {
|
||||
// This only works because nothing happen between `pop_used` and `add` that affects
|
||||
// the list of free descriptors in the queue, so `add` reuses the descriptor which
|
||||
// was just freed by `pop_used`.
|
||||
|
@ -61,7 +61,7 @@ impl NetworkDevice {
|
||||
for i in 0..QUEUE_SIZE {
|
||||
let mut rx_buffer = RxBuffer::new(RX_BUFFER_LEN, size_of::<VirtioNetHdr>());
|
||||
// FIEME: Replace rx_buffer with VM segment-based data structure to use dma mapping.
|
||||
let token = recv_queue.add(&[], &[rx_buffer.buf_mut()])?;
|
||||
let token = recv_queue.add_buf(&[], &[rx_buffer.buf_mut()])?;
|
||||
assert_eq!(i, token);
|
||||
assert_eq!(rx_buffers.put(rx_buffer) as u16, i);
|
||||
}
|
||||
@ -111,7 +111,7 @@ impl NetworkDevice {
|
||||
fn add_rx_buffer(&mut self, mut rx_buffer: RxBuffer) -> Result<(), VirtioNetError> {
|
||||
let token = self
|
||||
.recv_queue
|
||||
.add(&[], &[rx_buffer.buf_mut()])
|
||||
.add_buf(&[], &[rx_buffer.buf_mut()])
|
||||
.map_err(queue_to_network_error)?;
|
||||
assert!(self.rx_buffers.put_at(token as usize, rx_buffer).is_none());
|
||||
if self.recv_queue.should_notify() {
|
||||
@ -143,7 +143,7 @@ impl NetworkDevice {
|
||||
let header = VirtioNetHdr::default();
|
||||
let token = self
|
||||
.send_queue
|
||||
.add(&[header.as_bytes(), tx_buffer.buf()], &[])
|
||||
.add_buf(&[header.as_bytes(), tx_buffer.buf()], &[])
|
||||
.map_err(queue_to_network_error)?;
|
||||
|
||||
if self.send_queue.should_notify() {
|
||||
|
@ -6,7 +6,7 @@ use alloc::vec::Vec;
|
||||
use aster_frame::{
|
||||
io_mem::IoMem,
|
||||
offset_of,
|
||||
vm::{DmaCoherent, VmAllocOptions},
|
||||
vm::{DmaCoherent, VmAllocOptions, VmReader, VmWriter},
|
||||
};
|
||||
use aster_rights::{Dup, TRightSet, TRights, Write};
|
||||
use aster_util::{field_ptr, safe_ptr::SafePtr};
|
||||
@ -173,10 +173,10 @@ impl VirtQueue {
|
||||
})
|
||||
}
|
||||
|
||||
/// Add buffers to the virtqueue, return a token.
|
||||
/// Add buffers to the virtqueue, return a token. **This function will be removed in the future.**
|
||||
///
|
||||
/// Ref: linux virtio_ring.c virtqueue_add
|
||||
pub fn add(&mut self, inputs: &[&[u8]], outputs: &[&mut [u8]]) -> Result<u16, QueueError> {
|
||||
pub fn add_buf(&mut self, inputs: &[&[u8]], outputs: &[&mut [u8]]) -> Result<u16, QueueError> {
|
||||
// FIXME: use `DmaSteam` for inputs and outputs. Now because the upper device driver lacks the
|
||||
// ability to safely construct DmaStream from slice, slice is still used here.
|
||||
// pub fn add(
|
||||
@ -184,6 +184,7 @@ impl VirtQueue {
|
||||
// inputs: &[&DmaStream],
|
||||
// outputs: &[&mut DmaStream],
|
||||
// ) -> Result<u16, QueueError> {
|
||||
|
||||
if inputs.is_empty() && outputs.is_empty() {
|
||||
return Err(QueueError::InvalidArgs);
|
||||
}
|
||||
@ -196,7 +197,7 @@ impl VirtQueue {
|
||||
let mut last = self.free_head;
|
||||
for input in inputs.iter() {
|
||||
let desc = &self.descs[self.free_head as usize];
|
||||
set_buf(&desc.borrow_vm().restrict::<TRights![Write, Dup]>(), input);
|
||||
set_buf_slice(&desc.borrow_vm().restrict::<TRights![Write, Dup]>(), input);
|
||||
field_ptr!(desc, Descriptor, flags)
|
||||
.write(&DescFlags::NEXT)
|
||||
.unwrap();
|
||||
@ -205,7 +206,74 @@ impl VirtQueue {
|
||||
}
|
||||
for output in outputs.iter() {
|
||||
let desc = &mut self.descs[self.free_head as usize];
|
||||
set_buf(&desc.borrow_vm().restrict::<TRights![Write, Dup]>(), output);
|
||||
set_buf_slice(&desc.borrow_vm().restrict::<TRights![Write, Dup]>(), output);
|
||||
field_ptr!(desc, Descriptor, flags)
|
||||
.write(&(DescFlags::NEXT | DescFlags::WRITE))
|
||||
.unwrap();
|
||||
last = self.free_head;
|
||||
self.free_head = field_ptr!(desc, Descriptor, next).read().unwrap();
|
||||
}
|
||||
// set last_elem.next = NULL
|
||||
{
|
||||
let desc = &mut self.descs[last as usize];
|
||||
let mut flags: DescFlags = field_ptr!(desc, Descriptor, flags).read().unwrap();
|
||||
flags.remove(DescFlags::NEXT);
|
||||
field_ptr!(desc, Descriptor, flags).write(&flags).unwrap();
|
||||
}
|
||||
self.num_used += (inputs.len() + outputs.len()) as u16;
|
||||
|
||||
let avail_slot = self.avail_idx & (self.queue_size - 1);
|
||||
|
||||
{
|
||||
let ring_ptr: SafePtr<[u16; 64], &DmaCoherent> =
|
||||
field_ptr!(&self.avail, AvailRing, ring);
|
||||
let mut ring_slot_ptr = ring_ptr.cast::<u16>();
|
||||
ring_slot_ptr.add(avail_slot as usize);
|
||||
ring_slot_ptr.write(&head).unwrap();
|
||||
}
|
||||
// write barrier
|
||||
fence(Ordering::SeqCst);
|
||||
|
||||
// increase head of avail ring
|
||||
self.avail_idx = self.avail_idx.wrapping_add(1);
|
||||
field_ptr!(&self.avail, AvailRing, idx)
|
||||
.write(&self.avail_idx)
|
||||
.unwrap();
|
||||
|
||||
fence(Ordering::SeqCst);
|
||||
Ok(head)
|
||||
}
|
||||
|
||||
/// Add VmReader/VmWriter to the virtqueue, return a token.
|
||||
///
|
||||
/// Ref: linux virtio_ring.c virtqueue_add
|
||||
pub fn add_vm(
|
||||
&mut self,
|
||||
inputs: &[&VmReader],
|
||||
outputs: &[&VmWriter],
|
||||
) -> Result<u16, QueueError> {
|
||||
if inputs.is_empty() && outputs.is_empty() {
|
||||
return Err(QueueError::InvalidArgs);
|
||||
}
|
||||
if inputs.len() + outputs.len() + self.num_used as usize > self.queue_size as usize {
|
||||
return Err(QueueError::BufferTooSmall);
|
||||
}
|
||||
|
||||
// allocate descriptors from free list
|
||||
let head = self.free_head;
|
||||
let mut last = self.free_head;
|
||||
for input in inputs.iter() {
|
||||
let desc = &self.descs[self.free_head as usize];
|
||||
set_buf_reader(&desc.borrow_vm().restrict::<TRights![Write, Dup]>(), input);
|
||||
field_ptr!(desc, Descriptor, flags)
|
||||
.write(&DescFlags::NEXT)
|
||||
.unwrap();
|
||||
last = self.free_head;
|
||||
self.free_head = field_ptr!(desc, Descriptor, next).read().unwrap();
|
||||
}
|
||||
for output in outputs.iter() {
|
||||
let desc = &mut self.descs[self.free_head as usize];
|
||||
set_buf_writer(&desc.borrow_vm().restrict::<TRights![Write, Dup]>(), output);
|
||||
field_ptr!(desc, Descriptor, flags)
|
||||
.write(&(DescFlags::NEXT | DescFlags::WRITE))
|
||||
.unwrap();
|
||||
@ -366,20 +434,49 @@ pub struct Descriptor {
|
||||
next: u16,
|
||||
}
|
||||
|
||||
type DescriptorPtr<'a> = SafePtr<Descriptor, &'a DmaCoherent, TRightSet<TRights![Dup, Write]>>;
|
||||
|
||||
#[inline]
|
||||
#[allow(clippy::type_complexity)]
|
||||
fn set_buf(ptr: &SafePtr<Descriptor, &DmaCoherent, TRightSet<TRights![Dup, Write]>>, buf: &[u8]) {
|
||||
fn set_buf_slice(desc_ptr: &DescriptorPtr, buf: &[u8]) {
|
||||
// FIXME: use `DmaSteam` for buf. Now because the upper device driver lacks the
|
||||
// ability to safely construct DmaStream from slice, slice is still used here.
|
||||
let va = buf.as_ptr() as usize;
|
||||
let pa = aster_frame::vm::vaddr_to_paddr(va).unwrap();
|
||||
field_ptr!(ptr, Descriptor, addr)
|
||||
field_ptr!(desc_ptr, Descriptor, addr)
|
||||
.write(&(pa as u64))
|
||||
.unwrap();
|
||||
field_ptr!(ptr, Descriptor, len)
|
||||
field_ptr!(desc_ptr, Descriptor, len)
|
||||
.write(&(buf.len() as u32))
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
#[inline]
|
||||
#[allow(clippy::type_complexity)]
|
||||
fn set_buf_reader(desc_ptr: &DescriptorPtr, reader: &VmReader) {
|
||||
let va = reader.cursor() as usize;
|
||||
let pa = aster_frame::vm::vaddr_to_paddr(va).unwrap();
|
||||
field_ptr!(desc_ptr, Descriptor, addr)
|
||||
.write(&(pa as u64))
|
||||
.unwrap();
|
||||
field_ptr!(desc_ptr, Descriptor, len)
|
||||
.write(&(reader.remain() as u32))
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
#[inline]
|
||||
#[allow(clippy::type_complexity)]
|
||||
fn set_buf_writer(desc_ptr: &DescriptorPtr, writer: &VmWriter) {
|
||||
let va = writer.cursor() as usize;
|
||||
let pa = aster_frame::vm::vaddr_to_paddr(va).unwrap();
|
||||
field_ptr!(desc_ptr, Descriptor, addr)
|
||||
.write(&(pa as u64))
|
||||
.unwrap();
|
||||
field_ptr!(desc_ptr, Descriptor, len)
|
||||
.write(&(writer.avail() as u32))
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
bitflags! {
|
||||
/// Descriptor flags
|
||||
#[derive(Pod, Default)]
|
||||
|
@ -1,3 +1,10 @@
|
||||
use core::mem::size_of;
|
||||
|
||||
use alloc::vec::Vec;
|
||||
use aster_frame::{
|
||||
println,
|
||||
vm::{VmAllocOptions, VmIo},
|
||||
};
|
||||
use log::info;
|
||||
|
||||
pub fn init() {
|
||||
@ -10,18 +17,39 @@ pub fn init() {
|
||||
#[allow(unused)]
|
||||
fn block_device_test() {
|
||||
for (_, device) in aster_block::all_devices() {
|
||||
let mut write_buffer = [0u8; 512];
|
||||
let mut read_buffer = [0u8; 512];
|
||||
info!("write_buffer address:{:x}", write_buffer.as_ptr() as usize);
|
||||
info!("read_buffer address:{:x}", read_buffer.as_ptr() as usize);
|
||||
for i in 0..512 {
|
||||
for byte in write_buffer.iter_mut() {
|
||||
*byte = i as u8;
|
||||
let write_frame = VmAllocOptions::new(1).alloc_single().unwrap();
|
||||
let read_frame = VmAllocOptions::new(1).alloc_single().unwrap();
|
||||
info!("write_buffer address:{:x}", write_frame.start_paddr());
|
||||
info!("read_buffer address:{:x}", read_frame.start_paddr());
|
||||
|
||||
// init write frame
|
||||
for i in 0..=8 {
|
||||
let slice: [u8; 512] = [i; 512];
|
||||
write_frame.write_slice(i as usize * 512, &slice);
|
||||
}
|
||||
device.write_block(i as usize, &write_buffer);
|
||||
device.read_block(i as usize, &mut read_buffer);
|
||||
assert_eq!(write_buffer, read_buffer);
|
||||
|
||||
// Test multiple Writer & Reader
|
||||
let mut writers = Vec::with_capacity(8);
|
||||
for i in 0..8 {
|
||||
let writer = read_frame.writer().skip(i * 512).limit(512);
|
||||
writers.push(writer);
|
||||
}
|
||||
info!("block device test passed!");
|
||||
|
||||
let mut readers = Vec::with_capacity(8);
|
||||
for i in 0..8 {
|
||||
let reader = write_frame.reader().skip(i * 512).limit(512);
|
||||
readers.push(reader);
|
||||
}
|
||||
|
||||
device.write_block(0, readers.as_slice());
|
||||
device.read_block(0, writers.as_slice());
|
||||
let mut read_slice = [0u8; 512];
|
||||
let mut write_slice = [0u8; 512];
|
||||
for i in 0..8 {
|
||||
read_frame.read_bytes(i * size_of::<[u8; 512]>(), &mut read_slice);
|
||||
write_frame.read_bytes(i * size_of::<[u8; 512]>(), &mut write_slice);
|
||||
assert_eq!(read_slice, write_slice);
|
||||
}
|
||||
println!("block device test passed!");
|
||||
}
|
||||
}
|
||||
|
Reference in New Issue
Block a user