script: Make timers per-process (#34581)

Before all timers were managed by the Constellation process, meaning
that they had to trigger IPC calls to be scheduled and fired. Currently,
timers are only used in the `ScriptThread`, so it makes sense that they
are per-process.

This change restores the timer thread functionality that existed before
avoided entirely. Completion is done using a callback that is sent to
the timer thread similarly to how fetch is done. This allows reusing the
existing task queue without making any new channels.

Fixes #15219.

Signed-off-by: Martin Robinson <mrobinson@igalia.com>
This commit is contained in:
Martin Robinson 2024-12-24 10:53:35 +01:00 committed by GitHub
parent ff7626bfc6
commit 226299380d
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
21 changed files with 366 additions and 324 deletions

View file

@ -118,6 +118,7 @@ swapper = "0.1"
tempfile = "3"
tendril = { version = "0.4.1", features = ["encoding_rs"] }
time_03 = { workspace = true }
timers = { path = "../timers" }
tracing = { workspace = true, optional = true }
unicode-bidi = { workspace = true }
unicode-segmentation = { workspace = true }

View file

@ -94,6 +94,7 @@ pub trait WorkerEventLoopMethods {
fn from_control_msg(msg: Self::ControlMsg) -> Self::Event;
fn from_worker_msg(msg: Self::WorkerMsg) -> Self::Event;
fn from_devtools_msg(msg: DevtoolScriptControlMsg) -> Self::Event;
fn from_timer_msg() -> Self::Event;
fn control_receiver(&self) -> &Receiver<Self::ControlMsg>;
}
@ -110,19 +111,25 @@ pub fn run_worker_event_loop<T, WorkerMsg, Event>(
+ DomObject,
{
let scope = worker_scope.upcast::<WorkerGlobalScope>();
let devtools_receiver = scope.devtools_receiver();
let task_queue = worker_scope.task_queue();
let never = crossbeam_channel::never();
let devtools_receiver = scope.devtools_receiver().unwrap_or(&never);
let event = select! {
recv(worker_scope.control_receiver()) -> msg => T::from_control_msg(msg.unwrap()),
recv(task_queue.select()) -> msg => {
task_queue.take_tasks(msg.unwrap());
T::from_worker_msg(task_queue.recv().unwrap())
},
recv(devtools_receiver.unwrap_or(&crossbeam_channel::never())) -> msg =>
T::from_devtools_msg(msg.unwrap()),
recv(devtools_receiver) -> msg => T::from_devtools_msg(msg.unwrap()),
recv(scope.timer_scheduler().wait_channel()) -> _ => T::from_timer_msg(),
};
let mut sequential = vec![];
sequential.push(event);
scope.timer_scheduler().dispatch_completed_timers();
let mut sequential = vec![event];
// https://html.spec.whatwg.org/multipage/#worker-event-loop
// Once the WorkerGlobalScope's closing flag is set to true,
// the event loop's task queues must discard any further tasks
@ -132,14 +139,14 @@ pub fn run_worker_event_loop<T, WorkerMsg, Event>(
// Batch all events that are ready.
// The task queue will throttle non-priority tasks if necessary.
match task_queue.take_tasks_and_recv() {
Err(_) => match devtools_receiver.map(|port| port.try_recv()) {
None => {},
Some(Err(_)) => break,
Some(Ok(ev)) => sequential.push(T::from_devtools_msg(ev)),
Err(_) => match devtools_receiver.try_recv() {
Ok(message) => sequential.push(T::from_devtools_msg(message)),
Err(_) => break,
},
Ok(ev) => sequential.push(T::from_worker_msg(ev)),
}
}
// Step 3
for event in sequential {
let _realm = enter_realm(worker_scope);

View file

@ -110,6 +110,7 @@ pub enum MixedMessage {
Worker(DedicatedWorkerScriptMsg),
Devtools(DevtoolScriptControlMsg),
Control(DedicatedWorkerControlMsg),
Timer,
}
impl QueuedTaskConversion for DedicatedWorkerScriptMsg {
@ -234,6 +235,10 @@ impl WorkerEventLoopMethods for DedicatedWorkerGlobalScope {
MixedMessage::Devtools(msg)
}
fn from_timer_msg() -> MixedMessage {
MixedMessage::Timer
}
fn control_receiver(&self) -> &Receiver<DedicatedWorkerControlMsg> {
&self.control_receiver
}
@ -564,6 +569,7 @@ impl DedicatedWorkerGlobalScope {
MixedMessage::Control(DedicatedWorkerControlMsg::Exit) => {
return false;
},
MixedMessage::Timer => {},
}
true
}

View file

@ -55,7 +55,6 @@ impl DissimilarOriginWindow {
global_to_clone_from.mem_profiler_chan().clone(),
global_to_clone_from.time_profiler_chan().clone(),
global_to_clone_from.script_to_constellation_chan().clone(),
global_to_clone_from.scheduler_chan().clone(),
global_to_clone_from.resource_threads().clone(),
global_to_clone_from.origin().clone(),
global_to_clone_from.creation_url().clone(),

View file

@ -57,10 +57,10 @@ use script_traits::serializable::{BlobData, BlobImpl, FileBlob};
use script_traits::transferable::MessagePortImpl;
use script_traits::{
BroadcastMsg, GamepadEvent, GamepadSupportedHapticEffects, GamepadUpdateType, MessagePortMsg,
PortMessageTask, ScriptMsg, ScriptToConstellationChan, TimerEvent, TimerEventId,
TimerSchedulerMsg, TimerSource,
PortMessageTask, ScriptMsg, ScriptToConstellationChan,
};
use servo_url::{ImmutableOrigin, MutableOrigin, ServoUrl};
use timers::{BoxedTimerCallback, TimerEvent, TimerEventId, TimerEventRequest, TimerSource};
use uuid::Uuid;
#[cfg(feature = "webgpu")]
use webgpu::{DeviceLostReason, WebGPUDevice};
@ -134,7 +134,7 @@ use crate::script_module::{DynamicModuleList, ModuleScript, ModuleTree, ScriptFe
use crate::script_runtime::{
CanGc, CommonScriptMsg, JSContext as SafeJSContext, ScriptChan, ScriptPort, ThreadSafeJSContext,
};
use crate::script_thread::{MainThreadScriptChan, ScriptThread};
use crate::script_thread::{with_script_thread, MainThreadScriptChan, ScriptThread};
use crate::security_manager::CSPViolationReporter;
use crate::task::TaskCanceller;
use crate::task_source::dom_manipulation::DOMManipulationTaskSource;
@ -258,10 +258,6 @@ pub struct GlobalScope {
#[no_trace]
script_to_constellation_chan: ScriptToConstellationChan,
#[ignore_malloc_size_of = "channels are hard"]
#[no_trace]
scheduler_chan: IpcSender<TimerSchedulerMsg>,
/// <https://html.spec.whatwg.org/multipage/#in-error-reporting-mode>
in_error_reporting_mode: Cell<bool>,
@ -272,10 +268,7 @@ pub struct GlobalScope {
/// The mechanism by which time-outs and intervals are scheduled.
/// <https://html.spec.whatwg.org/multipage/#timers>
timers: OneshotTimers,
/// Have timers been initialized?
init_timers: Cell<bool>,
timers: OnceCell<OneshotTimers>,
/// The origin of the globalscope
#[no_trace]
@ -401,7 +394,8 @@ struct BroadcastListener {
}
/// A wrapper between timer events coming in over IPC, and the event-loop.
struct TimerListener {
#[derive(Clone)]
pub(crate) struct TimerListener {
canceller: TaskCanceller,
task_source: TimerTaskSource,
context: Trusted<GlobalScope>,
@ -542,7 +536,7 @@ impl BroadcastListener {
}
impl TimerListener {
/// Handle a timer-event coming-in over IPC,
/// Handle a timer-event coming from the [`timers::TimerScheduler`]
/// by queuing the appropriate task on the relevant event-loop.
fn handle(&self, event: TimerEvent) {
let context = self.context.clone();
@ -567,6 +561,10 @@ impl TimerListener {
&self.canceller,
);
}
pub fn into_callback(self) -> BoxedTimerCallback {
Box::new(move |timer_event| self.handle(timer_event))
}
}
impl MessageListener {
@ -783,7 +781,6 @@ impl GlobalScope {
mem_profiler_chan: profile_mem::ProfilerChan,
time_profiler_chan: profile_time::ProfilerChan,
script_to_constellation_chan: ScriptToConstellationChan,
scheduler_chan: IpcSender<TimerSchedulerMsg>,
resource_threads: ResourceThreads,
origin: MutableOrigin,
creation_url: Option<ServoUrl>,
@ -811,11 +808,9 @@ impl GlobalScope {
mem_profiler_chan,
time_profiler_chan,
script_to_constellation_chan,
scheduler_chan: scheduler_chan.clone(),
in_error_reporting_mode: Default::default(),
resource_threads,
timers: OneshotTimers::new(scheduler_chan),
init_timers: Default::default(),
timers: OnceCell::default(),
origin,
creation_url,
permission_state_invocation_results: Default::default(),
@ -861,34 +856,21 @@ impl GlobalScope {
false
}
/// Setup the IPC-to-event-loop glue for timers to schedule themselves.
fn setup_timers(&self) {
if self.init_timers.get() {
return;
}
self.init_timers.set(true);
let (timer_ipc_chan, timer_ipc_port) = ipc::channel().unwrap();
self.timers.setup_scheduling(timer_ipc_chan);
// Setup route from IPC to task-queue for the timer-task-source.
let context = Trusted::new(self);
let (task_source, canceller) = (
self.timer_task_source(),
self.task_canceller(TaskSourceName::Timer),
);
let timer_listener = TimerListener {
context,
task_source,
canceller,
};
ROUTER.add_typed_route(
timer_ipc_port,
Box::new(move |message| {
let event = message.unwrap();
timer_listener.handle(event);
}),
);
fn timers(&self) -> &OneshotTimers {
self.timers.get_or_init(|| {
let (task_source, canceller) = (
self.timer_task_source(),
self.task_canceller(TaskSourceName::Timer),
);
OneshotTimers::new(
self,
TimerListener {
context: Trusted::new(self),
task_source,
canceller,
},
)
})
}
/// <https://w3c.github.io/ServiceWorker/#get-the-service-worker-registration-object>
@ -2377,10 +2359,6 @@ impl GlobalScope {
self.script_to_constellation_chan().send(msg).unwrap();
}
pub fn scheduler_chan(&self) -> &IpcSender<TimerSchedulerMsg> {
&self.scheduler_chan
}
/// Get the `PipelineId` for this global scope.
pub fn pipeline_id(&self) -> PipelineId {
self.pipeline_id
@ -2409,6 +2387,16 @@ impl GlobalScope {
unreachable!();
}
/// Schedule a [`TimerEventRequest`] on this [`GlobalScope`]'s [`timers::TimerScheduler`].
/// Every Worker has its own scheduler, which handles events in the Worker event loop,
/// but `Window`s use a shared scheduler associated with their [`ScriptThread`].
pub(crate) fn schedule_timer(&self, request: TimerEventRequest) {
match self.downcast::<WorkerGlobalScope>() {
Some(worker_global) => worker_global.timer_scheduler().schedule_timer(request),
_ => with_script_thread(|script_thread| script_thread.schedule_timer(request)),
}
}
/// <https://html.spec.whatwg.org/multipage/#concept-settings-object-policy-container>
pub fn policy_container(&self) -> PolicyContainer {
if let Some(window) = self.downcast::<Window>() {
@ -2789,13 +2777,12 @@ impl GlobalScope {
callback: OneshotTimerCallback,
duration: Duration,
) -> OneshotTimerHandle {
self.setup_timers();
self.timers
self.timers()
.schedule_callback(callback, duration, self.timer_source())
}
pub fn unschedule_callback(&self, handle: OneshotTimerHandle) {
self.timers.unschedule_callback(handle);
self.timers().unschedule_callback(handle);
}
/// <https://html.spec.whatwg.org/multipage/#timer-initialisation-steps>
@ -2806,8 +2793,7 @@ impl GlobalScope {
timeout: Duration,
is_interval: IsInterval,
) -> i32 {
self.setup_timers();
self.timers.set_timeout_or_interval(
self.timers().set_timeout_or_interval(
self,
callback,
arguments,
@ -2818,7 +2804,7 @@ impl GlobalScope {
}
pub fn clear_timeout_or_interval(&self, handle: i32) {
self.timers.clear_timeout_or_interval(self, handle);
self.timers().clear_timeout_or_interval(self, handle);
}
pub fn queue_function_as_microtask(&self, callback: Rc<VoidFunction>) {
@ -2925,23 +2911,23 @@ impl GlobalScope {
}
pub fn fire_timer(&self, handle: TimerEventId, can_gc: CanGc) {
self.timers.fire_timer(handle, self, can_gc);
self.timers().fire_timer(handle, self, can_gc);
}
pub fn resume(&self) {
self.timers.resume();
self.timers().resume();
}
pub fn suspend(&self) {
self.timers.suspend();
self.timers().suspend();
}
pub fn slow_down_timers(&self) {
self.timers.slow_down();
self.timers().slow_down();
}
pub fn speed_up_timers(&self) {
self.timers.speed_up();
self.timers().speed_up();
}
fn timer_source(&self) -> TimerSource {

View file

@ -125,6 +125,7 @@ pub enum MixedMessage {
ServiceWorker(ServiceWorkerScriptMsg),
Devtools(DevtoolScriptControlMsg),
Control(ServiceWorkerControlMsg),
Timer,
}
#[derive(Clone, JSTraceable)]
@ -212,6 +213,10 @@ impl WorkerEventLoopMethods for ServiceWorkerGlobalScope {
MixedMessage::Devtools(msg)
}
fn from_timer_msg() -> MixedMessage {
MixedMessage::Timer
}
fn control_receiver(&self) -> &Receiver<ServiceWorkerControlMsg> {
&self.control_receiver
}
@ -433,6 +438,7 @@ impl ServiceWorkerGlobalScope {
MixedMessage::Control(ServiceWorkerControlMsg::Exit) => {
return false;
},
MixedMessage::Timer => {},
}
true
}

View file

@ -57,8 +57,8 @@ use script_layout_interface::{
use script_traits::webdriver_msg::{WebDriverJSError, WebDriverJSResult};
use script_traits::{
ConstellationControlMsg, DocumentState, LoadData, NavigationHistoryBehavior, ScriptMsg,
ScriptToConstellationChan, ScrollState, StructuredSerializedData, Theme, TimerSchedulerMsg,
WindowSizeData, WindowSizeType,
ScriptToConstellationChan, ScrollState, StructuredSerializedData, Theme, WindowSizeData,
WindowSizeType,
};
use selectors::attr::CaseSensitivity;
use servo_arc::Arc as ServoArc;
@ -2697,7 +2697,6 @@ impl Window {
devtools_chan: Option<IpcSender<ScriptToDevtoolsControlMsg>>,
constellation_chan: ScriptToConstellationChan,
control_chan: IpcSender<ConstellationControlMsg>,
scheduler_chan: IpcSender<TimerSchedulerMsg>,
pipelineid: PipelineId,
parent_info: Option<PipelineId>,
window_size: WindowSizeData,
@ -2739,7 +2738,6 @@ impl Window {
mem_profiler_chan,
time_profiler_chan,
constellation_chan,
scheduler_chan,
resource_threads,
origin,
Some(creator_url),

View file

@ -2,6 +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::{RefCell, RefMut};
use std::default::Default;
use std::rc::Rc;
use std::sync::atomic::{AtomicBool, Ordering};
@ -24,6 +25,7 @@ use net_traits::request::{
use net_traits::IpcSend;
use script_traits::WorkerGlobalScopeInit;
use servo_url::{MutableOrigin, ServoUrl};
use timers::TimerScheduler;
use uuid::Uuid;
use super::bindings::codegen::Bindings::MessagePortBinding::StructuredSerializeOptions;
@ -80,7 +82,6 @@ pub fn prepare_workerscope_init(
time_profiler_chan: global.time_profiler_chan().clone(),
from_devtools_sender: devtools_sender,
script_to_constellation_chan: global.script_to_constellation_chan().clone(),
scheduler_chan: global.scheduler_chan().clone(),
worker_id: worker_id.unwrap_or_else(|| WorkerId(Uuid::new_v4())),
pipeline_id: global.pipeline_id(),
origin: global.origin().immutable().clone(),
@ -129,6 +130,11 @@ pub struct WorkerGlobalScope {
#[no_trace]
navigation_start: CrossProcessInstant,
performance: MutNullableDom<Performance>,
/// A [`TimerScheduler`] used to schedule timers for this [`ServiceWorkerGlobalScope`].
/// Timers are handled in the service worker event loop.
#[no_trace]
timer_scheduler: RefCell<TimerScheduler>,
}
impl WorkerGlobalScope {
@ -158,7 +164,6 @@ impl WorkerGlobalScope {
init.mem_profiler_chan,
init.time_profiler_chan,
init.script_to_constellation_chan,
init.scheduler_chan,
init.resource_threads,
MutableOrigin::new(init.origin),
init.creation_url,
@ -183,6 +188,7 @@ impl WorkerGlobalScope {
_devtools_sender: init.from_devtools_sender,
navigation_start: CrossProcessInstant::now(),
performance: Default::default(),
timer_scheduler: RefCell::default(),
}
}
@ -242,6 +248,11 @@ impl WorkerGlobalScope {
pub fn policy_container(&self) -> Ref<PolicyContainer> {
self.policy_container.borrow()
}
/// Get a mutable reference to the [`TimerScheduler`] for this [`ServiceWorkerGlobalScope`].
pub(crate) fn timer_scheduler(&self) -> RefMut<TimerScheduler> {
self.timer_scheduler.borrow_mut()
}
}
impl WorkerGlobalScopeMethods<crate::DomTypeHolder> for WorkerGlobalScope {

View file

@ -15,7 +15,7 @@ use js::rust::Runtime;
use net_traits::image_cache::ImageCache;
use net_traits::ResourceThreads;
use profile_traits::{mem, time};
use script_traits::{Painter, ScriptMsg, ScriptToConstellationChan, TimerSchedulerMsg};
use script_traits::{Painter, ScriptMsg, ScriptToConstellationChan};
use servo_atoms::Atom;
use servo_url::{ImmutableOrigin, MutableOrigin, ServoUrl};
@ -93,7 +93,6 @@ impl WorkletGlobalScope {
init.mem_profiler_chan.clone(),
init.time_profiler_chan.clone(),
script_to_constellation_chan,
init.scheduler_chan.clone(),
init.resource_threads.clone(),
MutableOrigin::new(ImmutableOrigin::new_opaque()),
None,
@ -186,8 +185,6 @@ pub struct WorkletGlobalScopeInit {
pub devtools_chan: Option<IpcSender<ScriptToDevtoolsControlMsg>>,
/// Messages to send to constellation
pub to_constellation_sender: IpcSender<(PipelineId, ScriptMsg)>,
/// Message to send to the scheduler
pub scheduler_chan: IpcSender<TimerSchedulerMsg>,
/// The image cache
pub image_cache: Arc<dyn ImageCache>,
/// True if in headless mode

View file

@ -85,14 +85,15 @@ use script_traits::{
EventResult, InitialScriptState, JsEvalResult, LayoutMsg, LoadData, LoadOrigin,
MediaSessionActionType, MouseButton, MouseEventType, NavigationHistoryBehavior, NewLayoutInfo,
Painter, ProgressiveWebMetricType, ScriptMsg, ScriptToConstellationChan, ScrollState,
StructuredSerializedData, Theme, TimerSchedulerMsg, TouchEventType, TouchId,
UntrustedNodeAddress, UpdatePipelineIdReason, WheelDelta, WindowSizeData, WindowSizeType,
StructuredSerializedData, Theme, TouchEventType, TouchId, UntrustedNodeAddress,
UpdatePipelineIdReason, WheelDelta, WindowSizeData, WindowSizeType,
};
use servo_atoms::Atom;
use servo_config::opts;
use servo_url::{ImmutableOrigin, MutableOrigin, ServoUrl};
use style::dom::OpaqueNode;
use style::thread_state::{self, ThreadState};
use timers::{TimerEventRequest, TimerScheduler};
use url::Position;
#[cfg(feature = "webgpu")]
use webgpu::{WebGPUDevice, WebGPUMsg};
@ -274,7 +275,6 @@ impl InProgressLoad {
}
#[derive(Debug)]
#[allow(clippy::enum_variant_names)]
enum MixedMessage {
FromConstellation(ConstellationControlMsg),
FromScript(MainThreadScriptMsg),
@ -282,6 +282,7 @@ enum MixedMessage {
FromImageCache((PipelineId, PendingImageResponse)),
#[cfg(feature = "webgpu")]
FromWebGPUServer(WebGPUMsg),
TimerFired,
}
/// Messages used to control the script event loop.
@ -490,6 +491,11 @@ pub struct ScriptThread {
/// events in the event queue.
chan: MainThreadScriptChan,
/// A [`TimerScheduler`] used to schedule timers for this [`ScriptThread`]. Timers are handled
/// in the [`ScriptThread`] event loop.
#[no_trace]
timer_scheduler: RefCell<TimerScheduler>,
dom_manipulation_task_sender: Box<dyn ScriptChan>,
gamepad_task_sender: Box<dyn ScriptChan>,
@ -573,9 +579,6 @@ pub struct ScriptThread {
#[no_trace]
closed_pipelines: DomRefCell<HashSet<PipelineId>>,
#[no_trace]
scheduler_chan: IpcSender<TimerSchedulerMsg>,
#[no_trace]
content_process_shutdown_chan: Sender<()>,
@ -881,6 +884,11 @@ impl ScriptThread {
})
}
/// Schedule a [`TimerEventRequest`] on this [`ScriptThread`]'s [`TimerScheduler`].
pub(crate) fn schedule_timer(&self, request: TimerEventRequest) {
self.timer_scheduler.borrow_mut().schedule_timer(request);
}
// https://html.spec.whatwg.org/multipage/#await-a-stable-state
pub fn await_stable_state(task: Microtask) {
with_script_thread(|script_thread| {
@ -1052,7 +1060,6 @@ impl ScriptThread {
time_profiler_chan: script_thread.time_profiler_chan.clone(),
devtools_chan: script_thread.devtools_chan.clone(),
to_constellation_sender: script_thread.script_sender.clone(),
scheduler_chan: script_thread.scheduler_chan.clone(),
image_cache: script_thread.image_cache.clone(),
is_headless: script_thread.headless,
user_agent: script_thread.user_agent.clone(),
@ -1208,6 +1215,7 @@ impl ScriptThread {
closing,
chan: MainThreadScriptChan(chan.clone()),
timer_scheduler: Default::default(),
dom_manipulation_task_sender: boxed_script_sender.clone(),
gamepad_task_sender: boxed_script_sender.clone(),
media_element_task_sender: chan.clone(),
@ -1238,8 +1246,6 @@ impl ScriptThread {
topmost_mouse_over_target: MutNullableDom::new(Default::default()),
closed_pipelines: DomRefCell::new(HashSet::new()),
scheduler_chan: state.scheduler_chan,
content_process_shutdown_chan: state.content_process_shutdown_chan,
mutation_observer_microtask_queued: Default::default(),
@ -1683,10 +1689,6 @@ impl ScriptThread {
/// Handle incoming messages from other tasks and the task queue.
fn handle_msgs(&self, can_gc: CanGc) -> bool {
#[cfg(feature = "webgpu")]
use self::MixedMessage::FromWebGPUServer;
use self::MixedMessage::{FromConstellation, FromDevtools, FromImageCache, FromScript};
// Proritize rendering tasks and others, and gather all other events as `sequential`.
let mut sequential = vec![];
@ -1702,12 +1704,13 @@ impl ScriptThread {
.task_queue
.recv()
.expect("Spurious wake-up of the event-loop, task-queue has no tasks available");
FromScript(event)
MixedMessage::FromScript(event)
},
recv(self.control_port) -> msg => FromConstellation(msg.unwrap()),
recv(self.control_port) -> msg => MixedMessage::FromConstellation(msg.unwrap()),
recv(self.devtools_chan.as_ref().map(|_| &self.devtools_port).unwrap_or(&crossbeam_channel::never())) -> msg
=> FromDevtools(msg.unwrap()),
recv(self.image_cache_port) -> msg => FromImageCache(msg.unwrap()),
=> MixedMessage::FromDevtools(msg.unwrap()),
recv(self.image_cache_port) -> msg => MixedMessage::FromImageCache(msg.unwrap()),
recv(self.timer_scheduler.borrow().wait_channel()) -> _ => MixedMessage::TimerFired,
recv({
#[cfg(feature = "webgpu")]
{
@ -1720,7 +1723,7 @@ impl ScriptThread {
}) -> msg => {
#[cfg(feature = "webgpu")]
{
FromWebGPUServer(msg.unwrap())
MixedMessage::FromWebGPUServer(msg.unwrap())
}
#[cfg(not(feature = "webgpu"))]
{
@ -1733,6 +1736,11 @@ impl ScriptThread {
loop {
debug!("Handling event: {event:?}");
// Dispatch any completed timers, so that their tasks can be run below.
self.timer_scheduler
.borrow_mut()
.dispatch_completed_timers();
let pipeline_id = self.message_to_pipeline(&event);
let _realm = pipeline_id.map(|id| {
let global = self.documents.borrow().find_global(id);
@ -1744,7 +1752,9 @@ impl ScriptThread {
// This has to be handled before the ResizeMsg below,
// otherwise the page may not have been added to the
// child list yet, causing the find() to fail.
FromConstellation(ConstellationControlMsg::AttachLayout(new_layout_info)) => {
MixedMessage::FromConstellation(ConstellationControlMsg::AttachLayout(
new_layout_info,
)) => {
let pipeline_id = new_layout_info.new_pipeline_id;
self.profile_event(
ScriptThreadEventCategory::AttachLayout,
@ -1780,14 +1790,19 @@ impl ScriptThread {
},
)
},
FromConstellation(ConstellationControlMsg::Resize(id, size, size_type)) => {
MixedMessage::FromConstellation(ConstellationControlMsg::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), || {
MixedMessage::FromConstellation(ConstellationControlMsg::Viewport(id, rect)) => {
self.profile_event(ScriptThreadEventCategory::SetViewport, Some(id), || {
self.handle_viewport(id, rect);
}),
FromConstellation(ConstellationControlMsg::TickAllAnimations(
})
},
MixedMessage::FromConstellation(ConstellationControlMsg::TickAllAnimations(
pipeline_id,
tick_type,
)) => {
@ -1801,10 +1816,10 @@ impl ScriptThread {
)
}
},
FromConstellation(ConstellationControlMsg::SendEvent(id, event)) => {
MixedMessage::FromConstellation(ConstellationControlMsg::SendEvent(id, event)) => {
self.handle_event(id, event)
},
FromScript(MainThreadScriptMsg::Common(CommonScriptMsg::Task(
MixedMessage::FromScript(MainThreadScriptMsg::Common(CommonScriptMsg::Task(
_,
_,
_,
@ -1814,14 +1829,15 @@ impl ScriptThread {
// message handling, we run those steps only once at the end of each call of
// this function.
},
FromScript(MainThreadScriptMsg::Inactive) => {
MixedMessage::FromScript(MainThreadScriptMsg::Inactive) => {
// An event came-in from a document that is not fully-active, it has been stored by the task-queue.
// Continue without adding it to "sequential".
},
FromConstellation(ConstellationControlMsg::ExitFullScreen(id)) => self
.profile_event(ScriptThreadEventCategory::ExitFullscreen, Some(id), || {
MixedMessage::FromConstellation(ConstellationControlMsg::ExitFullScreen(id)) => {
self.profile_event(ScriptThreadEventCategory::ExitFullscreen, Some(id), || {
self.handle_exit_fullscreen(id, can_gc);
}),
})
},
_ => {
sequential.push(event);
},
@ -1838,19 +1854,19 @@ impl ScriptThread {
Err(_) => match &*self.webgpu_port.borrow() {
Some(p) => match p.try_recv() {
Err(_) => break,
Ok(ev) => event = FromWebGPUServer(ev),
Ok(ev) => event = MixedMessage::FromWebGPUServer(ev),
},
None => break,
},
Ok(ev) => event = FromImageCache(ev),
Ok(ev) => event = MixedMessage::FromImageCache(ev),
#[cfg(not(feature = "webgpu"))]
Err(_) => break,
},
Ok(ev) => event = FromDevtools(ev),
Ok(ev) => event = MixedMessage::FromDevtools(ev),
},
Ok(ev) => event = FromScript(ev),
Ok(ev) => event = MixedMessage::FromScript(ev),
},
Ok(ev) => event = FromConstellation(ev),
Ok(ev) => event = MixedMessage::FromConstellation(ev),
}
}
@ -1869,11 +1885,11 @@ impl ScriptThread {
if self.closing.load(Ordering::SeqCst) {
// If we've received the closed signal from the BHM, only handle exit messages.
match msg {
FromConstellation(ConstellationControlMsg::ExitScriptThread) => {
MixedMessage::FromConstellation(ConstellationControlMsg::ExitScriptThread) => {
self.handle_exit_script_thread_msg(can_gc);
return false;
},
FromConstellation(ConstellationControlMsg::ExitPipeline(
MixedMessage::FromConstellation(ConstellationControlMsg::ExitPipeline(
pipeline_id,
discard_browsing_context,
)) => {
@ -1890,20 +1906,25 @@ impl ScriptThread {
let exiting = self.profile_event(category, pipeline_id, move || {
match msg {
FromConstellation(ConstellationControlMsg::ExitScriptThread) => {
MixedMessage::FromConstellation(ConstellationControlMsg::ExitScriptThread) => {
self.handle_exit_script_thread_msg(can_gc);
return true;
},
FromConstellation(inner_msg) => {
MixedMessage::FromConstellation(inner_msg) => {
self.handle_msg_from_constellation(inner_msg, can_gc)
},
FromScript(inner_msg) => self.handle_msg_from_script(inner_msg),
FromDevtools(inner_msg) => self.handle_msg_from_devtools(inner_msg, can_gc),
FromImageCache(inner_msg) => self.handle_msg_from_image_cache(inner_msg),
MixedMessage::FromScript(inner_msg) => self.handle_msg_from_script(inner_msg),
MixedMessage::FromDevtools(inner_msg) => {
self.handle_msg_from_devtools(inner_msg, can_gc)
},
MixedMessage::FromImageCache(inner_msg) => {
self.handle_msg_from_image_cache(inner_msg)
},
#[cfg(feature = "webgpu")]
FromWebGPUServer(inner_msg) => {
MixedMessage::FromWebGPUServer(inner_msg) => {
self.handle_msg_from_webgpu_server(inner_msg, can_gc)
},
MixedMessage::TimerFired => {},
}
false
@ -1955,6 +1976,7 @@ impl ScriptThread {
},
#[cfg(feature = "webgpu")]
MixedMessage::FromWebGPUServer(_) => ScriptThreadEventCategory::WebGPUMsg,
MixedMessage::TimerFired => ScriptThreadEventCategory::TimerEvent,
}
}
@ -2047,7 +2069,6 @@ impl ScriptThread {
SetScrollStates(id, ..) => Some(id),
SetEpochPaintTime(id, ..) => Some(id),
},
MixedMessage::FromDevtools(_) => None,
MixedMessage::FromScript(ref inner_msg) => match *inner_msg {
MainThreadScriptMsg::Common(CommonScriptMsg::Task(_, _, pipeline_id, _)) => {
pipeline_id
@ -2059,6 +2080,7 @@ impl ScriptThread {
MainThreadScriptMsg::WakeUp => None,
},
MixedMessage::FromImageCache((pipeline_id, _)) => Some(pipeline_id),
MixedMessage::FromDevtools(_) | MixedMessage::TimerFired => None,
#[cfg(feature = "webgpu")]
MixedMessage::FromWebGPUServer(..) => None,
}
@ -3653,7 +3675,6 @@ impl ScriptThread {
self.devtools_chan.clone(),
script_to_constellation_chan,
self.control_chan.clone(),
self.scheduler_chan.clone(),
incomplete.pipeline_id,
incomplete.parent_info,
incomplete.window_size,

View file

@ -10,21 +10,21 @@ use std::rc::Rc;
use std::time::{Duration, Instant};
use deny_public_fields::DenyPublicFields;
use ipc_channel::ipc::IpcSender;
use js::jsapi::Heap;
use js::jsval::{JSVal, UndefinedValue};
use js::rust::HandleValue;
use script_traits::{TimerEvent, TimerEventId, TimerEventRequest, TimerSchedulerMsg, TimerSource};
use servo_config::pref;
use timers::{TimerEventId, TimerEventRequest, TimerSource};
use crate::dom::bindings::callback::ExceptionHandling::Report;
use crate::dom::bindings::cell::DomRefCell;
use crate::dom::bindings::codegen::Bindings::FunctionBinding::Function;
use crate::dom::bindings::reflector::DomObject;
use crate::dom::bindings::root::DomRoot;
use crate::dom::bindings::str::DOMString;
use crate::dom::document::FakeRequestAnimationFrameCallback;
use crate::dom::eventsource::EventSourceTimeoutCallback;
use crate::dom::globalscope::GlobalScope;
use crate::dom::globalscope::{GlobalScope, TimerListener};
use crate::dom::htmlmetaelement::RefreshRedirectDue;
use crate::dom::testbinding::TestBindingCallback;
use crate::dom::xmlhttprequest::XHRTimeoutCallback;
@ -37,17 +37,11 @@ pub struct OneshotTimerHandle(i32);
#[derive(DenyPublicFields, JSTraceable, MallocSizeOf)]
pub struct OneshotTimers {
global_scope: DomRoot<GlobalScope>,
#[ignore_malloc_size_of = "Missing malloc_size_of for task types"]
#[no_trace]
timer_listener: TimerListener,
js_timers: JsTimers,
#[ignore_malloc_size_of = "Defined in std"]
#[no_trace]
/// The sender, to be cloned for each timer,
/// on which the timer scheduler in the constellation can send an event
/// when the timer is due.
timer_event_chan: DomRefCell<Option<IpcSender<TimerEvent>>>,
#[ignore_malloc_size_of = "Defined in std"]
#[no_trace]
/// The sender to the timer scheduler in the constellation.
scheduler_chan: IpcSender<TimerSchedulerMsg>,
next_timer_handle: Cell<OneshotTimerHandle>,
timers: DomRefCell<Vec<OneshotTimer>>,
suspended_since: Cell<Option<Instant>>,
@ -124,11 +118,11 @@ impl PartialEq for OneshotTimer {
}
impl OneshotTimers {
pub fn new(scheduler_chan: IpcSender<TimerSchedulerMsg>) -> OneshotTimers {
pub fn new(global_scope: &GlobalScope, timer_listener: TimerListener) -> OneshotTimers {
OneshotTimers {
global_scope: DomRoot::from_ref(global_scope),
timer_listener,
js_timers: JsTimers::default(),
timer_event_chan: DomRefCell::new(None),
scheduler_chan,
next_timer_handle: Cell::new(OneshotTimerHandle(1)),
timers: DomRefCell::new(Vec::new()),
suspended_since: Cell::new(None),
@ -137,12 +131,6 @@ impl OneshotTimers {
}
}
pub fn setup_scheduling(&self, timer_event_chan: IpcSender<TimerEvent>) {
let mut chan = self.timer_event_chan.borrow_mut();
assert!(chan.is_none());
*chan = Some(timer_event_chan);
}
pub fn schedule_callback(
&self,
callback: OneshotTimerCallback,
@ -291,24 +279,19 @@ impl OneshotTimers {
}
let timers = self.timers.borrow();
let Some(timer) = timers.last() else {
return;
};
if let Some(timer) = timers.last() {
let expected_event_id = self.invalidate_expected_event_id();
let expected_event_id = self.invalidate_expected_event_id();
let event_request = TimerEventRequest {
callback: self.timer_listener.clone().into_callback(),
source: timer.source,
id: expected_event_id,
duration: timer.scheduled_for - Instant::now(),
};
let delay = timer.scheduled_for - Instant::now();
let request = TimerEventRequest(
self.timer_event_chan
.borrow()
.clone()
.expect("Timer event chan not setup to schedule timers."),
timer.source,
expected_event_id,
delay,
);
self.scheduler_chan
.send(TimerSchedulerMsg(request))
.unwrap();
}
self.global_scope.schedule_timer(event_request);
}
fn invalidate_expected_event_id(&self) -> TimerEventId {