diff --git a/esp-hal/CHANGELOG.md b/esp-hal/CHANGELOG.md index 33dcbeb596a..d7d7c8845b6 100644 --- a/esp-hal/CHANGELOG.md +++ b/esp-hal/CHANGELOG.md @@ -28,6 +28,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - TWAI support for ESP32-H2 (#2199) - Make `DmaDescriptor` methods public (#2237) - Added a way to configure watchdogs in `esp_hal::init` (#2180) +- Introduce `DmaRxStreamBuf` (#2242) - Implement `embedded_hal_async::delay::DelayNs` for `TIMGx` timers (#2084) ### Changed @@ -52,6 +53,7 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - The `NO_PIN` constant has been removed. (#2133) - MSRV bump to 1.79 (#2156) - Allow handling interrupts while trying to lock critical section on multi-core chips. (#2197) +- Migrate `Camera` to a move based API (#2242). - Removed the PS-RAM related features, replaced by `quad-psram`/`octal-psram`, `init_psram` takes a configuration parameter, it's now possible to auto-detect PS-RAM size (#2178) - `EspTwaiFrame` constructors now accept any type that converts into `esp_hal::twai::Id` (#2207) - Change `DmaTxBuf` to support PSRAM on `esp32s3` (#2161) diff --git a/esp-hal/MIGRATING-0.20.md b/esp-hal/MIGRATING-0.20.md index ea03064eb02..c76477ee4ec 100644 --- a/esp-hal/MIGRATING-0.20.md +++ b/esp-hal/MIGRATING-0.20.md @@ -210,6 +210,49 @@ We've replaced some usage of features with [esp-config](https://docs.rs/esp-conf + ESP_HAL_PLACE_SPI_DRIVER_IN_RAM=true ``` +## `Camera` driver now uses `DmaRxBuffer` and moves the driver into the transfer object. + +For one shot transfers. +```diff +let (rx_buffer, rx_descriptors, _, _) = dma_buffers!(32678, 0); ++ let dma_buf = DmaRxBuf::new(rx_descriptors, rx_buffer).unwrap(); + +let lcd_cam = LcdCam::new(peripherals.LCD_CAM); +let mut camera = Camera::new( + lcd_cam.cam, + channel.rx, +- rx_descriptors, + data_pins, + 20u32.MHz(), +); + +- let transfer = camera.read_dma(rx_buffer).unwrap(); +- transfer.wait(); ++ let transfer = camera.receive(dma_buf).unwrap(); ++ let (_, camera, buf) = transfer.wait(); +``` + +For circular transfers. +```diff +- let (rx_buffer, rx_descriptors, _, _) = dma_buffers!(32678, 0); ++ let dma_buf = dma_rx_stream_buffer!(32678); + +let lcd_cam = LcdCam::new(peripherals.LCD_CAM); +let mut camera = Camera::new( + lcd_cam.cam, + channel.rx, +- rx_descriptors, + data_pins, + 20u32.MHz(), +); + +- let mut transfer = camera.read_dma_circular(rx_buffer).unwrap(); ++ let mut transfer = camera.receive(dma_buf).unwrap(); +transfer.pop(&mut [.....]); +transfer.pop(&mut [.....]); +transfer.pop(&mut [.....]); +``` + ## PS-RAM Initializing PS-RAM now takes a chip specific config and returns start of the mapped memory and the size. diff --git a/esp-hal/src/dma/mod.rs b/esp-hal/src/dma/mod.rs index 264615e7433..dcb894f92b1 100644 --- a/esp-hal/src/dma/mod.rs +++ b/esp-hal/src/dma/mod.rs @@ -57,7 +57,7 @@ use core::{ cmp::min, fmt::Debug, marker::PhantomData, - ptr::addr_of_mut, + ptr::{addr_of_mut, null_mut}, sync::atomic::compiler_fence, }; @@ -709,6 +709,35 @@ macro_rules! dma_tx_buffer { }}; } +/// Convenience macro to create a [DmaRxStreamBuf] from buffer size and +/// optional chunk size (uses max if unspecified). +/// The buffer and descriptors are statically allocated and +/// used to create the [DmaRxStreamBuf]. +/// +/// Smaller chunk sizes are recommended for lower latency. +/// +/// ## Usage +/// ```rust,no_run +#[doc = crate::before_snippet!()] +/// use esp_hal::dma_rx_stream_buffer; +/// +/// let buf = dma_rx_stream_buffer!(32000); +/// let buf = dma_rx_stream_buffer!(32000, chunk_size = 1000); +/// # } +/// ``` +#[macro_export] +macro_rules! dma_rx_stream_buffer { + ($rx_size:expr) => { + dma_rx_stream_buffer!($rx_size, chunk_size = 4095) + }; + ($rx_size:expr, chunk_size = $chunk_size:expr) => {{ + let (buffer, descriptors) = + $crate::dma_buffers_impl!($rx_size, $chunk_size, is_circular = false); + + $crate::dma::DmaRxStreamBuf::new(descriptors, buffer).unwrap() + }}; +} + /// DMA Errors #[derive(Debug, Clone, Copy, PartialEq)] #[cfg_attr(feature = "defmt", derive(defmt::Format))] @@ -1364,18 +1393,28 @@ pub trait DmaChannel: crate::private::Sealed { pub trait Rx: crate::private::Sealed { fn init(&mut self, burst_mode: bool, priority: DmaPriority); + /// In the future, this will be removed in favour of + /// [Self::prepare_and_start]; unsafe fn prepare_transfer_without_start( &mut self, peri: DmaPeripheral, chain: &DescriptorChain, ) -> Result<(), DmaError>; + /// In the future, this will be removed in favour of + /// [Self::prepare_and_start]; unsafe fn prepare_transfer( &mut self, peri: DmaPeripheral, buffer: &mut BUF, ) -> Result<(), DmaError>; + fn prepare_and_start( + &mut self, + peri: DmaPeripheral, + buffer: BUF, + ) -> Result; + fn start_transfer(&mut self) -> Result<(), DmaError>; fn stop_transfer(&mut self); @@ -1436,17 +1475,17 @@ where &mut self, first_desc: *mut DmaDescriptor, peri: DmaPeripheral, - ) -> Result<(), DmaError> { + ) { compiler_fence(core::sync::atomic::Ordering::SeqCst); R::clear_in_interrupts(); R::reset_in(); R::set_in_descriptors(first_desc as u32); R::set_in_peripheral(peri as u8); - - Ok(()) } + /// In the future, this will be removed in favour of [Self::start]. + /// The error checking done in here can be deferred. fn start_transfer(&mut self) -> Result<(), DmaError> { R::start_in(); @@ -1461,6 +1500,10 @@ where R::stop_in(); } + fn start(&mut self) { + R::start_in(); + } + fn waker() -> &'static embassy_sync::waitqueue::AtomicWaker; } @@ -1532,7 +1575,8 @@ where } self.rx_impl - .prepare_transfer_without_start(chain.first() as _, peri) + .prepare_transfer_without_start(chain.first() as _, peri); + Ok(()) } unsafe fn prepare_transfer( @@ -1540,7 +1584,7 @@ where peri: DmaPeripheral, buffer: &mut BUF, ) -> Result<(), DmaError> { - let preparation = buffer.prepare(); + let preparation = buffer.prepare_mut(); // TODO: Get burst mode from DmaBuf. if self.burst_mode { @@ -1548,7 +1592,32 @@ where } self.rx_impl - .prepare_transfer_without_start(preparation.start, peri) + .prepare_transfer_without_start(preparation.start, peri); + + Ok(()) + } + + fn prepare_and_start( + &mut self, + peri: DmaPeripheral, + buffer: BUF, + ) -> Result { + // TODO: Get burst mode from DmaBuf. + if self.burst_mode { + return Err((DmaError::InvalidAlignment, buffer)); + } + + let (preparation, view) = buffer.prepare(); + + // SAFETY: DmaRxBuffer ensures user has kept to lifetime and validity guarantees. + unsafe { + self.rx_impl + .prepare_transfer_without_start(preparation.start, peri); + } + + self.rx_impl.start(); + + Ok(view) } fn start_transfer(&mut self) -> Result<(), DmaError> { @@ -1777,7 +1846,7 @@ where peri: DmaPeripheral, buffer: &mut BUF, ) -> Result<(), DmaError> { - let preparation = buffer.prepare(); + let preparation = buffer.prepare_mut(); cfg_if::cfg_if!( if #[cfg(esp32s3)] { if let Some(block_size) = preparation.block_size { @@ -1963,11 +2032,39 @@ pub struct Preparation { /// [DmaTxBuffer] is a DMA descriptor + memory combo that can be used for /// transmitting data from a DMA channel to a peripheral's FIFO. pub trait DmaTxBuffer { + /// A type providing operations that are safe to perform on the buffer + /// whilst the DMA is actively using it. + type View: DmaBufferView; + /// Prepares the buffer for an imminent transfer and returns /// information required to use this buffer. /// /// Note: This operation is idempotent. - fn prepare(&mut self) -> Preparation; + /// + /// In the future this will be deprecated in favour of [Self::prepare]. + fn prepare_mut(&mut self) -> Preparation; + + /// This method allows users to let the buffer implementation know that + /// [Self::prepare] will likely be called next. + /// + /// If [Self::prepare] is expensive, implementations are strongly encouraged + /// to do the expensive part in this function where possible. + /// + /// Users are encouraged to call this method if there are spare CPU cycles + /// available. This is typically whilst waiting for a previous transfer + /// to complete. + /// + /// This method may never be called so do not rely on it for correct + /// function. + /// + /// Note: This operation is idempotent. + fn prepare_hint(&mut self) { + // This method is optional. + } + + /// Prepares the buffer for an imminent transfer and returns + /// information required to use this buffer. + fn prepare(self) -> (Preparation, Self::View); /// Returns the maximum number of bytes that would be transmitted by this /// buffer. @@ -1984,11 +2081,37 @@ pub trait DmaTxBuffer { /// which resides in the last descriptor. There will be a separate trait in /// future to support multiple EOFs. pub trait DmaRxBuffer { + /// A type providing operations that are safe to perform on the buffer + /// whilst the DMA is actively using it. + type View: DmaBufferView; + /// Prepares the buffer for an imminent transfer and returns /// information required to use this buffer. /// /// Note: This operation is idempotent. - fn prepare(&mut self) -> Preparation; + fn prepare_mut(&mut self) -> Preparation; + + /// This method allows users to let the buffer implementation know that + /// [Self::prepare] will likely be called next. + /// + /// If [Self::prepare] is expensive, implementations are strongly encouraged + /// to do the expensive part in this function where possible. + /// + /// Users are encouraged to call this method if there are spare CPU cycles + /// available. This is typically whilst waiting for a previous transfer + /// to complete. + /// + /// This method may never be called so do not rely on it for correct + /// function. + /// + /// Note: This operation is idempotent. + fn prepare_hint(&mut self) { + // This method is optional. + } + + /// Prepares the buffer for an imminent transfer and returns + /// information required to use this buffer. + fn prepare(self) -> (Preparation, Self::View); /// Returns the maximum number of bytes that can be received by this buffer. /// @@ -1997,6 +2120,22 @@ pub trait DmaRxBuffer { fn length(&self) -> usize; } +/// This trait represents the subset of operations allowed to be performed on a +/// [DmaTxBuffer]/[DmaRxBuffer] whilst it is being used by the DMA. +pub trait DmaBufferView { + /// The buffer type that made this view. + type DmaBuffer; + + /// This is called after the DMA is done using the buffer. + fn release(self) -> Self::DmaBuffer; +} + +/// An in-progress view into [DmaRxBuf]/[DmaTxBuf]. +/// +/// In the future, this could support peeking into state of the +/// descriptors/buffers. +pub struct BufView(T); + /// Error returned from Dma[Rx|Tx|RxTx]Buf operations. #[derive(Debug)] #[cfg_attr(feature = "defmt", derive(defmt::Format))] @@ -2225,7 +2364,9 @@ impl DmaTxBuf { } impl DmaTxBuffer for DmaTxBuf { - fn prepare(&mut self) -> Preparation { + type View = BufView; + + fn prepare_mut(&mut self) -> Preparation { for desc in self.descriptors.iter_mut() { // Give ownership to the DMA desc.set_owner(Owner::Dma); @@ -2251,11 +2392,24 @@ impl DmaTxBuffer for DmaTxBuf { } } + fn prepare(mut self) -> (Preparation, BufView) { + let preparation = self.prepare_mut(); + (preparation, BufView(self)) + } + fn length(&self) -> usize { self.len() } } +impl DmaBufferView for BufView { + type DmaBuffer = DmaTxBuf; + + fn release(self) -> Self::DmaBuffer { + self.0 + } +} + /// DMA receive buffer /// /// This is a contiguous buffer linked together by DMA descriptors of length @@ -2464,7 +2618,9 @@ impl DmaRxBuf { } impl DmaRxBuffer for DmaRxBuf { - fn prepare(&mut self) -> Preparation { + type View = BufView; + + fn prepare_mut(&mut self) -> Preparation { for desc in self.descriptors.iter_mut() { // Give ownership to the DMA desc.set_owner(Owner::Dma); @@ -2488,11 +2644,24 @@ impl DmaRxBuffer for DmaRxBuf { } } + fn prepare(mut self) -> (Preparation, BufView) { + let preparation = self.prepare_mut(); + (preparation, BufView(self)) + } + fn length(&self) -> usize { self.len() } } +impl DmaBufferView for BufView { + type DmaBuffer = DmaRxBuf; + + fn release(self) -> Self::DmaBuffer { + self.0 + } +} + /// DMA transmit and receive buffer. /// /// This is a (single) contiguous buffer linked together by two sets of DMA @@ -2664,8 +2833,16 @@ impl DmaRxTxBuf { } } +/// A transmitting view into a [DmaRxTxBuf]. +pub struct DmaRxTxBufTxView(DmaRxTxBuf); + +/// A receiving view into a [DmaRxTxBuf]. +pub struct DmaRxTxBufRxView(DmaRxTxBuf); + impl DmaTxBuffer for DmaRxTxBuf { - fn prepare(&mut self) -> Preparation { + type View = DmaRxTxBufTxView; + + fn prepare_mut(&mut self) -> Preparation { for desc in self.tx_descriptors.iter_mut() { // Give ownership to the DMA desc.set_owner(Owner::Dma); @@ -2681,13 +2858,20 @@ impl DmaTxBuffer for DmaRxTxBuf { } } + fn prepare(mut self) -> (Preparation, DmaRxTxBufTxView) { + let preparation = DmaTxBuffer::prepare_mut(&mut self); + (preparation, DmaRxTxBufTxView(self)) + } + fn length(&self) -> usize { self.len() } } impl DmaRxBuffer for DmaRxTxBuf { - fn prepare(&mut self) -> Preparation { + type View = DmaRxTxBufRxView; + + fn prepare_mut(&mut self) -> Preparation { for desc in self.rx_descriptors.iter_mut() { // Give ownership to the DMA desc.set_owner(Owner::Dma); @@ -2711,11 +2895,360 @@ impl DmaRxBuffer for DmaRxTxBuf { } } + fn prepare(mut self) -> (Preparation, DmaRxTxBufRxView) { + let preparation = DmaRxBuffer::prepare_mut(&mut self); + (preparation, DmaRxTxBufRxView(self)) + } + fn length(&self) -> usize { self.len() } } +impl DmaBufferView for DmaRxTxBufRxView { + type DmaBuffer = DmaRxTxBuf; + + fn release(self) -> Self::DmaBuffer { + self.0 + } +} + +impl DmaBufferView for DmaRxTxBufTxView { + type DmaBuffer = DmaRxTxBuf; + + fn release(self) -> Self::DmaBuffer { + self.0 + } +} + +/// DMA Streaming Receive Buffer. +/// +/// This is a contiguous buffer linked together by DMA descriptors, and the +/// buffer is evenly distributed between each descriptor provided. +/// +/// It is used for continuously streaming data from a peripheral's FIFO. +/// +/// It does so by maintaining sliding window of descriptors that progresses when +/// you call [DmaRxStreamBufView::consume]. +/// +/// The list starts out like so `A (empty) -> B (empty) -> C (empty) -> D +/// (empty) -> NULL`. +/// +/// As the DMA writes to the buffers the list progresses like so: +/// - `A (empty) -> B (empty) -> C (empty) -> D (empty) -> NULL` +/// - `A (full) -> B (empty) -> C (empty) -> D (empty) -> NULL` +/// - `A (full) -> B (full) -> C (empty) -> D (empty) -> NULL` +/// - `A (full) -> B (full) -> C (full) -> D (empty) -> NULL` +/// +/// As you call [DmaRxStreamBufView::consume] the list (approximately) +/// progresses like so: +/// - `A (full) -> B (full) -> C (full) -> D (empty) -> NULL` +/// - `B (full) -> C (full) -> D (empty) -> A (empty) -> NULL` +/// - `C (full) -> D (empty) -> A (empty) -> B (empty) -> NULL` +/// - `D (empty) -> A (empty) -> B (empty) -> C (empty) -> NULL` +/// +/// If all the descriptors fill up, the [DmaRxInterrupt::DescriptorEmpty] +/// interrupt will fire and the DMA will stop writing, at which point it is up +/// to you to resume/restart the transfer. +/// +/// Note: This buffer will not tell you when this condition occurs, you should +/// check with the driver to see if the DMA has stopped. +/// +/// When constructing this buffer, it is important to tune the ratio between the +/// chunk size and buffer size appropriately. Smaller chunk sizes means you +/// receive data more frequently but this means the DMA interrupts +/// ([DmaRxInterrupt::Done]) also fire more frequently (if you use them). +/// +/// See [DmaRxStreamBufView] for APIs available whilst a transfer is in +/// progress. +pub struct DmaRxStreamBuf { + descriptors: &'static mut [DmaDescriptor], + buffer: &'static mut [u8], +} + +impl DmaRxStreamBuf { + /// Creates a new [DmaRxStreamBuf] evenly distributing the buffer between + /// the provided descriptors. + pub fn new( + descriptors: &'static mut [DmaDescriptor], + buffer: &'static mut [u8], + ) -> Result { + if !is_slice_in_dram(descriptors) { + return Err(DmaBufError::UnsupportedMemoryRegion); + } + if !is_slice_in_dram(buffer) { + return Err(DmaBufError::UnsupportedMemoryRegion); + } + + if descriptors.is_empty() { + return Err(DmaBufError::InsufficientDescriptors); + } + + // Evenly distribute the buffer between the descriptors. + let chunk_size = buffer.len() / descriptors.len(); + + if chunk_size > 4095 { + return Err(DmaBufError::InsufficientDescriptors); + } + + // Check that the last descriptor can hold the excess + let excess = buffer.len() % descriptors.len(); + if chunk_size + excess > 4095 { + return Err(DmaBufError::InsufficientDescriptors); + } + + // Link up all the descriptors (but not in a circle). + let mut next = null_mut(); + for desc in descriptors.iter_mut().rev() { + desc.next = next; + next = desc; + } + + let mut chunks = buffer.chunks_exact_mut(chunk_size); + for (desc, chunk) in descriptors.iter_mut().zip(chunks.by_ref()) { + desc.buffer = chunk.as_mut_ptr(); + desc.set_size(chunk.len()); + } + + let remainder = chunks.into_remainder(); + debug_assert_eq!(remainder.len(), excess); + + if !remainder.is_empty() { + // Append any excess to the last descriptor. + let last_descriptor = descriptors.last_mut().unwrap(); + last_descriptor.set_size(last_descriptor.size() + remainder.len()); + } + + Ok(Self { + descriptors, + buffer, + }) + } + + /// Consume the buf, returning the descriptors and buffer. + pub fn split(self) -> (&'static mut [DmaDescriptor], &'static mut [u8]) { + (self.descriptors, self.buffer) + } +} + +impl DmaRxBuffer for DmaRxStreamBuf { + type View = DmaRxStreamBufView; + + fn prepare_mut(&mut self) -> Preparation { + // It does not make sense to use this type without viewing it. Prefer DmaRxBuf + // for one shot transfers. + + unimplemented!() + } + + fn prepare(self) -> (Preparation, DmaRxStreamBufView) { + for desc in self.descriptors.iter_mut() { + // Give ownership of the buffer to the DMA hardware. + desc.set_owner(Owner::Dma); + // Clear the length so we know when the DMA sets it. + desc.set_length(0); + // Clear the EOF bit for the same reason. + desc.set_suc_eof(false); + } + + let preparation = Preparation { + start: self.descriptors.as_mut_ptr(), + block_size: None, + }; + ( + preparation, + DmaRxStreamBufView { + buf: self, + descriptor_idx: 0, + descriptor_offset: 0, + }, + ) + } + + fn length(&self) -> usize { + panic!("DmaCircularBuf doesn't have a length") + } +} + +/// A [DmaBufferView] into a [DmaRxStreamBuf] +pub struct DmaRxStreamBufView { + buf: DmaRxStreamBuf, + descriptor_idx: usize, + descriptor_offset: usize, +} + +impl DmaRxStreamBufView { + /// Returns the number of bytes that are available to read from the buf. + pub fn available_bytes(&self) -> usize { + let (tail, head) = self.buf.descriptors.split_at(self.descriptor_idx); + let mut result = 0; + for desc in head.iter().chain(tail) { + if desc.owner() == Owner::Dma { + break; + } + result += desc.len(); + } + result - self.descriptor_offset + } + + /// Reads as much as possible into the buf from the available data. + pub fn pop(&mut self, buf: &mut [u8]) -> usize { + if buf.is_empty() { + return 0; + } + let total_bytes = buf.len(); + + let mut remaining = buf; + loop { + let available = self.peek(); + if available.len() >= remaining.len() { + remaining.copy_from_slice(&available[0..remaining.len()]); + self.consume(remaining.len()); + let consumed = remaining.len(); + remaining = &mut remaining[consumed..]; + break; + } else { + let to_consume = available.len(); + remaining[0..to_consume].copy_from_slice(available); + self.consume(to_consume); + remaining = &mut remaining[to_consume..]; + } + } + + total_bytes - remaining.len() + } + + /// Returns a slice into the buffer containing available data. + /// This will be the longest possible contiguous slice into the buffer that + /// contains data that is available to read. + /// + /// Note: This function ignores EOFs, see [Self::peek_until_eof] if you need + /// EOF support. + pub fn peek(&self) -> &[u8] { + let (slice, _) = self.peek_internal(false); + slice + } + + /// Same as [Self::peek] but will not skip over any EOFs. + /// + /// It also returns a boolean indicating whether this slice ends with an EOF + /// or not. + pub fn peek_until_eof(&self) -> (&[u8], bool) { + self.peek_internal(true) + } + + /// Consumes the first `n` bytes from the available data, returning any + /// fully consumed descriptors back to the DMA. + /// This is typically called after [Self::peek]/[Self::peek_until_eof]. + /// + /// Returns the number of bytes that were actually consumed. + pub fn consume(&mut self, n: usize) -> usize { + let mut remaining_bytes_to_consume = n; + + loop { + let desc = &mut self.buf.descriptors[self.descriptor_idx]; + + if desc.owner() == Owner::Dma { + // Descriptor is still owned by DMA so it can't be read yet. + // This should only happen when there is no more data available to read. + break; + } + + let remaining_bytes_in_descriptor = desc.len() - self.descriptor_offset; + if remaining_bytes_to_consume < remaining_bytes_in_descriptor { + self.descriptor_offset += remaining_bytes_to_consume; + remaining_bytes_to_consume = 0; + break; + } + + // Reset the descriptor for reuse. + desc.set_owner(Owner::Dma); + desc.set_suc_eof(false); + desc.set_length(0); + + // Before connecting this descriptor to the end of the list, the next descriptor + // must be disconnected from this one to prevent the DMA from + // overtaking. + desc.next = null_mut(); + + let desc_ptr: *mut _ = desc; + + let prev_descriptor_index = self + .descriptor_idx + .checked_sub(1) + .unwrap_or(self.buf.descriptors.len() - 1); + + // Connect this consumed descriptor to the end of the chain. + self.buf.descriptors[prev_descriptor_index].next = desc_ptr; + + self.descriptor_idx += 1; + if self.descriptor_idx >= self.buf.descriptors.len() { + self.descriptor_idx = 0; + } + self.descriptor_offset = 0; + + remaining_bytes_to_consume -= remaining_bytes_in_descriptor; + } + + n - remaining_bytes_to_consume + } + + fn peek_internal(&self, stop_at_eof: bool) -> (&[u8], bool) { + let descriptors = &self.buf.descriptors[self.descriptor_idx..]; + + // There must be at least one descriptor. + debug_assert!(!descriptors.is_empty()); + + if descriptors.len() == 1 { + let last_descriptor = &descriptors[0]; + if last_descriptor.owner() == Owner::Dma { + // No data available. + (&[], false) + } else { + let length = last_descriptor.len() - self.descriptor_offset; + ( + &self.buf.buffer[self.buf.buffer.len() - length..], + last_descriptor.flags.suc_eof(), + ) + } + } else { + let chunk_size = descriptors[0].size(); + let mut found_eof = false; + + let mut number_of_contiguous_bytes = 0; + for desc in descriptors { + if desc.owner() == Owner::Dma { + break; + } + number_of_contiguous_bytes += desc.len(); + + if stop_at_eof && desc.flags.suc_eof() { + found_eof = true; + break; + } + // If the length is smaller than the size, the contiguous-ness ends here. + if desc.len() < desc.size() { + break; + } + } + + ( + &self.buf.buffer[chunk_size * self.descriptor_idx..][..number_of_contiguous_bytes] + [self.descriptor_offset..], + found_eof, + ) + } + } +} + +impl DmaBufferView for DmaRxStreamBufView { + type DmaBuffer = DmaRxStreamBuf; + + fn release(self) -> Self::DmaBuffer { + self.buf + } +} + pub(crate) mod dma_private { use super::*; diff --git a/esp-hal/src/lcd_cam/cam.rs b/esp-hal/src/lcd_cam/cam.rs index 004da522419..038107a0ad0 100644 --- a/esp-hal/src/lcd_cam/cam.rs +++ b/esp-hal/src/lcd_cam/cam.rs @@ -19,14 +19,14 @@ //! # use esp_hal::gpio::Io; //! # use esp_hal::lcd_cam::{cam::{Camera, RxEightBits}, LcdCam}; //! # use fugit::RateExtU32; -//! # use esp_hal::dma_buffers; +//! # use esp_hal::dma_rx_stream_buffer; //! # use esp_hal::dma::{Dma, DmaPriority}; //! # let io = Io::new(peripherals.GPIO, peripherals.IO_MUX); //! //! # let dma = Dma::new(peripherals.DMA); //! # let channel = dma.channel0; //! -//! # let (_, rx_descriptors, tx_buffer, tx_descriptors) = dma_buffers!(32678, 0); +//! # let dma_buf = dma_rx_stream_buffer!(20 * 1000, chunk_size = 1000); //! //! # let channel = channel.configure( //! # false, @@ -52,7 +52,6 @@ //! let mut camera = Camera::new( //! lcd_cam.cam, //! channel.rx, -//! rx_descriptors, //! data_pins, //! 20u32.MHz(), //! ) @@ -60,26 +59,27 @@ //! .with_master_clock(mclk_pin) //! .with_pixel_clock(pclk_pin) //! .with_ctrl_pins(vsync_pin, href_pin); +//! +//! let transfer = camera.receive(dma_buf).map_err(|e| e.0).unwrap(); +//! //! # } //! ``` +use core::ops::{Deref, DerefMut}; + use fugit::HertzU32; use crate::{ clock::Clocks, dma::{ - dma_private::{DmaSupport, DmaSupportRx}, ChannelRx, - DescriptorChain, + DmaBufferView, DmaChannel, - DmaDescriptor, DmaError, DmaPeripheral, - DmaTransferRx, - DmaTransferRxCircular, + DmaRxBuffer, LcdCamPeripheral, Rx, - WriteBuffer, }, gpio::{InputPin, InputSignal, OutputPin, OutputSignal, Pull}, lcd_cam::{cam::private::RxPins, private::calculate_clkm, BitOrder, ByteOrder}, @@ -129,9 +129,6 @@ pub struct Cam<'d> { pub struct Camera<'d, CH: DmaChannel> { lcd_cam: PeripheralRef<'d, LCD_CAM>, rx_channel: ChannelRx<'d, CH>, - rx_chain: DescriptorChain, - // 1 or 2 - bus_width: usize, } impl<'d, CH: DmaChannel> Camera<'d, CH> @@ -142,7 +139,6 @@ where pub fn new( cam: Cam<'d>, channel: ChannelRx<'d, CH>, - descriptors: &'static mut [DmaDescriptor], _pins: P, frequency: HertzU32, ) -> Self { @@ -176,7 +172,7 @@ where w.cam_rec_data_bytelen().bits(0); w.cam_line_int_num().bits(0); w.cam_vsync_filter_en().clear_bit(); - w.cam_2byte_en().clear_bit(); + w.cam_2byte_en().bit(P::BUS_WIDTH == 2); w.cam_clk_inv().clear_bit(); w.cam_de_inv().clear_bit(); w.cam_hsync_inv().clear_bit(); @@ -192,49 +188,10 @@ where Self { lcd_cam, rx_channel: channel, - rx_chain: DescriptorChain::new(descriptors), - bus_width: P::BUS_WIDTH, } } } -impl<'d, CH: DmaChannel> DmaSupport for Camera<'d, CH> { - fn peripheral_wait_dma(&mut self, _is_rx: bool, _is_tx: bool) { - loop { - // Wait for IN_SUC_EOF (i.e. VSYNC) - if self.rx_channel.is_done() { - break; - } - - // Or for IN_DSCR_EMPTY (i.e. No more buffer space) - if self.rx_channel.has_dscr_empty_error() { - break; - } - - // Or for IN_DSCR_ERR (i.e. bad descriptor) - if self.rx_channel.has_error() { - break; - } - } - } - - fn peripheral_dma_stop(&mut self) { - // TODO: Stop DMA?? self.instance.rx_channel.stop_transfer(); - } -} - -impl<'d, CH: DmaChannel> DmaSupportRx for Camera<'d, CH> { - type RX = ChannelRx<'d, CH>; - - fn rx(&mut self) -> &mut Self::RX { - &mut self.rx_channel - } - - fn chain(&mut self) -> &mut DescriptorChain { - &mut self.rx_chain - } -} - impl<'d, CH: DmaChannel> Camera<'d, CH> { /// Configures the byte order for the camera data. pub fn set_byte_order(&mut self, byte_order: ByteOrder) -> &mut Self { @@ -346,8 +303,12 @@ impl<'d, CH: DmaChannel> Camera<'d, CH> { self } - // Reset Camera control unit and Async Rx FIFO - fn reset_unit_and_fifo(&self) { + /// Starts a DMA transfer to receive data from the camera peripheral. + pub fn receive( + mut self, + buf: BUF, + ) -> Result, (DmaError, Self, BUF)> { + // Reset Camera control unit and Async Rx FIFO self.lcd_cam .cam_ctrl1() .modify(|_, w| w.cam_reset().set_bit()); @@ -360,60 +321,137 @@ impl<'d, CH: DmaChannel> Camera<'d, CH> { self.lcd_cam .cam_ctrl1() .modify(|_, w| w.cam_afifo_reset().clear_bit()); - } - // Start the Camera unit to listen for incoming DVP stream. - fn start_unit(&self) { - self.lcd_cam - .cam_ctrl() - .modify(|_, w| w.cam_update().set_bit()); + // Start DMA to receive incoming transfer. + let view = match self + .rx_channel + .prepare_and_start(DmaPeripheral::LcdCam, buf) + { + Ok(view) => view, + Err((err, buf)) => return Err((err, self, buf)), + }; + + // Start the Camera unit to listen for incoming DVP stream. + self.lcd_cam.cam_ctrl().modify(|_, w| { + // Automatically stops the camera unit once the GDMA Rx FIFO is full. + w.cam_stop_en().set_bit(); + + w.cam_update().set_bit() + }); self.lcd_cam .cam_ctrl1() .modify(|_, w| w.cam_start().set_bit()); + + Ok(CameraTransfer { + camera: self, + buffer_view: view, + }) + } +} + +/// Represents an ongoing (or potentially stopped) transfer from the Camera to a +/// DMA buffer. +pub struct CameraTransfer<'d, CH: DmaChannel, BUF: DmaRxBuffer> { + camera: Camera<'d, CH>, + buffer_view: BUF::View, +} + +impl<'d, CH: DmaChannel, BUF: DmaRxBuffer> CameraTransfer<'d, CH, BUF> { + /// Returns true when [Self::wait] will not block. + pub fn is_done(&self) -> bool { + // This peripheral doesn't really "complete". As long the camera (or anything + // pretending to be :D) sends data, it will receive it and pass it to the DMA. + // This implementation of is_done is an opinionated one. When the transfer is + // started, the CAM_STOP_EN bit is set, which tells the LCD_CAM to stop + // itself when the DMA stops emptying its async RX FIFO. This will + // typically be because the DMA ran out descriptors but there could be other + // reasons as well. + + // In the future, a user of esp_hal may not want this behaviour, which would be + // a reasonable ask. At which point is_done and wait would go away, and + // the driver will stop pretending that this peripheral has some kind of + // finish line. + + // For now, most people probably want this behaviour, so it shall be kept for + // the sake of familiarity and similarity with other drivers. + + self.camera + .lcd_cam + .cam_ctrl1() + .read() + .cam_start() + .bit_is_clear() } - fn start_dma( - &mut self, - circular: bool, - buf: &mut RXBUF, - ) -> Result<(), DmaError> { - let (ptr, len) = unsafe { buf.write_buffer() }; + /// Stops this transfer on the spot and returns the peripheral and buffer. + pub fn stop(mut self) -> (Camera<'d, CH>, BUF) { + self.stop_peripherals(); + let (camera, view) = self.release(); + (camera, view.release()) + } + + /// Waits for the transfer to stop and returns the peripheral and buffer. + /// + /// Note: The camera doesn't really "finish" its transfer, so what you're + /// really waiting for here is a DMA Error. You typically just want to + /// call [Self::stop] once you have the data you need. + pub fn wait(mut self) -> (Result<(), DmaError>, Camera<'d, CH>, BUF) { + while !self.is_done() {} + + // Stop the DMA as it doesn't know that the camera has stopped. + self.camera.rx_channel.stop_transfer(); + + // Note: There is no "done" interrupt to clear. + + let (camera, view) = self.release(); + + let result = if camera.rx_channel.has_error() { + Err(DmaError::DescriptorError) + } else { + Ok(()) + }; - assert!(len % self.bus_width == 0); + (result, camera, view.release()) + } + fn release(self) -> (Camera<'d, CH>, BUF::View) { unsafe { - self.rx_chain.fill_for_rx(circular, ptr as _, len)?; - self.rx_channel - .prepare_transfer_without_start(DmaPeripheral::LcdCam, &self.rx_chain)?; + let camera = core::ptr::read(&self.camera); + let view = core::ptr::read(&self.buffer_view); + core::mem::forget(self); + (camera, view) } - self.rx_channel.start_transfer() } - /// Starts a DMA transfer to receive data from the camera peripheral. - pub fn read_dma<'t, RXBUF: WriteBuffer>( - &'t mut self, - buf: &'t mut RXBUF, - ) -> Result, DmaError> { - self.reset_unit_and_fifo(); - // Start DMA to receive incoming transfer. - self.start_dma(false, buf)?; - self.start_unit(); + fn stop_peripherals(&mut self) { + // Stop the LCD_CAM peripheral. + self.camera + .lcd_cam + .cam_ctrl1() + .modify(|_, w| w.cam_start().clear_bit()); - Ok(DmaTransferRx::new(self)) + // Stop the DMA + self.camera.rx_channel.stop_transfer(); } +} - /// Starts a circular DMA transfer to receive data from the camera - /// peripheral. - pub fn read_dma_circular<'t, RXBUF: WriteBuffer>( - &'t mut self, - buf: &'t mut RXBUF, - ) -> Result, DmaError> { - self.reset_unit_and_fifo(); - // Start DMA to receive incoming transfer. - self.start_dma(true, buf)?; - self.start_unit(); +impl<'d, CH: DmaChannel, BUF: DmaRxBuffer> Deref for CameraTransfer<'d, CH, BUF> { + type Target = BUF::View; + + fn deref(&self) -> &Self::Target { + &self.buffer_view + } +} + +impl<'d, CH: DmaChannel, BUF: DmaRxBuffer> DerefMut for CameraTransfer<'d, CH, BUF> { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.buffer_view + } +} - Ok(DmaTransferRxCircular::new(self)) +impl<'d, CH: DmaChannel, BUF: DmaRxBuffer> Drop for CameraTransfer<'d, CH, BUF> { + fn drop(&mut self) { + self.stop_peripherals(); } } diff --git a/examples/src/bin/lcd_cam_ov2640.rs b/examples/src/bin/lcd_cam_ov2640.rs index 8afa95eef01..5aea427fd3f 100644 --- a/examples/src/bin/lcd_cam_ov2640.rs +++ b/examples/src/bin/lcd_cam_ov2640.rs @@ -27,7 +27,7 @@ use esp_backtrace as _; use esp_hal::{ delay::Delay, dma::{Dma, DmaPriority}, - dma_buffers, + dma_rx_stream_buffer, gpio::Io, i2c, i2c::I2C, @@ -38,7 +38,7 @@ use esp_hal::{ prelude::*, Blocking, }; -use esp_println::println; +use esp_println::{print, println}; #[entry] fn main() -> ! { @@ -49,7 +49,7 @@ fn main() -> ! { let dma = Dma::new(peripherals.DMA); let channel = dma.channel0; - let (rx_buffer, rx_descriptors, _, _) = dma_buffers!(32678, 0); + let dma_rx_buf = dma_rx_stream_buffer!(20 * 1000, chunk_size = 1000); let channel = channel.configure(false, DmaPriority::Priority0); @@ -71,22 +71,13 @@ fn main() -> ! { ); let lcd_cam = LcdCam::new(peripherals.LCD_CAM); - let mut camera = Camera::new( - lcd_cam.cam, - channel.rx, - rx_descriptors, - cam_data_pins, - 20u32.MHz(), - ) - .with_master_clock(cam_xclk) - .with_pixel_clock(cam_pclk) - .with_ctrl_pins(cam_vsync, cam_href); + let camera = Camera::new(lcd_cam.cam, channel.rx, cam_data_pins, 20u32.MHz()) + .with_master_clock(cam_xclk) + .with_pixel_clock(cam_pclk) + .with_ctrl_pins(cam_vsync, cam_href); let delay = Delay::new(); - let mut buffer = rx_buffer; - buffer.fill(0u8); - delay.delay_millis(500u32); let i2c = I2C::new(peripherals.I2C0, cam_siod, cam_sioc, 100u32.kHz()); @@ -101,10 +92,6 @@ fn main() -> ! { let pid = sccb.read(OV2640_ADDRESS, 0x0A).unwrap(); println!("Found PID of {:#02X}, and was expecting 0x26", pid); - // Start waiting for camera before initialising it to prevent missing the first few bytes. - // This can be improved with a VSYNC interrupt but would complicate this example. - let transfer = camera.read_dma(&mut buffer).unwrap(); - for (reg, value) in FIRST_BLOCK { sccb.write(OV2640_ADDRESS, *reg, *value).unwrap(); } @@ -116,20 +103,61 @@ fn main() -> ! { } } - transfer.wait().unwrap(); + // Start receiving data from the camera. + let mut transfer = camera.receive(dma_rx_buf).map_err(|e| e.0).unwrap(); + + // Skip the first 2 images. Each image ends with an EOF. + // We likely missed the first few bytes of the first image and the second image is likely + // garbage from the OV2640 focusing, calibrating, etc. + // Feel free to skip more images if the one captured below is still garbage. + for _ in 0..2 { + let mut total_bytes = 0; + loop { + let (data, ends_with_eof) = transfer.peek_until_eof(); + if data.is_empty() { + if transfer.is_done() { + panic!("We were too slow to read from the DMA"); + } + } else { + let bytes_peeked = data.len(); + transfer.consume(bytes_peeked); + total_bytes += bytes_peeked; + if ends_with_eof { + // Found the end of the image/frame. + println!("Skipped a {} byte image", total_bytes); + break; + } + } + } + } - // Note: JPEGs starts with "FF, D8, FF, E0" and end with "FF, D9" + println!("Frame data (parse with `xxd -r -p .txt image.jpg`):"); - let index_of_end = buffer.windows(2).position(|c| c[0] == 0xFF && c[1] == 0xD9); - let index_of_end = if let Some(idx) = index_of_end { - idx + 2 - } else { - println!("Failed to find JPEG terminator"); - buffer.len() - }; + // Note: JPEGs starts with "FF, D8, FF, E0" and end with "FF, D9". + // The OV2640 also sends some trailing zeros after the JPEG. This is expected. + + loop { + let (data, ends_with_eof) = transfer.peek_until_eof(); + if data.is_empty() { + if transfer.is_done() { + panic!("We were too slow to read from the DMA"); + } + } else { + for b in data { + print!("{:02X}, ", b); + } + let bytes_peeked = data.len(); + transfer.consume(bytes_peeked); + + if ends_with_eof { + // Found the end of the image/frame. + break; + } + } + } - println!("Frame data (parse with `xxd -r -p .txt image.jpg`):"); - println!("{:02X?}", &buffer[..index_of_end]); + // The full frame has been captured, the transfer can be stopped now. + let _ = transfer.stop(); loop {} }