From 775e25eaba2e30107f86f7dca20f0bce05136953 Mon Sep 17 00:00:00 2001 From: Yi Lin Date: Thu, 10 Mar 2022 21:08:44 +1300 Subject: [PATCH] Fix object ref SFT (#555) If a binding uses `ObjectReference` to point to an address that is not within our allocated memory, the SFT and the side metadata for the `ObjectReference` may not be initialized properly. This PR introduces a solution by introducing a constant `ObjectModel::OBJECT_REF_OFFSET_BEYOND_CELL`. If a binding may use object reference to point to an address beyond the cell we allocate, they need to provide a `Some` value for the constant. MMTk allocators need to guarantee that the object reference for an allocation won't point to a new chunk (so its metadata is properly initialized and is the same as the allocation result). This PR fixes https://github.com/mmtk/mmtk-jikesrvm/issues/102. This PR: * introduces a constant `ObjectModel::OBJECT_REF_OFFSET_BEYOND_CELL`. By default, it is `None`. If a binding may use `ObjectReference` to point to an address outside a cell, they need to set the constant to a proper `Some` value. * Introduces a module `util::alloc::object_ref_guard`, which includes a few methods that help allocators to check and avoid breaking the invariant. * adds a few assertions to check if our allocation addresses and object references have valid SFT entries. * introduces a space acquire lock so we can guarantee when pages are acquired from a page resource, the SFT for the pages are always initialized. * removes unnecessary methods in `Map32`. --- src/policy/mallocspace/global.rs | 10 +- src/policy/space.rs | 158 ++++++++++++++---- src/scheduler/gc_work.rs | 4 + src/util/alloc/allocator.rs | 4 + src/util/alloc/bumpallocator.rs | 3 +- src/util/alloc/immix_allocator.rs | 7 +- src/util/alloc/large_object_allocator.rs | 3 + src/util/alloc/malloc_allocator.rs | 29 +++- src/util/alloc/mod.rs | 13 ++ src/util/alloc/object_ref_guard.rs | 65 +++++++ src/util/heap/layout/map.rs | 9 - src/util/heap/layout/map32.rs | 40 +++-- src/util/heap/layout/vm_layout_constants.rs | 1 + src/util/metadata/side_metadata/constants.rs | 7 +- src/util/metadata/side_metadata/helpers_32.rs | 4 +- src/util/object_forwarding.rs | 2 + src/vm/object_model.rs | 9 + 17 files changed, 290 insertions(+), 78 deletions(-) create mode 100644 src/util/alloc/object_ref_guard.rs diff --git a/src/policy/mallocspace/global.rs b/src/policy/mallocspace/global.rs index 5bfc0d18b1..4246e8f2f6 100644 --- a/src/policy/mallocspace/global.rs +++ b/src/policy/mallocspace/global.rs @@ -240,9 +240,15 @@ impl MallocSpace { address } + pub fn free(&self, addr: Address) { + let offset_malloc_bit = is_offset_malloc(addr); + let bytes = get_malloc_usable_size(addr, offset_malloc_bit); + self.free_internal(addr, bytes, offset_malloc_bit); + } + // XXX optimize: We pass the bytes in to free as otherwise there were multiple // indirect call instructions in the generated assembly - pub fn free(&self, addr: Address, bytes: usize, offset_malloc_bit: bool) { + fn free_internal(&self, addr: Address, bytes: usize, offset_malloc_bit: bool) { if offset_malloc_bit { trace!("Free memory {:x}", addr); offset_free(addr); @@ -373,7 +379,7 @@ impl MallocSpace { trace!("Object {} has been allocated but not marked", object); // Free object - self.free(obj_start, bytes, offset_malloc); + self.free_internal(obj_start, bytes, offset_malloc); trace!("free object {}", object); unsafe { unset_alloc_bit_unsafe(object) }; diff --git a/src/policy/space.rs b/src/policy/space.rs index d8d15343bc..a7fd18edd9 100644 --- a/src/policy/space.rs +++ b/src/policy/space.rs @@ -28,6 +28,7 @@ use crate::util::memory; use crate::vm::VMBinding; use std::marker::PhantomData; +use std::sync::Mutex; use downcast_rs::Downcast; @@ -122,6 +123,7 @@ impl SFT for EmptySpaceSFT { } #[cfg(feature = "sanity")] fn is_sane(&self) -> bool { + warn!("Object in empty space!"); false } fn is_movable(&self) -> bool { @@ -154,8 +156,9 @@ impl SFT for EmptySpaceSFT { _worker: GCWorkerMutRef, ) -> ObjectReference { panic!( - "Call trace_object() on {:x}, which maps to an empty space", - _object + "Call trace_object() on {} (chunk {}), which maps to an empty space", + _object, + conversions::chunk_align_down(_object.to_address()), ) } } @@ -199,38 +202,42 @@ impl<'a> SFTMap<'a> { } fn log_update(&self, space: &(dyn SFT + Sync + 'static), start: Address, bytes: usize) { - debug!( - "Update SFT for [{}, {}) as {}", - start, - start + bytes, - space.name() - ); + debug!("Update SFT for Chunk {} as {}", start, space.name(),); let first = start.chunk_index(); - let last = conversions::chunk_align_up(start + bytes).chunk_index(); let start_chunk = chunk_index_to_address(first); - let end_chunk = chunk_index_to_address(last); debug!( - "Update SFT for {} bytes of [{} #{}, {} #{})", - bytes, start_chunk, first, end_chunk, last + "Update SFT for {} bytes of Chunk {} #{}", + bytes, start_chunk, first ); } fn trace_sft_map(&self) { - // For large heaps, it takes long to iterate each chunk. So check log level first. - if log::log_enabled!(log::Level::Trace) { - // print the entire SFT map - const SPACE_PER_LINE: usize = 10; - for i in (0..self.sft.len()).step_by(SPACE_PER_LINE) { - let max = if i + SPACE_PER_LINE > self.sft.len() { - self.sft.len() - } else { - i + SPACE_PER_LINE - }; - let chunks: Vec = (i..max).collect(); - let space_names: Vec<&str> = chunks.iter().map(|&x| self.sft[x].name()).collect(); - trace!("Chunk {}: {}", i, space_names.join(",")); - } + trace!("{}", self.print_sft_map()); + } + + // This can be used during debugging to print SFT map. + fn print_sft_map(&self) -> String { + // print the entire SFT map + let mut res = String::new(); + + const SPACE_PER_LINE: usize = 10; + for i in (0..self.sft.len()).step_by(SPACE_PER_LINE) { + let max = if i + SPACE_PER_LINE > self.sft.len() { + self.sft.len() + } else { + i + SPACE_PER_LINE + }; + let chunks: Vec = (i..max).collect(); + let space_names: Vec<&str> = chunks.iter().map(|&x| self.sft[x].name()).collect(); + res.push_str(&format!( + "{}: {}", + chunk_index_to_address(i), + space_names.join(",") + )); + res.push('\n'); } + + res } /// Update SFT map for the given address range. @@ -253,6 +260,13 @@ impl<'a> SFTMap<'a> { // TODO: We should clear a SFT entry when a space releases a chunk. #[allow(dead_code)] pub fn clear(&self, chunk_start: Address) { + if DEBUG_SFT { + debug!( + "Clear SFT for chunk {} (was {})", + chunk_start, + self.get(chunk_start).name() + ); + } assert!(chunk_start.is_aligned_to(BYTES_IN_CHUNK)); let chunk_idx = chunk_start.chunk_index(); self.set(chunk_idx, &EMPTY_SPACE_SFT); @@ -261,6 +275,14 @@ impl<'a> SFTMap<'a> { // Currently only used by 32 bits vm map #[allow(dead_code)] pub fn clear_by_index(&self, chunk_idx: usize) { + if DEBUG_SFT { + let chunk_start = chunk_index_to_address(chunk_idx); + debug!( + "Clear SFT for chunk {} by index (was {})", + chunk_start, + self.get(chunk_start).name() + ); + } self.set(chunk_idx, &EMPTY_SPACE_SFT) } @@ -298,6 +320,27 @@ impl<'a> SFTMap<'a> { } self.get(object.to_address()).is_mmtk_object(object) } + + /// Make sure we have valid SFT entries for the object reference. + #[cfg(debug_assertions)] + pub fn assert_valid_entries_for_object(&self, object: ObjectReference) { + let object_sft = self.get(object.to_address()); + let object_start_sft = self.get(VM::VMObjectModel::object_start_ref(object)); + + debug_assert!( + object_sft.name() != EMPTY_SFT_NAME, + "Object {} has empty SFT", + object + ); + debug_assert_eq!( + object_sft.name(), + object_start_sft.name(), + "Object {} has incorrect SFT entries (object start = {}, object = {}).", + object, + object_start_sft.name(), + object_sft.name() + ); + } } pub trait Space: 'static + SFT + Sync + Downcast { @@ -317,6 +360,12 @@ pub trait Space: 'static + SFT + Sync + Downcast { // initialize_collection() has to be called so we know GC is initialized. let allow_gc = should_poll && VM::VMActivePlan::global().is_initialized(); + // We need this lock: Othrewise, it is possible that one thread acquires pages in a new chunk, but not yet + // set SFT for it (in grow_space()), and another thread acquires pages in the same chunk, which is not + // a new chunk so grow_space() won't be called on it. The second thread could return a result in the chunk before + // its SFT is properly set. + let lock = self.common().acquire_lock.lock().unwrap(); + trace!("Reserving pages"); let pr = self.get_page_resource(); let pages_reserved = pr.reserve_pages(pages); @@ -327,6 +376,7 @@ pub trait Space: 'static + SFT + Sync + Downcast { debug!("Collection required"); assert!(allow_gc, "GC is not allowed here: collection is not initialized (did you call initialize_collection()?)."); pr.clear_request(pages_reserved); + drop(lock); // drop the lock before block VM::VMCollection::block_for_gc(VMMutatorThread(tls)); // We have checked that this is mutator unsafe { Address::zero() } } else { @@ -334,10 +384,14 @@ pub trait Space: 'static + SFT + Sync + Downcast { match pr.get_new_pages(self.common().descriptor, pages_reserved, pages, tls) { Ok(res) => { - // The following code was guarded by a page resource lock in Java MMTk. - // I think they are thread safe and we do not need a lock. So they - // are no longer guarded by a lock. If we see any issue here, considering - // adding a space lock here. + debug!( + "Got new pages {} ({} pages) for {} in chunk {}, new_chunk? {}", + res.start, + res.pages, + self.get_name(), + conversions::chunk_align_down(res.start), + res.new_chunk + ); let bytes = conversions::pages_to_bytes(res.pages); self.grow_space(res.start, bytes, res.new_chunk); // Mmap the pages and the side metadata, and handle error. In case of any error, @@ -360,6 +414,32 @@ pub trait Space: 'static + SFT + Sync + Downcast { memory::zero(res.start, bytes); } + // Some assertions + { + // --- Assert the start of the allocated region --- + // The start address SFT should be correct. + debug_assert_eq!(SFT_MAP.get(res.start).name(), self.get_name()); + // The start address is in our space. + debug_assert!(self.address_in_space(res.start)); + // The descriptor should be correct. + debug_assert_eq!( + self.common().vm_map().get_descriptor_for_address(res.start), + self.common().descriptor + ); + + // --- Assert the last byte in the allocated region --- + let last_byte = res.start + bytes - 1; + // The SFT for the last byte in the allocated memory should be correct. + debug_assert_eq!(SFT_MAP.get(last_byte).name(), self.get_name()); + // The last byte in the allocated memory should be in this space. + debug_assert!(self.address_in_space(last_byte)); + // The descriptor for the last byte should be correct. + debug_assert_eq!( + self.common().vm_map().get_descriptor_for_address(last_byte), + self.common().descriptor + ); + } + debug!("Space.acquire(), returned = {}", res.start); res.start } @@ -373,6 +453,7 @@ pub trait Space: 'static + SFT + Sync + Downcast { let gc_performed = VM::VMActivePlan::global().poll(true, self.as_space()); debug_assert!(gc_performed, "GC not performed when forced."); pr.clear_request(pages_reserved); + drop(lock); // drop the lock before block VM::VMCollection::block_for_gc(VMMutatorThread(tls)); // We asserted that this is mutator. unsafe { Address::zero() } } @@ -409,11 +490,14 @@ pub trait Space: 'static + SFT + Sync + Downcast { bytes, new_chunk ); - // FIXME: This assertion is too strict. See https://github.com/mmtk/mmtk-core/issues/374 - // debug_assert!( - // (new_chunk && start.is_aligned_to(BYTES_IN_CHUNK)) || !new_chunk, - // "should only grow space for new chunks at chunk-aligned start address" - // ); + + // If this is not a new chunk, the SFT for [start, start + bytes) should alreayd be initialized. + #[cfg(debug_assertions)] + if !new_chunk { + debug_assert!(SFT_MAP.get(start).name() != EMPTY_SFT_NAME, "In grow_space(start = {}, bytes = {}, new_chunk = {}), we have empty SFT entries (chunk for {} = {})", start, bytes, new_chunk, start, SFT_MAP.get(start).name()); + debug_assert!(SFT_MAP.get(start + bytes - 1).name() != EMPTY_SFT_NAME, "In grow_space(start = {}, bytes = {}, new_chunk = {}), we have empty SFT entries (chunk for {} = {}", start, bytes, new_chunk, start + bytes - 1, SFT_MAP.get(start + bytes - 1).name()); + } + if new_chunk { SFT_MAP.update(self.as_sft(), start, bytes); } @@ -549,6 +633,9 @@ pub struct CommonSpace { // TODO: This should be a constant for performance. pub needs_log_bit: bool, + /// A lock used during acquire() to make sure only one thread can allocate. + pub acquire_lock: Mutex<()>, + p: PhantomData, } @@ -589,6 +676,7 @@ impl CommonSpace { needs_log_bit: opt.needs_log_bit, metadata: opt.side_metadata_specs, p: PhantomData, + acquire_lock: Mutex::new(()), }; let vmrequest = opt.vmrequest; diff --git a/src/scheduler/gc_work.rs b/src/scheduler/gc_work.rs index 787d9f0f90..f7204df600 100644 --- a/src/scheduler/gc_work.rs +++ b/src/scheduler/gc_work.rs @@ -501,6 +501,10 @@ impl ProcessEdgesWork for SFTProcessEdges { return object; } + // Make sure we have valid SFT entries for the object. + #[cfg(debug_assertions)] + crate::mmtk::SFT_MAP.assert_valid_entries_for_object::(object); + // Erase type parameter let worker = GCWorkerMutRef::new(self.worker()); let trace = SFTProcessEdgesMutRef::new(self); diff --git a/src/util/alloc/allocator.rs b/src/util/alloc/allocator.rs index e5e66eb2ee..66bafc25e9 100644 --- a/src/util/alloc/allocator.rs +++ b/src/util/alloc/allocator.rs @@ -146,6 +146,10 @@ pub trait Allocator: Downcast { /// [`Collection::out_of_memory`] to inform the binding and then return a null pointer back to /// it. We have no assumptions on whether the VM will continue executing or abort immediately. /// + /// An allocator needs to make sure the object reference for the returned address is in the same + /// chunk as the returned address (so the side metadata and the SFT for an object reference is valid). + /// See [`crate::util::alloc::object_ref_guard`]. + /// /// Arguments: /// * `size`: the allocation size in bytes. /// * `align`: the required alignment in bytes. diff --git a/src/util/alloc/bumpallocator.rs b/src/util/alloc/bumpallocator.rs index 3dd946a0d8..41fdf992b0 100644 --- a/src/util/alloc/bumpallocator.rs +++ b/src/util/alloc/bumpallocator.rs @@ -1,4 +1,5 @@ use super::allocator::{align_allocation_no_fill, fill_alignment_gap}; +use super::object_ref_guard::adjust_thread_local_buffer_limit; use crate::util::Address; use crate::util::alloc::Allocator; @@ -25,7 +26,7 @@ pub struct BumpAllocator { impl BumpAllocator { pub fn set_limit(&mut self, cursor: Address, limit: Address) { self.cursor = cursor; - self.limit = limit; + self.limit = adjust_thread_local_buffer_limit::(limit); } pub fn reset(&mut self) { diff --git a/src/util/alloc/immix_allocator.rs b/src/util/alloc/immix_allocator.rs index 99eeafdae0..6b114e015b 100644 --- a/src/util/alloc/immix_allocator.rs +++ b/src/util/alloc/immix_allocator.rs @@ -1,4 +1,5 @@ use super::allocator::{align_allocation_no_fill, fill_alignment_gap}; +use super::object_ref_guard::adjust_thread_local_buffer_limit; use crate::plan::Plan; use crate::policy::immix::line::*; use crate::policy::immix::ImmixSpace; @@ -240,7 +241,7 @@ impl ImmixAllocator { if let Some(lines) = self.immix_space().get_next_available_lines(line) { // Find recyclable lines. Update the bump allocation cursor and limit. self.cursor = lines.start.start(); - self.limit = lines.end.start(); + self.limit = adjust_thread_local_buffer_limit::(lines.end.start()); trace!( "{:?}: acquire_recyclable_lines -> {:?} {:?} {:?}", self.tls, @@ -292,10 +293,10 @@ impl ImmixAllocator { trace!("{:?}: Acquired a new block {:?}", self.tls, block); if self.request_for_large { self.large_cursor = block.start(); - self.large_limit = block.end(); + self.large_limit = adjust_thread_local_buffer_limit::(block.end()); } else { self.cursor = block.start(); - self.limit = block.end(); + self.limit = adjust_thread_local_buffer_limit::(block.end()); } self.alloc(size, align, offset) } diff --git a/src/util/alloc/large_object_allocator.rs b/src/util/alloc/large_object_allocator.rs index c476db7971..d32a50eb76 100644 --- a/src/util/alloc/large_object_allocator.rs +++ b/src/util/alloc/large_object_allocator.rs @@ -32,6 +32,9 @@ impl Allocator for LargeObjectAllocator { } fn alloc(&mut self, size: usize, align: usize, offset: isize) -> Address { + #[cfg(debug_assertions)] + crate::util::alloc::object_ref_guard::assert_object_ref_in_cell::(size); + let cell: Address = self.alloc_slow(size, align, offset); // We may get a null ptr from alloc due to the VM being OOM if !cell.is_zero() { diff --git a/src/util/alloc/malloc_allocator.rs b/src/util/alloc/malloc_allocator.rs index 0fee3d5ce9..41200889b7 100644 --- a/src/util/alloc/malloc_allocator.rs +++ b/src/util/alloc/malloc_allocator.rs @@ -1,5 +1,6 @@ use crate::policy::mallocspace::MallocSpace; use crate::policy::space::Space; +use crate::util::alloc::object_ref_guard::object_ref_may_cross_chunk; use crate::util::alloc::Allocator; use crate::util::opaque_pointer::*; use crate::util::Address; @@ -36,14 +37,28 @@ impl Allocator for MallocAllocator { assert!(offset >= 0); let ret = self.space.alloc(self.tls, size, align, offset); - trace!( - "MallocSpace.alloc size = {}, align = {}, offset = {}, res = {}", - size, - align, - offset, + if !object_ref_may_cross_chunk::(ret) { ret - ); - ret + } else { + // The address we got does not pass object ref checks. We cache it and free it later. + // We free the results in the end to avoid malloc giving us the free'd address again. + // The creation of the vec is put here so for the common case where we succeed in the first allocation, + // we do not need to create this vec. + let mut to_free = vec![ret]; + loop { + let ret = self.space.alloc(self.tls, size, align, offset); + if object_ref_may_cross_chunk::(ret) { + // The result does not pass check. Cache it. + to_free.push(ret); + } else { + // The result passes the check. We free all the cached results, and return the new result. + for addr in to_free.iter() { + self.space.free(*addr); + } + return ret; + } + } + } } } diff --git a/src/util/alloc/mod.rs b/src/util/alloc/mod.rs index 7bf7529a78..e9abddc823 100644 --- a/src/util/alloc/mod.rs +++ b/src/util/alloc/mod.rs @@ -1,24 +1,37 @@ +///! Various allocators implementation. + +/// The allocator trait and allocation-related functions. pub(crate) mod allocator; pub use allocator::fill_alignment_gap; pub use allocator::AllocationError; pub use allocator::Allocator; +/// Functions to ensure an object reference for an allocation has valid metadata. +mod object_ref_guard; + +/// A list of all the allocators, embedded in Mutator pub(crate) mod allocators; pub use allocators::AllocatorSelector; +/// Bump pointer allocator mod bumpallocator; pub use bumpallocator::BumpAllocator; +/// Large object allocator mod large_object_allocator; pub use large_object_allocator::LargeObjectAllocator; +/// An alloactor backed by malloc mod malloc_allocator; pub use malloc_allocator::MallocAllocator; +/// Immix allocator pub mod immix_allocator; pub use self::immix_allocator::ImmixAllocator; +/// Mark compact allocator (actually a bump pointer allocator with an extra heade word) mod markcompact_allocator; pub use markcompact_allocator::MarkCompactAllocator; +/// Embedded metadata pages pub(crate) mod embedded_meta_data; diff --git a/src/util/alloc/object_ref_guard.rs b/src/util/alloc/object_ref_guard.rs new file mode 100644 index 0000000000..43ed549929 --- /dev/null +++ b/src/util/alloc/object_ref_guard.rs @@ -0,0 +1,65 @@ +//! This module includes functions to make sure the following invariant always holds: for each object we allocate (`[cell, cell + bytes)`), the metadata for +//! the object reference (`object_ref`) is always in the range of the allocated memory. Given that we always initialize metadata based on chunks, +//! we simply need to make sure that `object_ref` is in the same chunk as `[cell, cell + bytes)`. In other words, we avoid +//! allocating an address for which the object reference may be in another chunk. +//! +//! Note that where an ObjectReference points to is defined by a binding. We only have this problem if an object reference may point +//! to an address that is outside our allocated memory (`object_ref >= cell + bytes`). We ask a binding to specify +//! `ObjectModel::OBJECT_REF_OFFSET_BEYOND_CELL` if their object reference may point to an address outside the allocated +//! memory. `ObjectModel::OBJECT_REF_OFFSET_BEYOND_CELL` should be the max of `object_ref - cell`. +//! +//! There are various ways we deal with this. +//! * For allocators that have a thread local buffer, we can adjust the buffer limit to make sure that the last object allocated in the +//! buffer won't cross chunks. +//! * For allocators that allocate large objects, if the object size is larger than `OBJECT_REF_OFFSET_BEYOND_CELL`, it is guaranteed that +//! the object reference is within the allocated memory. +//! * For other allocators, we can check if the allocation result violates this invariant. + +use crate::util::heap::layout::vm_layout_constants::{BYTES_IN_CHUNK, CHUNK_MASK}; +use crate::util::Address; +use crate::vm::ObjectModel; +use crate::vm::VMBinding; + +/// Adjust limit for thread local buffer to make sure that we will not allocate objects whose object reference may +/// be in another chunk. +pub fn adjust_thread_local_buffer_limit(limit: Address) -> Address { + // We only need to adjust limit when the binding tells us that + // object ref may point outside the allocated memory and when limit is at chunk boundary + if let Some(offset) = VM::VMObjectModel::OBJECT_REF_OFFSET_BEYOND_CELL { + if limit.is_aligned_to(BYTES_IN_CHUNK) { + debug_assert!(limit.as_usize() > offset); + // We simply not use the last few bytes. This is a rare case anyway (expect less than 1% of slowpath allocation goes here). + // It should be possible for us to check if we can use the last few bytes to finish an allocation request when we 'exhaust' + // thread local buffer. But probably it won't give us much benefit and it complicates our allocation code. + return limit - offset; + } + } + + limit +} + +/// Assert that the object reference should always inside the allocation cell +#[cfg(debug_assertions)] +pub fn assert_object_ref_in_cell(size: usize) { + if VM::VMObjectModel::OBJECT_REF_OFFSET_BEYOND_CELL.is_none() { + return; + } + + // If the object ref offset is smaller than size, it is always inside the allocation cell. + debug_assert!( + size > VM::VMObjectModel::OBJECT_REF_OFFSET_BEYOND_CELL.unwrap(), + "Allocating objects of size {} may cross chunk (OBJECT_REF_OFFSET_BEYOND_CELL = {})", + size, + VM::VMObjectModel::OBJECT_REF_OFFSET_BEYOND_CELL.unwrap() + ); +} + +/// Check if the object reference for this allocation may cross and fall into the next chunk. +pub fn object_ref_may_cross_chunk(addr: Address) -> bool { + if VM::VMObjectModel::OBJECT_REF_OFFSET_BEYOND_CELL.is_none() { + return false; + } + + (addr & CHUNK_MASK) + VM::VMObjectModel::OBJECT_REF_OFFSET_BEYOND_CELL.unwrap() + >= BYTES_IN_CHUNK +} diff --git a/src/util/heap/layout/map.rs b/src/util/heap/layout/map.rs index 8be6432785..2df96c30b1 100644 --- a/src/util/heap/layout/map.rs +++ b/src/util/heap/layout/map.rs @@ -1,6 +1,5 @@ use crate::util::generic_freelist::GenericFreeList; use crate::util::heap::freelistpageresource::CommonFreeListPageResource; -use crate::util::heap::layout::vm_layout_constants::*; use crate::util::heap::space_descriptor::SpaceDescriptor; use crate::util::Address; @@ -47,13 +46,5 @@ pub trait Map: Sized { fn get_descriptor_for_address(&self, address: Address) -> SpaceDescriptor; - fn get_chunk_index(&self, address: Address) -> usize { - address >> LOG_BYTES_IN_CHUNK - } - - fn address_for_chunk_index(&self, chunk: usize) -> Address { - unsafe { Address::from_usize(chunk << LOG_BYTES_IN_CHUNK) } - } - fn add_to_cumulative_committed_pages(&self, pages: usize); } diff --git a/src/util/heap/layout/map32.rs b/src/util/heap/layout/map32.rs index e1254f077e..f990bd3944 100644 --- a/src/util/heap/layout/map32.rs +++ b/src/util/heap/layout/map32.rs @@ -55,11 +55,16 @@ impl Map for Map32 { let self_mut: &mut Self = unsafe { self.mut_self() }; let mut e = 0; while e < extent { - let index = self.get_chunk_index(start + e); + let index = (start + e).chunk_index(); assert!( self.descriptor_map[index].is_empty(), "Conflicting virtual address request" ); + debug!( + "Set descriptor {:?} for Chunk {}", + descriptor, + conversions::chunk_index_to_address(index) + ); self_mut.descriptor_map[index] = descriptor; // VM.barriers.objectArrayStoreNoGCBarrier(spaceMap, index, space); e += BYTES_IN_CHUNK; @@ -95,13 +100,13 @@ impl Map for Map32 { return unsafe { Address::zero() }; } self_mut.total_available_discontiguous_chunks -= chunks; - let rtn = self.address_for_chunk_index(chunk as _); + let rtn = conversions::chunk_index_to_address(chunk as _); self.insert(rtn, chunks << LOG_BYTES_IN_CHUNK, descriptor); if head.is_zero() { debug_assert!(self.next_link[chunk as usize] == 0); } else { - self_mut.next_link[chunk as usize] = self.get_chunk_index(head) as _; - self_mut.prev_link[self.get_chunk_index(head)] = chunk; + self_mut.next_link[chunk as usize] = head.chunk_index() as _; + self_mut.prev_link[head.chunk_index()] = chunk; } debug_assert!(self.prev_link[chunk as usize] == 0); rtn @@ -109,18 +114,18 @@ impl Map for Map32 { fn get_next_contiguous_region(&self, start: Address) -> Address { debug_assert!(start == conversions::chunk_align_down(start)); - let chunk = self.get_chunk_index(start); + let chunk = start.chunk_index(); if chunk == 0 || self.next_link[chunk] == 0 { unsafe { Address::zero() } } else { let a = self.next_link[chunk]; - self.address_for_chunk_index(a as _) + conversions::chunk_index_to_address(a as _) } } fn get_contiguous_region_chunks(&self, start: Address) -> usize { debug_assert!(start == conversions::chunk_align_down(start)); - let chunk = self.get_chunk_index(start); + let chunk = start.chunk_index(); self.region_map.size(chunk as i32) as _ } @@ -129,10 +134,11 @@ impl Map for Map32 { } fn free_all_chunks(&self, any_chunk: Address) { + debug!("free_all_chunks: {}", any_chunk); let (_sync, self_mut) = self.mut_self_with_sync(); debug_assert!(any_chunk == conversions::chunk_align_down(any_chunk)); if !any_chunk.is_zero() { - let chunk = self.get_chunk_index(any_chunk); + let chunk = any_chunk.chunk_index(); while self_mut.next_link[chunk] != 0 { let x = self_mut.next_link[chunk]; self_mut.free_contiguous_chunks_no_lock(x); @@ -146,9 +152,10 @@ impl Map for Map32 { } fn free_contiguous_chunks(&self, start: Address) -> usize { + debug!("free_contiguous_chunks: {}", start); let (_sync, self_mut) = self.mut_self_with_sync(); debug_assert!(start == conversions::chunk_align_down(start)); - let chunk = self.get_chunk_index(start); + let chunk = start.chunk_index(); self_mut.free_contiguous_chunks_no_lock(chunk as _) } @@ -158,8 +165,8 @@ impl Map for Map32 { let self_mut: &mut Self = unsafe { self.mut_self() }; /* establish bounds of discontiguous space */ let start_address = from; - let first_chunk = self.get_chunk_index(start_address); - let last_chunk = self.get_chunk_index(to); + let first_chunk = start_address.chunk_index(); + let last_chunk = to.chunk_index(); let unavail_start_chunk = last_chunk + 1; let trailing_chunks = MAX_CHUNKS - unavail_start_chunk; let pages = (1 + last_chunk - first_chunk) * PAGES_IN_CHUNK; @@ -227,7 +234,7 @@ impl Map for Map32 { } fn get_descriptor_for_address(&self, address: Address) -> SpaceDescriptor { - let index = self.get_chunk_index(address); + let index = address.chunk_index(); self.descriptor_map[index] } @@ -268,8 +275,13 @@ impl Map32 { self.prev_link[chunk as usize] = 0; self.next_link[chunk as usize] = 0; for offset in 0..chunks { - self.descriptor_map[(chunk + offset) as usize] = SpaceDescriptor::UNINITIALIZED; - SFT_MAP.clear_by_index((chunk + offset) as usize); + let index = (chunk + offset) as usize; + debug!( + "Clear descriptor for Chunk {}", + conversions::chunk_index_to_address(index) + ); + self.descriptor_map[index] = SpaceDescriptor::UNINITIALIZED; + SFT_MAP.clear_by_index(index); // VM.barriers.objectArrayStoreNoGCBarrier(spaceMap, chunk + offset, null); } chunks as _ diff --git a/src/util/heap/layout/vm_layout_constants.rs b/src/util/heap/layout/vm_layout_constants.rs index 5750d9e457..fd63f91a44 100644 --- a/src/util/heap/layout/vm_layout_constants.rs +++ b/src/util/heap/layout/vm_layout_constants.rs @@ -24,6 +24,7 @@ pub const LOG_BYTES_IN_CHUNK: usize = 22; /** Coarsest unit of address space allocation. */ pub const BYTES_IN_CHUNK: usize = 1 << LOG_BYTES_IN_CHUNK; +pub const CHUNK_MASK: usize = (1 << LOG_BYTES_IN_CHUNK) - 1; /** Coarsest unit of address space allocation, in pages */ pub const PAGES_IN_CHUNK: usize = 1 << (LOG_BYTES_IN_CHUNK as usize - LOG_BYTES_IN_PAGE as usize); diff --git a/src/util/metadata/side_metadata/constants.rs b/src/util/metadata/side_metadata/constants.rs index 41a2b99388..38f23f89e4 100644 --- a/src/util/metadata/side_metadata/constants.rs +++ b/src/util/metadata/side_metadata/constants.rs @@ -1,6 +1,6 @@ -use crate::util::heap::layout::vm_layout_constants::LOG_ADDRESS_SPACE; #[cfg(target_pointer_width = "32")] -use crate::util::heap::layout::vm_layout_constants::{BYTES_IN_CHUNK, LOG_BYTES_IN_CHUNK}; +use crate::util::heap::layout::vm_layout_constants::BYTES_IN_CHUNK; +use crate::util::heap::layout::vm_layout_constants::LOG_ADDRESS_SPACE; use crate::util::metadata::side_metadata::SideMetadataOffset; use crate::util::Address; @@ -62,9 +62,6 @@ pub(crate) const LOCAL_SIDE_METADATA_BASE_OFFSET: SideMetadataOffset = SideMetad pub(crate) const LOCAL_SIDE_METADATA_BASE_OFFSET: SideMetadataOffset = SideMetadataOffset::addr(LOCAL_SIDE_METADATA_BASE_ADDRESS); -#[cfg(target_pointer_width = "32")] -pub(super) const CHUNK_MASK: usize = (1 << LOG_BYTES_IN_CHUNK) - 1; - #[cfg(target_pointer_width = "32")] pub(super) const LOCAL_SIDE_METADATA_PER_CHUNK: usize = BYTES_IN_CHUNK >> LOG_LOCAL_SIDE_METADATA_WORST_CASE_RATIO; diff --git a/src/util/metadata/side_metadata/helpers_32.rs b/src/util/metadata/side_metadata/helpers_32.rs index 67eb3a81b0..a932d782e5 100644 --- a/src/util/metadata/side_metadata/helpers_32.rs +++ b/src/util/metadata/side_metadata/helpers_32.rs @@ -1,13 +1,13 @@ use super::SideMetadataSpec; use crate::util::{ constants::{self, LOG_BITS_IN_BYTE}, - heap::layout::vm_layout_constants::{BYTES_IN_CHUNK, LOG_BYTES_IN_CHUNK}, + heap::layout::vm_layout_constants::{BYTES_IN_CHUNK, CHUNK_MASK, LOG_BYTES_IN_CHUNK}, memory, Address, }; use std::io::Result; use super::constants::{ - CHUNK_MASK, LOCAL_SIDE_METADATA_BASE_ADDRESS, LOCAL_SIDE_METADATA_PER_CHUNK, + LOCAL_SIDE_METADATA_BASE_ADDRESS, LOCAL_SIDE_METADATA_PER_CHUNK, LOG_LOCAL_SIDE_METADATA_WORST_CASE_RATIO, }; #[cfg(test)] diff --git a/src/util/object_forwarding.rs b/src/util/object_forwarding.rs index 115cc0f6cb..4b167cb463 100644 --- a/src/util/object_forwarding.rs +++ b/src/util/object_forwarding.rs @@ -98,6 +98,8 @@ pub fn forward_object( Some(Ordering::SeqCst), ); } + #[cfg(debug_assertions)] + crate::mmtk::SFT_MAP.assert_valid_entries_for_object::(new_object); new_object } diff --git a/src/vm/object_model.rs b/src/vm/object_model.rs index afde2aa465..efab7a9581 100644 --- a/src/vm/object_model.rs +++ b/src/vm/object_model.rs @@ -248,6 +248,15 @@ pub trait ObjectModel { /// * `reference`: The object to be queried. fn get_type_descriptor(reference: ObjectReference) -> &'static [i8]; + /// For our allocation result `[cell, cell + bytes)`, if a binding's + /// definition of `ObjectReference` may point outside the cell (i.e. `object_ref >= cell + bytes`), + /// the binding needs to provide a `Some` value for this constant and + /// the value is the maximum of `object_ref - cell`. If a binding's + /// `ObjectReference` always points to an address in the cell (i.e. `[cell, cell + bytes)`), + /// they can leave this as `None`. + /// MMTk allocators use this value to make sure that the metadata for object reference is properly set. + const OBJECT_REF_OFFSET_BEYOND_CELL: Option = None; + /// Return the lowest address of the storage associated with an object. /// /// Arguments: