From 73eb83ded1287d2c46ccc84c36ee521ac8f85b43 Mon Sep 17 00:00:00 2001 From: Vecvec Date: Thu, 12 Jun 2025 13:40:10 +1200 Subject: [PATCH] [wgpu] Blas compaction (#7285) Co-authored-by: Connor Fitzgerald --- CHANGELOG.md | 1 + docs/api-specs/ray_tracing.md | 50 +++++ .../standalone/custom_backend/src/custom.rs | 10 +- tests/tests/wgpu-gpu/ray_tracing/as_build.rs | 154 +++++++++++++++ wgpu-core/src/command/ray_tracing.rs | 70 ++++++- wgpu-core/src/device/life.rs | 64 ++++++- wgpu-core/src/device/mod.rs | 8 + wgpu-core/src/device/queue.rs | 175 ++++++++++++++++-- wgpu-core/src/device/ray_tracing.rs | 82 +++++++- wgpu-core/src/device/resource.rs | 1 + wgpu-core/src/lock/rank.rs | 1 + wgpu-core/src/ray_tracing.rs | 47 ++++- wgpu-core/src/resource.rs | 155 +++++++++++++++- wgpu-core/src/track/blas.rs | 59 ++++++ wgpu-core/src/track/mod.rs | 6 +- wgpu-types/src/lib.rs | 4 +- wgpu/src/api/blas.rs | 48 ++++- wgpu/src/api/queue.rs | 13 ++ wgpu/src/backend/webgpu.rs | 21 ++- wgpu/src/backend/wgpu_core.rs | 66 ++++++- wgpu/src/dispatch.rs | 12 +- 21 files changed, 1002 insertions(+), 45 deletions(-) create mode 100644 wgpu-core/src/track/blas.rs diff --git a/CHANGELOG.md b/CHANGELOG.md index c997bd28a..f878251d0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -94,6 +94,7 @@ Naga now infers the correct binding layout when a resource appears only in an as #### General +- Support BLAS compaction in wgpu. By @Vecvec in [#7285](https://github.com/gfx-rs/wgpu/pull/7285). - Removed `MaintainBase` in favor of using `PollType`. By @waywardmonkeys in [#7508](https://github.com/gfx-rs/wgpu/pull/7508). - The `destroy` functions for buffers and textures in wgpu-core are now infallible. Previously, they returned an error if called multiple times for the same object. This only affects the wgpu-core API; the wgpu API already allowed multiple `destroy` calls. By @andyleiserson in [#7686](https://github.com/gfx-rs/wgpu/pull/7686) and [#7720](https://github.com/gfx-rs/wgpu/pull/7720). diff --git a/docs/api-specs/ray_tracing.md b/docs/api-specs/ray_tracing.md index 9d56cf1a6..915fd1c18 100644 --- a/docs/api-specs/ray_tracing.md +++ b/docs/api-specs/ray_tracing.md @@ -34,6 +34,53 @@ Before a [`Tlas`] is used in a shader it must - have all [`Blas`]es that it was last built with to have last been built in either the same build as this [`Tlas`] or an earlier build call. +### [`Blas`] compaction + +Once a [`Blas`] has been built, it can be compacted. Acceleration structures are allocated conservatively, without +knowing the exact data that is inside them. Once a [`Blas`] has been built, the driver can make data specific +optimisations to make the [`BLAS`] smaller. To begin compaction call [`Blas::prepare_compaction_async`] on it. This +method waits until all builds operating on the [`Blas`] are finished, prepares the [`Blas`] to be compacted, and runs +the given callback. To check whether the [`Blas`] is ready, you can also call [`Blas::ready_for_compaction`] instead of +waiting for the callback (useful if you are asynchronously compacting a large number of [`Blas`]es). Submitting a +rebuild of a [`Blas`] terminates any [`Blas::prepare_compaction_async`], preventing the callback from being called, and +making the [`Blas`] no longer ready to compact. Once a [`Blas`] is ready for compaction, it can be compacted using +[`Queue::compact_blas`] this returns the new compacted [`Blas`], which is independent of the [`Blas`] passed in. The +other [`Blas`] can be used for other things, including being rebuilt without affecting the new [`Blas`]. The returned +[`Blas`] behaves largely like the [`Blas`] it was created from, except that it can be neither rebuilt, nor compacted +again. + +An example of compaction being run when [`Blas`]es are ready, this would be in a situation when memory was not a major +problem, otherwise (e.g. if you get an out of memory error) you should compact immediately (and switching all +non-compacted [`Blas`]es to compacted ones). +```rust +use std::iter; +use wgpu::Blas; + +struct BlasToBeCompacted { + blas: Blas, + /// The index into the TlasInstance this BLAS is used in. + tlas_index: usize, +} + +fn render(/*whatever args you need to render*/) { + /* additional code to prepare the renderer */ + //An iterator of whatever BLASes you have called `prepare_compaction_async` on. + let blas_s_pending_compaction: impl Iterator = iter::empty(); + for blas_to_be_compacted in blas_s_pending_compaction { + if blas_to_be_compacted.blas.ready_for_compaction() { + let compacted_blas = queue.compact_blas(&blas_to_be_compacted.blas); + tlas_instance[blas_to_be_compacted.tlas_index].set_blas(&compacted_blas); + } + } + let mut encoder = + device.create_command_encoder(&wgpu::CommandEncoderDescriptor { label: None }); + /* do other preparations on th TlasInstance.*/ + encoder.build_acceleration_structures(iter::empty(), iter::once(&tlas_package)); + /* more render code */ + queue.submit(Some(encoder.finish())); +} +``` + [`Device::create_blas`]: https://wgpu.rs/doc/wgpu/struct.Device.html#method.create_blas [`Device::create_tlas`]: https://wgpu.rs/doc/wgpu/struct.Device.html#method.create_tlas [`Tlas`]: https://wgpu.rs/doc/wgpu/struct.Tlas.html @@ -41,6 +88,9 @@ Before a [`Tlas`] is used in a shader it must [`TlasInstance`]: https://wgpu.rs/doc/wgpu/struct.TlasInstance.html [`TlasPackage`]: https://wgpu.rs/doc/wgpu/struct.TlasPackage.html [`TlasPackage::tlas`]: https://wgpu.rs/doc/wgpu/struct.TlasPackage.html#method.tlas +[`Blas::prepare_compaction_async`]: https://wgpu.rs/doc/wgpu/struct.Blas.html#method.prepare_compaction_async +[`Blas::ready_for_compaction`]: https://wgpu.rs/doc/wgpu/struct.Blas.html#method.ready_for_compaction +[`Queue::compact_blas`]: https://wgpu.rs/doc/wgpu/struct.Queue.html#method.compact_blas ## `naga`'s raytracing API: diff --git a/examples/standalone/custom_backend/src/custom.rs b/examples/standalone/custom_backend/src/custom.rs index 40752b694..ec57413ec 100644 --- a/examples/standalone/custom_backend/src/custom.rs +++ b/examples/standalone/custom_backend/src/custom.rs @@ -3,9 +3,9 @@ use std::pin::Pin; use std::sync::Arc; use wgpu::custom::{ - AdapterInterface, ComputePipelineInterface, DeviceInterface, DispatchAdapter, DispatchDevice, - DispatchQueue, DispatchShaderModule, DispatchSurface, InstanceInterface, QueueInterface, - RequestAdapterFuture, ShaderModuleInterface, + AdapterInterface, ComputePipelineInterface, DeviceInterface, DispatchAdapter, DispatchBlas, + DispatchDevice, DispatchQueue, DispatchShaderModule, DispatchSurface, InstanceInterface, + QueueInterface, RequestAdapterFuture, ShaderModuleInterface, }; #[derive(Debug, Clone)] @@ -346,6 +346,10 @@ impl QueueInterface for CustomQueue { ) { unimplemented!() } + + fn compact_blas(&self, _blas: &DispatchBlas) -> (Option, DispatchBlas) { + unimplemented!() + } } #[derive(Debug)] diff --git a/tests/tests/wgpu-gpu/ray_tracing/as_build.rs b/tests/tests/wgpu-gpu/ray_tracing/as_build.rs index 2dc0e45f1..681c47624 100644 --- a/tests/tests/wgpu-gpu/ray_tracing/as_build.rs +++ b/tests/tests/wgpu-gpu/ray_tracing/as_build.rs @@ -41,6 +41,160 @@ fn unbuilt_blas(ctx: TestingContext) { ); } +#[gpu_test] +static UNBUILT_BLAS_COMPACTION: GpuTestConfiguration = GpuTestConfiguration::new() + .parameters( + TestParameters::default() + .test_features_limits() + .features(wgpu::Features::EXPERIMENTAL_RAY_TRACING_ACCELERATION_STRUCTURE) + // https://github.com/gfx-rs/wgpu/issues/6727 + .skip(FailureCase::backend_adapter(wgpu::Backends::VULKAN, "AMD")), + ) + .run_sync(unbuilt_blas_compaction); + +fn unbuilt_blas_compaction(ctx: TestingContext) { + let as_ctx = AsBuildContext::new( + &ctx, + AccelerationStructureFlags::ALLOW_COMPACTION, + AccelerationStructureFlags::empty(), + ); + + fail( + &ctx.device, + || { + // Prepare checks the BLAS has been built + as_ctx.blas.prepare_compaction_async(|_| {}) + }, + None, + ); +} + +#[gpu_test] +static BLAS_COMPACTION_WITHOUT_FLAGS: GpuTestConfiguration = GpuTestConfiguration::new() + .parameters( + TestParameters::default() + .test_features_limits() + .features(wgpu::Features::EXPERIMENTAL_RAY_TRACING_ACCELERATION_STRUCTURE) + // https://github.com/gfx-rs/wgpu/issues/6727 + .skip(FailureCase::backend_adapter(wgpu::Backends::VULKAN, "AMD")), + ) + .run_sync(blas_compaction_without_flags); + +fn blas_compaction_without_flags(ctx: TestingContext) { + let as_ctx = AsBuildContext::new( + &ctx, + AccelerationStructureFlags::empty(), + AccelerationStructureFlags::empty(), + ); + + let mut encoder = ctx + .device + .create_command_encoder(&CommandEncoderDescriptor::default()); + + encoder.build_acceleration_structures([&as_ctx.blas_build_entry()], []); + + ctx.queue.submit([encoder.finish()]); + + fail( + &ctx.device, + || { + // Prepare checks whether te BLAS is able to be compacted + as_ctx.blas.prepare_compaction_async(|_| {}) + }, + None, + ); +} + +#[gpu_test] +static UNPREPARED_BLAS_COMPACTION: GpuTestConfiguration = GpuTestConfiguration::new() + .parameters( + TestParameters::default() + .test_features_limits() + .features(wgpu::Features::EXPERIMENTAL_RAY_TRACING_ACCELERATION_STRUCTURE) + // https://github.com/gfx-rs/wgpu/issues/6727 + .skip(FailureCase::backend_adapter(wgpu::Backends::VULKAN, "AMD")), + ) + .run_sync(unprepared_blas_compaction); + +fn unprepared_blas_compaction(ctx: TestingContext) { + let as_ctx = AsBuildContext::new( + &ctx, + AccelerationStructureFlags::ALLOW_COMPACTION, + AccelerationStructureFlags::empty(), + ); + + let mut encoder = ctx + .device + .create_command_encoder(&CommandEncoderDescriptor::default()); + + encoder.build_acceleration_structures([&as_ctx.blas_build_entry()], []); + + ctx.queue.submit([encoder.finish()]); + + fail(&ctx.device, || ctx.queue.compact_blas(&as_ctx.blas), None); +} + +#[gpu_test] +static BLAS_COMPACTION: GpuTestConfiguration = GpuTestConfiguration::new() + .parameters( + TestParameters::default() + .test_features_limits() + .features(wgpu::Features::EXPERIMENTAL_RAY_TRACING_ACCELERATION_STRUCTURE) + // https://github.com/gfx-rs/wgpu/issues/6727 + .skip(FailureCase::backend_adapter(wgpu::Backends::VULKAN, "AMD")), + ) + .run_sync(blas_compaction); + +fn blas_compaction(ctx: TestingContext) { + let as_ctx = AsBuildContext::new( + &ctx, + AccelerationStructureFlags::ALLOW_COMPACTION, + AccelerationStructureFlags::empty(), + ); + + let mut encoder = ctx + .device + .create_command_encoder(&CommandEncoderDescriptor::default()); + + // Build the BLAS to be compacted (so compaction is valid). + encoder.build_acceleration_structures([&as_ctx.blas_build_entry()], []); + + ctx.queue.submit([encoder.finish()]); + + // Prepare the BLAS to be compacted. + let (send, recv) = std::sync::mpsc::channel(); + as_ctx.blas.prepare_compaction_async(move |res| { + res.unwrap(); + send.send(()).unwrap(); + }); + + // On native this will trigger the callback. + ctx.device.poll(PollType::Wait).unwrap(); + // Check that the callback actually gets called (this test will timeout if it doesn't). + recv.recv().unwrap(); + // This should return true because the callback has been called, and we haven't rebuilt the BLAS + assert!(as_ctx.blas.ready_for_compaction()); + + let compacted = ctx.queue.compact_blas(&as_ctx.blas); + + // This actually executes the compact call. + ctx.queue.submit([]); + + let mut fail_encoder = ctx + .device + .create_command_encoder(&CommandEncoderDescriptor::default()); + + // Try to build the compacted BLAS, this should fail. + let mut build_entry = as_ctx.blas_build_entry(); + build_entry.blas = &compacted; + + fail( + &ctx.device, + || fail_encoder.build_acceleration_structures([&build_entry], []), + None, + ); +} + #[gpu_test] static OUT_OF_ORDER_AS_BUILD: GpuTestConfiguration = GpuTestConfiguration::new() .parameters( diff --git a/wgpu-core/src/command/ray_tracing.rs b/wgpu-core/src/command/ray_tracing.rs index 5c84f5b4c..ffe588a27 100644 --- a/wgpu-core/src/command/ray_tracing.rs +++ b/wgpu-core/src/command/ray_tracing.rs @@ -22,7 +22,9 @@ use crate::{ TlasBuildEntry, TlasInstance, TlasPackage, TraceBlasBuildEntry, TraceBlasGeometries, TraceBlasTriangleGeometry, TraceTlasInstance, TraceTlasPackage, }, - resource::{AccelerationStructure, Blas, Buffer, Labeled, StagingBuffer, Tlas}, + resource::{ + AccelerationStructure, Blas, BlasCompactState, Buffer, Labeled, StagingBuffer, Tlas, + }, scratch::ScratchBuffer, snatch::SnatchGuard, track::PendingTransition, @@ -335,10 +337,16 @@ impl Global { let cmd_buf_raw = cmd_buf_data.encoder.open()?; + let mut blas_s_compactable = Vec::new(); let mut descriptors = Vec::new(); for storage in &blas_storage { - descriptors.push(map_blas(storage, scratch_buffer.raw(), &snatch_guard)?); + descriptors.push(map_blas( + storage, + scratch_buffer.raw(), + &snatch_guard, + &mut blas_s_compactable, + )?); } build_blas( @@ -348,6 +356,7 @@ impl Global { input_barriers, &descriptors, scratch_buffer_barrier, + blas_s_compactable, ); if tlas_present { @@ -664,10 +673,16 @@ impl Global { let cmd_buf_raw = cmd_buf_data.encoder.open()?; + let mut blas_s_compactable = Vec::new(); let mut descriptors = Vec::new(); for storage in &blas_storage { - descriptors.push(map_blas(storage, scratch_buffer.raw(), &snatch_guard)?); + descriptors.push(map_blas( + storage, + scratch_buffer.raw(), + &snatch_guard, + &mut blas_s_compactable, + )?); } build_blas( @@ -677,6 +692,7 @@ impl Global { input_barriers, &descriptors, scratch_buffer_barrier, + blas_s_compactable, ); if tlas_present { @@ -794,6 +810,15 @@ impl CommandBufferMutable { command_index_guard.next_acceleration_structure_build_command_index += 1; for blas in build.blas_s_built.iter() { + let mut state_lock = blas.compacted_state.lock(); + *state_lock = match *state_lock { + BlasCompactState::Compacted => { + unreachable!("Should be validated out in build.") + } + // Reset the compacted state to idle. This means any prepares, before mapping their + // internal buffer, will terminate. + _ => BlasCompactState::Idle, + }; *blas.built_index.write() = Some(build_command_index); } @@ -1215,6 +1240,10 @@ fn map_blas<'a>( storage: &'a BlasStore<'_>, scratch_buffer: &'a dyn hal::DynBuffer, snatch_guard: &'a SnatchGuard, + blases_compactable: &mut Vec<( + &'a dyn hal::DynBuffer, + &'a dyn hal::DynAccelerationStructure, + )>, ) -> Result< hal::BuildAccelerationStructureDescriptor< 'a, @@ -1231,12 +1260,27 @@ fn map_blas<'a>( if blas.update_mode == wgt::AccelerationStructureUpdateMode::PreferUpdate { log::info!("only rebuild implemented") } + let raw = blas.try_raw(snatch_guard)?; + + let state_lock = blas.compacted_state.lock(); + if let BlasCompactState::Compacted = *state_lock { + return Err(BuildAccelerationStructureError::CompactedBlas( + blas.error_ident(), + )); + } + + if blas + .flags + .contains(wgpu_types::AccelerationStructureFlags::ALLOW_COMPACTION) + { + blases_compactable.push((blas.compaction_buffer.as_ref().unwrap().as_ref(), raw)); + } Ok(hal::BuildAccelerationStructureDescriptor { entries, mode: hal::AccelerationStructureBuildMode::Build, flags: blas.flags, source_acceleration_structure: None, - destination_acceleration_structure: blas.try_raw(snatch_guard)?, + destination_acceleration_structure: raw, scratch_buffer, scratch_buffer_offset: *scratch_buffer_offset, }) @@ -1253,6 +1297,10 @@ fn build_blas<'a>( dyn hal::DynAccelerationStructure, >], scratch_buffer_barrier: hal::BufferBarrier, + blas_s_for_compaction: Vec<( + &'a dyn hal::DynBuffer, + &'a dyn hal::DynAccelerationStructure, + )>, ) { unsafe { cmd_buf_raw.transition_buffers(&input_barriers); @@ -1279,6 +1327,20 @@ fn build_blas<'a>( let mut source_usage = hal::AccelerationStructureUses::empty(); let mut destination_usage = hal::AccelerationStructureUses::empty(); + for &(buf, blas) in blas_s_for_compaction.iter() { + unsafe { + cmd_buf_raw.transition_buffers(&[hal::BufferBarrier { + buffer: buf, + usage: hal::StateTransition { + from: BufferUses::ACCELERATION_STRUCTURE_QUERY, + to: BufferUses::ACCELERATION_STRUCTURE_QUERY, + }, + }]) + } + unsafe { cmd_buf_raw.read_acceleration_structure_compact_size(blas, buf) } + destination_usage |= hal::AccelerationStructureUses::COPY_SRC; + } + if blas_present { source_usage |= hal::AccelerationStructureUses::BUILD_OUTPUT; destination_usage |= hal::AccelerationStructureUses::BUILD_INPUT diff --git a/wgpu-core/src/device/life.rs b/wgpu-core/src/device/life.rs index 7e20c53a9..67bb8a3c6 100644 --- a/wgpu-core/src/device/life.rs +++ b/wgpu-core/src/device/life.rs @@ -8,7 +8,8 @@ use crate::{ queue::{EncoderInFlight, SubmittedWorkDoneClosure, TempResource}, DeviceError, }, - resource::{Buffer, Texture, Trackable}, + ray_tracing::BlasCompactReadyPendingClosure, + resource::{Blas, Buffer, Texture, Trackable}, snatch::SnatchGuard, SubmissionIndex, }; @@ -32,6 +33,9 @@ struct ActiveSubmission { /// Buffers to be mapped once this submission has completed. mapped: Vec>, + /// BLASes to have their compacted size read back once this submission has completed. + compact_read_back: Vec>, + /// Command buffers used by this submission, and the encoder that owns them. /// /// [`wgpu_hal::Queue::submit`] requires the submitted command buffers to @@ -101,6 +105,23 @@ impl ActiveSubmission { false } + + /// Returns true if this submission contains the given blas. + /// + /// This only uses constant-time operations. + pub fn contains_blas(&self, blas: &Blas) -> bool { + for encoder in &self.encoders { + if encoder.trackers.blas_s.contains(blas) { + return true; + } + + if encoder.pending_blas_s.contains_key(&blas.tracker_index()) { + return true; + } + } + + false + } } #[derive(Clone, Debug, Error)] @@ -159,6 +180,10 @@ pub(crate) struct LifetimeTracker { /// queue submission still in flight. ready_to_map: Vec>, + /// BLASes the user has asked us to prepare to compact, and which are not used by any + /// queue submission still in flight. + ready_to_compact: Vec>, + /// Queue "on_submitted_work_done" closures that were initiated for while there is no /// currently pending submissions. These cannot be immediately invoked as they /// must happen _after_ all mapped buffer callbacks are mapped, so we defer them @@ -171,6 +196,7 @@ impl LifetimeTracker { Self { active: Vec::new(), ready_to_map: Vec::new(), + ready_to_compact: Vec::new(), work_done_closures: SmallVec::new(), } } @@ -185,6 +211,7 @@ impl LifetimeTracker { self.active.push(ActiveSubmission { index, mapped: Vec::new(), + compact_read_back: Vec::new(), encoders, work_done_closures: SmallVec::new(), }); @@ -207,6 +234,19 @@ impl LifetimeTracker { maybe_submission_index } + pub(crate) fn prepare_compact(&mut self, blas: &Arc) -> Option { + // Determine which BLASes are ready to map, and which must wait for the GPU. + let submission = self.active.iter_mut().rev().find(|a| a.contains_blas(blas)); + + let maybe_submission_index = submission.as_ref().map(|s| s.index); + + submission + .map_or(&mut self.ready_to_compact, |a| &mut a.compact_read_back) + .push(blas.clone()); + + maybe_submission_index + } + /// Returns the submission index of the most recent submission that uses the /// given buffer. pub fn get_buffer_latest_submission_index(&self, buffer: &Buffer) -> Option { @@ -270,6 +310,7 @@ impl LifetimeTracker { let mut work_done_closures: SmallVec<_> = self.work_done_closures.drain(..).collect(); for a in self.active.drain(..done_count) { self.ready_to_map.extend(a.mapped); + self.ready_to_compact.extend(a.compact_read_back); for encoder in a.encoders { // This involves actually decrementing the ref count of all command buffer // resources, so can be _very_ expensive. @@ -343,4 +384,25 @@ impl LifetimeTracker { } pending_callbacks } + /// Read back compact sizes from the BLASes in `self.ready_to_compact`. + /// + /// Return a list of mapping notifications to send. + /// + /// See the documentation for [`LifetimeTracker`] for details. + #[must_use] + pub(crate) fn handle_compact_read_back(&mut self) -> Vec { + if self.ready_to_compact.is_empty() { + return Vec::new(); + } + let mut pending_callbacks: Vec = + Vec::with_capacity(self.ready_to_compact.len()); + + for blas in self.ready_to_compact.drain(..) { + match blas.read_back_compact_size() { + Some(cb) => pending_callbacks.push(cb), + None => continue, + } + } + pending_callbacks + } } diff --git a/wgpu-core/src/device/mod.rs b/wgpu-core/src/device/mod.rs index a8d1329fa..8e6f28d55 100644 --- a/wgpu-core/src/device/mod.rs +++ b/wgpu-core/src/device/mod.rs @@ -5,6 +5,7 @@ use crate::{ binding_model, hub::Hub, id::{BindGroupLayoutId, PipelineLayoutId}, + ray_tracing::BlasCompactReadyPendingClosure, resource::{ Buffer, BufferAccessError, BufferAccessResult, BufferMapOperation, Labeled, ResourceErrorIdent, @@ -156,6 +157,7 @@ pub type BufferMapPendingClosure = (BufferMapOperation, BufferAccessResult); #[derive(Default)] pub struct UserClosures { pub mappings: Vec, + pub blas_compact_ready: Vec, pub submissions: SmallVec<[queue::SubmittedWorkDoneClosure; 1]>, pub device_lost_invocations: SmallVec<[DeviceLostInvocation; 1]>, } @@ -163,6 +165,7 @@ pub struct UserClosures { impl UserClosures { fn extend(&mut self, other: Self) { self.mappings.extend(other.mappings); + self.blas_compact_ready.extend(other.blas_compact_ready); self.submissions.extend(other.submissions); self.device_lost_invocations .extend(other.device_lost_invocations); @@ -179,6 +182,11 @@ impl UserClosures { callback(status); } } + for (mut operation, status) in self.blas_compact_ready { + if let Some(callback) = operation.take() { + callback(status); + } + } for closure in self.submissions { closure(); } diff --git a/wgpu-core/src/device/queue.rs b/wgpu-core/src/device/queue.rs index 746b0ddc5..8c5bf9d6c 100644 --- a/wgpu-core/src/device/queue.rs +++ b/wgpu-core/src/device/queue.rs @@ -1,19 +1,19 @@ -use alloc::{boxed::Box, sync::Arc, vec, vec::Vec}; +use alloc::{boxed::Box, string::ToString, sync::Arc, vec, vec::Vec}; use core::{ iter, mem::{self, ManuallyDrop}, + num::NonZeroU64, ptr::NonNull, sync::atomic::Ordering, }; - use smallvec::SmallVec; use thiserror::Error; +use wgt::AccelerationStructureFlags; use super::{life::LifetimeTracker, Device}; use crate::device::resource::CommandIndices; #[cfg(feature = "trace")] use crate::device::trace::Action; -use crate::scratch::ScratchBuffer; use crate::{ api_log, command::{ @@ -25,16 +25,19 @@ use crate::{ device::{DeviceError, WaitIdleError}, get_lowest_common_denom, global::Global, - id::{self, QueueId}, + id::{self, BlasId, QueueId}, init_tracker::{has_copy_partial_init_tracker_coverage, TextureInitRange}, - lock::{rank, Mutex, MutexGuard, RwLockWriteGuard}, + lock::{rank, Mutex, MutexGuard, RwLock, RwLockWriteGuard}, + ray_tracing::{BlasCompactReadyPendingClosure, CompactBlasError}, resource::{ - Buffer, BufferAccessError, BufferMapState, DestroyedBuffer, DestroyedResourceError, - DestroyedTexture, Fallible, FlushedStagingBuffer, InvalidResourceError, Labeled, - ParentDevice, ResourceErrorIdent, StagingBuffer, Texture, TextureInner, Trackable, + AccelerationStructure, Blas, BlasCompactState, Buffer, BufferAccessError, BufferMapState, + DestroyedBuffer, DestroyedResourceError, DestroyedTexture, Fallible, FlushedStagingBuffer, + InvalidResourceError, Labeled, ParentDevice, ResourceErrorIdent, StagingBuffer, Texture, + TextureInner, Trackable, TrackingData, }, resource_log, - snatch::SnatchGuard, + scratch::ScratchBuffer, + snatch::{SnatchGuard, Snatchable}, track::{self, Tracker, TrackerIndex}, FastHashMap, SubmissionIndex, }; @@ -116,16 +119,23 @@ impl Queue { ) -> ( SmallVec<[SubmittedWorkDoneClosure; 1]>, Vec, + Vec, bool, ) { let mut life_tracker = self.lock_life(); let submission_closures = life_tracker.triage_submissions(submission_index); let mapping_closures = life_tracker.handle_mapping(snatch_guard); + let blas_closures = life_tracker.handle_compact_read_back(); let queue_empty = life_tracker.queue_empty(); - (submission_closures, mapping_closures, queue_empty) + ( + submission_closures, + mapping_closures, + blas_closures, + queue_empty, + ) } } @@ -220,7 +230,7 @@ impl Drop for Queue { drop(fence); let snatch_guard = self.device.snatchable_lock.read(); - let (submission_closures, mapping_closures, queue_empty) = + let (submission_closures, mapping_closures, blas_compact_ready_closures, queue_empty) = self.maintain(last_successful_submission_index, &snatch_guard); drop(snatch_guard); @@ -228,6 +238,7 @@ impl Drop for Queue { let closures = crate::device::UserClosures { mappings: mapping_closures, + blas_compact_ready: blas_compact_ready_closures, submissions: submission_closures, device_lost_invocations: SmallVec::new(), }; @@ -275,6 +286,8 @@ pub(crate) struct EncoderInFlight { pub(crate) pending_buffers: FastHashMap>, /// These are the textures that have been tracked by `PendingWrites`. pub(crate) pending_textures: FastHashMap>, + /// These are the BLASes that have been tracked by `PendingWrites`. + pub(crate) pending_blas_s: FastHashMap>, } /// A private command encoder for writes made directly on the device @@ -312,6 +325,7 @@ pub(crate) struct PendingWrites { temp_resources: Vec, dst_buffers: FastHashMap>, dst_textures: FastHashMap>, + copied_blas_s: FastHashMap>, } impl PendingWrites { @@ -322,6 +336,7 @@ impl PendingWrites { temp_resources: Vec::new(), dst_buffers: FastHashMap::default(), dst_textures: FastHashMap::default(), + copied_blas_s: FastHashMap::default(), } } @@ -335,6 +350,11 @@ impl PendingWrites { .insert(texture.tracker_index(), texture.clone()); } + pub fn insert_blas(&mut self, blas: &Arc) { + self.copied_blas_s + .insert(blas.tracker_index(), blas.clone()); + } + pub fn contains_buffer(&self, buffer: &Arc) -> bool { self.dst_buffers.contains_key(&buffer.tracker_index()) } @@ -361,6 +381,7 @@ impl PendingWrites { if self.is_recording { let pending_buffers = mem::take(&mut self.dst_buffers); let pending_textures = mem::take(&mut self.dst_textures); + let pending_blas_s = mem::take(&mut self.copied_blas_s); let cmd_buf = unsafe { self.command_encoder.end_encoding() } .map_err(|e| device.handle_hal_error(e))?; @@ -385,11 +406,13 @@ impl PendingWrites { ), pending_buffers, pending_textures, + pending_blas_s, }; Ok(Some(encoder)) } else { self.dst_buffers.clear(); self.dst_textures.clear(); + self.copied_blas_s.clear(); Ok(None) } } @@ -1228,6 +1251,7 @@ impl Queue { .indirect_draw_validation_resources, pending_buffers: FastHashMap::default(), pending_textures: FastHashMap::default(), + pending_blas_s: FastHashMap::default(), }); } @@ -1381,6 +1405,84 @@ impl Queue { //TODO: flush pending writes self.lock_life().add_work_done_closure(closure) } + + pub fn compact_blas(&self, blas: &Arc) -> Result, CompactBlasError> { + profiling::scope!("Queue::compact_blas"); + api_log!("Queue::compact_blas"); + + self.same_device_as(blas.as_ref())?; + + let device = blas.device.clone(); + + let snatch_guard = device.snatchable_lock.read(); + + let BlasCompactState::Ready { size } = *blas.compacted_state.lock() else { + return Err(CompactBlasError::BlasNotReady); + }; + + let mut size_info = blas.size_info; + size_info.acceleration_structure_size = size; + + let mut pending_writes = self.pending_writes.lock(); + let cmd_buf_raw = pending_writes.activate(); + + let raw = unsafe { + device + .raw() + .create_acceleration_structure(&hal::AccelerationStructureDescriptor { + label: None, + size: size_info.acceleration_structure_size, + format: hal::AccelerationStructureFormat::BottomLevel, + allow_compaction: false, + }) + } + .map_err(DeviceError::from_hal)?; + + let src_raw = blas.try_raw(&snatch_guard)?; + + unsafe { + cmd_buf_raw.copy_acceleration_structure_to_acceleration_structure( + src_raw, + raw.as_ref(), + wgt::AccelerationStructureCopy::Compact, + ) + }; + + let handle = unsafe { + device + .raw() + .get_acceleration_structure_device_address(raw.as_ref()) + }; + + drop(snatch_guard); + + let mut command_indices_lock = device.command_indices.write(); + command_indices_lock.next_acceleration_structure_build_command_index += 1; + let built_index = + NonZeroU64::new(command_indices_lock.next_acceleration_structure_build_command_index) + .unwrap(); + + let new_blas = Arc::new(Blas { + raw: Snatchable::new(raw), + device: device.clone(), + size_info, + sizes: blas.sizes.clone(), + flags: blas.flags & !AccelerationStructureFlags::ALLOW_COMPACTION, + update_mode: blas.update_mode, + // Bypass the submit checks which update this because we don't submit this normally. + built_index: RwLock::new(rank::BLAS_BUILT_INDEX, Some(built_index)), + handle, + label: blas.label.clone() + " compacted", + tracking_data: TrackingData::new(blas.device.tracker_indices.blas_s.clone()), + compaction_buffer: None, + compacted_state: Mutex::new(rank::BLAS_COMPACTION_STATE, BlasCompactState::Compacted), + }); + + pending_writes.insert_blas(blas); + pending_writes.insert_blas(&new_blas); + + Ok(new_blas) + } } impl Global { @@ -1535,6 +1637,57 @@ impl Global { let result = queue.on_submitted_work_done(closure); result.unwrap_or(0) // '0' means no wait is necessary } + + pub fn queue_compact_blas( + &self, + queue_id: QueueId, + blas_id: BlasId, + id_in: Option, + ) -> (BlasId, Option, Option) { + api_log!("Queue::compact_blas {queue_id:?}, {blas_id:?}"); + + let fid = self.hub.blas_s.prepare(id_in); + + let queue = self.hub.queues.get(queue_id); + let blas = self.hub.blas_s.get(blas_id); + let device = &queue.device; + + // TODO: Tracing + + let error = 'error: { + match device.require_features( + wgpu_types::Features::EXPERIMENTAL_RAY_TRACING_ACCELERATION_STRUCTURE, + ) { + Ok(_) => {} + Err(err) => break 'error err.into(), + } + + let blas = match blas.get() { + Ok(blas) => blas, + Err(err) => break 'error err.into(), + }; + + let new_blas = match queue.compact_blas(&blas) { + Ok(blas) => blas, + Err(err) => break 'error err, + }; + + // We should have no more errors after this because we have marked the command encoder as successful. + let old_blas_size = blas.size_info.acceleration_structure_size; + let new_blas_size = new_blas.size_info.acceleration_structure_size; + let handle = new_blas.handle; + + let id = fid.assign(Fallible::Valid(new_blas)); + + api_log!("CommandEncoder::compact_blas {blas_id:?} (size: {old_blas_size}) -> {id:?} (size: {new_blas_size})"); + + return (id, Some(handle), None); + }; + + let id = fid.assign(Fallible::Invalid(Arc::new(error.to_string()))); + + (id, None, Some(error)) + } } fn validate_command_buffer( diff --git a/wgpu-core/src/device/ray_tracing.rs b/wgpu-core/src/device/ray_tracing.rs index 76c94f948..4e5583370 100644 --- a/wgpu-core/src/device/ray_tracing.rs +++ b/wgpu-core/src/device/ray_tracing.rs @@ -1,19 +1,24 @@ use alloc::{string::ToString as _, sync::Arc, vec::Vec}; -use core::mem::ManuallyDrop; +use core::mem::{size_of, ManuallyDrop}; -use crate::api_log; #[cfg(feature = "trace")] use crate::device::trace; -use crate::lock::rank; -use crate::resource::{Fallible, TrackingData}; -use crate::snatch::Snatchable; +use crate::device::DeviceError; use crate::{ + api_log, device::Device, global::Global, id::{self, BlasId, TlasId}, lock::RwLock, + lock::{rank, Mutex}, + ray_tracing::BlasPrepareCompactError, ray_tracing::{CreateBlasError, CreateTlasError}, - resource, LabelHelpers, + resource, + resource::{ + BlasCompactCallback, BlasCompactState, Fallible, InvalidResourceError, TrackingData, + }, + snatch::Snatchable, + LabelHelpers, }; use hal::AccelerationStructureTriangleIndices; use wgt::Features; @@ -105,12 +110,32 @@ impl Device { label: blas_desc.label.as_deref(), size: size_info.acceleration_structure_size, format: hal::AccelerationStructureFormat::BottomLevel, - // change this once compaction is implemented in wgpu-core - allow_compaction: false, + allow_compaction: blas_desc + .flags + .contains(wgpu_types::AccelerationStructureFlags::ALLOW_COMPACTION), }) } .map_err(|e| self.handle_hal_error_with_nonfatal_oom(e))?; + let compaction_buffer = if blas_desc + .flags + .contains(wgpu_types::AccelerationStructureFlags::ALLOW_COMPACTION) + { + Some(ManuallyDrop::new(unsafe { + self.raw() + .create_buffer(&hal::BufferDescriptor { + label: Some("(wgpu internal) compaction read-back buffer"), + size: size_of::() as wgpu_types::BufferAddress, + usage: wgpu_types::BufferUses::ACCELERATION_STRUCTURE_QUERY + | wgpu_types::BufferUses::MAP_READ, + memory_flags: hal::MemoryFlags::PREFER_COHERENT, + }) + .map_err(DeviceError::from_hal)? + })) + } else { + None + }; + let handle = unsafe { self.raw() .get_acceleration_structure_device_address(raw.as_ref()) @@ -127,6 +152,8 @@ impl Device { label: blas_desc.label.to_string(), built_index: RwLock::new(rank::BLAS_BUILT_INDEX, None), tracking_data: TrackingData::new(self.tracker_indices.blas_s.clone()), + compaction_buffer, + compacted_state: Mutex::new(rank::BLAS_COMPACTION_STATE, BlasCompactState::Idle), })) } @@ -311,4 +338,43 @@ impl Global { } } } + + pub fn blas_prepare_compact_async( + &self, + blas_id: BlasId, + callback: Option, + ) -> Result { + profiling::scope!("Blas::prepare_compact_async"); + api_log!("Blas::prepare_compact_async {blas_id:?}"); + + let hub = &self.hub; + + let compact_result = match hub.blas_s.get(blas_id).get() { + Ok(blas) => blas.prepare_compact_async(callback), + Err(e) => Err((callback, e.into())), + }; + + match compact_result { + Ok(submission_index) => Ok(submission_index), + Err((mut callback, err)) => { + if let Some(callback) = callback.take() { + callback(Err(err.clone())); + } + Err(err) + } + } + } + + pub fn ready_for_compaction(&self, blas_id: BlasId) -> Result { + profiling::scope!("Blas::prepare_compact_async"); + api_log!("Blas::prepare_compact_async {blas_id:?}"); + + let hub = &self.hub; + + let blas = hub.blas_s.get(blas_id).get()?; + + let lock = blas.compacted_state.lock(); + + Ok(matches!(*lock, BlasCompactState::Ready { .. })) + } } diff --git a/wgpu-core/src/device/resource.rs b/wgpu-core/src/device/resource.rs index 820c2d411..a5bbae722 100644 --- a/wgpu-core/src/device/resource.rs +++ b/wgpu-core/src/device/resource.rs @@ -541,6 +541,7 @@ impl Device { ( user_closures.submissions, user_closures.mappings, + user_closures.blas_compact_ready, queue_empty, ) = queue_result }; diff --git a/wgpu-core/src/lock/rank.rs b/wgpu-core/src/lock/rank.rs index 0be70ebe3..34d70a14e 100644 --- a/wgpu-core/src/lock/rank.rs +++ b/wgpu-core/src/lock/rank.rs @@ -148,6 +148,7 @@ define_lock_ranks! { rank TEXTURE_CLEAR_MODE "Texture::clear_mode" followed by { } rank TEXTURE_VIEWS "Texture::views" followed by { } rank BLAS_BUILT_INDEX "Blas::built_index" followed by { } + rank BLAS_COMPACTION_STATE "Blas::compaction_size" followed by { } rank TLAS_BUILT_INDEX "Tlas::built_index" followed by { } rank TLAS_DEPENDENCIES "Tlas::dependencies" followed by { } rank BUFFER_POOL "BufferPool::buffers" followed by { } diff --git a/wgpu-core/src/ray_tracing.rs b/wgpu-core/src/ray_tracing.rs index a70b4ad25..5b952b3be 100644 --- a/wgpu-core/src/ray_tracing.rs +++ b/wgpu-core/src/ray_tracing.rs @@ -5,6 +5,7 @@ // - lazy instance buffer allocation // - maybe share scratch and instance staging buffer allocation // - partial instance buffer uploads (api surface already designed with this in mind) +// - Batch BLAS read-backs (if it shows up in performance). // - ([non performance] extract function in build (rust function extraction with guards is a pain)) use alloc::{boxed::Box, sync::Arc, vec::Vec}; @@ -17,8 +18,8 @@ use crate::{ device::{DeviceError, MissingFeatures}, id::{BlasId, BufferId, TlasId}, resource::{ - Blas, DestroyedResourceError, InvalidResourceError, MissingBufferUsageError, - ResourceErrorIdent, Tlas, + Blas, BlasCompactCallback, BlasPrepareCompactResult, DestroyedResourceError, + InvalidResourceError, MissingBufferUsageError, ResourceErrorIdent, Tlas, }, }; @@ -117,6 +118,9 @@ pub enum BuildAccelerationStructureError { #[error("Blas {0:?} index formats are different, creation format: {1:?}, provided: {2:?}")] DifferentBlasIndexFormats(ResourceErrorIdent, Option, Option), + #[error("Blas {0:?} is compacted and so cannot be built")] + CompactedBlas(ResourceErrorIdent), + #[error("Blas {0:?} build sizes require index buffer but none was provided")] MissingIndexBuffer(ResourceErrorIdent), @@ -259,3 +263,42 @@ pub struct TraceTlasPackage { pub instances: Vec>, pub lowest_unmodified: u32, } + +#[derive(Clone, Debug, Error)] +pub enum BlasPrepareCompactError { + #[error(transparent)] + Device(#[from] DeviceError), + #[error(transparent)] + InvalidResource(#[from] InvalidResourceError), + #[error("Compaction is already being prepared")] + CompactionPreparingAlready, + #[error("Cannot compact an already compacted BLAS")] + DoubleCompaction, + #[error("BLAS is not yet built")] + NotBuilt, + #[error("BLAS does not support compaction (is AccelerationStructureFlags::ALLOW_COMPACTION missing?)")] + CompactionUnsupported, +} + +#[derive(Clone, Debug, Error)] +pub enum CompactBlasError { + #[error(transparent)] + Encoder(#[from] EncoderStateError), + + #[error(transparent)] + Device(#[from] DeviceError), + + #[error(transparent)] + InvalidResource(#[from] InvalidResourceError), + + #[error(transparent)] + DestroyedResource(#[from] DestroyedResourceError), + + #[error(transparent)] + MissingFeatures(#[from] MissingFeatures), + + #[error("BLAS was not prepared for compaction")] + BlasNotReady, +} + +pub type BlasCompactReadyPendingClosure = (Option, BlasPrepareCompactResult); diff --git a/wgpu-core/src/resource.rs b/wgpu-core/src/resource.rs index 8a3ec4465..95e9067e5 100644 --- a/wgpu-core/src/resource.rs +++ b/wgpu-core/src/resource.rs @@ -2,12 +2,11 @@ use alloc::{borrow::Cow, borrow::ToOwned as _, boxed::Box, string::String, sync: use core::{ borrow::Borrow, fmt, - mem::{self, ManuallyDrop}, + mem::{self, size_of, ManuallyDrop}, num::NonZeroU64, ops::Range, ptr::NonNull, }; - use smallvec::SmallVec; use thiserror::Error; use wgt::TextureSelector; @@ -28,6 +27,7 @@ use crate::{ }, init_tracker::{BufferInitTracker, TextureInitTracker}, lock::{rank, Mutex, RwLock}, + ray_tracing::{BlasCompactReadyPendingClosure, BlasPrepareCompactError}, resource_log, snatch::{SnatchGuard, Snatchable}, timestamp_normalization::TimestampNormalizationBindGroup, @@ -1974,6 +1974,45 @@ pub(crate) trait AccelerationStructure: Trackable { ) -> Result<&'a dyn hal::DynAccelerationStructure, DestroyedResourceError>; } +pub type BlasPrepareCompactResult = Result<(), BlasPrepareCompactError>; + +#[cfg(send_sync)] +pub type BlasCompactCallback = Box; +#[cfg(not(send_sync))] +pub type BlasCompactCallback = Box; + +pub(crate) struct BlasPendingCompact { + pub(crate) op: Option, + // hold the parent alive while the mapping is active + pub(crate) _parent_blas: Arc, +} + +impl fmt::Debug for BlasPendingCompact { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("BlasPendingCompact") + .field("op", &()) + .field("_parent_blas", &self._parent_blas) + .finish() + } +} + +#[derive(Debug)] +pub(crate) enum BlasCompactState { + /// Created from a compact operation. + Compacted, + /// Waiting for GPU to be done before mapping to get compacted size + Waiting(BlasPendingCompact), + /// Ready to be compacted + Ready { size: wgt::BufferAddress }, + /// Ready to prepare to compact. + Idle, +} + +#[cfg(send_sync)] +unsafe impl Send for BlasCompactState {} +#[cfg(send_sync)] +unsafe impl Sync for BlasCompactState {} + #[derive(Debug)] pub struct Blas { pub(crate) raw: Snatchable>, @@ -1987,17 +2026,26 @@ pub struct Blas { /// The `label` from the descriptor used to create the resource. pub(crate) label: String, pub(crate) tracking_data: TrackingData, + pub(crate) compaction_buffer: Option>>, + pub(crate) compacted_state: Mutex, } impl Drop for Blas { fn drop(&mut self) { resource_log!("Destroy raw {}", self.error_ident()); - // SAFETY: We are in the Drop impl, and we don't use self.raw anymore after this point. + // SAFETY: We are in the Drop impl, and we don't use self.raw or self.compaction_buffer anymore after this point. if let Some(raw) = self.raw.take() { unsafe { self.device.raw().destroy_acceleration_structure(raw); } } + if let Some(mut raw) = self.compaction_buffer.take() { + unsafe { + self.device + .raw() + .destroy_buffer(ManuallyDrop::take(&mut raw)) + } + } } } @@ -2013,6 +2061,107 @@ impl AccelerationStructure for Blas { } } +impl Blas { + pub(crate) fn prepare_compact_async( + self: &Arc, + op: Option, + ) -> Result, BlasPrepareCompactError)> { + let device = &self.device; + if let Err(e) = device.check_is_valid() { + return Err((op, e.into())); + } + + if self.built_index.read().is_none() { + return Err((op, BlasPrepareCompactError::NotBuilt)); + } + + if !self + .flags + .contains(wgt::AccelerationStructureFlags::ALLOW_COMPACTION) + { + return Err((op, BlasPrepareCompactError::CompactionUnsupported)); + } + + let mut state = self.compacted_state.lock(); + *state = match *state { + BlasCompactState::Compacted => { + return Err((op, BlasPrepareCompactError::DoubleCompaction)) + } + BlasCompactState::Waiting(_) => { + return Err((op, BlasPrepareCompactError::CompactionPreparingAlready)) + } + BlasCompactState::Ready { .. } => { + return Err((op, BlasPrepareCompactError::CompactionPreparingAlready)) + } + BlasCompactState::Idle => BlasCompactState::Waiting(BlasPendingCompact { + op, + _parent_blas: self.clone(), + }), + }; + + let submit_index = if let Some(queue) = device.get_queue() { + queue.lock_life().prepare_compact(self).unwrap_or(0) // '0' means no wait is necessary + } else { + // We can safely unwrap below since we just set the `compacted_state` to `BlasCompactState::Waiting`. + let (mut callback, status) = self.read_back_compact_size().unwrap(); + if let Some(callback) = callback.take() { + callback(status); + } + 0 + }; + + Ok(submit_index) + } + + /// This function returns [`None`] only if [`Self::compacted_state`] is not [`BlasCompactState::Waiting`]. + #[must_use] + pub(crate) fn read_back_compact_size(&self) -> Option { + let mut state = self.compacted_state.lock(); + let pending_compact = match mem::replace(&mut *state, BlasCompactState::Idle) { + BlasCompactState::Waiting(pending_mapping) => pending_mapping, + // Compaction cancelled e.g. by rebuild + BlasCompactState::Idle => return None, + BlasCompactState::Ready { .. } => { + unreachable!("This should be validated out by `prepare_for_compaction`") + } + _ => panic!("No pending mapping."), + }; + let status = { + let compaction_buffer = self.compaction_buffer.as_ref().unwrap().as_ref(); + unsafe { + let map_res = self.device.raw().map_buffer( + compaction_buffer, + 0..size_of::() as wgt::BufferAddress, + ); + match map_res { + Ok(mapping) => { + if !mapping.is_coherent { + // Clippy complains about this because it might not be intended, but + // this is intentional. + #[expect(clippy::single_range_in_vec_init)] + self.device.raw().flush_mapped_ranges( + compaction_buffer, + &[0..size_of::() as wgt::BufferAddress], + ); + } + let size = core::ptr::read_unaligned( + mapping.ptr.as_ptr().cast::(), + ); + self.device.raw().unmap_buffer(compaction_buffer); + if self.size_info.acceleration_structure_size != 0 { + debug_assert_ne!(size, 0); + } + *state = BlasCompactState::Ready { size }; + Ok(()) + } + Err(err) => Err(BlasPrepareCompactError::from(DeviceError::from_hal(err))), + } + } + }; + Some((pending_compact.op, status)) + } +} + crate::impl_resource_type!(Blas); crate::impl_labeled!(Blas); crate::impl_parent_device!(Blas); diff --git a/wgpu-core/src/track/blas.rs b/wgpu-core/src/track/blas.rs new file mode 100644 index 000000000..dec86aa1a --- /dev/null +++ b/wgpu-core/src/track/blas.rs @@ -0,0 +1,59 @@ +use crate::{ + resource::{Blas, Trackable}, + track::metadata::ResourceMetadata, +}; +use alloc::sync::Arc; + +/// A tracker that holds tracks BLASes. +/// +/// This is mostly a safe shell around [`ResourceMetadata`] +#[derive(Debug)] +pub(crate) struct BlasTracker { + metadata: ResourceMetadata>, + size: usize, +} + +impl BlasTracker { + pub fn new() -> Self { + Self { + metadata: ResourceMetadata::new(), + size: 0, + } + } + + /// Inserts a single resource into the resource tracker. + /// + /// Returns a reference to the newly inserted resource. + /// (This allows avoiding a clone/reference count increase in many cases.) + pub fn insert_single(&mut self, resource: Arc) -> &Arc { + let index = resource.tracker_index().as_usize(); + self.allow_index(index); + + unsafe { + // # SAFETY: we just allowed this resource, which makes the metadata object larger if + // it's not in bounds + self.metadata.insert(index, resource) + } + } + + /// Sets the size of all the vectors inside the tracker. + /// + /// Must be called with the highest possible Texture ID before + /// all unsafe functions are called. + pub fn set_size(&mut self, size: usize) { + self.size = size; + self.metadata.set_size(size) + } + + /// Extend the vectors to let the given index be valid. + fn allow_index(&mut self, index: usize) { + if index >= self.size { + self.set_size(index + 1); + } + } + + /// Returns true if the tracker owns the given texture. + pub fn contains(&self, blas: &Blas) -> bool { + self.metadata.contains(blas.tracker_index().as_usize()) + } +} diff --git a/wgpu-core/src/track/mod.rs b/wgpu-core/src/track/mod.rs index 12ede00a5..a878c5f39 100644 --- a/wgpu-core/src/track/mod.rs +++ b/wgpu-core/src/track/mod.rs @@ -95,6 +95,7 @@ Device <- CommandBuffer = insert(device.start, device.end, buffer.start, buffer. [`UsageScope`]: https://gpuweb.github.io/gpuweb/#programming-model-synchronization */ +mod blas; mod buffer; mod metadata; mod range; @@ -107,6 +108,7 @@ use crate::{ pipeline, resource::{self, Labeled, ResourceErrorIdent}, snatch::SnatchGuard, + track::blas::BlasTracker, }; use alloc::{sync::Arc, vec::Vec}; @@ -601,7 +603,7 @@ impl DeviceTracker { pub(crate) struct Tracker { pub buffers: BufferTracker, pub textures: TextureTracker, - pub blas_s: StatelessTracker, + pub blas_s: BlasTracker, pub tlas_s: StatelessTracker, pub views: StatelessTracker, pub bind_groups: StatelessTracker, @@ -616,7 +618,7 @@ impl Tracker { Self { buffers: BufferTracker::new(), textures: TextureTracker::new(), - blas_s: StatelessTracker::new(), + blas_s: BlasTracker::new(), tlas_s: StatelessTracker::new(), views: StatelessTracker::new(), bind_groups: StatelessTracker::new(), diff --git a/wgpu-types/src/lib.rs b/wgpu-types/src/lib.rs index 2cb16e393..dba446c3d 100644 --- a/wgpu-types/src/lib.rs +++ b/wgpu-types/src/lib.rs @@ -7565,8 +7565,8 @@ bitflags::bitflags!( /// Allow for incremental updates (no change in size), currently this is unimplemented /// and will build as normal (this is fine, update vs build should be unnoticeable) const ALLOW_UPDATE = 1 << 0; - /// Allow the acceleration structure to be compacted in a copy operation, the function - /// to compact is not currently implemented. + /// Allow the acceleration structure to be compacted in a copy operation + /// (`Blas::prepare_for_compaction`, `CommandEncoder::compact_blas`). const ALLOW_COMPACTION = 1 << 1; /// Optimize for fast ray tracing performance, recommended if the geometry is unlikely /// to change (e.g. in a game: non-interactive scene geometry) diff --git a/wgpu/src/api/blas.rs b/wgpu/src/api/blas.rs index fad3d0036..886154481 100644 --- a/wgpu/src/api/blas.rs +++ b/wgpu/src/api/blas.rs @@ -1,6 +1,6 @@ use alloc::{boxed::Box, vec::Vec}; -use wgt::WasmNotSendSync; +use wgt::{WasmNotSend, WasmNotSendSync}; use crate::dispatch; use crate::{Buffer, Label}; @@ -216,3 +216,49 @@ pub struct ContextBlasBuildEntry<'a> { #[expect(dead_code)] pub(crate) geometries: ContextBlasGeometries<'a>, } + +/// Error occurred when trying to asynchronously prepare a blas for compaction. +#[derive(Clone, PartialEq, Eq, Debug)] +pub struct BlasAsyncError; +static_assertions::assert_impl_all!(BlasAsyncError: Send, Sync); + +impl core::fmt::Display for BlasAsyncError { + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + write!( + f, + "Error occurred when trying to asynchronously prepare a blas for compaction" + ) + } +} + +impl core::error::Error for BlasAsyncError {} + +impl Blas { + /// Asynchronously prepares this BLAS for compaction. The callback is called once all builds + /// using this BLAS are finished and the BLAS is compactable. This can be checked using + /// [`Blas::ready_for_compaction`]. Rebuilding this BLAS will reset its compacted state, and it + /// will need to be prepared again. + /// + /// ### Interaction with other functions + /// On native, `queue.submit(..)` and polling devices (that is calling `instance.poll_all` or + /// `device.poll`) with [`PollType::Poll`] may call the callback. On native, polling devices with + /// [`PollType::Wait`] (or [`PollType::WaitForSubmissionIndex`] with a submission index greater + /// than the last submit the BLAS was used in) will guarantee callback is called. + /// + /// [`PollType::Poll`]: wgpu_types::PollType::Poll + /// [`PollType::Wait`]: wgpu_types::PollType::Wait + /// [`PollType::WaitForSubmissionIndex`]: wgpu_types::PollType::WaitForSubmissionIndex + pub fn prepare_compaction_async( + &self, + callback: impl FnOnce(Result<(), BlasAsyncError>) + WasmNotSend + 'static, + ) { + self.inner.prepare_compact_async(Box::new(callback)); + } + + /// Checks whether this BLAS is ready for compaction. The returned value is `true` if + /// [`Blas::prepare_compaction_async`]'s callback was called with a non-error value, otherwise + /// this is `false`. + pub fn ready_for_compaction(&self) -> bool { + self.inner.ready_for_compaction() + } +} diff --git a/wgpu/src/api/queue.rs b/wgpu/src/api/queue.rs index bc8d58182..0932a3149 100644 --- a/wgpu/src/api/queue.rs +++ b/wgpu/src/api/queue.rs @@ -301,4 +301,17 @@ impl Queue { hal_queue_callback(None) } } + + /// Compact a BLAS, it must have had [`Blas::prepare_compaction_async`] called on it and had the + /// callback provided called. + /// + /// The returned BLAS is more restricted than a normal BLAS because it may not be rebuilt or + /// compacted. + pub fn compact_blas(&self, blas: &Blas) -> Blas { + let (handle, dispatch) = self.inner.compact_blas(&blas.inner); + Blas { + handle, + inner: dispatch, + } + } } diff --git a/wgpu/src/backend/webgpu.rs b/wgpu/src/backend/webgpu.rs index 5f45c426f..97e5d6ed1 100644 --- a/wgpu/src/backend/webgpu.rs +++ b/wgpu/src/backend/webgpu.rs @@ -27,7 +27,10 @@ use wgt::Backends; use js_sys::Promise; use wasm_bindgen::{prelude::*, JsCast}; -use crate::{dispatch, Blas, SurfaceTargetUnsafe, Tlas}; +use crate::{ + dispatch::{self, BlasCompactCallback}, + Blas, SurfaceTargetUnsafe, Tlas, +}; use defined_non_null_js_value::DefinedNonNullJsValue; @@ -2581,6 +2584,13 @@ impl dispatch::QueueInterface for WebQueue { fn on_submitted_work_done(&self, _callback: dispatch::BoxSubmittedWorkDoneCallback) { unimplemented!("on_submitted_work_done is not yet implemented"); } + + fn compact_blas( + &self, + _blas: &dispatch::DispatchBlas, + ) -> (Option, dispatch::DispatchBlas) { + unimplemented!("Raytracing not implemented for web") + } } impl Drop for WebQueue { fn drop(&mut self) { @@ -2727,7 +2737,14 @@ impl Drop for WebTexture { } } -impl dispatch::BlasInterface for WebBlas {} +impl dispatch::BlasInterface for WebBlas { + fn prepare_compact_async(&self, _callback: BlasCompactCallback) { + unimplemented!("Raytracing not implemented for web") + } + fn ready_for_compaction(&self) -> bool { + unimplemented!("Raytracing not implemented for web") + } +} impl Drop for WebBlas { fn drop(&mut self) { // no-op diff --git a/wgpu/src/backend/wgpu_core.rs b/wgpu/src/backend/wgpu_core.rs index 290a725ba..d29f3527d 100644 --- a/wgpu/src/backend/wgpu_core.rs +++ b/wgpu/src/backend/wgpu_core.rs @@ -11,13 +11,16 @@ use core::{error::Error, fmt, future::ready, ops::Range, pin::Pin, ptr::NonNull, use arrayvec::ArrayVec; use smallvec::SmallVec; -use wgc::{command::bundle_ffi::*, error::ContextErrorSource, pipeline::CreateShaderModuleError}; +use wgc::{ + command::bundle_ffi::*, error::ContextErrorSource, pipeline::CreateShaderModuleError, + resource::BlasPrepareCompactResult, +}; use wgt::WasmNotSendSync; use crate::util::Mutex; use crate::{ api, - dispatch::{self, BufferMappedRangeInterface}, + dispatch::{self, BlasCompactCallback, BufferMappedRangeInterface}, BindingResource, Blas, BufferBinding, BufferDescriptor, CompilationInfo, CompilationMessage, CompilationMessageType, ErrorSource, Features, Label, LoadOp, MapMode, Operations, ShaderSource, SurfaceTargetUnsafe, TextureDescriptor, Tlas, @@ -644,7 +647,7 @@ pub struct CoreCommandEncoder { pub struct CoreBlas { pub(crate) context: ContextWgpuCore, id: wgc::id::BlasId, - // error_sink: ErrorSink, + error_sink: ErrorSink, } #[derive(Debug)] @@ -1547,7 +1550,7 @@ impl dispatch::DeviceInterface for CoreDevice { CoreBlas { context: self.context.clone(), id, - // error_sink: Arc::clone(&self.error_sink), + error_sink: Arc::clone(&self.error_sink), } .into(), ) @@ -1931,6 +1934,27 @@ impl dispatch::QueueInterface for CoreQueue { .0 .queue_on_submitted_work_done(self.id, callback); } + + fn compact_blas(&self, blas: &dispatch::DispatchBlas) -> (Option, dispatch::DispatchBlas) { + let (id, handle, error) = + self.context + .0 + .queue_compact_blas(self.id, blas.as_core().id, None); + + if let Some(cause) = error { + self.context + .handle_error_nolabel(&self.error_sink, cause, "Queue::compact_blas"); + } + ( + handle, + CoreBlas { + context: self.context.clone(), + id, + error_sink: Arc::clone(&self.error_sink), + } + .into(), + ) + } } impl Drop for CoreQueue { @@ -2102,7 +2126,39 @@ impl Drop for CoreTexture { } } -impl dispatch::BlasInterface for CoreBlas {} +impl dispatch::BlasInterface for CoreBlas { + fn prepare_compact_async(&self, callback: BlasCompactCallback) { + let callback: Option = + Some(Box::new(|status: BlasPrepareCompactResult| { + let res = status.map_err(|_| crate::BlasAsyncError); + callback(res); + })); + + match self.context.0.blas_prepare_compact_async(self.id, callback) { + Ok(_) => (), + Err(cause) => self.context.handle_error_nolabel( + &self.error_sink, + cause, + "Blas::prepare_compact_async", + ), + } + } + + fn ready_for_compaction(&self) -> bool { + match self.context.0.ready_for_compaction(self.id) { + Ok(ready) => ready, + Err(cause) => { + self.context.handle_error_nolabel( + &self.error_sink, + cause, + "Blas::ready_for_compaction", + ); + // A BLAS is definitely not ready for compaction if it's not valid + false + } + } + } +} impl Drop for CoreBlas { fn drop(&mut self) { diff --git a/wgpu/src/dispatch.rs b/wgpu/src/dispatch.rs index 640b6b212..f0c02d655 100644 --- a/wgpu/src/dispatch.rs +++ b/wgpu/src/dispatch.rs @@ -54,6 +54,11 @@ pub type BufferMapCallback = Box) #[cfg(not(send_sync))] pub type BufferMapCallback = Box) + 'static>; +#[cfg(send_sync)] +pub type BlasCompactCallback = Box) + Send + 'static>; +#[cfg(not(send_sync))] +pub type BlasCompactCallback = Box) + 'static>; + // remove when rust 1.86 #[cfg_attr(not(custom), expect(dead_code))] pub trait AsAny { @@ -222,6 +227,8 @@ pub trait QueueInterface: CommonTraits { fn get_timestamp_period(&self) -> f32; fn on_submitted_work_done(&self, callback: BoxSubmittedWorkDoneCallback); + + fn compact_blas(&self, blas: &DispatchBlas) -> (Option, DispatchBlas); } pub trait ShaderModuleInterface: CommonTraits { @@ -250,7 +257,10 @@ pub trait TextureInterface: CommonTraits { fn destroy(&self); } -pub trait BlasInterface: CommonTraits {} +pub trait BlasInterface: CommonTraits { + fn prepare_compact_async(&self, callback: BlasCompactCallback); + fn ready_for_compaction(&self) -> bool; +} pub trait TlasInterface: CommonTraits {} pub trait QuerySetInterface: CommonTraits {} pub trait PipelineLayoutInterface: CommonTraits {}