webgpu: Simplify presentation and handle cleared in script (#38717)

There are many important changes here:

- Generalize the presentation buffer into standalone staging buffers
that hold their own state. This allow them to be used by getImage.
- Move all clear handling to the ScriptThread and send the configuration
on each request present/getimage, thus avoiding any recreate/clearing
messages. This means that we prepare staging buffers lazily, on the
first request.

Try run for this change:
https://github.com/sagudev/servo/actions/runs/17341982368
Testing: This is covered by existing WebGPU CTS tests. There are some
bad expectations updates, but they are also on main (presumably from
last update the rendering work) although I think CTS is actually wrong
(see https://github.com/gpuweb/cts/issues/4440).
Fixes: #36820
Fixes: #37705
Fixes: #33368 (we now keep reference alive in hashmap)

---------

Signed-off-by: sagudev <16504129+sagudev@users.noreply.github.com>
This commit is contained in:
Sam 2025-09-09 05:35:12 +02:00 committed by GitHub
parent 8d2723b2c9
commit 1f0f079203
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
29 changed files with 1023 additions and 935 deletions

View file

@ -0,0 +1,776 @@
/* This Source Code Form is subject to the terms of the Mozilla Public
* License, v. 2.0. If a copy of the MPL was not distributed with this
* file, You can obtain one at https://mozilla.org/MPL/2.0/. */
//! Main process implementation of [GPUCanvasContext](https://www.w3.org/TR/webgpu/#canvas-context)
use std::ptr::NonNull;
use std::sync::{Arc, Mutex};
use arrayvec::ArrayVec;
use base::Epoch;
use compositing_traits::{
CrossProcessCompositorApi, ExternalImageSource, SerializableImageData,
WebrenderExternalImageApi,
};
use euclid::default::Size2D;
use ipc_channel::ipc::IpcSender;
use log::warn;
use pixels::{IpcSnapshot, Snapshot, SnapshotAlphaMode, SnapshotPixelFormat};
use rustc_hash::FxHashMap;
use webgpu_traits::{
ContextConfiguration, PRESENTATION_BUFFER_COUNT, PendingTexture, WebGPUContextId, WebGPUMsg,
};
use webrender_api::units::DeviceIntSize;
use webrender_api::{
ExternalImageData, ExternalImageId, ExternalImageType, ImageDescriptor, ImageDescriptorFlags,
ImageFormat, ImageKey,
};
use wgpu_core::device::HostMap;
use wgpu_core::global::Global;
use wgpu_core::id::{
self, BufferId, CommandBufferId, CommandEncoderId, DeviceId, QueueId, TextureId,
};
use wgpu_core::resource::{
BufferAccessError, BufferDescriptor, BufferMapOperation, CreateBufferError,
};
use wgpu_types::{
BufferUsages, COPY_BYTES_PER_ROW_ALIGNMENT, CommandBufferDescriptor, CommandEncoderDescriptor,
Extent3d, Origin3d, TexelCopyBufferInfo, TexelCopyBufferLayout, TexelCopyTextureInfo,
TextureAspect,
};
pub type WGPUImageMap = Arc<Mutex<FxHashMap<WebGPUContextId, ContextData>>>;
const fn image_data(context_id: WebGPUContextId) -> ExternalImageData {
ExternalImageData {
id: ExternalImageId(context_id.0),
channel_index: 0,
image_type: ExternalImageType::Buffer,
normalized_uvs: false,
}
}
/// Allocated buffer on GPU device
#[derive(Clone, Copy, Debug)]
struct Buffer {
device_id: DeviceId,
queue_id: QueueId,
size: u64,
}
impl Buffer {
/// Returns true if buffer is compatible with provided configuration
fn has_compatible_config(&self, config: &ContextConfiguration) -> bool {
config.device_id == self.device_id && self.size == config.buffer_size()
}
}
/// Mapped GPUBuffer
#[derive(Debug)]
struct MappedBuffer {
buffer: Buffer,
data: NonNull<u8>,
len: u64,
image_size: Size2D<u32>,
image_format: ImageFormat,
is_opaque: bool,
}
// Mapped buffer can be shared between safely (it's read-only)
unsafe impl Send for MappedBuffer {}
impl MappedBuffer {
const fn slice(&'_ self) -> &'_ [u8] {
// Safety: Pointer is from wgpu, and we only use it here
unsafe { std::slice::from_raw_parts(self.data.as_ptr(), self.len as usize) }
}
fn stride(&self) -> u32 {
(self.image_size.width * self.image_format.bytes_per_pixel() as u32)
.next_multiple_of(COPY_BYTES_PER_ROW_ALIGNMENT)
}
}
#[derive(Debug)]
enum StagingBufferState {
/// The Initial state: the buffer has yet to be created with only an
/// id reserved for it.
Unassigned,
/// The buffer is allocated in the WGPU Device and is ready to be used.
Available(Buffer),
/// `mapAsync` is currently running on the buffer.
Mapping(Buffer),
/// The buffer is currently mapped.
Mapped(MappedBuffer),
}
/// A staging buffer used for texture to buffer to CPU copy operations.
#[derive(Debug)]
struct StagingBuffer {
global: Arc<Global>,
buffer_id: BufferId,
state: StagingBufferState,
}
// [`StagingBuffer`] only used for reading (never for writing)
// so it is safe to share between threads.
unsafe impl Sync for StagingBuffer {}
impl StagingBuffer {
fn new(global: Arc<Global>, buffer_id: BufferId) -> Self {
Self {
global,
buffer_id,
state: StagingBufferState::Unassigned,
}
}
const fn is_mapped(&self) -> bool {
matches!(self.state, StagingBufferState::Mapped(..))
}
/// Return true if buffer can be used directly with provided config
/// without any additional work
fn is_available_and_has_compatible_config(&self, config: &ContextConfiguration) -> bool {
let StagingBufferState::Available(buffer) = &self.state else {
return false;
};
buffer.has_compatible_config(config)
}
/// Return true if buffer is not mapping or being mapped
const fn needs_assignment(&self) -> bool {
matches!(
self.state,
StagingBufferState::Unassigned | StagingBufferState::Available(_)
)
}
/// Make buffer available by unmapping / destroying it and then recreating it if needed.
fn ensure_available(&mut self, config: &ContextConfiguration) -> Result<(), CreateBufferError> {
let recreate = match &self.state {
StagingBufferState::Unassigned => true,
StagingBufferState::Available(buffer) |
StagingBufferState::Mapping(buffer) |
StagingBufferState::Mapped(MappedBuffer { buffer, .. }) => {
if buffer.has_compatible_config(config) {
let _ = self.global.buffer_unmap(self.buffer_id);
false
} else {
self.global.buffer_drop(self.buffer_id);
true
}
},
};
if recreate {
let buffer_size = config.buffer_size();
let (_, error) = self.global.device_create_buffer(
config.device_id,
&BufferDescriptor {
label: None,
size: buffer_size,
usage: BufferUsages::MAP_READ | BufferUsages::COPY_DST,
mapped_at_creation: false,
},
Some(self.buffer_id),
);
if let Some(error) = error {
return Err(error);
};
self.state = StagingBufferState::Available(Buffer {
device_id: config.device_id,
queue_id: config.queue_id,
size: buffer_size,
});
}
Ok(())
}
/// Makes buffer available and prepares command encoder
/// that will copy texture to this staging buffer.
///
/// Caller must submit command buffer to queue.
fn prepare_load_texture_command_buffer(
&mut self,
texture_id: TextureId,
encoder_id: CommandEncoderId,
config: &ContextConfiguration,
) -> Result<CommandBufferId, Box<dyn std::error::Error>> {
self.ensure_available(config)?;
let StagingBufferState::Available(buffer) = &self.state else {
unreachable!("Should be made available by `ensure_available`")
};
let device_id = buffer.device_id;
let command_descriptor = CommandEncoderDescriptor { label: None };
let (encoder_id, error) = self.global.device_create_command_encoder(
device_id,
&command_descriptor,
Some(encoder_id),
);
if let Some(error) = error {
return Err(error.into());
};
let buffer_info = TexelCopyBufferInfo {
buffer: self.buffer_id,
layout: TexelCopyBufferLayout {
offset: 0,
bytes_per_row: Some(config.stride()),
rows_per_image: None,
},
};
let texture_info = TexelCopyTextureInfo {
texture: texture_id,
mip_level: 0,
origin: Origin3d::ZERO,
aspect: TextureAspect::All,
};
let copy_size = Extent3d {
width: config.size.width,
height: config.size.height,
depth_or_array_layers: 1,
};
self.global.command_encoder_copy_texture_to_buffer(
encoder_id,
&texture_info,
&buffer_info,
&copy_size,
)?;
let (command_buffer_id, error) = self
.global
.command_encoder_finish(encoder_id, &CommandBufferDescriptor::default());
if let Some(error) = error {
return Err(error.into());
};
Ok(command_buffer_id)
}
/// Unmaps the buffer or cancels a mapping operation if one is in progress.
fn unmap(&mut self) {
match self.state {
StagingBufferState::Unassigned | StagingBufferState::Available(_) => {},
StagingBufferState::Mapping(buffer) |
StagingBufferState::Mapped(MappedBuffer { buffer, .. }) => {
let _ = self.global.buffer_unmap(self.buffer_id);
self.state = StagingBufferState::Available(buffer)
},
}
}
/// Obtain a snapshot from this buffer if is mapped or return `None` if it is not mapped.
fn snapshot(&self) -> Option<Snapshot> {
let StagingBufferState::Mapped(mapped) = &self.state else {
return None;
};
let format = match mapped.image_format {
ImageFormat::RGBA8 => SnapshotPixelFormat::RGBA,
ImageFormat::BGRA8 => SnapshotPixelFormat::BGRA,
_ => unreachable!("GPUCanvasContext does not support other formats per spec"),
};
let alpha_mode = if mapped.is_opaque {
SnapshotAlphaMode::AsOpaque {
premultiplied: false,
}
} else {
SnapshotAlphaMode::Transparent {
premultiplied: true,
}
};
let padded_byte_width = mapped.stride();
let data = mapped.slice();
let bytes_per_pixel = mapped.image_format.bytes_per_pixel() as usize;
let mut result_unpadded =
Vec::<u8>::with_capacity(mapped.image_size.area() as usize * bytes_per_pixel);
for row in 0..mapped.image_size.height {
let start = (row * padded_byte_width).try_into().ok()?;
result_unpadded
.extend(&data[start..start + mapped.image_size.width as usize * bytes_per_pixel]);
}
let mut snapshot =
Snapshot::from_vec(mapped.image_size, format, alpha_mode, result_unpadded);
if mapped.is_opaque {
snapshot.transform(SnapshotAlphaMode::Opaque, snapshot.format())
}
Some(snapshot)
}
}
impl Drop for StagingBuffer {
fn drop(&mut self) {
match self.state {
StagingBufferState::Unassigned => {},
StagingBufferState::Available(_) |
StagingBufferState::Mapping(_) |
StagingBufferState::Mapped(_) => {
self.global.buffer_drop(self.buffer_id);
},
}
}
}
#[derive(Default)]
pub struct WGPUExternalImages {
pub images: WGPUImageMap,
pub locked_ids: FxHashMap<WebGPUContextId, PresentationStagingBuffer>,
}
impl WebrenderExternalImageApi for WGPUExternalImages {
fn lock(&mut self, id: u64) -> (ExternalImageSource<'_>, Size2D<i32>) {
let id = WebGPUContextId(id);
let presentation = {
let mut webgpu_contexts = self.images.lock().unwrap();
webgpu_contexts
.get_mut(&id)
.and_then(|context_data| context_data.presentation.clone())
};
let Some(presentation) = presentation else {
return (ExternalImageSource::Invalid, Size2D::zero());
};
self.locked_ids.insert(id, presentation);
let presentation = self.locked_ids.get(&id).unwrap();
let StagingBufferState::Mapped(mapped_buffer) = &presentation.staging_buffer.state else {
unreachable!("Presentation staging buffer should be mapped")
};
let size = mapped_buffer.image_size;
(
ExternalImageSource::RawData(mapped_buffer.slice()),
size.cast().cast_unit(),
)
}
fn unlock(&mut self, id: u64) {
let id = WebGPUContextId(id);
let Some(presentation) = self.locked_ids.remove(&id) else {
return;
};
let mut webgpu_contexts = self.images.lock().unwrap();
if let Some(context_data) = webgpu_contexts.get_mut(&id) {
// We use this to return staging buffer if a newer one exists.
presentation.maybe_destroy(context_data);
} else {
// This will not free this buffer id in script,
// but that's okay because we still have many free ids.
drop(presentation);
}
}
}
/// Staging buffer currently used for presenting the epoch.
///
/// Users should [`ContextData::replace_presentation`] when done.
#[derive(Clone)]
pub struct PresentationStagingBuffer {
epoch: Epoch,
staging_buffer: Arc<StagingBuffer>,
}
impl PresentationStagingBuffer {
fn new(epoch: Epoch, staging_buffer: StagingBuffer) -> Self {
Self {
epoch,
staging_buffer: Arc::new(staging_buffer),
}
}
/// If the internal staging buffer is not shared,
/// unmap it and call [`ContextData::return_staging_buffer`] with it.
fn maybe_destroy(self, context_data: &mut ContextData) {
if let Some(mut staging_buffer) = Arc::into_inner(self.staging_buffer) {
staging_buffer.unmap();
context_data.return_staging_buffer(staging_buffer);
}
}
}
/// The embedder process-side representation of what is the `GPUCanvasContext` in script.
pub struct ContextData {
/// The [`ImageKey`] of the WebRender image associated with this context.
image_key: ImageKey,
/// Staging buffers that are not actively used.
///
/// Staging buffer here are either [`StagingBufferState::Unassigned`] or [`StagingBufferState::Available`].
/// They are removed from here when they are in process of being mapped or are already mapped.
inactive_staging_buffers: ArrayVec<StagingBuffer, PRESENTATION_BUFFER_COUNT>,
/// The [`PresentationStagingBuffer`] of the most recent presentation. This will
/// be `None` directly after initialization, as clearing is handled completely in
/// the `ScriptThread`.
presentation: Option<PresentationStagingBuffer>,
/// Next epoch to be used
next_epoch: Epoch,
}
impl ContextData {
fn new(
image_key: ImageKey,
global: &Arc<Global>,
buffer_ids: ArrayVec<id::BufferId, PRESENTATION_BUFFER_COUNT>,
) -> Self {
Self {
image_key,
inactive_staging_buffers: buffer_ids
.iter()
.map(|buffer_id| StagingBuffer::new(global.clone(), *buffer_id))
.collect(),
presentation: None,
next_epoch: Epoch(1),
}
}
/// Returns `None` if no staging buffer is unused or failure when making it available
fn get_or_make_available_buffer(
&'_ mut self,
config: &ContextConfiguration,
) -> Option<StagingBuffer> {
self.inactive_staging_buffers
.iter()
// Try to get first preallocated GPUBuffer.
.position(|staging_buffer| {
staging_buffer.is_available_and_has_compatible_config(config)
})
// Fall back to the first inactive staging buffer.
.or_else(|| {
self.inactive_staging_buffers
.iter()
.position(|staging_buffer| staging_buffer.needs_assignment())
})
// Or just the use first one.
.or_else(|| {
if self.inactive_staging_buffers.is_empty() {
None
} else {
Some(0)
}
})
.and_then(|index| {
let mut staging_buffer = self.inactive_staging_buffers.remove(index);
if staging_buffer.ensure_available(config).is_ok() {
Some(staging_buffer)
} else {
// If we fail to make it available, return it to the list of inactive staging buffers.
self.inactive_staging_buffers.push(staging_buffer);
None
}
})
}
/// Destroy the context that this [`ContextData`] represents,
/// freeing all of its buffers, and deleting the associated WebRender image.
fn destroy(
self,
script_sender: &IpcSender<WebGPUMsg>,
compositor_api: &CrossProcessCompositorApi,
) {
// This frees the id in the `ScriptThread`.
for staging_buffer in self.inactive_staging_buffers {
if let Err(error) = script_sender.send(WebGPUMsg::FreeBuffer(staging_buffer.buffer_id))
{
warn!(
"Unable to send FreeBuffer({:?}) ({error})",
staging_buffer.buffer_id
);
};
}
compositor_api.delete_image(self.image_key);
}
/// Advance the [`Epoch`] and return the new one.
fn next_epoch(&mut self) -> Epoch {
let epoch = self.next_epoch;
self.next_epoch.next();
epoch
}
/// If the given [`PresentationStagingBuffer`] is for a newer presentation, replace the existing
/// one. Deallocate the older one by calling [`Self::return_staging_buffer`] on it.
fn replace_presentation(&mut self, presentation: PresentationStagingBuffer) {
let stale_presentation = if presentation.epoch >=
self.presentation
.as_ref()
.map(|p| p.epoch)
.unwrap_or(Epoch(0))
{
self.presentation.replace(presentation)
} else {
Some(presentation)
};
if let Some(stale_presentation) = stale_presentation {
stale_presentation.maybe_destroy(self);
}
}
fn clear_presentation(&mut self) {
if let Some(stale_presentation) = self.presentation.take() {
stale_presentation.maybe_destroy(self);
}
}
fn return_staging_buffer(&mut self, staging_buffer: StagingBuffer) {
self.inactive_staging_buffers.push(staging_buffer)
}
}
impl crate::WGPU {
pub(crate) fn create_context(
&self,
context_id: WebGPUContextId,
image_key: ImageKey,
size: DeviceIntSize,
buffer_ids: ArrayVec<id::BufferId, PRESENTATION_BUFFER_COUNT>,
) {
let context_data = ContextData::new(image_key, &self.global, buffer_ids);
self.compositor_api.add_image(
image_key,
ImageDescriptor {
format: ImageFormat::BGRA8,
size,
stride: None,
offset: 0,
flags: ImageDescriptorFlags::empty(),
},
SerializableImageData::External(image_data(context_id)),
);
assert!(
self.wgpu_image_map
.lock()
.unwrap()
.insert(context_id, context_data)
.is_none(),
"Context should be created only once!"
);
}
pub(crate) fn get_image(
&self,
context_id: WebGPUContextId,
pending_texture: Option<PendingTexture>,
sender: IpcSender<IpcSnapshot>,
) {
let mut webgpu_contexts = self.wgpu_image_map.lock().unwrap();
let context_data = webgpu_contexts.get_mut(&context_id).unwrap();
if let Some(PendingTexture {
texture_id,
encoder_id,
configuration,
}) = pending_texture
{
let Some(staging_buffer) = context_data.get_or_make_available_buffer(&configuration)
else {
warn!("Failure obtaining available staging buffer");
sender
.send(Snapshot::cleared(configuration.size).as_ipc())
.unwrap();
return;
};
let epoch = context_data.next_epoch();
let wgpu_image_map = self.wgpu_image_map.clone();
let sender = sender.clone();
drop(webgpu_contexts);
self.texture_download(
texture_id,
encoder_id,
staging_buffer,
configuration,
move |staging_buffer| {
let mut webgpu_contexts = wgpu_image_map.lock().unwrap();
let context_data = webgpu_contexts.get_mut(&context_id).unwrap();
sender
.send(
staging_buffer
.snapshot()
.unwrap_or_else(|| Snapshot::cleared(configuration.size))
.as_ipc(),
)
.unwrap();
if staging_buffer.is_mapped() {
context_data.replace_presentation(PresentationStagingBuffer::new(
epoch,
staging_buffer,
));
} else {
// failure
context_data.return_staging_buffer(staging_buffer);
}
},
);
} else {
sender
.send(
context_data
.presentation
.as_ref()
.and_then(|presentation_staging_buffer| {
presentation_staging_buffer.staging_buffer.snapshot()
})
.unwrap_or_else(Snapshot::empty)
.as_ipc(),
)
.unwrap();
}
}
/// Read the texture to the staging buffer, map it to CPU memory, and update the
/// image in WebRender when complete.
pub(crate) fn present(
&self,
context_id: WebGPUContextId,
pending_texture: Option<PendingTexture>,
size: Size2D<u32>,
canvas_epoch: Epoch,
) {
let mut webgpu_contexts = self.wgpu_image_map.lock().unwrap();
let context_data = webgpu_contexts.get_mut(&context_id).unwrap();
let image_key = context_data.image_key;
let Some(PendingTexture {
texture_id,
encoder_id,
configuration,
}) = pending_texture
else {
context_data.clear_presentation();
self.compositor_api.update_image(
image_key,
ImageDescriptor {
format: ImageFormat::BGRA8,
size: size.cast_unit().cast(),
stride: None,
offset: 0,
flags: ImageDescriptorFlags::empty(),
},
SerializableImageData::External(image_data(context_id)),
Some(canvas_epoch),
);
return;
};
let Some(staging_buffer) = context_data.get_or_make_available_buffer(&configuration) else {
warn!("Failure obtaining available staging buffer");
context_data.clear_presentation();
self.compositor_api.update_image(
image_key,
configuration.into(),
SerializableImageData::External(image_data(context_id)),
Some(canvas_epoch),
);
return;
};
let epoch = context_data.next_epoch();
let wgpu_image_map = self.wgpu_image_map.clone();
let compositor_api = self.compositor_api.clone();
drop(webgpu_contexts);
self.texture_download(
texture_id,
encoder_id,
staging_buffer,
configuration,
move |staging_buffer| {
let mut webgpu_contexts = wgpu_image_map.lock().unwrap();
let context_data = webgpu_contexts.get_mut(&context_id).unwrap();
if staging_buffer.is_mapped() {
context_data.replace_presentation(PresentationStagingBuffer::new(
epoch,
staging_buffer,
));
} else {
context_data.return_staging_buffer(staging_buffer);
context_data.clear_presentation();
}
// update image in WR
compositor_api.update_image(
image_key,
configuration.into(),
SerializableImageData::External(image_data(context_id)),
Some(canvas_epoch),
);
},
);
}
/// Copies data from provided texture using `encoder_id` to the provided [`StagingBuffer`].
///
/// `callback` is guaranteed to be called.
///
/// Returns a [`StagingBuffer`] with the [`StagingBufferState::Mapped`] state
/// on success or [`StagingBufferState::Available`] on failure.
fn texture_download(
&self,
texture_id: TextureId,
encoder_id: CommandEncoderId,
mut staging_buffer: StagingBuffer,
config: ContextConfiguration,
callback: impl FnOnce(StagingBuffer) + Send + 'static,
) {
let Ok(command_buffer_id) =
staging_buffer.prepare_load_texture_command_buffer(texture_id, encoder_id, &config)
else {
return callback(staging_buffer);
};
let StagingBufferState::Available(buffer) = &staging_buffer.state else {
unreachable!("`prepare_load_texture_command_buffer` should make buffer available")
};
let buffer_id = staging_buffer.buffer_id;
let buffer_size = buffer.size;
{
let _guard = self.poller.lock();
let result = self
.global
.queue_submit(buffer.queue_id, &[command_buffer_id]);
if result.is_err() {
return callback(staging_buffer);
}
}
staging_buffer.state = match staging_buffer.state {
StagingBufferState::Available(buffer) => StagingBufferState::Mapping(buffer),
_ => unreachable!("`prepare_load_texture_command_buffer` should make buffer available"),
};
let map_callback = {
let token = self.poller.token();
Box::new(move |result: Result<(), BufferAccessError>| {
drop(token);
staging_buffer.state = match staging_buffer.state {
StagingBufferState::Mapping(buffer) => {
if let Ok((data, len)) = result.and_then(|_| {
staging_buffer.global.buffer_get_mapped_range(
staging_buffer.buffer_id,
0,
Some(buffer.size),
)
}) {
StagingBufferState::Mapped(MappedBuffer {
buffer,
data,
len,
image_size: config.size,
image_format: config.format,
is_opaque: config.is_opaque,
})
} else {
StagingBufferState::Available(buffer)
}
},
_ => {
unreachable!("Mapping buffer should have StagingBufferState::Mapping state")
},
};
callback(staging_buffer);
})
};
let map_op = BufferMapOperation {
host: HostMap::Read,
callback: Some(map_callback),
};
// error is handled by map_callback
let _ = self
.global
.buffer_map_async(buffer_id, 0, Some(buffer_size), map_op);
self.poller.wake();
}
pub(crate) fn destroy_context(&mut self, context_id: WebGPUContextId) {
self.wgpu_image_map
.lock()
.unwrap()
.remove(&context_id)
.unwrap()
.destroy(&self.script_sender, &self.compositor_api);
}
}

View file

@ -2,9 +2,9 @@
* License, v. 2.0. If a copy of the MPL was not distributed with this
* file, You can obtain one at https://mozilla.org/MPL/2.0/. */
use canvas_context::WGPUImageMap;
pub use canvas_context::{ContextData, WGPUExternalImages};
use log::warn;
use swapchain::WGPUImageMap;
pub use swapchain::{ContextData, WGPUExternalImages};
use webgpu_traits::{WebGPU, WebGPUMsg};
use wgpu_thread::WGPU;
pub use {wgpu_core as wgc, wgpu_types as wgt};
@ -19,7 +19,7 @@ use compositing_traits::{CrossProcessCompositorApi, WebrenderExternalImageRegist
use ipc_channel::ipc::{self, IpcReceiver};
use servo_config::pref;
pub mod swapchain;
pub mod canvas_context;
pub fn start_webgpu_thread(
compositor_api: CrossProcessCompositorApi,

View file

@ -1,596 +0,0 @@
/* This Source Code Form is subject to the terms of the Mozilla Public
* License, v. 2.0. If a copy of the MPL was not distributed with this
* file, You can obtain one at https://mozilla.org/MPL/2.0/. */
use std::ptr::NonNull;
use std::slice;
use std::sync::{Arc, Mutex};
use arrayvec::ArrayVec;
use base::Epoch;
use compositing_traits::{
CrossProcessCompositorApi, ExternalImageSource, SerializableImageData,
WebrenderExternalImageApi,
};
use euclid::default::Size2D;
use ipc_channel::ipc::IpcSender;
use log::{error, warn};
use pixels::{IpcSnapshot, Snapshot, SnapshotAlphaMode, SnapshotPixelFormat};
use rustc_hash::FxHashMap;
use serde::{Deserialize, Serialize};
use webgpu_traits::{
ContextConfiguration, Error, PRESENTATION_BUFFER_COUNT, WebGPUContextId, WebGPUMsg,
};
use webrender_api::units::DeviceIntSize;
use webrender_api::{
ExternalImageData, ExternalImageId, ExternalImageType, ImageDescriptor, ImageDescriptorFlags,
ImageFormat, ImageKey,
};
use wgpu_core::device::HostMap;
use wgpu_core::global::Global;
use wgpu_core::id;
use wgpu_core::resource::{BufferAccessError, BufferMapOperation};
use crate::wgt;
const DEFAULT_IMAGE_FORMAT: ImageFormat = ImageFormat::RGBA8;
pub type WGPUImageMap = Arc<Mutex<FxHashMap<WebGPUContextId, ContextData>>>;
/// Presentation id encodes current configuration and current image
/// so that async presentation does not update context with older data
#[derive(Clone, Copy, Debug, Deserialize, Eq, Hash, Ord, PartialEq, PartialOrd, Serialize)]
struct PresentationId(u64);
struct GPUPresentationBuffer {
global: Arc<Global>,
buffer_id: id::BufferId,
data: NonNull<u8>,
size: usize,
}
// This is safe because `GPUPresentationBuffer` holds exclusive access to ptr
unsafe impl Send for GPUPresentationBuffer {}
unsafe impl Sync for GPUPresentationBuffer {}
impl GPUPresentationBuffer {
fn new(global: Arc<Global>, buffer_id: id::BufferId, buffer_size: u64) -> Self {
let (data, size) = global
.buffer_get_mapped_range(buffer_id, 0, Some(buffer_size))
.unwrap();
GPUPresentationBuffer {
global,
buffer_id,
data,
size: size as usize,
}
}
fn slice(&self) -> &[u8] {
unsafe { slice::from_raw_parts(self.data.as_ptr(), self.size) }
}
}
impl Drop for GPUPresentationBuffer {
fn drop(&mut self) {
let _ = self.global.buffer_unmap(self.buffer_id);
}
}
#[derive(Default)]
pub struct WGPUExternalImages {
pub images: WGPUImageMap,
pub locked_ids: FxHashMap<WebGPUContextId, Vec<u8>>,
}
impl WebrenderExternalImageApi for WGPUExternalImages {
fn lock(&mut self, id: u64) -> (ExternalImageSource<'_>, Size2D<i32>) {
let id = WebGPUContextId(id);
let webgpu_contexts = self.images.lock().unwrap();
let context_data = webgpu_contexts.get(&id).unwrap();
let size = context_data.image_desc.size().cast_unit();
let data = if let Some(present_buffer) = context_data
.swap_chain
.as_ref()
.and_then(|swap_chain| swap_chain.data.as_ref())
{
present_buffer.slice().to_vec()
} else {
context_data.dummy_data()
};
self.locked_ids.insert(id, data);
(
ExternalImageSource::RawData(self.locked_ids.get(&id).unwrap().as_slice()),
size,
)
}
fn unlock(&mut self, id: u64) {
let id = WebGPUContextId(id);
self.locked_ids.remove(&id);
}
}
/// States of presentation buffer
#[derive(Clone, Copy, Debug, Default, Eq, Ord, PartialEq, PartialOrd)]
enum PresentationBufferState {
/// Initial state, buffer has yet to be created,
/// only its id is reserved
#[default]
Unassigned,
/// Buffer is already created and ready to be used immediately
Available,
/// Buffer is currently running mapAsync
Mapping,
/// Buffer is currently actively mapped to be used by wr
Mapped,
}
struct SwapChain {
device_id: id::DeviceId,
queue_id: id::QueueId,
data: Option<GPUPresentationBuffer>,
}
#[derive(Clone, Copy, Debug, PartialEq)]
pub struct WebGPUImageDescriptor(pub ImageDescriptor);
impl WebGPUImageDescriptor {
fn new(format: ImageFormat, size: DeviceIntSize, is_opaque: bool) -> Self {
let stride = ((size.width * format.bytes_per_pixel()) |
(wgt::COPY_BYTES_PER_ROW_ALIGNMENT as i32 - 1)) +
1;
Self(ImageDescriptor {
format,
size,
stride: Some(stride),
offset: 0,
flags: if is_opaque {
ImageDescriptorFlags::IS_OPAQUE
} else {
ImageDescriptorFlags::empty()
},
})
}
fn default(size: DeviceIntSize) -> Self {
Self::new(DEFAULT_IMAGE_FORMAT, size, false)
}
/// Returns true if needs image update (if it's changed)
fn update(&mut self, new: Self) -> bool {
if self.0 != new.0 {
self.0 = new.0;
true
} else {
false
}
}
fn buffer_stride(&self) -> i32 {
self.0
.stride
.expect("Stride should be set by WebGPUImageDescriptor")
}
fn buffer_size(&self) -> wgt::BufferAddress {
(self.buffer_stride() * self.0.size.height) as wgt::BufferAddress
}
fn size(&self) -> DeviceIntSize {
self.0.size
}
}
pub struct ContextData {
image_key: ImageKey,
image_desc: WebGPUImageDescriptor,
image_data: ExternalImageData,
buffer_ids: ArrayVec<(id::BufferId, PresentationBufferState), PRESENTATION_BUFFER_COUNT>,
/// If there is no associated swapchain the context is dummy (transparent black)
swap_chain: Option<SwapChain>,
/// Next presentation id to be returned
next_presentation_id: PresentationId,
/// Current id that is presented/configured
///
/// This value only grows
current_presentation_id: PresentationId,
}
impl ContextData {
/// Init ContextData as dummy (transparent black)
fn new(
context_id: WebGPUContextId,
image_key: ImageKey,
size: DeviceIntSize,
buffer_ids: ArrayVec<id::BufferId, PRESENTATION_BUFFER_COUNT>,
) -> Self {
let image_data = ExternalImageData {
id: ExternalImageId(context_id.0),
channel_index: 0,
image_type: ExternalImageType::Buffer,
normalized_uvs: false,
};
Self {
image_key,
image_desc: WebGPUImageDescriptor::default(size),
image_data,
swap_chain: None,
buffer_ids: buffer_ids
.iter()
.map(|&buffer_id| (buffer_id, PresentationBufferState::Unassigned))
.collect(),
current_presentation_id: PresentationId(0),
next_presentation_id: PresentationId(1),
}
}
fn dummy_data(&self) -> Vec<u8> {
vec![0; self.image_desc.buffer_size() as usize]
}
/// Returns id of available buffer
/// and sets state to PresentationBufferState::Mapping
fn get_available_buffer(&'_ mut self, global: &Arc<Global>) -> Option<id::BufferId> {
assert!(self.swap_chain.is_some());
if let Some((buffer_id, buffer_state)) = self
.buffer_ids
.iter_mut()
.find(|(_, state)| *state == PresentationBufferState::Available)
{
*buffer_state = PresentationBufferState::Mapping;
Some(*buffer_id)
} else if let Some((buffer_id, buffer_state)) = self
.buffer_ids
.iter_mut()
.find(|(_, state)| *state == PresentationBufferState::Unassigned)
{
*buffer_state = PresentationBufferState::Mapping;
let buffer_id = *buffer_id;
let buffer_desc = wgt::BufferDescriptor {
label: None,
size: self.image_desc.buffer_size(),
usage: wgt::BufferUsages::MAP_READ | wgt::BufferUsages::COPY_DST,
mapped_at_creation: false,
};
let _ = global.device_create_buffer(
self.swap_chain.as_ref().unwrap().device_id,
&buffer_desc,
Some(buffer_id),
);
Some(buffer_id)
} else {
error!("No available presentation buffer: {:?}", self.buffer_ids);
None
}
}
fn get_buffer_state(&mut self, buffer_id: id::BufferId) -> &mut PresentationBufferState {
&mut self
.buffer_ids
.iter_mut()
.find(|(id, _)| *id == buffer_id)
.expect("Presentation buffer should have associated state")
.1
}
fn unmap_old_buffer(&mut self, presentation_buffer: GPUPresentationBuffer) {
assert!(self.swap_chain.is_some());
let buffer_state = self.get_buffer_state(presentation_buffer.buffer_id);
assert_eq!(*buffer_state, PresentationBufferState::Mapped);
*buffer_state = PresentationBufferState::Available;
drop(presentation_buffer);
}
fn destroy_swapchain(&mut self, global: &Arc<Global>) {
drop(self.swap_chain.take());
// free all buffers
for (buffer_id, buffer_state) in &mut self.buffer_ids {
match buffer_state {
PresentationBufferState::Unassigned => {
/* These buffer were not yet created in wgpu */
},
_ => {
global.buffer_drop(*buffer_id);
},
}
*buffer_state = PresentationBufferState::Unassigned;
}
}
fn destroy(
mut self,
global: &Arc<Global>,
script_sender: &IpcSender<WebGPUMsg>,
compositor_api: &CrossProcessCompositorApi,
) {
self.destroy_swapchain(global);
for (buffer_id, _) in self.buffer_ids {
if let Err(e) = script_sender.send(WebGPUMsg::FreeBuffer(buffer_id)) {
warn!("Unable to send FreeBuffer({:?}) ({:?})", buffer_id, e);
};
}
compositor_api.delete_image(self.image_key);
}
/// Returns true if presentation id was updated (was newer)
fn check_and_update_presentation_id(&mut self, presentation_id: PresentationId) -> bool {
if presentation_id > self.current_presentation_id {
self.current_presentation_id = presentation_id;
true
} else {
false
}
}
/// Returns new presentation id
fn next_presentation_id(&mut self) -> PresentationId {
let res = PresentationId(self.next_presentation_id.0);
self.next_presentation_id.0 += 1;
res
}
}
impl crate::WGPU {
pub(crate) fn create_context(
&self,
context_id: WebGPUContextId,
image_key: ImageKey,
size: DeviceIntSize,
buffer_ids: ArrayVec<id::BufferId, PRESENTATION_BUFFER_COUNT>,
) {
let context_data = ContextData::new(context_id, image_key, size, buffer_ids);
self.compositor_api.add_image(
image_key,
context_data.image_desc.0,
SerializableImageData::External(context_data.image_data),
);
assert!(
self.wgpu_image_map
.lock()
.unwrap()
.insert(context_id, context_data)
.is_none(),
"Context should be created only once!"
);
}
pub(crate) fn get_image(&self, context_id: WebGPUContextId) -> IpcSnapshot {
let webgpu_contexts = self.wgpu_image_map.lock().unwrap();
let context_data = webgpu_contexts.get(&context_id).unwrap();
let size = context_data.image_desc.size().cast().cast_unit();
let data = if let Some(present_buffer) = context_data
.swap_chain
.as_ref()
.and_then(|swap_chain| swap_chain.data.as_ref())
{
let format = match context_data.image_desc.0.format {
ImageFormat::RGBA8 => SnapshotPixelFormat::RGBA,
ImageFormat::BGRA8 => SnapshotPixelFormat::BGRA,
_ => unimplemented!(),
};
let alpha_mode = if context_data.image_desc.0.is_opaque() {
SnapshotAlphaMode::AsOpaque {
premultiplied: false,
}
} else {
SnapshotAlphaMode::Transparent {
premultiplied: true,
}
};
Snapshot::from_vec(size, format, alpha_mode, present_buffer.slice().to_vec())
} else {
Snapshot::cleared(size)
};
data.as_ipc()
}
pub(crate) fn update_context(
&self,
context_id: WebGPUContextId,
size: DeviceIntSize,
config: Option<ContextConfiguration>,
) {
let mut webgpu_contexts = self.wgpu_image_map.lock().unwrap();
let context_data = webgpu_contexts.get_mut(&context_id).unwrap();
let presentation_id = context_data.next_presentation_id();
context_data.check_and_update_presentation_id(presentation_id);
// If configuration is not provided
// the context will be dummy/empty until recreation
let needs_image_update = if let Some(config) = config {
let new_image_desc =
WebGPUImageDescriptor::new(config.format(), size, config.is_opaque);
let needs_swapchain_rebuild = context_data.swap_chain.is_none() ||
new_image_desc.buffer_size() != context_data.image_desc.buffer_size();
if needs_swapchain_rebuild {
context_data.destroy_swapchain(&self.global);
context_data.swap_chain = Some(SwapChain {
device_id: config.device_id,
queue_id: config.queue_id,
data: None,
});
}
context_data.image_desc.update(new_image_desc)
} else {
context_data.destroy_swapchain(&self.global);
context_data
.image_desc
.update(WebGPUImageDescriptor::default(size))
};
if needs_image_update {
self.compositor_api.update_image(
context_data.image_key,
context_data.image_desc.0,
SerializableImageData::External(context_data.image_data),
None,
);
}
}
/// Copies data async from provided texture using encoder_id to available staging presentation buffer
pub(crate) fn swapchain_present(
&mut self,
context_id: WebGPUContextId,
encoder_id: id::Id<id::markers::CommandEncoder>,
texture_id: id::Id<id::markers::Texture>,
canvas_epoch: Option<Epoch>,
) -> Result<(), Box<dyn std::error::Error>> {
fn err<T: std::error::Error + 'static>(e: Option<T>) -> Result<(), T> {
if let Some(error) = e {
Err(error)
} else {
Ok(())
}
}
let global = &self.global;
let device_id;
let queue_id;
let buffer_id;
let image_desc;
let presentation_id;
{
if let Some(context_data) = self.wgpu_image_map.lock().unwrap().get_mut(&context_id) {
let Some(swap_chain) = context_data.swap_chain.as_ref() else {
return Ok(());
};
device_id = swap_chain.device_id;
queue_id = swap_chain.queue_id;
buffer_id = context_data.get_available_buffer(global).unwrap();
image_desc = context_data.image_desc;
presentation_id = context_data.next_presentation_id();
} else {
return Ok(());
}
}
let comm_desc = wgt::CommandEncoderDescriptor { label: None };
let (encoder_id, error) =
global.device_create_command_encoder(device_id, &comm_desc, Some(encoder_id));
err(error)?;
let buffer_cv = wgt::TexelCopyBufferInfo {
buffer: buffer_id,
layout: wgt::TexelCopyBufferLayout {
offset: 0,
bytes_per_row: Some(image_desc.buffer_stride() as u32),
rows_per_image: None,
},
};
let texture_cv = wgt::TexelCopyTextureInfo {
texture: texture_id,
mip_level: 0,
origin: wgt::Origin3d::ZERO,
aspect: wgt::TextureAspect::All,
};
let copy_size = wgt::Extent3d {
width: image_desc.size().width as u32,
height: image_desc.size().height as u32,
depth_or_array_layers: 1,
};
global.command_encoder_copy_texture_to_buffer(
encoder_id,
&texture_cv,
&buffer_cv,
&copy_size,
)?;
let (command_buffer_id, error) =
global.command_encoder_finish(encoder_id, &wgt::CommandBufferDescriptor::default());
err(error)?;
{
let _guard = self.poller.lock();
global
.queue_submit(queue_id, &[command_buffer_id])
.map_err(|(_, error)| Error::from_error(error))?;
}
let callback = {
let global = Arc::clone(&self.global);
let wgpu_image_map = Arc::clone(&self.wgpu_image_map);
let compositor_api = self.compositor_api.clone();
let token = self.poller.token();
Box::new(move |result| {
drop(token);
update_wr_image(
result,
global,
buffer_id,
wgpu_image_map,
context_id,
compositor_api,
image_desc,
presentation_id,
canvas_epoch,
);
})
};
let map_op = BufferMapOperation {
host: HostMap::Read,
callback: Some(callback),
};
global.buffer_map_async(buffer_id, 0, Some(image_desc.buffer_size()), map_op)?;
self.poller.wake();
Ok(())
}
pub(crate) fn destroy_context(&mut self, context_id: WebGPUContextId) {
self.wgpu_image_map
.lock()
.unwrap()
.remove(&context_id)
.unwrap()
.destroy(&self.global, &self.script_sender, &self.compositor_api);
}
}
#[allow(clippy::too_many_arguments)]
fn update_wr_image(
result: Result<(), BufferAccessError>,
global: Arc<Global>,
buffer_id: id::BufferId,
wgpu_image_map: WGPUImageMap,
context_id: WebGPUContextId,
compositor_api: CrossProcessCompositorApi,
image_desc: WebGPUImageDescriptor,
presentation_id: PresentationId,
canvas_epoch: Option<Epoch>,
) {
match result {
Ok(()) => {
if let Some(context_data) = wgpu_image_map.lock().unwrap().get_mut(&context_id) {
if !context_data.check_and_update_presentation_id(presentation_id) {
let buffer_state = context_data.get_buffer_state(buffer_id);
if *buffer_state == PresentationBufferState::Mapping {
let _ = global.buffer_unmap(buffer_id);
*buffer_state = PresentationBufferState::Available;
}
// throw away all work, because we are too old
return;
}
assert_eq!(image_desc, context_data.image_desc);
let buffer_state = context_data.get_buffer_state(buffer_id);
assert_eq!(*buffer_state, PresentationBufferState::Mapping);
*buffer_state = PresentationBufferState::Mapped;
let presentation_buffer =
GPUPresentationBuffer::new(global, buffer_id, image_desc.buffer_size());
let Some(swap_chain) = context_data.swap_chain.as_mut() else {
return;
};
let old_presentation_buffer = swap_chain.data.replace(presentation_buffer);
compositor_api.update_image(
context_data.image_key,
context_data.image_desc.0,
SerializableImageData::External(context_data.image_data),
canvas_epoch,
);
if let Some(old_presentation_buffer) = old_presentation_buffer {
context_data.unmap_old_buffer(old_presentation_buffer)
}
} else {
error!("WebGPU Context {:?} is destroyed", context_id);
}
},
_ => error!("Could not map buffer({:?})", buffer_id),
}
}

View file

@ -36,8 +36,8 @@ use wgpu_types::MemoryHints;
use wgt::InstanceDescriptor;
pub use {wgpu_core as wgc, wgpu_types as wgt};
use crate::canvas_context::WGPUImageMap;
use crate::poll_thread::Poller;
use crate::swapchain::WGPUImageMap;
#[derive(Eq, Hash, PartialEq)]
pub(crate) struct DeviceScope {
@ -509,32 +509,19 @@ impl WGPU {
};
self.create_context(context_id, image_key, size, buffer_ids);
},
WebGPURequest::UpdateContext {
WebGPURequest::Present {
context_id,
pending_texture,
size,
configuration,
} => {
self.update_context(context_id, size, configuration);
},
WebGPURequest::SwapChainPresent {
context_id,
texture_id,
encoder_id,
canvas_epoch,
} => {
let result = self.swapchain_present(
context_id,
encoder_id,
texture_id,
canvas_epoch,
);
if let Err(e) = result {
log::error!("Error occured in SwapChainPresent: {e:?}");
}
},
WebGPURequest::GetImage { context_id, sender } => {
sender.send(self.get_image(context_id)).unwrap()
self.present(context_id, pending_texture, size, canvas_epoch);
},
WebGPURequest::GetImage {
context_id,
pending_texture,
sender,
} => self.get_image(context_id, pending_texture, sender),
WebGPURequest::ValidateTextureDescriptor {
device_id,
texture_id,
@ -1185,25 +1172,22 @@ impl WGPU {
let device_scope = devices
.get_mut(&device_id)
.expect("Device should not be dropped by this point");
if let Some(error_scope_stack) = &mut device_scope.error_scope_stack {
if let Some(error_scope) = error_scope_stack.pop() {
if let Err(e) = sender.send(Ok(
// TODO: Do actual selection instead of selecting first error
error_scope.errors.first().cloned(),
)) {
warn!(
"Unable to send {:?} to poperrorscope: {e:?}",
error_scope.errors
);
let result =
if let Some(error_scope_stack) = &mut device_scope.error_scope_stack {
if let Some(error_scope) = error_scope_stack.pop() {
Ok(
// TODO: Do actual selection instead of selecting first error
error_scope.errors.first().cloned(),
)
} else {
Err(PopError::Empty)
}
} else if let Err(e) = sender.send(Err(PopError::Empty)) {
warn!("Unable to send PopError::Empty: {e:?}");
}
} else {
// device lost
if let Err(e) = sender.send(Err(PopError::Lost)) {
warn!("Unable to send PopError::Lost due {e:?}");
}
} else {
// This means the device has been lost.
Err(PopError::Lost)
};
if let Err(error) = sender.send(result) {
warn!("Error while sending PopErrorScope result: {error}");
}
},
WebGPURequest::ComputeGetBindGroupLayout {