diff --git a/.github/scripts/ci-test.sh b/.github/scripts/ci-test.sh index 19c54b66b7..ddfb635042 100755 --- a/.github/scripts/ci-test.sh +++ b/.github/scripts/ci-test.sh @@ -12,6 +12,11 @@ if [[ $arch == "x86_64" && $os == "linux" ]]; then fi ALL_PLANS=$(sed -n '/enum PlanSelector/,/}/p' src/util/options.rs | sed -e 's;//.*;;g' -e '/^$/d' -e 's/,//g' | xargs | grep -o '{.*}' | grep -o '\w\+') +ALL_DISCONTIGUOUS_PLANS=$(echo -n "$ALL_PLANS" | sed '/Compressor/d') + +if [[ $arch == "x86" ]]; then + ALL_PLANS=$ALL_DISCONTIGUOUS_PLANS +fi # Test with mock VM: # - Find all the files that start with mock_test_ @@ -25,6 +30,8 @@ find ./src ./tests -type f -name "mock_test_*" | while read -r file; do PLANS=$(sed -n 's/^\/\/ *GITHUB-CI: *MMTK_PLAN=//p' $file | tr ',' '\n') if [[ $PLANS == 'all' ]]; then PLANS=$ALL_PLANS + elif [[ $PLANS == 'discontiguous' ]]; then + PLANS=$ALL_DISCONTIGUOUS_PLANS elif [[ -z $PLANS ]]; then PLANS=NoGC fi diff --git a/Cargo.toml b/Cargo.toml index 05b6de0832..7f46b63910 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -197,6 +197,9 @@ extreme_assertions = [] # Enable multiple spaces for NoGC, each allocator maps to an individual ImmortalSpace. nogc_multi_space = [] +# Disable multiple spaces for Compressor. +compressor_single_space = [] + # To collect statistics for each GC work packet. Enabling this may introduce a small overhead (several percentage slowdown on benchmark time). work_packet_stats = [] diff --git a/src/plan/compressor/gc_work.rs b/src/plan/compressor/gc_work.rs new file mode 100644 index 0000000000..888f072fdd --- /dev/null +++ b/src/plan/compressor/gc_work.rs @@ -0,0 +1,114 @@ +use super::global::Compressor; +use crate::policy::compressor::CompressorSpace; +use crate::policy::compressor::{TRACE_KIND_FORWARD_ROOT, TRACE_KIND_MARK}; +use crate::scheduler::gc_work::PlanProcessEdges; +use crate::scheduler::gc_work::*; +use crate::scheduler::GCWork; +use crate::scheduler::GCWorker; +use crate::scheduler::WorkBucketStage; +use crate::vm::ActivePlan; +use crate::vm::Scanning; +use crate::vm::VMBinding; +use crate::MMTK; +use std::marker::PhantomData; + +/// Iterate through the heap and calculate the new location of live objects. +pub struct CalculateForwardingAddress { + compressor_space: &'static CompressorSpace, +} + +impl GCWork for CalculateForwardingAddress { + fn do_work(&mut self, _worker: &mut GCWorker, _mmtk: &'static MMTK) { + self.compressor_space.calculate_offset_vector(); + } +} + +impl CalculateForwardingAddress { + pub fn new(compressor_space: &'static CompressorSpace) -> Self { + Self { compressor_space } + } +} + +/// Create another round of root scanning work packets +/// to update object references. +pub struct UpdateReferences { + plan: *const Compressor, + p: PhantomData, +} + +unsafe impl Send for UpdateReferences {} + +impl GCWork for UpdateReferences { + fn do_work(&mut self, worker: &mut GCWorker, mmtk: &'static MMTK) { + // The following needs to be done right before the second round of root scanning + VM::VMScanning::prepare_for_roots_re_scanning(); + mmtk.state.prepare_for_stack_scanning(); + // Prepare common and base spaces for the 2nd round of transitive closure + let plan_mut = unsafe { &mut *(self.plan as *mut Compressor) }; + plan_mut.common.release(worker.tls, true); + plan_mut.common.prepare(worker.tls, true); + #[cfg(feature = "extreme_assertions")] + mmtk.slot_logger.reset(); + + // We do two passes of transitive closures. We clear the live bytes from the first pass. + mmtk.scheduler + .worker_group + .get_and_clear_worker_live_bytes(); + + for mutator in VM::VMActivePlan::mutators() { + mmtk.scheduler.work_buckets[WorkBucketStage::SecondRoots].add(ScanMutatorRoots::< + CompressorForwardingWorkContext, + >(mutator)); + } + + mmtk.scheduler.work_buckets[WorkBucketStage::SecondRoots] + .add(ScanVMSpecificRoots::>::new()); + } +} + +impl UpdateReferences { + pub fn new(plan: &Compressor) -> Self { + Self { + plan, + p: PhantomData, + } + } +} + +/// Compact live objects based on the previously-calculated forwarding pointers. +pub struct Compact { + compressor_space: &'static CompressorSpace, +} + +impl GCWork for Compact { + fn do_work(&mut self, worker: &mut GCWorker, _mmtk: &'static MMTK) { + self.compressor_space.compact(worker); + } +} + +impl Compact { + pub fn new(compressor_space: &'static CompressorSpace) -> Self { + Self { compressor_space } + } +} + +/// Marking trace +pub type MarkingProcessEdges = PlanProcessEdges, TRACE_KIND_MARK>; +/// Forwarding trace +pub type ForwardingProcessEdges = PlanProcessEdges, TRACE_KIND_FORWARD_ROOT>; + +pub struct CompressorWorkContext(std::marker::PhantomData); +impl crate::scheduler::GCWorkContext for CompressorWorkContext { + type VM = VM; + type PlanType = Compressor; + type DefaultProcessEdges = MarkingProcessEdges; + type PinningProcessEdges = UnsupportedProcessEdges; +} + +pub struct CompressorForwardingWorkContext(std::marker::PhantomData); +impl crate::scheduler::GCWorkContext for CompressorForwardingWorkContext { + type VM = VM; + type PlanType = Compressor; + type DefaultProcessEdges = ForwardingProcessEdges; + type PinningProcessEdges = UnsupportedProcessEdges; +} diff --git a/src/plan/compressor/global.rs b/src/plan/compressor/global.rs new file mode 100644 index 0000000000..e6c8a26c69 --- /dev/null +++ b/src/plan/compressor/global.rs @@ -0,0 +1,197 @@ +use super::gc_work::CompressorWorkContext; +use super::gc_work::{ + CalculateForwardingAddress, Compact, ForwardingProcessEdges, MarkingProcessEdges, + UpdateReferences, +}; +use crate::plan::compressor::mutator::ALLOCATOR_MAPPING; +use crate::plan::global::CreateGeneralPlanArgs; +use crate::plan::global::CreateSpecificPlanArgs; +use crate::plan::global::{BasePlan, CommonPlan}; +use crate::plan::AllocationSemantics; +use crate::plan::Plan; +use crate::plan::PlanConstraints; +use crate::policy::compressor::CompressorSpace; +use crate::policy::space::Space; +use crate::scheduler::gc_work::*; +use crate::scheduler::GCWorkScheduler; +use crate::scheduler::WorkBucketStage; +use crate::util::alloc::allocators::AllocatorSelector; +use crate::util::heap::gc_trigger::SpaceStats; +#[allow(unused_imports)] +use crate::util::heap::VMRequest; +use crate::util::metadata::side_metadata::SideMetadataContext; +use crate::util::opaque_pointer::*; +use crate::vm::VMBinding; +use enum_map::EnumMap; +use mmtk_macros::{HasSpaces, PlanTraceObject}; + +/// Compressor implements a stop-the-world and serial implementation of +/// the Compressor, as described in Kermany and Petrank, +/// [The Compressor: concurrent, incremental, and parallel compaction](https://dl.acm.org/doi/10.1145/1133255.1134023). +#[derive(HasSpaces, PlanTraceObject)] +pub struct Compressor { + #[parent] + pub common: CommonPlan, + #[space] + pub compressor_space: CompressorSpace, +} + +/// The plan constraints for the Compressor plan. +pub const COMPRESSOR_CONSTRAINTS: PlanConstraints = PlanConstraints { + moves_objects: true, + needs_forward_after_liveness: true, + ..PlanConstraints::default() +}; + +impl Plan for Compressor { + fn constraints(&self) -> &'static PlanConstraints { + &COMPRESSOR_CONSTRAINTS + } + + fn collection_required(&self, space_full: bool, _space: Option>) -> bool { + self.base().collection_required(self, space_full) + } + + fn common(&self) -> &CommonPlan { + &self.common + } + + fn base(&self) -> &BasePlan { + &self.common.base + } + + fn base_mut(&mut self) -> &mut BasePlan { + &mut self.common.base + } + + fn prepare(&mut self, tls: VMWorkerThread) { + self.common.prepare(tls, true); + self.compressor_space.prepare(); + } + + fn release(&mut self, tls: VMWorkerThread) { + self.common.release(tls, true); + self.compressor_space.release(); + } + + fn end_of_gc(&mut self, tls: VMWorkerThread) { + self.common.end_of_gc(tls); + } + + fn get_allocator_mapping(&self) -> &'static EnumMap { + &ALLOCATOR_MAPPING + } + + fn schedule_collection(&'static self, scheduler: &GCWorkScheduler) { + // TODO use schedule_common once it can work with the Compressor + // The main issue there is that we need to ForwardingProcessEdges + // in FinalizableForwarding. + + // Stop & scan mutators (mutator scanning can happen before STW) + scheduler.work_buckets[WorkBucketStage::Unconstrained] + .add(StopMutators::>::new()); + + // Prepare global/collectors/mutators + scheduler.work_buckets[WorkBucketStage::Prepare] + .add(Prepare::>::new(self)); + + scheduler.work_buckets[WorkBucketStage::CalculateForwarding].add( + CalculateForwardingAddress::::new(&self.compressor_space), + ); + // do another trace to update references + scheduler.work_buckets[WorkBucketStage::SecondRoots].add(UpdateReferences::::new(self)); + scheduler.work_buckets[WorkBucketStage::Compact] + .add(Compact::::new(&self.compressor_space)); + + // Release global/collectors/mutators + scheduler.work_buckets[WorkBucketStage::Release] + .add(Release::>::new(self)); + + // Reference processing + if !*self.base().options.no_reference_types { + use crate::util::reference_processor::{ + PhantomRefProcessing, SoftRefProcessing, WeakRefProcessing, + }; + scheduler.work_buckets[WorkBucketStage::SoftRefClosure] + .add(SoftRefProcessing::>::new()); + scheduler.work_buckets[WorkBucketStage::WeakRefClosure] + .add(WeakRefProcessing::::new()); + scheduler.work_buckets[WorkBucketStage::PhantomRefClosure] + .add(PhantomRefProcessing::::new()); + + use crate::util::reference_processor::RefForwarding; + scheduler.work_buckets[WorkBucketStage::RefForwarding] + .add(RefForwarding::>::new()); + + use crate::util::reference_processor::RefEnqueue; + scheduler.work_buckets[WorkBucketStage::Release].add(RefEnqueue::::new()); + } + + // Finalization + if !*self.base().options.no_finalizer { + use crate::util::finalizable_processor::{Finalization, ForwardFinalization}; + // finalization + // treat finalizable objects as roots and perform a closure (marking) + // must be done before calculating forwarding pointers + scheduler.work_buckets[WorkBucketStage::FinalRefClosure] + .add(Finalization::>::new()); + // update finalizable object references + // must be done before compacting + scheduler.work_buckets[WorkBucketStage::FinalizableForwarding] + .add(ForwardFinalization::>::new()); + } + + // VM-specific weak ref processing + scheduler.work_buckets[WorkBucketStage::VMRefClosure] + .set_sentinel(Box::new(VMProcessWeakRefs::>::new())); + + // VM-specific weak ref forwarding + scheduler.work_buckets[WorkBucketStage::VMRefForwarding] + .add(VMForwardWeakRefs::>::new()); + + // VM-specific work after forwarding, possible to implement ref enququing. + scheduler.work_buckets[WorkBucketStage::Release].add(VMPostForwarding::::default()); + + // Analysis GC work + #[cfg(feature = "analysis")] + { + use crate::util::analysis::GcHookWork; + scheduler.work_buckets[WorkBucketStage::Unconstrained].add(GcHookWork); + } + #[cfg(feature = "sanity")] + scheduler.work_buckets[WorkBucketStage::Final] + .add(crate::util::sanity::sanity_checker::ScheduleSanityGC::::new(self)); + } + + fn current_gc_may_move_object(&self) -> bool { + true + } + + fn get_used_pages(&self) -> usize { + self.compressor_space.reserved_pages() + self.common.get_used_pages() + } +} + +impl Compressor { + pub fn new(args: CreateGeneralPlanArgs) -> Self { + let mut plan_args = CreateSpecificPlanArgs { + global_args: args, + constraints: &COMPRESSOR_CONSTRAINTS, + global_side_metadata_specs: SideMetadataContext::new_global_specs(&[]), + }; + + let res = Compressor { + compressor_space: CompressorSpace::new(plan_args.get_space_args( + "compressor_space", + true, + false, + VMRequest::discontiguous(), + )), + common: CommonPlan::new(plan_args), + }; + + res.verify_side_metadata_sanity(); + + res + } +} diff --git a/src/plan/compressor/mod.rs b/src/plan/compressor/mod.rs new file mode 100644 index 0000000000..61501cdb78 --- /dev/null +++ b/src/plan/compressor/mod.rs @@ -0,0 +1,5 @@ +pub(super) mod gc_work; +pub(super) mod global; +pub(super) mod mutator; + +pub use self::global::Compressor; diff --git a/src/plan/compressor/mutator.rs b/src/plan/compressor/mutator.rs new file mode 100644 index 0000000000..c999ddf2c9 --- /dev/null +++ b/src/plan/compressor/mutator.rs @@ -0,0 +1,72 @@ +use crate::plan::compressor::Compressor; +use crate::plan::mutator_context::common_prepare_func; +use crate::plan::mutator_context::Mutator; +use crate::plan::mutator_context::MutatorBuilder; +use crate::plan::mutator_context::MutatorConfig; +use crate::plan::mutator_context::{ + create_allocator_mapping, create_space_mapping, ReservedAllocators, +}; +use crate::plan::AllocationSemantics; +use crate::util::alloc::allocators::AllocatorSelector; +use crate::util::alloc::BumpAllocator; +use crate::util::{VMMutatorThread, VMWorkerThread}; +use crate::vm::VMBinding; +use crate::MMTK; +use enum_map::{enum_map, EnumMap}; + +const RESERVED_ALLOCATORS: ReservedAllocators = ReservedAllocators { + n_bump_pointer: 1, + ..ReservedAllocators::DEFAULT +}; + +lazy_static! { + /// When compressor_single_space is enabled, force all allocations to go to the default allocator and space. + static ref ALLOCATOR_MAPPING_SINGLE_SPACE: EnumMap = enum_map! { + _ => AllocatorSelector::BumpPointer(0), + }; + pub static ref ALLOCATOR_MAPPING: EnumMap = { + if cfg!(feature = "compressor_single_space") { + *ALLOCATOR_MAPPING_SINGLE_SPACE + } else { + let mut map = create_allocator_mapping(RESERVED_ALLOCATORS, true); + map[AllocationSemantics::Default] = AllocatorSelector::BumpPointer(0); + map + } + }; +} + +pub fn create_compressor_mutator( + mutator_tls: VMMutatorThread, + mmtk: &'static MMTK, +) -> Mutator { + let plan = mmtk.get_plan().downcast_ref::>().unwrap(); + let config = MutatorConfig { + allocator_mapping: &ALLOCATOR_MAPPING, + space_mapping: Box::new({ + let mut vec = create_space_mapping( + RESERVED_ALLOCATORS, + !cfg!(feature = "compressor_single_space"), + plan, + ); + vec.push((AllocatorSelector::BumpPointer(0), &plan.compressor_space)); + vec + }), + prepare_func: &common_prepare_func, + release_func: &compressor_mutator_release, + }; + + let builder = MutatorBuilder::new(mutator_tls, mmtk, config); + builder.build() +} + +pub fn compressor_mutator_release(mutator: &mut Mutator, _tls: VMWorkerThread) { + // reset the thread-local allocation bump pointer + let bump_allocator = unsafe { + mutator + .allocators + .get_allocator_mut(mutator.config.allocator_mapping[AllocationSemantics::Default]) + } + .downcast_mut::>() + .unwrap(); + bump_allocator.reset(); +} diff --git a/src/plan/global.rs b/src/plan/global.rs index 5a5bb38ab5..cd087bc20f 100644 --- a/src/plan/global.rs +++ b/src/plan/global.rs @@ -58,6 +58,9 @@ pub fn create_mutator( PlanSelector::StickyImmix => { crate::plan::sticky::immix::mutator::create_stickyimmix_mutator(tls, mmtk) } + PlanSelector::Compressor => { + crate::plan::compressor::mutator::create_compressor_mutator(tls, mmtk) + } }) } @@ -91,6 +94,9 @@ pub fn create_plan( PlanSelector::StickyImmix => { Box::new(crate::plan::sticky::immix::StickyImmix::new(args)) as Box> } + PlanSelector::Compressor => { + Box::new(crate::plan::compressor::Compressor::new(args)) as Box> + } }; // We have created Plan in the heap, and we won't explicitly move it. diff --git a/src/plan/mod.rs b/src/plan/mod.rs index 74fcac2811..e86a7fe590 100644 --- a/src/plan/mod.rs +++ b/src/plan/mod.rs @@ -44,6 +44,7 @@ mod generational; /// Sticky plans (using sticky marks for generational behaviors without a copying nursery) mod sticky; +mod compressor; mod immix; mod markcompact; mod marksweep; diff --git a/src/policy/compressor/compressorspace.rs b/src/policy/compressor/compressorspace.rs new file mode 100644 index 0000000000..91d18555c6 --- /dev/null +++ b/src/policy/compressor/compressorspace.rs @@ -0,0 +1,334 @@ +use crate::plan::VectorObjectQueue; +use crate::policy::compressor::forwarding; +use crate::policy::gc_work::{TraceKind, TRACE_KIND_TRANSITIVE_PIN}; +use crate::policy::sft::GCWorkerMutRef; +use crate::policy::sft::SFT; +use crate::policy::space::{CommonSpace, Space}; +use crate::scheduler::GCWorker; +use crate::util::copy::CopySemantics; +use crate::util::heap::{MonotonePageResource, PageResource}; +use crate::util::metadata::extract_side_metadata; +#[cfg(feature = "vo_bit")] +use crate::util::metadata::vo_bit; +use crate::util::metadata::MetadataSpec; +use crate::util::object_enum::{self, ObjectEnumerator}; +use crate::util::{Address, ObjectReference}; +use crate::vm::slot::Slot; +use crate::{vm::*, ObjectQueue}; +use atomic::Ordering; + +pub(crate) const TRACE_KIND_MARK: TraceKind = 0; +pub(crate) const TRACE_KIND_FORWARD_ROOT: TraceKind = 1; + +/// CompressorSpace is a stop-the-world and serial implementation of +/// the Compressor, as described in Kermany and Petrank, +/// [The Compressor: concurrent, incremental, and parallel compaction](https://dl.acm.org/doi/10.1145/1133255.1134023). +pub struct CompressorSpace { + common: CommonSpace, + pr: MonotonePageResource, + forwarding: forwarding::ForwardingMetadata, +} + +pub(crate) const GC_MARK_BIT_MASK: u8 = 1; + +impl SFT for CompressorSpace { + fn name(&self) -> &'static str { + self.get_name() + } + + fn get_forwarded_object(&self, object: ObjectReference) -> Option { + Some(self.forward(object, false)) + } + + fn is_live(&self, object: ObjectReference) -> bool { + Self::is_marked(object) + } + + #[cfg(feature = "object_pinning")] + fn pin_object(&self, _object: ObjectReference) -> bool { + panic!("Cannot pin/unpin objects of CompressorSpace.") + } + + #[cfg(feature = "object_pinning")] + fn unpin_object(&self, _object: ObjectReference) -> bool { + panic!("Cannot pin/unpin objects of CompressorSpace.") + } + + #[cfg(feature = "object_pinning")] + fn is_object_pinned(&self, _object: ObjectReference) -> bool { + false + } + + fn is_movable(&self) -> bool { + true + } + + fn initialize_object_metadata(&self, _object: ObjectReference, _alloc: bool) { + #[cfg(feature = "vo_bit")] + crate::util::metadata::vo_bit::set_vo_bit(_object); + } + + #[cfg(feature = "sanity")] + fn is_sane(&self) -> bool { + true + } + + #[cfg(feature = "is_mmtk_object")] + fn is_mmtk_object(&self, addr: Address) -> Option { + crate::util::metadata::vo_bit::is_vo_bit_set_for_addr(addr) + } + + #[cfg(feature = "is_mmtk_object")] + fn find_object_from_internal_pointer( + &self, + ptr: Address, + max_search_bytes: usize, + ) -> Option { + crate::util::metadata::vo_bit::find_object_from_internal_pointer::( + ptr, + max_search_bytes, + ) + } + + fn sft_trace_object( + &self, + _queue: &mut VectorObjectQueue, + _object: ObjectReference, + _worker: GCWorkerMutRef, + ) -> ObjectReference { + // We should not use trace_object for compressor space. + // Depending on which trace it is, we should manually call either trace_mark or trace_forward. + panic!("sft_trace_object() cannot be used with Compressor space") + } + + fn debug_print_object_info(&self, object: ObjectReference) { + println!("marked = {}", CompressorSpace::::is_marked(object)); + self.common.debug_print_object_global_info(object); + } +} + +impl Space for CompressorSpace { + fn as_space(&self) -> &dyn Space { + self + } + + fn as_sft(&self) -> &(dyn SFT + Sync + 'static) { + self + } + + fn get_page_resource(&self) -> &dyn PageResource { + &self.pr + } + + fn maybe_get_page_resource_mut(&mut self) -> Option<&mut dyn PageResource> { + Some(&mut self.pr) + } + + fn common(&self) -> &CommonSpace { + &self.common + } + + fn initialize_sft(&self, sft_map: &mut dyn crate::policy::sft_map::SFTMap) { + self.common().initialize_sft(self.as_sft(), sft_map) + } + + fn release_multiple_pages(&mut self, _start: Address) { + panic!("compressorspace only releases pages enmasse") + } + + fn enumerate_objects(&self, enumerator: &mut dyn ObjectEnumerator) { + object_enum::enumerate_blocks_from_monotonic_page_resource(enumerator, &self.pr); + } +} + +impl crate::policy::gc_work::PolicyTraceObject for CompressorSpace { + fn trace_object( + &self, + queue: &mut Q, + object: ObjectReference, + _copy: Option, + _worker: &mut GCWorker, + ) -> ObjectReference { + debug_assert!( + KIND != TRACE_KIND_TRANSITIVE_PIN, + "Compressor does not support transitive pin trace." + ); + if KIND == TRACE_KIND_MARK { + self.trace_mark_object(queue, object) + } else if KIND == TRACE_KIND_FORWARD_ROOT { + self.trace_forward_root(queue, object) + } else { + unreachable!() + } + } + fn may_move_objects() -> bool { + if KIND == TRACE_KIND_MARK { + false + } else if KIND == TRACE_KIND_FORWARD_ROOT { + true + } else { + unreachable!() + } + } +} + +impl CompressorSpace { + pub fn new(args: crate::policy::space::PlanCreateSpaceArgs) -> Self { + let vm_map = args.vm_map; + assert!( + !args.vmrequest.is_discontiguous(), + "The Compressor requires a contiguous heap" + ); + let local_specs = extract_side_metadata(&[ + MetadataSpec::OnSide(forwarding::MARK_SPEC), + MetadataSpec::OnSide(forwarding::OFFSET_VECTOR_SPEC), + ]); + let common = CommonSpace::new(args.into_policy_args(true, false, local_specs)); + + CompressorSpace { + pr: MonotonePageResource::new_contiguous(common.start, common.extent, vm_map), + forwarding: forwarding::ForwardingMetadata::new(common.start), + common, + } + } + + pub fn prepare(&self) { + for (from_start, size) in self.pr.iterate_allocated_regions() { + forwarding::MARK_SPEC.bzero_metadata(from_start, size); + } + } + + pub fn release(&self) { + self.forwarding.release(); + } + + pub fn trace_mark_object( + &self, + queue: &mut Q, + object: ObjectReference, + ) -> ObjectReference { + #[cfg(feature = "vo_bit")] + debug_assert!( + crate::util::metadata::vo_bit::is_vo_bit_set(object), + "{:x}: VO bit not set", + object + ); + if CompressorSpace::::test_and_mark(object) { + queue.enqueue(object); + self.forwarding.mark_end_of_object(object); + } + object + } + + pub fn trace_forward_root( + &self, + _queue: &mut Q, + object: ObjectReference, + ) -> ObjectReference { + self.forward(object, true) + } + + pub fn test_and_mark(object: ObjectReference) -> bool { + let old = forwarding::MARK_SPEC.fetch_or_atomic( + object.to_raw_address(), + GC_MARK_BIT_MASK, + Ordering::SeqCst, + ); + (old & GC_MARK_BIT_MASK) == 0 + } + + pub fn is_marked(object: ObjectReference) -> bool { + let old_value = + forwarding::MARK_SPEC.load_atomic::(object.to_raw_address(), Ordering::SeqCst); + let mark_bit = old_value & GC_MARK_BIT_MASK; + mark_bit != 0 + } + + pub fn calculate_offset_vector(&self) { + self.forwarding.calculate_offset_vector(&self.pr); + } + + pub fn forward(&self, object: ObjectReference, _vo_bit_valid: bool) -> ObjectReference { + // We can't expect the VO bit to be valid whilst in the compaction loop. + // If we are fixing a reference to an object which precedes the referent + // the VO bit will have been cleared already. + // Thus the assertion really only is any good whilst we are fixing + // the roots. + #[cfg(feature = "vo_bit")] + if _vo_bit_valid { + debug_assert!( + crate::util::metadata::vo_bit::is_vo_bit_set(object), + "{:x}: VO bit not set", + object + ); + } + ObjectReference::from_raw_address(self.forwarding.forward(object.to_raw_address())).unwrap() + } + + fn heap_span(&self) -> (Address, Address) { + (self.forwarding.first_address, self.pr.cursor()) + } + + pub fn compact(&self, worker: &mut GCWorker) { + let mut to = Address::ZERO; + // The allocator will never cause an object to span multiple regions, + // but the Compressor may move an object to span multiple regions. + // Thus we must treat all regions as one contiguous space when + // walking the mark bitmap. + let (start, end) = self.heap_span(); + #[cfg(feature = "vo_bit")] + { + #[cfg(debug_assertions)] + self.forwarding + .scan_marked_objects(start, end, &mut |object: ObjectReference| { + debug_assert!( + crate::util::metadata::vo_bit::is_vo_bit_set(object), + "{:x}: VO bit not set", + object + ); + }); + for (region_start, size) in self.pr.iterate_allocated_regions() { + crate::util::metadata::vo_bit::bzero_vo_bit(region_start, size); + } + } + + self.forwarding + .scan_marked_objects(start, end, &mut |obj: ObjectReference| { + // We set the end bits based on the sizes of objects when they are + // marked, and we compute the live data and thus the forwarding + // addresses based on those sizes. The forwarding addresses would be + // incorrect if the sizes of objects were to change. + let copied_size = VM::VMObjectModel::get_size_when_copied(obj); + debug_assert!(copied_size == VM::VMObjectModel::get_current_size(obj)); + let new_object = self.forward(obj, false); + debug_assert!( + new_object.to_raw_address() >= to, + "{0} < {to}", + new_object.to_raw_address() + ); + // copy object + trace!(" copy from {} to {}", obj, new_object); + let end_of_new_object = VM::VMObjectModel::copy_to(obj, new_object, Address::ZERO); + // update VO bit + #[cfg(feature = "vo_bit")] + vo_bit::set_vo_bit(new_object); + to = new_object.to_object_start::() + copied_size; + debug_assert_eq!(end_of_new_object, to); + // update references in object + if VM::VMScanning::support_slot_enqueuing(worker.tls, new_object) { + VM::VMScanning::scan_object(worker.tls, new_object, &mut |s: VM::VMSlot| { + if let Some(o) = s.load() { + s.store(self.forward(o, false)); + } + }); + } else { + VM::VMScanning::scan_object_and_trace_edges(worker.tls, new_object, &mut |o| { + self.forward(o, false) + }); + } + }); + debug!("Compact end: to = {}", to); + // reset the bump pointer + self.pr.reset_cursor(to); + } +} diff --git a/src/policy/compressor/forwarding.rs b/src/policy/compressor/forwarding.rs new file mode 100644 index 0000000000..c14a3a6daa --- /dev/null +++ b/src/policy/compressor/forwarding.rs @@ -0,0 +1,164 @@ +use crate::policy::compressor::GC_MARK_BIT_MASK; +use crate::util::constants::MIN_OBJECT_SIZE; +use crate::util::heap::MonotonePageResource; +use crate::util::metadata::side_metadata::spec_defs::{COMPRESSOR_MARK, COMPRESSOR_OFFSET_VECTOR}; +use crate::util::metadata::side_metadata::SideMetadataSpec; +use crate::util::{Address, ObjectReference}; +use crate::vm::object_model::ObjectModel; +use crate::vm::VMBinding; +use atomic::Ordering; +use std::marker::PhantomData; +use std::sync::atomic::AtomicBool; + +/// A finite-state machine which processes the positions of mark bits, +/// and accumulates the size of live data that it has seen. +/// +/// The Compressor caches the state of the transducer at the start of +/// each block by serialising the state using [`Transducer::encode`]; +/// the state can then be deserialised using [`Transducer::decode`]. +#[derive(Debug)] +struct Transducer { + live: usize, + last_bit_seen: Address, + in_object: bool, +} +impl Transducer { + pub fn new() -> Self { + Self { + live: 0, + last_bit_seen: Address::ZERO, + in_object: false, + } + } + pub fn step(&mut self, address: Address) { + if self.in_object { + self.live += address - self.last_bit_seen + MIN_OBJECT_SIZE; + } + self.in_object = !self.in_object; + self.last_bit_seen = address; + } + + pub fn encode(&self, address: Address) -> usize { + if self.in_object { + // We count the space between the last mark bit and + // the current address as live when we stop in the + // middle of an object. + self.live + (address - self.last_bit_seen) + 1 + } else { + self.live + } + } + + pub fn decode(offset: usize, address: Address) -> Self { + Transducer { + live: offset & !1, + last_bit_seen: address, + in_object: (offset & 1) == 1, + } + } +} + +pub struct ForwardingMetadata { + pub(crate) first_address: Address, + calculated: AtomicBool, + vm: PhantomData, +} + +// A block in the Compressor is the granularity at which we record +// the live data prior to the start of each block. We set it to 512 bytes +// following the paper. +pub(crate) const LOG_BLOCK_SIZE: usize = 9; +pub(crate) const BLOCK_SIZE: usize = 1 << LOG_BLOCK_SIZE; +pub(crate) const MARK_SPEC: SideMetadataSpec = COMPRESSOR_MARK; +pub(crate) const OFFSET_VECTOR_SPEC: SideMetadataSpec = COMPRESSOR_OFFSET_VECTOR; + +impl ForwardingMetadata { + pub fn new(start: Address) -> ForwardingMetadata { + ForwardingMetadata { + first_address: start, + calculated: AtomicBool::new(false), + vm: PhantomData, + } + } + + pub fn mark_end_of_object(&self, object: ObjectReference) { + let end_of_object = + object.to_raw_address() + VM::VMObjectModel::get_current_size(object) - MIN_OBJECT_SIZE; + #[cfg(debug_assertions)] + { + use crate::util::metadata::side_metadata::{address_to_meta_address, meta_byte_lshift}; + // We require to be able to iterate upon start and end bits in the + // same bitmap. Therefore the start and end bits cannot be the + // same, else we would only encounter one of the two bits. + let a1 = address_to_meta_address(&MARK_SPEC, object.to_raw_address()); + let s1 = meta_byte_lshift(&MARK_SPEC, object.to_raw_address()); + let a2 = address_to_meta_address(&MARK_SPEC, end_of_object); + let s2 = meta_byte_lshift(&MARK_SPEC, end_of_object); + debug_assert!( + (a1, s1) < (a2, s2), + "The start and end mark bits should be different bits" + ); + } + + MARK_SPEC.fetch_or_atomic(end_of_object, GC_MARK_BIT_MASK, Ordering::SeqCst); + } + + pub fn calculate_offset_vector(&self, pr: &MonotonePageResource) { + let mut state = Transducer::new(); + let last_block = (pr.cursor() - self.first_address) / BLOCK_SIZE; + debug!("calculating offset of {last_block} blocks"); + for block in 0..last_block { + let block_start = self.first_address + (block * BLOCK_SIZE); + let block_end = block_start + BLOCK_SIZE; + OFFSET_VECTOR_SPEC.store_atomic::( + block_start, + state.encode(block_start), + Ordering::Relaxed, + ); + MARK_SPEC.scan_non_zero_values::(block_start, block_end, &mut |addr: Address| { + state.step(addr); + }); + } + self.calculated.store(true, Ordering::Relaxed); + } + + pub fn release(&self) { + self.calculated.store(false, Ordering::Relaxed); + } + + pub fn forward(&self, address: Address) -> Address { + debug_assert!( + self.calculated.load(Ordering::Relaxed), + "forward() should only be called when we have calculated an offset vector" + ); + let block_number = (address - self.first_address) / BLOCK_SIZE; + let block_address = self.first_address + (block_number * BLOCK_SIZE); + let mut state = Transducer::decode( + OFFSET_VECTOR_SPEC.load_atomic::(block_address, Ordering::Relaxed), + block_address, + ); + // The transducer in this implementation computes the offset + // relative to the start of the heap; whereas Total-Live-Data in + // the paper computes the offset relative to the start of the block. + MARK_SPEC.scan_non_zero_values::(block_address, address, &mut |addr: Address| { + state.step(addr); + }); + self.first_address + state.live + } + + pub fn scan_marked_objects( + &self, + start: Address, + end: Address, + f: &mut impl FnMut(ObjectReference), + ) { + let mut in_object = false; + MARK_SPEC.scan_non_zero_values::(start, end, &mut |addr: Address| { + if !in_object { + let object = ObjectReference::from_raw_address(addr).unwrap(); + f(object); + } + in_object = !in_object; + }); + } +} diff --git a/src/policy/compressor/mod.rs b/src/policy/compressor/mod.rs new file mode 100644 index 0000000000..dfe9b56a07 --- /dev/null +++ b/src/policy/compressor/mod.rs @@ -0,0 +1,4 @@ +pub mod compressorspace; +pub mod forwarding; + +pub use compressorspace::*; diff --git a/src/policy/markcompactspace.rs b/src/policy/markcompactspace.rs index 2f3689f70b..c709ab499a 100644 --- a/src/policy/markcompactspace.rs +++ b/src/policy/markcompactspace.rs @@ -42,8 +42,6 @@ impl SFT for MarkCompactSpace { } fn is_live(&self, object: ObjectReference) -> bool { - // Sanity checker cannot use this method to do the verification - // since the mark bit will be cleared during the second trace(update forwarding pointer) Self::is_marked(object) } diff --git a/src/policy/mod.rs b/src/policy/mod.rs index c5763a213f..1d29d34f4a 100644 --- a/src/policy/mod.rs +++ b/src/policy/mod.rs @@ -21,6 +21,7 @@ pub mod gc_work; pub mod sft; pub mod sft_map; +pub mod compressor; pub mod copyspace; pub mod immix; pub mod immortalspace; diff --git a/src/util/metadata/side_metadata/helpers.rs b/src/util/metadata/side_metadata/helpers.rs index adb56a28e1..2a9104c492 100644 --- a/src/util/metadata/side_metadata/helpers.rs +++ b/src/util/metadata/side_metadata/helpers.rs @@ -224,7 +224,7 @@ pub(super) const fn metadata_address_range_size(metadata_spec: &SideMetadataSpec 1usize << (VMLayout::LOG_ARCH_ADDRESS_SPACE - log_data_meta_ratio(metadata_spec)) } -pub(super) fn meta_byte_lshift(metadata_spec: &SideMetadataSpec, data_addr: Address) -> u8 { +pub(crate) fn meta_byte_lshift(metadata_spec: &SideMetadataSpec, data_addr: Address) -> u8 { let bits_num_log = metadata_spec.log_num_of_bits as i32; if bits_num_log >= 3 { return 0; diff --git a/src/util/metadata/side_metadata/spec_defs.rs b/src/util/metadata/side_metadata/spec_defs.rs index 86482c3d01..4d18985ce8 100644 --- a/src/util/metadata/side_metadata/spec_defs.rs +++ b/src/util/metadata/side_metadata/spec_defs.rs @@ -94,6 +94,10 @@ define_side_metadata_specs!( MS_LOCAL_FREE = (global: false, log_num_of_bits: LOG_BITS_IN_ADDRESS, log_bytes_in_region: crate::policy::marksweepspace::native_ms::Block::LOG_BYTES), // First cell of thread free list in block for native mimalloc MS_THREAD_FREE = (global: false, log_num_of_bits: LOG_BITS_IN_ADDRESS, log_bytes_in_region: crate::policy::marksweepspace::native_ms::Block::LOG_BYTES), + // Start and end marks by Compressor + COMPRESSOR_MARK = (global: false, log_num_of_bits: 0, log_bytes_in_region: LOG_MIN_OBJECT_SIZE as usize), + // Block offset vectors by Compressor + COMPRESSOR_OFFSET_VECTOR = (global: false, log_num_of_bits: LOG_BITS_IN_ADDRESS, log_bytes_in_region: crate::policy::compressor::forwarding::LOG_BLOCK_SIZE), ); #[cfg(test)] diff --git a/src/util/options.rs b/src/util/options.rs index abfada69c8..a8df06faff 100644 --- a/src/util/options.rs +++ b/src/util/options.rs @@ -46,6 +46,8 @@ pub enum PlanSelector { Immix, /// A mark-compact collector that marks objects and performs Cheney-style copying. MarkCompact, + /// A mark-compact collector that uses Compressor-style bitmaps. + Compressor, /// An Immix collector that uses a sticky mark bit to allow generational behaviors without a copying nursery. StickyImmix, } diff --git a/src/vm/tests/mock_tests/mock_test_allocator_info.rs b/src/vm/tests/mock_tests/mock_test_allocator_info.rs index 3761b2e42b..fc288e8041 100644 --- a/src/vm/tests/mock_tests/mock_test_allocator_info.rs +++ b/src/vm/tests/mock_tests/mock_test_allocator_info.rs @@ -28,6 +28,7 @@ pub fn test_allocator_info() { | PlanSelector::GenCopy | PlanSelector::GenImmix | PlanSelector::MarkCompact + | PlanSelector::Compressor | PlanSelector::StickyImmix => { // These plans all use bump pointer allocator. let AllocatorInfo::BumpPointer { diff --git a/src/vm/tests/mock_tests/mock_test_vm_layout_compressed_pointer.rs b/src/vm/tests/mock_tests/mock_test_vm_layout_compressed_pointer.rs index 658733f683..4cc750ee26 100644 --- a/src/vm/tests/mock_tests/mock_test_vm_layout_compressed_pointer.rs +++ b/src/vm/tests/mock_tests/mock_test_vm_layout_compressed_pointer.rs @@ -1,4 +1,4 @@ -// GITHUB-CI: MMTK_PLAN=all +// GITHUB-CI: MMTK_PLAN=discontiguous use super::mock_test_prelude::*; use super::mock_test_vm_layout_default::test_with_vm_layout;