mmtk/policy/immix/
immixspace.rs

1use super::defrag::StatsForDefrag;
2use super::line::*;
3use super::{block::*, defrag::Defrag};
4use crate::plan::VectorObjectQueue;
5use crate::policy::gc_work::{TraceKind, DEFAULT_TRACE, TRACE_KIND_TRANSITIVE_PIN};
6use crate::policy::sft::GCWorkerMutRef;
7use crate::policy::sft::SFT;
8use crate::policy::sft_map::SFTMap;
9use crate::policy::space::{CommonSpace, Space};
10use crate::util::alloc::allocator::AllocationOptions;
11use crate::util::alloc::allocator::AllocatorContext;
12use crate::util::constants::LOG_BYTES_IN_PAGE;
13use crate::util::heap::chunk_map::*;
14use crate::util::heap::BlockPageResource;
15use crate::util::heap::PageResource;
16use crate::util::linear_scan::{Region, RegionIterator};
17use crate::util::metadata::log_bit::UnlogBitsOperation;
18use crate::util::metadata::side_metadata::SideMetadataSpec;
19#[cfg(feature = "vo_bit")]
20use crate::util::metadata::vo_bit;
21use crate::util::metadata::{self, MetadataSpec};
22use crate::util::object_enum::ObjectEnumerator;
23use crate::util::object_forwarding;
24use crate::util::{copy::*, epilogue, object_enum};
25use crate::util::{Address, ObjectReference};
26use crate::vm::*;
27use crate::{
28    plan::ObjectQueue,
29    scheduler::{GCWork, GCWorkScheduler, GCWorker, WorkBucketStage},
30    util::opaque_pointer::{VMThread, VMWorkerThread},
31    MMTK,
32};
33use atomic::Ordering;
34use std::sync::{atomic::AtomicU8, atomic::AtomicUsize, Arc};
35
36pub(crate) const TRACE_KIND_FAST: TraceKind = 0;
37pub(crate) const TRACE_KIND_DEFRAG: TraceKind = 1;
38
39pub struct ImmixSpace<VM: VMBinding> {
40    common: CommonSpace<VM>,
41    pr: BlockPageResource<VM, Block>,
42    /// Allocation status for all chunks in immix space
43    pub chunk_map: ChunkMap,
44    /// Current line mark state
45    pub line_mark_state: AtomicU8,
46    /// Line mark state in previous GC
47    line_unavail_state: AtomicU8,
48    /// A list of all reusable blocks
49    pub reusable_blocks: ReusableBlockPool,
50    /// Defrag utilities
51    pub(super) defrag: Defrag,
52    /// How many lines have been consumed since last GC?
53    lines_consumed: AtomicUsize,
54    /// Object mark state
55    mark_state: u8,
56    /// Work packet scheduler
57    scheduler: Arc<GCWorkScheduler<VM>>,
58    /// Some settings for this space
59    space_args: ImmixSpaceArgs,
60}
61
62/// Some arguments for Immix Space.
63pub struct ImmixSpaceArgs {
64    /// Whether this ImmixSpace instance contains both young and old objects.
65    /// This affects the updating of valid-object bits.  If some lines or blocks of this ImmixSpace
66    /// instance contain young objects, their VO bits need to be updated during this GC.  Currently
67    /// only StickyImmix is affected.  GenImmix allocates young objects in a separete CopySpace
68    /// nursery and its VO bits can be cleared in bulk.
69    pub mixed_age: bool,
70    /// Disable copying for this Immix space.
71    pub never_move_objects: bool,
72}
73
74unsafe impl<VM: VMBinding> Sync for ImmixSpace<VM> {}
75
76impl<VM: VMBinding> SFT for ImmixSpace<VM> {
77    fn name(&self) -> &'static str {
78        self.get_name()
79    }
80
81    fn get_forwarded_object(&self, object: ObjectReference) -> Option<ObjectReference> {
82        // If we never move objects, look no further.
83        if !self.is_movable() {
84            return None;
85        }
86
87        if object_forwarding::is_forwarded::<VM>(object) {
88            Some(object_forwarding::read_forwarding_pointer::<VM>(object))
89        } else {
90            None
91        }
92    }
93
94    fn is_live(&self, object: ObjectReference) -> bool {
95        // If the mark bit is set, it is live.
96        if self.is_marked(object) {
97            return true;
98        }
99
100        // If we never move objects, look no further.
101        if !self.is_movable() {
102            return false;
103        }
104
105        // If the object is forwarded, it is live, too.
106        object_forwarding::is_forwarded::<VM>(object)
107    }
108    #[cfg(feature = "object_pinning")]
109    fn pin_object(&self, object: ObjectReference) -> bool {
110        VM::VMObjectModel::LOCAL_PINNING_BIT_SPEC.pin_object::<VM>(object)
111    }
112    #[cfg(feature = "object_pinning")]
113    fn unpin_object(&self, object: ObjectReference) -> bool {
114        VM::VMObjectModel::LOCAL_PINNING_BIT_SPEC.unpin_object::<VM>(object)
115    }
116    #[cfg(feature = "object_pinning")]
117    fn is_object_pinned(&self, object: ObjectReference) -> bool {
118        VM::VMObjectModel::LOCAL_PINNING_BIT_SPEC.is_object_pinned::<VM>(object)
119    }
120    fn is_movable(&self) -> bool {
121        !self.space_args.never_move_objects
122    }
123
124    #[cfg(feature = "sanity")]
125    fn is_sane(&self) -> bool {
126        true
127    }
128    fn initialize_object_metadata(&self, _object: ObjectReference) {
129        #[cfg(feature = "vo_bit")]
130        crate::util::metadata::vo_bit::set_vo_bit(_object);
131    }
132    #[cfg(feature = "is_mmtk_object")]
133    fn is_mmtk_object(&self, addr: Address) -> Option<ObjectReference> {
134        crate::util::metadata::vo_bit::is_vo_bit_set_for_addr(addr)
135    }
136    #[cfg(feature = "is_mmtk_object")]
137    fn find_object_from_internal_pointer(
138        &self,
139        ptr: Address,
140        max_search_bytes: usize,
141    ) -> Option<ObjectReference> {
142        // We don't need to search more than the max object size in the immix space.
143        let search_bytes = usize::min(super::MAX_IMMIX_OBJECT_SIZE, max_search_bytes);
144        crate::util::metadata::vo_bit::find_object_from_internal_pointer::<VM>(ptr, search_bytes)
145    }
146    fn sft_trace_object(
147        &self,
148        _queue: &mut VectorObjectQueue,
149        _object: ObjectReference,
150        _worker: GCWorkerMutRef,
151    ) -> ObjectReference {
152        panic!("We do not use SFT to trace objects for Immix. sft_trace_object() cannot be used.")
153    }
154
155    fn debug_print_object_info(&self, object: ObjectReference) {
156        println!("marked  = {}", self.is_marked(object));
157        println!(
158            "line marked = {}",
159            Line::from_unaligned_address(object.to_raw_address()).is_marked(self.mark_state)
160        );
161        println!(
162            "block state = {:?}",
163            Block::from_unaligned_address(object.to_raw_address()).get_state()
164        );
165        object_forwarding::debug_print_object_forwarding_info::<VM>(object);
166        self.common.debug_print_object_global_info(object);
167    }
168}
169
170impl<VM: VMBinding> Space<VM> for ImmixSpace<VM> {
171    fn as_space(&self) -> &dyn Space<VM> {
172        self
173    }
174    fn as_sft(&self) -> &(dyn SFT + Sync + 'static) {
175        self
176    }
177    fn get_page_resource(&self) -> &dyn PageResource<VM> {
178        &self.pr
179    }
180    fn maybe_get_page_resource_mut(&mut self) -> Option<&mut dyn PageResource<VM>> {
181        Some(&mut self.pr)
182    }
183    fn common(&self) -> &CommonSpace<VM> {
184        &self.common
185    }
186    fn initialize_sft(&self, sft_map: &mut dyn SFTMap) {
187        self.common().initialize_sft(self.as_sft(), sft_map)
188    }
189    fn release_multiple_pages(&mut self, _start: Address) {
190        panic!("immixspace only releases pages enmasse")
191    }
192    fn set_copy_for_sft_trace(&mut self, _semantics: Option<CopySemantics>) {
193        panic!("We do not use SFT to trace objects for Immix. set_copy_context() cannot be used.")
194    }
195
196    fn enumerate_objects(&self, enumerator: &mut dyn ObjectEnumerator) {
197        object_enum::enumerate_blocks_from_chunk_map::<Block>(enumerator, &self.chunk_map);
198    }
199
200    fn clear_side_log_bits(&self) {
201        // Remove the following warning if we have a legitimate use case.
202        warn!("ImmixSpace::clear_side_log_bits is single-treaded.  Consider clearing side metadata in per-chunk work packets.");
203
204        let log_bit = VM::VMObjectModel::GLOBAL_LOG_BIT_SPEC.extract_side_spec();
205        for chunk in self.chunk_map.all_chunks() {
206            log_bit.bzero_metadata(chunk.start(), Chunk::BYTES);
207        }
208    }
209
210    fn set_side_log_bits(&self) {
211        // Remove the following warning if we have a legitimate use case.
212        warn!("ImmixSpace::set_side_log_bits is single-treaded.  Consider setting side metadata in per-chunk work packets.");
213
214        let log_bit = VM::VMObjectModel::GLOBAL_LOG_BIT_SPEC.extract_side_spec();
215        for chunk in self.chunk_map.all_chunks() {
216            log_bit.bset_metadata(chunk.start(), Chunk::BYTES);
217        }
218    }
219}
220
221impl<VM: VMBinding> crate::policy::gc_work::PolicyTraceObject<VM> for ImmixSpace<VM> {
222    fn trace_object<Q: ObjectQueue, const KIND: TraceKind>(
223        &self,
224        queue: &mut Q,
225        object: ObjectReference,
226        copy: Option<CopySemantics>,
227        worker: &mut GCWorker<VM>,
228    ) -> ObjectReference {
229        if KIND == TRACE_KIND_TRANSITIVE_PIN {
230            self.trace_object_without_moving(queue, object)
231        } else if KIND == TRACE_KIND_DEFRAG {
232            if Block::containing(object).is_defrag_source() {
233                debug_assert!(self.in_defrag());
234                debug_assert!(
235                    !crate::plan::is_nursery_gc(worker.mmtk.get_plan()),
236                    "Calling PolicyTraceObject on Immix in nursery GC"
237                );
238                self.trace_object_with_opportunistic_copy(
239                    queue,
240                    object,
241                    copy.unwrap(),
242                    worker,
243                    // This should not be nursery collection. Nursery collection does not use PolicyTraceObject.
244                    false,
245                )
246            } else {
247                self.trace_object_without_moving(queue, object)
248            }
249        } else if KIND == TRACE_KIND_FAST {
250            self.trace_object_without_moving(queue, object)
251        } else {
252            unreachable!()
253        }
254    }
255
256    fn post_scan_object(&self, object: ObjectReference) {
257        if super::MARK_LINE_AT_SCAN_TIME && !super::BLOCK_ONLY {
258            debug_assert!(self.in_space(object));
259            self.mark_lines(object);
260        }
261    }
262
263    #[allow(clippy::if_same_then_else)] // DEFAULT_TRACE needs a workaround which is documented below.
264    fn may_move_objects<const KIND: TraceKind>() -> bool {
265        if KIND == TRACE_KIND_DEFRAG {
266            true
267        } else if KIND == TRACE_KIND_FAST || KIND == TRACE_KIND_TRANSITIVE_PIN {
268            false
269        } else if KIND == DEFAULT_TRACE {
270            // FIXME: This is hacky. When we do a default trace, this should be a nonmoving space.
271            // The only exception is the nursery GC for sticky immix, for which, we use default trace.
272            // This function is only used for PlanProcessEdges, and for sticky immix nursery GC, we use
273            // GenNurseryProcessEdges. So it still works. But this is quite hacky anyway.
274            // See https://github.com/mmtk/mmtk-core/issues/1314 for details.
275            false
276        } else {
277            unreachable!()
278        }
279    }
280}
281
282impl<VM: VMBinding> ImmixSpace<VM> {
283    #[allow(unused)]
284    const UNMARKED_STATE: u8 = 0;
285    const MARKED_STATE: u8 = 1;
286
287    /// Get side metadata specs
288    fn side_metadata_specs() -> Vec<SideMetadataSpec> {
289        metadata::extract_side_metadata(&if super::BLOCK_ONLY {
290            vec![
291                MetadataSpec::OnSide(Block::DEFRAG_STATE_TABLE),
292                MetadataSpec::OnSide(Block::MARK_TABLE),
293                *VM::VMObjectModel::LOCAL_MARK_BIT_SPEC,
294                *VM::VMObjectModel::LOCAL_FORWARDING_BITS_SPEC,
295                *VM::VMObjectModel::LOCAL_FORWARDING_POINTER_SPEC,
296                #[cfg(feature = "object_pinning")]
297                *VM::VMObjectModel::LOCAL_PINNING_BIT_SPEC,
298            ]
299        } else {
300            vec![
301                MetadataSpec::OnSide(Line::MARK_TABLE),
302                MetadataSpec::OnSide(Block::DEFRAG_STATE_TABLE),
303                MetadataSpec::OnSide(Block::MARK_TABLE),
304                *VM::VMObjectModel::LOCAL_MARK_BIT_SPEC,
305                *VM::VMObjectModel::LOCAL_FORWARDING_BITS_SPEC,
306                *VM::VMObjectModel::LOCAL_FORWARDING_POINTER_SPEC,
307                #[cfg(feature = "object_pinning")]
308                *VM::VMObjectModel::LOCAL_PINNING_BIT_SPEC,
309            ]
310        })
311    }
312
313    pub fn new(
314        args: crate::policy::space::PlanCreateSpaceArgs<VM>,
315        mut space_args: ImmixSpaceArgs,
316    ) -> Self {
317        if args.unlog_traced_object {
318            assert!(
319                args.constraints.needs_log_bit,
320                "Invalid args when the plan does not use log bit"
321            );
322        }
323
324        // Make sure we override the space args if we force non moving Immix
325        if cfg!(feature = "immix_non_moving") && !space_args.never_move_objects {
326            info!(
327                "Overriding never_moves_objects for Immix Space {}, as the immix_non_moving feature is set. Block size: 2^{}",
328                args.name,
329                Block::LOG_BYTES,
330            );
331            space_args.never_move_objects = true;
332        }
333
334        // validate features
335        if super::BLOCK_ONLY {
336            assert!(
337                space_args.never_move_objects,
338                "Block-only immix must not move objects"
339            );
340        }
341        assert!(
342            Block::LINES / 2 <= u8::MAX as usize - 2,
343            "Number of lines in a block should not exceed BlockState::MARK_MARKED"
344        );
345
346        #[cfg(feature = "vo_bit")]
347        vo_bit::helper::validate_config::<VM>();
348        let vm_map = args.vm_map;
349        let scheduler = args.scheduler.clone();
350        let common =
351            CommonSpace::new(args.into_policy_args(true, false, Self::side_metadata_specs()));
352        let space_index = common.descriptor.get_index();
353        ImmixSpace {
354            pr: if common.vmrequest.is_discontiguous() {
355                BlockPageResource::new_discontiguous(
356                    Block::LOG_PAGES,
357                    vm_map,
358                    scheduler.num_workers(),
359                )
360            } else {
361                BlockPageResource::new_contiguous(
362                    Block::LOG_PAGES,
363                    common.start,
364                    common.extent,
365                    vm_map,
366                    scheduler.num_workers(),
367                )
368            },
369            common,
370            chunk_map: ChunkMap::new(space_index),
371            line_mark_state: AtomicU8::new(Line::RESET_MARK_STATE),
372            line_unavail_state: AtomicU8::new(Line::RESET_MARK_STATE),
373            lines_consumed: AtomicUsize::new(0),
374            reusable_blocks: ReusableBlockPool::new(scheduler.num_workers()),
375            defrag: Defrag::default(),
376            // Set to the correct mark state when inititialized. We cannot rely on prepare to set it (prepare may get skipped in nursery GCs).
377            mark_state: Self::MARKED_STATE,
378            scheduler: scheduler.clone(),
379            space_args,
380        }
381    }
382
383    /// Flush the thread-local queues in BlockPageResource
384    pub fn flush_page_resource(&self) {
385        self.reusable_blocks.flush_all();
386        #[cfg(target_pointer_width = "64")]
387        self.pr.flush_all()
388    }
389
390    /// Get the number of defrag headroom pages.
391    pub fn defrag_headroom_pages(&self) -> usize {
392        self.defrag.defrag_headroom_pages(self)
393    }
394
395    /// Check if current GC is a defrag GC.
396    pub fn in_defrag(&self) -> bool {
397        self.defrag.in_defrag()
398    }
399
400    /// check if the current GC should do defragmentation.
401    pub fn decide_whether_to_defrag(
402        &self,
403        emergency_collection: bool,
404        collect_whole_heap: bool,
405        collection_attempts: usize,
406        user_triggered_collection: bool,
407        full_heap_system_gc: bool,
408    ) -> bool {
409        self.defrag.decide_whether_to_defrag(
410            self.is_defrag_enabled(),
411            emergency_collection,
412            collect_whole_heap,
413            collection_attempts,
414            user_triggered_collection,
415            self.reusable_blocks.len() == 0,
416            full_heap_system_gc,
417            *self.common.options.immix_always_defrag,
418        );
419        self.defrag.in_defrag()
420    }
421
422    /// Get work packet scheduler
423    fn scheduler(&self) -> &GCWorkScheduler<VM> {
424        &self.scheduler
425    }
426
427    pub(crate) fn prepare(
428        &mut self,
429        major_gc: bool,
430        plan_stats: Option<StatsForDefrag>,
431        unlog_bits_op: UnlogBitsOperation,
432    ) {
433        if major_gc {
434            // Update mark_state
435            if VM::VMObjectModel::LOCAL_MARK_BIT_SPEC.is_on_side() {
436                self.mark_state = Self::MARKED_STATE;
437            } else {
438                // For header metadata, we use cyclic mark bits.
439                unimplemented!("cyclic mark bits is not supported at the moment");
440            }
441
442            // Prepare defrag info
443            if self.is_defrag_enabled() {
444                self.defrag.prepare(self, plan_stats.unwrap());
445            }
446
447            // Prepare each block for GC
448            let threshold = self.defrag.defrag_spill_threshold.load(Ordering::Acquire);
449            // # Safety: ImmixSpace reference is always valid within this collection cycle.
450            let space = unsafe { &*(self as *const Self) };
451            let work_packets = self.chunk_map.generate_tasks(|chunk| {
452                Box::new(PrepareBlockState {
453                    space,
454                    chunk,
455                    defrag_threshold: if space.in_defrag() {
456                        Some(threshold)
457                    } else {
458                        None
459                    },
460                    unlog_bits_op,
461                })
462            });
463            self.scheduler().work_buckets[WorkBucketStage::Prepare].bulk_add(work_packets);
464
465            if !super::BLOCK_ONLY {
466                self.line_mark_state.fetch_add(1, Ordering::AcqRel);
467                if self.line_mark_state.load(Ordering::Acquire) > Line::MAX_MARK_STATE {
468                    self.line_mark_state
469                        .store(Line::RESET_MARK_STATE, Ordering::Release);
470                }
471            }
472        }
473
474        #[cfg(feature = "vo_bit")]
475        if vo_bit::helper::need_to_clear_vo_bits_before_tracing::<VM>() {
476            let maybe_scope = if major_gc {
477                // If it is major GC, we always clear all VO bits because we are doing full-heap
478                // tracing.
479                Some(VOBitsClearingScope::FullGC)
480            } else if self.space_args.mixed_age {
481                // StickyImmix nursery GC.
482                // Some lines (or blocks) contain only young objects,
483                // while other lines (or blocks) contain only old objects.
484                if super::BLOCK_ONLY {
485                    // Block only.  Young objects are only allocated into fully empty blocks.
486                    // Only clear unmarked blocks.
487                    Some(VOBitsClearingScope::BlockOnly)
488                } else {
489                    // Young objects are allocated into empty lines.
490                    // Only clear unmarked lines.
491                    let line_mark_state = self.line_mark_state.load(Ordering::SeqCst);
492                    Some(VOBitsClearingScope::Line {
493                        state: line_mark_state,
494                    })
495                }
496            } else {
497                // GenImmix nursery GC.  We do nothing to the ImmixSpace because the nursery is a
498                // separate CopySpace.  It'll clear its own VO bits.
499                None
500            };
501
502            if let Some(scope) = maybe_scope {
503                let work_packets = self
504                    .chunk_map
505                    .generate_tasks(|chunk| Box::new(ClearVOBitsAfterPrepare { chunk, scope }));
506                self.scheduler.work_buckets[WorkBucketStage::ClearVOBits].bulk_add(work_packets);
507            }
508        }
509    }
510
511    /// Release for the immix space.
512    pub(crate) fn release(&mut self, major_gc: bool, unlog_bits_op: UnlogBitsOperation) {
513        if major_gc {
514            // Update line_unavail_state for hole searching after this GC.
515            if !super::BLOCK_ONLY {
516                self.line_unavail_state.store(
517                    self.line_mark_state.load(Ordering::Acquire),
518                    Ordering::Release,
519                );
520            }
521        }
522        // Clear reusable blocks list
523        if !super::BLOCK_ONLY {
524            self.reusable_blocks.reset();
525        }
526        // Sweep chunks and blocks
527        let work_packets = self.generate_sweep_tasks(unlog_bits_op);
528        self.scheduler().work_buckets[WorkBucketStage::Release].bulk_add(work_packets);
529
530        self.lines_consumed.store(0, Ordering::Relaxed);
531    }
532
533    /// This is called when a GC finished.
534    /// Return whether this GC was a defrag GC, as a plan may want to know this.
535    pub fn end_of_gc(&mut self) -> bool {
536        let did_defrag = self.defrag.in_defrag();
537        if self.is_defrag_enabled() {
538            self.defrag.reset_in_defrag();
539        }
540        did_defrag
541    }
542
543    /// Generate chunk sweep tasks
544    fn generate_sweep_tasks(&self, unlog_bits_op: UnlogBitsOperation) -> Vec<Box<dyn GCWork<VM>>> {
545        self.defrag.mark_histograms.lock().clear();
546        // # Safety: ImmixSpace reference is always valid within this collection cycle.
547        let space = unsafe { &*(self as *const Self) };
548        let epilogue = Arc::new(FlushPageResource {
549            space,
550            counter: AtomicUsize::new(0),
551        });
552        let tasks = self.chunk_map.generate_tasks(|chunk| {
553            Box::new(SweepChunk {
554                space,
555                chunk,
556                unlog_bits_op,
557                epilogue: epilogue.clone(),
558            })
559        });
560        epilogue.counter.store(tasks.len(), Ordering::SeqCst);
561        tasks
562    }
563
564    /// Release a block.
565    pub fn release_block(&self, block: Block) {
566        block.deinit();
567        self.pr.release_block(block);
568    }
569
570    /// Allocate a clean block.
571    pub fn get_clean_block(
572        &self,
573        tls: VMThread,
574        copy: bool,
575        alloc_options: AllocationOptions,
576    ) -> Option<Block> {
577        let block_address = self.acquire(tls, Block::PAGES, alloc_options);
578        if block_address.is_zero() {
579            return None;
580        }
581        self.defrag.notify_new_clean_block(copy);
582        let block = Block::from_aligned_address(block_address);
583        block.init(copy);
584        self.chunk_map.set_allocated(block.chunk(), true);
585        self.lines_consumed
586            .fetch_add(Block::LINES, Ordering::SeqCst);
587        Some(block)
588    }
589
590    /// Pop a reusable block from the reusable block list.
591    pub fn get_reusable_block(&self, copy: bool) -> Option<Block> {
592        if super::BLOCK_ONLY {
593            return None;
594        }
595        loop {
596            if let Some(block) = self.reusable_blocks.pop() {
597                // Skip blocks that should be evacuated.
598                if copy && block.is_defrag_source() {
599                    continue;
600                }
601
602                // Get available lines. Do this before block.init which will reset block state.
603                let lines_delta = match block.get_state() {
604                    BlockState::Reusable { unavailable_lines } => {
605                        Block::LINES - unavailable_lines as usize
606                    }
607                    BlockState::Unmarked => Block::LINES,
608                    _ => unreachable!("{:?} {:?}", block, block.get_state()),
609                };
610                self.lines_consumed.fetch_add(lines_delta, Ordering::SeqCst);
611
612                block.init(copy);
613                return Some(block);
614            } else {
615                return None;
616            }
617        }
618    }
619
620    /// Trace and mark objects without evacuation.
621    pub fn trace_object_without_moving(
622        &self,
623        queue: &mut impl ObjectQueue,
624        object: ObjectReference,
625    ) -> ObjectReference {
626        #[cfg(feature = "vo_bit")]
627        vo_bit::helper::on_trace_object::<VM>(object);
628
629        if self.attempt_mark(object, self.mark_state) {
630            // Mark block and lines
631            if !super::BLOCK_ONLY {
632                if !super::MARK_LINE_AT_SCAN_TIME {
633                    self.mark_lines(object);
634                }
635            } else {
636                Block::containing(object).set_state(BlockState::Marked);
637            }
638
639            #[cfg(feature = "vo_bit")]
640            vo_bit::helper::on_object_marked::<VM>(object);
641
642            // Visit node
643            queue.enqueue(object);
644            self.unlog_object_if_needed(object);
645            return object;
646        }
647        object
648    }
649
650    /// Trace object and do evacuation if required.
651    #[allow(clippy::assertions_on_constants)]
652    pub fn trace_object_with_opportunistic_copy(
653        &self,
654        queue: &mut impl ObjectQueue,
655        object: ObjectReference,
656        semantics: CopySemantics,
657        worker: &mut GCWorker<VM>,
658        nursery_collection: bool,
659    ) -> ObjectReference {
660        let copy_context = worker.get_copy_context_mut();
661        debug_assert!(!super::BLOCK_ONLY);
662
663        #[cfg(feature = "vo_bit")]
664        vo_bit::helper::on_trace_object::<VM>(object);
665
666        let forwarding_status = object_forwarding::attempt_to_forward::<VM>(object);
667        if object_forwarding::state_is_forwarded_or_being_forwarded(forwarding_status) {
668            // We lost the forwarding race as some other thread has set the forwarding word; wait
669            // until the object has been forwarded by the winner. Note that the object may not
670            // necessarily get forwarded since Immix opportunistically moves objects.
671            #[allow(clippy::let_and_return)]
672            let new_object =
673                object_forwarding::spin_and_get_forwarded_object::<VM>(object, forwarding_status);
674            #[cfg(debug_assertions)]
675            {
676                if new_object == object {
677                    debug_assert!(
678                        self.is_marked(object) || self.defrag.space_exhausted() || self.is_pinned(object),
679                        "Forwarded object is the same as original object {} even though it should have been copied",
680                        object,
681                    );
682                } else {
683                    // new_object != object
684                    debug_assert!(
685                        !Block::containing(new_object).is_defrag_source(),
686                        "Block {:?} containing forwarded object {} should not be a defragmentation source",
687                        Block::containing(new_object),
688                        new_object,
689                    );
690                }
691            }
692            new_object
693        } else if self.is_marked(object) {
694            // We won the forwarding race but the object is already marked so we clear the
695            // forwarding status and return the unmoved object
696            object_forwarding::clear_forwarding_bits::<VM>(object);
697            object
698        } else {
699            // We won the forwarding race; actually forward and copy the object if it is not pinned
700            // and we have sufficient space in our copy allocator
701            let new_object = if self.is_pinned(object)
702                || (!nursery_collection && self.defrag.space_exhausted())
703            {
704                self.attempt_mark(object, self.mark_state);
705                object_forwarding::clear_forwarding_bits::<VM>(object);
706                Block::containing(object).set_state(BlockState::Marked);
707
708                #[cfg(feature = "vo_bit")]
709                vo_bit::helper::on_object_marked::<VM>(object);
710
711                if !super::MARK_LINE_AT_SCAN_TIME {
712                    self.mark_lines(object);
713                }
714
715                self.unlog_object_if_needed(object);
716
717                object
718            } else {
719                // We are forwarding objects. When the copy allocator allocates the block, it should
720                // mark the block. So we do not need to explicitly mark it here.
721
722                object_forwarding::forward_object::<VM>(
723                    object,
724                    semantics,
725                    copy_context,
726                    |new_object| {
727                        // post_copy should have set the unlog bit
728                        // if `unlog_traced_object` is true.
729                        debug_assert!(
730                            !self.common.unlog_traced_object
731                                || VM::VMObjectModel::GLOBAL_LOG_BIT_SPEC
732                                    .is_unlogged::<VM>(new_object, Ordering::Relaxed)
733                        );
734                        #[cfg(feature = "vo_bit")]
735                        vo_bit::helper::on_object_forwarded::<VM>(new_object);
736                    },
737                )
738            };
739            debug_assert_eq!(
740                Block::containing(new_object).get_state(),
741                BlockState::Marked
742            );
743
744            queue.enqueue(new_object);
745            debug_assert!(new_object.is_live());
746            new_object
747        }
748    }
749
750    fn unlog_object_if_needed(&self, object: ObjectReference) {
751        if self.common.unlog_traced_object {
752            // Make sure the side metadata for the line can fit into one byte. For smaller line size, we should
753            // use `mark_as_unlogged` instead to mark the bit.
754            const_assert!(
755                Line::BYTES
756                    >= (1
757                        << (crate::util::constants::LOG_BITS_IN_BYTE
758                            + crate::util::constants::LOG_MIN_OBJECT_SIZE))
759            );
760            const_assert_eq!(
761                crate::vm::object_model::specs::VMGlobalLogBitSpec::LOG_NUM_BITS,
762                0
763            ); // We should put this to the addition, but type casting is not allowed in constant assertions.
764
765            // Every immix line is 256 bytes, which is mapped to 4 bytes in the side metadata.
766            // If we have one object in the line that is mature, we can assume all the objects in the line are mature objects.
767            // So we can just mark the byte.
768            VM::VMObjectModel::GLOBAL_LOG_BIT_SPEC
769                .mark_byte_as_unlogged::<VM>(object, Ordering::Relaxed);
770        }
771    }
772
773    /// Mark all the lines that the given object spans.
774    #[allow(clippy::assertions_on_constants)]
775    pub fn mark_lines(&self, object: ObjectReference) {
776        debug_assert!(!super::BLOCK_ONLY);
777        Line::mark_lines_for_object::<VM>(object, self.line_mark_state.load(Ordering::Acquire));
778    }
779
780    /// Atomically mark an object.
781    fn attempt_mark(&self, object: ObjectReference, mark_state: u8) -> bool {
782        loop {
783            let old_value = VM::VMObjectModel::LOCAL_MARK_BIT_SPEC.load_atomic::<VM, u8>(
784                object,
785                None,
786                Ordering::SeqCst,
787            );
788            if old_value == mark_state {
789                return false;
790            }
791
792            if VM::VMObjectModel::LOCAL_MARK_BIT_SPEC
793                .compare_exchange_metadata::<VM, u8>(
794                    object,
795                    old_value,
796                    mark_state,
797                    None,
798                    Ordering::SeqCst,
799                    Ordering::SeqCst,
800                )
801                .is_ok()
802            {
803                break;
804            }
805        }
806        true
807    }
808
809    /// Check if an object is marked.
810    fn is_marked_with(&self, object: ObjectReference, mark_state: u8) -> bool {
811        let old_value = VM::VMObjectModel::LOCAL_MARK_BIT_SPEC.load_atomic::<VM, u8>(
812            object,
813            None,
814            Ordering::SeqCst,
815        );
816        old_value == mark_state
817    }
818
819    pub(crate) fn is_marked(&self, object: ObjectReference) -> bool {
820        self.is_marked_with(object, self.mark_state)
821    }
822
823    /// Check if an object is pinned.
824    fn is_pinned(&self, _object: ObjectReference) -> bool {
825        #[cfg(feature = "object_pinning")]
826        return self.is_object_pinned(_object);
827
828        #[cfg(not(feature = "object_pinning"))]
829        false
830    }
831
832    /// Hole searching.
833    ///
834    /// Linearly scan lines in a block to search for the next
835    /// hole, starting from the given line. If we find available lines,
836    /// return a tuple of the start line and the end line (non-inclusive).
837    ///
838    /// Returns None if the search could not find any more holes.
839    #[allow(clippy::assertions_on_constants)]
840    pub fn get_next_available_lines(&self, search_start: Line) -> Option<(Line, Line)> {
841        debug_assert!(!super::BLOCK_ONLY);
842        let unavail_state = self.line_unavail_state.load(Ordering::Acquire);
843        let current_state = self.line_mark_state.load(Ordering::Acquire);
844        let block = search_start.block();
845        let mark_data = block.line_mark_table();
846        let start_cursor = search_start.get_index_within_block();
847        let mut cursor = start_cursor;
848        // Find start
849        while cursor < mark_data.len() {
850            let mark = mark_data.get(cursor);
851            if mark != unavail_state && mark != current_state {
852                break;
853            }
854            cursor += 1;
855        }
856        if cursor == mark_data.len() {
857            return None;
858        }
859        let start = search_start.next_nth(cursor - start_cursor);
860        // Find limit
861        while cursor < mark_data.len() {
862            let mark = mark_data.get(cursor);
863            if mark == unavail_state || mark == current_state {
864                break;
865            }
866            cursor += 1;
867        }
868        let end = search_start.next_nth(cursor - start_cursor);
869        debug_assert!(RegionIterator::<Line>::new(start, end)
870            .all(|line| !line.is_marked(unavail_state) && !line.is_marked(current_state)));
871        Some((start, end))
872    }
873
874    pub fn is_last_gc_exhaustive(&self, did_defrag_for_last_gc: bool) -> bool {
875        if self.is_defrag_enabled() {
876            did_defrag_for_last_gc
877        } else {
878            // If defrag is disabled, every GC is exhaustive.
879            true
880        }
881    }
882
883    pub(crate) fn get_pages_allocated(&self) -> usize {
884        self.lines_consumed.load(Ordering::SeqCst) >> (LOG_BYTES_IN_PAGE - Line::LOG_BYTES as u8)
885    }
886
887    /// Post copy routine for Immix copy contexts
888    fn post_copy(&self, object: ObjectReference, _bytes: usize) {
889        // Mark the object
890        VM::VMObjectModel::LOCAL_MARK_BIT_SPEC.store_atomic::<VM, u8>(
891            object,
892            self.mark_state,
893            None,
894            Ordering::SeqCst,
895        );
896        // Mark the line
897        if !super::MARK_LINE_AT_SCAN_TIME {
898            self.mark_lines(object);
899        }
900        if self.common.unlog_traced_object {
901            VM::VMObjectModel::GLOBAL_LOG_BIT_SPEC
902                .mark_byte_as_unlogged::<VM>(object, Ordering::Relaxed);
903        }
904    }
905
906    pub(crate) fn prefer_copy_on_nursery_gc(&self) -> bool {
907        self.is_nursery_copy_enabled()
908    }
909
910    pub(crate) fn is_nursery_copy_enabled(&self) -> bool {
911        !self.space_args.never_move_objects && !cfg!(feature = "sticky_immix_non_moving_nursery")
912    }
913
914    pub(crate) fn is_defrag_enabled(&self) -> bool {
915        !self.space_args.never_move_objects
916    }
917}
918
919/// A work packet to prepare each block for a major GC.
920/// Performs the action on a range of chunks.
921pub struct PrepareBlockState<VM: VMBinding> {
922    #[allow(dead_code)]
923    pub space: &'static ImmixSpace<VM>,
924    pub chunk: Chunk,
925    pub defrag_threshold: Option<usize>,
926    pub unlog_bits_op: UnlogBitsOperation,
927}
928
929impl<VM: VMBinding> PrepareBlockState<VM> {
930    /// Clear object mark table
931    fn reset_object_mark(&self) {
932        // NOTE: We reset the mark bits because cyclic mark bit is currently not supported, yet.
933        // See `ImmixSpace::prepare`.
934        if let MetadataSpec::OnSide(side) = *VM::VMObjectModel::LOCAL_MARK_BIT_SPEC {
935            side.bzero_metadata(self.chunk.start(), Chunk::BYTES);
936        }
937    }
938}
939
940impl<VM: VMBinding> GCWork<VM> for PrepareBlockState<VM> {
941    fn do_work(&mut self, _worker: &mut GCWorker<VM>, mmtk: &'static MMTK<VM>) {
942        // Clear object mark table for this chunk
943        self.reset_object_mark();
944        // Iterate over all blocks in this chunk
945        for block in self.chunk.iter_region::<Block>() {
946            let state = block.get_state();
947            // Skip unallocated blocks.
948            if state == BlockState::Unallocated {
949                continue;
950            }
951            // Check if this block needs to be defragmented.
952            let is_defrag_source = if !self.space.is_defrag_enabled() {
953                // Do not set any block as defrag source if defrag is disabled.
954                false
955            } else if *mmtk.options.immix_defrag_every_block {
956                // Set every block as defrag source if so desired.
957                true
958            } else if let Some(defrag_threshold) = self.defrag_threshold {
959                // This GC is a defrag GC.
960                block.get_holes() > defrag_threshold
961            } else {
962                // Not a defrag GC.
963                false
964            };
965            block.set_as_defrag_source(is_defrag_source);
966            // Clear block mark data.
967            block.set_state(BlockState::Unmarked);
968            debug_assert!(!block.get_state().is_reusable());
969            debug_assert_ne!(block.get_state(), BlockState::Marked);
970        }
971
972        self.unlog_bits_op
973            .execute::<VM>(self.chunk.start(), Chunk::BYTES);
974    }
975}
976
977/// Chunk sweeping work packet.
978struct SweepChunk<VM: VMBinding> {
979    space: &'static ImmixSpace<VM>,
980    chunk: Chunk,
981    unlog_bits_op: UnlogBitsOperation,
982    /// A destructor invoked when all `SweepChunk` packets are finished.
983    epilogue: Arc<FlushPageResource<VM>>,
984}
985
986impl<VM: VMBinding> GCWork<VM> for SweepChunk<VM> {
987    fn do_work(&mut self, _worker: &mut GCWorker<VM>, mmtk: &'static MMTK<VM>) {
988        assert!(self.space.chunk_map.get(self.chunk).unwrap().is_allocated());
989
990        let mut histogram = self.space.defrag.new_histogram();
991        let line_mark_state = if super::BLOCK_ONLY {
992            None
993        } else {
994            Some(self.space.line_mark_state.load(Ordering::Acquire))
995        };
996        // Hints for clearing side forwarding bits.
997        let is_moving_gc = mmtk.get_plan().current_gc_may_move_object();
998        let is_defrag_gc = self.space.defrag.in_defrag();
999        // number of allocated blocks.
1000        let mut allocated_blocks = 0;
1001        // Iterate over all allocated blocks in this chunk.
1002        for block in self
1003            .chunk
1004            .iter_region::<Block>()
1005            .filter(|block| block.get_state() != BlockState::Unallocated)
1006        {
1007            // Clear side forwarding bits.
1008            // In the beginning of the next GC, no side forwarding bits shall be set.
1009            // In this way, we can omit clearing forwarding bits when copying object.
1010            // See `GCWorkerCopyContext::post_copy`.
1011            // Note, `block.sweep()` overwrites `DEFRAG_STATE_TABLE` with the number of holes,
1012            // but we need it to know if a block is a defrag source.
1013            // We clear forwarding bits before `block.sweep()`.
1014            if let MetadataSpec::OnSide(side) = *VM::VMObjectModel::LOCAL_FORWARDING_BITS_SPEC {
1015                if is_moving_gc {
1016                    let objects_may_move = if is_defrag_gc {
1017                        // If it is a defrag GC, we only clear forwarding bits for defrag sources.
1018                        block.is_defrag_source()
1019                    } else {
1020                        // Otherwise, it must be a nursery GC of StickyImmix with copying nursery.
1021                        // We don't have information about which block contains moved objects,
1022                        // so we have to clear forwarding bits for all blocks.
1023                        true
1024                    };
1025                    if objects_may_move {
1026                        side.bzero_metadata(block.start(), Block::BYTES);
1027                    }
1028                }
1029            }
1030
1031            if !block.sweep(self.space, &mut histogram, line_mark_state) {
1032                // Block is live. Increment the allocated block count.
1033                allocated_blocks += 1;
1034            }
1035        }
1036        probe!(mmtk, sweep_chunk, allocated_blocks);
1037        // Set this chunk as free if there is not live blocks.
1038        if allocated_blocks == 0 {
1039            self.space.chunk_map.set_allocated(self.chunk, false)
1040        }
1041        self.space.defrag.add_completed_mark_histogram(histogram);
1042
1043        self.unlog_bits_op
1044            .execute::<VM>(self.chunk.start(), Chunk::BYTES);
1045
1046        self.epilogue.finish_one_work_packet();
1047    }
1048}
1049
1050/// Count number of remaining work pacets, and flush page resource if all packets are finished.
1051struct FlushPageResource<VM: VMBinding> {
1052    space: &'static ImmixSpace<VM>,
1053    counter: AtomicUsize,
1054}
1055
1056impl<VM: VMBinding> FlushPageResource<VM> {
1057    /// Called after a related work packet is finished.
1058    fn finish_one_work_packet(&self) {
1059        if 1 == self.counter.fetch_sub(1, Ordering::SeqCst) {
1060            // We've finished releasing all the dead blocks to the BlockPageResource's thread-local queues.
1061            // Now flush the BlockPageResource.
1062            self.space.flush_page_resource()
1063        }
1064    }
1065}
1066
1067impl<VM: VMBinding> Drop for FlushPageResource<VM> {
1068    fn drop(&mut self) {
1069        epilogue::debug_assert_counter_zero(&self.counter, "FlushPageResource::counter");
1070    }
1071}
1072
1073use crate::policy::copy_context::PolicyCopyContext;
1074use crate::util::alloc::Allocator;
1075use crate::util::alloc::ImmixAllocator;
1076
1077/// Normal immix copy context. It has one copying Immix allocator.
1078/// Most immix plans use this copy context.
1079pub struct ImmixCopyContext<VM: VMBinding> {
1080    allocator: ImmixAllocator<VM>,
1081}
1082
1083impl<VM: VMBinding> PolicyCopyContext for ImmixCopyContext<VM> {
1084    type VM = VM;
1085
1086    fn prepare(&mut self) {
1087        self.allocator.reset();
1088    }
1089    fn release(&mut self) {
1090        self.allocator.reset();
1091    }
1092    fn alloc_copy(
1093        &mut self,
1094        _original: ObjectReference,
1095        bytes: usize,
1096        align: usize,
1097        offset: usize,
1098    ) -> Address {
1099        self.allocator.alloc(bytes, align, offset)
1100    }
1101    fn post_copy(&mut self, obj: ObjectReference, bytes: usize) {
1102        self.get_space().post_copy(obj, bytes)
1103    }
1104}
1105
1106impl<VM: VMBinding> ImmixCopyContext<VM> {
1107    pub(crate) fn new(
1108        tls: VMWorkerThread,
1109        context: Arc<AllocatorContext<VM>>,
1110        space: &'static ImmixSpace<VM>,
1111    ) -> Self {
1112        ImmixCopyContext {
1113            allocator: ImmixAllocator::new(tls.0, Some(space), context, true),
1114        }
1115    }
1116
1117    fn get_space(&self) -> &ImmixSpace<VM> {
1118        self.allocator.immix_space()
1119    }
1120}
1121
1122/// Hybrid Immix copy context. It includes two different immix allocators. One with `copy = true`
1123/// is used for defrag GCs, and the other is used for other purposes (such as promoting objects from
1124/// nursery to Immix mature space). This is used by generational immix.
1125pub struct ImmixHybridCopyContext<VM: VMBinding> {
1126    copy_allocator: ImmixAllocator<VM>,
1127    defrag_allocator: ImmixAllocator<VM>,
1128}
1129
1130impl<VM: VMBinding> PolicyCopyContext for ImmixHybridCopyContext<VM> {
1131    type VM = VM;
1132
1133    fn prepare(&mut self) {
1134        self.copy_allocator.reset();
1135        self.defrag_allocator.reset();
1136    }
1137    fn release(&mut self) {
1138        self.copy_allocator.reset();
1139        self.defrag_allocator.reset();
1140    }
1141    fn alloc_copy(
1142        &mut self,
1143        _original: ObjectReference,
1144        bytes: usize,
1145        align: usize,
1146        offset: usize,
1147    ) -> Address {
1148        if self.get_space().in_defrag() {
1149            self.defrag_allocator.alloc(bytes, align, offset)
1150        } else {
1151            self.copy_allocator.alloc(bytes, align, offset)
1152        }
1153    }
1154    fn post_copy(&mut self, obj: ObjectReference, bytes: usize) {
1155        self.get_space().post_copy(obj, bytes)
1156    }
1157}
1158
1159impl<VM: VMBinding> ImmixHybridCopyContext<VM> {
1160    pub(crate) fn new(
1161        tls: VMWorkerThread,
1162        context: Arc<AllocatorContext<VM>>,
1163        space: &'static ImmixSpace<VM>,
1164    ) -> Self {
1165        ImmixHybridCopyContext {
1166            copy_allocator: ImmixAllocator::new(tls.0, Some(space), context.clone(), false),
1167            defrag_allocator: ImmixAllocator::new(tls.0, Some(space), context, true),
1168        }
1169    }
1170
1171    fn get_space(&self) -> &ImmixSpace<VM> {
1172        // Both copy allocators should point to the same space.
1173        debug_assert_eq!(
1174            self.defrag_allocator.immix_space().common().descriptor,
1175            self.copy_allocator.immix_space().common().descriptor
1176        );
1177        // Just get the space from either allocator
1178        self.defrag_allocator.immix_space()
1179    }
1180}
1181
1182#[cfg(feature = "vo_bit")]
1183#[derive(Clone, Copy)]
1184enum VOBitsClearingScope {
1185    /// Clear all VO bits in all blocks.
1186    FullGC,
1187    /// Clear unmarked blocks, only.
1188    BlockOnly,
1189    /// Clear unmarked lines, only.  (i.e. lines with line mark state **not** equal to `state`).
1190    Line { state: u8 },
1191}
1192
1193/// A work packet to clear VO bit metadata after Prepare.
1194#[cfg(feature = "vo_bit")]
1195struct ClearVOBitsAfterPrepare {
1196    chunk: Chunk,
1197    scope: VOBitsClearingScope,
1198}
1199
1200#[cfg(feature = "vo_bit")]
1201impl<VM: VMBinding> GCWork<VM> for ClearVOBitsAfterPrepare {
1202    fn do_work(&mut self, _worker: &mut GCWorker<VM>, _mmtk: &'static MMTK<VM>) {
1203        match self.scope {
1204            VOBitsClearingScope::FullGC => {
1205                vo_bit::bzero_vo_bit(self.chunk.start(), Chunk::BYTES);
1206            }
1207            VOBitsClearingScope::BlockOnly => {
1208                self.clear_blocks(None);
1209            }
1210            VOBitsClearingScope::Line { state } => {
1211                self.clear_blocks(Some(state));
1212            }
1213        }
1214    }
1215}
1216
1217#[cfg(feature = "vo_bit")]
1218impl ClearVOBitsAfterPrepare {
1219    fn clear_blocks(&mut self, line_mark_state: Option<u8>) {
1220        for block in self
1221            .chunk
1222            .iter_region::<Block>()
1223            .filter(|block| block.get_state() != BlockState::Unallocated)
1224        {
1225            block.clear_vo_bits_for_unmarked_regions(line_mark_state);
1226        }
1227    }
1228}