wgpu_core/device/
queue.rs

1#[cfg(feature = "trace")]
2use crate::device::trace::Action;
3use crate::{
4    api_log,
5    command::{
6        extract_texture_selector, validate_linear_texture_data, validate_texture_copy_range,
7        ClearError, CommandAllocator, CommandBuffer, CopySide, ImageCopyTexture, TransferError,
8    },
9    conv,
10    device::{life::ResourceMaps, DeviceError, WaitIdleError},
11    get_lowest_common_denom,
12    global::Global,
13    hal_api::HalApi,
14    hal_label,
15    id::{self, DeviceId, QueueId},
16    init_tracker::{has_copy_partial_init_tracker_coverage, TextureInitRange},
17    lock::{rank, Mutex},
18    resource::{
19        Buffer, BufferAccessError, BufferMapState, DestroyedBuffer, DestroyedTexture, Resource,
20        ResourceInfo, ResourceType, StagingBuffer, Texture, TextureInner,
21    },
22    resource_log, track, FastHashMap, SubmissionIndex,
23};
24
25use hal::{CommandEncoder as _, Device as _, Queue as _};
26use smallvec::SmallVec;
27
28use std::{
29    iter, mem, ptr,
30    sync::{atomic::Ordering, Arc},
31};
32use thiserror::Error;
33
34use super::Device;
35
36pub struct Queue<A: HalApi> {
37    pub(crate) device: Option<Arc<Device<A>>>,
38    pub(crate) raw: Option<A::Queue>,
39    pub(crate) info: ResourceInfo<Queue<A>>,
40}
41
42impl<A: HalApi> Resource for Queue<A> {
43    const TYPE: ResourceType = "Queue";
44
45    type Marker = crate::id::markers::Queue;
46
47    fn as_info(&self) -> &ResourceInfo<Self> {
48        &self.info
49    }
50
51    fn as_info_mut(&mut self) -> &mut ResourceInfo<Self> {
52        &mut self.info
53    }
54}
55
56impl<A: HalApi> Drop for Queue<A> {
57    fn drop(&mut self) {
58        let queue = self.raw.take().unwrap();
59        self.device.as_ref().unwrap().release_queue(queue);
60    }
61}
62
63/// Number of command buffers that we generate from the same pool
64/// for the write_xxx commands, before the pool is recycled.
65///
66/// If we don't stop at some point, the pool will grow forever,
67/// without a concrete moment of when it can be cleared.
68const WRITE_COMMAND_BUFFERS_PER_POOL: usize = 64;
69
70#[repr(C)]
71pub struct SubmittedWorkDoneClosureC {
72    pub callback: unsafe extern "C" fn(user_data: *mut u8),
73    pub user_data: *mut u8,
74}
75
76#[cfg(send_sync)]
77unsafe impl Send for SubmittedWorkDoneClosureC {}
78
79pub struct SubmittedWorkDoneClosure {
80    // We wrap this so creating the enum in the C variant can be unsafe,
81    // allowing our call function to be safe.
82    inner: SubmittedWorkDoneClosureInner,
83}
84
85#[cfg(send_sync)]
86type SubmittedWorkDoneCallback = Box<dyn FnOnce() + Send + 'static>;
87#[cfg(not(send_sync))]
88type SubmittedWorkDoneCallback = Box<dyn FnOnce() + 'static>;
89
90enum SubmittedWorkDoneClosureInner {
91    Rust { callback: SubmittedWorkDoneCallback },
92    C { inner: SubmittedWorkDoneClosureC },
93}
94
95impl SubmittedWorkDoneClosure {
96    pub fn from_rust(callback: SubmittedWorkDoneCallback) -> Self {
97        Self {
98            inner: SubmittedWorkDoneClosureInner::Rust { callback },
99        }
100    }
101
102    /// # Safety
103    ///
104    /// - The callback pointer must be valid to call with the provided `user_data`
105    ///   pointer.
106    ///
107    /// - Both pointers must point to `'static` data, as the callback may happen at
108    ///   an unspecified time.
109    pub unsafe fn from_c(inner: SubmittedWorkDoneClosureC) -> Self {
110        Self {
111            inner: SubmittedWorkDoneClosureInner::C { inner },
112        }
113    }
114
115    pub(crate) fn call(self) {
116        match self.inner {
117            SubmittedWorkDoneClosureInner::Rust { callback } => callback(),
118            // SAFETY: the contract of the call to from_c says that this unsafe is sound.
119            SubmittedWorkDoneClosureInner::C { inner } => unsafe {
120                (inner.callback)(inner.user_data)
121            },
122        }
123    }
124}
125
126#[repr(C)]
127#[derive(Debug, Copy, Clone)]
128pub struct WrappedSubmissionIndex {
129    pub queue_id: QueueId,
130    pub index: SubmissionIndex,
131}
132
133/// A texture or buffer to be freed soon.
134///
135/// This is just a tagged raw texture or buffer, generally about to be added to
136/// some other more specific container like:
137///
138/// - `PendingWrites::temp_resources`: resources used by queue writes and
139///   unmaps, waiting to be folded in with the next queue submission
140///
141/// - `ActiveSubmission::last_resources`: temporary resources used by a queue
142///   submission, to be freed when it completes
143///
144/// - `LifetimeTracker::free_resources`: resources to be freed in the next
145///   `maintain` call, no longer used anywhere
146#[derive(Debug)]
147pub enum TempResource<A: HalApi> {
148    Buffer(Arc<Buffer<A>>),
149    StagingBuffer(Arc<StagingBuffer<A>>),
150    DestroyedBuffer(Arc<DestroyedBuffer<A>>),
151    DestroyedTexture(Arc<DestroyedTexture<A>>),
152    Texture(Arc<Texture<A>>),
153}
154
155/// A series of raw [`CommandBuffer`]s that have been submitted to a
156/// queue, and the [`wgpu_hal::CommandEncoder`] that built them.
157///
158/// [`CommandBuffer`]: hal::Api::CommandBuffer
159/// [`wgpu_hal::CommandEncoder`]: hal::CommandEncoder
160pub(crate) struct EncoderInFlight<A: HalApi> {
161    raw: A::CommandEncoder,
162    cmd_buffers: Vec<A::CommandBuffer>,
163}
164
165impl<A: HalApi> EncoderInFlight<A> {
166    /// Free all of our command buffers.
167    ///
168    /// Return the command encoder, fully reset and ready to be
169    /// reused.
170    pub(crate) unsafe fn land(mut self) -> A::CommandEncoder {
171        unsafe { self.raw.reset_all(self.cmd_buffers.into_iter()) };
172        self.raw
173    }
174}
175
176/// A private command encoder for writes made directly on the device
177/// or queue.
178///
179/// Operations like `buffer_unmap`, `queue_write_buffer`, and
180/// `queue_write_texture` need to copy data to the GPU. At the hal
181/// level, this must be done by encoding and submitting commands, but
182/// these operations are not associated with any specific wgpu command
183/// buffer.
184///
185/// Instead, `Device::pending_writes` owns one of these values, which
186/// has its own hal command encoder and resource lists. The commands
187/// accumulated here are automatically submitted to the queue the next
188/// time the user submits a wgpu command buffer, ahead of the user's
189/// commands.
190///
191/// Important:
192/// When locking pending_writes be sure that tracker is not locked
193/// and try to lock trackers for the minimum timespan possible
194///
195/// All uses of [`StagingBuffer`]s end up here.
196#[derive(Debug)]
197pub(crate) struct PendingWrites<A: HalApi> {
198    pub command_encoder: A::CommandEncoder,
199
200    /// True if `command_encoder` is in the "recording" state, as
201    /// described in the docs for the [`wgpu_hal::CommandEncoder`]
202    /// trait.
203    ///
204    /// [`wgpu_hal::CommandEncoder`]: hal::CommandEncoder
205    pub is_recording: bool,
206
207    pub temp_resources: Vec<TempResource<A>>,
208    pub dst_buffers: FastHashMap<id::BufferId, Arc<Buffer<A>>>,
209    pub dst_textures: FastHashMap<id::TextureId, Arc<Texture<A>>>,
210
211    /// All command buffers allocated from `command_encoder`.
212    pub executing_command_buffers: Vec<A::CommandBuffer>,
213}
214
215impl<A: HalApi> PendingWrites<A> {
216    pub fn new(command_encoder: A::CommandEncoder) -> Self {
217        Self {
218            command_encoder,
219            is_recording: false,
220            temp_resources: Vec::new(),
221            dst_buffers: FastHashMap::default(),
222            dst_textures: FastHashMap::default(),
223            executing_command_buffers: Vec::new(),
224        }
225    }
226
227    pub fn dispose(mut self, device: &A::Device) {
228        unsafe {
229            if self.is_recording {
230                self.command_encoder.discard_encoding();
231            }
232            self.command_encoder
233                .reset_all(self.executing_command_buffers.into_iter());
234            device.destroy_command_encoder(self.command_encoder);
235        }
236
237        self.temp_resources.clear();
238    }
239
240    pub fn consume_temp(&mut self, resource: TempResource<A>) {
241        self.temp_resources.push(resource);
242    }
243
244    fn consume(&mut self, buffer: Arc<StagingBuffer<A>>) {
245        self.temp_resources
246            .push(TempResource::StagingBuffer(buffer));
247    }
248
249    fn pre_submit(&mut self) -> Result<Option<&A::CommandBuffer>, DeviceError> {
250        self.dst_buffers.clear();
251        self.dst_textures.clear();
252        if self.is_recording {
253            let cmd_buf = unsafe { self.command_encoder.end_encoding()? };
254            self.is_recording = false;
255            self.executing_command_buffers.push(cmd_buf);
256
257            return Ok(self.executing_command_buffers.last());
258        }
259
260        Ok(None)
261    }
262
263    #[must_use]
264    fn post_submit(
265        &mut self,
266        command_allocator: &CommandAllocator<A>,
267        device: &A::Device,
268        queue: &A::Queue,
269    ) -> Option<EncoderInFlight<A>> {
270        if self.executing_command_buffers.len() >= WRITE_COMMAND_BUFFERS_PER_POOL {
271            let new_encoder = command_allocator.acquire_encoder(device, queue).unwrap();
272            Some(EncoderInFlight {
273                raw: mem::replace(&mut self.command_encoder, new_encoder),
274                cmd_buffers: mem::take(&mut self.executing_command_buffers),
275            })
276        } else {
277            None
278        }
279    }
280
281    pub fn activate(&mut self) -> &mut A::CommandEncoder {
282        if !self.is_recording {
283            unsafe {
284                self.command_encoder
285                    .begin_encoding(Some("(wgpu internal) PendingWrites"))
286                    .unwrap();
287            }
288            self.is_recording = true;
289        }
290        &mut self.command_encoder
291    }
292
293    pub fn deactivate(&mut self) {
294        if self.is_recording {
295            unsafe {
296                self.command_encoder.discard_encoding();
297            }
298            self.is_recording = false;
299        }
300    }
301}
302
303fn prepare_staging_buffer<A: HalApi>(
304    device: &Arc<Device<A>>,
305    size: wgt::BufferAddress,
306    instance_flags: wgt::InstanceFlags,
307) -> Result<(StagingBuffer<A>, *mut u8), DeviceError> {
308    profiling::scope!("prepare_staging_buffer");
309    let stage_desc = hal::BufferDescriptor {
310        label: hal_label(Some("(wgpu internal) Staging"), instance_flags),
311        size,
312        usage: hal::BufferUses::MAP_WRITE | hal::BufferUses::COPY_SRC,
313        memory_flags: hal::MemoryFlags::TRANSIENT,
314    };
315
316    let buffer = unsafe { device.raw().create_buffer(&stage_desc)? };
317    let mapping = unsafe { device.raw().map_buffer(&buffer, 0..size) }?;
318
319    let staging_buffer = StagingBuffer {
320        raw: Mutex::new(rank::STAGING_BUFFER_RAW, Some(buffer)),
321        device: device.clone(),
322        size,
323        info: ResourceInfo::new(
324            "<StagingBuffer>",
325            Some(device.tracker_indices.staging_buffers.clone()),
326        ),
327        is_coherent: mapping.is_coherent,
328    };
329
330    Ok((staging_buffer, mapping.ptr.as_ptr()))
331}
332
333impl<A: HalApi> StagingBuffer<A> {
334    unsafe fn flush(&self, device: &A::Device) -> Result<(), DeviceError> {
335        if !self.is_coherent {
336            unsafe {
337                device.flush_mapped_ranges(
338                    self.raw.lock().as_ref().unwrap(),
339                    iter::once(0..self.size),
340                )
341            };
342        }
343        unsafe { device.unmap_buffer(self.raw.lock().as_ref().unwrap())? };
344        Ok(())
345    }
346}
347
348#[derive(Clone, Debug, Error)]
349#[error("Queue is invalid")]
350pub struct InvalidQueue;
351
352#[derive(Clone, Debug, Error)]
353#[non_exhaustive]
354pub enum QueueWriteError {
355    #[error(
356        "Device of queue ({:?}) does not match device of write recipient ({:?})",
357        queue_device_id,
358        target_device_id
359    )]
360    DeviceMismatch {
361        queue_device_id: DeviceId,
362        target_device_id: DeviceId,
363    },
364    #[error(transparent)]
365    Queue(#[from] DeviceError),
366    #[error(transparent)]
367    Transfer(#[from] TransferError),
368    #[error(transparent)]
369    MemoryInitFailure(#[from] ClearError),
370}
371
372#[derive(Clone, Debug, Error)]
373#[non_exhaustive]
374pub enum QueueSubmitError {
375    #[error(transparent)]
376    Queue(#[from] DeviceError),
377    #[error("Buffer {0:?} is destroyed")]
378    DestroyedBuffer(id::BufferId),
379    #[error("Texture {0:?} is destroyed")]
380    DestroyedTexture(id::TextureId),
381    #[error(transparent)]
382    Unmap(#[from] BufferAccessError),
383    #[error("Buffer {0:?} is still mapped")]
384    BufferStillMapped(id::BufferId),
385    #[error("Surface output was dropped before the command buffer got submitted")]
386    SurfaceOutputDropped,
387    #[error("Surface was unconfigured before the command buffer got submitted")]
388    SurfaceUnconfigured,
389    #[error("GPU got stuck :(")]
390    StuckGpu,
391}
392
393//TODO: move out common parts of write_xxx.
394
395impl Global {
396    pub fn queue_write_buffer<A: HalApi>(
397        &self,
398        queue_id: QueueId,
399        buffer_id: id::BufferId,
400        buffer_offset: wgt::BufferAddress,
401        data: &[u8],
402    ) -> Result<(), QueueWriteError> {
403        profiling::scope!("Queue::write_buffer");
404        api_log!("Queue::write_buffer {buffer_id:?} {}bytes", data.len());
405
406        let hub = A::hub(self);
407
408        let buffer_device_id = hub
409            .buffers
410            .get(buffer_id)
411            .map_err(|_| TransferError::InvalidBuffer(buffer_id))?
412            .device
413            .as_info()
414            .id();
415
416        let queue = hub
417            .queues
418            .get(queue_id)
419            .map_err(|_| DeviceError::InvalidQueueId)?;
420
421        let device = queue.device.as_ref().unwrap();
422
423        {
424            let queue_device_id = device.as_info().id();
425            if buffer_device_id != queue_device_id {
426                return Err(QueueWriteError::DeviceMismatch {
427                    queue_device_id,
428                    target_device_id: buffer_device_id,
429                });
430            }
431        }
432
433        let data_size = data.len() as wgt::BufferAddress;
434
435        #[cfg(feature = "trace")]
436        if let Some(ref mut trace) = *device.trace.lock() {
437            let data_path = trace.make_binary("bin", data);
438            trace.add(Action::WriteBuffer {
439                id: buffer_id,
440                data: data_path,
441                range: buffer_offset..buffer_offset + data_size,
442                queued: true,
443            });
444        }
445
446        if data_size == 0 {
447            log::trace!("Ignoring write_buffer of size 0");
448            return Ok(());
449        }
450
451        // Platform validation requires that the staging buffer always be
452        // freed, even if an error occurs. All paths from here must call
453        // `device.pending_writes.consume`.
454        let (staging_buffer, staging_buffer_ptr) =
455            prepare_staging_buffer(device, data_size, device.instance_flags)?;
456        let mut pending_writes = device.pending_writes.lock();
457        let pending_writes = pending_writes.as_mut().unwrap();
458
459        let stage_fid = hub.staging_buffers.request();
460        let staging_buffer = stage_fid.init(staging_buffer);
461
462        if let Err(flush_error) = unsafe {
463            profiling::scope!("copy");
464            ptr::copy_nonoverlapping(data.as_ptr(), staging_buffer_ptr, data.len());
465            staging_buffer.flush(device.raw())
466        } {
467            pending_writes.consume(staging_buffer);
468            return Err(flush_error.into());
469        }
470
471        let result = self.queue_write_staging_buffer_impl(
472            device,
473            pending_writes,
474            &staging_buffer,
475            buffer_id,
476            buffer_offset,
477        );
478
479        pending_writes.consume(staging_buffer);
480        result
481    }
482
483    pub fn queue_create_staging_buffer<A: HalApi>(
484        &self,
485        queue_id: QueueId,
486        buffer_size: wgt::BufferSize,
487        id_in: Option<id::StagingBufferId>,
488    ) -> Result<(id::StagingBufferId, *mut u8), QueueWriteError> {
489        profiling::scope!("Queue::create_staging_buffer");
490        let hub = A::hub(self);
491
492        let queue = hub
493            .queues
494            .get(queue_id)
495            .map_err(|_| DeviceError::InvalidQueueId)?;
496
497        let device = queue.device.as_ref().unwrap();
498
499        let (staging_buffer, staging_buffer_ptr) =
500            prepare_staging_buffer(device, buffer_size.get(), device.instance_flags)?;
501
502        let fid = hub.staging_buffers.prepare(id_in);
503        let (id, _) = fid.assign(Arc::new(staging_buffer));
504        resource_log!("Queue::create_staging_buffer {id:?}");
505
506        Ok((id, staging_buffer_ptr))
507    }
508
509    pub fn queue_write_staging_buffer<A: HalApi>(
510        &self,
511        queue_id: QueueId,
512        buffer_id: id::BufferId,
513        buffer_offset: wgt::BufferAddress,
514        staging_buffer_id: id::StagingBufferId,
515    ) -> Result<(), QueueWriteError> {
516        profiling::scope!("Queue::write_staging_buffer");
517        let hub = A::hub(self);
518
519        let queue = hub
520            .queues
521            .get(queue_id)
522            .map_err(|_| DeviceError::InvalidQueueId)?;
523
524        let device = queue.device.as_ref().unwrap();
525
526        let staging_buffer = hub.staging_buffers.unregister(staging_buffer_id);
527        if staging_buffer.is_none() {
528            return Err(QueueWriteError::Transfer(TransferError::InvalidBuffer(
529                buffer_id,
530            )));
531        }
532        let staging_buffer = staging_buffer.unwrap();
533        let mut pending_writes = device.pending_writes.lock();
534        let pending_writes = pending_writes.as_mut().unwrap();
535
536        // At this point, we have taken ownership of the staging_buffer from the
537        // user. Platform validation requires that the staging buffer always
538        // be freed, even if an error occurs. All paths from here must call
539        // `device.pending_writes.consume`.
540        if let Err(flush_error) = unsafe { staging_buffer.flush(device.raw()) } {
541            pending_writes.consume(staging_buffer);
542            return Err(flush_error.into());
543        }
544
545        let result = self.queue_write_staging_buffer_impl(
546            device,
547            pending_writes,
548            &staging_buffer,
549            buffer_id,
550            buffer_offset,
551        );
552
553        pending_writes.consume(staging_buffer);
554        result
555    }
556
557    pub fn queue_validate_write_buffer<A: HalApi>(
558        &self,
559        _queue_id: QueueId,
560        buffer_id: id::BufferId,
561        buffer_offset: u64,
562        buffer_size: u64,
563    ) -> Result<(), QueueWriteError> {
564        profiling::scope!("Queue::validate_write_buffer");
565        let hub = A::hub(self);
566
567        let buffer = hub
568            .buffers
569            .get(buffer_id)
570            .map_err(|_| TransferError::InvalidBuffer(buffer_id))?;
571
572        self.queue_validate_write_buffer_impl(&buffer, buffer_id, buffer_offset, buffer_size)?;
573
574        Ok(())
575    }
576
577    fn queue_validate_write_buffer_impl<A: HalApi>(
578        &self,
579        buffer: &Buffer<A>,
580        buffer_id: id::BufferId,
581        buffer_offset: u64,
582        buffer_size: u64,
583    ) -> Result<(), TransferError> {
584        if !buffer.usage.contains(wgt::BufferUsages::COPY_DST) {
585            return Err(TransferError::MissingCopyDstUsageFlag(
586                Some(buffer_id),
587                None,
588            ));
589        }
590        if buffer_size % wgt::COPY_BUFFER_ALIGNMENT != 0 {
591            return Err(TransferError::UnalignedCopySize(buffer_size));
592        }
593        if buffer_offset % wgt::COPY_BUFFER_ALIGNMENT != 0 {
594            return Err(TransferError::UnalignedBufferOffset(buffer_offset));
595        }
596        if buffer_offset + buffer_size > buffer.size {
597            return Err(TransferError::BufferOverrun {
598                start_offset: buffer_offset,
599                end_offset: buffer_offset + buffer_size,
600                buffer_size: buffer.size,
601                side: CopySide::Destination,
602            });
603        }
604
605        Ok(())
606    }
607
608    fn queue_write_staging_buffer_impl<A: HalApi>(
609        &self,
610        device: &Device<A>,
611        pending_writes: &mut PendingWrites<A>,
612        staging_buffer: &StagingBuffer<A>,
613        buffer_id: id::BufferId,
614        buffer_offset: u64,
615    ) -> Result<(), QueueWriteError> {
616        let hub = A::hub(self);
617
618        let (dst, transition) = {
619            let buffer_guard = hub.buffers.read();
620            let dst = buffer_guard
621                .get(buffer_id)
622                .map_err(|_| TransferError::InvalidBuffer(buffer_id))?;
623            let mut trackers = device.trackers.lock();
624            trackers
625                .buffers
626                .set_single(dst, hal::BufferUses::COPY_DST)
627                .ok_or(TransferError::InvalidBuffer(buffer_id))?
628        };
629        let snatch_guard = device.snatchable_lock.read();
630        let dst_raw = dst
631            .raw
632            .get(&snatch_guard)
633            .ok_or(TransferError::InvalidBuffer(buffer_id))?;
634
635        if dst.device.as_info().id() != device.as_info().id() {
636            return Err(DeviceError::WrongDevice.into());
637        }
638
639        let src_buffer_size = staging_buffer.size;
640        self.queue_validate_write_buffer_impl(&dst, buffer_id, buffer_offset, src_buffer_size)?;
641
642        dst.info
643            .use_at(device.active_submission_index.load(Ordering::Relaxed) + 1);
644
645        let region = wgt::BufferSize::new(src_buffer_size).map(|size| hal::BufferCopy {
646            src_offset: 0,
647            dst_offset: buffer_offset,
648            size,
649        });
650        let inner_buffer = staging_buffer.raw.lock();
651        let barriers = iter::once(hal::BufferBarrier {
652            buffer: inner_buffer.as_ref().unwrap(),
653            usage: hal::BufferUses::MAP_WRITE..hal::BufferUses::COPY_SRC,
654        })
655        .chain(transition.map(|pending| pending.into_hal(&dst, &snatch_guard)));
656        let encoder = pending_writes.activate();
657        unsafe {
658            encoder.transition_buffers(barriers);
659            encoder.copy_buffer_to_buffer(
660                inner_buffer.as_ref().unwrap(),
661                dst_raw,
662                region.into_iter(),
663            );
664        }
665        let dst = hub.buffers.get(buffer_id).unwrap();
666        pending_writes.dst_buffers.insert(buffer_id, dst.clone());
667
668        // Ensure the overwritten bytes are marked as initialized so
669        // they don't need to be nulled prior to mapping or binding.
670        {
671            dst.initialization_status
672                .write()
673                .drain(buffer_offset..(buffer_offset + src_buffer_size));
674        }
675
676        Ok(())
677    }
678
679    pub fn queue_write_texture<A: HalApi>(
680        &self,
681        queue_id: QueueId,
682        destination: &ImageCopyTexture,
683        data: &[u8],
684        data_layout: &wgt::ImageDataLayout,
685        size: &wgt::Extent3d,
686    ) -> Result<(), QueueWriteError> {
687        profiling::scope!("Queue::write_texture");
688        api_log!("Queue::write_texture {:?} {size:?}", destination.texture);
689
690        let hub = A::hub(self);
691
692        let queue = hub
693            .queues
694            .get(queue_id)
695            .map_err(|_| DeviceError::InvalidQueueId)?;
696
697        let device = queue.device.as_ref().unwrap();
698
699        #[cfg(feature = "trace")]
700        if let Some(ref mut trace) = *device.trace.lock() {
701            let data_path = trace.make_binary("bin", data);
702            trace.add(Action::WriteTexture {
703                to: *destination,
704                data: data_path,
705                layout: *data_layout,
706                size: *size,
707            });
708        }
709
710        if size.width == 0 || size.height == 0 || size.depth_or_array_layers == 0 {
711            log::trace!("Ignoring write_texture of size 0");
712            return Ok(());
713        }
714
715        let dst = hub
716            .textures
717            .get(destination.texture)
718            .map_err(|_| TransferError::InvalidTexture(destination.texture))?;
719
720        if dst.device.as_info().id().into_queue_id() != queue_id {
721            return Err(DeviceError::WrongDevice.into());
722        }
723
724        if !dst.desc.usage.contains(wgt::TextureUsages::COPY_DST) {
725            return Err(
726                TransferError::MissingCopyDstUsageFlag(None, Some(destination.texture)).into(),
727            );
728        }
729
730        // Note: Doing the copy range validation early is important because ensures that the
731        // dimensions are not going to cause overflow in other parts of the validation.
732        let (hal_copy_size, array_layer_count) =
733            validate_texture_copy_range(destination, &dst.desc, CopySide::Destination, size)?;
734
735        let (selector, dst_base) = extract_texture_selector(destination, size, &dst)?;
736
737        if !dst_base.aspect.is_one() {
738            return Err(TransferError::CopyAspectNotOne.into());
739        }
740
741        if !conv::is_valid_copy_dst_texture_format(dst.desc.format, destination.aspect) {
742            return Err(TransferError::CopyToForbiddenTextureFormat {
743                format: dst.desc.format,
744                aspect: destination.aspect,
745            }
746            .into());
747        }
748
749        // Note: `_source_bytes_per_array_layer` is ignored since we
750        // have a staging copy, and it can have a different value.
751        let (_, _source_bytes_per_array_layer) = validate_linear_texture_data(
752            data_layout,
753            dst.desc.format,
754            destination.aspect,
755            data.len() as wgt::BufferAddress,
756            CopySide::Source,
757            size,
758            false,
759        )?;
760
761        if dst.desc.format.is_depth_stencil_format() {
762            device
763                .require_downlevel_flags(wgt::DownlevelFlags::DEPTH_TEXTURE_AND_BUFFER_COPIES)
764                .map_err(TransferError::from)?;
765        }
766
767        let (block_width, block_height) = dst.desc.format.block_dimensions();
768        let width_blocks = size.width / block_width;
769        let height_blocks = size.height / block_height;
770
771        let block_rows_per_image = data_layout.rows_per_image.unwrap_or(
772            // doesn't really matter because we need this only if we copy
773            // more than one layer, and then we validate for this being not
774            // None
775            height_blocks,
776        );
777
778        let block_size = dst
779            .desc
780            .format
781            .block_copy_size(Some(destination.aspect))
782            .unwrap();
783        let bytes_per_row_alignment =
784            get_lowest_common_denom(device.alignments.buffer_copy_pitch.get() as u32, block_size);
785        let stage_bytes_per_row =
786            wgt::math::align_to(block_size * width_blocks, bytes_per_row_alignment);
787
788        let block_rows_in_copy =
789            (size.depth_or_array_layers - 1) * block_rows_per_image + height_blocks;
790        let stage_size = stage_bytes_per_row as u64 * block_rows_in_copy as u64;
791
792        let mut pending_writes = device.pending_writes.lock();
793        let pending_writes = pending_writes.as_mut().unwrap();
794        let encoder = pending_writes.activate();
795
796        // If the copy does not fully cover the layers, we need to initialize to
797        // zero *first* as we don't keep track of partial texture layer inits.
798        //
799        // Strictly speaking we only need to clear the areas of a layer
800        // untouched, but this would get increasingly messy.
801        let init_layer_range = if dst.desc.dimension == wgt::TextureDimension::D3 {
802            // volume textures don't have a layer range as array volumes aren't supported
803            0..1
804        } else {
805            destination.origin.z..destination.origin.z + size.depth_or_array_layers
806        };
807        let mut dst_initialization_status = dst.initialization_status.write();
808        if dst_initialization_status.mips[destination.mip_level as usize]
809            .check(init_layer_range.clone())
810            .is_some()
811        {
812            if has_copy_partial_init_tracker_coverage(size, destination.mip_level, &dst.desc) {
813                for layer_range in dst_initialization_status.mips[destination.mip_level as usize]
814                    .drain(init_layer_range)
815                    .collect::<Vec<std::ops::Range<u32>>>()
816                {
817                    let mut trackers = device.trackers.lock();
818                    crate::command::clear_texture(
819                        &dst,
820                        TextureInitRange {
821                            mip_range: destination.mip_level..(destination.mip_level + 1),
822                            layer_range,
823                        },
824                        encoder,
825                        &mut trackers.textures,
826                        &device.alignments,
827                        device.zero_buffer.as_ref().unwrap(),
828                        &device.snatchable_lock.read(),
829                    )
830                    .map_err(QueueWriteError::from)?;
831                }
832            } else {
833                dst_initialization_status.mips[destination.mip_level as usize]
834                    .drain(init_layer_range);
835            }
836        }
837
838        let snatch_guard = device.snatchable_lock.read();
839
840        // Re-get `dst` immutably here, so that the mutable borrow of the
841        // `texture_guard.get` above ends in time for the `clear_texture`
842        // call above. Since we've held `texture_guard` the whole time, we know
843        // the texture hasn't gone away in the mean time, so we can unwrap.
844        let dst = hub.textures.get(destination.texture).unwrap();
845        dst.info
846            .use_at(device.active_submission_index.load(Ordering::Relaxed) + 1);
847
848        let dst_raw = dst
849            .raw(&snatch_guard)
850            .ok_or(TransferError::InvalidTexture(destination.texture))?;
851
852        let bytes_per_row = data_layout
853            .bytes_per_row
854            .unwrap_or(width_blocks * block_size);
855
856        // Platform validation requires that the staging buffer always be
857        // freed, even if an error occurs. All paths from here must call
858        // `device.pending_writes.consume`.
859        let (staging_buffer, staging_buffer_ptr) =
860            prepare_staging_buffer(device, stage_size, device.instance_flags)?;
861
862        let stage_fid = hub.staging_buffers.request();
863        let staging_buffer = stage_fid.init(staging_buffer);
864
865        if stage_bytes_per_row == bytes_per_row {
866            profiling::scope!("copy aligned");
867            // Fast path if the data is already being aligned optimally.
868            unsafe {
869                ptr::copy_nonoverlapping(
870                    data.as_ptr().offset(data_layout.offset as isize),
871                    staging_buffer_ptr,
872                    stage_size as usize,
873                );
874            }
875        } else {
876            profiling::scope!("copy chunked");
877            // Copy row by row into the optimal alignment.
878            let copy_bytes_per_row = stage_bytes_per_row.min(bytes_per_row) as usize;
879            for layer in 0..size.depth_or_array_layers {
880                let rows_offset = layer * block_rows_per_image;
881                for row in 0..height_blocks {
882                    unsafe {
883                        ptr::copy_nonoverlapping(
884                            data.as_ptr().offset(
885                                data_layout.offset as isize
886                                    + (rows_offset + row) as isize * bytes_per_row as isize,
887                            ),
888                            staging_buffer_ptr.offset(
889                                (rows_offset + row) as isize * stage_bytes_per_row as isize,
890                            ),
891                            copy_bytes_per_row,
892                        );
893                    }
894                }
895            }
896        }
897
898        if let Err(e) = unsafe { staging_buffer.flush(device.raw()) } {
899            pending_writes.consume(staging_buffer);
900            return Err(e.into());
901        }
902
903        let regions = (0..array_layer_count).map(|rel_array_layer| {
904            let mut texture_base = dst_base.clone();
905            texture_base.array_layer += rel_array_layer;
906            hal::BufferTextureCopy {
907                buffer_layout: wgt::ImageDataLayout {
908                    offset: rel_array_layer as u64
909                        * block_rows_per_image as u64
910                        * stage_bytes_per_row as u64,
911                    bytes_per_row: Some(stage_bytes_per_row),
912                    rows_per_image: Some(block_rows_per_image),
913                },
914                texture_base,
915                size: hal_copy_size,
916            }
917        });
918
919        {
920            let inner_buffer = staging_buffer.raw.lock();
921            let barrier = hal::BufferBarrier {
922                buffer: inner_buffer.as_ref().unwrap(),
923                usage: hal::BufferUses::MAP_WRITE..hal::BufferUses::COPY_SRC,
924            };
925
926            let mut trackers = device.trackers.lock();
927            let transition = trackers
928                .textures
929                .set_single(&dst, selector, hal::TextureUses::COPY_DST)
930                .ok_or(TransferError::InvalidTexture(destination.texture))?;
931            unsafe {
932                encoder.transition_textures(transition.map(|pending| pending.into_hal(dst_raw)));
933                encoder.transition_buffers(iter::once(barrier));
934                encoder.copy_buffer_to_texture(inner_buffer.as_ref().unwrap(), dst_raw, regions);
935            }
936        }
937
938        pending_writes.consume(staging_buffer);
939        pending_writes
940            .dst_textures
941            .insert(destination.texture, dst.clone());
942
943        Ok(())
944    }
945
946    #[cfg(webgl)]
947    pub fn queue_copy_external_image_to_texture<A: HalApi>(
948        &self,
949        queue_id: QueueId,
950        source: &wgt::ImageCopyExternalImage,
951        destination: crate::command::ImageCopyTextureTagged,
952        size: wgt::Extent3d,
953    ) -> Result<(), QueueWriteError> {
954        profiling::scope!("Queue::copy_external_image_to_texture");
955
956        let hub = A::hub(self);
957
958        let queue = hub
959            .queues
960            .get(queue_id)
961            .map_err(|_| DeviceError::InvalidQueueId)?;
962
963        let device = queue.device.as_ref().unwrap();
964
965        if size.width == 0 || size.height == 0 || size.depth_or_array_layers == 0 {
966            log::trace!("Ignoring write_texture of size 0");
967            return Ok(());
968        }
969
970        let mut needs_flag = false;
971        needs_flag |= matches!(source.source, wgt::ExternalImageSource::OffscreenCanvas(_));
972        needs_flag |= source.origin != wgt::Origin2d::ZERO;
973        needs_flag |= destination.color_space != wgt::PredefinedColorSpace::Srgb;
974        #[allow(clippy::bool_comparison)]
975        if matches!(source.source, wgt::ExternalImageSource::ImageBitmap(_)) {
976            needs_flag |= source.flip_y != false;
977            needs_flag |= destination.premultiplied_alpha != false;
978        }
979
980        if needs_flag {
981            device
982                .require_downlevel_flags(wgt::DownlevelFlags::UNRESTRICTED_EXTERNAL_TEXTURE_COPIES)
983                .map_err(TransferError::from)?;
984        }
985
986        let src_width = source.source.width();
987        let src_height = source.source.height();
988
989        let dst = hub.textures.get(destination.texture).unwrap();
990
991        if !conv::is_valid_external_image_copy_dst_texture_format(dst.desc.format) {
992            return Err(
993                TransferError::ExternalCopyToForbiddenTextureFormat(dst.desc.format).into(),
994            );
995        }
996        if dst.desc.dimension != wgt::TextureDimension::D2 {
997            return Err(TransferError::InvalidDimensionExternal(destination.texture).into());
998        }
999        if !dst.desc.usage.contains(wgt::TextureUsages::COPY_DST) {
1000            return Err(
1001                TransferError::MissingCopyDstUsageFlag(None, Some(destination.texture)).into(),
1002            );
1003        }
1004        if !dst
1005            .desc
1006            .usage
1007            .contains(wgt::TextureUsages::RENDER_ATTACHMENT)
1008        {
1009            return Err(
1010                TransferError::MissingRenderAttachmentUsageFlag(destination.texture).into(),
1011            );
1012        }
1013        if dst.desc.sample_count != 1 {
1014            return Err(TransferError::InvalidSampleCount {
1015                sample_count: dst.desc.sample_count,
1016            }
1017            .into());
1018        }
1019
1020        if source.origin.x + size.width > src_width {
1021            return Err(TransferError::TextureOverrun {
1022                start_offset: source.origin.x,
1023                end_offset: source.origin.x + size.width,
1024                texture_size: src_width,
1025                dimension: crate::resource::TextureErrorDimension::X,
1026                side: CopySide::Source,
1027            }
1028            .into());
1029        }
1030        if source.origin.y + size.height > src_height {
1031            return Err(TransferError::TextureOverrun {
1032                start_offset: source.origin.y,
1033                end_offset: source.origin.y + size.height,
1034                texture_size: src_height,
1035                dimension: crate::resource::TextureErrorDimension::Y,
1036                side: CopySide::Source,
1037            }
1038            .into());
1039        }
1040        if size.depth_or_array_layers != 1 {
1041            return Err(TransferError::TextureOverrun {
1042                start_offset: 0,
1043                end_offset: size.depth_or_array_layers,
1044                texture_size: 1,
1045                dimension: crate::resource::TextureErrorDimension::Z,
1046                side: CopySide::Source,
1047            }
1048            .into());
1049        }
1050
1051        // Note: Doing the copy range validation early is important because ensures that the
1052        // dimensions are not going to cause overflow in other parts of the validation.
1053        let (hal_copy_size, _) = validate_texture_copy_range(
1054            &destination.to_untagged(),
1055            &dst.desc,
1056            CopySide::Destination,
1057            &size,
1058        )?;
1059
1060        let (selector, dst_base) =
1061            extract_texture_selector(&destination.to_untagged(), &size, &dst)?;
1062
1063        let mut pending_writes = device.pending_writes.lock();
1064        let encoder = pending_writes.as_mut().unwrap().activate();
1065
1066        // If the copy does not fully cover the layers, we need to initialize to
1067        // zero *first* as we don't keep track of partial texture layer inits.
1068        //
1069        // Strictly speaking we only need to clear the areas of a layer
1070        // untouched, but this would get increasingly messy.
1071        let init_layer_range = if dst.desc.dimension == wgt::TextureDimension::D3 {
1072            // volume textures don't have a layer range as array volumes aren't supported
1073            0..1
1074        } else {
1075            destination.origin.z..destination.origin.z + size.depth_or_array_layers
1076        };
1077        let mut dst_initialization_status = dst.initialization_status.write();
1078        if dst_initialization_status.mips[destination.mip_level as usize]
1079            .check(init_layer_range.clone())
1080            .is_some()
1081        {
1082            if has_copy_partial_init_tracker_coverage(&size, destination.mip_level, &dst.desc) {
1083                for layer_range in dst_initialization_status.mips[destination.mip_level as usize]
1084                    .drain(init_layer_range)
1085                    .collect::<Vec<std::ops::Range<u32>>>()
1086                {
1087                    let mut trackers = device.trackers.lock();
1088                    crate::command::clear_texture(
1089                        &dst,
1090                        TextureInitRange {
1091                            mip_range: destination.mip_level..(destination.mip_level + 1),
1092                            layer_range,
1093                        },
1094                        encoder,
1095                        &mut trackers.textures,
1096                        &device.alignments,
1097                        device.zero_buffer.as_ref().unwrap(),
1098                        &device.snatchable_lock.read(),
1099                    )
1100                    .map_err(QueueWriteError::from)?;
1101                }
1102            } else {
1103                dst_initialization_status.mips[destination.mip_level as usize]
1104                    .drain(init_layer_range);
1105            }
1106        }
1107        dst.info
1108            .use_at(device.active_submission_index.load(Ordering::Relaxed) + 1);
1109
1110        let snatch_guard = device.snatchable_lock.read();
1111        let dst_raw = dst
1112            .raw(&snatch_guard)
1113            .ok_or(TransferError::InvalidTexture(destination.texture))?;
1114
1115        let regions = hal::TextureCopy {
1116            src_base: hal::TextureCopyBase {
1117                mip_level: 0,
1118                array_layer: 0,
1119                origin: source.origin.to_3d(0),
1120                aspect: hal::FormatAspects::COLOR,
1121            },
1122            dst_base,
1123            size: hal_copy_size,
1124        };
1125
1126        unsafe {
1127            let mut trackers = device.trackers.lock();
1128            let transitions = trackers
1129                .textures
1130                .set_single(&dst, selector, hal::TextureUses::COPY_DST)
1131                .ok_or(TransferError::InvalidTexture(destination.texture))?;
1132            encoder.transition_textures(transitions.map(|pending| pending.into_hal(dst_raw)));
1133            encoder.copy_external_image_to_texture(
1134                source,
1135                dst_raw,
1136                destination.premultiplied_alpha,
1137                iter::once(regions),
1138            );
1139        }
1140
1141        Ok(())
1142    }
1143
1144    pub fn queue_submit<A: HalApi>(
1145        &self,
1146        queue_id: QueueId,
1147        command_buffer_ids: &[id::CommandBufferId],
1148    ) -> Result<WrappedSubmissionIndex, QueueSubmitError> {
1149        profiling::scope!("Queue::submit");
1150        api_log!("Queue::submit {queue_id:?}");
1151
1152        let (submit_index, callbacks) = {
1153            let hub = A::hub(self);
1154
1155            let queue = hub
1156                .queues
1157                .get(queue_id)
1158                .map_err(|_| DeviceError::InvalidQueueId)?;
1159
1160            let device = queue.device.as_ref().unwrap();
1161
1162            let snatch_guard = device.snatchable_lock.read();
1163
1164            // Fence lock must be acquired after the snatch lock everywhere to avoid deadlocks.
1165            let mut fence = device.fence.write();
1166            let fence = fence.as_mut().unwrap();
1167            let submit_index = device
1168                .active_submission_index
1169                .fetch_add(1, Ordering::Relaxed)
1170                + 1;
1171            let mut active_executions = Vec::new();
1172
1173            let mut used_surface_textures = track::TextureUsageScope::default();
1174
1175            // Use a hashmap here to deduplicate the surface textures that are used in the command buffers.
1176            // This avoids vulkan deadlocking from the same surface texture being submitted multiple times.
1177            let mut submit_surface_textures_owned = FastHashMap::default();
1178
1179            {
1180                let mut command_buffer_guard = hub.command_buffers.write();
1181
1182                if !command_buffer_ids.is_empty() {
1183                    profiling::scope!("prepare");
1184
1185                    //TODO: if multiple command buffers are submitted, we can re-use the last
1186                    // native command buffer of the previous chain instead of always creating
1187                    // a temporary one, since the chains are not finished.
1188                    let mut temp_suspected = device.temp_suspected.lock();
1189                    {
1190                        let mut suspected = temp_suspected.replace(ResourceMaps::new()).unwrap();
1191                        suspected.clear();
1192                    }
1193
1194                    // finish all the command buffers first
1195                    for &cmb_id in command_buffer_ids {
1196                        // we reset the used surface textures every time we use
1197                        // it, so make sure to set_size on it.
1198                        used_surface_textures.set_size(device.tracker_indices.textures.size());
1199
1200                        #[allow(unused_mut)]
1201                        let mut cmdbuf = match command_buffer_guard.replace_with_error(cmb_id) {
1202                            Ok(cmdbuf) => cmdbuf,
1203                            Err(_) => continue,
1204                        };
1205
1206                        if cmdbuf.device.as_info().id().into_queue_id() != queue_id {
1207                            return Err(DeviceError::WrongDevice.into());
1208                        }
1209
1210                        #[cfg(feature = "trace")]
1211                        if let Some(ref mut trace) = *device.trace.lock() {
1212                            trace.add(Action::Submit(
1213                                submit_index,
1214                                cmdbuf
1215                                    .data
1216                                    .lock()
1217                                    .as_mut()
1218                                    .unwrap()
1219                                    .commands
1220                                    .take()
1221                                    .unwrap(),
1222                            ));
1223                        }
1224                        if !cmdbuf.is_finished() {
1225                            let cmdbuf = Arc::into_inner(cmdbuf).expect(
1226                                "Command buffer cannot be destroyed because is still in use",
1227                            );
1228                            device.destroy_command_buffer(cmdbuf);
1229                            continue;
1230                        }
1231
1232                        // optimize the tracked states
1233                        // cmdbuf.trackers.optimize();
1234                        {
1235                            let cmd_buf_data = cmdbuf.data.lock();
1236                            let cmd_buf_trackers = &cmd_buf_data.as_ref().unwrap().trackers;
1237
1238                            // update submission IDs
1239                            for buffer in cmd_buf_trackers.buffers.used_resources() {
1240                                let tracker_index = buffer.info.tracker_index();
1241                                let raw_buf = match buffer.raw.get(&snatch_guard) {
1242                                    Some(raw) => raw,
1243                                    None => {
1244                                        return Err(QueueSubmitError::DestroyedBuffer(
1245                                            buffer.info.id(),
1246                                        ));
1247                                    }
1248                                };
1249                                buffer.info.use_at(submit_index);
1250                                if buffer.is_unique() {
1251                                    if let BufferMapState::Active { .. } = *buffer.map_state.lock()
1252                                    {
1253                                        log::warn!("Dropped buffer has a pending mapping.");
1254                                        unsafe { device.raw().unmap_buffer(raw_buf) }
1255                                            .map_err(DeviceError::from)?;
1256                                    }
1257                                    temp_suspected
1258                                        .as_mut()
1259                                        .unwrap()
1260                                        .buffers
1261                                        .insert(tracker_index, buffer.clone());
1262                                } else {
1263                                    match *buffer.map_state.lock() {
1264                                        BufferMapState::Idle => (),
1265                                        _ => {
1266                                            return Err(QueueSubmitError::BufferStillMapped(
1267                                                buffer.info.id(),
1268                                            ))
1269                                        }
1270                                    }
1271                                }
1272                            }
1273                            for texture in cmd_buf_trackers.textures.used_resources() {
1274                                let tracker_index = texture.info.tracker_index();
1275                                let should_extend = match texture.inner.get(&snatch_guard) {
1276                                    None => {
1277                                        return Err(QueueSubmitError::DestroyedTexture(
1278                                            texture.info.id(),
1279                                        ));
1280                                    }
1281                                    Some(TextureInner::Native { .. }) => false,
1282                                    Some(TextureInner::Surface { ref raw, .. }) => {
1283                                        if raw.is_some() {
1284                                            // Compare the Arcs by pointer as Textures don't implement Eq.
1285                                            submit_surface_textures_owned
1286                                                .insert(Arc::as_ptr(&texture), texture.clone());
1287                                        }
1288
1289                                        true
1290                                    }
1291                                };
1292                                texture.info.use_at(submit_index);
1293                                if texture.is_unique() {
1294                                    temp_suspected
1295                                        .as_mut()
1296                                        .unwrap()
1297                                        .textures
1298                                        .insert(tracker_index, texture.clone());
1299                                }
1300                                if should_extend {
1301                                    unsafe {
1302                                        used_surface_textures
1303                                            .merge_single(&texture, None, hal::TextureUses::PRESENT)
1304                                            .unwrap();
1305                                    };
1306                                }
1307                            }
1308                            for texture_view in cmd_buf_trackers.views.used_resources() {
1309                                texture_view.info.use_at(submit_index);
1310                                if texture_view.is_unique() {
1311                                    temp_suspected.as_mut().unwrap().texture_views.insert(
1312                                        texture_view.as_info().tracker_index(),
1313                                        texture_view.clone(),
1314                                    );
1315                                }
1316                            }
1317                            {
1318                                for bg in cmd_buf_trackers.bind_groups.used_resources() {
1319                                    bg.info.use_at(submit_index);
1320                                    // We need to update the submission indices for the contained
1321                                    // state-less (!) resources as well, so that they don't get
1322                                    // deleted too early if the parent bind group goes out of scope.
1323                                    for view in bg.used.views.used_resources() {
1324                                        view.info.use_at(submit_index);
1325                                    }
1326                                    for sampler in bg.used.samplers.used_resources() {
1327                                        sampler.info.use_at(submit_index);
1328                                    }
1329                                    if bg.is_unique() {
1330                                        temp_suspected
1331                                            .as_mut()
1332                                            .unwrap()
1333                                            .bind_groups
1334                                            .insert(bg.as_info().tracker_index(), bg.clone());
1335                                    }
1336                                }
1337                            }
1338                            // assert!(cmd_buf_trackers.samplers.is_empty());
1339                            for compute_pipeline in
1340                                cmd_buf_trackers.compute_pipelines.used_resources()
1341                            {
1342                                compute_pipeline.info.use_at(submit_index);
1343                                if compute_pipeline.is_unique() {
1344                                    temp_suspected.as_mut().unwrap().compute_pipelines.insert(
1345                                        compute_pipeline.as_info().tracker_index(),
1346                                        compute_pipeline.clone(),
1347                                    );
1348                                }
1349                            }
1350                            for render_pipeline in
1351                                cmd_buf_trackers.render_pipelines.used_resources()
1352                            {
1353                                render_pipeline.info.use_at(submit_index);
1354                                if render_pipeline.is_unique() {
1355                                    temp_suspected.as_mut().unwrap().render_pipelines.insert(
1356                                        render_pipeline.as_info().tracker_index(),
1357                                        render_pipeline.clone(),
1358                                    );
1359                                }
1360                            }
1361                            for query_set in cmd_buf_trackers.query_sets.used_resources() {
1362                                query_set.info.use_at(submit_index);
1363                                if query_set.is_unique() {
1364                                    temp_suspected.as_mut().unwrap().query_sets.insert(
1365                                        query_set.as_info().tracker_index(),
1366                                        query_set.clone(),
1367                                    );
1368                                }
1369                            }
1370                            for bundle in cmd_buf_trackers.bundles.used_resources() {
1371                                bundle.info.use_at(submit_index);
1372                                // We need to update the submission indices for the contained
1373                                // state-less (!) resources as well, excluding the bind groups.
1374                                // They don't get deleted too early if the bundle goes out of scope.
1375                                for render_pipeline in
1376                                    bundle.used.render_pipelines.read().used_resources()
1377                                {
1378                                    render_pipeline.info.use_at(submit_index);
1379                                }
1380                                for query_set in bundle.used.query_sets.read().used_resources() {
1381                                    query_set.info.use_at(submit_index);
1382                                }
1383                                if bundle.is_unique() {
1384                                    temp_suspected
1385                                        .as_mut()
1386                                        .unwrap()
1387                                        .render_bundles
1388                                        .insert(bundle.as_info().tracker_index(), bundle.clone());
1389                                }
1390                            }
1391                        }
1392                        let mut baked = cmdbuf.from_arc_into_baked();
1393                        // execute resource transitions
1394                        unsafe {
1395                            baked
1396                                .encoder
1397                                .begin_encoding(hal_label(
1398                                    Some("(wgpu internal) Transit"),
1399                                    device.instance_flags,
1400                                ))
1401                                .map_err(DeviceError::from)?
1402                        };
1403                        log::trace!("Stitching command buffer {:?} before submission", cmb_id);
1404
1405                        //Note: locking the trackers has to be done after the storages
1406                        let mut trackers = device.trackers.lock();
1407                        baked
1408                            .initialize_buffer_memory(&mut *trackers, &snatch_guard)
1409                            .map_err(|err| QueueSubmitError::DestroyedBuffer(err.0))?;
1410                        baked
1411                            .initialize_texture_memory(&mut *trackers, device, &snatch_guard)
1412                            .map_err(|err| QueueSubmitError::DestroyedTexture(err.0))?;
1413                        //Note: stateless trackers are not merged:
1414                        // device already knows these resources exist.
1415                        CommandBuffer::insert_barriers_from_tracker(
1416                            &mut baked.encoder,
1417                            &mut *trackers,
1418                            &baked.trackers,
1419                            &snatch_guard,
1420                        );
1421
1422                        let transit = unsafe { baked.encoder.end_encoding().unwrap() };
1423                        baked.list.insert(0, transit);
1424
1425                        // Transition surface textures into `Present` state.
1426                        // Note: we could technically do it after all of the command buffers,
1427                        // but here we have a command encoder by hand, so it's easier to use it.
1428                        if !used_surface_textures.is_empty() {
1429                            unsafe {
1430                                baked
1431                                    .encoder
1432                                    .begin_encoding(hal_label(
1433                                        Some("(wgpu internal) Present"),
1434                                        device.instance_flags,
1435                                    ))
1436                                    .map_err(DeviceError::from)?
1437                            };
1438                            trackers
1439                                .textures
1440                                .set_from_usage_scope(&used_surface_textures);
1441                            let (transitions, textures) =
1442                                trackers.textures.drain_transitions(&snatch_guard);
1443                            let texture_barriers = transitions
1444                                .into_iter()
1445                                .enumerate()
1446                                .map(|(i, p)| p.into_hal(textures[i].unwrap().raw().unwrap()));
1447                            let present = unsafe {
1448                                baked.encoder.transition_textures(texture_barriers);
1449                                baked.encoder.end_encoding().unwrap()
1450                            };
1451                            baked.list.push(present);
1452                            used_surface_textures = track::TextureUsageScope::default();
1453                        }
1454
1455                        // done
1456                        active_executions.push(EncoderInFlight {
1457                            raw: baked.encoder,
1458                            cmd_buffers: baked.list,
1459                        });
1460                    }
1461
1462                    log::trace!("Device after submission {}", submit_index);
1463                }
1464            }
1465
1466            let mut pending_writes = device.pending_writes.lock();
1467            let pending_writes = pending_writes.as_mut().unwrap();
1468
1469            {
1470                used_surface_textures.set_size(hub.textures.read().len());
1471                for (&id, texture) in pending_writes.dst_textures.iter() {
1472                    match texture.inner.get(&snatch_guard) {
1473                        None => {
1474                            return Err(QueueSubmitError::DestroyedTexture(id));
1475                        }
1476                        Some(TextureInner::Native { .. }) => {}
1477                        Some(TextureInner::Surface { ref raw, .. }) => {
1478                            if raw.is_some() {
1479                                // Compare the Arcs by pointer as Textures don't implement Eq
1480                                submit_surface_textures_owned
1481                                    .insert(Arc::as_ptr(texture), texture.clone());
1482                            }
1483
1484                            unsafe {
1485                                used_surface_textures
1486                                    .merge_single(texture, None, hal::TextureUses::PRESENT)
1487                                    .unwrap()
1488                            };
1489                        }
1490                    }
1491                }
1492
1493                if !used_surface_textures.is_empty() {
1494                    let mut trackers = device.trackers.lock();
1495
1496                    trackers
1497                        .textures
1498                        .set_from_usage_scope(&used_surface_textures);
1499                    let (transitions, textures) =
1500                        trackers.textures.drain_transitions(&snatch_guard);
1501                    let texture_barriers = transitions
1502                        .into_iter()
1503                        .enumerate()
1504                        .map(|(i, p)| p.into_hal(textures[i].unwrap().raw().unwrap()));
1505                    unsafe {
1506                        pending_writes
1507                            .command_encoder
1508                            .transition_textures(texture_barriers);
1509                    };
1510                }
1511            }
1512
1513            let refs = pending_writes
1514                .pre_submit()?
1515                .into_iter()
1516                .chain(
1517                    active_executions
1518                        .iter()
1519                        .flat_map(|pool_execution| pool_execution.cmd_buffers.iter()),
1520                )
1521                .collect::<Vec<_>>();
1522
1523            let mut submit_surface_textures =
1524                SmallVec::<[_; 2]>::with_capacity(submit_surface_textures_owned.len());
1525
1526            for texture in submit_surface_textures_owned.values() {
1527                submit_surface_textures.extend(match texture.inner.get(&snatch_guard) {
1528                    Some(TextureInner::Surface { raw, .. }) => raw.as_ref(),
1529                    _ => None,
1530                });
1531            }
1532
1533            unsafe {
1534                queue
1535                    .raw
1536                    .as_ref()
1537                    .unwrap()
1538                    .submit(&refs, &submit_surface_textures, (fence, submit_index))
1539                    .map_err(DeviceError::from)?;
1540            }
1541
1542            profiling::scope!("cleanup");
1543            if let Some(pending_execution) = pending_writes.post_submit(
1544                &device.command_allocator,
1545                device.raw(),
1546                queue.raw.as_ref().unwrap(),
1547            ) {
1548                active_executions.push(pending_execution);
1549            }
1550
1551            // this will register the new submission to the life time tracker
1552            let mut pending_write_resources = mem::take(&mut pending_writes.temp_resources);
1553            device.lock_life().track_submission(
1554                submit_index,
1555                pending_write_resources.drain(..),
1556                active_executions,
1557            );
1558
1559            // This will schedule destruction of all resources that are no longer needed
1560            // by the user but used in the command stream, among other things.
1561            let (closures, _) = match device.maintain(fence, wgt::Maintain::Poll, snatch_guard) {
1562                Ok(closures) => closures,
1563                Err(WaitIdleError::Device(err)) => return Err(QueueSubmitError::Queue(err)),
1564                Err(WaitIdleError::StuckGpu) => return Err(QueueSubmitError::StuckGpu),
1565                Err(WaitIdleError::WrongSubmissionIndex(..)) => unreachable!(),
1566            };
1567
1568            // pending_write_resources has been drained, so it's empty, but we
1569            // want to retain its heap allocation.
1570            pending_writes.temp_resources = pending_write_resources;
1571            device.lock_life().post_submit();
1572
1573            (submit_index, closures)
1574        };
1575
1576        // the closures should execute with nothing locked!
1577        callbacks.fire();
1578
1579        Ok(WrappedSubmissionIndex {
1580            queue_id,
1581            index: submit_index,
1582        })
1583    }
1584
1585    pub fn queue_get_timestamp_period<A: HalApi>(
1586        &self,
1587        queue_id: QueueId,
1588    ) -> Result<f32, InvalidQueue> {
1589        let hub = A::hub(self);
1590        match hub.queues.get(queue_id) {
1591            Ok(queue) => Ok(unsafe { queue.raw.as_ref().unwrap().get_timestamp_period() }),
1592            Err(_) => Err(InvalidQueue),
1593        }
1594    }
1595
1596    pub fn queue_on_submitted_work_done<A: HalApi>(
1597        &self,
1598        queue_id: QueueId,
1599        closure: SubmittedWorkDoneClosure,
1600    ) -> Result<(), InvalidQueue> {
1601        api_log!("Queue::on_submitted_work_done {queue_id:?}");
1602
1603        //TODO: flush pending writes
1604        let hub = A::hub(self);
1605        match hub.queues.get(queue_id) {
1606            Ok(queue) => queue
1607                .device
1608                .as_ref()
1609                .unwrap()
1610                .lock_life()
1611                .add_work_done_closure(closure),
1612            Err(_) => return Err(InvalidQueue),
1613        }
1614        Ok(())
1615    }
1616}