From 4515e9b6f960d80d2de10cda36940d31cdf28842 Mon Sep 17 00:00:00 2001 From: Kunshan Wang Date: Tue, 18 Apr 2023 18:31:52 +0800 Subject: [PATCH] No coordinator work Remove the concept of "coordinator work packets". Now ScheduleCollection and StopMutators are both executed on ordinary workers. The only work packet executed by the coordinator is EndOfGC. Simplified the interaction between the coordinator and the workers. The coordinator only responds to the event that "all workers have parked". Removed the workers-to-coordinator channel. WorkerMonitor now has two Condvars, one (the existing one) for notifying workers about more work available, and another for notifying the coordinator that all workers have parked. --- src/scheduler/controller.rs | 93 +++++--------------- src/scheduler/controller/channel.rs | 114 ------------------------ src/scheduler/gc_work.rs | 27 ++---- src/scheduler/mod.rs | 1 - src/scheduler/scheduler.rs | 62 ++++--------- src/scheduler/work.rs | 7 -- src/scheduler/worker.rs | 130 +++++++++++++++++++++------- src/vm/collection.rs | 9 +- 8 files changed, 145 insertions(+), 298 deletions(-) delete mode 100644 src/scheduler/controller/channel.rs diff --git a/src/scheduler/controller.rs b/src/scheduler/controller.rs index e970899241..8e124d0836 100644 --- a/src/scheduler/controller.rs +++ b/src/scheduler/controller.rs @@ -3,19 +3,16 @@ //! MMTk has many GC threads. There are many GC worker threads and one GC controller thread. //! The GC controller thread responds to GC requests and coordinates the workers to perform GC. -use std::sync::{Arc, Condvar, Mutex}; +use std::sync::Arc; use crate::plan::gc_requester::GCRequester; use crate::scheduler::gc_work::{EndOfGC, ScheduleCollection}; +use crate::scheduler::{GCWork, WorkBucketStage}; use crate::util::VMWorkerThread; use crate::vm::VMBinding; use crate::MMTK; -use self::channel::{Event, Receiver}; - -use super::{CoordinatorWork, GCWorkScheduler, GCWorker}; - -pub(crate) mod channel; +use super::{GCWorkScheduler, GCWorker}; /// The thread local struct for the GC controller, the counterpart of `GCWorker`. pub struct GCController { @@ -25,8 +22,6 @@ pub struct GCController { requester: Arc>, /// The reference to the scheduler. scheduler: Arc>, - /// Receive coordinator work packets and notifications from GC workers through this. - receiver: Receiver, /// The `GCWorker` is used to execute packets. The controller is also a `GCWorker`. coordinator_worker: GCWorker, } @@ -36,14 +31,12 @@ impl GCController { mmtk: &'static MMTK, requester: Arc>, scheduler: Arc>, - receiver: Receiver, coordinator_worker: GCWorker, ) -> Box> { Box::new(Self { mmtk, requester, scheduler, - receiver, coordinator_worker, }) } @@ -79,76 +72,49 @@ impl GCController { return true; } - // If all fo the above failed, it means GC has finished. + // If all of the above failed, it means GC has finished. false } - /// Reset the "all workers parked" state and resume workers. - fn reset_and_resume_workers(&mut self) { - self.receiver.reset_all_workers_parked(); - self.scheduler.worker_monitor.notify_work_available(true); - debug!("Workers resumed"); - } - /// Handle the "all workers have parked" event. Return true if GC is finished. fn on_all_workers_parked(&mut self) -> bool { - assert!(self.scheduler.all_activated_buckets_are_empty()); + self.scheduler.assert_all_activated_buckets_are_empty(); let new_work_available = self.find_more_work_for_workers(); - if new_work_available { - self.reset_and_resume_workers(); - // If there is more work to do, GC has not finished. - return false; - } - - assert!(self.scheduler.all_buckets_empty()); - - true - } - - /// Process an event. Return true if the GC is finished. - fn process_event(&mut self, message: Event) -> bool { - match message { - Event::Work(mut work) => { - self.execute_coordinator_work(work.as_mut(), true); - false - } - Event::AllParked => self.on_all_workers_parked(), - } + // GC finishes if there is no new work to do. + !new_work_available } /// Coordinate workers to perform GC in response to a GC request. pub fn do_gc_until_completion(&mut self) { let gc_start = std::time::Instant::now(); - // Schedule collection. - self.execute_coordinator_work(&mut ScheduleCollection, true); - - // Tell GC trigger that GC started - this happens after ScheduleCollection so we - // will know what kind of GC this is (e.g. nursery vs mature in gen copy, defrag vs fast in Immix) - self.mmtk - .plan - .base() - .gc_trigger - .policy - .on_gc_start(self.mmtk); - - // React to worker-generated events until finished. + + debug_assert!( + self.scheduler.worker_monitor.debug_is_sleeping(), + "Workers are still doing work when GC started." + ); + + // Add a ScheduleCollection work packet. It is the seed of other work packets. + self.scheduler.work_buckets[WorkBucketStage::Unconstrained].add(ScheduleCollection); + + // Resume the workers and gradually open more buckets when they stop together. loop { - let event = self.receiver.poll_event(); - let finished = self.process_event(event); + self.scheduler.worker_monitor.resume_and_wait(); + let finished = self.on_all_workers_parked(); if finished { break; } } // All GC workers must have parked by now. - debug_assert!(self.scheduler.worker_monitor.debug_is_group_sleeping()); + debug_assert!(self.scheduler.worker_monitor.debug_is_sleeping()); debug_assert!(!self.scheduler.worker_group.has_designated_work()); + debug_assert!(self.scheduler.all_buckets_empty()); // Deactivate all work buckets to prepare for the next GC. // NOTE: There is no need to hold any lock. - // All GC workers are doing "group sleeping" now, + // Workers are in the `Sleeping` state. // so they will not wake up while we deactivate buckets. self.scheduler.deactivate_all(); @@ -163,21 +129,8 @@ impl GCController { let mut end_of_gc = EndOfGC { elapsed: gc_start.elapsed(), }; - - self.execute_coordinator_work(&mut end_of_gc, false); + end_of_gc.do_work_with_stat(&mut self.coordinator_worker, self.mmtk); self.scheduler.debug_assert_all_buckets_deactivated(); } - - fn execute_coordinator_work( - &mut self, - work: &mut dyn CoordinatorWork, - notify_workers: bool, - ) { - work.do_work_with_stat(&mut self.coordinator_worker, self.mmtk); - - if notify_workers { - self.reset_and_resume_workers(); - }; - } } diff --git a/src/scheduler/controller/channel.rs b/src/scheduler/controller/channel.rs deleted file mode 100644 index 548f2c6e38..0000000000 --- a/src/scheduler/controller/channel.rs +++ /dev/null @@ -1,114 +0,0 @@ -use std::collections::VecDeque; - -use super::*; - -/// A one-way channel for workers to send coordinator packets and notifications to the controller. -struct Channel { - sync: Mutex>, - cond: Condvar, -} - -/// The synchronized parts of `Channel`. -struct ChannelSync { - /// Pending coordinator work packets. - coordinator_packets: VecDeque>>, - /// Whether all workers have parked. - /// - /// NOTE: This field is set to `true` by the last parked worker. - /// It is used to notify the coordinator about the event that all workers have parked. - /// To resume workers from "group sleeping", use `WorkerMonitor::notify_work_available`. - all_workers_parked: bool, -} - -/// Each worker holds an instance of this. -/// -/// It wraps a channel, and only allows workers to access it in expected ways. -pub struct Sender { - chan: Arc>, -} - -impl Clone for Sender { - fn clone(&self) -> Self { - Self { - chan: self.chan.clone(), - } - } -} - -impl Sender { - /// Send a coordinator work packet to the coordinator. - pub fn add_coordinator_work(&self, work: Box>) { - let mut sync = self.chan.sync.lock().unwrap(); - sync.coordinator_packets.push_back(work); - debug!("A worker has sent a coordinator work packet."); - self.chan.cond.notify_one(); - } - - /// Notify the coordinator that all workers have parked. - pub fn notify_all_workers_parked(&self) { - let mut sync = self.chan.sync.lock().unwrap(); - sync.all_workers_parked = true; - debug!("Notified the coordinator that all workers have parked."); - self.chan.cond.notify_one(); - } -} - -/// The coordinator holds an instance of this. -/// -/// It wraps a channel, and only allows the coordinator to access it in expected ways. -pub struct Receiver { - chan: Arc>, -} - -impl Receiver { - /// Get an event. - pub(super) fn poll_event(&self) -> Event { - let mut sync = self.chan.sync.lock().unwrap(); - loop { - // Make sure the coordinator always sees packets before seeing "all parked". - if let Some(work) = sync.coordinator_packets.pop_front() { - debug!("Received a coordinator packet."); - return Event::Work(work); - } - - if sync.all_workers_parked { - debug!("Observed all workers parked."); - return Event::AllParked; - } - - sync = self.chan.cond.wait(sync).unwrap(); - } - } - - /// Reset the "all workers have parked" flag. - pub fn reset_all_workers_parked(&self) { - let mut sync = self.chan.sync.lock().unwrap(); - sync.all_workers_parked = false; - debug!("The all_workers_parked state is reset."); - } -} - -/// This type represents the events the `Receiver` observes. -pub(crate) enum Event { - /// Send a work-packet to the coordinator thread. - Work(Box>), - /// Notify the coordinator thread that all GC tasks are finished. - /// When sending this message, all the work buckets should be - /// empty, and all the workers should be parked. - AllParked, -} - -/// Create a Sender-Receiver pair. -pub(crate) fn make_channel() -> (Sender, Receiver) { - let chan = Arc::new(Channel { - sync: Mutex::new(ChannelSync { - coordinator_packets: Default::default(), - all_workers_parked: false, - }), - cond: Default::default(), - }); - - let sender = Sender { chan: chan.clone() }; - let receiver = Receiver { chan }; - (sender, receiver) -} diff --git a/src/scheduler/gc_work.rs b/src/scheduler/gc_work.rs index dee41d00fb..2fb579c5f4 100644 --- a/src/scheduler/gc_work.rs +++ b/src/scheduler/gc_work.rs @@ -15,11 +15,16 @@ pub struct ScheduleCollection; impl GCWork for ScheduleCollection { fn do_work(&mut self, worker: &mut GCWorker, mmtk: &'static MMTK) { mmtk.plan.schedule_collection(worker.scheduler()); + + // Tell GC trigger that GC started. + // We now know what kind of GC this is (e.g. nursery vs mature in gen copy, defrag vs fast in Immix) + // TODO: Depending on the OS scheduling, other workers can run so fast that they can finish + // everything in the `Unconstrained` and the `Prepare` buckets before we execute the next + // statement. Consider if there is a better place to call `on_gc_start`. + mmtk.plan.base().gc_trigger.policy.on_gc_start(mmtk); } } -impl CoordinatorWork for ScheduleCollection {} - /// The global GC Preparation Work /// This work packet invokes prepare() for the plan (which will invoke prepare() for each space), and /// pushes work packets for preparing mutators and collectors. @@ -174,15 +179,6 @@ impl StopMutators { impl GCWork for StopMutators { fn do_work(&mut self, worker: &mut GCWorker, mmtk: &'static MMTK) { - // If the VM requires that only the coordinator thread can stop the world, - // we delegate the work to the coordinator. - if ::VMCollection::COORDINATOR_ONLY_STW && !worker.is_coordinator() { - worker - .sender - .add_coordinator_work(Box::new(StopMutators::::new())); - return; - } - trace!("stop_all_mutators start"); mmtk.plan.base().prepare_for_stack_scanning(); ::VMCollection::stop_all_mutators(worker.tls, |mutator| { @@ -217,8 +213,6 @@ impl GCWork for StopMutators { } } -impl CoordinatorWork for StopMutators {} - #[derive(Default)] pub struct EndOfGC { pub elapsed: std::time::Duration, @@ -244,11 +238,6 @@ impl GCWork for EndOfGC { mmtk.edge_logger.reset(); } - if ::VMCollection::COORDINATOR_ONLY_STW { - assert!(worker.is_coordinator(), - "VM only allows coordinator to resume mutators, but the current worker is not the coordinator."); - } - mmtk.plan.base().set_gc_status(GcStatus::NotInGC); // Reset the triggering information. @@ -258,8 +247,6 @@ impl GCWork for EndOfGC { } } -impl CoordinatorWork for EndOfGC {} - /// This implements `ObjectTracer` by forwarding the `trace_object` calls to the wrapped /// `ProcessEdgesWork` instance. struct ProcessEdgesWorkTracer { diff --git a/src/scheduler/mod.rs b/src/scheduler/mod.rs index cad3c9137c..3851490766 100644 --- a/src/scheduler/mod.rs +++ b/src/scheduler/mod.rs @@ -10,7 +10,6 @@ mod stat; pub(self) mod work_counter; mod work; -pub use work::CoordinatorWork; pub use work::GCWork; pub(crate) use work::GCWorkContext; diff --git a/src/scheduler/scheduler.rs b/src/scheduler/scheduler.rs index 8dbf8ba390..40147acffe 100644 --- a/src/scheduler/scheduler.rs +++ b/src/scheduler/scheduler.rs @@ -94,16 +94,12 @@ impl GCWorkScheduler { /// Create GC threads, including the controller thread and all workers. pub fn spawn_gc_threads(self: &Arc, mmtk: &'static MMTK, tls: VMThread) { - // Create the communication channel. - let (sender, receiver) = controller::channel::make_channel(); - // Spawn the controller thread. let coordinator_worker = GCWorker::new( mmtk, usize::MAX, self.clone(), true, - sender.clone(), self.coordinator_worker_shared.clone(), deque::Worker::new_fifo(), ); @@ -111,12 +107,11 @@ impl GCWorkScheduler { mmtk, mmtk.plan.base().gc_requester.clone(), self.clone(), - receiver, coordinator_worker, ); VM::VMCollection::spawn_gc_thread(tls, GCThreadContext::::Controller(gc_controller)); - self.worker_group.spawn(mmtk, sender, tls) + self.worker_group.spawn(mmtk, tls) } /// Resolve the affinity of a thread. @@ -307,13 +302,21 @@ impl GCWorkScheduler { } /// Check if all the work buckets are empty - pub(crate) fn all_activated_buckets_are_empty(&self) -> bool { - for bucket in self.work_buckets.values() { - if bucket.is_activated() && !bucket.is_drained() { - return false; + pub(crate) fn assert_all_activated_buckets_are_empty(&self) { + let mut error_example = None; + for (id, bucket) in self.work_buckets.iter() { + if bucket.is_activated() && !bucket.is_empty() { + error!("Work bucket {:?} is active but not empty!", id); + // This error can be hard to reproduce. + // If an error happens in the release build where logs are turned off, + // we should show at least one abnormal bucket in the panic message + // so that we still have some information for debugging. + error_example = Some(id); } } - true + if let Some(id) = error_example { + panic!("Some active buckets (such as {:?}) are not empty.", id); + } } /// Get a schedulable work packet without retry. @@ -376,48 +379,13 @@ impl GCWorkScheduler { } fn poll_slow(&self, worker: &GCWorker) -> Box> { - // Note: The lock is released during `wait` in the loop. - let mut sync = self.worker_monitor.sync.lock().unwrap(); loop { // Retry polling if let Some(work) = self.poll_schedulable_work(worker) { return work; } - // Park this worker - let all_parked = sync.inc_parked_workers(); - - if all_parked { - // If all workers are parked, enter "group sleeping" and notify controller. - sync.group_sleep = true; - debug!("Entered group-sleeping state"); - worker.sender.notify_all_workers_parked(); - } else { - // Otherwise wait until notified. - // Note: The condition for this `cond.wait` is "more work is available". - // If this worker spuriously wakes up, then in the next loop iteration, the - // `poll_schedulable_work` invocation above will fail, and the worker will reach - // here and wait again. - sync = self.worker_monitor.cond.wait(sync).unwrap(); - } - - // Keep waiting if we have entered "group sleeping" state. - // The coordinator will let the worker leave the "group sleeping" state - // once the coordinator finished its work. - // - // Note: `wait_while` checks `sync.group_sleep` before actually starting to wait. - // This is expected because the controller may run so fast that it opened new buckets - // and unset `sync.group_sleep` before we even reached here. If that happens, waiting - // blindly will result in all workers sleeping forever. So we should always check - // `sync.group_sleep` before waiting. - sync = self - .worker_monitor - .cond - .wait_while(sync, |sync| sync.group_sleep) - .unwrap(); - - // Unpark this worker. - sync.dec_parked_workers(); + self.worker_monitor.park_and_wait(worker); } } diff --git a/src/scheduler/work.rs b/src/scheduler/work.rs index 98cb463895..2621fcedc0 100644 --- a/src/scheduler/work.rs +++ b/src/scheduler/work.rs @@ -4,13 +4,6 @@ use crate::vm::VMBinding; #[cfg(feature = "work_packet_stats")] use std::any::{type_name, TypeId}; -/// A special kind of work that will execute on the coordinator (i.e. controller) thread -/// -/// The coordinator thread holds the global monitor lock when executing `CoordinatorWork`s. -/// So, directly adding new work to any buckets will cause dead lock. -/// For this case, use `WorkBucket::add_with_priority_unsync` instead. -pub trait CoordinatorWork: 'static + Send + GCWork {} - pub trait GCWork: 'static + Send { /// Define the work for this packet. However, this is not supposed to be called directly. /// Usually `do_work_with_stat()` should be used. diff --git a/src/scheduler/worker.rs b/src/scheduler/worker.rs index 1b2c91fa71..438ee9e7d2 100644 --- a/src/scheduler/worker.rs +++ b/src/scheduler/worker.rs @@ -55,8 +55,35 @@ impl GCWorkerShared { /// NOTE: All fields are public in order to support the complex control structure /// in `GCWorkScheduler::poll_slow`. pub(crate) struct WorkerMonitor { - pub sync: Mutex, - pub cond: Condvar, + /// The synchronized part. + sync: Mutex, + /// This is notified when new work is made available for the workers. + /// Particularly, it is notified when + /// - `sync.worker_group_state` is transitioned to `Working` because + /// - some workers still have designated work, or + /// - some sentinel work packets are added to their drained buckets, or + /// - some work buckets are opened, or + /// - any work packet is added to any open bucket. + /// Workers wait on this condvar. + work_available: Condvar, + /// This is notified when all workers parked. + /// The coordinator waits on this condvar. + all_workers_parked: Condvar, +} + +/// The state of the worker group. The worker group alternates between the `Sleeping` and the +/// `Working` state. Workers execute work packets in the `Working` state, but once workers entered +/// the `Sleeping` state, they can not continue until the coordinator explicitly transitions the +/// state back to `Working` after it found more work for workers to do. +#[derive(Clone, Copy, PartialEq, Eq, Debug)] +enum WorkerGroupState { + /// In this state, the coordinator can open new buckets and close buckets, + /// but workers cannot execute any packets or get any work packets from any buckets. + /// Workers cannot unpark in this state. + Sleeping, + /// In this state, workers can get work packets from open buckets, + /// but no buckets can be opened or closed. + Working, } /// The synchronized part of `WorkerMonitor`. @@ -65,13 +92,8 @@ pub(crate) struct WorkerMonitorSync { worker_count: usize, /// Number of parked workers. parked_workers: usize, - /// This flag is set to true when all workers have parked, - /// and cleared if a work packet is added to an open bucket, or a new bucket is opened. - /// No workers can unpark while this flag is set. - /// - /// Note that testing this flag is *not* equivalent to testing `parked_workers == num_workers`. - /// This field is used to *receive notification* for the event of more work becoming available. - pub group_sleep: bool, + /// The worker group state. + worker_group_state: WorkerGroupState, } impl WorkerMonitor { @@ -80,28 +102,82 @@ impl WorkerMonitor { sync: Mutex::new(WorkerMonitorSync { worker_count, parked_workers: 0, - group_sleep: false, + worker_group_state: WorkerGroupState::Sleeping, }), - cond: Default::default(), + work_available: Default::default(), + all_workers_parked: Default::default(), } } /// Wake up workers when more work packets are made available for workers. - /// This function will get workers out of the "group sleeping" state. + /// This function is called when adding work packets to buckets. + /// This function doesn't change the `work_group_state` variable. + /// If workers are in the `Sleeping` , use `resume_and_wait` to resume workers. pub fn notify_work_available(&self, all: bool) { - let mut sync = self.sync.lock().unwrap(); - sync.group_sleep = false; if all { - self.cond.notify_all(); + self.work_available.notify_all(); } else { - self.cond.notify_one(); + self.work_available.notify_one(); } } - /// Test if workers are in group sleeping state. Used for debugging. - pub fn debug_is_group_sleeping(&self) -> bool { + /// Wake up workers and wait until they transition to `Sleeping` state again. + /// This is called by the coordinator. + pub fn resume_and_wait(&self) { + let mut sync = self.sync.lock().unwrap(); + sync.worker_group_state = WorkerGroupState::Working; + self.work_available.notify_all(); + let _sync = self + .all_workers_parked + .wait_while(sync, |sync| { + sync.worker_group_state == WorkerGroupState::Working + }) + .unwrap(); + } + + /// Test if the worker group is in the `Sleeping` state. + pub fn debug_is_sleeping(&self) -> bool { let sync = self.sync.lock().unwrap(); - sync.group_sleep + sync.worker_group_state == WorkerGroupState::Sleeping + } + + /// Park until more work is available. + /// The argument `worker` indicates this function can only be called by workers. + pub fn park_and_wait(&self, worker: &GCWorker) { + let mut sync = self.sync.lock().unwrap(); + + // Park this worker + let all_parked = sync.inc_parked_workers(); + trace!("Worker {} parked.", worker.ordinal); + + if all_parked { + // If all workers are parked, enter "Sleeping" state and notify controller. + sync.worker_group_state = WorkerGroupState::Sleeping; + debug!( + "Worker {} notifies the coordinator that all workerer parked.", + worker.ordinal + ); + self.all_workers_parked.notify_one(); + } else { + // Otherwise wait until notified. + // Note: The condition for this `cond.wait` is "more work is available". + // If this worker spuriously wakes up, then in the next loop iteration, the + // `poll_schedulable_work` invocation above will fail, and the worker will reach + // here and wait again. + sync = self.work_available.wait(sync).unwrap(); + } + + // If we are in the `Sleeping` state, wait until leaving that state. + sync = self + .work_available + .wait_while(sync, |sync| { + sync.worker_group_state == WorkerGroupState::Sleeping + }) + .unwrap(); + + // Unpark this worker. + sync.dec_parked_workers(); + trace!("Worker {} unparked.", worker.ordinal); } } @@ -110,7 +186,7 @@ impl WorkerMonitorSync { /// Called before a worker is parked. /// /// Return true if all the workers are parked. - pub fn inc_parked_workers(&mut self) -> bool { + fn inc_parked_workers(&mut self) -> bool { let old = self.parked_workers; debug_assert!(old < self.worker_count); let new = old + 1; @@ -120,7 +196,7 @@ impl WorkerMonitorSync { /// Decrease the packed-workers counter. /// Called after a worker is resumed from the parked state. - pub fn dec_parked_workers(&mut self) { + fn dec_parked_workers(&mut self) { let old = self.parked_workers; debug_assert!(old <= self.worker_count); debug_assert!(old > 0); @@ -141,8 +217,6 @@ pub struct GCWorker { scheduler: Arc>, /// The copy context, used to implement copying GC. copy: GCWorkerCopyContext, - /// The sending end of the channel to send message to the controller thread. - pub(crate) sender: controller::channel::Sender, /// The reference to the MMTk instance. pub mmtk: &'static MMTK, /// True if this struct is the embedded GCWorker of the controller thread. @@ -177,7 +251,6 @@ impl GCWorker { ordinal: ThreadId, scheduler: Arc>, is_coordinator: bool, - sender: controller::channel::Sender, shared: Arc>, local_work_buffer: deque::Worker>>, ) -> Self { @@ -186,7 +259,6 @@ impl GCWorker { ordinal, // We will set this later copy: GCWorkerCopyContext::new_non_copy(), - sender, scheduler, mmtk, is_coordinator, @@ -300,12 +372,7 @@ impl WorkerGroup { } /// Spawn all the worker threads - pub fn spawn( - &self, - mmtk: &'static MMTK, - sender: controller::channel::Sender, - tls: VMThread, - ) { + pub fn spawn(&self, mmtk: &'static MMTK, tls: VMThread) { let mut unspawned_local_work_queues = self.unspawned_local_work_queues.lock().unwrap(); // Spawn each worker thread. for (ordinal, shared) in self.workers_shared.iter().enumerate() { @@ -314,7 +381,6 @@ impl WorkerGroup { ordinal, mmtk.scheduler.clone(), false, - sender.clone(), shared.clone(), unspawned_local_work_queues.pop().unwrap(), )); diff --git a/src/vm/collection.rs b/src/vm/collection.rs index 8ff08bcbd7..8e117d9e6f 100644 --- a/src/vm/collection.rs +++ b/src/vm/collection.rs @@ -12,13 +12,8 @@ pub enum GCThreadContext { /// VM-specific methods for garbage collection. pub trait Collection { - /// If true, only the coordinator thread can call stop_all_mutators and the resume_mutators methods. - /// If false, any GC thread can call these methods. - /// - /// This constant exists because some VMs require the thread that resumes a thread to be the same thread that - /// stopped it. The MMTk Core will use the appropriate thread to stop or start the world according to the value of - /// this constant. If a VM does not have such a requirement, the VM binding shall set this to false to reduce an - /// unnecessary context switch. + /// This const existed for historical reasons. It has no effect now. + #[deprecated] const COORDINATOR_ONLY_STW: bool = true; /// Stop all the mutator threads. MMTk calls this method when it requires all the mutator to yield for a GC.