Migrate Camera to a move based API (#2242)

* Migrate Camera to a move based API

* update

* update

* ManuallyDrop

* Add default channel to transfer struct

---------

Co-authored-by: Dominic Fischer <git@dominicfischer.me>
This commit is contained in:
Dominic Fischer 2024-10-02 14:23:14 +01:00 committed by GitHub
parent 8789ca3c3d
commit 30aef580e3
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
5 changed files with 668 additions and 131 deletions

View File

@ -29,6 +29,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)
- Added `Efuse::read_bit` (#2259)
@ -56,6 +57,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)

View File

@ -227,6 +227,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.

View File

@ -53,7 +53,13 @@
//!
//! For convenience you can use the [crate::dma_buffers] macro.
use core::{cmp::min, fmt::Debug, marker::PhantomData, sync::atomic::compiler_fence};
use core::{
cmp::min,
fmt::Debug,
marker::PhantomData,
ptr::null_mut,
sync::atomic::compiler_fence,
};
trait Word: crate::private::Sealed {}
@ -722,6 +728,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, 1000);
/// # }
/// ```
#[macro_export]
macro_rules! dma_rx_stream_buffer {
($rx_size:expr) => {
$crate::dma_rx_stream_buffer!($rx_size, 4095)
};
($rx_size:expr, $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))]
@ -2116,12 +2151,22 @@ pub struct Preparation {
/// The implementing type must keep all its descriptors and the buffers they
/// point to valid while the buffer is being transferred.
pub unsafe trait DmaTxBuffer {
/// A type providing operations that are safe to perform on the buffer
/// whilst the DMA is actively using it.
type View;
/// 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;
/// This is called before the DMA starts using the buffer.
fn into_view(self) -> Self::View;
/// This is called after the DMA is done using the buffer.
fn from_view(view: Self::View) -> Self;
/// Returns the maximum number of bytes that would be transmitted by this
/// buffer.
///
@ -2142,12 +2187,22 @@ pub unsafe trait DmaTxBuffer {
/// The implementing type must keep all its descriptors and the buffers they
/// point to valid while the buffer is being transferred.
pub unsafe trait DmaRxBuffer {
/// A type providing operations that are safe to perform on the buffer
/// whilst the DMA is actively using it.
type View;
/// 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;
/// This is called before the DMA starts using the buffer.
fn into_view(self) -> Self::View;
/// This is called after the DMA is done using the buffer.
fn from_view(view: Self::View) -> Self;
/// Returns the maximum number of bytes that can be received by this buffer.
///
/// This is a convenience hint for SPI. Most peripherals don't care how long
@ -2155,6 +2210,12 @@ pub unsafe trait DmaRxBuffer {
fn length(&self) -> usize;
}
/// An in-progress view into [DmaRxBuf]/[DmaTxBuf].
///
/// In the future, this could support peeking into state of the
/// descriptors/buffers.
pub struct BufView<T>(T);
/// Error returned from Dma[Rx|Tx|RxTx]Buf operations.
#[derive(Debug, PartialEq, Clone, Copy)]
#[cfg_attr(feature = "defmt", derive(defmt::Format))]
@ -2335,6 +2396,8 @@ impl DmaTxBuf {
}
unsafe impl DmaTxBuffer for DmaTxBuf {
type View = BufView<DmaTxBuf>;
fn prepare(&mut self) -> Preparation {
for desc in self.descriptors.linked_iter_mut() {
// In non-circular mode, we only set `suc_eof` for the last descriptor to signal
@ -2358,6 +2421,14 @@ unsafe impl DmaTxBuffer for DmaTxBuf {
}
}
fn into_view(self) -> BufView<DmaTxBuf> {
BufView(self)
}
fn from_view(view: Self::View) -> Self {
view.0
}
fn length(&self) -> usize {
self.len()
}
@ -2483,6 +2554,8 @@ impl DmaRxBuf {
}
unsafe impl DmaRxBuffer for DmaRxBuf {
type View = BufView<DmaRxBuf>;
fn prepare(&mut self) -> Preparation {
for desc in self.descriptors.linked_iter_mut() {
desc.reset_for_rx();
@ -2494,6 +2567,14 @@ unsafe impl DmaRxBuffer for DmaRxBuf {
}
}
fn into_view(self) -> BufView<DmaRxBuf> {
BufView(self)
}
fn from_view(view: Self::View) -> Self {
view.0
}
fn length(&self) -> usize {
self.len()
}
@ -2595,6 +2676,8 @@ impl DmaRxTxBuf {
}
unsafe impl DmaTxBuffer for DmaRxTxBuf {
type View = BufView<DmaRxTxBuf>;
fn prepare(&mut self) -> Preparation {
for desc in self.tx_descriptors.linked_iter_mut() {
// In non-circular mode, we only set `suc_eof` for the last descriptor to signal
@ -2608,12 +2691,22 @@ unsafe impl DmaTxBuffer for DmaRxTxBuf {
}
}
fn into_view(self) -> BufView<DmaRxTxBuf> {
BufView(self)
}
fn from_view(view: Self::View) -> Self {
view.0
}
fn length(&self) -> usize {
self.len()
}
}
unsafe impl DmaRxBuffer for DmaRxTxBuf {
type View = BufView<DmaRxTxBuf>;
fn prepare(&mut self) -> Preparation {
for desc in self.rx_descriptors.linked_iter_mut() {
desc.reset_for_rx();
@ -2625,11 +2718,330 @@ unsafe impl DmaRxBuffer for DmaRxTxBuf {
}
}
fn into_view(self) -> BufView<DmaRxTxBuf> {
BufView(self)
}
fn from_view(view: Self::View) -> Self {
view.0
}
fn length(&self) -> usize {
self.len()
}
}
/// 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<Self, DmaBufError> {
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)
}
}
unsafe impl DmaRxBuffer for DmaRxStreamBuf {
type View = DmaRxStreamBufView;
fn prepare(&mut self) -> Preparation {
for desc in self.descriptors.iter_mut() {
desc.reset_for_rx();
}
Preparation {
start: self.descriptors.as_mut_ptr(),
block_size: None,
}
}
fn into_view(self) -> DmaRxStreamBufView {
DmaRxStreamBufView {
buf: self,
descriptor_idx: 0,
descriptor_offset: 0,
}
}
fn from_view(view: Self::View) -> Self {
view.buf
}
fn length(&self) -> usize {
panic!("DmaCircularBuf doesn't have a length")
}
}
/// A view 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,
)
}
}
}
pub(crate) mod dma_private {
use super::*;

View File

@ -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, 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,27 +59,30 @@
//! .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::{
mem::ManuallyDrop,
ops::{Deref, DerefMut},
};
use fugit::HertzU32;
use crate::{
clock::Clocks,
dma::{
dma_private::{DmaSupport, DmaSupportRx},
AnyDmaChannel,
ChannelRx,
DescriptorChain,
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},
@ -130,9 +132,6 @@ pub struct Cam<'d> {
pub struct Camera<'d, CH: DmaChannel = AnyDmaChannel> {
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>
@ -143,7 +142,6 @@ where
pub fn new<P: RxPins>(
cam: Cam<'d>,
channel: ChannelRx<'d, CH>,
descriptors: &'static mut [DmaDescriptor],
_pins: P,
frequency: HertzU32,
) -> Self {
@ -177,7 +175,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();
@ -193,49 +191,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 {
@ -347,8 +306,12 @@ impl<'d, CH: DmaChannel> Camera<'d, CH> {
self
}
/// Starts a DMA transfer to receive data from the camera peripheral.
pub fn receive<BUF: DmaRxBuffer>(
mut self,
mut buf: BUF,
) -> Result<CameraTransfer<'d, BUF, CH>, (DmaError, Self, BUF)> {
// Reset Camera control unit and Async Rx FIFO
fn reset_unit_and_fifo(&self) {
self.lcd_cam
.cam_ctrl1()
.modify(|_, w| w.cam_reset().set_bit());
@ -361,60 +324,149 @@ impl<'d, CH: DmaChannel> Camera<'d, CH> {
self.lcd_cam
.cam_ctrl1()
.modify(|_, w| w.cam_afifo_reset().clear_bit());
// Start DMA to receive incoming transfer.
let result = unsafe {
self.rx_channel
.prepare_transfer(DmaPeripheral::LcdCam, &mut buf)
.and_then(|_| self.rx_channel.start_transfer())
};
if let Err(e) = result {
return Err((e, self, buf));
}
// 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());
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: ManuallyDrop::new(self),
buffer_view: ManuallyDrop::new(buf.into_view()),
})
}
}
/// Represents an ongoing (or potentially stopped) transfer from the Camera to a
/// DMA buffer.
pub struct CameraTransfer<'d, BUF: DmaRxBuffer, CH: DmaChannel = AnyDmaChannel> {
camera: ManuallyDrop<Camera<'d, CH>>,
buffer_view: ManuallyDrop<BUF::View>,
}
impl<'d, BUF: DmaRxBuffer, CH: DmaChannel> CameraTransfer<'d, BUF, CH> {
/// 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<RXBUF: WriteBuffer>(
&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, BUF::from_view(view))
}
assert!(len % self.bus_width == 0);
/// 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(())
};
(result, camera, BUF::from_view(view))
}
fn release(mut self) -> (Camera<'d, CH>, BUF::View) {
// SAFETY: Since forget is called on self, we know that self.camera and
// self.buffer_view won't be touched again.
let result = unsafe {
let camera = ManuallyDrop::take(&mut self.camera);
let view = ManuallyDrop::take(&mut self.buffer_view);
(camera, view)
};
core::mem::forget(self);
result
}
fn stop_peripherals(&mut self) {
// Stop the LCD_CAM peripheral.
self.camera
.lcd_cam
.cam_ctrl1()
.modify(|_, w| w.cam_start().clear_bit());
// Stop the DMA
self.camera.rx_channel.stop_transfer();
}
}
impl<'d, BUF: DmaRxBuffer, CH: DmaChannel> Deref for CameraTransfer<'d, BUF, CH> {
type Target = BUF::View;
fn deref(&self) -> &Self::Target {
&self.buffer_view
}
}
impl<'d, BUF: DmaRxBuffer, CH: DmaChannel> DerefMut for CameraTransfer<'d, BUF, CH> {
fn deref_mut(&mut self) -> &mut Self::Target {
&mut self.buffer_view
}
}
impl<'d, BUF: DmaRxBuffer, CH: DmaChannel> Drop for CameraTransfer<'d, BUF, CH> {
fn drop(&mut self) {
self.stop_peripherals();
// SAFETY: This is Drop, we know that self.camera and self.buffer_view
// won't be touched again.
unsafe {
self.rx_chain.fill_for_rx(circular, ptr as _, len)?;
self.rx_channel
.prepare_transfer_without_start(DmaPeripheral::LcdCam, &self.rx_chain)?;
ManuallyDrop::drop(&mut self.camera);
ManuallyDrop::drop(&mut self.buffer_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<DmaTransferRx<'_, Self>, DmaError> {
self.reset_unit_and_fifo();
// Start DMA to receive incoming transfer.
self.start_dma(false, buf)?;
self.start_unit();
Ok(DmaTransferRx::new(self))
}
/// 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<DmaTransferRxCircular<'_, Self>, DmaError> {
self.reset_unit_and_fifo();
// Start DMA to receive incoming transfer.
self.start_dma(true, buf)?;
self.start_unit();
Ok(DmaTransferRxCircular::new(self))
}
}

View File

@ -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, 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(),
)
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();
// Note: JPEGs starts with "FF, D8, FF, E0" and end with "FF, D9"
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
// 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 {
println!("Failed to find JPEG terminator");
buffer.len()
};
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;
}
}
}
}
println!("Frame data (parse with `xxd -r -p <uart>.txt image.jpg`):");
println!("{:02X?}", &buffer[..index_of_end]);
// 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;
}
}
}
// The full frame has been captured, the transfer can be stopped now.
let _ = transfer.stop();
loop {}
}