mmtk/scheduler/
gc_work.rs

1use super::work_bucket::WorkBucketStage;
2use super::*;
3use crate::global_state::GcStatus;
4use crate::plan::ObjectsClosure;
5use crate::plan::VectorObjectQueue;
6use crate::util::*;
7use crate::vm::slot::Slot;
8use crate::vm::*;
9use crate::*;
10use std::marker::PhantomData;
11use std::ops::{Deref, DerefMut};
12
13pub struct ScheduleCollection;
14
15impl<VM: VMBinding> GCWork<VM> for ScheduleCollection {
16    fn do_work(&mut self, worker: &mut GCWorker<VM>, mmtk: &'static MMTK<VM>) {
17        // Tell GC trigger that GC started.
18        mmtk.gc_trigger.policy.on_gc_start(mmtk);
19
20        // Determine collection kind
21        let is_emergency = mmtk.state.set_collection_kind(
22            mmtk.get_plan().last_collection_was_exhaustive(),
23            mmtk.gc_trigger.policy.can_heap_size_grow(),
24        );
25        if is_emergency {
26            mmtk.get_plan().notify_emergency_collection();
27        }
28        // Set to GcPrepare
29        mmtk.set_gc_status(GcStatus::GcPrepare);
30
31        // Let the plan to schedule collection work
32        mmtk.get_plan().schedule_collection(worker.scheduler());
33    }
34}
35
36/// The global GC Preparation Work
37/// This work packet invokes prepare() for the plan (which will invoke prepare() for each space), and
38/// pushes work packets for preparing mutators and collectors.
39/// We should only have one such work packet per GC, before any actual GC work starts.
40/// We assume this work packet is the only running work packet that accesses plan, and there should
41/// be no other concurrent work packet that accesses plan (read or write). Otherwise, there may
42/// be a race condition.
43pub struct Prepare<C: GCWorkContext> {
44    pub plan: *const C::PlanType,
45}
46
47unsafe impl<C: GCWorkContext> Send for Prepare<C> {}
48
49impl<C: GCWorkContext> Prepare<C> {
50    pub fn new(plan: *const C::PlanType) -> Self {
51        Self { plan }
52    }
53}
54
55impl<C: GCWorkContext> GCWork<C::VM> for Prepare<C> {
56    fn do_work(&mut self, worker: &mut GCWorker<C::VM>, mmtk: &'static MMTK<C::VM>) {
57        trace!("Prepare Global");
58        // We assume this is the only running work packet that accesses plan at the point of execution
59        let plan_mut: &mut C::PlanType = unsafe { &mut *(self.plan as *const _ as *mut _) };
60        plan_mut.prepare(worker.tls);
61
62        if plan_mut.constraints().needs_prepare_mutator {
63            let prepare_mutator_packets = <C::VM as VMBinding>::VMActivePlan::mutators()
64                .map(|mutator| Box::new(PrepareMutator::<C::VM>::new(mutator)) as _)
65                .collect::<Vec<_>>();
66            // Just in case the VM binding is inconsistent about the number of mutators and the actual mutator list.
67            debug_assert_eq!(
68                prepare_mutator_packets.len(),
69                <C::VM as VMBinding>::VMActivePlan::number_of_mutators()
70            );
71            mmtk.scheduler.work_buckets[WorkBucketStage::Prepare].bulk_add(prepare_mutator_packets);
72        }
73
74        for w in &mmtk.scheduler.worker_group.workers_shared {
75            let result = w.designated_work.push(Box::new(PrepareCollector));
76            debug_assert!(result.is_ok());
77        }
78    }
79}
80
81/// The mutator GC Preparation Work
82pub struct PrepareMutator<VM: VMBinding> {
83    // The mutator reference has static lifetime.
84    // It is safe because the actual lifetime of this work-packet will not exceed the lifetime of a GC.
85    pub mutator: &'static mut Mutator<VM>,
86}
87
88impl<VM: VMBinding> PrepareMutator<VM> {
89    pub fn new(mutator: &'static mut Mutator<VM>) -> Self {
90        Self { mutator }
91    }
92}
93
94impl<VM: VMBinding> GCWork<VM> for PrepareMutator<VM> {
95    fn do_work(&mut self, worker: &mut GCWorker<VM>, _mmtk: &'static MMTK<VM>) {
96        trace!("Prepare Mutator");
97        self.mutator.prepare(worker.tls);
98    }
99}
100
101/// The collector GC Preparation Work
102#[derive(Default)]
103pub struct PrepareCollector;
104
105impl<VM: VMBinding> GCWork<VM> for PrepareCollector {
106    fn do_work(&mut self, worker: &mut GCWorker<VM>, mmtk: &'static MMTK<VM>) {
107        trace!("Prepare Collector");
108        worker.get_copy_context_mut().prepare();
109        mmtk.get_plan().prepare_worker(worker);
110    }
111}
112
113/// The global GC release Work
114/// This work packet invokes release() for the plan (which will invoke release() for each space), and
115/// pushes work packets for releasing mutators and collectors.
116/// We should only have one such work packet per GC, after all actual GC work ends.
117/// We assume this work packet is the only running work packet that accesses plan, and there should
118/// be no other concurrent work packet that accesses plan (read or write). Otherwise, there may
119/// be a race condition.
120pub struct Release<C: GCWorkContext> {
121    pub plan: *const C::PlanType,
122}
123
124impl<C: GCWorkContext> Release<C> {
125    pub fn new(plan: *const C::PlanType) -> Self {
126        Self { plan }
127    }
128}
129
130unsafe impl<C: GCWorkContext> Send for Release<C> {}
131
132impl<C: GCWorkContext + 'static> GCWork<C::VM> for Release<C> {
133    fn do_work(&mut self, worker: &mut GCWorker<C::VM>, mmtk: &'static MMTK<C::VM>) {
134        trace!("Release Global");
135
136        mmtk.gc_trigger.policy.on_gc_release(mmtk);
137        // We assume this is the only running work packet that accesses plan at the point of execution
138
139        let plan_mut: &mut C::PlanType = unsafe { &mut *(self.plan as *const _ as *mut _) };
140        plan_mut.release(worker.tls);
141
142        let release_mutator_packets = <C::VM as VMBinding>::VMActivePlan::mutators()
143            .map(|mutator| Box::new(ReleaseMutator::<C::VM>::new(mutator)) as _)
144            .collect::<Vec<_>>();
145        // Just in case the VM binding is inconsistent about the number of mutators and the actual mutator list.
146        debug_assert_eq!(
147            release_mutator_packets.len(),
148            <C::VM as VMBinding>::VMActivePlan::number_of_mutators()
149        );
150        mmtk.scheduler.work_buckets[WorkBucketStage::Release].bulk_add(release_mutator_packets);
151
152        for w in &mmtk.scheduler.worker_group.workers_shared {
153            let result = w.designated_work.push(Box::new(ReleaseCollector));
154            debug_assert!(result.is_ok());
155        }
156    }
157}
158
159/// The mutator release Work
160pub struct ReleaseMutator<VM: VMBinding> {
161    // The mutator reference has static lifetime.
162    // It is safe because the actual lifetime of this work-packet will not exceed the lifetime of a GC.
163    pub mutator: &'static mut Mutator<VM>,
164}
165
166impl<VM: VMBinding> ReleaseMutator<VM> {
167    pub fn new(mutator: &'static mut Mutator<VM>) -> Self {
168        Self { mutator }
169    }
170}
171
172impl<VM: VMBinding> GCWork<VM> for ReleaseMutator<VM> {
173    fn do_work(&mut self, worker: &mut GCWorker<VM>, _mmtk: &'static MMTK<VM>) {
174        trace!("Release Mutator");
175        self.mutator.release(worker.tls);
176    }
177}
178
179/// The collector release Work
180#[derive(Default)]
181pub struct ReleaseCollector;
182
183impl<VM: VMBinding> GCWork<VM> for ReleaseCollector {
184    fn do_work(&mut self, worker: &mut GCWorker<VM>, _mmtk: &'static MMTK<VM>) {
185        trace!("Release Collector");
186        worker.get_copy_context_mut().release();
187    }
188}
189
190/// Stop all mutators
191///
192/// TODO: Smaller work granularity
193#[derive(Default)]
194pub struct StopMutators<C: GCWorkContext> {
195    /// If this is true, we skip creating [`ScanMutatorRoots`] work packets for mutators.
196    /// By default, this is false.
197    skip_mutator_roots: bool,
198    /// Flush mutators once they are stopped. By default this is false. [`ScanMutatorRoots`] will flush mutators.
199    flush_mutator: bool,
200    phantom: PhantomData<C>,
201}
202
203impl<C: GCWorkContext> StopMutators<C> {
204    pub fn new() -> Self {
205        Self {
206            skip_mutator_roots: false,
207            flush_mutator: false,
208            phantom: PhantomData,
209        }
210    }
211
212    /// Create a `StopMutators` work packet that does not create `ScanMutatorRoots` work packets for mutators, and will simply flush mutators.
213    pub fn new_no_scan_roots() -> Self {
214        Self {
215            skip_mutator_roots: true,
216            flush_mutator: true,
217            phantom: PhantomData,
218        }
219    }
220}
221
222impl<C: GCWorkContext> GCWork<C::VM> for StopMutators<C> {
223    fn do_work(&mut self, worker: &mut GCWorker<C::VM>, mmtk: &'static MMTK<C::VM>) {
224        trace!("stop_all_mutators start");
225        mmtk.state.prepare_for_stack_scanning();
226        <C::VM as VMBinding>::VMCollection::stop_all_mutators(worker.tls, |mutator| {
227            // TODO: The stack scanning work won't start immediately, as the `Prepare` bucket is not opened yet (the bucket is opened in notify_mutators_paused).
228            // Should we push to Unconstrained instead?
229
230            if self.flush_mutator {
231                mutator.flush();
232            }
233            if !self.skip_mutator_roots {
234                mmtk.scheduler.work_buckets[WorkBucketStage::Prepare]
235                    .add(ScanMutatorRoots::<C>(mutator));
236            }
237        });
238        trace!("stop_all_mutators end");
239        mmtk.get_plan().notify_mutators_paused(&mmtk.scheduler);
240        mmtk.scheduler.notify_mutators_paused(mmtk);
241        mmtk.scheduler.work_buckets[WorkBucketStage::Prepare].add(ScanVMSpecificRoots::<C>::new());
242    }
243}
244
245/// This implements `ObjectTracer` by forwarding the `trace_object` calls to the wrapped
246/// `ProcessEdgesWork` instance.
247pub(crate) struct ProcessEdgesWorkTracer<E: ProcessEdgesWork> {
248    process_edges_work: E,
249    stage: WorkBucketStage,
250}
251
252impl<E: ProcessEdgesWork> ObjectTracer for ProcessEdgesWorkTracer<E> {
253    /// Forward the `trace_object` call to the underlying `ProcessEdgesWork`,
254    /// and flush as soon as the underlying buffer of `process_edges_work` is full.
255    fn trace_object(&mut self, object: ObjectReference) -> ObjectReference {
256        let result = self.process_edges_work.trace_object(object);
257        self.flush_if_full();
258        result
259    }
260}
261
262impl<E: ProcessEdgesWork> ProcessEdgesWorkTracer<E> {
263    fn flush_if_full(&mut self) {
264        if self.process_edges_work.nodes.is_full() {
265            self.flush();
266        }
267    }
268
269    pub fn flush_if_not_empty(&mut self) {
270        if !self.process_edges_work.nodes.is_empty() {
271            self.flush();
272        }
273    }
274
275    fn flush(&mut self) {
276        let next_nodes = self.process_edges_work.pop_nodes();
277        assert!(!next_nodes.is_empty());
278        let work_packet = self.process_edges_work.create_scan_work(next_nodes);
279        let worker = self.process_edges_work.worker();
280        worker.scheduler().work_buckets[self.stage].add(work_packet);
281    }
282}
283
284/// This type implements `ObjectTracerContext` by creating a temporary `ProcessEdgesWork` during
285/// the call to `with_tracer`, making use of its `trace_object` method.  It then creates work
286/// packets using the methods of the `ProcessEdgesWork` and add the work packet into the given
287/// `stage`.
288pub(crate) struct ProcessEdgesWorkTracerContext<E: ProcessEdgesWork> {
289    stage: WorkBucketStage,
290    phantom_data: PhantomData<E>,
291}
292
293impl<E: ProcessEdgesWork> Clone for ProcessEdgesWorkTracerContext<E> {
294    fn clone(&self) -> Self {
295        Self { ..*self }
296    }
297}
298
299impl<E: ProcessEdgesWork> ObjectTracerContext<E::VM> for ProcessEdgesWorkTracerContext<E> {
300    type TracerType = ProcessEdgesWorkTracer<E>;
301
302    fn with_tracer<R, F>(&self, worker: &mut GCWorker<E::VM>, func: F) -> R
303    where
304        F: FnOnce(&mut Self::TracerType) -> R,
305    {
306        let mmtk = worker.mmtk;
307
308        // Prepare the underlying ProcessEdgesWork
309        let mut process_edges_work = E::new(vec![], false, mmtk, self.stage);
310        // FIXME: This line allows us to omit the borrowing lifetime of worker.
311        // We should refactor ProcessEdgesWork so that it uses `worker` locally, not as a member.
312        process_edges_work.set_worker(worker);
313
314        // Cretae the tracer.
315        let mut tracer = ProcessEdgesWorkTracer {
316            process_edges_work,
317            stage: self.stage,
318        };
319
320        // The caller can use the tracer here.
321        let result = func(&mut tracer);
322
323        // Flush the queued nodes.
324        tracer.flush_if_not_empty();
325
326        result
327    }
328}
329
330/// Delegate to the VM binding for weak reference processing.
331///
332/// Some VMs (e.g. v8) do not have a Java-like global weak reference storage, and the
333/// processing of those weakrefs may be more complex. For such case, we delegate to the
334/// VM binding to process weak references.
335///
336/// NOTE: This will replace `{Soft,Weak,Phantom}RefProcessing` and `Finalization` in the future.
337pub struct VMProcessWeakRefs<E: ProcessEdgesWork> {
338    phantom_data: PhantomData<E>,
339}
340
341impl<E: ProcessEdgesWork> VMProcessWeakRefs<E> {
342    pub fn new() -> Self {
343        Self {
344            phantom_data: PhantomData,
345        }
346    }
347}
348
349impl<E: ProcessEdgesWork> GCWork<E::VM> for VMProcessWeakRefs<E> {
350    fn do_work(&mut self, worker: &mut GCWorker<E::VM>, _mmtk: &'static MMTK<E::VM>) {
351        trace!("VMProcessWeakRefs");
352
353        let stage = WorkBucketStage::VMRefClosure;
354
355        let need_to_repeat = {
356            let tracer_factory = ProcessEdgesWorkTracerContext::<E> {
357                stage,
358                phantom_data: PhantomData,
359            };
360            <E::VM as VMBinding>::VMScanning::process_weak_refs(worker, tracer_factory)
361        };
362
363        if need_to_repeat {
364            // Schedule Self as the new sentinel so we'll call `process_weak_refs` again after the
365            // current transitive closure.
366            let new_self = Box::new(Self::new());
367
368            worker.scheduler().work_buckets[stage].set_sentinel(new_self);
369        }
370    }
371}
372
373/// Delegate to the VM binding for forwarding weak references.
374///
375/// Some VMs (e.g. v8) do not have a Java-like global weak reference storage, and the
376/// processing of those weakrefs may be more complex. For such case, we delegate to the
377/// VM binding to process weak references.
378///
379/// NOTE: This will replace `RefForwarding` and `ForwardFinalization` in the future.
380pub struct VMForwardWeakRefs<E: ProcessEdgesWork> {
381    phantom_data: PhantomData<E>,
382}
383
384impl<E: ProcessEdgesWork> VMForwardWeakRefs<E> {
385    pub fn new() -> Self {
386        Self {
387            phantom_data: PhantomData,
388        }
389    }
390}
391
392impl<E: ProcessEdgesWork> GCWork<E::VM> for VMForwardWeakRefs<E> {
393    fn do_work(&mut self, worker: &mut GCWorker<E::VM>, _mmtk: &'static MMTK<E::VM>) {
394        trace!("VMForwardWeakRefs");
395
396        let stage = WorkBucketStage::VMRefForwarding;
397
398        let tracer_factory = ProcessEdgesWorkTracerContext::<E> {
399            stage,
400            phantom_data: PhantomData,
401        };
402        <E::VM as VMBinding>::VMScanning::forward_weak_refs(worker, tracer_factory)
403    }
404}
405
406/// This work packet calls `Collection::post_forwarding`.
407///
408/// NOTE: This will replace `RefEnqueue` in the future.
409///
410/// NOTE: Although this work packet runs in parallel with the `Release` work packet, it does not
411/// access the `Plan` instance.
412#[derive(Default)]
413pub struct VMPostForwarding<VM: VMBinding> {
414    phantom_data: PhantomData<VM>,
415}
416
417impl<VM: VMBinding> GCWork<VM> for VMPostForwarding<VM> {
418    fn do_work(&mut self, worker: &mut GCWorker<VM>, _mmtk: &'static MMTK<VM>) {
419        trace!("VMPostForwarding start");
420        <VM as VMBinding>::VMCollection::post_forwarding(worker.tls);
421        trace!("VMPostForwarding end");
422    }
423}
424
425pub struct ScanMutatorRoots<C: GCWorkContext>(pub &'static mut Mutator<C::VM>);
426
427impl<C: GCWorkContext> GCWork<C::VM> for ScanMutatorRoots<C> {
428    fn do_work(&mut self, worker: &mut GCWorker<C::VM>, mmtk: &'static MMTK<C::VM>) {
429        trace!("ScanMutatorRoots for mutator {:?}", self.0.get_tls());
430        let mutators = <C::VM as VMBinding>::VMActivePlan::number_of_mutators();
431        let factory = ProcessEdgesWorkRootsWorkFactory::<
432            C::VM,
433            C::DefaultProcessEdges,
434            C::PinningProcessEdges,
435        >::new(mmtk);
436        <C::VM as VMBinding>::VMScanning::scan_roots_in_mutator_thread(
437            worker.tls,
438            unsafe { &mut *(self.0 as *mut _) },
439            factory,
440        );
441        self.0.flush();
442
443        if mmtk.state.inform_stack_scanned(mutators) {
444            <C::VM as VMBinding>::VMScanning::notify_initial_thread_scan_complete(
445                false, worker.tls,
446            );
447            mmtk.set_gc_status(GcStatus::GcProper);
448        }
449    }
450}
451
452#[derive(Default)]
453pub struct ScanVMSpecificRoots<C: GCWorkContext>(PhantomData<C>);
454
455impl<C: GCWorkContext> ScanVMSpecificRoots<C> {
456    pub fn new() -> Self {
457        Self(PhantomData)
458    }
459}
460
461impl<C: GCWorkContext> GCWork<C::VM> for ScanVMSpecificRoots<C> {
462    fn do_work(&mut self, worker: &mut GCWorker<C::VM>, mmtk: &'static MMTK<C::VM>) {
463        trace!("ScanStaticRoots");
464        let factory = ProcessEdgesWorkRootsWorkFactory::<
465            C::VM,
466            C::DefaultProcessEdges,
467            C::PinningProcessEdges,
468        >::new(mmtk);
469        <C::VM as VMBinding>::VMScanning::scan_vm_specific_roots(worker.tls, factory);
470    }
471}
472
473pub struct ProcessEdgesBase<VM: VMBinding> {
474    pub slots: Vec<VM::VMSlot>,
475    pub nodes: VectorObjectQueue,
476    mmtk: &'static MMTK<VM>,
477    // Use raw pointer for fast pointer dereferencing, instead of using `Option<&'static mut GCWorker<E::VM>>`.
478    // Because a copying gc will dereference this pointer at least once for every object copy.
479    worker: *mut GCWorker<VM>,
480    pub roots: bool,
481    pub bucket: WorkBucketStage,
482}
483
484unsafe impl<VM: VMBinding> Send for ProcessEdgesBase<VM> {}
485
486impl<VM: VMBinding> ProcessEdgesBase<VM> {
487    // Requires an MMTk reference. Each plan-specific type that uses ProcessEdgesBase can get a static plan reference
488    // at creation. This avoids overhead for dynamic dispatch or downcasting plan for each object traced.
489    pub fn new(
490        slots: Vec<VM::VMSlot>,
491        roots: bool,
492        mmtk: &'static MMTK<VM>,
493        bucket: WorkBucketStage,
494    ) -> Self {
495        #[cfg(feature = "extreme_assertions")]
496        if crate::util::slot_logger::should_check_duplicate_slots(mmtk.get_plan()) {
497            for slot in &slots {
498                // log slot, panic if already logged
499                mmtk.slot_logger.log_slot(*slot);
500            }
501        }
502        Self {
503            slots,
504            nodes: VectorObjectQueue::new(),
505            mmtk,
506            worker: std::ptr::null_mut(),
507            roots,
508            bucket,
509        }
510    }
511    pub fn set_worker(&mut self, worker: &mut GCWorker<VM>) {
512        self.worker = worker;
513    }
514
515    pub fn worker(&self) -> &'static mut GCWorker<VM> {
516        unsafe { &mut *self.worker }
517    }
518
519    pub fn mmtk(&self) -> &'static MMTK<VM> {
520        self.mmtk
521    }
522
523    pub fn plan(&self) -> &'static dyn Plan<VM = VM> {
524        self.mmtk.get_plan()
525    }
526
527    /// Pop all nodes from nodes, and clear nodes to an empty vector.
528    pub fn pop_nodes(&mut self) -> Vec<ObjectReference> {
529        self.nodes.take()
530    }
531
532    pub fn is_roots(&self) -> bool {
533        self.roots
534    }
535}
536
537/// A short-hand for `<E::VM as VMBinding>::VMSlot`.
538pub type SlotOf<E> = <<E as ProcessEdgesWork>::VM as VMBinding>::VMSlot;
539
540/// An abstract trait for work packets that process object graph edges.  Its method
541/// [`ProcessEdgesWork::trace_object`] traces an object and, upon first visit, enqueues it into an
542/// internal queue inside the `ProcessEdgesWork` instance.  Each implementation of this trait
543/// implement `trace_object` differently.  During [`Plan::schedule_collection`], plans select
544/// (usually via `GCWorkContext`) specialized implementations of this trait to be used during each
545/// trace according the nature of each trace, such as whether it is a nursery collection, whether it
546/// is a defrag collection, whether it pins objects, etc.
547///
548/// This trait was originally designed for work packets that process object graph edges represented
549/// as slots.  The constructor [`ProcessEdgesWork::new`] takes a vector of slots, and the created
550/// work packet will trace the objects pointed by the object reference in each slot using the
551/// `trace_object` method, and update the slot if the GC moves the target object when tracing.
552///
553/// This trait can also be used merely as a provider of the `trace_object` method by giving it an
554/// empty vector of slots.  This is useful for node-enqueuing tracing
555/// ([`Scanning::scan_object_and_trace_edges`]) as well as weak reference processing
556/// ([`Scanning::process_weak_refs`] as well as `ReferenceProcessor` and `FinalizableProcessor`).
557/// In those cases, the caller passes the reference to the target object to `trace_object`, an the
558/// caller is responsible for updating the slots according the return value of `trace_object`.
559///
560/// TODO: We should refactor this trait to decouple it from slots. See:
561/// <https://github.com/mmtk/mmtk-core/issues/599>
562pub trait ProcessEdgesWork:
563    Send + 'static + Sized + DerefMut + Deref<Target = ProcessEdgesBase<Self::VM>>
564{
565    /// The associate type for the VM.
566    type VM: VMBinding;
567
568    /// The work packet type for scanning objects when using this ProcessEdgesWork.
569    type ScanObjectsWorkType: ScanObjectsWork<Self::VM>;
570
571    /// The maximum number of slots that should be put to one of this work packets.
572    /// The caller who creates a work packet of this trait should be responsible to
573    /// comply with this capacity.
574    /// Higher capacity means the packet will take longer to finish, and may lead to
575    /// bad load balancing. On the other hand, lower capacity would lead to higher cost
576    /// on scheduling many small work packets. It is important to find a proper capacity.
577    const CAPACITY: usize = EDGES_WORK_BUFFER_SIZE;
578    /// Do we update object reference? This has to be true for a moving GC.
579    const OVERWRITE_REFERENCE: bool = true;
580    /// If true, we do object scanning in this work packet with the same worker without scheduling overhead.
581    /// If false, we will add object scanning work packets to the global queue and allow other workers to work on it.
582    const SCAN_OBJECTS_IMMEDIATELY: bool = true;
583
584    /// Create a [`ProcessEdgesWork`].
585    ///
586    /// Arguments:
587    /// * `slots`: a vector of slots.
588    /// * `roots`: are the objects root reachable objects?
589    /// * `mmtk`: a reference to the MMTK instance.
590    /// * `bucket`: which work bucket this packet belongs to. Further work generated from this packet will also be put to the same bucket.
591    fn new(
592        slots: Vec<SlotOf<Self>>,
593        roots: bool,
594        mmtk: &'static MMTK<Self::VM>,
595        bucket: WorkBucketStage,
596    ) -> Self;
597
598    /// Trace an MMTk object. The implementation should forward this call to the policy-specific
599    /// `trace_object()` methods, depending on which space this object is in.
600    /// If the object is not in any MMTk space, the implementation should forward the call to
601    /// `ActivePlan::vm_trace_object()` to let the binding handle the tracing.
602    fn trace_object(&mut self, object: ObjectReference) -> ObjectReference;
603
604    /// If the work includes roots, we will store the roots somewhere so for sanity GC, we can do another
605    /// transitive closure from the roots.
606    #[cfg(feature = "sanity")]
607    fn cache_roots_for_sanity_gc(&mut self) {
608        assert!(self.roots);
609        self.mmtk()
610            .sanity_checker
611            .lock()
612            .unwrap()
613            .add_root_slots(self.slots.clone());
614    }
615
616    /// Start the a scan work packet. If SCAN_OBJECTS_IMMEDIATELY, the work packet will be executed immediately, in this method.
617    /// Otherwise, the work packet will be added the Closure work bucket and will be dispatched later by the scheduler.
618    fn start_or_dispatch_scan_work(&mut self, mut work_packet: impl GCWork<Self::VM>) {
619        if Self::SCAN_OBJECTS_IMMEDIATELY {
620            // We execute this `scan_objects_work` immediately.
621            // This is expected to be a useful optimization because,
622            // say for _pmd_ with 200M heap, we're likely to have 50000~60000 `ScanObjects` work packets
623            // being dispatched (similar amount to `ProcessEdgesWork`).
624            // Executing these work packets now can remarkably reduce the global synchronization time.
625            work_packet.do_work(self.worker(), self.mmtk);
626        } else {
627            debug_assert!(self.bucket != WorkBucketStage::Unconstrained);
628            self.mmtk.scheduler.work_buckets[self.bucket].add(work_packet);
629        }
630    }
631
632    /// Create an object-scanning work packet to be used for this ProcessEdgesWork.
633    ///
634    /// `roots` indicates if we are creating a packet for root scanning.  It is only true when this
635    /// method is called to handle `RootsWorkFactory::create_process_pinning_roots_work`.
636    fn create_scan_work(&self, nodes: Vec<ObjectReference>) -> Self::ScanObjectsWorkType;
637
638    /// Flush the nodes in ProcessEdgesBase, and create a ScanObjects work packet for it. If the node set is empty,
639    /// this method will simply return with no work packet created.
640    fn flush(&mut self) {
641        let nodes = self.pop_nodes();
642        if !nodes.is_empty() {
643            self.start_or_dispatch_scan_work(self.create_scan_work(nodes));
644        }
645    }
646
647    /// Process a slot, including loading the object reference from the memory slot,
648    /// trace the object and store back the new object reference if necessary.
649    fn process_slot(&mut self, slot: SlotOf<Self>) {
650        let Some(object) = slot.load() else {
651            // Skip slots that are not holding an object reference.
652            return;
653        };
654        let new_object = self.trace_object(object);
655        if Self::OVERWRITE_REFERENCE && new_object != object {
656            slot.store(new_object);
657        }
658    }
659
660    /// Process all the slots in the work packet.
661    fn process_slots(&mut self) {
662        probe!(mmtk, process_slots, self.slots.len(), self.is_roots());
663        for i in 0..self.slots.len() {
664            self.process_slot(self.slots[i])
665        }
666    }
667}
668
669impl<E: ProcessEdgesWork> GCWork<E::VM> for E {
670    fn do_work(&mut self, worker: &mut GCWorker<E::VM>, _mmtk: &'static MMTK<E::VM>) {
671        self.set_worker(worker);
672        self.process_slots();
673        if !self.nodes.is_empty() {
674            self.flush();
675        }
676        #[cfg(feature = "sanity")]
677        if self.roots && !_mmtk.is_in_sanity() {
678            self.cache_roots_for_sanity_gc();
679        }
680        trace!("ProcessEdgesWork End");
681    }
682}
683
684/// A general implementation of [`ProcessEdgesWork`] using SFT. A plan can always implement their
685/// own [`ProcessEdgesWork`] instances. However, most plans can use this work packet for tracing amd
686/// they do not need to provide a plan-specific trace object work packet. If they choose to use this
687/// type, they need to provide a correct implementation for some related methods (such as
688/// `Space.set_copy_for_sft_trace()`, `SFT.sft_trace_object()`). Some plans are not using this type,
689/// mostly due to more complex tracing. Either it is impossible to use this type, or there is
690/// performance overheads for using this general trace type. In such cases, they implement their
691/// specific [`ProcessEdgesWork`] instances.
692// TODO: This is not used any more. Should we remove it?
693#[allow(dead_code)]
694pub struct SFTProcessEdges<VM: VMBinding> {
695    pub base: ProcessEdgesBase<VM>,
696}
697
698impl<VM: VMBinding> ProcessEdgesWork for SFTProcessEdges<VM> {
699    type VM = VM;
700    type ScanObjectsWorkType = ScanObjects<Self>;
701
702    fn new(
703        slots: Vec<SlotOf<Self>>,
704        roots: bool,
705        mmtk: &'static MMTK<VM>,
706        bucket: WorkBucketStage,
707    ) -> Self {
708        let base = ProcessEdgesBase::new(slots, roots, mmtk, bucket);
709        Self { base }
710    }
711
712    fn trace_object(&mut self, object: ObjectReference) -> ObjectReference {
713        use crate::policy::sft::GCWorkerMutRef;
714
715        // Erase <VM> type parameter
716        let worker = GCWorkerMutRef::new(self.worker());
717
718        // Invoke trace object on sft
719        let sft = unsafe { crate::mmtk::SFT_MAP.get_unchecked(object.to_raw_address()) };
720        sft.sft_trace_object(&mut self.base.nodes, object, worker)
721    }
722
723    fn create_scan_work(&self, nodes: Vec<ObjectReference>) -> ScanObjects<Self> {
724        ScanObjects::<Self>::new(nodes, false, self.bucket)
725    }
726}
727
728/// An implementation of `RootsWorkFactory` that creates work packets based on `ProcessEdgesWork`
729/// for handling roots.  The `DPE` and the `PPE` type parameters correspond to the
730/// `DefaultProcessEdge` and the `PinningProcessEdges` type members of the [`GCWorkContext`] trait.
731pub(crate) struct ProcessEdgesWorkRootsWorkFactory<
732    VM: VMBinding,
733    DPE: ProcessEdgesWork<VM = VM>,
734    PPE: ProcessEdgesWork<VM = VM>,
735> {
736    mmtk: &'static MMTK<VM>,
737    phantom: PhantomData<(DPE, PPE)>,
738}
739
740impl<VM: VMBinding, DPE: ProcessEdgesWork<VM = VM>, PPE: ProcessEdgesWork<VM = VM>> Clone
741    for ProcessEdgesWorkRootsWorkFactory<VM, DPE, PPE>
742{
743    fn clone(&self) -> Self {
744        Self {
745            mmtk: self.mmtk,
746            phantom: PhantomData,
747        }
748    }
749}
750
751/// For USDT tracepoints for roots.
752/// Keep in sync with `tools/tracing/timeline/visualize.py`.
753#[repr(usize)]
754enum RootsKind {
755    NORMAL = 0,
756    PINNING = 1,
757    TPINNING = 2,
758}
759
760impl<VM: VMBinding, DPE: ProcessEdgesWork<VM = VM>, PPE: ProcessEdgesWork<VM = VM>>
761    RootsWorkFactory<VM::VMSlot> for ProcessEdgesWorkRootsWorkFactory<VM, DPE, PPE>
762{
763    fn create_process_roots_work(&mut self, slots: Vec<VM::VMSlot>) {
764        // Note: We should use the same USDT name "mmtk:roots" for all the three kinds of roots. A
765        // VM binding may not call all of the three methods in this impl. For example, the OpenJDK
766        // binding only calls `create_process_roots_work`, and the Ruby binding only calls
767        // `create_process_pinning_roots_work`. Because `ProcessEdgesWorkRootsWorkFactory<VM, DPE,
768        // PPE>` is a generic type, the Rust compiler emits the function bodies on demand, so the
769        // resulting machine code may not contain all three USDT trace points.  If they have
770        // different names, and our `capture.bt` mentions all of them, `bpftrace` may complain that
771        // it cannot find one or more of those USDT trace points in the binary.
772        probe!(mmtk, roots, RootsKind::NORMAL, slots.len());
773        crate::memory_manager::add_work_packet(
774            self.mmtk,
775            WorkBucketStage::Closure,
776            DPE::new(slots, true, self.mmtk, WorkBucketStage::Closure),
777        );
778    }
779
780    fn create_process_pinning_roots_work(&mut self, nodes: Vec<ObjectReference>) {
781        probe!(mmtk, roots, RootsKind::PINNING, nodes.len());
782        // Will process roots within the PinningRootsTrace bucket
783        // And put work in the Closure bucket
784        crate::memory_manager::add_work_packet(
785            self.mmtk,
786            WorkBucketStage::PinningRootsTrace,
787            ProcessRootNodes::<VM, PPE, DPE>::new(nodes, WorkBucketStage::Closure),
788        );
789    }
790
791    fn create_process_tpinning_roots_work(&mut self, nodes: Vec<ObjectReference>) {
792        probe!(mmtk, roots, RootsKind::TPINNING, nodes.len());
793        crate::memory_manager::add_work_packet(
794            self.mmtk,
795            WorkBucketStage::TPinningClosure,
796            ProcessRootNodes::<VM, PPE, PPE>::new(nodes, WorkBucketStage::TPinningClosure),
797        );
798    }
799}
800
801impl<VM: VMBinding, DPE: ProcessEdgesWork<VM = VM>, PPE: ProcessEdgesWork<VM = VM>>
802    ProcessEdgesWorkRootsWorkFactory<VM, DPE, PPE>
803{
804    fn new(mmtk: &'static MMTK<VM>) -> Self {
805        Self {
806            mmtk,
807            phantom: PhantomData,
808        }
809    }
810}
811
812impl<VM: VMBinding> Deref for SFTProcessEdges<VM> {
813    type Target = ProcessEdgesBase<VM>;
814    fn deref(&self) -> &Self::Target {
815        &self.base
816    }
817}
818
819impl<VM: VMBinding> DerefMut for SFTProcessEdges<VM> {
820    fn deref_mut(&mut self) -> &mut Self::Target {
821        &mut self.base
822    }
823}
824
825/// Trait for a work packet that scans objects
826pub trait ScanObjectsWork<VM: VMBinding>: GCWork<VM> + Sized {
827    /// The associated ProcessEdgesWork for processing the outgoing edges of the objects in this
828    /// packet.
829    type E: ProcessEdgesWork<VM = VM>;
830
831    /// Called after each object is scanned.
832    fn post_scan_object(&self, object: ObjectReference);
833
834    /// Return the work bucket for this work packet and its derived work packets.
835    fn get_bucket(&self) -> WorkBucketStage;
836
837    /// The common code for ScanObjects and PlanScanObjects.
838    fn do_work_common(
839        &self,
840        buffer: &[ObjectReference],
841        worker: &mut GCWorker<<Self::E as ProcessEdgesWork>::VM>,
842        mmtk: &'static MMTK<<Self::E as ProcessEdgesWork>::VM>,
843    ) {
844        let tls = worker.tls;
845
846        let objects_to_scan = buffer;
847
848        // Scan the objects in the list that supports slot-enququing.
849        let mut scan_later = vec![];
850        {
851            let mut closure = ObjectsClosure::<Self::E>::new(worker, self.get_bucket());
852
853            // For any object we need to scan, we count its live bytes.
854            // Check the option outside the loop for better performance.
855            if crate::util::rust_util::unlikely(*mmtk.get_options().count_live_bytes_in_gc) {
856                // Borrow before the loop.
857                let mut live_bytes_stats = closure.worker.shared.live_bytes_per_space.borrow_mut();
858                for object in objects_to_scan.iter().copied() {
859                    crate::scheduler::worker::GCWorkerShared::<VM>::increase_live_bytes(
860                        &mut live_bytes_stats,
861                        object,
862                    );
863                }
864            }
865
866            for object in objects_to_scan.iter().copied() {
867                if <VM as VMBinding>::VMScanning::support_slot_enqueuing(tls, object) {
868                    trace!("Scan object (slot) {}", object);
869                    // If an object supports slot-enqueuing, we enqueue its slots.
870                    <VM as VMBinding>::VMScanning::scan_object(tls, object, &mut closure);
871                    self.post_scan_object(object);
872                } else {
873                    // If an object does not support slot-enqueuing, we have to use
874                    // `Scanning::scan_object_and_trace_edges` and offload the job of updating the
875                    // reference field to the VM.
876                    //
877                    // However, at this point, `closure` is borrowing `worker`.
878                    // So we postpone the processing of objects that needs object enqueuing
879                    scan_later.push(object);
880                }
881            }
882        }
883
884        let total_objects = objects_to_scan.len();
885        let scan_and_trace = scan_later.len();
886        probe!(mmtk, scan_objects, total_objects, scan_and_trace);
887
888        // If any object does not support slot-enqueuing, we process them now.
889        if !scan_later.is_empty() {
890            let object_tracer_context = ProcessEdgesWorkTracerContext::<Self::E> {
891                stage: self.get_bucket(),
892                phantom_data: PhantomData,
893            };
894
895            object_tracer_context.with_tracer(worker, |object_tracer| {
896                // Scan objects and trace their outgoing edges at the same time.
897                for object in scan_later.iter().copied() {
898                    trace!("Scan object (node) {}", object);
899                    <VM as VMBinding>::VMScanning::scan_object_and_trace_edges(
900                        tls,
901                        object,
902                        object_tracer,
903                    );
904                    self.post_scan_object(object);
905                }
906            });
907        }
908    }
909}
910
911/// Scan objects and enqueue the slots of the objects.  For objects that do not support
912/// slot-enqueuing, this work packet also traces their outgoing edges directly.
913///
914/// This work packet does not execute policy-specific post-scanning hooks
915/// (it won't call `post_scan_object()` in [`policy::gc_work::PolicyTraceObject`]).
916/// It should be used only for policies that do not perform policy-specific actions when scanning
917/// an object.
918pub struct ScanObjects<Edges: ProcessEdgesWork> {
919    buffer: Vec<ObjectReference>,
920    #[allow(unused)]
921    concurrent: bool,
922    phantom: PhantomData<Edges>,
923    bucket: WorkBucketStage,
924}
925
926impl<Edges: ProcessEdgesWork> ScanObjects<Edges> {
927    pub fn new(buffer: Vec<ObjectReference>, concurrent: bool, bucket: WorkBucketStage) -> Self {
928        Self {
929            buffer,
930            concurrent,
931            phantom: PhantomData,
932            bucket,
933        }
934    }
935}
936
937impl<VM: VMBinding, E: ProcessEdgesWork<VM = VM>> ScanObjectsWork<VM> for ScanObjects<E> {
938    type E = E;
939
940    fn get_bucket(&self) -> WorkBucketStage {
941        self.bucket
942    }
943
944    fn post_scan_object(&self, _object: ObjectReference) {
945        // Do nothing.
946    }
947}
948
949impl<E: ProcessEdgesWork> GCWork<E::VM> for ScanObjects<E> {
950    fn do_work(&mut self, worker: &mut GCWorker<E::VM>, mmtk: &'static MMTK<E::VM>) {
951        trace!("ScanObjects");
952        self.do_work_common(&self.buffer, worker, mmtk);
953        trace!("ScanObjects End");
954    }
955}
956
957use crate::mmtk::MMTK;
958use crate::plan::Plan;
959use crate::plan::PlanTraceObject;
960use crate::policy::gc_work::TraceKind;
961
962/// This provides an implementation of [`crate::scheduler::gc_work::ProcessEdgesWork`]. A plan that implements
963/// `PlanTraceObject` can use this work packet for tracing objects.
964pub struct PlanProcessEdges<
965    VM: VMBinding,
966    P: Plan<VM = VM> + PlanTraceObject<VM>,
967    const KIND: TraceKind,
968> {
969    plan: &'static P,
970    base: ProcessEdgesBase<VM>,
971}
972
973impl<VM: VMBinding, P: PlanTraceObject<VM> + Plan<VM = VM>, const KIND: TraceKind> ProcessEdgesWork
974    for PlanProcessEdges<VM, P, KIND>
975{
976    type VM = VM;
977    type ScanObjectsWorkType = PlanScanObjects<Self, P>;
978
979    fn new(
980        slots: Vec<SlotOf<Self>>,
981        roots: bool,
982        mmtk: &'static MMTK<VM>,
983        bucket: WorkBucketStage,
984    ) -> Self {
985        let base = ProcessEdgesBase::new(slots, roots, mmtk, bucket);
986        let plan = base.plan().downcast_ref::<P>().unwrap();
987        Self { plan, base }
988    }
989
990    fn create_scan_work(&self, nodes: Vec<ObjectReference>) -> Self::ScanObjectsWorkType {
991        PlanScanObjects::<Self, P>::new(self.plan, nodes, false, self.bucket)
992    }
993
994    fn trace_object(&mut self, object: ObjectReference) -> ObjectReference {
995        // We cannot borrow `self` twice in a call, so we extract `worker` as a local variable.
996        let worker = self.worker();
997        self.plan
998            .trace_object::<VectorObjectQueue, KIND>(&mut self.base.nodes, object, worker)
999    }
1000
1001    fn process_slot(&mut self, slot: SlotOf<Self>) {
1002        let Some(object) = slot.load() else {
1003            // Skip slots that are not holding an object reference.
1004            return;
1005        };
1006        let new_object = self.trace_object(object);
1007        if P::may_move_objects::<KIND>() && new_object != object {
1008            slot.store(new_object);
1009        }
1010    }
1011}
1012
1013// Impl Deref/DerefMut to ProcessEdgesBase for PlanProcessEdges
1014impl<VM: VMBinding, P: PlanTraceObject<VM> + Plan<VM = VM>, const KIND: TraceKind> Deref
1015    for PlanProcessEdges<VM, P, KIND>
1016{
1017    type Target = ProcessEdgesBase<VM>;
1018    fn deref(&self) -> &Self::Target {
1019        &self.base
1020    }
1021}
1022
1023impl<VM: VMBinding, P: PlanTraceObject<VM> + Plan<VM = VM>, const KIND: TraceKind> DerefMut
1024    for PlanProcessEdges<VM, P, KIND>
1025{
1026    fn deref_mut(&mut self) -> &mut Self::Target {
1027        &mut self.base
1028    }
1029}
1030
1031/// This is an alternative to `ScanObjects` that calls the `post_scan_object` of the policy
1032/// selected by the plan.  It is applicable to plans that derive `PlanTraceObject`.
1033pub struct PlanScanObjects<E: ProcessEdgesWork, P: Plan<VM = E::VM> + PlanTraceObject<E::VM>> {
1034    plan: &'static P,
1035    buffer: Vec<ObjectReference>,
1036    #[allow(dead_code)]
1037    concurrent: bool,
1038    phantom: PhantomData<E>,
1039    bucket: WorkBucketStage,
1040}
1041
1042impl<E: ProcessEdgesWork, P: Plan<VM = E::VM> + PlanTraceObject<E::VM>> PlanScanObjects<E, P> {
1043    pub fn new(
1044        plan: &'static P,
1045        buffer: Vec<ObjectReference>,
1046        concurrent: bool,
1047        bucket: WorkBucketStage,
1048    ) -> Self {
1049        Self {
1050            plan,
1051            buffer,
1052            concurrent,
1053            phantom: PhantomData,
1054            bucket,
1055        }
1056    }
1057}
1058
1059impl<E: ProcessEdgesWork, P: Plan<VM = E::VM> + PlanTraceObject<E::VM>> ScanObjectsWork<E::VM>
1060    for PlanScanObjects<E, P>
1061{
1062    type E = E;
1063
1064    fn get_bucket(&self) -> WorkBucketStage {
1065        self.bucket
1066    }
1067
1068    fn post_scan_object(&self, object: ObjectReference) {
1069        self.plan.post_scan_object(object);
1070    }
1071}
1072
1073impl<E: ProcessEdgesWork, P: Plan<VM = E::VM> + PlanTraceObject<E::VM>> GCWork<E::VM>
1074    for PlanScanObjects<E, P>
1075{
1076    fn do_work(&mut self, worker: &mut GCWorker<E::VM>, mmtk: &'static MMTK<E::VM>) {
1077        trace!("PlanScanObjects");
1078        self.do_work_common(&self.buffer, worker, mmtk);
1079        trace!("PlanScanObjects End");
1080    }
1081}
1082
1083/// This work packet processes pinning roots.
1084///
1085/// The `roots` member holds a list of `ObjectReference` to objects directly pointed by roots.
1086/// These objects will be traced using `R2OPE` (Root-to-Object Process Edges).
1087///
1088/// After that, it will create work packets for tracing their children.  Those work packets (and
1089/// the work packets further created by them) will use `O2OPE` (Object-to-Object Process Edges) as
1090/// their `ProcessEdgesWork` implementations.
1091///
1092/// Because `roots` are pinning roots, `R2OPE` must be a `ProcessEdgesWork` that never moves any
1093/// object.
1094///
1095/// The choice of `O2OPE` determines whether the `roots` are transitively pinning or not.
1096///
1097/// -   If `O2OPE` is set to a `ProcessEdgesWork` that never moves objects, all descendents of
1098///     `roots` will not be moved in this GC.  That implements transitive pinning roots.
1099/// -   If `O2OPE` may move objects, then this `ProcessRootsNode<VM, R2OPE, O2OPE>` work packet
1100///     will only pin the objects in `roots` (because `R2OPE` must not move objects anyway), but
1101///     not their descendents.
1102pub(crate) struct ProcessRootNodes<
1103    VM: VMBinding,
1104    R2OPE: ProcessEdgesWork<VM = VM>,
1105    O2OPE: ProcessEdgesWork<VM = VM>,
1106> {
1107    phantom: PhantomData<(VM, R2OPE, O2OPE)>,
1108    roots: Vec<ObjectReference>,
1109    bucket: WorkBucketStage,
1110}
1111
1112impl<VM: VMBinding, R2OPE: ProcessEdgesWork<VM = VM>, O2OPE: ProcessEdgesWork<VM = VM>>
1113    ProcessRootNodes<VM, R2OPE, O2OPE>
1114{
1115    pub fn new(nodes: Vec<ObjectReference>, bucket: WorkBucketStage) -> Self {
1116        Self {
1117            phantom: PhantomData,
1118            roots: nodes,
1119            bucket,
1120        }
1121    }
1122}
1123
1124impl<VM: VMBinding, R2OPE: ProcessEdgesWork<VM = VM>, O2OPE: ProcessEdgesWork<VM = VM>> GCWork<VM>
1125    for ProcessRootNodes<VM, R2OPE, O2OPE>
1126{
1127    fn do_work(&mut self, worker: &mut GCWorker<VM>, mmtk: &'static MMTK<VM>) {
1128        trace!("ProcessRootNodes");
1129
1130        #[cfg(feature = "sanity")]
1131        {
1132            if !mmtk.is_in_sanity() {
1133                mmtk.sanity_checker
1134                    .lock()
1135                    .unwrap()
1136                    .add_root_nodes(self.roots.clone());
1137            }
1138        }
1139
1140        let num_roots = self.roots.len();
1141
1142        // This step conceptually traces the edges from root slots to the objects they point to.
1143        // However, VMs that deliver root objects instead of root slots are incapable of updating
1144        // root slots.  Therefore, we call `trace_object` on those objects, and assert the GC
1145        // doesn't move those objects because we cannot store the updated references back to the
1146        // slots.
1147        //
1148        // The `root_objects_to_scan` variable will hold those root objects which are traced for the
1149        // first time.  We will create a work packet for scanning those roots.
1150        let root_objects_to_scan = {
1151            // We create an instance of E to use its `trace_object` method and its object queue.
1152            let mut process_edges_work =
1153                R2OPE::new(vec![], true, mmtk, WorkBucketStage::PinningRootsTrace);
1154            process_edges_work.set_worker(worker);
1155
1156            for object in self.roots.iter().copied() {
1157                let new_object = process_edges_work.trace_object(object);
1158                debug_assert_eq!(
1159                    object, new_object,
1160                    "Object moved while tracing root unmovable root object: {} -> {}",
1161                    object, new_object
1162                );
1163            }
1164
1165            // This contains root objects that are visited the first time.
1166            // It is sufficient to only scan these objects.
1167            process_edges_work.nodes.take()
1168        };
1169
1170        let num_enqueued_nodes = root_objects_to_scan.len();
1171        probe!(mmtk, process_root_nodes, num_roots, num_enqueued_nodes);
1172
1173        if !root_objects_to_scan.is_empty() {
1174            let process_edges_work = O2OPE::new(vec![], false, mmtk, self.bucket);
1175            let work = process_edges_work.create_scan_work(root_objects_to_scan);
1176            crate::memory_manager::add_work_packet(mmtk, self.bucket, work);
1177        }
1178
1179        trace!("ProcessRootNodes End");
1180    }
1181}
1182
1183/// A `ProcessEdgesWork` type that panics when any of its method is used.
1184/// This is currently used for plans that do not support transitively pinning.
1185#[derive(Default)]
1186pub struct UnsupportedProcessEdges<VM: VMBinding> {
1187    phantom: PhantomData<VM>,
1188}
1189
1190impl<VM: VMBinding> Deref for UnsupportedProcessEdges<VM> {
1191    type Target = ProcessEdgesBase<VM>;
1192    fn deref(&self) -> &Self::Target {
1193        panic!("unsupported!")
1194    }
1195}
1196
1197impl<VM: VMBinding> DerefMut for UnsupportedProcessEdges<VM> {
1198    fn deref_mut(&mut self) -> &mut Self::Target {
1199        panic!("unsupported!")
1200    }
1201}
1202
1203impl<VM: VMBinding> ProcessEdgesWork for UnsupportedProcessEdges<VM> {
1204    type VM = VM;
1205
1206    type ScanObjectsWorkType = ScanObjects<Self>;
1207
1208    fn new(
1209        _slots: Vec<SlotOf<Self>>,
1210        _roots: bool,
1211        _mmtk: &'static MMTK<Self::VM>,
1212        _bucket: WorkBucketStage,
1213    ) -> Self {
1214        panic!("unsupported!")
1215    }
1216
1217    fn trace_object(&mut self, _object: ObjectReference) -> ObjectReference {
1218        panic!("unsupported!")
1219    }
1220
1221    fn create_scan_work(&self, _nodes: Vec<ObjectReference>) -> Self::ScanObjectsWorkType {
1222        panic!("unsupported!")
1223    }
1224}