From 815ed10b5f3a49ef56da425f1e3653681bd98170 Mon Sep 17 00:00:00 2001 From: Gregory Terzian <2792687+gterzian@users.noreply.github.com> Date: Wed, 30 Jul 2025 21:03:28 +0800 Subject: [PATCH] background_hang_monitor: ensure workers run until monitored components do (#38322) Shut-down of the background hang monitor(bhm) is currently problematic: - it does not always run until the monitored script-thread does(see "BackgroundHangMonitor has gone away" mentioned in https://github.com/servo/servo/issues/34158). - it shuts-down before the constellation(good, so actually https://github.com/servo/servo/issues/24850 was "fixed" but in a way that introduced a new problem), but using a mechanism that allows it to shutdown before script(the problem above). - there are various mechanism(see the doc comments removed by this PR) in place which are meant to ensure a clean shutdown despite the above problems; those are complicated, and become unnecessary once those problems are fixed. All of the above is fixed by the changes in this PR, which ensure the bhm does not shut-down before script, and also maintains the invariant that it must shut-down before the constellation(in single-process mode) or before the main thread(in multi-process mode), but using a mechanism which allows it to keep running until script shuts-down. An unnecessary option around the exit signal is also removed. As a positive side-effect, it also ensures that any script-thread is shut-down before the constellation(because for the bhm worker to exit, the monitored script must have exited first), so this should also fix a host of other problems noted in https://github.com/servo/servo/issues/30849, but each should be confirmed independently(and various other improvements seem possible in their specific contexts, such as joining on script threads, and removing the `ScriptThreadMessage::ExitScriptThread`). Fixes: https://github.com/servo/servo/issues/24850 and part of https://github.com/servo/servo/issues/34158 Testing: Unit tests in `component/background_hang_monitor/tests`. Also manually tested loading "about-blank" in single- and multi-process mode. --------- Signed-off-by: gterzian <2792687+gterzian@users.noreply.github.com> --- .../background_hang_monitor.rs | 210 +++++------------- .../tests/hang_monitor_tests.rs | 54 +++-- components/constellation/constellation.rs | 62 ++++-- components/constellation/pipeline.rs | 3 +- components/script/script_thread.rs | 2 +- components/servo/lib.rs | 11 +- .../shared/background_hang_monitor/lib.rs | 7 +- 7 files changed, 147 insertions(+), 202 deletions(-) diff --git a/components/background_hang_monitor/background_hang_monitor.rs b/components/background_hang_monitor/background_hang_monitor.rs index 68f4f39f3a4..8e0d7797710 100644 --- a/components/background_hang_monitor/background_hang_monitor.rs +++ b/components/background_hang_monitor/background_hang_monitor.rs @@ -2,10 +2,8 @@ * 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::Cell; use std::collections::{HashMap, VecDeque}; -use std::sync::{Arc, Weak}; -use std::thread; +use std::thread::{self, Builder, JoinHandle}; use std::time::{Duration, Instant}; use background_hang_monitor_api::{ @@ -16,46 +14,33 @@ use background_hang_monitor_api::{ use crossbeam_channel::{Receiver, Sender, after, never, select, unbounded}; use ipc_channel::ipc::{IpcReceiver, IpcSender}; use ipc_channel::router::ROUTER; -use log::warn; use crate::sampler::{NativeStack, Sampler}; #[derive(Clone)] pub struct HangMonitorRegister { - sender: Weak>, - tether: Sender, + sender: MonitoredComponentSender, monitoring_enabled: bool, } impl HangMonitorRegister { - /// Start a new hang monitor worker, and return a handle to register components for monitoring. + /// Start a new hang monitor worker, and return a handle to register components for monitoring, + /// as well as a join handle on the worker thread. pub fn init( constellation_chan: IpcSender, control_port: IpcReceiver, monitoring_enabled: bool, - ) -> Box { - // Create a channel to pass messages of type `MonitoredComponentMsg`. - // See the discussion in `::register_component` for why we wrap - // the sender with `Arc` and why `HangMonitorRegister` only maintains - // a weak reference to it. + ) -> (Box, JoinHandle<()>) { let (sender, port) = unbounded(); - let sender = Arc::new(sender); - let sender_weak = Arc::downgrade(&sender); + let sender_clone = sender.clone(); - // Create a "tether" channel, whose sole purpose is to keep the worker - // thread alive. The worker thread will terminates when all copies of - // `tether` are dropped. - let (tether, tether_port) = unbounded(); - - let _ = thread::Builder::new() + let join_handle = Builder::new() .name("BackgroundHangMonitor".to_owned()) .spawn(move || { let mut monitor = BackgroundHangMonitorWorker::new( constellation_chan, control_port, - (sender, port), - tether_port, + port, monitoring_enabled, ); while monitor.run() { @@ -63,11 +48,13 @@ impl HangMonitorRegister { } }) .expect("Couldn't start BHM worker."); - Box::new(HangMonitorRegister { - sender: sender_weak, - tether, - monitoring_enabled, - }) + ( + Box::new(HangMonitorRegister { + sender: sender_clone, + monitoring_enabled, + }), + join_handle, + ) } } @@ -80,11 +67,10 @@ impl BackgroundHangMonitorRegister for HangMonitorRegister { component_id: MonitoredComponentId, transient_hang_timeout: Duration, permanent_hang_timeout: Duration, - exit_signal: Option>, + exit_signal: Box, ) -> Box { let bhm_chan = BackgroundHangMonitorChan::new( self.sender.clone(), - self.tether.clone(), component_id, self.monitoring_enabled, ); @@ -124,54 +110,12 @@ impl BackgroundHangMonitorRegister for HangMonitorRegister { ))] let sampler = crate::sampler::DummySampler::new_boxed(); - // When a component is registered, and there's an exit request that - // reached BHM, we want an exit signal to be delivered to the - // component's exit signal handler eventually. However, there's a race - // condition between the reception of `BackgroundHangMonitorControlMsg:: - // Exit` and `MonitoredComponentMsg::Register` that needs to handled - // carefully. When the worker receives an `Exit` message, it stops - // processing messages, and any further `Register` messages sent to the - // worker thread are ignored. If the submissions of `Exit` and - // `Register` messages are far apart enough, the channel is closed by - // the time the client attempts to send a `Register` message, and - // therefore the client can figure out by `Sender::send`'s return value - // that it must deliver an exit signal. However, if these message - // submissions are close enough, the `Register` message is still sent, - // but the worker thread might exit before it sees the message, leaving - // the message unprocessed and the exit signal unsent. - // - // To fix this, we wrap the exit signal handler in an RAII wrapper of - // type `SignalToExitOnDrop` to automatically send a signal when it's - // dropped. This way, we can make sure the exit signal is sent even if - // the message couldn't reach the worker thread and be processed. - // - // However, as it turns out, `crossbeam-channel`'s channels don't drop - // remaining messages until all associated senders *and* receivers are - // dropped. This means the exit signal won't be delivered as long as - // there's at least one `HangMonitorRegister` or - // `BackgroundHangMonitorChan` maintaining a copy of the sender. To work - // around this and guarantee a rapid delivery of the exit signal, the - // sender is wrapped in `Arc`, and only the worker thread maintains a - // strong reference, thus ensuring both the sender and receiver are - // dropped as soon as the worker thread exits. - let exit_signal = SignalToExitOnDrop(exit_signal); - - // If the tether is dropped after this call, the worker thread might - // exit before processing the `Register` message because there's no - // implicit ordering guarantee between two channels. If this happens, - // an exit signal will be sent despite we haven't received a - // corresponding exit request. To enforce the correct ordering and - // prevent a false exit signal from being sent, we include a copy of - // `self.tether` in the `Register` message. - let tether = self.tether.clone(); - bhm_chan.send(MonitoredComponentMsg::Register( sampler, thread::current().name().map(str::to_owned), transient_hang_timeout, permanent_hang_timeout, exit_signal, - tether, )); Box::new(bhm_chan) } @@ -191,8 +135,7 @@ enum MonitoredComponentMsg { Option, Duration, Duration, - SignalToExitOnDrop, - Sender, + Box, ), /// Unregister component for monitoring. Unregister, @@ -202,54 +145,32 @@ enum MonitoredComponentMsg { NotifyWait, } -/// Stable equivalent to the `!` type -enum Never {} - /// A wrapper around a sender to the monitor, /// which will send the Id of the monitored component along with each message, /// and keep track of whether the monitor is still listening on the other end. struct BackgroundHangMonitorChan { - sender: Weak>, - _tether: Sender, + sender: MonitoredComponentSender, component_id: MonitoredComponentId, - disconnected: Cell, monitoring_enabled: bool, } impl BackgroundHangMonitorChan { fn new( - sender: Weak>, - tether: Sender, + sender: MonitoredComponentSender, component_id: MonitoredComponentId, monitoring_enabled: bool, ) -> Self { BackgroundHangMonitorChan { sender, - _tether: tether, component_id, - disconnected: Default::default(), monitoring_enabled, } } fn send(&self, msg: MonitoredComponentMsg) { - if self.disconnected.get() { - return; - } - - // The worker thread owns both the receiver *and* the only strong - // reference to the sender. An `upgrade` failure means the latter is - // gone, and a `send` failure means the former is gone. They are dropped - // simultaneously, but we might observe an intermediate state. - if self - .sender - .upgrade() - .and_then(|sender| sender.send((self.component_id.clone(), msg)).ok()) - .is_none() - { - warn!("BackgroundHangMonitor has gone away"); - self.disconnected.set(true); - } + self.sender + .send((self.component_id.clone(), msg)) + .expect("BHM is gone"); } } @@ -272,33 +193,6 @@ impl BackgroundHangMonitor for BackgroundHangMonitorChan { } } -/// Wraps [`BackgroundHangMonitorExitSignal`] and calls `signal_to_exit` when -/// dropped. -struct SignalToExitOnDrop(Option>); - -impl SignalToExitOnDrop { - /// Call `BackgroundHangMonitorExitSignal::signal_to_exit` now. - fn signal_to_exit(&mut self) { - if let Some(signal) = self.0.take() { - signal.signal_to_exit(); - } - } - - /// Disassociate `BackgroundHangMonitorExitSignal` from itself, preventing - /// `BackgroundHangMonitorExitSignal::signal_to_exit` from being called in - /// the future. - fn release(&mut self) { - self.0 = None; - } -} - -impl Drop for SignalToExitOnDrop { - #[inline] - fn drop(&mut self) { - self.signal_to_exit(); - } -} - struct MonitoredComponent { sampler: Box, last_activity: Instant, @@ -308,7 +202,7 @@ struct MonitoredComponent { sent_transient_alert: bool, sent_permanent_alert: bool, is_waiting: bool, - exit_signal: SignalToExitOnDrop, + exit_signal: Box, } struct Sample(MonitoredComponentId, Instant, NativeStack); @@ -318,8 +212,6 @@ struct BackgroundHangMonitorWorker { monitored_components: HashMap, constellation_chan: IpcSender, port: Receiver<(MonitoredComponentId, MonitoredComponentMsg)>, - _port_sender: Arc>, - tether_port: Receiver, control_port: Receiver, sampling_duration: Option, sampling_max_duration: Option, @@ -328,6 +220,7 @@ struct BackgroundHangMonitorWorker { sampling_baseline: Instant, samples: VecDeque, monitoring_enabled: bool, + shutting_down: bool, } type MonitoredComponentSender = Sender<(MonitoredComponentId, MonitoredComponentMsg)>; @@ -337,8 +230,7 @@ impl BackgroundHangMonitorWorker { fn new( constellation_chan: IpcSender, control_port: IpcReceiver, - (port_sender, port): (Arc, MonitoredComponentReceiver), - tether_port: Receiver, + port: MonitoredComponentReceiver, monitoring_enabled: bool, ) -> Self { let control_port = ROUTER.route_ipc_receiver_to_new_crossbeam_receiver(control_port); @@ -347,8 +239,6 @@ impl BackgroundHangMonitorWorker { monitored_components: Default::default(), constellation_chan, port, - _port_sender: port_sender, - tether_port, control_port, sampling_duration: None, sampling_max_duration: None, @@ -357,6 +247,7 @@ impl BackgroundHangMonitorWorker { creation: Instant::now(), samples: Default::default(), monitoring_enabled, + shutting_down: Default::default(), } } @@ -415,19 +306,14 @@ impl BackgroundHangMonitorWorker { let received = select! { recv(self.port) -> event => { - // Since we own the `Arc>`, the channel never - // gets disconnected. - Some(event.unwrap()) - }, - recv(self.tether_port) -> _ => { - // This arm can only reached by a tether disconnection - // All associated `HangMonitorRegister` and - // `BackgroundHangMonitorChan` have been dropped. Suppress - // `signal_to_exit` and exit the BHM. - for component in self.monitored_components.values_mut() { - component.exit_signal.release(); + if let Ok(event) = event { + Some(event) + } else { + // All senders have dropped, + // which means all monitored components have shut down, + // and so we can as well. + return false; } - return false; }, recv(self.control_port) -> event => { match event { @@ -444,16 +330,21 @@ impl BackgroundHangMonitorWorker { } None }, - Ok(BackgroundHangMonitorControlMsg::Exit(sender)) => { + Ok(BackgroundHangMonitorControlMsg::Exit) => { for component in self.monitored_components.values_mut() { component.exit_signal.signal_to_exit(); } - // Confirm exit with to the constellation. - let _ = sender.send(()); + // Note the start of shutdown, + // to ensure exit propagates, + // even to components that have yet to register themselves, + // from this point on. + self.shutting_down = true; - // Also exit the BHM. - return false; + // Keep running; this worker thread will shutdown + // when the monitored components have shutdown, + // which we know has happened when `self.port` disconnects. + None }, Err(_) => return false, } @@ -492,9 +383,16 @@ impl BackgroundHangMonitorWorker { transient_hang_timeout, permanent_hang_timeout, exit_signal, - _tether, ), ) => { + // If we are shutting down, + // propagate it to the component, + // and register it(the component will unregister itself + // as part of handling the exit). + if self.shutting_down { + exit_signal.signal_to_exit(); + } + let component = MonitoredComponent { sampler, last_activity: Instant::now(), @@ -517,13 +415,9 @@ impl BackgroundHangMonitorWorker { ); }, (component_id, MonitoredComponentMsg::Unregister) => { - let (_, mut component) = self - .monitored_components + self.monitored_components .remove_entry(&component_id) .expect("Received Unregister for an unknown component"); - - // Prevent `signal_to_exit` from being called - component.exit_signal.release(); }, (component_id, MonitoredComponentMsg::NotifyActivity(annotation)) => { let component = self diff --git a/components/background_hang_monitor/tests/hang_monitor_tests.rs b/components/background_hang_monitor/tests/hang_monitor_tests.rs index a35ef7bfda4..8c1b089d06d 100644 --- a/components/background_hang_monitor/tests/hang_monitor_tests.rs +++ b/components/background_hang_monitor/tests/hang_monitor_tests.rs @@ -27,16 +27,23 @@ fn test_hang_monitoring() { ipc::channel().expect("ipc channel failure"); let (_sampler_sender, sampler_receiver) = ipc::channel().expect("ipc channel failure"); - let background_hang_monitor_register = HangMonitorRegister::init( + let (background_hang_monitor_register, join_handle) = HangMonitorRegister::init( background_hang_monitor_ipc_sender.clone(), sampler_receiver, true, ); + + struct BHMExitSignal; + + impl BackgroundHangMonitorExitSignal for BHMExitSignal { + fn signal_to_exit(&self) {} + } + let background_hang_monitor = background_hang_monitor_register.register_component( MonitoredComponentId(TEST_PIPELINE_ID, MonitoredComponentType::Script), Duration::from_millis(10), Duration::from_millis(1000), - None, + Box::new(BHMExitSignal), ); // Start an activity. @@ -119,6 +126,11 @@ fn test_hang_monitoring() { // Still no new alerts because the hang monitor has shut-down already. assert!(background_hang_monitor_receiver.try_recv().is_err()); + + // Join on the worker thread(channels are dropped above). + join_handle + .join() + .expect("Failed to join on the BHM worker thread"); } #[test] @@ -131,16 +143,23 @@ fn test_hang_monitoring_unregister() { ipc::channel().expect("ipc channel failure"); let (_sampler_sender, sampler_receiver) = ipc::channel().expect("ipc channel failure"); - let background_hang_monitor_register = HangMonitorRegister::init( + let (background_hang_monitor_register, join_handle) = HangMonitorRegister::init( background_hang_monitor_ipc_sender.clone(), sampler_receiver, true, ); + + struct BHMExitSignal; + + impl BackgroundHangMonitorExitSignal for BHMExitSignal { + fn signal_to_exit(&self) {} + } + let background_hang_monitor = background_hang_monitor_register.register_component( MonitoredComponentId(TEST_PIPELINE_ID, MonitoredComponentType::Script), Duration::from_millis(10), Duration::from_millis(1000), - None, + Box::new(BHMExitSignal), ); // Start an activity. @@ -155,6 +174,13 @@ fn test_hang_monitoring_unregister() { // No new alert yet assert!(background_hang_monitor_receiver.try_recv().is_err()); + + // Drop the channels and join on the worker thread. + drop(background_hang_monitor); + drop(background_hang_monitor_register); + join_handle + .join() + .expect("Failed to join on the BHM worker thread"); } // Perform two certain steps in `test_hang_monitoring_exit_signal_inner` in @@ -218,15 +244,13 @@ fn test_hang_monitoring_exit_signal_inner(op_order: fn(&mut dyn FnMut(), &mut dy })); // Init a worker, without active monitoring. - let background_hang_monitor_register = HangMonitorRegister::init( + let (background_hang_monitor_register, join_handle) = HangMonitorRegister::init( background_hang_monitor_ipc_sender.clone(), control_receiver, false, ); let mut background_hang_monitor = None; - let (exit_sender, exit_receiver) = ipc::channel().expect("Failed to create IPC channel!"); - let mut exit_sender = Some(exit_sender); // `op_order` determines the order in which these two closures are // executed. @@ -237,24 +261,26 @@ fn test_hang_monitoring_exit_signal_inner(op_order: fn(&mut dyn FnMut(), &mut dy MonitoredComponentId(TEST_PIPELINE_ID, MonitoredComponentType::Script), Duration::from_millis(10), Duration::from_millis(1000), - Some(signal.take().unwrap()), + signal.take().unwrap(), )); }, &mut || { // Send the exit message. control_sender - .send(BackgroundHangMonitorControlMsg::Exit( - exit_sender.take().unwrap(), - )) + .send(BackgroundHangMonitorControlMsg::Exit) .unwrap(); }, ); - // Assert we receive a confirmation back. - assert!(exit_receiver.recv().is_ok()); - // Assert we get the exit signal. while !closing.load(Ordering::SeqCst) { thread::sleep(Duration::from_millis(10)); } + + // Drop the channels and join on the worker thread. + drop(background_hang_monitor); + drop(background_hang_monitor_register); + join_handle + .join() + .expect("Failed to join on the BHM worker thread"); } diff --git a/components/constellation/constellation.rs b/components/constellation/constellation.rs index 6ae2eb0fe25..e27c00f1377 100644 --- a/components/constellation/constellation.rs +++ b/components/constellation/constellation.rs @@ -92,6 +92,7 @@ use std::marker::PhantomData; use std::mem::replace; use std::rc::{Rc, Weak}; use std::sync::{Arc, Mutex}; +use std::thread::JoinHandle; use std::{process, thread}; use background_hang_monitor::HangMonitorRegister; @@ -281,6 +282,9 @@ pub struct Constellation { /// None when in multiprocess mode. background_monitor_register: Option>, + /// In single process mode, a join handle on the BHM worker thread. + background_monitor_register_join_handle: Option>, + /// Channels to control all background-hang monitors. /// TODO: store them on the relevant BrowsingContextGroup, /// so that they could be controlled on a "per-tab/event-loop" basis. @@ -598,23 +602,28 @@ where // If we are in multiprocess mode, // a dedicated per-process hang monitor will be initialized later inside the content process. // See run_content_process in servo/lib.rs - let (background_monitor_register, background_hang_monitor_control_ipc_senders) = - if opts::get().multiprocess { - (None, vec![]) - } else { - let ( - background_hang_monitor_control_ipc_sender, - background_hang_monitor_control_ipc_receiver, - ) = ipc::channel().expect("ipc channel failure"); - ( - Some(HangMonitorRegister::init( - background_hang_monitor_ipc_sender.clone(), - background_hang_monitor_control_ipc_receiver, - opts::get().background_hang_monitor, - )), - vec![background_hang_monitor_control_ipc_sender], - ) - }; + let ( + background_monitor_register, + background_monitor_register_join_handle, + background_hang_monitor_control_ipc_senders, + ) = if opts::get().multiprocess { + (None, None, vec![]) + } else { + let ( + background_hang_monitor_control_ipc_sender, + background_hang_monitor_control_ipc_receiver, + ) = ipc::channel().expect("ipc channel failure"); + let (register, join_handle) = HangMonitorRegister::init( + background_hang_monitor_ipc_sender.clone(), + background_hang_monitor_control_ipc_receiver, + opts::get().background_hang_monitor, + ); + ( + Some(register), + Some(join_handle), + vec![background_hang_monitor_control_ipc_sender], + ) + }; let swmanager_receiver = route_ipc_receiver_to_new_crossbeam_receiver_preserving_errors( @@ -639,6 +648,7 @@ where background_hang_monitor_sender: background_hang_monitor_ipc_sender, background_hang_monitor_receiver, background_monitor_register, + background_monitor_register_join_handle, background_monitor_control_senders: background_hang_monitor_control_ipc_senders, script_receiver, compositor_receiver, @@ -2440,15 +2450,13 @@ where // even when currently hanging(on JS or sync XHR). // This must be done before starting the process of closing all pipelines. for chan in &self.background_monitor_control_senders { - let (exit_ipc_sender, exit_ipc_receiver) = - ipc::channel().expect("Failed to create IPC channel!"); - if let Err(e) = chan.send(BackgroundHangMonitorControlMsg::Exit(exit_ipc_sender)) { + // Note: the bhm worker thread will continue to run + // until all monitored components have exited, + // at which point we can join on the thread(done in `handle_shutdown`). + if let Err(e) = chan.send(BackgroundHangMonitorControlMsg::Exit) { warn!("error communicating with bhm: {}", e); continue; } - if exit_ipc_receiver.recv().is_err() { - warn!("Failed to receive exit confirmation from BHM."); - } } // Close the top-level browsing contexts @@ -2508,6 +2516,14 @@ where fn handle_shutdown(&mut self) { debug!("Handling shutdown."); + // In single process mode, join on the background hang monitor worker thread. + drop(self.background_monitor_register.take()); + if let Some(join_handle) = self.background_monitor_register_join_handle.take() { + join_handle + .join() + .expect("Failed to join on the BHM background thread."); + } + // At this point, there are no active pipelines, // so we can safely block on other threads, without worrying about deadlock. // Channels to receive signals when threads are done exiting. diff --git a/components/constellation/pipeline.rs b/components/constellation/pipeline.rs index b49d8d1c927..6f4923000c2 100644 --- a/components/constellation/pipeline.rs +++ b/components/constellation/pipeline.rs @@ -5,6 +5,7 @@ use std::collections::HashSet; use std::rc::Rc; use std::sync::Arc; +use std::thread::JoinHandle; use background_hang_monitor::HangMonitorRegister; use background_hang_monitor_api::{ @@ -558,7 +559,7 @@ impl UnprivilegedPipelineContent { pub fn register_with_background_hang_monitor( &mut self, - ) -> Box { + ) -> (Box, JoinHandle<()>) { HangMonitorRegister::init( self.background_hang_monitor_to_constellation_chan.clone(), self.bhm_control_port.take().expect("no sampling profiler?"), diff --git a/components/script/script_thread.rs b/components/script/script_thread.rs index fb393bc00d6..1d4b8bd304c 100644 --- a/components/script/script_thread.rs +++ b/components/script/script_thread.rs @@ -892,7 +892,7 @@ impl ScriptThread { MonitoredComponentId(state.id, MonitoredComponentType::Script), Duration::from_millis(1000), Duration::from_millis(5000), - Some(Box::new(background_hang_monitor_exit_signal)), + Box::new(background_hang_monitor_exit_signal), ); let (image_cache_sender, image_cache_receiver) = unbounded(); diff --git a/components/servo/lib.rs b/components/servo/lib.rs index a37ec18e10b..e9cf346fe82 100644 --- a/components/servo/lib.rs +++ b/components/servo/lib.rs @@ -1258,7 +1258,8 @@ pub fn run_content_process(token: String) { set_logger(content.script_to_constellation_chan().clone()); - let background_hang_monitor_register = content.register_with_background_hang_monitor(); + let (background_hang_monitor_register, join_handle) = + content.register_with_background_hang_monitor(); let layout_factory = Arc::new(LayoutFactoryImpl()); content.register_system_memory_reporter(); @@ -1268,6 +1269,14 @@ pub fn run_content_process(token: String) { layout_factory, background_hang_monitor_register, ); + + // Since wait_for_completion is true, + // here we know that the script-thread + // will exit(or already has), + // and so we can join on the BHM worker thread. + join_handle + .join() + .expect("Failed to join on the BHM background thread."); }, UnprivilegedContent::ServiceWorker(content) => { content.start::(); diff --git a/components/shared/background_hang_monitor/lib.rs b/components/shared/background_hang_monitor/lib.rs index 1eb70d0e92e..f2f2c84c6cc 100644 --- a/components/shared/background_hang_monitor/lib.rs +++ b/components/shared/background_hang_monitor/lib.rs @@ -10,7 +10,6 @@ use std::time::Duration; use std::{fmt, mem}; use base::id::PipelineId; -use ipc_channel::ipc::IpcSender; use serde::{Deserialize, Serialize}; #[derive(Clone, Copy, Debug, Deserialize, Serialize)] @@ -170,7 +169,7 @@ pub trait BackgroundHangMonitorRegister: BackgroundHangMonitorClone + Send { component: MonitoredComponentId, transient_hang_timeout: Duration, permanent_hang_timeout: Duration, - exit_signal: Option>, + exit_signal: Box, ) -> Box; } @@ -208,6 +207,6 @@ pub trait BackgroundHangMonitorExitSignal: Send { pub enum BackgroundHangMonitorControlMsg { /// Toggle the sampler, with a given sampling rate and max total sampling duration. ToggleSampler(Duration, Duration), - /// Exit, and propagate the signal to monitored components. - Exit(IpcSender<()>), + /// Propagate exit signal to monitored components, and shutdown when they have. + Exit, }