Parcourir la source

Merge pull request #61 from rcore-os/net

Refactor virtio network device APIs to improve driver performance
chyyuu il y a 2 ans
Parent
commit
af3c127b40

+ 2 - 11
examples/aarch64/src/main.rs

@@ -23,7 +23,7 @@ use hal::HalImpl;
 use log::{debug, error, info, trace, warn, LevelFilter};
 use psci::system_off;
 use virtio_drivers::{
-    device::{blk::VirtIOBlk, console::VirtIOConsole, gpu::VirtIOGpu, net::VirtIONet},
+    device::{blk::VirtIOBlk, console::VirtIOConsole, gpu::VirtIOGpu},
     transport::{
         mmio::{MmioTransport, VirtIOHeader},
         pci::{
@@ -114,7 +114,7 @@ fn virtio_device(transport: impl Transport) {
     match transport.device_type() {
         DeviceType::Block => virtio_blk(transport),
         DeviceType::GPU => virtio_gpu(transport),
-        DeviceType::Network => virtio_net(transport),
+        // DeviceType::Network => virtio_net(transport), // currently is unsupported without alloc
         DeviceType::Console => virtio_console(transport),
         t => warn!("Unrecognized virtio device: {:?}", t),
     }
@@ -165,15 +165,6 @@ fn virtio_gpu<T: Transport>(transport: T) {
     info!("virtio-gpu test finished");
 }
 
-fn virtio_net<T: Transport>(transport: T) {
-    let mut net = VirtIONet::<HalImpl, T>::new(transport).expect("failed to create net driver");
-    let mut buf = [0u8; 0x100];
-    let len = net.recv(&mut buf).expect("failed to recv");
-    info!("recv: {:?}", &buf[..len]);
-    net.send(&buf[..len]).expect("failed to send");
-    info!("virtio-net test finished");
-}
-
 fn virtio_console<T: Transport>(transport: T) {
     let mut console =
         VirtIOConsole::<HalImpl, T>::new(transport).expect("Failed to create console driver");

+ 16 - 1
examples/riscv/Cargo.toml

@@ -6,10 +6,25 @@ edition = "2018"
 
 # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
 
+[features]
+tcp = ["smoltcp"]
+default = ["tcp"]
+
 [dependencies]
 log = "0.4"
-riscv = "0.6"
+riscv = "0.10"
 opensbi-rt = { git = "https://github.com/rcore-os/opensbi-rt.git", rev = "abdfeb72" }
 fdt = "0.1.4"
 virtio-drivers = { path = "../.." }
 lazy_static = { version = "1.4", features = ["spin_no_std"] }
+
+[dependencies.smoltcp]
+version = "0.9.1"
+optional = true
+default-features = false
+features = [
+  "alloc", "log",   # no std
+  "medium-ethernet",
+  "proto-ipv4",
+  "socket-raw", "socket-icmp", "socket-udp", "socket-tcp",
+]

+ 11 - 3
examples/riscv/Makefile

@@ -4,6 +4,8 @@ mode := release
 kernel := target/$(target)/$(mode)/riscv
 img := target/$(target)/$(mode)/img
 
+tcp ?= off
+
 sysroot := $(shell rustc --print sysroot)
 objdump := $(shell find $(sysroot) -name llvm-objdump) --arch-name=$(arch)
 objcopy := $(shell find $(sysroot) -name llvm-objcopy)
@@ -13,6 +15,12 @@ ifeq ($(mode), release)
 	BUILD_ARGS += --release
 endif
 
+ifeq ($(tcp), on)
+	BUILD_ARGS += --features tcp
+else
+	BUILD_ARGS += --no-default-features
+endif
+
 .PHONY: kernel build clean qemu run env
 
 build: $(bin)
@@ -38,7 +46,7 @@ clean:
 
 qemu-legacy: kernel $(img)
 # Wait a few seconds, then try to open a connection to the VM so it can test its networking.
-	( sleep 4 && nc localhost 5555 -v < /dev/null) &
+	( sleep 4 && echo "hello" | nc localhost 5555 -N -v) &
 	qemu-system-$(arch) \
 	  $(QEMU_ARGS) \
 		-machine virt \
@@ -54,7 +62,7 @@ qemu-legacy: kernel $(img)
 
 qemu: kernel $(img)
 # Wait a few seconds, then try to open a connection to the VM so it can test its networking.
-	( sleep 4 && nc localhost 5555 -v < /dev/null) &
+	( sleep 4 && echo "hello" | nc localhost 5555 -N -v) &
 	qemu-system-$(arch) \
 	  $(QEMU_ARGS) \
 		-machine virt \
@@ -73,4 +81,4 @@ qemu: kernel $(img)
 $(img):
 	dd if=/dev/zero of=$@ bs=512 count=32
 
-run: build qemu-legacy qemu
+run: build qemu-legacy qemu

+ 36 - 8
examples/riscv/src/main.rs

@@ -2,13 +2,16 @@
 #![no_main]
 #![deny(warnings)]
 
+#[macro_use]
+extern crate log;
+
 extern crate alloc;
 extern crate opensbi_rt;
 
 use alloc::vec;
 use core::ptr::NonNull;
 use fdt::{node::FdtNode, standard_nodes::Compatible, Fdt};
-use log::{info, warn, LevelFilter};
+use log::LevelFilter;
 use virtio_drivers::{
     device::{blk::VirtIOBlk, gpu::VirtIOGpu, input::VirtIOInput, net::VirtIONet},
     transport::{
@@ -20,6 +23,12 @@ use virtio_impl::HalImpl;
 
 mod virtio_impl;
 
+#[cfg(feature = "tcp")]
+mod tcp;
+
+const NET_BUFFER_LEN: usize = 2048;
+const NET_QUEUE_SIZE: usize = 16;
+
 #[no_mangle]
 extern "C" fn main(_hartid: usize, device_tree_paddr: usize) {
     log::set_max_level(LevelFilter::Info);
@@ -114,7 +123,7 @@ fn virtio_gpu<T: Transport>(transport: T) {
     gpu.flush().expect("failed to flush");
     //delay some time
     info!("virtio-gpu show graphics....");
-    for _ in 0..100000 {
+    for _ in 0..10000 {
         for _ in 0..100000 {
             unsafe {
                 core::arch::asm!("nop");
@@ -137,10 +146,29 @@ fn virtio_input<T: Transport>(transport: T) {
 }
 
 fn virtio_net<T: Transport>(transport: T) {
-    let mut net = VirtIONet::<HalImpl, T>::new(transport).expect("failed to create net driver");
-    let mut buf = [0u8; 0x100];
-    let len = net.recv(&mut buf).expect("failed to recv");
-    info!("recv: {:?}", &buf[..len]);
-    net.send(&buf[..len]).expect("failed to send");
-    info!("virtio-net test finished");
+    let net = VirtIONet::<HalImpl, T, NET_QUEUE_SIZE>::new(transport, NET_BUFFER_LEN)
+        .expect("failed to create net driver");
+    info!("MAC address: {:02x?}", net.mac_address());
+
+    #[cfg(not(feature = "tcp"))]
+    {
+        let mut net = net;
+        loop {
+            match net.receive() {
+                Ok(buf) => {
+                    info!("RECV {} bytes: {:02x?}", buf.packet_len(), buf.packet());
+                    let tx_buf = virtio_drivers::device::net::TxBuffer::from(buf.packet());
+                    net.send(tx_buf).expect("failed to send");
+                    net.recycle_rx_buffer(buf).unwrap();
+                    break;
+                }
+                Err(virtio_drivers::Error::NotReady) => continue,
+                Err(err) => panic!("failed to recv: {:?}", err),
+            }
+        }
+        info!("virtio-net test finished");
+    }
+
+    #[cfg(feature = "tcp")]
+    tcp::test_echo_server(net);
 }

+ 180 - 0
examples/riscv/src/tcp.rs

@@ -0,0 +1,180 @@
+//! Simple echo server over TCP.
+//!
+//! Ref: https://github.com/smoltcp-rs/smoltcp/blob/master/examples/server.rs
+
+use alloc::{borrow::ToOwned, rc::Rc, vec, vec::Vec};
+use core::{cell::RefCell, str::FromStr};
+
+use smoltcp::iface::{Config, Interface, SocketSet};
+use smoltcp::phy::{Device, DeviceCapabilities, Medium, RxToken, TxToken};
+use smoltcp::wire::{EthernetAddress, IpAddress, IpCidr, Ipv4Address};
+use smoltcp::{socket::tcp, time::Instant};
+use virtio_drivers::device::net::{RxBuffer, VirtIONet};
+use virtio_drivers::{transport::Transport, Error};
+
+use super::{HalImpl, NET_QUEUE_SIZE};
+
+type DeviceImpl<T> = VirtIONet<HalImpl, T, NET_QUEUE_SIZE>;
+
+const IP: &str = "10.0.2.15"; // QEMU user networking default IP
+const GATEWAY: &str = "10.0.2.2"; // QEMU user networking gateway
+const PORT: u16 = 5555;
+
+struct DeviceWrapper<T: Transport> {
+    inner: Rc<RefCell<DeviceImpl<T>>>,
+}
+
+impl<T: Transport> DeviceWrapper<T> {
+    fn new(dev: DeviceImpl<T>) -> Self {
+        DeviceWrapper {
+            inner: Rc::new(RefCell::new(dev)),
+        }
+    }
+
+    fn mac_address(&self) -> EthernetAddress {
+        EthernetAddress(self.inner.borrow().mac_address())
+    }
+}
+
+impl<T: Transport> Device for DeviceWrapper<T> {
+    type RxToken<'a> = VirtioRxToken<T> where Self: 'a;
+    type TxToken<'a> = VirtioTxToken<T> where Self: 'a;
+
+    fn receive(&mut self, _timestamp: Instant) -> Option<(Self::RxToken<'_>, Self::TxToken<'_>)> {
+        match self.inner.borrow_mut().receive() {
+            Ok(buf) => Some((
+                VirtioRxToken(self.inner.clone(), buf),
+                VirtioTxToken(self.inner.clone()),
+            )),
+            Err(Error::NotReady) => None,
+            Err(err) => panic!("receive failed: {}", err),
+        }
+    }
+
+    fn transmit(&mut self, _timestamp: Instant) -> Option<Self::TxToken<'_>> {
+        Some(VirtioTxToken(self.inner.clone()))
+    }
+
+    fn capabilities(&self) -> DeviceCapabilities {
+        let mut caps = DeviceCapabilities::default();
+        caps.max_transmission_unit = 1536;
+        caps.max_burst_size = Some(1);
+        caps.medium = Medium::Ethernet;
+        caps
+    }
+}
+
+struct VirtioRxToken<T: Transport>(Rc<RefCell<DeviceImpl<T>>>, RxBuffer);
+struct VirtioTxToken<T: Transport>(Rc<RefCell<DeviceImpl<T>>>);
+
+impl<T: Transport> RxToken for VirtioRxToken<T> {
+    fn consume<R, F>(self, f: F) -> R
+    where
+        F: FnOnce(&mut [u8]) -> R,
+    {
+        let mut rx_buf = self.1;
+        trace!(
+            "RECV {} bytes: {:02X?}",
+            rx_buf.packet_len(),
+            rx_buf.packet()
+        );
+        let result = f(rx_buf.packet_mut());
+        self.0.borrow_mut().recycle_rx_buffer(rx_buf).unwrap();
+        result
+    }
+}
+
+impl<T: Transport> TxToken for VirtioTxToken<T> {
+    fn consume<R, F>(self, len: usize, f: F) -> R
+    where
+        F: FnOnce(&mut [u8]) -> R,
+    {
+        let mut dev = self.0.borrow_mut();
+        let mut tx_buf = dev.new_tx_buffer(len);
+        let result = f(tx_buf.packet_mut());
+        trace!("SEND {} bytes: {:02X?}", len, tx_buf.packet());
+        dev.send(tx_buf).unwrap();
+        result
+    }
+}
+
+pub fn test_echo_server<T: Transport>(dev: DeviceImpl<T>) {
+    let mut device = DeviceWrapper::new(dev);
+
+    // Create interface
+    let mut config = Config::new();
+    config.random_seed = 0x2333;
+    if device.capabilities().medium == Medium::Ethernet {
+        config.hardware_addr = Some(device.mac_address().into());
+    }
+
+    let mut iface = Interface::new(config, &mut device);
+    iface.update_ip_addrs(|ip_addrs| {
+        ip_addrs
+            .push(IpCidr::new(IpAddress::from_str(IP).unwrap(), 24))
+            .unwrap();
+    });
+
+    iface
+        .routes_mut()
+        .add_default_ipv4_route(Ipv4Address::from_str(GATEWAY).unwrap())
+        .unwrap();
+
+    // Create sockets
+    let tcp_rx_buffer = tcp::SocketBuffer::new(vec![0; 1024]);
+    let tcp_tx_buffer = tcp::SocketBuffer::new(vec![0; 1024]);
+    let tcp_socket = tcp::Socket::new(tcp_rx_buffer, tcp_tx_buffer);
+
+    let mut sockets = SocketSet::new(vec![]);
+    let tcp_handle = sockets.add(tcp_socket);
+
+    info!("start a reverse echo server...");
+    let mut tcp_active = false;
+    loop {
+        let timestamp = Instant::from_micros_const(riscv::register::time::read() as i64 / 10);
+        iface.poll(timestamp, &mut device, &mut sockets);
+
+        // tcp:PORT: echo with reverse
+        let socket = sockets.get_mut::<tcp::Socket>(tcp_handle);
+        if !socket.is_open() {
+            info!("listening on port {}...", PORT);
+            socket.listen(PORT).unwrap();
+        }
+
+        if socket.is_active() && !tcp_active {
+            info!("tcp:{} connected", PORT);
+        } else if !socket.is_active() && tcp_active {
+            info!("tcp:{} disconnected", PORT);
+        }
+        tcp_active = socket.is_active();
+
+        if socket.may_recv() {
+            let data = socket
+                .recv(|buffer| {
+                    let recvd_len = buffer.len();
+                    if !buffer.is_empty() {
+                        debug!("tcp:{} recv {} bytes: {:?}", PORT, recvd_len, buffer);
+                        let mut lines = buffer
+                            .split(|&b| b == b'\n')
+                            .map(ToOwned::to_owned)
+                            .collect::<Vec<_>>();
+                        for line in lines.iter_mut() {
+                            line.reverse();
+                        }
+                        let data = lines.join(&b'\n');
+                        (recvd_len, data)
+                    } else {
+                        (0, vec![])
+                    }
+                })
+                .unwrap();
+            if socket.can_send() && !data.is_empty() {
+                debug!("tcp:{} send data: {:?}", PORT, data);
+                socket.send_slice(&data[..]).unwrap();
+            }
+        } else if socket.may_send() {
+            info!("tcp:{} close", PORT);
+            socket.close();
+        }
+    }
+}

+ 2 - 1
src/device/input.rs

@@ -76,6 +76,7 @@ impl<H: Hal, T: Transport> VirtIOInput<H, T> {
                     .pop_used(token, &[], &mut [event.as_bytes_mut()])
                     .ok()?;
             }
+            let event_saved = *event;
             // requeue
             // Safe because buffer lasts as long as the queue.
             if let Ok(new_token) = unsafe { self.event_queue.add(&[], &mut [event.as_bytes_mut()]) }
@@ -87,7 +88,7 @@ impl<H: Hal, T: Transport> VirtIOInput<H, T> {
                 if self.event_queue.should_notify() {
                     self.transport.notify(QUEUE_EVENT);
                 }
-                return Some(*event);
+                return Some(event_saved);
             }
         }
         None

+ 1 - 0
src/device/mod.rs

@@ -5,4 +5,5 @@ pub mod console;
 pub mod gpu;
 #[cfg(feature = "alloc")]
 pub mod input;
+#[cfg(feature = "alloc")]
 pub mod net;

+ 192 - 33
src/device/net.rs

@@ -4,13 +4,95 @@ use crate::hal::Hal;
 use crate::queue::VirtQueue;
 use crate::transport::Transport;
 use crate::volatile::{volread, ReadOnly};
-use crate::Result;
+use crate::{Error, Result};
+use alloc::{vec, vec::Vec};
 use bitflags::bitflags;
-use core::mem::{size_of, MaybeUninit};
-use log::{debug, info};
+use core::mem::size_of;
+use log::{debug, info, warn};
 use zerocopy::{AsBytes, FromBytes};
 
-const QUEUE_SIZE: u16 = 2;
+const MAX_BUFFER_LEN: usize = 65535;
+const MIN_BUFFER_LEN: usize = 1526;
+const NET_HDR_SIZE: usize = size_of::<VirtioNetHdr>();
+
+/// A buffer used for transmitting.
+pub struct TxBuffer(Vec<u8>);
+
+/// A buffer used for receiving.
+pub struct RxBuffer {
+    buf: Vec<usize>, // for alignment
+    packet_len: usize,
+    idx: usize,
+}
+
+impl TxBuffer {
+    /// Constructs the buffer from the given slice.
+    pub fn from(buf: &[u8]) -> Self {
+        Self(Vec::from(buf))
+    }
+
+    /// Returns the network packet length.
+    pub fn packet_len(&self) -> usize {
+        self.0.len()
+    }
+
+    /// Returns the network packet as a slice.
+    pub fn packet(&self) -> &[u8] {
+        self.0.as_slice()
+    }
+
+    /// Returns the network packet as a mutable slice.
+    pub fn packet_mut(&mut self) -> &mut [u8] {
+        self.0.as_mut_slice()
+    }
+}
+
+impl RxBuffer {
+    /// Allocates a new buffer with length `buf_len`.
+    fn new(idx: usize, buf_len: usize) -> Self {
+        Self {
+            buf: vec![0; buf_len / size_of::<usize>()],
+            packet_len: 0,
+            idx,
+        }
+    }
+
+    /// Set the network packet length.
+    fn set_packet_len(&mut self, packet_len: usize) {
+        self.packet_len = packet_len
+    }
+
+    /// Returns the network packet length (witout header).
+    pub const fn packet_len(&self) -> usize {
+        self.packet_len
+    }
+
+    /// Returns all data in the buffer, including both the header and the packet.
+    pub fn as_bytes(&self) -> &[u8] {
+        self.buf.as_bytes()
+    }
+
+    /// Returns all data in the buffer with the mutable reference,
+    /// including both the header and the packet.
+    pub fn as_bytes_mut(&mut self) -> &mut [u8] {
+        self.buf.as_bytes_mut()
+    }
+
+    /// Returns the reference of the header.
+    pub fn header(&self) -> &VirtioNetHdr {
+        unsafe { &*(self.buf.as_ptr() as *const VirtioNetHdr) }
+    }
+
+    /// Returns the network packet as a slice.
+    pub fn packet(&self) -> &[u8] {
+        &self.buf.as_bytes()[NET_HDR_SIZE..NET_HDR_SIZE + self.packet_len]
+    }
+
+    /// Returns the network packet as a mutable slice.
+    pub fn packet_mut(&mut self) -> &mut [u8] {
+        &mut self.buf.as_bytes_mut()[NET_HDR_SIZE..NET_HDR_SIZE + self.packet_len]
+    }
+}
 
 /// The virtio network device is a virtual ethernet card.
 ///
@@ -19,16 +101,17 @@ const QUEUE_SIZE: u16 = 2;
 /// Empty buffers are placed in one virtqueue for receiving packets, and
 /// outgoing packets are enqueued into another for transmission in that order.
 /// A third command queue is used to control advanced filtering features.
-pub struct VirtIONet<H: Hal, T: Transport> {
+pub struct VirtIONet<H: Hal, T: Transport, const QUEUE_SIZE: usize> {
     transport: T,
     mac: EthernetAddress,
-    recv_queue: VirtQueue<H, { QUEUE_SIZE as usize }>,
-    send_queue: VirtQueue<H, { QUEUE_SIZE as usize }>,
+    recv_queue: VirtQueue<H, QUEUE_SIZE>,
+    send_queue: VirtQueue<H, QUEUE_SIZE>,
+    rx_buffers: [Option<RxBuffer>; QUEUE_SIZE],
 }
 
-impl<H: Hal, T: Transport> VirtIONet<H, T> {
+impl<H: Hal, T: Transport, const QUEUE_SIZE: usize> VirtIONet<H, T, QUEUE_SIZE> {
     /// Create a new VirtIO-Net driver.
-    pub fn new(mut transport: T) -> Result<Self> {
+    pub fn new(mut transport: T, buf_len: usize) -> Result<Self> {
         transport.begin_init(|features| {
             let features = Features::from_bits_truncate(features);
             info!("Device features {:?}", features);
@@ -41,11 +124,37 @@ impl<H: Hal, T: Transport> VirtIONet<H, T> {
         // Safe because config points to a valid MMIO region for the config space.
         unsafe {
             mac = volread!(config, mac);
-            debug!("Got MAC={:?}, status={:?}", mac, volread!(config, status));
+            debug!(
+                "Got MAC={:02x?}, status={:?}",
+                mac,
+                volread!(config, status)
+            );
+        }
+
+        if !(MIN_BUFFER_LEN..=MAX_BUFFER_LEN).contains(&buf_len) {
+            warn!(
+                "Receive buffer len {} is not in range [{}, {}]",
+                buf_len, MIN_BUFFER_LEN, MAX_BUFFER_LEN
+            );
+            return Err(Error::InvalidParam);
         }
 
-        let recv_queue = VirtQueue::new(&mut transport, QUEUE_RECEIVE)?;
         let send_queue = VirtQueue::new(&mut transport, QUEUE_TRANSMIT)?;
+        let mut recv_queue = VirtQueue::new(&mut transport, QUEUE_RECEIVE)?;
+
+        const NONE_BUF: Option<RxBuffer> = None;
+        let mut rx_buffers = [NONE_BUF; QUEUE_SIZE];
+        for (i, rx_buf_place) in rx_buffers.iter_mut().enumerate() {
+            let mut rx_buf = RxBuffer::new(i, buf_len);
+            // Safe because the buffer lives as long as the queue.
+            let token = unsafe { recv_queue.add(&[], &mut [rx_buf.as_bytes_mut()])? };
+            assert_eq!(token, i as u16);
+            *rx_buf_place = Some(rx_buf);
+        }
+
+        if recv_queue.should_notify() {
+            transport.notify(QUEUE_RECEIVE);
+        }
 
         transport.finish_init();
 
@@ -54,6 +163,7 @@ impl<H: Hal, T: Transport> VirtIONet<H, T> {
             mac,
             recv_queue,
             send_queue,
+            rx_buffers,
         })
     }
 
@@ -63,7 +173,7 @@ impl<H: Hal, T: Transport> VirtIONet<H, T> {
     }
 
     /// Get MAC address.
-    pub fn mac(&self) -> EthernetAddress {
+    pub fn mac_address(&self) -> EthernetAddress {
         self.mac
     }
 
@@ -77,24 +187,66 @@ impl<H: Hal, T: Transport> VirtIONet<H, T> {
         self.recv_queue.can_pop()
     }
 
-    /// Receive a packet.
-    pub fn recv(&mut self, buf: &mut [u8]) -> Result<usize> {
-        let mut header = MaybeUninit::<Header>::uninit();
-        let header_buf = unsafe { (*header.as_mut_ptr()).as_bytes_mut() };
-        let len = self.recv_queue.add_notify_wait_pop(
-            &[],
-            &mut [header_buf, buf],
-            &mut self.transport,
-        )?;
-        // let header = unsafe { header.assume_init() };
-        Ok(len as usize - size_of::<Header>())
+    /// Receives a [`RxBuffer`] from network. If currently no data, returns an
+    /// error with type [`Error::NotReady`].
+    ///
+    /// It will try to pop a buffer that completed data reception in the
+    /// NIC queue.
+    pub fn receive(&mut self) -> Result<RxBuffer> {
+        if let Some(token) = self.recv_queue.peek_used() {
+            let mut rx_buf = self.rx_buffers[token as usize]
+                .take()
+                .ok_or(Error::WrongToken)?;
+            if token as usize != rx_buf.idx {
+                return Err(Error::WrongToken);
+            }
+
+            // Safe because `token` == `rx_buf.idx`, we are passing the same
+            // buffer as we passed to `VirtQueue::add` and it is still valid.
+            let len = unsafe {
+                self.recv_queue
+                    .pop_used(token, &[], &mut [rx_buf.as_bytes_mut()])?
+            };
+            if (len as usize) < NET_HDR_SIZE {
+                Err(Error::IoError)
+            } else {
+                rx_buf.set_packet_len(len as usize - NET_HDR_SIZE);
+                Ok(rx_buf)
+            }
+        } else {
+            Err(Error::NotReady)
+        }
+    }
+
+    /// Gives back the ownership of `rx_buf`, and recycles it for next use.
+    ///
+    /// It will add the buffer back to the NIC queue.
+    pub fn recycle_rx_buffer(&mut self, mut rx_buf: RxBuffer) -> Result {
+        let old_token = rx_buf.idx;
+        // Safe because we take the ownership of `rx_buf` back to `rx_buffers`,
+        // it lives as long as the queue.
+        let new_token = unsafe { self.recv_queue.add(&[], &mut [rx_buf.as_bytes_mut()]) }?;
+        if new_token as usize != old_token {
+            return Err(Error::WrongToken);
+        }
+        self.rx_buffers[old_token] = Some(rx_buf);
+        if self.recv_queue.should_notify() {
+            self.transport.notify(QUEUE_RECEIVE);
+        }
+        Ok(())
     }
 
-    /// Send a packet.
-    pub fn send(&mut self, buf: &[u8]) -> Result {
-        let header = unsafe { MaybeUninit::<Header>::zeroed().assume_init() };
+    /// Allocate a new buffer for transmitting.
+    pub fn new_tx_buffer(&self, buf_len: usize) -> TxBuffer {
+        TxBuffer(vec![0; buf_len])
+    }
+
+    /// Sends a [`TxBuffer`] to the network, and blocks until the request
+    /// completed.
+    pub fn send(&mut self, tx_buf: TxBuffer) -> Result {
+        let header = VirtioNetHdr::default();
         self.send_queue.add_notify_wait_pop(
-            &[header.as_bytes(), buf],
+            &[header.as_bytes(), tx_buf.packet()],
             &mut [],
             &mut self.transport,
         )?;
@@ -102,7 +254,7 @@ impl<H: Hal, T: Transport> VirtIONet<H, T> {
     }
 }
 
-impl<H: Hal, T: Transport> Drop for VirtIONet<H, T> {
+impl<H: Hal, T: Transport, const QUEUE_SIZE: usize> Drop for VirtIONet<H, T, QUEUE_SIZE> {
     fn drop(&mut self) {
         // Clear any pointers pointing to DMA regions, so the device doesn't try to access them
         // after they have been freed.
@@ -190,26 +342,33 @@ bitflags! {
 struct Config {
     mac: ReadOnly<EthernetAddress>,
     status: ReadOnly<Status>,
+    max_virtqueue_pairs: ReadOnly<u16>,
+    mtu: ReadOnly<u16>,
 }
 
 type EthernetAddress = [u8; 6];
 
-// virtio 5.1.6 Device Operation
+/// VirtIO 5.1.6 Device Operation:
+///
+/// Packets are transmitted by placing them in the transmitq1. . .transmitqN,
+/// and buffers for incoming packets are placed in the receiveq1. . .receiveqN.
+/// In each case, the packet itself is preceded by a header.
 #[repr(C)]
-#[derive(AsBytes, Debug, FromBytes)]
-struct Header {
+#[derive(AsBytes, Debug, Default, FromBytes)]
+pub struct VirtioNetHdr {
     flags: Flags,
     gso_type: GsoType,
     hdr_len: u16, // cannot rely on this
     gso_size: u16,
     csum_start: u16,
     csum_offset: u16,
+    // num_buffers: u16, // only available when the feature MRG_RXBUF is negotiated.
     // payload starts from here
 }
 
 bitflags! {
     #[repr(transparent)]
-    #[derive(AsBytes, FromBytes)]
+    #[derive(AsBytes, Default, FromBytes)]
     struct Flags: u8 {
         const NEEDS_CSUM = 1;
         const DATA_VALID = 2;
@@ -218,7 +377,7 @@ bitflags! {
 }
 
 #[repr(transparent)]
-#[derive(AsBytes, Debug, Copy, Clone, Eq, FromBytes, PartialEq)]
+#[derive(AsBytes, Debug, Copy, Clone, Default, Eq, FromBytes, PartialEq)]
 struct GsoType(u8);
 
 impl GsoType {