mmtk/policy/marksweepspace/malloc_ms/
global.rs

1use super::metadata::*;
2use crate::plan::ObjectQueue;
3use crate::plan::VectorObjectQueue;
4use crate::policy::sft::GCWorkerMutRef;
5use crate::policy::sft::SFT;
6use crate::policy::space::CommonSpace;
7use crate::scheduler::GCWorkScheduler;
8use crate::util::heap::chunk_map::Chunk;
9use crate::util::heap::chunk_map::ChunkMap;
10use crate::util::heap::gc_trigger::GCTrigger;
11use crate::util::heap::space_descriptor::SpaceDescriptor;
12use crate::util::heap::PageResource;
13use crate::util::linear_scan::Region;
14use crate::util::malloc::library::{BYTES_IN_MALLOC_PAGE, LOG_BYTES_IN_MALLOC_PAGE};
15use crate::util::malloc::malloc_ms_util::*;
16use crate::util::metadata::side_metadata;
17use crate::util::metadata::side_metadata::{
18    SideMetadataContext, SideMetadataSanity, SideMetadataSpec,
19};
20use crate::util::metadata::MetadataSpec;
21use crate::util::object_enum::ObjectEnumerator;
22use crate::util::opaque_pointer::*;
23use crate::util::Address;
24use crate::util::ObjectReference;
25use crate::util::{conversions, metadata};
26use crate::vm::VMBinding;
27use crate::vm::{ActivePlan, Collection, ObjectModel};
28use crate::{policy::space::Space, util::heap::layout::vm_layout::BYTES_IN_CHUNK};
29#[cfg(debug_assertions)]
30use std::collections::HashMap;
31use std::marker::PhantomData;
32#[cfg(debug_assertions)]
33use std::sync::atomic::AtomicU32;
34use std::sync::atomic::{AtomicUsize, Ordering};
35use std::sync::Arc;
36use std::sync::Mutex;
37// If true, we will use a hashmap to store all the allocated memory from malloc, and use it
38// to make sure our allocation is correct.
39#[cfg(debug_assertions)]
40const ASSERT_ALLOCATION: bool = false;
41
42/// This space uses malloc to get new memory, and performs mark-sweep for the memory.
43pub struct MallocSpace<VM: VMBinding> {
44    phantom: PhantomData<VM>,
45    active_bytes: AtomicUsize,
46    active_pages: AtomicUsize,
47    metadata: SideMetadataContext,
48    /// Work packet scheduler
49    scheduler: Arc<GCWorkScheduler<VM>>,
50    gc_trigger: Arc<GCTrigger<VM>>,
51    descriptor: SpaceDescriptor,
52    chunk_map: ChunkMap,
53    mmap_metadata_lock: Mutex<()>,
54    // Mapping between allocated address and its size - this is used to check correctness.
55    // Size will be set to zero when the memory is freed.
56    #[cfg(debug_assertions)]
57    active_mem: Mutex<HashMap<Address, usize>>,
58    // The following fields are used for checking correctness of the parallel sweep implementation
59    // as we need to check how many live bytes exist against `active_bytes` when the last sweep
60    // work packet is executed
61    #[cfg(debug_assertions)]
62    pub total_work_packets: AtomicU32,
63    #[cfg(debug_assertions)]
64    pub completed_work_packets: AtomicU32,
65    #[cfg(debug_assertions)]
66    pub work_live_bytes: AtomicUsize,
67}
68
69impl<VM: VMBinding> SFT for MallocSpace<VM> {
70    fn name(&self) -> &'static str {
71        self.get_name()
72    }
73
74    fn is_live(&self, object: ObjectReference) -> bool {
75        is_marked::<VM>(object, Ordering::SeqCst)
76    }
77
78    #[cfg(feature = "object_pinning")]
79    fn pin_object(&self, _object: ObjectReference) -> bool {
80        false
81    }
82
83    #[cfg(feature = "object_pinning")]
84    fn unpin_object(&self, _object: ObjectReference) -> bool {
85        false
86    }
87
88    #[cfg(feature = "object_pinning")]
89    fn is_object_pinned(&self, _object: ObjectReference) -> bool {
90        false
91    }
92
93    fn is_movable(&self) -> bool {
94        false
95    }
96
97    #[cfg(feature = "sanity")]
98    fn is_sane(&self) -> bool {
99        true
100    }
101
102    // For malloc space, we need to further check the VO bit.
103    fn is_in_space(&self, object: ObjectReference) -> bool {
104        self.is_alloced_by_malloc(object)
105    }
106
107    /// For malloc space, we just use the side metadata.
108    #[cfg(feature = "is_mmtk_object")]
109    fn is_mmtk_object(&self, addr: Address) -> Option<ObjectReference> {
110        debug_assert!(!addr.is_zero());
111        // `addr` cannot be mapped by us. It should be mapped by the malloc library.
112        debug_assert!(!addr.is_mapped());
113        self.has_object_alloced_by_malloc(addr)
114    }
115
116    #[cfg(feature = "is_mmtk_object")]
117    fn find_object_from_internal_pointer(
118        &self,
119        ptr: Address,
120        max_search_bytes: usize,
121    ) -> Option<ObjectReference> {
122        crate::util::metadata::vo_bit::find_object_from_internal_pointer::<VM>(
123            ptr,
124            max_search_bytes,
125        )
126    }
127
128    fn initialize_object_metadata(&self, object: ObjectReference) {
129        trace!("initialize_object_metadata for object {}", object);
130        set_vo_bit(object);
131    }
132
133    fn sft_trace_object(
134        &self,
135        queue: &mut VectorObjectQueue,
136        object: ObjectReference,
137        _worker: GCWorkerMutRef,
138    ) -> ObjectReference {
139        self.trace_object(queue, object)
140    }
141}
142
143impl<VM: VMBinding> Space<VM> for MallocSpace<VM> {
144    fn as_space(&self) -> &dyn Space<VM> {
145        self
146    }
147
148    fn as_sft(&self) -> &(dyn SFT + Sync + 'static) {
149        self
150    }
151
152    fn get_page_resource(&self) -> &dyn PageResource<VM> {
153        unreachable!()
154    }
155
156    fn maybe_get_page_resource_mut(&mut self) -> Option<&mut dyn PageResource<VM>> {
157        None
158    }
159
160    fn common(&self) -> &CommonSpace<VM> {
161        unreachable!()
162    }
163
164    fn get_gc_trigger(&self) -> &GCTrigger<VM> {
165        self.gc_trigger.as_ref()
166    }
167
168    fn initialize_sft(&self, _sft_map: &mut dyn crate::policy::sft_map::SFTMap) {
169        // Do nothing - we will set sft when we get new results from malloc
170    }
171
172    fn release_multiple_pages(&mut self, _start: Address) {
173        unreachable!()
174    }
175
176    // We have assertions in a debug build. We allow this pattern for the release build.
177    #[allow(clippy::let_and_return)]
178    fn in_space(&self, object: ObjectReference) -> bool {
179        let ret = self.is_alloced_by_malloc(object);
180
181        #[cfg(debug_assertions)]
182        if ASSERT_ALLOCATION {
183            let addr = object.to_object_start::<VM>();
184            let active_mem = self.active_mem.lock().unwrap();
185            if ret {
186                // The VO bit tells that the object is in space.
187                debug_assert!(
188                    *active_mem.get(&addr).unwrap() != 0,
189                    "active mem check failed for {} (object {}) - was freed",
190                    addr,
191                    object
192                );
193            } else {
194                // The VO bit tells that the object is not in space. It could never be allocated, or have been freed.
195                debug_assert!(
196                    (!active_mem.contains_key(&addr))
197                        || (active_mem.contains_key(&addr) && *active_mem.get(&addr).unwrap() == 0),
198                    "mem check failed for {} (object {}): allocated = {}, size = {:?}",
199                    addr,
200                    object,
201                    active_mem.contains_key(&addr),
202                    if active_mem.contains_key(&addr) {
203                        active_mem.get(&addr)
204                    } else {
205                        None
206                    }
207                );
208            }
209        }
210        ret
211    }
212
213    fn address_in_space(&self, _start: Address) -> bool {
214        unreachable!("We do not know if an address is in malloc space. Use in_space() to check if an object is in malloc space.")
215    }
216
217    fn get_name(&self) -> &'static str {
218        "MallocSpace"
219    }
220
221    fn estimate_side_meta_pages(&self, data_pages: usize) -> usize {
222        self.metadata.calculate_reserved_pages(data_pages)
223    }
224
225    #[allow(clippy::assertions_on_constants)]
226    fn reserved_pages(&self) -> usize {
227        use crate::util::constants::LOG_BYTES_IN_PAGE;
228        // Assume malloc pages are no smaller than 4K pages. Otherwise the substraction below will fail.
229        debug_assert!(LOG_BYTES_IN_MALLOC_PAGE >= LOG_BYTES_IN_PAGE);
230        let data_pages = self.active_pages.load(Ordering::SeqCst)
231            << (LOG_BYTES_IN_MALLOC_PAGE - LOG_BYTES_IN_PAGE);
232        let meta_pages = self.estimate_side_meta_pages(data_pages);
233        data_pages + meta_pages
234    }
235
236    fn verify_side_metadata_sanity(&self, side_metadata_sanity_checker: &mut SideMetadataSanity) {
237        side_metadata_sanity_checker
238            .verify_metadata_context(std::any::type_name::<Self>(), &self.metadata)
239    }
240
241    fn enumerate_objects(&self, _enumerator: &mut dyn ObjectEnumerator) {
242        unimplemented!()
243    }
244
245    fn clear_side_log_bits(&self) {
246        unimplemented!()
247    }
248
249    fn set_side_log_bits(&self) {
250        unimplemented!()
251    }
252}
253
254use crate::scheduler::GCWorker;
255use crate::util::copy::CopySemantics;
256
257impl<VM: VMBinding> crate::policy::gc_work::PolicyTraceObject<VM> for MallocSpace<VM> {
258    fn trace_object<Q: ObjectQueue, const KIND: crate::policy::gc_work::TraceKind>(
259        &self,
260        queue: &mut Q,
261        object: ObjectReference,
262        _copy: Option<CopySemantics>,
263        _worker: &mut GCWorker<VM>,
264    ) -> ObjectReference {
265        self.trace_object(queue, object)
266    }
267
268    fn may_move_objects<const KIND: crate::policy::gc_work::TraceKind>() -> bool {
269        false
270    }
271}
272
273// Actually no max object size.
274#[allow(dead_code)]
275pub const MAX_OBJECT_SIZE: usize = crate::util::constants::MAX_INT;
276
277impl<VM: VMBinding> MallocSpace<VM> {
278    pub fn extend_global_side_metadata_specs(specs: &mut Vec<SideMetadataSpec>) {
279        // MallocSpace needs to use VO bit. If the feature is turned on, the VO bit spec is in the global specs.
280        // Otherwise, we manually add it.
281        if !cfg!(feature = "vo_bit") {
282            specs.push(crate::util::metadata::vo_bit::VO_BIT_SIDE_METADATA_SPEC);
283        }
284    }
285
286    pub fn new(args: crate::policy::space::PlanCreateSpaceArgs<VM>) -> Self {
287        if *args.options.count_live_bytes_in_gc {
288            // The implementation of counting live bytes needs a SpaceDescriptor which we do not have for MallocSpace.
289            // Besides we cannot meaningfully measure the live bytes vs total pages for MallocSpace.
290            panic!("count_live_bytes_in_gc is not supported by MallocSpace");
291        }
292        let descriptor = SpaceDescriptor::create_descriptor();
293        let chunk_map = ChunkMap::new(descriptor.get_index());
294        MallocSpace {
295            phantom: PhantomData,
296            active_bytes: AtomicUsize::new(0),
297            active_pages: AtomicUsize::new(0),
298            metadata: SideMetadataContext {
299                global: args.global_side_metadata_specs.clone(),
300                local: metadata::extract_side_metadata(&[
301                    MetadataSpec::OnSide(ACTIVE_PAGE_METADATA_SPEC),
302                    MetadataSpec::OnSide(OFFSET_MALLOC_METADATA_SPEC),
303                    *VM::VMObjectModel::LOCAL_MARK_BIT_SPEC,
304                ]),
305            },
306            scheduler: args.scheduler.clone(),
307            gc_trigger: args.gc_trigger,
308            descriptor,
309            chunk_map,
310            mmap_metadata_lock: Mutex::new(()),
311            #[cfg(debug_assertions)]
312            active_mem: Mutex::new(HashMap::new()),
313            #[cfg(debug_assertions)]
314            total_work_packets: AtomicU32::new(0),
315            #[cfg(debug_assertions)]
316            completed_work_packets: AtomicU32::new(0),
317            #[cfg(debug_assertions)]
318            work_live_bytes: AtomicUsize::new(0),
319        }
320    }
321
322    /// Set multiple pages, starting from the given address, for the given size, and increase the active page count if we set any page mark in the region.
323    /// This is a thread-safe method, and can be used during mutator phase when mutators may access the same page.
324    /// Performance-wise, this method may impose overhead, as we are doing a compare-exchange for every page in the range.
325    fn set_page_mark(&self, start: Address, size: usize) {
326        // Set first page
327        let mut page = start.align_down(BYTES_IN_MALLOC_PAGE);
328        let mut used_pages = 0;
329
330        // It is important to go to the end of the object, which may span a page boundary
331        while page < start + size {
332            if compare_exchange_set_page_mark(page) {
333                used_pages += 1;
334            }
335
336            page += BYTES_IN_MALLOC_PAGE;
337        }
338
339        if used_pages != 0 {
340            self.active_pages.fetch_add(used_pages, Ordering::SeqCst);
341        }
342    }
343
344    fn set_chunk_mark(&self, start: Address, size: usize) {
345        let mut chunk = start.align_down(BYTES_IN_CHUNK);
346        while chunk < start + size {
347            self.chunk_map
348                .set_allocated(Chunk::from_aligned_address(chunk), true);
349            chunk += BYTES_IN_CHUNK;
350        }
351    }
352
353    /// Unset multiple pages, starting from the given address, for the given size, and decrease the active page count if we unset any page mark in the region
354    ///
355    /// # Safety
356    /// We need to ensure that only one GC thread is accessing the range.
357    unsafe fn unset_page_mark(&self, start: Address, size: usize) {
358        debug_assert!(start.is_aligned_to(BYTES_IN_MALLOC_PAGE));
359        debug_assert!(crate::util::conversions::raw_is_aligned(
360            size,
361            BYTES_IN_MALLOC_PAGE
362        ));
363        let mut page = start;
364        let mut cleared_pages = 0;
365        while page < start + size {
366            if is_page_marked_unsafe(page) {
367                cleared_pages += 1;
368                unset_page_mark_unsafe(page);
369            }
370            page += BYTES_IN_MALLOC_PAGE;
371        }
372
373        if cleared_pages != 0 {
374            self.active_pages.fetch_sub(cleared_pages, Ordering::SeqCst);
375        }
376    }
377
378    pub fn alloc(&self, tls: VMThread, size: usize, align: usize, offset: usize) -> Address {
379        // TODO: Should refactor this and Space.acquire()
380        if self.get_gc_trigger().poll(false, Some(self)) {
381            assert!(VM::VMActivePlan::is_mutator(tls), "Polling in GC worker");
382            VM::VMCollection::block_for_gc(VMMutatorThread(tls));
383            return unsafe { Address::zero() };
384        }
385
386        let (address, is_offset_malloc) = alloc::<VM>(size, align, offset);
387        if !address.is_zero() {
388            let actual_size = get_malloc_usable_size(address, is_offset_malloc);
389
390            if !self.is_meta_space_mapped(address, actual_size) {
391                // Map the metadata space for the associated chunk
392                self.map_metadata_and_update_bound(address, actual_size);
393                // Update SFT
394                assert!(crate::mmtk::SFT_MAP.has_sft_entry(address)); // make sure the address is okay with our SFT map
395                unsafe { crate::mmtk::SFT_MAP.update(self, address, actual_size) };
396            }
397
398            // Set chunk marks for the current object
399            self.set_chunk_mark(address, actual_size);
400
401            // Set page marks for current object
402            self.set_page_mark(address, actual_size);
403            self.active_bytes.fetch_add(actual_size, Ordering::SeqCst);
404
405            if is_offset_malloc {
406                set_offset_malloc_bit(address);
407            }
408
409            #[cfg(debug_assertions)]
410            if ASSERT_ALLOCATION {
411                debug_assert!(actual_size != 0);
412                self.active_mem.lock().unwrap().insert(address, actual_size);
413            }
414        }
415
416        address
417    }
418
419    /// Check if metadata is mapped for a range [addr, addr + size). Metadata is mapped per chunk,
420    /// we will go through all the chunks for [address, address + size), and check if they are mapped.
421    /// If any of the chunks is not mapped, return false. Otherwise return true.
422    fn is_meta_space_mapped(&self, address: Address, size: usize) -> bool {
423        let mut chunk = address.align_down(BYTES_IN_CHUNK);
424        while chunk < address + size {
425            // is the chunk already mapped?
426            if !self.is_meta_space_mapped_for_address(chunk) {
427                return false;
428            }
429            chunk += BYTES_IN_CHUNK;
430        }
431        true
432    }
433
434    /// Check if metadata is mapped for a given address. We check with the chunk map: if the side metadata
435    /// for the chunk map is mapped, and if it is allocated in the chunk map.
436    fn is_meta_space_mapped_for_address(&self, address: Address) -> bool {
437        let is_chunk_map_mapped = |chunk_start: Address| {
438            const CHUNK_MAP_MAX_META_ADDRESS: Address =
439                ChunkMap::ALLOC_TABLE.upper_bound_address_for_contiguous();
440            let meta_address =
441                side_metadata::address_to_meta_address(&ChunkMap::ALLOC_TABLE, chunk_start);
442            if meta_address < CHUNK_MAP_MAX_META_ADDRESS {
443                meta_address.is_mapped()
444            } else {
445                false
446            }
447        };
448        let chunk_start = address.align_down(BYTES_IN_CHUNK);
449        is_chunk_map_mapped(chunk_start)
450            && self
451                .chunk_map
452                .get(Chunk::from_aligned_address(chunk_start))
453                .is_some()
454    }
455
456    pub fn free(&self, addr: Address) {
457        let offset_malloc_bit = is_offset_malloc(addr);
458        let bytes = get_malloc_usable_size(addr, offset_malloc_bit);
459        self.free_internal(addr, bytes, offset_malloc_bit);
460    }
461
462    // XXX optimize: We pass the bytes in to free as otherwise there were multiple
463    // indirect call instructions in the generated assembly
464    fn free_internal(&self, addr: Address, bytes: usize, offset_malloc_bit: bool) {
465        if offset_malloc_bit {
466            trace!("Free memory {:x}", addr);
467            offset_free(addr);
468            unsafe { unset_offset_malloc_bit_unsafe(addr) };
469        } else {
470            let ptr = addr.to_mut_ptr();
471            trace!("Free memory {:?}", ptr);
472            unsafe {
473                free(ptr);
474            }
475        }
476
477        self.active_bytes.fetch_sub(bytes, Ordering::SeqCst);
478
479        #[cfg(debug_assertions)]
480        if ASSERT_ALLOCATION {
481            self.active_mem.lock().unwrap().insert(addr, 0).unwrap();
482        }
483    }
484
485    pub fn trace_object<Q: ObjectQueue>(
486        &self,
487        queue: &mut Q,
488        object: ObjectReference,
489    ) -> ObjectReference {
490        assert!(
491            self.in_space(object),
492            "Cannot mark an object {} that was not alloced by malloc.",
493            object,
494        );
495
496        if !is_marked::<VM>(object, Ordering::Relaxed) {
497            set_mark_bit::<VM>(object, Ordering::SeqCst);
498            queue.enqueue(object);
499        }
500
501        object
502    }
503
504    fn map_metadata_and_update_bound(&self, addr: Address, size: usize) {
505        // Acquire the lock before
506        let _lock = self.mmap_metadata_lock.lock().unwrap();
507
508        // Mmap metadata for each chunk
509        let map_metadata_space_for_chunk = |start: Address| {
510            debug_assert!(start.is_aligned_to(BYTES_IN_CHUNK));
511            // Attempt to map the local metadata for the policy.
512            // Note that this might fail. For example, we have marked a chunk as active but later we freed all
513            // the objects in it, and unset its chunk bit. However, we do not free its metadata. So for the chunk,
514            // its chunk bit is mapped, but not marked, and all its local metadata is also mapped.
515            let mmap_metadata_result =
516                self.metadata
517                    .try_map_metadata_space(start, BYTES_IN_CHUNK, self.get_name());
518            debug_assert!(
519                mmap_metadata_result.is_ok(),
520                "mmap sidemetadata failed for chunk_start ({})",
521                start
522            );
523            // Set the chunk mark at the end. So if we have chunk mark set, we know we have mapped side metadata
524            // for the chunk.
525            trace!("set chunk mark bit for {}", start);
526            self.chunk_map
527                .set_allocated(Chunk::from_aligned_address(start), true);
528        };
529
530        // Go through each chunk, and map for them.
531        let mut chunk = conversions::chunk_align_down(addr);
532        while chunk < addr + size {
533            map_metadata_space_for_chunk(chunk);
534            chunk += BYTES_IN_CHUNK;
535        }
536    }
537
538    /// Check if a given object was allocated by malloc
539    pub fn is_alloced_by_malloc(&self, object: ObjectReference) -> bool {
540        self.is_meta_space_mapped_for_address(object.to_raw_address())
541            && crate::util::metadata::vo_bit::is_vo_bit_set(object)
542    }
543
544    /// Check if there is an object allocated by malloc at the address.
545    ///
546    /// This function doesn't check if `addr` is aligned.
547    /// If not, it will try to load the VO bit for the address rounded down to the metadata's granularity.
548    #[cfg(feature = "is_mmtk_object")]
549    pub fn has_object_alloced_by_malloc(&self, addr: Address) -> Option<ObjectReference> {
550        if !self.is_meta_space_mapped_for_address(addr) {
551            return None;
552        }
553        crate::util::metadata::vo_bit::is_vo_bit_set_for_addr(addr)
554    }
555
556    pub fn prepare(&mut self, _full_heap: bool) {}
557
558    pub fn release(&mut self) {
559        use crate::scheduler::WorkBucketStage;
560        let space = unsafe { &*(self as *const Self) };
561        let work_packets = self.chunk_map.generate_tasks(|chunk| {
562            Box::new(MSSweepChunk {
563                ms: space,
564                chunk: chunk.start(),
565            })
566        });
567
568        debug!("Generated {} sweep work packets", work_packets.len());
569        #[cfg(debug_assertions)]
570        {
571            self.total_work_packets
572                .store(work_packets.len() as u32, Ordering::SeqCst);
573            self.completed_work_packets.store(0, Ordering::SeqCst);
574            self.work_live_bytes.store(0, Ordering::SeqCst);
575        }
576
577        self.scheduler.work_buckets[WorkBucketStage::Release].bulk_add(work_packets);
578    }
579
580    pub fn end_of_gc(&mut self) {}
581
582    pub fn sweep_chunk(&self, chunk_start: Address) {
583        // Call the relevant sweep function depending on the location of the mark bits
584        match *VM::VMObjectModel::LOCAL_MARK_BIT_SPEC {
585            MetadataSpec::OnSide(local_mark_bit_side_spec) => {
586                self.sweep_chunk_mark_on_side(chunk_start, local_mark_bit_side_spec);
587            }
588            _ => {
589                self.sweep_chunk_mark_in_header(chunk_start);
590            }
591        }
592    }
593
594    /// Given an object in MallocSpace, return its malloc address, whether it is an offset malloc, and malloc size
595    fn get_malloc_addr_size(object: ObjectReference) -> (Address, bool, usize) {
596        let obj_start = object.to_object_start::<VM>();
597        let offset_malloc_bit = is_offset_malloc(obj_start);
598        let bytes = get_malloc_usable_size(obj_start, offset_malloc_bit);
599        (obj_start, offset_malloc_bit, bytes)
600    }
601
602    /// Clean up for an empty chunk
603    fn clean_up_empty_chunk(&self, chunk_start: Address) {
604        // Clear the chunk map
605        self.chunk_map
606            .set_allocated(Chunk::from_aligned_address(chunk_start), false);
607        // Clear the SFT entry
608        unsafe { crate::mmtk::SFT_MAP.clear(chunk_start) };
609        // Clear the page marks - we are the only GC thread that is accessing this chunk
610        unsafe { self.unset_page_mark(chunk_start, BYTES_IN_CHUNK) };
611    }
612
613    /// Sweep an object if it is dead, and unset page marks for empty pages before this object.
614    /// Return true if the object is swept.
615    fn sweep_object(&self, object: ObjectReference, empty_page_start: &mut Address) -> bool {
616        let (obj_start, offset_malloc, bytes) = Self::get_malloc_addr_size(object);
617
618        // We are the only thread that is dealing with the object. We can use non-atomic methods for the metadata.
619        if !unsafe { is_marked_unsafe::<VM>(object) } {
620            // Dead object
621            trace!("Object {} has been allocated but not marked", object);
622
623            // Free object
624            self.free_internal(obj_start, bytes, offset_malloc);
625            trace!("free object {}", object);
626            unsafe { unset_vo_bit_unsafe(object) };
627
628            true
629        } else {
630            // Live object that we have marked
631
632            // Unset marks for free pages and update last_object_end
633            if !empty_page_start.is_zero() {
634                // unset marks for pages since last object
635                let current_page = object
636                    .to_object_start::<VM>()
637                    .align_down(BYTES_IN_MALLOC_PAGE);
638                if current_page > *empty_page_start {
639                    // we are the only GC thread that is accessing this chunk
640                    unsafe {
641                        self.unset_page_mark(*empty_page_start, current_page - *empty_page_start)
642                    };
643                }
644            }
645
646            // Update last_object_end
647            *empty_page_start = (obj_start + bytes).align_up(BYTES_IN_MALLOC_PAGE);
648
649            false
650        }
651    }
652
653    /// Used when each chunk is done. Only called in debug build.
654    #[cfg(debug_assertions)]
655    fn debug_sweep_chunk_done(&self, live_bytes_in_the_chunk: usize) {
656        debug!(
657            "Used bytes after releasing: {}",
658            self.active_bytes.load(Ordering::SeqCst)
659        );
660
661        let completed_packets = self.completed_work_packets.fetch_add(1, Ordering::SeqCst) + 1;
662        self.work_live_bytes
663            .fetch_add(live_bytes_in_the_chunk, Ordering::SeqCst);
664
665        if completed_packets == self.total_work_packets.load(Ordering::Relaxed) {
666            trace!(
667                "work_live_bytes = {}, live_bytes = {}, active_bytes = {}",
668                self.work_live_bytes.load(Ordering::Relaxed),
669                live_bytes_in_the_chunk,
670                self.active_bytes.load(Ordering::Relaxed)
671            );
672            debug_assert_eq!(
673                self.work_live_bytes.load(Ordering::Relaxed),
674                self.active_bytes.load(Ordering::Relaxed)
675            );
676        }
677    }
678
679    /// This function is called when the mark bits sit on the side metadata.
680    /// This has been optimized with the use of bulk loading and bulk zeroing of
681    /// metadata.
682    ///
683    /// This function uses non-atomic accesses to side metadata (although these
684    /// non-atomic accesses should not have race conditions associated with them)
685    /// as well as calls libc functions (`malloc_usable_size()`, `free()`)
686    fn sweep_chunk_mark_on_side(&self, chunk_start: Address, mark_bit_spec: SideMetadataSpec) {
687        // We can do xor on bulk for mark bits and valid object bits. If the result is zero, that means
688        // the objects in it are all alive (both valid object bit and mark bit is set), and we do not
689        // need to do anything for the region. Otherwise, we will sweep each single object in the region.
690        // Note: Enabling this would result in inaccurate page accounting. We disable this by default, and
691        // we will sweep object one by one.
692        const BULK_XOR_ON_MARK_BITS: bool = false;
693
694        if BULK_XOR_ON_MARK_BITS {
695            #[cfg(debug_assertions)]
696            let mut live_bytes = 0;
697
698            debug!("Check active chunk {:?}", chunk_start);
699            let mut address = chunk_start;
700            let chunk_end = chunk_start + BYTES_IN_CHUNK;
701
702            debug_assert!(
703                crate::util::metadata::vo_bit::VO_BIT_SIDE_METADATA_SPEC.log_bytes_in_region
704                    == mark_bit_spec.log_bytes_in_region,
705                "VO-bit and mark-bit metadata have different minimum object sizes!"
706            );
707
708            // For bulk xor'ing 128-bit vectors on architectures with vector instructions
709            // Each bit represents an object of LOG_MIN_OBJ_SIZE size
710            let bulk_load_size: usize = 128
711                * (1 << crate::util::metadata::vo_bit::VO_BIT_SIDE_METADATA_SPEC
712                    .log_bytes_in_region);
713
714            // The start of a possibly empty page. This will be updated during the sweeping, and always points to the next page of last live objects.
715            let mut empty_page_start = Address::ZERO;
716
717            // Scan the chunk by every 'bulk_load_size' region.
718            while address < chunk_end {
719                let alloc_128: u128 = unsafe {
720                    load128(
721                        &crate::util::metadata::vo_bit::VO_BIT_SIDE_METADATA_SPEC,
722                        address,
723                    )
724                };
725                let mark_128: u128 = unsafe { load128(&mark_bit_spec, address) };
726
727                // Check if there are dead objects in the bulk loaded region
728                if alloc_128 ^ mark_128 != 0 {
729                    let end = address + bulk_load_size;
730
731                    // We will do non atomic load on the VO bit, as this is the only thread that access the VO bit for a chunk.
732                    // Linear scan through the bulk load region.
733                    let bulk_load_scan = crate::util::linear_scan::ObjectIterator::<
734                        VM,
735                        MallocObjectSize<VM>,
736                        false,
737                    >::new(address, end);
738                    for object in bulk_load_scan {
739                        self.sweep_object(object, &mut empty_page_start);
740                    }
741                } else {
742                    // TODO we aren't actually accounting for the case where an object is alive and spans
743                    // a page boundary as we don't know what the object sizes are/what is alive in the bulk region
744                    if alloc_128 != 0 {
745                        empty_page_start = address + bulk_load_size;
746                    }
747                }
748
749                // We have processed this bulk load memory. Step to the next.
750                address += bulk_load_size;
751                debug_assert!(address.is_aligned_to(bulk_load_size));
752            }
753
754            // Linear scan through the chunk, and add up all the live object sizes.
755            // We have to do this as a separate pass, as in the above pass, we did not go through all the live objects
756            #[cfg(debug_assertions)]
757            {
758                let chunk_linear_scan = crate::util::linear_scan::ObjectIterator::<
759                    VM,
760                    MallocObjectSize<VM>,
761                    false,
762                >::new(chunk_start, chunk_end);
763                for object in chunk_linear_scan {
764                    let (obj_start, _, bytes) = Self::get_malloc_addr_size(object);
765
766                    if ASSERT_ALLOCATION {
767                        debug_assert!(
768                            self.active_mem.lock().unwrap().contains_key(&obj_start),
769                            "Address {} with VO bit is not in active_mem",
770                            obj_start
771                        );
772                        debug_assert_eq!(
773                            self.active_mem.lock().unwrap().get(&obj_start),
774                            Some(&bytes),
775                            "Address {} size in active_mem does not match the size from malloc_usable_size",
776                            obj_start
777                        );
778                    }
779
780                    debug_assert!(
781                        unsafe { is_marked_unsafe::<VM>(object) },
782                        "Dead object = {} found after sweep",
783                        object
784                    );
785
786                    live_bytes += bytes;
787                }
788            }
789
790            // Clear all the mark bits
791            mark_bit_spec.bzero_metadata(chunk_start, BYTES_IN_CHUNK);
792
793            // If we never updated empty_page_start, the entire chunk is empty.
794            if empty_page_start.is_zero() {
795                self.clean_up_empty_chunk(chunk_start);
796            }
797
798            #[cfg(debug_assertions)]
799            self.debug_sweep_chunk_done(live_bytes);
800        } else {
801            self.sweep_each_object_in_chunk(chunk_start);
802        }
803    }
804
805    /// This sweep function is called when the mark bit sits in the object header
806    ///
807    /// This function uses non-atomic accesses to side metadata (although these
808    /// non-atomic accesses should not have race conditions associated with them)
809    /// as well as calls libc functions (`malloc_usable_size()`, `free()`)
810    fn sweep_chunk_mark_in_header(&self, chunk_start: Address) {
811        self.sweep_each_object_in_chunk(chunk_start)
812    }
813
814    fn sweep_each_object_in_chunk(&self, chunk_start: Address) {
815        #[cfg(debug_assertions)]
816        let mut live_bytes = 0;
817
818        debug!("Check active chunk {:?}", chunk_start);
819
820        // The start of a possibly empty page. This will be updated during the sweeping, and always points to the next page of last live objects.
821        let mut empty_page_start = Address::ZERO;
822
823        let chunk_linear_scan = crate::util::linear_scan::ObjectIterator::<
824            VM,
825            MallocObjectSize<VM>,
826            false,
827        >::new(chunk_start, chunk_start + BYTES_IN_CHUNK);
828
829        for object in chunk_linear_scan {
830            #[cfg(debug_assertions)]
831            if ASSERT_ALLOCATION {
832                let (obj_start, _, bytes) = Self::get_malloc_addr_size(object);
833                debug_assert!(
834                    self.active_mem.lock().unwrap().contains_key(&obj_start),
835                    "Address {} with VO bit is not in active_mem",
836                    obj_start
837                );
838                debug_assert_eq!(
839                    self.active_mem.lock().unwrap().get(&obj_start),
840                    Some(&bytes),
841                    "Address {} size in active_mem does not match the size from malloc_usable_size",
842                    obj_start
843                );
844            }
845
846            let live = !self.sweep_object(object, &mut empty_page_start);
847            if live {
848                // Live object. Unset mark bit.
849                // We should be the only thread that access this chunk, it is okay to use non-atomic store.
850                unsafe { unset_mark_bit::<VM>(object) };
851
852                #[cfg(debug_assertions)]
853                {
854                    // Accumulate live bytes
855                    let (_, _, bytes) = Self::get_malloc_addr_size(object);
856                    live_bytes += bytes;
857                }
858            }
859        }
860
861        // If we never updated empty_page_start, the entire chunk is empty.
862        if empty_page_start.is_zero() {
863            self.clean_up_empty_chunk(chunk_start);
864        } else if empty_page_start < chunk_start + BYTES_IN_CHUNK {
865            // This is for the edge case where we have a live object and then no other live
866            // objects afterwards till the end of the chunk. For example consider chunk
867            // 0x0-0x400000 where only one object at 0x100 is alive. We will unset page bits
868            // for 0x0-0x100 but then not unset it for the pages after 0x100. This checks
869            // if we have empty pages at the end of a chunk that needs to be cleared.
870            unsafe {
871                self.unset_page_mark(
872                    empty_page_start,
873                    chunk_start + BYTES_IN_CHUNK - empty_page_start,
874                )
875            };
876        }
877
878        #[cfg(debug_assertions)]
879        self.debug_sweep_chunk_done(live_bytes);
880    }
881}
882
883struct MallocObjectSize<VM>(PhantomData<VM>);
884impl<VM: VMBinding> crate::util::linear_scan::LinearScanObjectSize for MallocObjectSize<VM> {
885    fn size(object: ObjectReference) -> usize {
886        let (_, _, bytes) = MallocSpace::<VM>::get_malloc_addr_size(object);
887        bytes
888    }
889}
890
891use crate::scheduler::GCWork;
892use crate::MMTK;
893
894/// Simple work packet that just sweeps a single chunk
895pub struct MSSweepChunk<VM: VMBinding> {
896    ms: &'static MallocSpace<VM>,
897    // starting address of a chunk
898    chunk: Address,
899}
900
901impl<VM: VMBinding> GCWork<VM> for MSSweepChunk<VM> {
902    fn do_work(&mut self, _worker: &mut GCWorker<VM>, _mmtk: &'static MMTK<VM>) {
903        self.ms.sweep_chunk(self.chunk);
904    }
905}