Skip to content

Commit

Permalink
Fix object ref SFT (mmtk#555)
Browse files Browse the repository at this point in the history
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 mmtk/mmtk-jikesrvm#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`.
  • Loading branch information
qinsoon authored and tianleq committed Apr 4, 2022
1 parent 7a57f11 commit 775e25e
Show file tree
Hide file tree
Showing 17 changed files with 290 additions and 78 deletions.
10 changes: 8 additions & 2 deletions src/policy/mallocspace/global.rs
Original file line number Diff line number Diff line change
Expand Up @@ -240,9 +240,15 @@ impl<VM: VMBinding> MallocSpace<VM> {
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);
Expand Down Expand Up @@ -373,7 +379,7 @@ impl<VM: VMBinding> MallocSpace<VM> {
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) };

Expand Down
158 changes: 123 additions & 35 deletions src/policy/space.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ use crate::util::memory;

use crate::vm::VMBinding;
use std::marker::PhantomData;
use std::sync::Mutex;

use downcast_rs::Downcast;

Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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()),
)
}
}
Expand Down Expand Up @@ -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<usize> = (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<usize> = (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.
Expand All @@ -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);
Expand All @@ -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)
}

Expand Down Expand Up @@ -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<VM: VMBinding>(&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<VM: VMBinding>: 'static + SFT + Sync + Downcast {
Expand All @@ -317,6 +360,12 @@ pub trait Space<VM: VMBinding>: '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);
Expand All @@ -327,17 +376,22 @@ pub trait Space<VM: VMBinding>: '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 {
debug!("Collection not required");

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,
Expand All @@ -360,6 +414,32 @@ pub trait Space<VM: VMBinding>: '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
}
Expand All @@ -373,6 +453,7 @@ pub trait Space<VM: VMBinding>: '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() }
}
Expand Down Expand Up @@ -409,11 +490,14 @@ pub trait Space<VM: VMBinding>: '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);
}
Expand Down Expand Up @@ -549,6 +633,9 @@ pub struct CommonSpace<VM: VMBinding> {
// 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<VM>,
}

Expand Down Expand Up @@ -589,6 +676,7 @@ impl<VM: VMBinding> CommonSpace<VM> {
needs_log_bit: opt.needs_log_bit,
metadata: opt.side_metadata_specs,
p: PhantomData,
acquire_lock: Mutex::new(()),
};

let vmrequest = opt.vmrequest;
Expand Down
4 changes: 4 additions & 0 deletions src/scheduler/gc_work.rs
Original file line number Diff line number Diff line change
Expand Up @@ -501,6 +501,10 @@ impl<VM: VMBinding> ProcessEdgesWork for SFTProcessEdges<VM> {
return object;
}

// Make sure we have valid SFT entries for the object.
#[cfg(debug_assertions)]
crate::mmtk::SFT_MAP.assert_valid_entries_for_object::<VM>(object);

// Erase <VM> type parameter
let worker = GCWorkerMutRef::new(self.worker());
let trace = SFTProcessEdgesMutRef::new(self);
Expand Down
4 changes: 4 additions & 0 deletions src/util/alloc/allocator.rs
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,10 @@ pub trait Allocator<VM: VMBinding>: 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.
Expand Down
3 changes: 2 additions & 1 deletion src/util/alloc/bumpallocator.rs
Original file line number Diff line number Diff line change
@@ -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;
Expand All @@ -25,7 +26,7 @@ pub struct BumpAllocator<VM: VMBinding> {
impl<VM: VMBinding> BumpAllocator<VM> {
pub fn set_limit(&mut self, cursor: Address, limit: Address) {
self.cursor = cursor;
self.limit = limit;
self.limit = adjust_thread_local_buffer_limit::<VM>(limit);
}

pub fn reset(&mut self) {
Expand Down
7 changes: 4 additions & 3 deletions src/util/alloc/immix_allocator.rs
Original file line number Diff line number Diff line change
@@ -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;
Expand Down Expand Up @@ -240,7 +241,7 @@ impl<VM: VMBinding> ImmixAllocator<VM> {
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::<VM>(lines.end.start());
trace!(
"{:?}: acquire_recyclable_lines -> {:?} {:?} {:?}",
self.tls,
Expand Down Expand Up @@ -292,10 +293,10 @@ impl<VM: VMBinding> ImmixAllocator<VM> {
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::<VM>(block.end());
} else {
self.cursor = block.start();
self.limit = block.end();
self.limit = adjust_thread_local_buffer_limit::<VM>(block.end());
}
self.alloc(size, align, offset)
}
Expand Down
3 changes: 3 additions & 0 deletions src/util/alloc/large_object_allocator.rs
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,9 @@ impl<VM: VMBinding> Allocator<VM> for LargeObjectAllocator<VM> {
}

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::<VM>(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() {
Expand Down
Loading

0 comments on commit 775e25e

Please sign in to comment.