[wgpu] Blas compaction (#7285)

Co-authored-by: Connor Fitzgerald <connorwadefitzgerald@gmail.com>
This commit is contained in:
Vecvec 2025-06-12 13:40:10 +12:00 committed by GitHub
parent efbac5dfba
commit 73eb83ded1
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
21 changed files with 1002 additions and 45 deletions

View File

@ -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).

View File

@ -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<Item = BlasToBeCompacted> = 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:

View File

@ -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<u64>, DispatchBlas) {
unimplemented!()
}
}
#[derive(Debug)]

View File

@ -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(

View File

@ -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<dyn hal::DynBuffer>,
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

View File

@ -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<Arc<Buffer>>,
/// BLASes to have their compacted size read back once this submission has completed.
compact_read_back: Vec<Arc<Blas>>,
/// 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<Arc<Buffer>>,
/// 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<Arc<Blas>>,
/// 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<Blas>) -> Option<SubmissionIndex> {
// 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<SubmissionIndex> {
@ -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<BlasCompactReadyPendingClosure> {
if self.ready_to_compact.is_empty() {
return Vec::new();
}
let mut pending_callbacks: Vec<BlasCompactReadyPendingClosure> =
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
}
}

View File

@ -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<BufferMapPendingClosure>,
pub blas_compact_ready: Vec<BlasCompactReadyPendingClosure>,
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();
}

View File

@ -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<super::BufferMapPendingClosure>,
Vec<BlasCompactReadyPendingClosure>,
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<TrackerIndex, Arc<Buffer>>,
/// These are the textures that have been tracked by `PendingWrites`.
pub(crate) pending_textures: FastHashMap<TrackerIndex, Arc<Texture>>,
/// These are the BLASes that have been tracked by `PendingWrites`.
pub(crate) pending_blas_s: FastHashMap<TrackerIndex, Arc<Blas>>,
}
/// A private command encoder for writes made directly on the device
@ -312,6 +325,7 @@ pub(crate) struct PendingWrites {
temp_resources: Vec<TempResource>,
dst_buffers: FastHashMap<TrackerIndex, Arc<Buffer>>,
dst_textures: FastHashMap<TrackerIndex, Arc<Texture>>,
copied_blas_s: FastHashMap<TrackerIndex, Arc<Blas>>,
}
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<Blas>) {
self.copied_blas_s
.insert(blas.tracker_index(), blas.clone());
}
pub fn contains_buffer(&self, buffer: &Arc<Buffer>) -> 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<Blas>) -> Result<Arc<Blas>, 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>,
) -> (BlasId, Option<u64>, Option<CompactBlasError>) {
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(

View File

@ -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::<wgpu_types::BufferAddress>() 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<BlasCompactCallback>,
) -> Result<crate::SubmissionIndex, BlasPrepareCompactError> {
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<bool, InvalidResourceError> {
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 { .. }))
}
}

View File

@ -541,6 +541,7 @@ impl Device {
(
user_closures.submissions,
user_closures.mappings,
user_closures.blas_compact_ready,
queue_empty,
) = queue_result
};

View File

@ -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 { }

View File

@ -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<IndexFormat>, Option<IndexFormat>),
#[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<Option<TraceTlasInstance>>,
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<BlasCompactCallback>, BlasPrepareCompactResult);

View File

@ -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<dyn FnOnce(BlasPrepareCompactResult) + Send + 'static>;
#[cfg(not(send_sync))]
pub type BlasCompactCallback = Box<dyn FnOnce(BlasPrepareCompactResult) + 'static>;
pub(crate) struct BlasPendingCompact {
pub(crate) op: Option<BlasCompactCallback>,
// hold the parent alive while the mapping is active
pub(crate) _parent_blas: Arc<Blas>,
}
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<Box<dyn hal::DynAccelerationStructure>>,
@ -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<ManuallyDrop<Box<dyn hal::DynBuffer>>>,
pub(crate) compacted_state: Mutex<BlasCompactState>,
}
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<Self>,
op: Option<BlasCompactCallback>,
) -> Result<SubmissionIndex, (Option<BlasCompactCallback>, 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<BlasCompactReadyPendingClosure> {
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::<wgpu_types::BufferAddress>() 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::<wgpu_types::BufferAddress>() as wgt::BufferAddress],
);
}
let size = core::ptr::read_unaligned(
mapping.ptr.as_ptr().cast::<wgt::BufferAddress>(),
);
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);

View File

@ -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<Arc<Blas>>,
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<Blas>) -> &Arc<Blas> {
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())
}
}

View File

@ -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<resource::Blas>,
pub blas_s: BlasTracker,
pub tlas_s: StatelessTracker<resource::Tlas>,
pub views: StatelessTracker<resource::TextureView>,
pub bind_groups: StatelessTracker<binding_model::BindGroup>,
@ -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(),

View File

@ -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)

View File

@ -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()
}
}

View File

@ -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,
}
}
}

View File

@ -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<u64>, 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

View File

@ -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<u64>, 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<wgc::resource::BlasCompactCallback> =
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) {

View File

@ -54,6 +54,11 @@ pub type BufferMapCallback = Box<dyn FnOnce(Result<(), crate::BufferAsyncError>)
#[cfg(not(send_sync))]
pub type BufferMapCallback = Box<dyn FnOnce(Result<(), crate::BufferAsyncError>) + 'static>;
#[cfg(send_sync)]
pub type BlasCompactCallback = Box<dyn FnOnce(Result<(), crate::BlasAsyncError>) + Send + 'static>;
#[cfg(not(send_sync))]
pub type BlasCompactCallback = Box<dyn FnOnce(Result<(), crate::BlasAsyncError>) + '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<u64>, 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 {}