• Home
  • Line#
  • Scopes#
  • Navigate#
  • Raw
  • Download
1 //! Driver for VirtIO socket devices.
2 #![deny(unsafe_op_in_unsafe_fn)]
3 
4 use super::error::SocketError;
5 use super::protocol::{
6     Feature, StreamShutdown, VirtioVsockConfig, VirtioVsockHdr, VirtioVsockOp, VsockAddr,
7     VMADDR_CID_HOST,
8 };
9 use super::DEFAULT_RX_BUFFER_SIZE;
10 use crate::config::read_config;
11 use crate::hal::{DeviceHal, Hal};
12 use crate::queue::{owning::OwningQueue, DeviceVirtQueue, VirtQueue};
13 use crate::transport::{DeviceTransport, Transport};
14 use crate::Result;
15 use core::mem::size_of;
16 use log::debug;
17 use zerocopy::{FromBytes, IntoBytes};
18 
19 pub(crate) const RX_QUEUE_IDX: u16 = 0;
20 pub(crate) const TX_QUEUE_IDX: u16 = 1;
21 const EVENT_QUEUE_IDX: u16 = 2;
22 
23 pub(crate) const QUEUE_SIZE: usize = 8;
24 const SUPPORTED_FEATURES: Feature = Feature::RING_EVENT_IDX.union(Feature::RING_INDIRECT_DESC);
25 
26 /// Information about a particular vsock connection.
27 #[derive(Clone, Debug, Default, PartialEq, Eq)]
28 pub struct ConnectionInfo {
29     /// The address of the peer.
30     pub dst: VsockAddr,
31     /// The local port number associated with the connection.
32     pub src_port: u32,
33     /// The last `buf_alloc` value the peer sent to us, indicating how much receive buffer space in
34     /// bytes it has allocated for packet bodies.
35     peer_buf_alloc: u32,
36     /// The last `fwd_cnt` value the peer sent to us, indicating how many bytes of packet bodies it
37     /// has finished processing.
38     peer_fwd_cnt: u32,
39     /// The number of bytes of packet bodies which we have sent to the peer.
40     tx_cnt: u32,
41     /// The number of bytes of buffer space we have allocated to receive packet bodies from the
42     /// peer.
43     pub buf_alloc: u32,
44     /// The number of bytes of packet bodies which we have received from the peer and handled.
45     fwd_cnt: u32,
46     /// Whether we have recently requested credit from the peer.
47     ///
48     /// This is set to true when we send a `VIRTIO_VSOCK_OP_CREDIT_REQUEST`, and false when we
49     /// receive a `VIRTIO_VSOCK_OP_CREDIT_UPDATE`.
50     has_pending_credit_request: bool,
51 }
52 
53 impl ConnectionInfo {
54     /// Creates a new `ConnectionInfo` for the given peer address and local port, and default values
55     /// for everything else.
new(destination: VsockAddr, src_port: u32) -> Self56     pub fn new(destination: VsockAddr, src_port: u32) -> Self {
57         Self {
58             dst: destination,
59             src_port,
60             ..Default::default()
61         }
62     }
63 
64     /// Updates this connection info with the peer buffer allocation and forwarded count from the
65     /// given event.
update_for_event(&mut self, event: &VsockEvent)66     pub fn update_for_event(&mut self, event: &VsockEvent) {
67         self.peer_buf_alloc = event.buffer_status.buffer_allocation;
68         self.peer_fwd_cnt = event.buffer_status.forward_count;
69 
70         if let VsockEventType::CreditUpdate = event.event_type {
71             self.has_pending_credit_request = false;
72         }
73     }
74 
75     /// Increases the forwarded count recorded for this connection by the given number of bytes.
76     ///
77     /// This should be called once received data has been passed to the client, so there is buffer
78     /// space available for more.
done_forwarding(&mut self, length: usize)79     pub fn done_forwarding(&mut self, length: usize) {
80         self.fwd_cnt += length as u32;
81     }
82 
83     /// Returns the number of bytes of RX buffer space the peer has available to receive packet body
84     /// data from us.
peer_free(&self) -> u3285     fn peer_free(&self) -> u32 {
86         self.peer_buf_alloc - (self.tx_cnt - self.peer_fwd_cnt)
87     }
88 
new_header(&self, src_cid: u64) -> VirtioVsockHdr89     fn new_header(&self, src_cid: u64) -> VirtioVsockHdr {
90         VirtioVsockHdr {
91             src_cid: src_cid.into(),
92             dst_cid: self.dst.cid.into(),
93             src_port: self.src_port.into(),
94             dst_port: self.dst.port.into(),
95             buf_alloc: self.buf_alloc.into(),
96             fwd_cnt: self.fwd_cnt.into(),
97             ..Default::default()
98         }
99     }
100 }
101 
102 /// An event received from a VirtIO socket device.
103 #[derive(Clone, Debug, Eq, PartialEq)]
104 pub struct VsockEvent {
105     /// The source of the event, i.e. the peer who sent it.
106     pub source: VsockAddr,
107     /// The destination of the event, i.e. the CID and port on our side.
108     pub destination: VsockAddr,
109     /// The peer's buffer status for the connection.
110     pub buffer_status: VsockBufferStatus,
111     /// The type of event.
112     pub event_type: VsockEventType,
113 }
114 
115 impl VsockEvent {
116     /// Returns whether the event matches the given connection.
matches_connection(&self, connection_info: &ConnectionInfo, guest_cid: u64) -> bool117     pub fn matches_connection(&self, connection_info: &ConnectionInfo, guest_cid: u64) -> bool {
118         self.source == connection_info.dst
119             && self.destination.cid == guest_cid
120             && self.destination.port == connection_info.src_port
121     }
122 
from_header(header: &VirtioVsockHdr) -> Result<Self>123     fn from_header(header: &VirtioVsockHdr) -> Result<Self> {
124         let op = header.op()?;
125         let buffer_status = VsockBufferStatus {
126             buffer_allocation: header.buf_alloc.into(),
127             forward_count: header.fwd_cnt.into(),
128         };
129         let source = header.source();
130         let destination = header.destination();
131 
132         let event_type = match op {
133             VirtioVsockOp::Request => {
134                 header.check_data_is_empty()?;
135                 VsockEventType::ConnectionRequest
136             }
137             VirtioVsockOp::Response => {
138                 header.check_data_is_empty()?;
139                 VsockEventType::Connected
140             }
141             VirtioVsockOp::CreditUpdate => {
142                 header.check_data_is_empty()?;
143                 VsockEventType::CreditUpdate
144             }
145             VirtioVsockOp::Rst | VirtioVsockOp::Shutdown => {
146                 header.check_data_is_empty()?;
147                 debug!("Disconnected from the peer");
148                 let reason = if op == VirtioVsockOp::Rst {
149                     DisconnectReason::Reset
150                 } else {
151                     DisconnectReason::Shutdown
152                 };
153                 VsockEventType::Disconnected { reason }
154             }
155             VirtioVsockOp::Rw => VsockEventType::Received {
156                 length: header.len() as usize,
157             },
158             VirtioVsockOp::CreditRequest => {
159                 header.check_data_is_empty()?;
160                 VsockEventType::CreditRequest
161             }
162             VirtioVsockOp::Invalid => return Err(SocketError::InvalidOperation.into()),
163         };
164 
165         Ok(VsockEvent {
166             source,
167             destination,
168             buffer_status,
169             event_type,
170         })
171     }
172 }
173 
174 #[derive(Clone, Debug, Eq, PartialEq)]
175 pub struct VsockBufferStatus {
176     pub buffer_allocation: u32,
177     pub forward_count: u32,
178 }
179 
180 /// The reason why a vsock connection was closed.
181 #[derive(Copy, Clone, Debug, Eq, PartialEq)]
182 pub enum DisconnectReason {
183     /// The peer has either closed the connection in response to our shutdown request, or forcibly
184     /// closed it of its own accord.
185     Reset,
186     /// The peer asked to shut down the connection.
187     Shutdown,
188 }
189 
190 /// Details of the type of an event received from a VirtIO socket.
191 #[derive(Clone, Debug, Eq, PartialEq)]
192 pub enum VsockEventType {
193     /// The peer requests to establish a connection with us.
194     ConnectionRequest,
195     /// The connection was successfully established.
196     Connected,
197     /// The connection was closed.
198     Disconnected {
199         /// The reason for the disconnection.
200         reason: DisconnectReason,
201     },
202     /// Data was received on the connection.
203     Received {
204         /// The length of the data in bytes.
205         length: usize,
206     },
207     /// The peer requests us to send a credit update.
208     CreditRequest,
209     /// The peer just sent us a credit update with nothing else.
210     CreditUpdate,
211 }
212 
213 /// Low-level driver for a VirtIO socket device.
214 ///
215 /// You probably want to use [`VsockConnectionManager`](super::VsockConnectionManager) rather than
216 /// using this directly.
217 ///
218 /// `RX_BUFFER_SIZE` is the size in bytes of each buffer used in the RX virtqueue. This must be
219 /// bigger than `size_of::<VirtioVsockHdr>()`.
220 pub struct VirtIOSocket<H: Hal, T: Transport, const RX_BUFFER_SIZE: usize = DEFAULT_RX_BUFFER_SIZE>
221 {
222     transport: T,
223     /// Virtqueue to receive packets.
224     rx: OwningQueue<H, QUEUE_SIZE, RX_BUFFER_SIZE>,
225     tx: VirtQueue<H, { QUEUE_SIZE }>,
226     /// Virtqueue to receive events from the device.
227     event: VirtQueue<H, { QUEUE_SIZE }>,
228     /// The guest_cid field contains the guest’s context ID, which uniquely identifies
229     /// the device for its lifetime. The upper 32 bits of the CID are reserved and zeroed.
230     guest_cid: u64,
231 }
232 
233 impl<H: Hal, T: Transport, const RX_BUFFER_SIZE: usize> Drop
234     for VirtIOSocket<H, T, RX_BUFFER_SIZE>
235 {
drop(&mut self)236     fn drop(&mut self) {
237         // Clear any pointers pointing to DMA regions, so the device doesn't try to access them
238         // after they have been freed.
239         self.transport.queue_unset(RX_QUEUE_IDX);
240         self.transport.queue_unset(TX_QUEUE_IDX);
241         self.transport.queue_unset(EVENT_QUEUE_IDX);
242     }
243 }
244 
245 impl<H: Hal, T: Transport, const RX_BUFFER_SIZE: usize> VirtIOSocket<H, T, RX_BUFFER_SIZE> {
246     /// Create a new VirtIO Vsock driver.
new(mut transport: T) -> Result<Self>247     pub fn new(mut transport: T) -> Result<Self> {
248         assert!(RX_BUFFER_SIZE > size_of::<VirtioVsockHdr>());
249 
250         let negotiated_features = transport.begin_init(SUPPORTED_FEATURES);
251 
252         let guest_cid = transport.read_consistent(|| {
253             Ok(
254                 read_config!(transport, VirtioVsockConfig, guest_cid_low)? as u64
255                     | (read_config!(transport, VirtioVsockConfig, guest_cid_high)? as u64) << 32,
256             )
257         })?;
258         debug!("guest cid: {guest_cid:?}");
259 
260         let rx = VirtQueue::new(
261             &mut transport,
262             RX_QUEUE_IDX,
263             negotiated_features.contains(Feature::RING_INDIRECT_DESC),
264             negotiated_features.contains(Feature::RING_EVENT_IDX),
265         )?;
266         let tx = VirtQueue::new(
267             &mut transport,
268             TX_QUEUE_IDX,
269             negotiated_features.contains(Feature::RING_INDIRECT_DESC),
270             negotiated_features.contains(Feature::RING_EVENT_IDX),
271         )?;
272         let event = VirtQueue::new(
273             &mut transport,
274             EVENT_QUEUE_IDX,
275             negotiated_features.contains(Feature::RING_INDIRECT_DESC),
276             negotiated_features.contains(Feature::RING_EVENT_IDX),
277         )?;
278 
279         let rx = OwningQueue::new(rx)?;
280 
281         transport.finish_init();
282         if rx.should_notify() {
283             transport.notify(RX_QUEUE_IDX);
284         }
285 
286         Ok(Self {
287             transport,
288             rx,
289             tx,
290             event,
291             guest_cid,
292         })
293     }
294 
295     /// Returns the CID which has been assigned to this guest.
guest_cid(&self) -> u64296     pub fn guest_cid(&self) -> u64 {
297         self.guest_cid
298     }
299 
300     /// Sends a request to connect to the given destination.
301     ///
302     /// This returns as soon as the request is sent; you should wait until `poll` returns a
303     /// `VsockEventType::Connected` event indicating that the peer has accepted the connection
304     /// before sending data.
connect(&mut self, connection_info: &ConnectionInfo) -> Result305     pub fn connect(&mut self, connection_info: &ConnectionInfo) -> Result {
306         let header = VirtioVsockHdr {
307             op: VirtioVsockOp::Request.into(),
308             ..connection_info.new_header(self.guest_cid)
309         };
310         // Sends a header only packet to the TX queue to connect the device to the listening socket
311         // at the given destination.
312         self.send_packet_to_tx_queue(&header, &[])
313     }
314 
315     /// Accepts the given connection from a peer.
accept(&mut self, connection_info: &ConnectionInfo) -> Result316     pub fn accept(&mut self, connection_info: &ConnectionInfo) -> Result {
317         <Self as VirtIOSocketManager>::accept(self, connection_info)
318     }
319 
320     /// Requests the peer to send us a credit update for the given connection.
request_credit(&mut self, connection_info: &ConnectionInfo) -> Result321     pub fn request_credit(&mut self, connection_info: &ConnectionInfo) -> Result {
322         <Self as VirtIOSocketManager>::request_credit(self, connection_info)
323     }
324 
325     /// Sends the buffer to the destination.
send(&mut self, buffer: &[u8], connection_info: &mut ConnectionInfo) -> Result326     pub fn send(&mut self, buffer: &[u8], connection_info: &mut ConnectionInfo) -> Result {
327         <Self as VirtIOSocketManager>::send(self, buffer, connection_info)
328     }
329 
330     /// Tells the peer how much buffer space we have to receive data.
credit_update(&mut self, connection_info: &ConnectionInfo) -> Result331     pub fn credit_update(&mut self, connection_info: &ConnectionInfo) -> Result {
332         <Self as VirtIOSocketManager>::credit_update(self, connection_info)
333     }
334 
335     /// Polls the RX virtqueue for the next event, and calls the given handler function to handle
336     /// it.
poll( &mut self, handler: impl FnOnce(VsockEvent, &[u8]) -> Result<Option<VsockEvent>>, ) -> Result<Option<VsockEvent>>337     pub fn poll(
338         &mut self,
339         handler: impl FnOnce(VsockEvent, &[u8]) -> Result<Option<VsockEvent>>,
340     ) -> Result<Option<VsockEvent>> {
341         <Self as VirtIOSocketManager>::poll(self, handler)
342     }
343 
344     /// Requests to shut down the connection cleanly, sending hints about whether we will send or
345     /// receive more data.
346     ///
347     /// This returns as soon as the request is sent; you should wait until `poll` returns a
348     /// `VsockEventType::Disconnected` event if you want to know that the peer has acknowledged the
349     /// shutdown.
shutdown_with_hints( &mut self, connection_info: &ConnectionInfo, hints: StreamShutdown, ) -> Result350     pub fn shutdown_with_hints(
351         &mut self,
352         connection_info: &ConnectionInfo,
353         hints: StreamShutdown,
354     ) -> Result {
355         <Self as VirtIOSocketManager>::shutdown_with_hints(self, connection_info, hints)
356     }
357 
358     /// Requests to shut down the connection cleanly, telling the peer that we won't send or receive
359     /// any more data.
360     ///
361     /// This returns as soon as the request is sent; you should wait until `poll` returns a
362     /// `VsockEventType::Disconnected` event if you want to know that the peer has acknowledged the
363     /// shutdown.
shutdown(&mut self, connection_info: &ConnectionInfo) -> Result364     pub fn shutdown(&mut self, connection_info: &ConnectionInfo) -> Result {
365         <Self as VirtIOSocketManager>::shutdown(self, connection_info)
366     }
367 
368     /// Forcibly closes the connection without waiting for the peer.
force_close(&mut self, connection_info: &ConnectionInfo) -> Result369     pub fn force_close(&mut self, connection_info: &ConnectionInfo) -> Result {
370         <Self as VirtIOSocketManager>::force_close(self, connection_info)
371     }
372 
send_packet_to_tx_queue(&mut self, header: &VirtioVsockHdr, buffer: &[u8]) -> Result373     fn send_packet_to_tx_queue(&mut self, header: &VirtioVsockHdr, buffer: &[u8]) -> Result {
374         let _len = if buffer.is_empty() {
375             self.tx
376                 .add_notify_wait_pop(&[header.as_bytes()], &mut [], &mut self.transport)?
377         } else {
378             self.tx.add_notify_wait_pop(
379                 &[header.as_bytes(), buffer],
380                 &mut [],
381                 &mut self.transport,
382             )?
383         };
384         Ok(())
385     }
386 }
387 
388 impl<H: Hal, T: Transport, const RX_BUFFER_SIZE: usize> VirtIOSocketManager
389     for VirtIOSocket<H, T, RX_BUFFER_SIZE>
390 {
local_cid(&self) -> u64391     fn local_cid(&self) -> u64 {
392         self.guest_cid()
393     }
send_packet_to_queue(&mut self, header: &VirtioVsockHdr, buffer: &[u8]) -> Result394     fn send_packet_to_queue(&mut self, header: &VirtioVsockHdr, buffer: &[u8]) -> Result {
395         self.send_packet_to_tx_queue(header, buffer)
396     }
poll( &mut self, handler: impl FnOnce(VsockEvent, &[u8]) -> Result<Option<VsockEvent>>, ) -> Result<Option<VsockEvent>>397     fn poll(
398         &mut self,
399         handler: impl FnOnce(VsockEvent, &[u8]) -> Result<Option<VsockEvent>>,
400     ) -> Result<Option<VsockEvent>> {
401         self.rx.poll(&mut self.transport, |buffer| {
402             let (header, body) = read_header_and_body(buffer)?;
403             VsockEvent::from_header(&header).and_then(|event| handler(event, body))
404         })
405     }
406 }
407 
408 /// A low-level interface for a vsock device implementation
409 pub struct VirtIOSocketDevice<H: DeviceHal, T: DeviceTransport> {
410     transport: T,
411     rx: DeviceVirtQueue<H, { QUEUE_SIZE }>,
412     tx: DeviceVirtQueue<H, { QUEUE_SIZE }>,
413     event: DeviceVirtQueue<H, { QUEUE_SIZE }>,
414 }
415 
416 impl<H: DeviceHal, T: DeviceTransport> VirtIOSocketDevice<H, T> {
417     /// Create a new VirtIO Vsock device.
new(mut transport: T) -> Result<Self>418     pub fn new(mut transport: T) -> Result<Self> {
419         let rx = DeviceVirtQueue::new(&mut transport, RX_QUEUE_IDX)?;
420         let tx = DeviceVirtQueue::new(&mut transport, TX_QUEUE_IDX)?;
421         let event = DeviceVirtQueue::new(&mut transport, EVENT_QUEUE_IDX)?;
422         Ok(Self {
423             transport,
424             rx,
425             tx,
426             event,
427         })
428     }
429 }
430 
431 impl<H: DeviceHal, T: DeviceTransport> VirtIOSocketManager for VirtIOSocketDevice<H, T> {
local_cid(&self) -> u64432     fn local_cid(&self) -> u64 {
433         VMADDR_CID_HOST
434     }
send_packet_to_queue(&mut self, header: &VirtioVsockHdr, buffer: &[u8]) -> Result435     fn send_packet_to_queue(&mut self, header: &VirtioVsockHdr, buffer: &[u8]) -> Result {
436         if buffer.is_empty() {
437             self.rx
438                 .wait_pop_add_notify(&[header.as_bytes()], &mut self.transport)?
439         } else {
440             self.rx
441                 .wait_pop_add_notify(&[header.as_bytes(), buffer], &mut self.transport)?
442         }
443         Ok(())
444     }
poll( &mut self, handler: impl FnOnce(VsockEvent, &[u8]) -> Result<Option<VsockEvent>>, ) -> Result<Option<VsockEvent>>445     fn poll(
446         &mut self,
447         handler: impl FnOnce(VsockEvent, &[u8]) -> Result<Option<VsockEvent>>,
448     ) -> Result<Option<VsockEvent>> {
449         self.tx.poll(&mut self.transport, |buffer| {
450             let (header, body) = read_header_and_body(buffer)?;
451             VsockEvent::from_header(&header).and_then(|event| handler(event, body))
452         })
453     }
454 }
455 pub trait VirtIOSocketManager {
local_cid(&self) -> u64456     fn local_cid(&self) -> u64;
send_packet_to_queue(&mut self, header: &VirtioVsockHdr, buffer: &[u8]) -> Result457     fn send_packet_to_queue(&mut self, header: &VirtioVsockHdr, buffer: &[u8]) -> Result;
poll( &mut self, handler: impl FnOnce(VsockEvent, &[u8]) -> Result<Option<VsockEvent>>, ) -> Result<Option<VsockEvent>>458     fn poll(
459         &mut self,
460         handler: impl FnOnce(VsockEvent, &[u8]) -> Result<Option<VsockEvent>>,
461     ) -> Result<Option<VsockEvent>>;
462 
463     /// Accepts the given connection from a peer.
accept(&mut self, connection_info: &ConnectionInfo) -> Result464     fn accept(&mut self, connection_info: &ConnectionInfo) -> Result {
465         let header = VirtioVsockHdr {
466             op: VirtioVsockOp::Response.into(),
467             ..connection_info.new_header(self.local_cid())
468         };
469         self.send_packet_to_queue(&header, &[])
470     }
471 
472     /// Requests the peer to send us a credit update for the given connection.
request_credit(&mut self, connection_info: &ConnectionInfo) -> Result473     fn request_credit(&mut self, connection_info: &ConnectionInfo) -> Result {
474         let header = VirtioVsockHdr {
475             op: VirtioVsockOp::CreditRequest.into(),
476             ..connection_info.new_header(self.local_cid())
477         };
478         self.send_packet_to_queue(&header, &[])
479     }
480 
481     /// Sends the buffer to the destination.
send(&mut self, buffer: &[u8], connection_info: &mut ConnectionInfo) -> Result482     fn send(&mut self, buffer: &[u8], connection_info: &mut ConnectionInfo) -> Result {
483         self.check_peer_buffer_is_sufficient(connection_info, buffer.len())?;
484 
485         let len = buffer.len() as u32;
486         let header = VirtioVsockHdr {
487             op: VirtioVsockOp::Rw.into(),
488             len: len.into(),
489             ..connection_info.new_header(self.local_cid())
490         };
491         connection_info.tx_cnt += len;
492         self.send_packet_to_queue(&header, buffer)
493     }
494 
check_peer_buffer_is_sufficient( &mut self, connection_info: &mut ConnectionInfo, buffer_len: usize, ) -> Result495     fn check_peer_buffer_is_sufficient(
496         &mut self,
497         connection_info: &mut ConnectionInfo,
498         buffer_len: usize,
499     ) -> Result {
500         if connection_info.peer_free() as usize >= buffer_len {
501             Ok(())
502         } else {
503             // Request an update of the cached peer credit, if we haven't already done so, and tell
504             // the caller to try again later.
505             if !connection_info.has_pending_credit_request {
506                 self.request_credit(connection_info)?;
507                 connection_info.has_pending_credit_request = true;
508             }
509             Err(SocketError::InsufficientBufferSpaceInPeer.into())
510         }
511     }
512 
513     /// Tells the peer how much buffer space we have to receive data.
credit_update(&mut self, connection_info: &ConnectionInfo) -> Result514     fn credit_update(&mut self, connection_info: &ConnectionInfo) -> Result {
515         let header = VirtioVsockHdr {
516             op: VirtioVsockOp::CreditUpdate.into(),
517             ..connection_info.new_header(self.local_cid())
518         };
519         self.send_packet_to_queue(&header, &[])
520     }
521 
522     /// Requests to shut down the connection cleanly, sending hints about whether we will send or
523     /// receive more data.
524     ///
525     /// This returns as soon as the request is sent; you should wait until `poll` returns a
526     /// `VsockEventType::Disconnected` event if you want to know that the peer has acknowledged the
527     /// shutdown.
shutdown_with_hints( &mut self, connection_info: &ConnectionInfo, hints: StreamShutdown, ) -> Result528     fn shutdown_with_hints(
529         &mut self,
530         connection_info: &ConnectionInfo,
531         hints: StreamShutdown,
532     ) -> Result {
533         let header = VirtioVsockHdr {
534             op: VirtioVsockOp::Shutdown.into(),
535             flags: hints.into(),
536             ..connection_info.new_header(self.local_cid())
537         };
538         self.send_packet_to_queue(&header, &[])
539     }
540 
541     /// Requests to shut down the connection cleanly, telling the peer that we won't send or receive
542     /// any more data.
543     ///
544     /// This returns as soon as the request is sent; you should wait until `poll` returns a
545     /// `VsockEventType::Disconnected` event if you want to know that the peer has acknowledged the
546     /// shutdown.
shutdown(&mut self, connection_info: &ConnectionInfo) -> Result547     fn shutdown(&mut self, connection_info: &ConnectionInfo) -> Result {
548         self.shutdown_with_hints(
549             connection_info,
550             StreamShutdown::SEND | StreamShutdown::RECEIVE,
551         )
552     }
553 
554     /// Forcibly closes the connection without waiting for the peer.
force_close(&mut self, connection_info: &ConnectionInfo) -> Result555     fn force_close(&mut self, connection_info: &ConnectionInfo) -> Result {
556         let header = VirtioVsockHdr {
557             op: VirtioVsockOp::Rst.into(),
558             ..connection_info.new_header(self.local_cid())
559         };
560         self.send_packet_to_queue(&header, &[])?;
561         Ok(())
562     }
563 }
564 
read_header_and_body(buffer: &[u8]) -> Result<(VirtioVsockHdr, &[u8])>565 fn read_header_and_body(buffer: &[u8]) -> Result<(VirtioVsockHdr, &[u8])> {
566     // This could fail if the device returns a buffer used length shorter than the header size.
567     let header = VirtioVsockHdr::read_from_prefix(buffer)
568         .map_err(|_| SocketError::BufferTooShort)?
569         .0;
570     let body_length = header.len() as usize;
571 
572     // This could fail if the device returns an unreasonably long body length.
573     let data_end = size_of::<VirtioVsockHdr>()
574         .checked_add(body_length)
575         .ok_or(SocketError::InvalidNumber)?;
576     // This could fail if the device returns a body length longer than buffer used length it
577     // returned.
578     let data = buffer
579         .get(size_of::<VirtioVsockHdr>()..data_end)
580         .ok_or(SocketError::BufferTooShort)?;
581     Ok((header, data))
582 }
583 
584 #[cfg(test)]
585 mod tests {
586     use super::*;
587     use crate::{
588         config::ReadOnly,
589         hal::fake::FakeHal,
590         transport::{
591             fake::{FakeTransport, QueueStatus, State},
592             DeviceType,
593         },
594     };
595     use alloc::{sync::Arc, vec};
596     use std::sync::Mutex;
597 
598     #[test]
config()599     fn config() {
600         let config_space = VirtioVsockConfig {
601             guest_cid_low: ReadOnly::new(66),
602             guest_cid_high: ReadOnly::new(0),
603         };
604         let state = Arc::new(Mutex::new(State::new(
605             vec![
606                 QueueStatus::default(),
607                 QueueStatus::default(),
608                 QueueStatus::default(),
609             ],
610             config_space,
611         )));
612         let transport = FakeTransport {
613             device_type: DeviceType::Socket,
614             max_queue_size: 32,
615             device_features: 0,
616             state: state.clone(),
617         };
618         let socket =
619             VirtIOSocket::<FakeHal, FakeTransport<VirtioVsockConfig>>::new(transport).unwrap();
620         assert_eq!(socket.guest_cid(), 0x00_0000_0042);
621     }
622 }
623