libservo: Add a WebView::take_screenshot() API and use it for reftests (#39583)

This change adds a new API to the `WebView` for capturing screenshots.
This makes it possible to:

 - use the reftest waiting infrastructure via the API
   easily.
 - take more than a single screenshot in one Servo run.
 - take screenshots, but still paint the `WebView` normally prior
   to the moment that the screenshot is ready, instead of preventing
   all non-screenshot-ready paints while taking a screenshot.

In addition, the previous infrastructure, the `wait_for_stable_image`
option is removed completely.

Testing: This change is tested by the passing of the WPT tests,
as they commonly use the screenshot feature.

Signed-off-by: Martin Robinson <mrobinson@igalia.com>
Co-authored-by: Delan Azabani <dazabani@igalia.com>
This commit is contained in:
Martin Robinson 2025-09-30 13:39:47 +02:00 committed by GitHub
parent 6995e60ee1
commit 6ffc0cd482
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
27 changed files with 701 additions and 468 deletions

2
Cargo.lock generated
View file

@ -1478,6 +1478,7 @@ dependencies = [
"embedder_traits",
"euclid",
"gleam",
"image",
"ipc-channel",
"libc",
"log",
@ -5001,6 +5002,7 @@ dependencies = [
"gleam",
"gstreamer",
"http 1.3.1",
"image",
"ipc-channel",
"keyboard-types",
"layout",

View file

@ -31,6 +31,7 @@ dpi = { workspace = true }
embedder_traits = { workspace = true }
euclid = { workspace = true }
gleam = { workspace = true }
image = { workspace = true }
ipc-channel = { workspace = true }
libc = { workspace = true }
log = { workspace = true }

View file

@ -25,8 +25,11 @@ use compositing_traits::{
use constellation_traits::{EmbedderToConstellationMessage, PaintMetricEvent};
use crossbeam_channel::Sender;
use dpi::PhysicalSize;
use embedder_traits::{CompositorHitTestResult, InputEvent, ShutdownState, ViewportDetails};
use embedder_traits::{
CompositorHitTestResult, InputEvent, ScreenshotCaptureError, ShutdownState, ViewportDetails,
};
use euclid::{Point2D, Rect, Scale, Size2D, Transform3D};
use image::RgbaImage;
use ipc_channel::ipc::{self, IpcSharedMemory};
use log::{debug, info, trace, warn};
use pixels::{CorsStatus, ImageFrame, ImageMetadata, PixelFormat, RasterImage};
@ -36,7 +39,7 @@ use profile_traits::mem::{
use profile_traits::time::{self as profile_time, ProfilerCategory};
use profile_traits::{path, time_profile};
use rustc_hash::{FxHashMap, FxHashSet};
use servo_config::{opts, pref};
use servo_config::pref;
use servo_geometry::DeviceIndependentPixel;
use style_traits::CSSPixel;
use webrender::{CaptureBits, RenderApi, Transaction};
@ -54,29 +57,10 @@ use webrender_api::{
use crate::InitialCompositorState;
use crate::refresh_driver::RefreshDriver;
use crate::screenshot::ScreenshotTaker;
use crate::webview_manager::WebViewManager;
use crate::webview_renderer::{PinchZoomResult, UnknownWebView, WebViewRenderer};
#[derive(Debug, PartialEq)]
pub enum UnableToComposite {
NotReadyToPaintImage(NotReadyToPaint),
}
#[derive(Debug, PartialEq)]
pub enum NotReadyToPaint {
JustNotifiedConstellation,
WaitingOnConstellation,
}
/// Holds the state when running reftests that determines when it is
/// safe to save the output image.
#[derive(Clone, Copy, Debug, PartialEq)]
enum ReadyState {
Unknown,
WaitingForConstellationReply,
ReadyToSaveImage,
}
/// An option to control what kind of WebRender debugging is enabled while Servo is running.
#[derive(Clone)]
pub enum WebRenderDebugOption {
@ -137,10 +121,6 @@ pub struct IOCompositor {
/// Tracks whether or not the view needs to be repainted.
needs_repaint: Cell<RepaintReason>,
/// Used by the logic that determines when it is safe to output an
/// image for the reftest framework.
ready_to_save_state: ReadyState,
/// The webrender renderer.
webrender: Option<webrender::Renderer>,
@ -150,6 +130,9 @@ pub struct IOCompositor {
/// The number of frames pending to receive from WebRender.
pending_frames: Cell<usize>,
/// A [`ScreenshotTaker`] responsible for handling all screenshot requests.
screenshot_taker: ScreenshotTaker,
/// A handle to the memory profiler which will automatically unregister
/// when it's dropped.
_mem_profiler_registration: ProfilerRegistration,
@ -333,10 +316,10 @@ impl IOCompositor {
})),
webview_renderers: WebViewManager::default(),
needs_repaint: Cell::default(),
ready_to_save_state: ReadyState::Unknown,
webrender: Some(state.webrender),
rendering_context: state.rendering_context,
pending_frames: Cell::new(0),
pending_frames: Default::default(),
screenshot_taker: Default::default(),
_mem_profiler_registration: registration,
};
@ -358,6 +341,10 @@ impl IOCompositor {
}
}
pub(crate) fn rendering_context(&self) -> &dyn RenderingContext {
&*self.rendering_context
}
pub fn rendering_context_size(&self) -> Size2D<u32, DevicePixel> {
self.rendering_context.size2d()
}
@ -373,7 +360,11 @@ impl IOCompositor {
}
}
fn set_needs_repaint(&self, reason: RepaintReason) {
pub(crate) fn webview_renderer(&self, webview_id: WebViewId) -> Option<&WebViewRenderer> {
self.webview_renderers.get(webview_id)
}
pub(crate) fn set_needs_repaint(&self, reason: RepaintReason) {
let mut needs_repaint = self.needs_repaint.get();
needs_repaint.insert(reason);
self.needs_repaint.set(needs_repaint);
@ -496,18 +487,6 @@ impl IOCompositor {
};
webview_renderer.on_touch_event_processed(result);
},
CompositorMsg::IsReadyToSaveImageReply(is_ready) => {
assert_eq!(
self.ready_to_save_state,
ReadyState::WaitingForConstellationReply
);
if is_ready && self.pending_frames.get() == 0 {
self.ready_to_save_state = ReadyState::ReadyToSaveImage;
} else {
self.ready_to_save_state = ReadyState::Unknown;
}
self.set_needs_repaint(RepaintReason::ReadyForScreenshot);
},
CompositorMsg::SetThrottled(webview_id, pipeline_id, throttled) => {
let Some(webview_renderer) = self.webview_renderers.get_mut(webview_id) else {
@ -532,14 +511,8 @@ impl IOCompositor {
}
},
CompositorMsg::NewWebRenderFrameReady(_document_id, recomposite_needed) => {
self.handle_new_webrender_frame_ready(recomposite_needed);
},
CompositorMsg::LoadComplete(_) => {
if opts::get().wait_for_stable_image {
self.set_needs_repaint(RepaintReason::ReadyForScreenshot);
}
CompositorMsg::NewWebRenderFrameReady(..) => {
unreachable!("New WebRender frames should be handled in the caller.");
},
CompositorMsg::SendInitialTransaction(webview_id, pipeline_id) => {
@ -697,7 +670,10 @@ impl IOCompositor {
let mut global = self.global.borrow_mut();
global.frame_delayer.set_pending_frame(true);
if global.frame_delayer.needs_new_frame() {
if !global.frame_delayer.needs_new_frame() {
return;
}
let mut transaction = Transaction::new();
self.generate_frame(&mut transaction, RenderReasons::SCENE);
global.send_transaction(transaction);
@ -709,7 +685,8 @@ impl IOCompositor {
),
);
global.frame_delayer.set_pending_frame(false);
}
self.screenshot_taker
.prepare_screenshot_requests_for_render(self)
},
CompositorMsg::GenerateImageKey(sender) => {
@ -759,6 +736,8 @@ impl IOCompositor {
waiting_pipelines,
),
);
self.screenshot_taker
.prepare_screenshot_requests_for_render(self);
}
global.send_transaction(txn);
@ -821,6 +800,13 @@ impl IOCompositor {
webview.set_viewport_description(viewport_description);
}
},
CompositorMsg::ScreenshotReadinessReponse(webview_id, pipelines_and_epochs) => {
self.screenshot_taker.handle_screenshot_readiness_reply(
webview_id,
pipelines_and_epochs,
self,
);
},
}
}
@ -860,10 +846,6 @@ impl IOCompositor {
rendering_group_id,
);
},
CompositorMsg::NewWebRenderFrameReady(..) => {
// Subtract from the number of pending frames, but do not do any compositing.
self.pending_frames.set(self.pending_frames.get() - 1);
},
_ => {
debug!("Ignoring message ({:?} while shutting down", msg);
},
@ -895,8 +877,8 @@ impl IOCompositor {
/// Queue a new frame in the transaction and increase the pending frames count.
pub(crate) fn generate_frame(&self, transaction: &mut Transaction, reason: RenderReasons) {
self.pending_frames.set(self.pending_frames.get() + 1);
transaction.generate_frame(0, true /* present */, reason);
self.pending_frames.set(self.pending_frames.get() + 1);
}
/// Set the root pipeline for our WebRender scene to a display list that consists of an iframe
@ -1188,78 +1170,18 @@ impl IOCompositor {
.any(WebViewRenderer::animation_callbacks_running)
}
/// Query the constellation to see if the current compositor
/// output matches the current frame tree output, and if the
/// associated script threads are idle.
fn is_ready_to_paint_image_output(&mut self) -> Result<(), NotReadyToPaint> {
match self.ready_to_save_state {
ReadyState::Unknown => {
// Unsure if the output image is stable.
// Collect the currently painted epoch of each pipeline that is
// complete (i.e. has *all* layers painted to the requested epoch).
// This gets sent to the constellation for comparison with the current
// frame tree.
let mut pipeline_epochs = FxHashMap::default();
for id in self
.webview_renderers
.iter()
.flat_map(WebViewRenderer::pipeline_ids)
{
if let Some(WebRenderEpoch(epoch)) = self
.webrender
.as_ref()
.and_then(|wr| wr.current_epoch(self.webrender_document(), id.into()))
{
let epoch = Epoch(epoch);
pipeline_epochs.insert(*id, epoch);
}
}
// Pass the pipeline/epoch states to the constellation and check
// if it's safe to output the image.
let msg = EmbedderToConstellationMessage::IsReadyToSaveImage(pipeline_epochs);
if let Err(e) = self.global.borrow().constellation_sender.send(msg) {
warn!("Sending ready to save to constellation failed ({:?}).", e);
}
self.ready_to_save_state = ReadyState::WaitingForConstellationReply;
Err(NotReadyToPaint::JustNotifiedConstellation)
},
ReadyState::WaitingForConstellationReply => {
// If waiting on a reply from the constellation to the last
// query if the image is stable, then assume not ready yet.
Err(NotReadyToPaint::WaitingOnConstellation)
},
ReadyState::ReadyToSaveImage => {
// Constellation has replied at some point in the past
// that the current output image is stable and ready
// for saving.
// Reset the flag so that we check again in the future
// TODO: only reset this if we load a new document?
self.ready_to_save_state = ReadyState::Unknown;
Ok(())
},
}
}
/// Render the WebRender scene to the active `RenderingContext`. If successful, trigger
/// the next round of animations. Return false if unable to render.
pub fn render(&mut self) -> bool {
/// Render the WebRender scene to the active `RenderingContext`.
pub fn render(&mut self) {
self.global
.borrow()
.refresh_driver
.notify_will_paint(self.webview_renderers.iter());
if let Err(error) = self.render_inner() {
warn!("Unable to render: {error:?}");
return false;
}
self.render_inner();
// We've painted the default target, which means that from the embedder's perspective,
// the scene no longer needs to be repainted.
self.needs_repaint.set(RepaintReason::empty());
true
}
/// Render the WebRender scene to the shared memory, without updating other state of this
@ -1268,8 +1190,8 @@ impl IOCompositor {
&mut self,
webview_id: WebViewId,
page_rect: Option<Rect<f32, CSSPixel>>,
) -> Result<Option<RasterImage>, UnableToComposite> {
self.render_inner()?;
) -> Option<RasterImage> {
self.render_inner();
let size = self.rendering_context.size2d().to_i32();
let rect = if let Some(rect) = page_rect {
@ -1294,8 +1216,7 @@ impl IOCompositor {
DeviceIntRect::from_origin_and_size(Point2D::origin(), size)
};
Ok(self
.rendering_context
self.rendering_context
.read_to_image(rect)
.map(|image| RasterImage {
metadata: ImageMetadata {
@ -1312,11 +1233,11 @@ impl IOCompositor {
bytes: ipc::IpcSharedMemory::from_bytes(&image),
id: None,
cors_status: CorsStatus::Safe,
}))
})
}
#[servo_tracing::instrument(skip_all)]
fn render_inner(&mut self) -> Result<(), UnableToComposite> {
fn render_inner(&mut self) {
if let Err(err) = self.rendering_context.make_current() {
warn!("Failed to make the rendering context current: {:?}", err);
}
@ -1326,12 +1247,6 @@ impl IOCompositor {
webrender.update();
}
if opts::get().wait_for_stable_image {
if let Err(result) = self.is_ready_to_paint_image_output() {
return Err(UnableToComposite::NotReadyToPaintImage(result));
}
}
self.rendering_context.prepare_for_rendering();
let time_profiler_chan = self.global.borrow().time_profiler_chan.clone();
@ -1353,7 +1268,7 @@ impl IOCompositor {
);
self.send_pending_paint_metrics_messages_after_composite();
Ok(())
self.screenshot_taker.maybe_take_screenshots(self);
}
/// Send all pending paint metrics messages after a composite operation, which may advance
@ -1480,22 +1395,18 @@ impl IOCompositor {
#[servo_tracing::instrument(skip_all)]
pub fn handle_messages(&mut self, mut messages: Vec<CompositorMsg>) {
// Check for new messages coming from the other threads in the system.
let mut found_recomposite_msg = false;
messages.retain(|message| {
match message {
CompositorMsg::NewWebRenderFrameReady(..) if found_recomposite_msg => {
// Only take one of duplicate NewWebRendeFrameReady messages, but do subtract
// one frame from the pending frames.
// Pull out the `NewWebRenderFrameReady` messages from the list of messages and handle them
// at the end of this function. This prevents overdraw when more than a single message of
// this type of received. In addition, if any of these frames need a repaint, that reflected
// when calling `handle_new_webrender_frame_ready`.
let mut repaint_needed = false;
messages.retain(|message| match message {
CompositorMsg::NewWebRenderFrameReady(_, need_repaint) => {
self.pending_frames.set(self.pending_frames.get() - 1);
repaint_needed |= need_repaint;
false
},
CompositorMsg::NewWebRenderFrameReady(..) => {
found_recomposite_msg = true;
true
},
_ => true,
}
});
for message in messages {
@ -1504,6 +1415,8 @@ impl IOCompositor {
return;
}
}
self.handle_new_webrender_frame_ready(repaint_needed);
}
#[servo_tracing::instrument(skip_all)]
@ -1704,14 +1617,40 @@ impl IOCompositor {
}
}
fn handle_new_webrender_frame_ready(&mut self, recomposite_needed: bool) {
self.pending_frames.set(self.pending_frames.get() - 1);
if recomposite_needed {
fn handle_new_webrender_frame_ready(&mut self, repaint_needed: bool) {
if repaint_needed {
self.refresh_cursor();
}
if recomposite_needed || self.animation_callbacks_running() {
if repaint_needed || self.animation_callbacks_running() {
self.set_needs_repaint(RepaintReason::NewWebRenderFrame);
}
// If we received a new frame and a repaint isn't necessary, it may be that this
// is the last frame that was pending. In that case, trigger a manual repaint so
// that the screenshot can be taken at the end of the repaint procedure.
if !repaint_needed {
self.screenshot_taker
.maybe_trigger_paint_for_screenshot(self);
}
}
/// Whether or not the renderer is waiting on a frame, either because it has been sent
/// to WebRender and is not ready yet or because the [`FrameDelayer`] is delaying a frame
/// waiting for asynchronous (canvas) image updates to complete.
pub(crate) fn has_pending_frames(&self) -> bool {
self.pending_frames.get() != 0 || self.global.borrow().frame_delayer.pending_frame
}
pub fn request_screenshot(
&self,
webview_id: WebViewId,
callback: Box<dyn FnOnce(Result<RgbaImage, ScreenshotCaptureError>) + 'static>,
) {
self.screenshot_taker
.request_screenshot(webview_id, callback);
let _ = self.global.borrow().constellation_sender.send(
EmbedderToConstellationMessage::RequestScreenshotReadiness(webview_id),
);
}
}

View file

@ -24,6 +24,7 @@ mod tracing;
mod compositor;
mod refresh_driver;
mod screenshot;
mod touch;
mod webview_manager;
mod webview_renderer;

View file

@ -0,0 +1,187 @@
/* 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::cell::RefCell;
use std::rc::Rc;
use base::Epoch;
use base::id::{PipelineId, WebViewId};
use embedder_traits::ScreenshotCaptureError;
use image::RgbaImage;
use rustc_hash::FxHashMap;
use crate::IOCompositor;
use crate::compositor::RepaintReason;
pub(crate) struct ScreenshotRequest {
webview_id: WebViewId,
phase: ScreenshotRequestPhase,
callback: Box<dyn FnOnce(Result<RgbaImage, ScreenshotCaptureError>) + 'static>,
}
/// Screenshots requests happen in three phases:
#[derive(PartialEq)]
pub(crate) enum ScreenshotRequestPhase {
/// A request is sent to the Constellation, asking each Pipeline in a WebView,
/// to report the display list epoch to render for the screenshot. Each Pipeline
/// will wait to send an epoch that happens after the Pipeline is ready in a
/// variety of ways:
///
/// - The `load` event has fired.
/// - All render blocking elements are no longer blocking the rendering.
/// - All images are loaded and displayed.
/// - All web fonts are loaded.
/// - The `reftest-wait` and `test-wait` classes have been removed from the root element.
/// - The rendering is up-to-date
///
/// When all Pipelines have reported this epoch to the Constellation it sends a
/// ScreenshotReadinessResponse back to the renderer.
ConstellationRequest,
/// The renderer has received the ScreenshotReadinessReponse from the Constellation
/// and is now waiting for all display lists to be received from the Pipelines and
/// sent to WebRender.
WaitingOnPipelineDisplayLists(Rc<FxHashMap<PipelineId, Epoch>>),
/// Once the renderer has received all of the Pipeline display lists necessary to take
/// the screenshot and uploaded them to WebRender, it waits for an appropriate frame to
/// be ready. Currently this just waits for the [`FrameDelayer`] to stop delaying frames
/// and for there to be no pending WebRender frames (ones sent to WebRender that are not
/// ready yet). Once that happens, and a potential extra repaint is triggered, the renderer
/// will take the screenshot and fufill the request.
WaitingOnFrame,
}
#[derive(Default)]
pub(crate) struct ScreenshotTaker {
/// A vector of pending screenshots to be taken. These will be resolved once the
/// pages have finished loading all content and the rendering reflects the finished
/// state. See [`ScreenshotRequestPhase`] for more information.
requests: RefCell<Vec<ScreenshotRequest>>,
}
impl ScreenshotTaker {
pub(crate) fn request_screenshot(
&self,
webview_id: WebViewId,
callback: Box<dyn FnOnce(Result<RgbaImage, ScreenshotCaptureError>) + 'static>,
) {
self.requests.borrow_mut().push(ScreenshotRequest {
webview_id,
phase: ScreenshotRequestPhase::ConstellationRequest,
callback,
});
}
pub(crate) fn handle_screenshot_readiness_reply(
&self,
webview_id: WebViewId,
expected_epochs: FxHashMap<PipelineId, Epoch>,
renderer: &IOCompositor,
) {
let expected_epochs = Rc::new(expected_epochs);
for screenshot_request in self.requests.borrow_mut().iter_mut() {
if screenshot_request.webview_id != webview_id ||
screenshot_request.phase != ScreenshotRequestPhase::ConstellationRequest
{
continue;
}
screenshot_request.phase =
ScreenshotRequestPhase::WaitingOnPipelineDisplayLists(expected_epochs.clone());
}
// Maybe when the message is received, the renderer already has the all of the necessary
// display lists from the Pipelines. In that case, the renderer should move immediately
// to the next phase of the screenshot state machine.
self.prepare_screenshot_requests_for_render(renderer);
}
pub(crate) fn prepare_screenshot_requests_for_render(&self, renderer: &IOCompositor) {
let mut any_became_ready = false;
for screenshot_request in self.requests.borrow_mut().iter_mut() {
let ScreenshotRequestPhase::WaitingOnPipelineDisplayLists(pipelines) =
&screenshot_request.phase
else {
continue;
};
let Some(webview) = renderer.webview_renderer(screenshot_request.webview_id) else {
continue;
};
if pipelines.iter().all(|(pipeline_id, expected_epoch)| {
webview
.pipelines
.get(pipeline_id)
.and_then(|pipeline| pipeline.display_list_epoch)
.is_some_and(|epoch| epoch >= *expected_epoch)
}) {
screenshot_request.phase = ScreenshotRequestPhase::WaitingOnFrame;
any_became_ready = true;
}
}
// If there are now screenshots waiting on a frame, and there are no pending frames,
// immediately trigger a repaint so that screenshots can be taken when the repaint
// is done.
if any_became_ready {
self.maybe_trigger_paint_for_screenshot(renderer);
}
}
pub(crate) fn maybe_trigger_paint_for_screenshot(&self, renderer: &IOCompositor) {
if renderer.has_pending_frames() {
return;
}
if self.requests.borrow().iter().any(|screenshot_request| {
matches!(
screenshot_request.phase,
ScreenshotRequestPhase::WaitingOnFrame
)
}) {
renderer.set_needs_repaint(RepaintReason::ReadyForScreenshot);
}
}
pub(crate) fn maybe_take_screenshots(&self, renderer: &IOCompositor) {
if renderer.has_pending_frames() {
return;
}
let mut requests = self.requests.borrow_mut();
if requests.is_empty() {
return;
}
// TODO: This can eventually just be `extract_if`. We need to have ownership
// of the ScreenshotRequest in order to call the `FnOnce` callabck.
let screenshots = requests.drain(..);
*requests = screenshots
.filter_map(|screenshot_request| {
if !matches!(
screenshot_request.phase,
ScreenshotRequestPhase::WaitingOnFrame
) {
return Some(screenshot_request);
}
let callback = screenshot_request.callback;
let Some(webview_renderer) =
renderer.webview_renderer(screenshot_request.webview_id)
else {
callback(Err(ScreenshotCaptureError::WebViewDoesNotExist));
return None;
};
let result = renderer
.rendering_context()
.read_to_image(webview_renderer.rect.to_i32())
.ok_or(ScreenshotCaptureError::CouldNotReadImage);
callback(result);
None
})
.collect();
}
}

View file

@ -34,11 +34,9 @@ mod from_constellation {
Self::CreateOrUpdateWebView(..) => target!("CreateOrUpdateWebView"),
Self::RemoveWebView(..) => target!("RemoveWebView"),
Self::TouchEventProcessed(..) => target!("TouchEventProcessed"),
Self::IsReadyToSaveImageReply(..) => target!("IsReadyToSaveImageReply"),
Self::SetThrottled(..) => target!("SetThrottled"),
Self::NewWebRenderFrameReady(..) => target!("NewWebRenderFrameReady"),
Self::PipelineExited(..) => target!("PipelineExited"),
Self::LoadComplete(..) => target!("LoadComplete"),
Self::SendInitialTransaction(..) => target!("SendInitialTransaction"),
Self::SendScrollNode(..) => target!("SendScrollNode"),
Self::SendDisplayList { .. } => target!("SendDisplayList"),
@ -54,6 +52,7 @@ mod from_constellation {
Self::Viewport(..) => target!("Viewport"),
Self::GenerateImageKeysForPipeline(..) => target!("GenerateImageKeysForPipeline"),
Self::DelayNewFrameForCanvas(..) => target!("DelayFramesForCanvas"),
Self::ScreenshotReadinessReponse(..) => target!("ScreenshotReadinessResponse"),
}
}
}

View file

@ -3,7 +3,7 @@
* file, You can obtain one at https://mozilla.org/MPL/2.0/. */
use std::cell::RefCell;
use std::collections::hash_map::{Entry, Keys};
use std::collections::hash_map::Entry;
use std::rc::Rc;
use base::id::{PipelineId, WebViewId};
@ -132,10 +132,6 @@ impl WebViewRenderer {
.any(PipelineDetails::animation_callbacks_running)
}
pub(crate) fn pipeline_ids(&self) -> Keys<'_, PipelineId, PipelineDetails> {
self.pipelines.keys()
}
pub(crate) fn animating(&self) -> bool {
self.animating
}

View file

@ -15,11 +15,6 @@ use servo_url::ServoUrl;
/// Global flags for Servo, currently set on the command line.
#[derive(Clone, Debug, Deserialize, Serialize)]
pub struct Opts {
/// Whether or not Servo should wait for web content to go into an idle state, therefore
/// likely producing a stable output image. This is useful for taking screenshots of pages
/// after they have loaded.
pub wait_for_stable_image: bool,
/// `None` to disable the time profiler or `Some` to enable it with:
///
/// - an interval in seconds to cause it to produce output on that interval.
@ -167,7 +162,6 @@ pub enum OutputOptions {
impl Default for Opts {
fn default() -> Self {
Self {
wait_for_stable_image: false,
time_profiling: None,
time_profiler_trace_path: None,
nonincremental_layout: false,

View file

@ -85,7 +85,7 @@
//! See <https://github.com/servo/servo/issues/14704>
use std::borrow::ToOwned;
use std::cell::OnceCell;
use std::cell::{Cell, OnceCell, RefCell};
use std::collections::hash_map::Entry;
use std::collections::{HashMap, HashSet, VecDeque};
use std::marker::PhantomData;
@ -120,8 +120,9 @@ use constellation_traits::{
EmbedderToConstellationMessage, IFrameLoadInfo, IFrameLoadInfoWithData, IFrameSandboxState,
IFrameSizeMsg, Job, LoadData, LoadOrigin, LogEntry, MessagePortMsg, NavigationHistoryBehavior,
PaintMetricEvent, PortMessageTask, PortTransferInfo, SWManagerMsg, SWManagerSenders,
ScriptToConstellationChan, ScriptToConstellationMessage, ServiceWorkerManagerFactory,
ServiceWorkerMsg, StructuredSerializedData, TraversalDirection, WindowSizeType,
ScreenshotReadinessResponse, ScriptToConstellationChan, ScriptToConstellationMessage,
ServiceWorkerManagerFactory, ServiceWorkerMsg, StructuredSerializedData, TraversalDirection,
WindowSizeType,
};
use crossbeam_channel::{Receiver, Select, Sender, unbounded};
use devtools_traits::{
@ -506,6 +507,11 @@ pub struct Constellation<STF, SWF> {
/// Pending viewport changes for browsing contexts that are not
/// yet known to the constellation.
pending_viewport_changes: HashMap<BrowsingContextId, ViewportDetails>,
/// Pending screenshot readiness requests. These are collected until the screenshot is
/// ready to take place, at which point the Constellation informs the renderer that it
/// can start the process of taking the screenshot.
screenshot_readiness_requests: Vec<ScreenshotReadinessRequest>,
}
/// State needed to construct a constellation.
@ -572,18 +578,6 @@ pub struct InitialConstellationState {
pub async_runtime: Box<dyn AsyncRuntime>,
}
/// When we are running reftests, we save an image to compare against a reference.
/// This enum gives the possible states of preparing such an image.
#[derive(Debug, PartialEq)]
enum ReadyToSave {
NoTopLevelBrowsingContext,
PendingChanges,
DocumentLoading,
EpochMismatch,
PipelineUnknown,
Ready,
}
/// When we are exiting a pipeline, we can either force exiting or not.
/// A normal exit waits for the compositor to update its state before
/// exiting, and delegates layout exit to script. A forced exit does
@ -768,6 +762,7 @@ where
rippy_data,
)),
pending_viewport_changes: Default::default(),
screenshot_readiness_requests: Vec::new(),
};
constellation.run();
@ -1454,14 +1449,6 @@ where
NavigationHistoryBehavior::Push,
);
},
EmbedderToConstellationMessage::IsReadyToSaveImage(pipeline_states) => {
let is_ready = self.handle_is_ready_to_save_image(pipeline_states);
debug!("Ready to save image {:?}.", is_ready);
self.compositor_proxy
.send(CompositorMsg::IsReadyToSaveImageReply(
is_ready == ReadyToSave::Ready,
));
},
// Create a new top level browsing context. Will use response_chan to return
// the browsing context id.
EmbedderToConstellationMessage::NewWebView(url, webview_id, viewport_details) => {
@ -1605,6 +1592,9 @@ where
));
}
},
EmbedderToConstellationMessage::RequestScreenshotReadiness(webview_id) => {
self.handle_request_screenshot_readiness(webview_id)
},
}
}
@ -1864,13 +1854,6 @@ where
ScriptToConstellationMessage::SetDocumentState(state) => {
self.document_states.insert(source_pipeline_id, state);
},
ScriptToConstellationMessage::SetLayoutEpoch(epoch, response_sender) => {
if let Some(pipeline) = self.pipelines.get_mut(&source_pipeline_id) {
pipeline.layout_epoch = epoch;
}
response_sender.send(true).unwrap_or_default();
},
ScriptToConstellationMessage::LogEntry(thread_name, entry) => {
self.handle_log_entry(Some(webview_id), thread_name, entry);
},
@ -2025,6 +2008,9 @@ where
}
}
},
ScriptToConstellationMessage::RespondToScreenshotReadinessRequest(response) => {
self.handle_screenshot_readiness_response(source_pipeline_id, response);
},
}
}
@ -3772,6 +3758,8 @@ where
ExitPipelineMode::Normal,
);
}
self.send_screenshot_readiness_requests_to_pipelines();
}
#[servo_tracing::instrument(skip_all)]
@ -3790,19 +3778,7 @@ where
.get(&BrowsingContextId::from(webview_id))
.map(|ctx| ctx.pipeline_id == pipeline_id)
.unwrap_or(false);
if pipeline_is_top_level_pipeline {
// Is there any pending pipeline that will replace the current top level pipeline
let current_top_level_pipeline_will_be_replaced = self
.pending_changes
.iter()
.any(|change| change.browsing_context_id == webview_id);
if !current_top_level_pipeline_will_be_replaced {
// Notify embedder and compositor top level document finished loading.
self.compositor_proxy
.send(CompositorMsg::LoadComplete(webview_id));
}
} else {
if !pipeline_is_top_level_pipeline {
self.handle_subframe_loaded(pipeline_id);
}
}
@ -5037,15 +5013,20 @@ where
debug!("{}: Document ready to activate", pipeline_id);
// Find the pending change whose new pipeline id is pipeline_id.
let pending_index = self
let Some(pending_index) = self
.pending_changes
.iter()
.rposition(|change| change.new_pipeline_id == pipeline_id);
.rposition(|change| change.new_pipeline_id == pipeline_id)
else {
return;
};
// If it is found, remove it from the pending changes, and make it
// the active document of its frame.
if let Some(pending_index) = pending_index {
let change = self.pending_changes.swap_remove(pending_index);
self.send_screenshot_readiness_requests_to_pipelines();
// Notify the parent (if there is one).
let parent_pipeline_id = match change.new_browsing_context_info {
// This will be a new browsing context.
@ -5075,7 +5056,6 @@ where
}
self.change_session_history(change);
}
}
/// Called when the window is resized.
#[servo_tracing::instrument(skip_all)]
@ -5101,88 +5081,108 @@ where
self.switch_fullscreen_mode(browsing_context_id);
}
/// Checks the state of all script and layout pipelines to see if they are idle
/// and compares the current layout state to what the compositor has. This is used
/// to check if the output image is "stable" and can be written as a screenshot
/// for reftests.
/// Since this function is only used in reftests, we do not harden it against panic.
#[servo_tracing::instrument(skip_all)]
fn handle_is_ready_to_save_image(
&mut self,
pipeline_states: FxHashMap<PipelineId, Epoch>,
) -> ReadyToSave {
// Note that this function can panic, due to ipc-channel creation
// failure. Avoiding this panic would require a mechanism for dealing
// with low-resource scenarios.
//
// If there is no focus browsing context yet, the initial page has
// not loaded, so there is nothing to save yet.
let Some(webview_id) = self.webviews.focused_webview().map(|(id, _)| id) else {
return ReadyToSave::NoTopLevelBrowsingContext;
};
fn handle_request_screenshot_readiness(&mut self, webview_id: WebViewId) {
self.screenshot_readiness_requests
.push(ScreenshotReadinessRequest {
webview_id,
pipeline_states: Default::default(),
state: Default::default(),
});
self.send_screenshot_readiness_requests_to_pipelines();
}
fn send_screenshot_readiness_requests_to_pipelines(&mut self) {
// If there are pending loads, wait for those to complete.
if !self.pending_changes.is_empty() {
return ReadyToSave::PendingChanges;
return;
}
// Step through the fully active browsing contexts, checking that the script thread is idle,
// and that the current epoch of the layout matches what the compositor has painted. If all
// these conditions are met, then the output image should not change and a reftest
// screenshot can safely be written.
for browsing_context in self.fully_active_browsing_contexts_iter(webview_id) {
for screenshot_request in &self.screenshot_readiness_requests {
// Ignore this request if it is not pending.
if screenshot_request.state.get() != ScreenshotRequestState::Pending {
return;
}
*screenshot_request.pipeline_states.borrow_mut() = self
.fully_active_browsing_contexts_iter(screenshot_request.webview_id)
.filter_map(|browsing_context| {
let pipeline_id = browsing_context.pipeline_id;
trace!(
"{}: Checking readiness of {}",
browsing_context.id, pipeline_id
);
let pipeline = match self.pipelines.get(&pipeline_id) {
None => {
warn!("{}: Screenshot while closing", pipeline_id);
continue;
},
Some(pipeline) => pipeline,
let Some(pipeline) = self.pipelines.get(&pipeline_id) else {
// This can happen while Servo is shutting down, so just ignore it for now.
return None;
};
// See if this pipeline has reached idle script state yet.
match self.document_states.get(&browsing_context.pipeline_id) {
Some(&DocumentState::Idle) => {},
Some(&DocumentState::Pending) | None => {
return ReadyToSave::DocumentLoading;
},
}
// Check the visible rectangle for this pipeline. If the constellation has received a
// size for the pipeline, then its painting should be up to date.
//
// If the rectangle for this pipeline is zero sized, it will
// never be painted. In this case, don't query the layout
// thread as it won't contribute to the final output image.
// If the rectangle for this BrowsingContext is zero, it will never be
// painted. In this case, don't query screenshot readiness as it won't
// contribute to the final output image.
if browsing_context.viewport_details.size == Size2D::zero() {
continue;
return None;
}
// Get the epoch that the compositor has drawn for this pipeline and then check if the
// last laid out epoch matches what the compositor has drawn. If they match (and script
// is idle) then this pipeline won't change again and can be considered stable.
let compositor_epoch = pipeline_states.get(&browsing_context.pipeline_id);
match compositor_epoch {
Some(compositor_epoch) => {
if pipeline.layout_epoch != *compositor_epoch {
return ReadyToSave::EpochMismatch;
}
},
None => {
// The compositor doesn't know about this pipeline yet.
// Assume it hasn't rendered yet.
return ReadyToSave::PipelineUnknown;
},
let _ = pipeline
.event_loop
.send(ScriptThreadMessage::RequestScreenshotReadiness(pipeline_id));
Some((pipeline_id, None))
})
.collect();
screenshot_request
.state
.set(ScreenshotRequestState::WaitingOnScript);
}
}
// All script threads are idle and layout epochs match compositor, so output image!
ReadyToSave::Ready
#[servo_tracing::instrument(skip_all)]
fn handle_screenshot_readiness_response(
&mut self,
updated_pipeline_id: PipelineId,
response: ScreenshotReadinessResponse,
) {
if self.screenshot_readiness_requests.is_empty() {
return;
}
self.screenshot_readiness_requests
.retain(|screenshot_request| {
if screenshot_request.state.get() != ScreenshotRequestState::WaitingOnScript {
return true;
}
let mut has_pending_pipeline = false;
let mut pipeline_states = screenshot_request.pipeline_states.borrow_mut();
pipeline_states.retain(|pipeline_id, state| {
if *pipeline_id != updated_pipeline_id {
has_pending_pipeline |= state.is_none();
return true;
}
match response {
ScreenshotReadinessResponse::Ready(epoch) => {
*state = Some(epoch);
true
},
ScreenshotReadinessResponse::NoLongerActive => false,
}
});
if has_pending_pipeline {
return true;
}
let pipelines_and_epochs = pipeline_states
.iter()
.map(|(pipeline_id, epoch)| {
(
*pipeline_id,
epoch.expect("Should have an epoch when pipeline is ready."),
)
})
.collect();
self.compositor_proxy
.send(CompositorMsg::ScreenshotReadinessReponse(
screenshot_request.webview_id,
pipelines_and_epochs,
));
false
});
}
/// Get the current activity of a pipeline.
@ -5536,6 +5536,12 @@ where
// Inform script, compositor that this pipeline has exited.
pipeline.send_exit_message_to_script(dbc);
self.send_screenshot_readiness_requests_to_pipelines();
self.handle_screenshot_readiness_response(
pipeline_id,
ScreenshotReadinessResponse::NoLongerActive,
);
debug!("{}: Closed", pipeline_id);
}
@ -5709,3 +5715,27 @@ where
CanvasPaintThread::start(self.compositor_proxy.cross_process_compositor_api.clone())
}
}
/// When a [`ScreenshotReadinessRequest`] is received from the renderer, the [`Constellation`]
/// go through a variety of states to process them. This data structure represents those states.
#[derive(Clone, Copy, Default, PartialEq)]
enum ScreenshotRequestState {
/// The [`Constellation`] has received the [`ScreenshotReadinessRequest`], but has not yet
/// forwarded it to the [`Pipeline`]'s of the requests's WebView. This is likely because there
/// are still pending navigation changes in the [`Constellation`]. Once those changes are resolved
/// the request will be forwarded to the [`Pipeline`]s.
#[default]
Pending,
/// The [`Constellation`] has forwarded the [`ScreenshotReadinessRequest`] to the [`Pipeline`]s of
/// the corresponding `WebView`. The [`Pipeline`]s are waiting for a variety of things to happen in
/// order to report what appropriate display list epoch is for the screenshot. Once they all report
/// back, the [`Constellation`] considers that the request is handled, and the renderer is responsible
/// for waiting to take the screenshot.
WaitingOnScript,
}
struct ScreenshotReadinessRequest {
webview_id: WebViewId,
state: Cell<ScreenshotRequestState>,
pipeline_states: RefCell<FxHashMap<PipelineId, Option<Epoch>>>,
}

View file

@ -11,7 +11,6 @@ use background_hang_monitor::HangMonitorRegister;
use background_hang_monitor_api::{
BackgroundHangMonitorControlMsg, BackgroundHangMonitorRegister, HangMonitorAlert,
};
use base::Epoch;
use base::generic_channel::{self, GenericReceiver, GenericSender};
use base::id::{
BrowsingContextId, HistoryStateId, PipelineId, PipelineNamespace, PipelineNamespaceId,
@ -104,10 +103,6 @@ pub struct Pipeline {
/// The title of this pipeline's document.
pub title: String,
/// The last compositor [`Epoch`] that was laid out in this pipeline if "exit after load" is
/// enabled.
pub layout_epoch: Epoch,
pub focus_sequence: FocusSequenceNumber,
}
@ -400,7 +395,6 @@ impl Pipeline {
history_states: HashSet::new(),
completely_loaded: false,
title: String::new(),
layout_epoch: Epoch(0),
focus_sequence: FocusSequenceNumber::default(),
};

View file

@ -50,7 +50,6 @@ mod from_compositor {
fn log_target(&self) -> &'static str {
match self {
Self::Exit => target!("Exit"),
Self::IsReadyToSaveImage(..) => target!("IsReadyToSaveImage"),
Self::AllowNavigationResponse(..) => target!("AllowNavigationResponse"),
Self::LoadUrl(..) => target!("LoadUrl"),
Self::ClearCache => target!("ClearCache"),
@ -82,6 +81,7 @@ mod from_compositor {
Self::NoLongerWaitingOnAsynchronousImageUpdates(..) => {
target!("NoLongerWaitingOnCanvas")
},
Self::RequestScreenshotReadiness(..) => target!("RequestScreenshotReadiness"),
}
}
}
@ -163,7 +163,6 @@ mod from_script {
Self::CreateAuxiliaryWebView(..) => target!("ScriptNewAuxiliary"),
Self::ActivateDocument => target!("ActivateDocument"),
Self::SetDocumentState(..) => target!("SetDocumentState"),
Self::SetLayoutEpoch(..) => target!("SetLayoutEpoch"),
Self::SetFinalUrl(..) => target!("SetFinalUrl"),
Self::TouchEventProcessed(..) => target!("TouchEventProcessed"),
Self::LogEntry(..) => target!("LogEntry"),
@ -183,6 +182,9 @@ mod from_script {
Self::WebDriverInputComplete(..) => target!("WebDriverInputComplete"),
Self::FinishJavaScriptEvaluation(..) => target!("FinishJavaScriptEvaluation"),
Self::ForwardKeyboardScroll(..) => target!("ForwardKeyboardScroll"),
Self::RespondToScreenshotReadinessRequest(..) => {
target!("RespondToScreenshotReadinessRequest")
},
}
}
}

View file

@ -26,8 +26,9 @@ use bluetooth_traits::BluetoothRequest;
use canvas_traits::webgl::WebGLChan;
use compositing_traits::CrossProcessCompositorApi;
use constellation_traits::{
DocumentState, LoadData, LoadOrigin, NavigationHistoryBehavior, ScriptToConstellationChan,
ScriptToConstellationMessage, StructuredSerializedData, WindowSizeType,
LoadData, LoadOrigin, NavigationHistoryBehavior, ScreenshotReadinessResponse,
ScriptToConstellationChan, ScriptToConstellationMessage, StructuredSerializedData,
WindowSizeType,
};
use crossbeam_channel::{Sender, unbounded};
use cssparser::SourceLocation;
@ -42,7 +43,7 @@ use embedder_traits::{
use euclid::default::{Point2D as UntypedPoint2D, Rect as UntypedRect, Size2D as UntypedSize2D};
use euclid::{Point2D, Scale, Size2D, Vector2D};
use fonts::FontContext;
use ipc_channel::ipc::{self, IpcSender};
use ipc_channel::ipc::IpcSender;
use js::glue::DumpJSStack;
use js::jsapi::{
GCReason, Heap, JS_GC, JSAutoRealm, JSContext as RawJSContext, JSObject, JSPROP_ENUMERATE,
@ -79,7 +80,7 @@ use script_bindings::root::Root;
use script_traits::{ConstellationInputEvent, ScriptThreadMessage};
use selectors::attr::CaseSensitivity;
use servo_arc::Arc as ServoArc;
use servo_config::{opts, pref};
use servo_config::pref;
use servo_geometry::{DeviceIndependentIntRect, f32_rect_to_au_rect};
use servo_url::{ImmutableOrigin, MutableOrigin, ServoUrl};
use storage_traits::StorageThreads;
@ -447,6 +448,9 @@ pub(crate) struct Window {
/// The window proxies the script thread knows.
#[ignore_malloc_size_of = "Rc"]
script_window_proxies: Rc<ScriptWindowProxies>,
/// Whether or not this [`Window`] has a pending screenshot readiness request.
has_pending_screenshot_readiness_request: Cell<bool>,
}
impl Window {
@ -2335,17 +2339,18 @@ impl Window {
}
document.update_animations_post_reflow();
self.update_constellation_epoch();
reflow_result.reflow_phases_run
}
pub(crate) fn maybe_send_idle_document_state_to_constellation(&self) {
if !opts::get().wait_for_stable_image {
return;
pub(crate) fn request_screenshot_readiness(&self) {
self.has_pending_screenshot_readiness_request.set(true);
self.maybe_resolve_pending_screenshot_readiness_requests();
}
if self.has_sent_idle_message.get() {
pub(crate) fn maybe_resolve_pending_screenshot_readiness_requests(&self) {
let pending_request = self.has_pending_screenshot_readiness_request.get();
if !pending_request {
return;
}
@ -2382,17 +2387,18 @@ impl Window {
return;
}
// When all these conditions are met, notify the constellation
// that this pipeline is ready to write the image (from the script thread
// perspective at least).
debug!(
"{:?}: Sending DocumentState::Idle to Constellation",
self.pipeline_id()
// When all these conditions are met, notify the Constellation that we are ready to
// have our screenshot taken, when the given layout Epoch has been rendered.
let epoch = self.layout.borrow().current_epoch();
let pipeline_id = self.pipeline_id();
debug!("Ready to take screenshot of {pipeline_id:?} at epoch={epoch:?}");
self.send_to_constellation(
ScriptToConstellationMessage::RespondToScreenshotReadinessRequest(
ScreenshotReadinessResponse::Ready(epoch),
),
);
self.send_to_constellation(ScriptToConstellationMessage::SetDocumentState(
DocumentState::Idle,
));
self.has_sent_idle_message.set(true);
self.has_pending_screenshot_readiness_request.set(false);
}
/// If parsing has taken a long time and reflows are still waiting for the `load` event,
@ -2456,24 +2462,6 @@ impl Window {
self.layout_blocker.get().layout_blocked()
}
/// If writing a screenshot, synchronously update the layout epoch that it set
/// in the constellation.
pub(crate) fn update_constellation_epoch(&self) {
if !opts::get().wait_for_stable_image {
return;
}
let epoch = self.layout.borrow().current_epoch();
debug!(
"{:?}: Updating constellation epoch: {epoch:?}",
self.pipeline_id()
);
let (sender, receiver) = ipc::channel().expect("Failed to create IPC channel!");
let event = ScriptToConstellationMessage::SetLayoutEpoch(epoch, sender);
self.send_to_constellation(event);
let _ = receiver.recv();
}
/// Trigger a reflow that is required by a certain queries.
pub(crate) fn layout_reflow(&self, query_msg: QueryMsg) {
self.reflow(ReflowGoal::LayoutQuery(query_msg));
@ -3203,9 +3191,7 @@ impl Window {
node.dirty(NodeDamage::Other);
}
}
}
impl Window {
#[allow(unsafe_code)]
#[allow(clippy::too_many_arguments)]
pub(crate) fn new(
@ -3339,6 +3325,7 @@ impl Window {
report_list: Default::default(),
endpoints_list: Default::default(),
script_window_proxies: ScriptThread::window_proxies(),
has_pending_screenshot_readiness_request: Default::default(),
});
WindowBinding::Wrap::<crate::DomTypeHolder>(GlobalScope::get_cx(), win)

View file

@ -102,6 +102,7 @@ impl MixedMessage {
ScriptThreadMessage::PreferencesUpdated(..) => None,
ScriptThreadMessage::NoLongerWaitingOnAsychronousImageUpdates(_) => None,
ScriptThreadMessage::ForwardKeyboardScroll(id, _) => Some(*id),
ScriptThreadMessage::RequestScreenshotReadiness(id) => Some(*id),
},
MixedMessage::FromScript(inner_msg) => match inner_msg {
MainThreadScriptMsg::Common(CommonScriptMsg::Task(_, _, pipeline_id, _)) => {

View file

@ -38,8 +38,9 @@ use canvas_traits::webgl::WebGLPipeline;
use chrono::{DateTime, Local};
use compositing_traits::{CrossProcessCompositorApi, PipelineExitSource};
use constellation_traits::{
JsEvalResult, LoadData, LoadOrigin, NavigationHistoryBehavior, ScriptToConstellationChan,
ScriptToConstellationMessage, StructuredSerializedData, WindowSizeType,
JsEvalResult, LoadData, LoadOrigin, NavigationHistoryBehavior, ScreenshotReadinessResponse,
ScriptToConstellationChan, ScriptToConstellationMessage, StructuredSerializedData,
WindowSizeType,
};
use crossbeam_channel::unbounded;
use data_url::mime::Mime;
@ -1297,17 +1298,14 @@ impl ScriptThread {
}
}
/// If waiting for an idle `Pipeline` state in order to dump a screenshot at
/// the right time, inform the `Constellation` this `Pipeline` has entered
/// the idle state when applicable.
fn maybe_send_idle_document_state_to_constellation(&self) {
if !opts::get().wait_for_stable_image {
return;
}
/// If any `Pipeline`s are waiting to become ready for the purpose of taking a
/// screenshot, check to see if the `Pipeline` is now ready and send a message to the
/// Constellation, if so.
fn maybe_resolve_pending_screenshot_readiness_requests(&self) {
for (_, document) in self.documents.borrow().iter() {
document
.window()
.maybe_send_idle_document_state_to_constellation();
.maybe_resolve_pending_screenshot_readiness_requests();
}
}
@ -1534,7 +1532,7 @@ impl ScriptThread {
self.update_the_rendering(can_gc);
self.maybe_fulfill_font_ready_promises(can_gc);
self.maybe_send_idle_document_state_to_constellation();
self.maybe_resolve_pending_screenshot_readiness_requests();
// This must happen last to detect if any change above makes a rendering update necessary.
self.maybe_schedule_rendering_opportunity_after_ipc_message(built_any_display_lists);
@ -1911,6 +1909,9 @@ impl ScriptThread {
document.event_handler().do_keyboard_scroll(scroll);
}
},
ScriptThreadMessage::RequestScreenshotReadiness(pipeline_id) => {
self.handle_request_screenshot_readiness(pipeline_id);
},
}
}
@ -3912,6 +3913,19 @@ impl ScriptThread {
pub(crate) fn is_servo_privileged(url: ServoUrl) -> bool {
with_script_thread(|script_thread| script_thread.privileged_urls.contains(&url))
}
fn handle_request_screenshot_readiness(&self, pipeline_id: PipelineId) {
let Some(window) = self.documents.borrow().find_window(pipeline_id) else {
let _ = self.senders.pipeline_to_constellation_sender.send((
pipeline_id,
ScriptToConstellationMessage::RespondToScreenshotReadinessRequest(
ScreenshotReadinessResponse::NoLongerActive,
),
));
return;
};
window.request_screenshot_readiness();
}
}
impl Drop for ScriptThread {

View file

@ -85,6 +85,7 @@ euclid = { workspace = true }
fonts = { path = "../fonts" }
gleam = { workspace = true }
gstreamer = { workspace = true, optional = true }
image = { workspace = true }
ipc-channel = { workspace = true }
keyboard-types = { workspace = true }
layout = { path = "../layout" }

View file

@ -1070,14 +1070,10 @@ impl Servo {
}
}
let res = self
let img = self
.compositor
.borrow_mut()
.render_to_shared_memory(webview_id, page_rect);
if let Err(ref e) = res {
error!("Error retrieving PNG: {:?}", e);
}
let img = res.unwrap_or(None);
if let Err(e) = response_sender.send(Ok(img)) {
error!("Sending reply to create png failed ({:?}).", e);
}

View file

@ -14,9 +14,11 @@ use constellation_traits::{EmbedderToConstellationMessage, TraversalDirection};
use dpi::PhysicalSize;
use embedder_traits::{
Cursor, Image, InputEvent, JSValue, JavaScriptEvaluationError, LoadStatus,
MediaSessionActionType, ScreenGeometry, Theme, TraversalId, ViewportDetails,
MediaSessionActionType, ScreenGeometry, ScreenshotCaptureError, Theme, TraversalId,
ViewportDetails,
};
use euclid::{Point2D, Scale, Size2D};
use image::RgbaImage;
use servo_geometry::DeviceIndependentPixel;
use url::Url;
use webrender_api::ScrollLocation;
@ -568,11 +570,9 @@ impl WebView {
));
}
/// Paint the contents of this [`WebView`] into its `RenderingContext`. This will
/// always paint, unless the `Opts::wait_for_stable_image` option is enabled. In
/// that case, this might do nothing. Returns true if a paint was actually performed.
pub fn paint(&self) -> bool {
self.inner().compositor.borrow_mut().render()
/// Paint the contents of this [`WebView`] into its `RenderingContext`.
pub fn paint(&self) {
self.inner().compositor.borrow_mut().render();
}
/// Evaluate the specified string of JavaScript code. Once execution is complete or an error
@ -588,6 +588,31 @@ impl WebView {
Box::new(callback),
);
}
/// Asynchronously take a screenshot of the [`WebView`] contents. This method will
/// wait until the [`WebView`] is ready before the screenshot is taken. This includes
/// waiting for:
///
/// - all frames to fire their `load` event.
/// - all render blocking elements, such as stylesheets included via the `<link>`
/// element, to stop blocking the rendering.
/// - all images to be loaded and displayed.
/// - all web fonts are loaded.
/// - the `reftest-wait` and `test-wait` classes have been removed from the root element.
/// - the rendering is up-to-date
///
/// Once all these conditions are met and the rendering does not have any pending frames
/// to render, the provided `callback` will be called with the results of the screenshot
/// operation.
pub fn take_screenshot(
&self,
callback: impl FnOnce(Result<RgbaImage, ScreenshotCaptureError>) + 'static,
) {
self.inner()
.compositor
.borrow()
.request_screenshot(self.id(), Box::new(callback));
}
}
/// A structure used to expose a view of the [`WebView`] to the Servo

View file

@ -87,8 +87,6 @@ pub enum CompositorMsg {
RemoveWebView(WebViewId),
/// Script has handled a touch event, and either prevented or allowed default actions.
TouchEventProcessed(WebViewId, TouchEventResult),
/// A reply to the compositor asking if the output image is stable.
IsReadyToSaveImageReply(bool),
/// Set whether to use less resources by stopping animations.
SetThrottled(WebViewId, PipelineId, bool),
/// WebRender has produced a new frame. This message informs the compositor that
@ -100,8 +98,6 @@ pub enum CompositorMsg {
/// they have fully shut it down, to avoid recreating it due to any subsequent
/// messages.
PipelineExited(WebViewId, PipelineId, PipelineExitSource),
/// The load of a page has completed
LoadComplete(WebViewId),
/// Inform WebRender of the existence of this pipeline.
SendInitialTransaction(WebViewId, WebRenderPipelineId),
/// Perform a scroll operation.
@ -164,6 +160,9 @@ pub enum CompositorMsg {
CollectMemoryReport(ReportsChan),
/// A top-level frame has parsed a viewport metatag and is sending the new constraints.
Viewport(WebViewId, ViewportDescription),
/// Let the compositor know that the given WebView is ready to have a screenshot taken
/// after the given pipeline's epochs have been rendered.
ScreenshotReadinessReponse(WebViewId, FxHashMap<PipelineId, Epoch>),
}
impl Debug for CompositorMsg {

View file

@ -504,6 +504,16 @@ pub enum KeyboardScroll {
End,
}
#[derive(Debug, Deserialize, Serialize)]
pub enum ScreenshotReadinessResponse {
/// The Pipeline associated with this response, is ready for a screenshot at the
/// provided [`Epoch`].
Ready(Epoch),
/// The Pipeline associated with this response is no longer active and should be
/// ignored for the purposes of the screenshot.
NoLongerActive,
}
/// Messages from the script to the constellation.
#[derive(Deserialize, IntoStaticStr, Serialize)]
pub enum ScriptToConstellationMessage {
@ -644,8 +654,6 @@ pub enum ScriptToConstellationMessage {
ActivateDocument,
/// Set the document state for a pipeline (used by screenshot / reftests)
SetDocumentState(DocumentState),
/// Update the layout epoch in the constellation (used by screenshot / reftests).
SetLayoutEpoch(Epoch, IpcSender<bool>),
/// Update the pipeline Url, which can change after redirections.
SetFinalUrl(ServoUrl),
/// Script has handled a touch event, and either prevented or allowed default actions.
@ -691,6 +699,8 @@ pub enum ScriptToConstellationMessage {
WebDriverInputComplete(WebDriverMessageId),
/// Forward a keyboard scroll operation from an `<iframe>` to a parent pipeline.
ForwardKeyboardScroll(PipelineId, KeyboardScroll),
/// Notify the Constellation of the screenshot readiness of a given pipeline.
RespondToScreenshotReadinessRequest(ScreenshotReadinessResponse),
}
impl fmt::Debug for ScriptToConstellationMessage {

View file

@ -15,7 +15,6 @@ use std::collections::VecDeque;
use std::fmt;
use std::time::Duration;
use base::Epoch;
use base::cross_process_instant::CrossProcessInstant;
use base::id::{MessagePortId, PipelineId, WebViewId};
use embedder_traits::{
@ -41,8 +40,6 @@ use webrender_api::{ExternalScrollId, ImageKey};
pub enum EmbedderToConstellationMessage {
/// Exit the constellation.
Exit,
/// Query the constellation to see if the current compositor output is stable
IsReadyToSaveImage(FxHashMap<PipelineId, Epoch>),
/// Whether to allow script to navigate.
AllowNavigationResponse(PipelineId, bool),
/// Request to load a page.
@ -109,6 +106,9 @@ pub enum EmbedderToConstellationMessage {
SetWebDriverResponseSender(IpcSender<WebDriverCommandResponse>),
/// A set of preferences were updated with the given new values.
PreferencesUpdated(Vec<(&'static str, PrefValue)>),
/// Request preparation for a screenshot of the given WebView. The Constellation will
/// send a message to the Embedder when the screenshot is ready to be taken.
RequestScreenshotReadiness(WebViewId),
}
/// A description of a paint metric that is sent from the Servo renderer to the

View file

@ -1063,6 +1063,15 @@ pub enum JavaScriptEvaluationError {
SerializationError,
}
#[derive(Clone, Debug, Deserialize, PartialEq, Serialize)]
pub enum ScreenshotCaptureError {
/// The screenshot request failed to read the screenshot image from the `WebView`'s
/// `RenderingContext`.
CouldNotReadImage,
/// The WebView that this screenshot request was made for no longer exists.
WebViewDoesNotExist,
}
#[derive(Clone, Copy, Debug, Deserialize, Serialize)]
pub struct RgbColor {
pub red: u8,

View file

@ -268,6 +268,10 @@ pub enum ScriptThreadMessage {
NoLongerWaitingOnAsychronousImageUpdates(PipelineId),
/// Forward a keyboard scroll operation from an `<iframe>` to a parent pipeline.
ForwardKeyboardScroll(PipelineId, KeyboardScroll),
/// Request readiness for a screenshot from the given pipeline. The pipeline will
/// respond when it is ready to take the screenshot or will not be able to take it
/// in the future.
RequestScreenshotReadiness(PipelineId),
}
impl fmt::Debug for ScriptThreadMessage {

View file

@ -2,7 +2,7 @@
* 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::cell::{Ref, RefCell, RefMut};
use std::cell::{Cell, Ref, RefCell, RefMut};
use std::collections::HashMap;
use std::collections::hash_map::Entry;
use std::mem;
@ -11,6 +11,7 @@ use std::rc::Rc;
use crossbeam_channel::Receiver;
use embedder_traits::webdriver::WebDriverSenders;
use image::{DynamicImage, ImageFormat};
use keyboard_types::ShortcutMatcher;
use log::{error, info};
use servo::base::generic_channel::GenericSender;
@ -31,7 +32,6 @@ use super::dialog::Dialog;
use super::gamepad::GamepadSupport;
use super::keyutils::CMD_OR_CONTROL;
use super::window_trait::WindowPortsMethods;
use crate::output_image::save_output_image_if_necessary;
use crate::prefs::ServoShellPreferences;
pub(crate) enum AppState {
@ -91,6 +91,10 @@ pub struct RunningAppStateInner {
/// List of webviews that have favicon textures which are not yet uploaded
/// to the GPU by egui.
pending_favicon_loads: Vec<WebViewId>,
/// Whether or not the application has achieved stable image output. This is used
/// for the `exit_after_stable_image` option.
achieved_stable_image: Rc<Cell<bool>>,
}
impl Drop for RunningAppState {
@ -123,6 +127,7 @@ impl RunningAppState {
need_repaint: false,
dialog_amount_changed: false,
pending_favicon_loads: Default::default(),
achieved_stable_image: Default::default(),
}),
}
}
@ -178,24 +183,12 @@ impl RunningAppState {
let Some(webview) = self.focused_webview() else {
return;
};
if !webview.paint() {
return;
}
// This needs to be done before presenting(), because `ReneringContext::read_to_image` reads
// from the back buffer.
save_output_image_if_necessary(
&self.servoshell_preferences,
&self.inner().window.rendering_context(),
);
webview.paint();
let mut inner_mut = self.inner_mut();
inner_mut.window.rendering_context().present();
inner_mut.need_repaint = false;
if self.servoshell_preferences.exit_after_stable_image {
self.servo().start_shutting_down();
}
}
/// Spins the internal application event loop.
@ -220,6 +213,12 @@ impl RunningAppState {
self.inner_mut().dialog_amount_changed = false;
if self.servoshell_preferences.exit_after_stable_image &&
self.inner().achieved_stable_image.get()
{
self.servo.start_shutting_down();
}
PumpResult::Continue {
need_update,
need_window_redraw,
@ -492,6 +491,44 @@ impl RunningAppState {
pub(crate) fn take_pending_favicon_loads(&self) -> Vec<WebViewId> {
mem::take(&mut self.inner_mut().pending_favicon_loads)
}
/// If we are exiting after achieving a stable image or we want to save the display of the
/// [`WebView`] to an image file, request a screenshot of the [`WebView`].
fn maybe_request_screenshot(&self, webview: WebView) {
let output_path = self.servoshell_preferences.output_image_path.clone();
if !self.servoshell_preferences.exit_after_stable_image && output_path.is_none() {
return;
}
// Never request more than a single screenshot for now.
let achieved_stable_image = self.inner().achieved_stable_image.clone();
if achieved_stable_image.get() {
return;
}
webview.take_screenshot(move |image| {
achieved_stable_image.set(true);
let Some(output_path) = output_path else {
return;
};
let image = match image {
Ok(image) => image,
Err(error) => {
error!("Could not take screenshot: {error:?}");
return;
},
};
let image_format = ImageFormat::from_path(&output_path).unwrap_or(ImageFormat::Png);
if let Err(error) =
DynamicImage::ImageRgba8(image).save_with_format(output_path, image_format)
{
error!("Failed to save screenshot: {error}.");
}
});
}
}
struct ServoShellServoDelegate;
@ -653,6 +690,7 @@ impl WebViewDelegate for RunningAppState {
{
let _ = sender.send(WebDriverLoadStatus::Complete);
}
self.maybe_request_screenshot(webview);
}
}

View file

@ -14,6 +14,7 @@ use embedder_traits::{
WebDriverJSResult,
};
use euclid::{Point2D, Rect, Scale, Size2D, Vector2D};
use image::{DynamicImage, ImageFormat};
use keyboard_types::{CompositionEvent, CompositionState, Key, KeyState, NamedKey};
use log::{debug, error, info, warn};
use raw_window_handle::{RawWindowHandle, WindowHandle};
@ -25,14 +26,13 @@ use servo::webrender_api::ScrollLocation;
use servo::webrender_api::units::{DeviceIntRect, DeviceIntSize, DevicePixel};
use servo::{
AllowOrDenyRequest, ImeEvent, InputEvent, LoadStatus, MouseButtonEvent, MouseMoveEvent,
NavigationRequest, PermissionRequest, RenderingContext, Servo, ServoDelegate, ServoError,
SimpleDialog, TraversalId, WebDriverCommandMsg, WebDriverLoadStatus, WebDriverScriptCommand,
WebView, WebViewBuilder, WebViewDelegate, WindowRenderingContext,
NavigationRequest, PermissionRequest, Servo, ServoDelegate, ServoError, SimpleDialog,
TraversalId, WebDriverCommandMsg, WebDriverLoadStatus, WebDriverScriptCommand, WebView,
WebViewBuilder, WebViewDelegate, WindowRenderingContext,
};
use url::Url;
use crate::egl::host_trait::HostTrait;
use crate::output_image::save_output_image_if_necessary;
use crate::prefs::ServoShellPreferences;
#[derive(Clone, Debug)]
@ -108,6 +108,10 @@ struct RunningAppStateInner {
/// The HiDPI scaling factor to use for the display of [`WebView`]s.
hidpi_scale_factor: Scale<f32, DeviceIndependentPixel, DevicePixel>,
/// Whether or not the application has achieved stable image output. This is used
/// for the `exit_after_stable_image` option.
achieved_stable_image: Rc<Cell<bool>>,
}
struct ServoShellServoDelegate {
@ -173,6 +177,7 @@ impl WebViewDelegate for RunningAppState {
{
let _ = sender.send(WebDriverLoadStatus::Complete);
}
self.maybe_request_screenshot(webview);
}
#[cfg(feature = "tracing")]
@ -367,6 +372,7 @@ impl RunningAppState {
focused_webview_id: None,
animating_state_changed,
hidpi_scale_factor: Scale::new(hidpi_scale_factor),
achieved_stable_image: Default::default(),
}),
});
@ -943,17 +949,56 @@ impl RunningAppState {
}
pub fn present_if_needed(&self) {
if self.inner().need_present {
self.inner_mut().need_present = false;
if !self.active_webview().paint() {
if !self.inner().need_present {
return;
}
save_output_image_if_necessary(&self.servoshell_preferences, &self.rendering_context);
self.rendering_context.present();
if self.servoshell_preferences.exit_after_stable_image {
self.inner_mut().need_present = false;
self.active_webview().paint();
if self.servoshell_preferences.exit_after_stable_image &&
self.inner().achieved_stable_image.get()
{
self.request_shutdown();
}
}
/// If we are exiting after achieving a stable image or we want to save the display of the
/// [`WebView`] to an image file, request a screenshot of the [`WebView`].
fn maybe_request_screenshot(&self, webview: WebView) {
let output_path = self.servoshell_preferences.output_image_path.clone();
if !self.servoshell_preferences.exit_after_stable_image && output_path.is_none() {
return;
}
// Never request more than a single screenshot for now.
let achieved_stable_image = self.inner().achieved_stable_image.clone();
if achieved_stable_image.get() {
return;
}
webview.take_screenshot(move |image| {
achieved_stable_image.set(true);
let Some(output_path) = output_path else {
return;
};
let image = match image {
Ok(image) => image,
Err(error) => {
error!("Could not take screenshot: {error:?}");
return;
},
};
let image_format = ImageFormat::from_path(&output_path).unwrap_or(ImageFormat::Png);
if let Err(error) =
DynamicImage::ImageRgba8(image).save_with_format(output_path, image_format)
{
error!("Failed to save screenshot: {error}.");
}
});
}
}

View file

@ -15,7 +15,6 @@ mod crash_handler;
pub(crate) mod desktop;
#[cfg(any(target_os = "android", target_env = "ohos"))]
mod egl;
mod output_image;
#[cfg(not(any(target_os = "android", target_env = "ohos")))]
mod panic_hook;
mod parser;

View file

@ -1,39 +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::rc::Rc;
use euclid::Point2D;
use image::{DynamicImage, ImageFormat};
use log::error;
use servo::RenderingContext;
use servo::webrender_api::units::DeviceIntRect;
use crate::prefs::ServoShellPreferences;
/// This needs to be done before presenting(), because `ReneringContext::read_to_image` reads
/// from the back buffer. This does nothing if the preference `output_image_path` is not set.
pub(crate) fn save_output_image_if_necessary<T>(
prefs: &ServoShellPreferences,
rendering_context: &Rc<T>,
) where
T: RenderingContext + ?Sized,
{
let Some(output_path) = prefs.output_image_path.as_ref() else {
return;
};
let size = rendering_context.size2d().to_i32();
let viewport_rect = DeviceIntRect::from_origin_and_size(Point2D::origin(), size);
let Some(image) = rendering_context.read_to_image(viewport_rect) else {
error!("Failed to read output image.");
return;
};
let image_format = ImageFormat::from_path(output_path).unwrap_or(ImageFormat::Png);
if let Err(error) = DynamicImage::ImageRgba8(image).save_with_format(output_path, image_format)
{
error!("Failed to save {output_path}: {error}.");
}
}

View file

@ -694,7 +694,6 @@ pub(crate) fn parse_command_line_arguments(args: Vec<String>) -> ArgumentParsing
let opts = Opts {
debug: debug_options,
wait_for_stable_image: cmd_args.exit,
time_profiling: cmd_args.profile,
time_profiler_trace_path: cmd_args
.profiler_trace_path