script: Start rework to better match the specification HTML event loop (#31505)

* Fix the HTML event-loop: add a update the rendering task

add rendering task source

sketch structure to update the rendering

resize steps

composition events

fix warnings in rendering task source

refactor handling of composition events: put window and doc for pipeline on top

set script as user interacting in update the rendering task

fmt

add todos for other steps, put all compositor handling logic in one place

update the rendering: evaluate media queries and report changes

update the rendering: update animations and send events

update the rendering: run animation frames

update the rendering: order docs

put rendering related info on documents map

tidy

update the rendering: add issue numbers to todos

update the rendering: reflow as last step

update the rendering: add todo for top layer removals

note rendering opportunity when ticking animations for testing

fix double borrow crash in css/basic-transition

fix faster reversing of transitions test

undo ordering of docs

bypass not fully-active pipeline task throttling for rendering tasks

ensure tasks are dequed from task queue

prioritize update the rendering task

remove servo media stuff from set activity

tidy

debug

update the rendering: perform microtask checkpoint after task

tidy-up

only run evaluate media queries if resized

re-add evaluation of media queries for each rendering task, re-prioritize rendering tasks, re-add microtask checkpoint for all sequential messages

re-structure resize steps, and their interaction with evaluating media queries and reacting to environment changes

update the rendering: remove reflow call at the end

update webmessaging expectations

update to FAIL /html/browsers/browsing-the-web/navigating-across-documents/initial-empty-document/load-pageshow-events-iframe-contentWindow.html

update to FAIL load-pageshow-events-window-open.html

add issue number for ordering of docs

nits

move batching of mouse move event to document info

nits

add doc for mouse move event index

reset mouse move event index when taking pending compositor events

fix replacing mouse move event

nits

* move update the rendering related data to document

* move re-taking of tasks to try_recv

* address nits

* change task queue try_recv into take_tasks_and_recv, with nits

* refactor process_pending_compositor_events

* when updating the rendering, return early if script cannot continue running

* use an instant for the last render opportunity time

* nits

* remove handle_tick_all_animations

* use a vec for pending resize and compositor events

* fix spec links

* Fix a few other nits before landing

---------

Co-authored-by: Martin Robinson <mrobinson@igalia.com>
This commit is contained in:
Gregory Terzian 2024-05-13 17:23:03 +08:00 committed by GitHub
parent 77c50ad356
commit 1d66ea2b27
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
11 changed files with 528 additions and 283 deletions

View file

@ -132,7 +132,7 @@ pub fn run_worker_event_loop<T, WorkerMsg, Event>(
while !scope.is_closing() {
// Batch all events that are ready.
// The task queue will throttle non-priority tasks if necessary.
match task_queue.try_recv() {
match task_queue.take_tasks_and_recv() {
Err(_) => match devtools_port.map(|port| port.try_recv()) {
None => {},
Some(Err(_)) => break,

View file

@ -46,8 +46,9 @@ use profile_traits::ipc as profile_ipc;
use profile_traits::time::{TimerMetadata, TimerMetadataFrameType, TimerMetadataReflowType};
use script_layout_interface::{PendingRestyle, ReflowGoal, TrustedNodeAddress};
use script_traits::{
AnimationState, DocumentActivity, MouseButton, MouseEventType, MsDuration, ScriptMsg,
TouchEventType, TouchId, UntrustedNodeAddress, WheelDelta,
AnimationState, AnimationTickType, CompositorEvent, DocumentActivity, MouseButton,
MouseEventType, MsDuration, ScriptMsg, TouchEventType, TouchId, UntrustedNodeAddress,
WheelDelta,
};
use servo_arc::Arc;
use servo_atoms::Atom;
@ -446,6 +447,16 @@ pub struct Document {
dirty_root: MutNullableDom<Element>,
/// <https://html.spec.whatwg.org/multipage/#will-declaratively-refresh>
declarative_refresh: DomRefCell<Option<DeclarativeRefresh>>,
/// Pending composition events, to be handled at the next rendering opportunity.
#[no_trace]
#[ignore_malloc_size_of = "CompositorEvent contains data from outside crates"]
pending_compositor_events: DomRefCell<Vec<CompositorEvent>>,
/// The index of the last mouse move event in the pending compositor events queue.
mouse_move_event_index: DomRefCell<Option<usize>>,
/// Pending animation ticks, to be handled at the next rendering opportunity.
#[no_trace]
#[ignore_malloc_size_of = "AnimationTickType contains data from an outside crate"]
pending_animation_ticks: DomRefCell<AnimationTickType>,
}
#[derive(JSTraceable, MallocSizeOf)]
@ -1217,7 +1228,7 @@ impl Document {
}
#[allow(unsafe_code)]
pub unsafe fn handle_mouse_event(
pub unsafe fn handle_mouse_button_event(
&self,
button: MouseButton,
client_point: Point2D<f32>,
@ -3211,9 +3222,56 @@ impl Document {
animations: DomRefCell::new(Animations::new()),
dirty_root: Default::default(),
declarative_refresh: Default::default(),
pending_animation_ticks: Default::default(),
pending_compositor_events: Default::default(),
mouse_move_event_index: Default::default(),
}
}
/// Note a pending animation tick, to be processed at the next `update_the_rendering` task.
pub fn note_pending_animation_tick(&self, tick_type: AnimationTickType) {
self.pending_animation_ticks.borrow_mut().extend(tick_type);
}
/// As part of a `update_the_rendering` task, tick all pending animations.
pub fn tick_all_animations(&self) {
let tick_type = mem::take(&mut *self.pending_animation_ticks.borrow_mut());
if tick_type.contains(AnimationTickType::REQUEST_ANIMATION_FRAME) {
self.run_the_animation_frame_callbacks();
}
if tick_type.contains(AnimationTickType::CSS_ANIMATIONS_AND_TRANSITIONS) {
self.maybe_mark_animating_nodes_as_dirty();
}
}
/// Note a pending compositor event, to be processed at the next `update_the_rendering` task.
pub fn note_pending_compositor_event(&self, event: CompositorEvent) {
let mut pending_compositor_events = self.pending_compositor_events.borrow_mut();
if matches!(event, CompositorEvent::MouseMoveEvent { .. }) {
// First try to replace any existing mouse move event.
if let Some(mouse_move_event) = self
.mouse_move_event_index
.borrow()
.and_then(|index| pending_compositor_events.get_mut(index))
{
*mouse_move_event = event;
return;
}
*self.mouse_move_event_index.borrow_mut() =
Some(self.pending_compositor_events.borrow().len());
}
self.pending_compositor_events.borrow_mut().push(event);
}
/// Get pending compositor events, for processing within an `update_the_rendering` task.
pub fn take_pending_compositor_events(&self) -> Vec<CompositorEvent> {
// Reset the mouse event index.
*self.mouse_move_event_index.borrow_mut() = None;
mem::take(&mut *self.pending_compositor_events.borrow_mut())
}
pub fn set_csp_list(&self, csp_list: Option<CspList>) {
*self.csp_list.borrow_mut() = csp_list;
}

View file

@ -12,7 +12,7 @@ use std::ptr::NonNull;
use std::rc::Rc;
use std::sync::atomic::Ordering;
use std::sync::{Arc, Mutex};
use std::{cmp, env};
use std::{cmp, env, mem};
use app_units::Au;
use backtrace::Backtrace;
@ -220,9 +220,9 @@ pub struct Window {
#[no_trace]
devtools_marker_sender: DomRefCell<Option<IpcSender<Option<TimelineMarker>>>>,
/// Pending resize event, if any.
/// Pending resize events, if any.
#[no_trace]
resize_event: Cell<Option<(WindowSizeData, WindowSizeType)>>,
resize_events: DomRefCell<Vec<(WindowSizeData, WindowSizeType)>>,
/// Parent id associated with this page, if any.
#[no_trace]
@ -2328,14 +2328,12 @@ impl Window {
.set(self.pending_reflow_count.get() + 1);
}
pub fn set_resize_event(&self, event: WindowSizeData, event_type: WindowSizeType) {
self.resize_event.set(Some((event, event_type)));
pub fn add_resize_event(&self, event: WindowSizeData, event_type: WindowSizeType) {
self.resize_events.borrow_mut().push((event, event_type));
}
pub fn steal_resize_event(&self) -> Option<(WindowSizeData, WindowSizeType)> {
let event = self.resize_event.get();
self.resize_event.set(None);
event
pub fn steal_resize_events(&self) -> Vec<(WindowSizeData, WindowSizeType)> {
mem::take(&mut self.resize_events.borrow_mut())
}
pub fn set_page_clip_rect_with_new_viewport(&self, viewport: UntypedRect<f32>) -> bool {
@ -2459,7 +2457,6 @@ impl Window {
);
event.upcast::<Event>().fire(mql.upcast::<EventTarget>());
}
self.Document().react_to_environment_changes();
}
/// Set whether to use less resources by running timers at a heavily limited rate.
@ -2606,7 +2603,7 @@ impl Window {
bluetooth_thread,
bluetooth_extra_permission_data: BluetoothExtraPermissionData::new(),
page_clip_rect: Cell::new(MaxRect::max_rect()),
resize_event: Default::default(),
resize_events: Default::default(),
window_size: Cell::new(window_size),
current_viewport: Cell::new(initial_viewport.to_untyped()),
suppress_reflow: Cell::new(true),

View file

@ -75,18 +75,13 @@ use script_layout_interface::{
Layout, LayoutConfig, LayoutFactory, ReflowGoal, ScriptThreadFactory,
};
use script_traits::webdriver_msg::WebDriverScriptCommand;
use script_traits::CompositorEvent::{
CompositionEvent, GamepadEvent, IMEDismissedEvent, KeyboardEvent, MouseButtonEvent,
MouseMoveEvent, ResizeEvent, TouchEvent, WheelEvent,
};
use script_traits::{
AnimationTickType, CompositorEvent, ConstellationControlMsg, DiscardBrowsingContext,
DocumentActivity, EventResult, HistoryEntryReplacement, InitialScriptState, JsEvalResult,
LayoutControlMsg, LayoutMsg, LoadData, LoadOrigin, MediaSessionActionType, MouseButton,
MouseEventType, NewLayoutInfo, Painter, ProgressiveWebMetricType, ScriptMsg,
ScriptToConstellationChan, StructuredSerializedData, TimerSchedulerMsg, TouchEventType,
TouchId, UntrustedNodeAddress, UpdatePipelineIdReason, WebrenderIpcSender, WheelDelta,
WindowSizeData, WindowSizeType,
CompositorEvent, ConstellationControlMsg, DiscardBrowsingContext, DocumentActivity,
EventResult, HistoryEntryReplacement, InitialScriptState, JsEvalResult, LayoutControlMsg,
LayoutMsg, LoadData, LoadOrigin, MediaSessionActionType, MouseButton, MouseEventType,
NewLayoutInfo, Painter, ProgressiveWebMetricType, ScriptMsg, ScriptToConstellationChan,
StructuredSerializedData, TimerSchedulerMsg, TouchEventType, TouchId, UntrustedNodeAddress,
UpdatePipelineIdReason, WebrenderIpcSender, WheelDelta, WindowSizeData, WindowSizeType,
};
use servo_atoms::Atom;
use servo_config::opts;
@ -160,6 +155,7 @@ use crate::task_source::networking::NetworkingTaskSource;
use crate::task_source::performance_timeline::PerformanceTimelineTaskSource;
use crate::task_source::port_message::PortMessageQueue;
use crate::task_source::remote_event::RemoteEventTaskSource;
use crate::task_source::rendering::RenderingTaskSource;
use crate::task_source::timer::TimerTaskSource;
use crate::task_source::user_interaction::UserInteractionTaskSource;
use crate::task_source::websocket::WebsocketTaskSource;
@ -517,6 +513,10 @@ unsafe_no_jsmanaged_fields!(TaskQueue<MainThreadScriptMsg>);
// ScriptThread instances are rooted on creation, so this is okay
#[allow(crown::unrooted_must_root)]
pub struct ScriptThread {
/// <https://html.spec.whatwg.org/multipage/#last-render-opportunity-time>
last_render_opportunity_time: DomRefCell<Option<Instant>>,
/// Used to batch rendering opportunities
has_queued_update_the_rendering_task: DomRefCell<bool>,
/// The documents for pipelines managed by this thread
documents: DomRefCell<Documents>,
/// The window proxies known by this thread
@ -575,6 +575,8 @@ pub struct ScriptThread {
remote_event_task_sender: Box<dyn ScriptChan>,
rendering_task_sender: Box<dyn ScriptChan>,
/// A channel to hand out to threads that need to respond to a message from the script thread.
#[no_trace]
control_chan: IpcSender<ConstellationControlMsg>,
@ -1345,6 +1347,8 @@ impl ScriptThread {
ScriptThread {
documents: DomRefCell::new(Documents::default()),
last_render_opportunity_time: Default::default(),
has_queued_update_the_rendering_task: Default::default(),
window_proxies: DomRefCell::new(HashMapTracedValues::new()),
incomplete_loads: DomRefCell::new(vec![]),
incomplete_parser_contexts: IncompleteParserContexts(RefCell::new(vec![])),
@ -1372,6 +1376,7 @@ impl ScriptThread {
performance_timeline_task_sender: boxed_script_sender.clone(),
timer_task_sender: boxed_script_sender.clone(),
remote_event_task_sender: boxed_script_sender.clone(),
rendering_task_sender: boxed_script_sender.clone(),
history_traversal_task_sender: chan.clone(),
@ -1471,28 +1476,311 @@ impl ScriptThread {
debug!("Stopped script thread.");
}
/// Handle incoming control messages.
/// <https://drafts.csswg.org/cssom-view/#document-run-the-resize-steps>
fn run_the_resize_steps(
&self,
id: PipelineId,
size: WindowSizeData,
size_type: WindowSizeType,
) {
self.profile_event(ScriptThreadEventCategory::Resize, Some(id), || {
self.handle_resize_event(id, size, size_type);
});
}
/// Process a compositor mouse move event.
fn process_mouse_move_event(
&self,
document: &Document,
window: &Window,
point: Point2D<f32>,
node_address: Option<UntrustedNodeAddress>,
pressed_mouse_buttons: u16,
) {
// Get the previous target temporarily
let prev_mouse_over_target = self.topmost_mouse_over_target.get();
unsafe {
document.handle_mouse_move_event(
point,
&self.topmost_mouse_over_target,
node_address,
pressed_mouse_buttons,
)
}
// Short-circuit if nothing changed
if self.topmost_mouse_over_target.get() == prev_mouse_over_target {
return;
}
let mut state_already_changed = false;
// Notify Constellation about the topmost anchor mouse over target.
if let Some(target) = self.topmost_mouse_over_target.get() {
if let Some(anchor) = target
.upcast::<Node>()
.inclusive_ancestors(ShadowIncluding::No)
.filter_map(DomRoot::downcast::<HTMLAnchorElement>)
.next()
{
let status = anchor
.upcast::<Element>()
.get_attribute(&ns!(), &local_name!("href"))
.and_then(|href| {
let value = href.value();
let url = document.url();
url.join(&value).map(|url| url.to_string()).ok()
});
let event = EmbedderMsg::Status(status);
window.send_to_embedder(event);
state_already_changed = true;
}
}
// We might have to reset the anchor state
if !state_already_changed {
if let Some(target) = prev_mouse_over_target {
if target
.upcast::<Node>()
.inclusive_ancestors(ShadowIncluding::No)
.filter_map(DomRoot::downcast::<HTMLAnchorElement>)
.next()
.is_some()
{
let event = EmbedderMsg::Status(None);
window.send_to_embedder(event);
}
}
}
}
/// Process compositor events as part of a "update the rendering task".
fn process_pending_compositor_events(&self, pipeline_id: PipelineId) {
let Some(document) = self.documents.borrow().find_document(pipeline_id) else {
warn!("Processing pending compositor events for closed pipeline {pipeline_id}.");
return;
};
// Do not handle events if the BC has been, or is being, discarded
if document.window().Closed() {
warn!("Compositor event sent to a pipeline with a closed window {pipeline_id}.");
return;
}
ScriptThread::set_user_interacting(true);
let window = document.window();
let _realm = enter_realm(document.window());
for event in document.take_pending_compositor_events().into_iter() {
match event {
CompositorEvent::ResizeEvent(new_size, size_type) => {
window.add_resize_event(new_size, size_type);
},
CompositorEvent::MouseButtonEvent(
event_type,
button,
point,
node_address,
point_in_node,
pressed_mouse_buttons,
) => {
self.handle_mouse_button_event(
pipeline_id,
event_type,
button,
point,
node_address,
point_in_node,
pressed_mouse_buttons,
);
},
CompositorEvent::MouseMoveEvent(point, node_address, pressed_mouse_buttons) => {
self.process_mouse_move_event(
&document,
&window,
point,
node_address,
pressed_mouse_buttons,
);
},
CompositorEvent::TouchEvent(event_type, identifier, point, node_address) => {
let touch_result = self.handle_touch_event(
pipeline_id,
event_type,
identifier,
point,
node_address,
);
match (event_type, touch_result) {
(TouchEventType::Down, TouchEventResult::Processed(handled)) => {
let result = if handled {
// TODO: Wait to see if preventDefault is called on the first touchmove event.
EventResult::DefaultAllowed
} else {
EventResult::DefaultPrevented
};
let message = ScriptMsg::TouchEventProcessed(result);
self.script_sender.send((pipeline_id, message)).unwrap();
},
_ => {
// TODO: Calling preventDefault on a touchup event should prevent clicks.
},
}
},
CompositorEvent::WheelEvent(delta, point, node_address) => {
self.handle_wheel_event(pipeline_id, delta, point, node_address);
},
CompositorEvent::KeyboardEvent(key_event) => {
document.dispatch_key_event(key_event);
},
CompositorEvent::IMEDismissedEvent => {
document.ime_dismissed();
},
CompositorEvent::CompositionEvent(composition_event) => {
document.dispatch_composition_event(composition_event);
},
CompositorEvent::GamepadEvent(gamepad_event) => {
let global = window.upcast::<GlobalScope>();
global.handle_gamepad_event(gamepad_event);
},
}
}
ScriptThread::set_user_interacting(false);
}
/// <https://html.spec.whatwg.org/multipage/#update-the-rendering>
fn update_the_rendering(&self) {
*self.has_queued_update_the_rendering_task.borrow_mut() = false;
if !self.can_continue_running_inner() {
return;
}
// TODO: The specification says to filter out non-renderable documents,
// as well as those for which a rendering update would be unnecessary,
// but this isn't happening here.
let pipeline_and_docs: Vec<(PipelineId, DomRoot<Document>)> = self
.documents
.borrow()
.iter()
.map(|(id, document)| (id, DomRoot::from_ref(&*document)))
.collect();
// Note: the spec reads: "for doc in docs" at each step
// whereas this runs all steps per doc in docs.
for (pipeline_id, document) in pipeline_and_docs {
// TODO(#32004): The rendering should be updated according parent and shadow root order
// in the specification, but this isn't happening yet.
// TODO(#31581): The steps in the "Revealing the document" section need to be implemente
// `process_pending_compositor_events` handles the focusing steps as well as other events
// from the compositor.
// TODO: Should this be broken and to match the specification more closely? For instance see
// https://html.spec.whatwg.org/multipage/#flush-autofocus-candidates.
self.process_pending_compositor_events(pipeline_id);
// TODO(#31665): Implement the "run the scroll steps" from
// https://drafts.csswg.org/cssom-view/#document-run-the-scroll-steps.
for (size, size_type) in document.window().steal_resize_events().into_iter() {
// Resize steps.
self.run_the_resize_steps(pipeline_id, size, size_type);
// Evaluate media queries and report changes.
document
.window()
.evaluate_media_queries_and_report_changes();
// https://html.spec.whatwg.org/multipage/#img-environment-changes
// As per the spec, this can be run at any time.
document.react_to_environment_changes()
}
// Update animations and send events.
self.update_animations_and_send_events();
// TODO(#31866): Implement "run the fullscreen steps" from
// https://fullscreen.spec.whatwg.org/multipage/#run-the-fullscreen-steps.
// TODO(#31868): Implement the "context lost steps" from
// https://html.spec.whatwg.org/multipage/#context-lost-steps.
// Run the animation frame callbacks.
document.tick_all_animations();
// TODO(#31006): Implement the resize observer steps.
// TODO(#31870): Implement step 17: if the focused area of doc is not a focusable area,
// then run the focusing steps for document's viewport.
// TODO: Perform pending transition operations from
// https://drafts.csswg.org/css-view-transitions/#perform-pending-transition-operations.
// TODO(#31021): Run the update intersection observations steps from
// https://w3c.github.io/IntersectionObserver/#run-the-update-intersection-observations-steps
// TODO: Mark paint timing from https://w3c.github.io/paint-timing.
// TODO(#31871): Update the rendering: consolidate all reflow calls into one here?
// TODO: Process top layer removals according to
// https://drafts.csswg.org/css-position-4/#process-top-layer-removals.
}
}
/// <https://html.spec.whatwg.org/multipage/#event-loop-processing-model:rendering-opportunity>
fn rendering_opportunity(&self, pipeline_id: PipelineId) {
*self.last_render_opportunity_time.borrow_mut() = Some(Instant::now());
// Note: the pipeline should be a navigable with a rendering opportunity,
// and we should use this opportunity to queue one task for each navigable with
// an opportunity in this script-thread.
let Some(document) = self.documents.borrow().find_document(pipeline_id) else {
warn!("Trying to update the rendering for closed pipeline {pipeline_id}.");
return;
};
let window = document.window();
let task_manager = window.task_manager();
let rendering_task_source = task_manager.rendering_task_source();
let canceller = task_manager.task_canceller(TaskSourceName::Rendering);
if *self.has_queued_update_the_rendering_task.borrow() {
return;
}
*self.has_queued_update_the_rendering_task.borrow_mut() = true;
// Queues a task to update the rendering.
// <https://html.spec.whatwg.org/multipage/#event-loop-processing-model:queue-a-global-task>
let _ = rendering_task_source.queue_with_canceller(
task!(update_the_rendering: move || {
// Note: spec says to queue a task using the navigable's active window,
// but then updates the rendering for all docs in the same event-loop.
SCRIPT_THREAD_ROOT.with(|root| {
if let Some(script_thread) = root.get() {
let script_thread = unsafe {&*script_thread};
script_thread.update_the_rendering();
}
})
}),
&canceller,
);
}
/// Handle incoming messages from other tasks and the task queue.
fn handle_msgs(&self) -> bool {
use self::MixedMessage::{
FromConstellation, FromDevtools, FromImageCache, FromScript, FromWebGPUServer,
};
// Handle pending resize events.
// Gather them first to avoid a double mut borrow on self.
let mut resizes = vec![];
for (id, document) in self.documents.borrow().iter() {
// Only process a resize if layout is idle.
if let Some((size, size_type)) = document.window().steal_resize_event() {
resizes.push((id, size, size_type));
}
}
for (id, size, size_type) in resizes {
self.handle_event(id, ResizeEvent(size, size_type));
}
// Store new resizes, and gather all other events.
// Proritize rendering tasks and others, and gather all other events as `sequential`.
let mut sequential = vec![];
// Notify the background-hang-monitor we are waiting for an event.
@ -1518,9 +1806,6 @@ impl ScriptThread {
};
debug!("Got event.");
// Squash any pending resize, reflow, animation tick, and mouse-move events in the queue.
let mut mouse_move_event_index = None;
let mut animation_ticks = HashSet::new();
loop {
let pipeline_id = self.message_to_pipeline(&event);
let _realm = pipeline_id.map(|id| {
@ -1570,10 +1855,7 @@ impl ScriptThread {
)
},
FromConstellation(ConstellationControlMsg::Resize(id, size, size_type)) => {
// step 7.7
self.profile_event(ScriptThreadEventCategory::Resize, Some(id), || {
self.handle_resize(id, size, size_type);
})
self.handle_resize_message(id, size, size_type);
},
FromConstellation(ConstellationControlMsg::Viewport(id, rect)) => self
.profile_event(ScriptThreadEventCategory::SetViewport, Some(id), || {
@ -1584,20 +1866,33 @@ impl ScriptThread {
self.handle_set_scroll_state(id, &scroll_state);
})
},
FromConstellation(ConstellationControlMsg::TickAllAnimations(pipeline_id, _)) => {
if !animation_ticks.contains(&pipeline_id) {
animation_ticks.insert(pipeline_id);
sequential.push(event);
FromConstellation(ConstellationControlMsg::TickAllAnimations(
pipeline_id,
tick_type,
)) => {
if let Some(doc) = self.documents.borrow().find_document(pipeline_id) {
self.rendering_opportunity(pipeline_id);
doc.note_pending_animation_tick(tick_type);
} else {
warn!(
"Trying to note pending animation tick for closed pipeline {}.",
pipeline_id
)
}
},
FromConstellation(ConstellationControlMsg::SendEvent(_, MouseMoveEvent(..))) => {
match mouse_move_event_index {
None => {
mouse_move_event_index = Some(sequential.len());
sequential.push(event);
FromConstellation(ConstellationControlMsg::SendEvent(id, event)) => {
self.handle_event(id, event)
},
Some(index) => sequential[index] = event,
}
FromScript(MainThreadScriptMsg::Common(CommonScriptMsg::Task(
_,
task,
_pipeline_id,
TaskSourceName::Rendering,
))) => {
// Run the "update the rendering" task.
task.run_box();
// Always perform a microtrask checkpoint after running a task.
self.perform_a_microtask_checkpoint();
},
FromScript(MainThreadScriptMsg::Inactive) => {
// An event came-in from a document that is not fully-active, it has been stored by the task-queue.
@ -1616,7 +1911,7 @@ impl ScriptThread {
// and check for more resize events. If there are no events pending, we'll move
// on and execute the sequential non-resize events we've seen.
match self.control_port.try_recv() {
Err(_) => match self.task_queue.try_recv() {
Err(_) => match self.task_queue.take_tasks_and_recv() {
Err(_) => match self.devtools_port.try_recv() {
Err(_) => match self.image_cache_port.try_recv() {
Err(_) => match &*self.webgpu_port.borrow() {
@ -1636,14 +1931,10 @@ impl ScriptThread {
}
}
// Step 11.10 from https://html.spec.whatwg.org/multipage/#event-loops.
self.update_animations_and_send_events();
// Process the gathered events.
debug!("Processing events.");
for msg in sequential {
debug!("Processing event {:?}.", msg);
let category = self.categorize_msg(&msg);
let pipeline_id = self.message_to_pipeline(&msg);
@ -1691,6 +1982,7 @@ impl ScriptThread {
}
// https://html.spec.whatwg.org/multipage/#event-loop-processing-model step 6
// TODO(#32003): A microtask checkpoint is only supposed to be performed after running a task.
self.perform_a_microtask_checkpoint();
}
@ -1982,7 +2274,6 @@ impl ScriptThread {
ConstellationControlMsg::UnloadDocument(pipeline_id) => {
self.handle_unload_document(pipeline_id)
},
ConstellationControlMsg::SendEvent(id, event) => self.handle_event(id, event),
ConstellationControlMsg::ResizeInactive(id, new_size) => {
self.handle_resize_inactive_msg(id, new_size)
},
@ -2044,9 +2335,6 @@ impl ScriptThread {
ConstellationControlMsg::WebDriverScriptCommand(pipeline_id, msg) => {
self.handle_webdriver_msg(pipeline_id, msg)
},
ConstellationControlMsg::TickAllAnimations(pipeline_id, tick_type) => {
self.handle_tick_all_animations(pipeline_id, tick_type)
},
ConstellationControlMsg::WebFontLoaded(pipeline_id) => {
self.handle_web_font_loaded(pipeline_id)
},
@ -2089,6 +2377,8 @@ impl ScriptThread {
msg @ ConstellationControlMsg::SetScrollState(..) |
msg @ ConstellationControlMsg::Resize(..) |
msg @ ConstellationControlMsg::ExitFullScreen(..) |
msg @ ConstellationControlMsg::SendEvent(..) |
msg @ ConstellationControlMsg::TickAllAnimations(..) |
msg @ ConstellationControlMsg::ExitScriptThread => {
panic!("should have handled {:?} already", msg)
},
@ -2453,10 +2743,18 @@ impl ScriptThread {
}
}
fn handle_resize(&self, id: PipelineId, size: WindowSizeData, size_type: WindowSizeType) {
/// Batch window resize operations into a single "update the rendering" task,
/// or, if a load is in progress, set the window size directly.
fn handle_resize_message(
&self,
id: PipelineId,
size: WindowSizeData,
size_type: WindowSizeType,
) {
let window = self.documents.borrow().find_window(id);
if let Some(ref window) = window {
window.set_resize_event(size, size_type);
self.rendering_opportunity(id);
window.add_resize_event(size, size_type);
return;
}
let mut loads = self.incomplete_loads.borrow_mut();
@ -2879,6 +3177,10 @@ impl ScriptThread {
RemoteEventTaskSource(self.remote_event_task_sender.clone(), pipeline_id)
}
fn rendering_task_source(&self, pipeline_id: PipelineId) -> RenderingTaskSource {
RenderingTaskSource(self.rendering_task_sender.clone(), pipeline_id)
}
pub fn timer_task_source(&self, pipeline_id: PipelineId) -> TimerTaskSource {
TimerTaskSource(self.timer_task_sender.clone(), pipeline_id)
}
@ -2989,23 +3291,12 @@ impl ScriptThread {
pub fn handle_tick_all_animations_for_testing(id: PipelineId) {
SCRIPT_THREAD_ROOT.with(|root| {
let script_thread = unsafe { &*root.get().unwrap() };
script_thread
.handle_tick_all_animations(id, AnimationTickType::CSS_ANIMATIONS_AND_TRANSITIONS);
});
}
/// Handles when layout finishes all animation in one tick
fn handle_tick_all_animations(&self, id: PipelineId, tick_type: AnimationTickType) {
let document = match self.documents.borrow().find_document(id) {
Some(document) => document,
None => return warn!("Message sent to closed pipeline {}.", id),
let Some(document) = script_thread.documents.borrow().find_document(id) else {
warn!("Animation tick for tests for closed pipeline {id}.");
return;
};
if tick_type.contains(AnimationTickType::REQUEST_ANIMATION_FRAME) {
document.run_the_animation_frame_callbacks();
}
if tick_type.contains(AnimationTickType::CSS_ANIMATIONS_AND_TRANSITIONS) {
document.maybe_mark_animating_nodes_as_dirty();
}
});
}
/// Handles a Web font being loaded. Does nothing if the page no longer exists.
@ -3242,6 +3533,7 @@ impl ScriptThread {
self.port_message_queue(incomplete.pipeline_id),
self.user_interaction_task_source(incomplete.pipeline_id),
self.remote_event_task_source(incomplete.pipeline_id),
self.rendering_task_source(incomplete.pipeline_id),
self.timer_task_source(incomplete.pipeline_id),
self.websocket_task_source(incomplete.pipeline_id),
);
@ -3477,190 +3769,19 @@ impl ScriptThread {
window.reflow(ReflowGoal::Full, reason);
}
/// This is the main entry point for receiving and dispatching DOM events.
///
/// TODO: Actually perform DOM event dispatch.
/// Queue compositor events for later dispatching as part of a
/// `update_the_rendering` task.
fn handle_event(&self, pipeline_id: PipelineId, event: CompositorEvent) {
// Do not handle events if the pipeline exited.
let window = match self.documents.borrow().find_window(pipeline_id) {
Some(win) => win,
None => {
return warn!(
"Compositor event sent to a pipeline that already exited {}.",
pipeline_id
)
},
};
// Do not handle events if the BC has been, or is being, discarded
if window.Closed() {
return warn!(
"Compositor event sent to a pipeline with a closed window {}.",
pipeline_id
);
}
let _realm = enter_realm(&*window);
// Assuming all CompositionEvent are generated by user interactions.
ScriptThread::set_user_interacting(true);
match event {
ResizeEvent(new_size, size_type) => {
self.handle_resize_event(pipeline_id, new_size, size_type);
},
MouseButtonEvent(
event_type,
button,
point,
node_address,
point_in_node,
pressed_mouse_buttons,
) => {
self.handle_mouse_event(
pipeline_id,
event_type,
button,
point,
node_address,
point_in_node,
pressed_mouse_buttons,
);
},
MouseMoveEvent(point, node_address, pressed_mouse_buttons) => {
let document = match self.documents.borrow().find_document(pipeline_id) {
Some(document) => document,
None => return warn!("Message sent to closed pipeline {}.", pipeline_id),
};
let window = document.window();
// Get the previous target temporarily
let prev_mouse_over_target = self.topmost_mouse_over_target.get();
unsafe {
document.handle_mouse_move_event(
point,
&self.topmost_mouse_over_target,
node_address,
pressed_mouse_buttons,
)
}
// Short-circuit if nothing changed
if self.topmost_mouse_over_target.get() == prev_mouse_over_target {
let Some(document) = self.documents.borrow().find_document(pipeline_id) else {
warn!("Compositor event sent to closed pipeline {pipeline_id}.");
return;
}
let mut state_already_changed = false;
// Notify Constellation about the topmost anchor mouse over target.
if let Some(target) = self.topmost_mouse_over_target.get() {
if let Some(anchor) = target
.upcast::<Node>()
.inclusive_ancestors(ShadowIncluding::No)
.filter_map(DomRoot::downcast::<HTMLAnchorElement>)
.next()
{
let status = anchor
.upcast::<Element>()
.get_attribute(&ns!(), &local_name!("href"))
.and_then(|href| {
let value = href.value();
let url = document.url();
url.join(&value).map(|url| url.to_string()).ok()
});
let event = EmbedderMsg::Status(status);
window.send_to_embedder(event);
state_already_changed = true;
}
}
// We might have to reset the anchor state
if !state_already_changed {
if let Some(target) = prev_mouse_over_target {
if target
.upcast::<Node>()
.inclusive_ancestors(ShadowIncluding::No)
.filter_map(DomRoot::downcast::<HTMLAnchorElement>)
.next()
.is_some()
{
let event = EmbedderMsg::Status(None);
window.send_to_embedder(event);
}
}
}
},
TouchEvent(event_type, identifier, point, node_address) => {
let touch_result = self.handle_touch_event(
pipeline_id,
event_type,
identifier,
point,
node_address,
);
match (event_type, touch_result) {
(TouchEventType::Down, TouchEventResult::Processed(handled)) => {
let result = if handled {
// TODO: Wait to see if preventDefault is called on the first touchmove event.
EventResult::DefaultAllowed
} else {
EventResult::DefaultPrevented
};
let message = ScriptMsg::TouchEventProcessed(result);
self.script_sender.send((pipeline_id, message)).unwrap();
},
_ => {
// TODO: Calling preventDefault on a touchup event should prevent clicks.
},
}
},
WheelEvent(delta, point, node_address) => {
self.handle_wheel_event(pipeline_id, delta, point, node_address);
},
KeyboardEvent(key_event) => {
let document = match self.documents.borrow().find_document(pipeline_id) {
Some(document) => document,
None => return warn!("Message sent to closed pipeline {}.", pipeline_id),
};
document.dispatch_key_event(key_event);
},
IMEDismissedEvent => {
let document = match self.documents.borrow().find_document(pipeline_id) {
Some(document) => document,
None => return warn!("Message sent to closed pipeline {}.", pipeline_id),
};
document.ime_dismissed();
},
CompositionEvent(composition_event) => {
let document = match self.documents.borrow().find_document(pipeline_id) {
Some(document) => document,
None => return warn!("Message sent to closed pipeline {}.", pipeline_id),
};
document.dispatch_composition_event(composition_event);
},
GamepadEvent(gamepad_event) => {
let window = match self.documents.borrow().find_window(pipeline_id) {
Some(window) => window,
None => return warn!("Message sent to closed pipeline {}.", pipeline_id),
};
let global = window.upcast::<GlobalScope>();
global.handle_gamepad_event(gamepad_event);
},
}
ScriptThread::set_user_interacting(false);
self.rendering_opportunity(pipeline_id);
document.note_pending_compositor_event(event);
}
#[allow(clippy::too_many_arguments)]
fn handle_mouse_event(
fn handle_mouse_button_event(
&self,
pipeline_id: PipelineId,
mouse_event_type: MouseEventType,
@ -3670,12 +3791,12 @@ impl ScriptThread {
point_in_node: Option<Point2D<f32>>,
pressed_mouse_buttons: u16,
) {
let document = match self.documents.borrow().find_document(pipeline_id) {
Some(document) => document,
None => return warn!("Message sent to closed pipeline {}.", pipeline_id),
let Some(document) = self.documents.borrow().find_document(pipeline_id) else {
warn!("Message sent to closed pipeline {pipeline_id}.");
return;
};
unsafe {
document.handle_mouse_event(
document.handle_mouse_button_event(
button,
point,
mouse_event_type,
@ -3711,9 +3832,9 @@ impl ScriptThread {
point: Point2D<f32>,
node_address: Option<UntrustedNodeAddress>,
) {
let document = match self.documents.borrow().find_document(pipeline_id) {
Some(document) => document,
None => return warn!("Message sent to closed pipeline {}.", pipeline_id),
let Some(document) = self.documents.borrow().find_document(pipeline_id) else {
warn!("Message sent to closed pipeline {pipeline_id}.");
return;
};
unsafe { document.handle_wheel_event(wheel_delta, point, node_address) };
}
@ -3814,11 +3935,6 @@ impl ScriptThread {
);
uievent.upcast::<Event>().fire(window.upcast());
}
// https://html.spec.whatwg.org/multipage/#event-loop-processing-model
// Step 7.7 - evaluate media queries and report changes
// Since we have resized, we need to re-evaluate MQLs
window.evaluate_media_queries_and_report_changes();
}
/// Instructs the constellation to fetch the document that will be loaded. Stores the InProgressLoad

View file

@ -17,6 +17,7 @@ use crate::task_source::networking::NetworkingTaskSource;
use crate::task_source::performance_timeline::PerformanceTimelineTaskSource;
use crate::task_source::port_message::PortMessageQueue;
use crate::task_source::remote_event::RemoteEventTaskSource;
use crate::task_source::rendering::RenderingTaskSource;
use crate::task_source::timer::TimerTaskSource;
use crate::task_source::user_interaction::UserInteractionTaskSource;
use crate::task_source::websocket::WebsocketTaskSource;
@ -59,6 +60,8 @@ pub struct TaskManager {
#[ignore_malloc_size_of = "task sources are hard"]
remote_event_task_source: RemoteEventTaskSource,
#[ignore_malloc_size_of = "task sources are hard"]
rendering_task_source: RenderingTaskSource,
#[ignore_malloc_size_of = "task sources are hard"]
timer_task_source: TimerTaskSource,
#[ignore_malloc_size_of = "task sources are hard"]
websocket_task_source: WebsocketTaskSource,
@ -77,6 +80,7 @@ impl TaskManager {
port_message_queue: PortMessageQueue,
user_interaction_task_source: UserInteractionTaskSource,
remote_event_task_source: RemoteEventTaskSource,
rendering_task_source: RenderingTaskSource,
timer_task_source: TimerTaskSource,
websocket_task_source: WebsocketTaskSource,
) -> Self {
@ -91,6 +95,7 @@ impl TaskManager {
port_message_queue,
user_interaction_task_source,
remote_event_task_source,
rendering_task_source,
timer_task_source,
websocket_task_source,
task_cancellers: Default::default(),
@ -177,6 +182,14 @@ impl TaskManager {
RemoteEvent
);
task_source_functions!(
self,
rendering_task_source_with_canceller,
rendering_task_source,
RenderingTaskSource,
Rendering
);
task_source_functions!(
self,
timer_task_source_with_canceller,

View file

@ -145,6 +145,12 @@ impl<T: QueuedTaskConversion> TaskQueue<T> {
}
for msg in incoming {
// Always run "update the rendering" tasks,
// TODO: fix "fully active" concept for iframes.
if let Some(TaskSourceName::Rendering) = msg.task_source_name() {
self.msg_queue.borrow_mut().push_back(msg);
continue;
}
if let Some(pipeline_id) = msg.pipeline_id() {
if !fully_active.contains(&pipeline_id) {
self.store_task_for_inactive_pipeline(msg, &pipeline_id);
@ -186,8 +192,9 @@ impl<T: QueuedTaskConversion> TaskQueue<T> {
self.msg_queue.borrow_mut().pop_front().ok_or(())
}
/// Same as recv.
pub fn try_recv(&self) -> Result<T, ()> {
/// Take all tasks again and then run `recv()`.
pub fn take_tasks_and_recv(&self) -> Result<T, ()> {
self.take_tasks(T::wake_up_msg());
self.recv()
}

View file

@ -11,6 +11,7 @@ pub mod networking;
pub mod performance_timeline;
pub mod port_message;
pub mod remote_event;
pub mod rendering;
pub mod timer;
pub mod user_interaction;
pub mod websocket;
@ -36,6 +37,8 @@ pub enum TaskSourceName {
PortMessage,
UserInteraction,
RemoteEvent,
/// <https://html.spec.whatwg.org/multipage/#rendering-task-source>
Rendering,
MediaElement,
Websocket,
Timer,

View file

@ -0,0 +1,45 @@
/* 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::fmt;
use std::result::Result;
use msg::constellation_msg::PipelineId;
use crate::script_runtime::{CommonScriptMsg, ScriptChan, ScriptThreadEventCategory};
use crate::task::{TaskCanceller, TaskOnce};
use crate::task_source::{TaskSource, TaskSourceName};
#[derive(JSTraceable)]
pub struct RenderingTaskSource(pub Box<dyn ScriptChan + Send>, #[no_trace] pub PipelineId);
impl Clone for RenderingTaskSource {
fn clone(&self) -> RenderingTaskSource {
RenderingTaskSource(self.0.clone(), self.1)
}
}
impl fmt::Debug for RenderingTaskSource {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "RenderingTaskSource(...)")
}
}
impl TaskSource for RenderingTaskSource {
const NAME: TaskSourceName = TaskSourceName::Rendering;
fn queue_with_canceller<T>(&self, task: T, canceller: &TaskCanceller) -> Result<(), ()>
where
T: TaskOnce + 'static,
{
let msg_task = CommonScriptMsg::Task(
ScriptThreadEventCategory::ScriptEvent,
Box::new(canceller.wrap_task(task)),
Some(self.1),
RenderingTaskSource::NAME,
);
self.0.send(msg_task).map_err(|_| ())
}
}

View file

@ -762,7 +762,7 @@ pub struct IFrameLoadInfoWithData {
}
bitflags! {
#[derive(Deserialize, Serialize)]
#[derive(Default, Deserialize, Serialize)]
/// Specifies if rAF should be triggered and/or CSS Animations and Transitions.
pub struct AnimationTickType: u8 {
/// Trigger a call to requestAnimationFrame.

View file

@ -2,6 +2,9 @@
[load & pageshow event do not fire on contentWindow of <iframe> element created with no src]
expected: FAIL
[load & pageshow events do not fire on contentWindow of <iframe> element created with src='']
expected: FAIL
[load & pageshow events do not fire on contentWindow of <iframe> element created with src='about:blank?foo']
expected: FAIL

View file

@ -2,6 +2,9 @@
[load event does not fire on window.open()]
expected: FAIL
[load event does not fire on window.open('about:blank')]
expected: FAIL
[load event does not fire on window.open('')]
expected: FAIL