提交 ad3a5b8a 编写于 作者: T tschatzl

8054818: Refactor HeapRegionSeq to manage heap region and auxiliary data

Summary: Let HeapRegionSeq manage the heap region and auxiliary data to decrease the amount of responsibilities of G1CollectedHeap, and encapsulate this work from other code.
Reviewed-by: jwilhelm, jmasa, mgerdin, brutisso
上级 60259367
......@@ -45,8 +45,8 @@ import sun.jvm.hotspot.types.TypeDataBase;
public class G1CollectedHeap extends SharedHeap {
// HeapRegionSeq _seq;
static private long hrsFieldOffset;
// MemRegion _g1_committed;
static private long g1CommittedFieldOffset;
// MemRegion _g1_reserved;
static private long g1ReservedFieldOffset;
// size_t _summary_bytes_used;
static private CIntegerField summaryBytesUsedField;
// G1MonitoringSupport* _g1mm;
......@@ -68,7 +68,6 @@ public class G1CollectedHeap extends SharedHeap {
Type type = db.lookupType("G1CollectedHeap");
hrsFieldOffset = type.getField("_hrs").getOffset();
g1CommittedFieldOffset = type.getField("_g1_committed").getOffset();
summaryBytesUsedField = type.getCIntegerField("_summary_bytes_used");
g1mmField = type.getAddressField("_g1mm");
oldSetFieldOffset = type.getField("_old_set").getOffset();
......@@ -76,9 +75,7 @@ public class G1CollectedHeap extends SharedHeap {
}
public long capacity() {
Address g1CommittedAddr = addr.addOffsetTo(g1CommittedFieldOffset);
MemRegion g1Committed = new MemRegion(g1CommittedAddr);
return g1Committed.byteSize();
return hrs().capacity();
}
public long used() {
......
......@@ -93,19 +93,35 @@ public class G1HeapRegionTable extends VMObject {
private class HeapRegionIterator implements Iterator<HeapRegion> {
private long index;
private long length;
private HeapRegion next;
public HeapRegion positionToNext() {
HeapRegion result = next;
while (index < length && at(index) == null) {
index++;
}
if (index < length) {
next = at(index);
index++; // restart search at next element
} else {
next = null;
}
return result;
}
@Override
public boolean hasNext() { return index < length; }
public boolean hasNext() { return next != null; }
@Override
public HeapRegion next() { return at(index++); }
public HeapRegion next() { return positionToNext(); }
@Override
public void remove() { /* not supported */ }
public void remove() { /* not supported */ }
HeapRegionIterator(long committedLength) {
HeapRegionIterator(long totalLength) {
index = 0;
length = committedLength;
length = totalLength;
positionToNext();
}
}
......
......@@ -43,7 +43,7 @@ public class HeapRegionSeq extends VMObject {
// G1HeapRegionTable _regions
static private long regionsFieldOffset;
// uint _committed_length
static private CIntegerField committedLengthField;
static private CIntegerField numCommittedField;
static {
VM.registerVMInitializedObserver(new Observer() {
......@@ -57,7 +57,7 @@ public class HeapRegionSeq extends VMObject {
Type type = db.lookupType("HeapRegionSeq");
regionsFieldOffset = type.getField("_regions").getOffset();
committedLengthField = type.getCIntegerField("_committed_length");
numCommittedField = type.getCIntegerField("_num_committed");
}
private G1HeapRegionTable regions() {
......@@ -66,16 +66,20 @@ public class HeapRegionSeq extends VMObject {
regionsAddr);
}
public long capacity() {
return length() * HeapRegion.grainBytes();
}
public long length() {
return regions().length();
}
public long committedLength() {
return committedLengthField.getValue(addr);
return numCommittedField.getValue(addr);
}
public Iterator<HeapRegion> heapRegionIterator() {
return regions().heapRegionIterator(committedLength());
return regions().heapRegionIterator(length());
}
public HeapRegionSeq(Address addr) {
......
......@@ -728,14 +728,13 @@ ConcurrentMark::ConcurrentMark(G1CollectedHeap* g1h, ReservedSpace heap_rs) :
_completed_initialization = true;
}
void ConcurrentMark::update_g1_committed(bool force) {
void ConcurrentMark::update_heap_boundaries(MemRegion bounds, bool force) {
// If concurrent marking is not in progress, then we do not need to
// update _heap_end.
if (!concurrent_marking_in_progress() && !force) return;
MemRegion committed = _g1h->g1_committed();
assert(committed.start() == _heap_start, "start shouldn't change");
HeapWord* new_end = committed.end();
assert(bounds.start() == _heap_start, "start shouldn't change");
HeapWord* new_end = bounds.end();
if (new_end > _heap_end) {
// The heap has been expanded.
......@@ -826,7 +825,7 @@ void ConcurrentMark::set_concurrency_and_phase(uint active_tasks, bool concurren
assert(out_of_regions(),
err_msg("only way to get here: _finger: "PTR_FORMAT", _heap_end: "PTR_FORMAT,
p2i(_finger), p2i(_heap_end)));
update_g1_committed(true);
update_heap_boundaries(_g1h->g1_committed(), true);
}
}
......@@ -2194,7 +2193,7 @@ void ConcurrentMark::completeCleanup() {
// Noone else should be accessing the _cleanup_list at this point,
// so it's not necessary to take any locks
while (!_cleanup_list.is_empty()) {
HeapRegion* hr = _cleanup_list.remove_head();
HeapRegion* hr = _cleanup_list.remove_region(true /* from_head */);
assert(hr != NULL, "Got NULL from a non-empty list");
hr->par_clear();
tmp_free_list.add_ordered(hr);
......
......@@ -798,7 +798,7 @@ public:
// that CM is notified of where the new end of the heap is. It
// doesn't do anything if concurrent_marking_in_progress() is false,
// unless the force parameter is true.
void update_g1_committed(bool force = false);
void update_heap_boundaries(MemRegion bounds, bool force = false);
bool isMarked(oop p) const {
assert(p != NULL && p->is_oop(), "expected an oop");
......
......@@ -675,8 +675,7 @@ G1BlockOffsetArrayContigSpace::set_for_starts_humongous(HeapWord* new_top) {
assert(new_top <= _end, "_end should have already been updated");
// The first BOT entry should have offset 0.
zero_bottom_entry();
initialize_threshold();
reset_bot();
alloc_block(_bottom, new_top);
}
......
......@@ -231,10 +231,6 @@ public:
void set_bottom(HeapWord* new_bottom);
// Updates all the BlockOffsetArray's sharing this shared array to
// reflect the current "top"'s of their spaces.
void update_offset_arrays();
// Return the appropriate index into "_offset_array" for "p".
inline size_t index_for(const void* p) const;
......@@ -480,6 +476,8 @@ class G1BlockOffsetArrayContigSpace: public G1BlockOffsetArray {
blk_start, blk_end);
}
// Zero out the entry for _bottom (offset will be zero).
void zero_bottom_entry();
public:
G1BlockOffsetArrayContigSpace(G1BlockOffsetSharedArray* array, MemRegion mr);
......@@ -487,8 +485,10 @@ class G1BlockOffsetArrayContigSpace: public G1BlockOffsetArray {
// bottom of the covered region.
HeapWord* initialize_threshold();
// Zero out the entry for _bottom (offset will be zero).
void zero_bottom_entry();
void reset_bot() {
zero_bottom_entry();
initialize_threshold();
}
// Return the next threshold, the point at which the table should be
// updated.
......
......@@ -244,19 +244,9 @@ private:
static size_t _humongous_object_threshold_in_words;
// Storage for the G1 heap.
VirtualSpace _g1_storage;
MemRegion _g1_reserved;
// The part of _g1_storage that is currently committed.
MemRegion _g1_committed;
// The master free list. It will satisfy all new region allocations.
FreeRegionList _free_list;
// The secondary free list which contains regions that have been
// freed up during the cleanup process. This will be appended to the
// master free list when appropriate.
// freed up during the cleanup process. This will be appended to
// the master free list when appropriate.
FreeRegionList _secondary_free_list;
// It keeps track of the old regions.
......@@ -520,14 +510,6 @@ protected:
// humongous object, set is_old to true. If not, to false.
HeapRegion* new_region(size_t word_size, bool is_old, bool do_expand);
// Attempt to satisfy a humongous allocation request of the given
// size by finding a contiguous set of free regions of num_regions
// length and remove them from the master free list. Return the
// index of the first region or G1_NULL_HRS_INDEX if the search
// was unsuccessful.
uint humongous_obj_allocate_find_first(uint num_regions,
size_t word_size);
// Initialize a contiguous set of free regions of length num_regions
// and starting at index first so that they appear as a single
// humongous region.
......@@ -1193,27 +1175,20 @@ public:
virtual size_t unsafe_max_alloc();
virtual bool is_maximal_no_gc() const {
return _g1_storage.uncommitted_size() == 0;
return _hrs.available() == 0;
}
// The total number of regions in the heap.
uint n_regions() const { return _hrs.length(); }
// The current number of regions in the heap.
uint num_regions() const { return _hrs.length(); }
// The max number of regions in the heap.
uint max_regions() const { return _hrs.max_length(); }
// The number of regions that are completely free.
uint free_regions() const { return _free_list.length(); }
uint num_free_regions() const { return _hrs.num_free_regions(); }
// The number of regions that are not completely free.
uint used_regions() const { return n_regions() - free_regions(); }
// The number of regions available for "regular" expansion.
uint expansion_regions() const { return _expansion_regions; }
// Factory method for HeapRegion instances. It will return NULL if
// the allocation fails.
HeapRegion* new_heap_region(uint hrs_index, HeapWord* bottom);
uint num_used_regions() const { return num_regions() - num_free_regions(); }
void verify_not_dirty_region(HeapRegion* hr) PRODUCT_RETURN;
void verify_dirty_region(HeapRegion* hr) PRODUCT_RETURN;
......@@ -1262,7 +1237,7 @@ public:
#ifdef ASSERT
bool is_on_master_free_list(HeapRegion* hr) {
return hr->containing_set() == &_free_list;
return _hrs.is_free(hr);
}
#endif // ASSERT
......@@ -1274,7 +1249,7 @@ public:
}
void append_secondary_free_list() {
_free_list.add_ordered(&_secondary_free_list);
_hrs.insert_list_into_free_list(&_secondary_free_list);
}
void append_secondary_free_list_if_not_empty_with_lock() {
......@@ -1380,19 +1355,19 @@ public:
// Return "TRUE" iff the given object address is in the reserved
// region of g1.
bool is_in_g1_reserved(const void* p) const {
return _g1_reserved.contains(p);
return _hrs.reserved().contains(p);
}
// Returns a MemRegion that corresponds to the space that has been
// reserved for the heap
MemRegion g1_reserved() {
return _g1_reserved;
MemRegion g1_reserved() const {
return _hrs.reserved();
}
// Returns a MemRegion that corresponds to the space that has been
// committed in the heap
MemRegion g1_committed() {
return _g1_committed;
return _hrs.committed();
}
virtual bool is_in_closed_subset(const void* p) const;
......@@ -1432,6 +1407,8 @@ public:
// within the heap.
inline uint addr_to_region(HeapWord* addr) const;
inline HeapWord* bottom_addr_for_region(uint index) const;
// Divide the heap region sequence into "chunks" of some size (the number
// of regions divided by the number of parallel threads times some
// overpartition factor, currently 4). Assumes that this will be called
......@@ -1445,10 +1422,10 @@ public:
// setting the claim value of the second and subsequent regions of the
// chunk.) For now requires that "doHeapRegion" always returns "false",
// i.e., that a closure never attempt to abort a traversal.
void heap_region_par_iterate_chunked(HeapRegionClosure* blk,
uint worker,
uint no_of_par_workers,
jint claim_value);
void heap_region_par_iterate_chunked(HeapRegionClosure* cl,
uint worker_id,
uint num_workers,
jint claim_value) const;
// It resets all the region claim values to the default.
void reset_heap_region_claim_values();
......@@ -1473,11 +1450,6 @@ public:
// starting region for iterating over the current collection set.
HeapRegion* start_cset_region_for_worker(uint worker_i);
// This is a convenience method that is used by the
// HeapRegionIterator classes to calculate the starting region for
// each worker so that they do not all start from the same region.
HeapRegion* start_region_for_worker(uint worker_i, uint no_of_par_workers);
// Iterate over the regions (if any) in the current collection set.
void collection_set_iterate(HeapRegionClosure* blk);
......
......@@ -47,19 +47,21 @@ inline uint G1CollectedHeap::addr_to_region(HeapWord* addr) const {
return (uint)(pointer_delta(addr, _reserved.start(), sizeof(uint8_t)) >> HeapRegion::LogOfHRGrainBytes);
}
inline HeapWord* G1CollectedHeap::bottom_addr_for_region(uint index) const {
return _hrs.reserved().start() + index * HeapRegion::GrainWords;
}
template <class T>
inline HeapRegion*
G1CollectedHeap::heap_region_containing_raw(const T addr) const {
inline HeapRegion* G1CollectedHeap::heap_region_containing_raw(const T addr) const {
assert(addr != NULL, "invariant");
assert(_g1_reserved.contains((const void*) addr),
assert(is_in_g1_reserved((const void*) addr),
err_msg("Address "PTR_FORMAT" is outside of the heap ranging from ["PTR_FORMAT" to "PTR_FORMAT")",
p2i((void*)addr), p2i(_g1_reserved.start()), p2i(_g1_reserved.end())));
p2i((void*)addr), p2i(g1_reserved().start()), p2i(g1_reserved().end())));
return _hrs.addr_to_region((HeapWord*) addr);
}
template <class T>
inline HeapRegion*
G1CollectedHeap::heap_region_containing(const T addr) const {
inline HeapRegion* G1CollectedHeap::heap_region_containing(const T addr) const {
HeapRegion* hr = heap_region_containing_raw(addr);
if (hr->continuesHumongous()) {
return hr->humongous_start_region();
......@@ -89,10 +91,9 @@ inline bool G1CollectedHeap::obj_in_cs(oop obj) {
return r != NULL && r->in_collection_set();
}
inline HeapWord*
G1CollectedHeap::attempt_allocation(size_t word_size,
unsigned int* gc_count_before_ret,
int* gclocker_retry_count_ret) {
inline HeapWord* G1CollectedHeap::attempt_allocation(size_t word_size,
unsigned int* gc_count_before_ret,
int* gclocker_retry_count_ret) {
assert_heap_not_locked_and_not_at_safepoint();
assert(!isHumongous(word_size), "attempt_allocation() should not "
"be called for humongous allocation requests");
......@@ -252,8 +253,7 @@ G1CollectedHeap::set_evacuation_failure_alot_for_current_gc() {
}
}
inline bool
G1CollectedHeap::evacuation_should_fail() {
inline bool G1CollectedHeap::evacuation_should_fail() {
if (!G1EvacuationFailureALot || !_evacuation_failure_alot_for_current_gc) {
return false;
}
......
......@@ -455,7 +455,7 @@ void G1CollectorPolicy::init() {
} else {
_young_list_fixed_length = _young_gen_sizer->min_desired_young_length();
}
_free_regions_at_end_of_collection = _g1->free_regions();
_free_regions_at_end_of_collection = _g1->num_free_regions();
update_young_list_target_length();
// We may immediately start allocating regions and placing them on the
......@@ -828,7 +828,7 @@ void G1CollectorPolicy::record_full_collection_end() {
record_survivor_regions(0, NULL, NULL);
_free_regions_at_end_of_collection = _g1->free_regions();
_free_regions_at_end_of_collection = _g1->num_free_regions();
// Reset survivors SurvRateGroup.
_survivor_surv_rate_group->reset();
update_young_list_target_length();
......@@ -1180,7 +1180,7 @@ void G1CollectorPolicy::record_collection_pause_end(double pause_time_ms, Evacua
_in_marking_window = new_in_marking_window;
_in_marking_window_im = new_in_marking_window_im;
_free_regions_at_end_of_collection = _g1->free_regions();
_free_regions_at_end_of_collection = _g1->num_free_regions();
update_young_list_target_length();
// Note that _mmu_tracker->max_gc_time() returns the time in seconds.
......@@ -1202,7 +1202,7 @@ void G1CollectorPolicy::record_heap_size_info_at_start(bool full) {
_survivor_used_bytes_before_gc = young_list->survivor_used_bytes();
_heap_capacity_bytes_before_gc = _g1->capacity();
_heap_used_bytes_before_gc = _g1->used();
_cur_collection_pause_used_regions_at_start = _g1->used_regions();
_cur_collection_pause_used_regions_at_start = _g1->num_used_regions();
_eden_capacity_bytes_before_gc =
(_young_list_target_length * HeapRegion::GrainBytes) - _survivor_used_bytes_before_gc;
......@@ -1617,7 +1617,7 @@ void
G1CollectorPolicy::record_concurrent_mark_cleanup_end(int no_of_gc_threads) {
_collectionSetChooser->clear();
uint region_num = _g1->n_regions();
uint region_num = _g1->num_regions();
if (G1CollectedHeap::use_parallel_gc_threads()) {
const uint OverpartitionFactor = 4;
uint WorkUnit;
......@@ -1638,7 +1638,7 @@ G1CollectorPolicy::record_concurrent_mark_cleanup_end(int no_of_gc_threads) {
MAX2(region_num / (uint) (ParallelGCThreads * OverpartitionFactor),
MinWorkUnit);
}
_collectionSetChooser->prepare_for_par_region_addition(_g1->n_regions(),
_collectionSetChooser->prepare_for_par_region_addition(_g1->num_regions(),
WorkUnit);
ParKnownGarbageTask parKnownGarbageTask(_collectionSetChooser,
(int) WorkUnit);
......@@ -1935,7 +1935,7 @@ uint G1CollectorPolicy::calc_max_old_cset_length() {
// of them are available.
G1CollectedHeap* g1h = G1CollectedHeap::heap();
const size_t region_num = g1h->n_regions();
const size_t region_num = g1h->num_regions();
const size_t perc = (size_t) G1OldCSetRegionThresholdPercent;
size_t result = region_num * perc / 100;
// emulate ceiling
......
......@@ -344,11 +344,6 @@ HeapWord* HeapRegion::next_block_start_careful(HeapWord* addr) {
return low;
}
#ifdef _MSC_VER // the use of 'this' below gets a warning, make it go away
#pragma warning( disable:4355 ) // 'this' : used in base member initializer list
#endif // _MSC_VER
HeapRegion::HeapRegion(uint hrs_index,
G1BlockOffsetSharedArray* sharedOffsetArray,
MemRegion mr) :
......@@ -360,7 +355,7 @@ HeapRegion::HeapRegion(uint hrs_index,
_claimed(InitialClaimValue), _evacuation_failed(false),
_prev_marked_bytes(0), _next_marked_bytes(0), _gc_efficiency(0.0),
_young_type(NotYoung), _next_young_region(NULL),
_next_dirty_cards_region(NULL), _next(NULL), _prev(NULL), _pending_removal(false),
_next_dirty_cards_region(NULL), _next(NULL), _prev(NULL),
#ifdef ASSERT
_containing_set(NULL),
#endif // ASSERT
......@@ -369,14 +364,20 @@ HeapRegion::HeapRegion(uint hrs_index,
_predicted_bytes_to_copy(0)
{
_rem_set = new HeapRegionRemSet(sharedOffsetArray, this);
assert(HeapRegionRemSet::num_par_rem_sets() > 0, "Invariant.");
initialize(mr);
}
void HeapRegion::initialize(MemRegion mr, bool clear_space, bool mangle_space) {
assert(_rem_set->is_empty(), "Remembered set must be empty");
G1OffsetTableContigSpace::initialize(mr, clear_space, mangle_space);
_orig_end = mr.end();
// Note that initialize() will set the start of the unmarked area of the
// region.
hr_clear(false /*par*/, false /*clear_space*/);
set_top(bottom());
record_top_and_timestamp();
assert(HeapRegionRemSet::num_par_rem_sets() > 0, "Invariant.");
}
CompactibleSpace* HeapRegion::next_compaction_space() const {
......@@ -907,7 +908,7 @@ void HeapRegion::verify(VerifyOption vo,
}
// If it returns false, verify_for_object() will output the
// appropriate messasge.
// appropriate message.
if (do_bot_verify &&
!g1->is_obj_dead(obj, this) &&
!_offsets.verify_for_object(p, obj_size)) {
......@@ -1038,8 +1039,7 @@ void G1OffsetTableContigSpace::clear(bool mangle_space) {
set_top(bottom());
set_saved_mark_word(bottom());
CompactibleSpace::clear(mangle_space);
_offsets.zero_bottom_entry();
_offsets.initialize_threshold();
reset_bot();
}
void G1OffsetTableContigSpace::set_bottom(HeapWord* new_bottom) {
......@@ -1129,9 +1129,11 @@ G1OffsetTableContigSpace(G1BlockOffsetSharedArray* sharedOffsetArray,
_gc_time_stamp(0)
{
_offsets.set_space(this);
// false ==> we'll do the clearing if there's clearing to be done.
CompactibleSpace::initialize(mr, false, SpaceDecorator::Mangle);
}
void G1OffsetTableContigSpace::initialize(MemRegion mr, bool clear_space, bool mangle_space) {
CompactibleSpace::initialize(mr, clear_space, mangle_space);
_top = bottom();
_offsets.zero_bottom_entry();
_offsets.initialize_threshold();
reset_bot();
}
......@@ -62,7 +62,7 @@ class nmethod;
p2i((_hr_)->bottom()), p2i((_hr_)->top()), p2i((_hr_)->end())
// sentinel value for hrs_index
#define G1_NULL_HRS_INDEX ((uint) -1)
#define G1_NO_HRS_INDEX ((uint) -1)
// A dirty card to oop closure for heap regions. It
// knows how to get the G1 heap and how to use the bitmap
......@@ -146,6 +146,9 @@ class G1OffsetTableContigSpace: public CompactibleSpace {
HeapWord* top() const { return _top; }
protected:
// Reset the G1OffsetTableContigSpace.
virtual void initialize(MemRegion mr, bool clear_space, bool mangle_space);
HeapWord** top_addr() { return &_top; }
// Allocation helpers (return NULL if full).
inline HeapWord* allocate_impl(size_t word_size, HeapWord* end_value);
......@@ -200,8 +203,7 @@ class G1OffsetTableContigSpace: public CompactibleSpace {
virtual void print() const;
void reset_bot() {
_offsets.zero_bottom_entry();
_offsets.initialize_threshold();
_offsets.reset_bot();
}
void update_bot_for_object(HeapWord* start, size_t word_size) {
......@@ -264,7 +266,6 @@ class HeapRegion: public G1OffsetTableContigSpace {
#ifdef ASSERT
HeapRegionSetBase* _containing_set;
#endif // ASSERT
bool _pending_removal;
// For parallel heapRegion traversal.
jint _claimed;
......@@ -333,6 +334,12 @@ class HeapRegion: public G1OffsetTableContigSpace {
G1BlockOffsetSharedArray* sharedOffsetArray,
MemRegion mr);
// Initializing the HeapRegion not only resets the data structure, but also
// resets the BOT for that heap region.
// The default values for clear_space means that we will do the clearing if
// there's clearing to be done ourselves. We also always mangle the space.
virtual void initialize(MemRegion mr, bool clear_space = false, bool mangle_space = SpaceDecorator::Mangle);
static int LogOfHRGrainBytes;
static int LogOfHRGrainWords;
......@@ -553,26 +560,6 @@ class HeapRegion: public G1OffsetTableContigSpace {
// to provide a dummy version of it.
#endif // ASSERT
// If we want to remove regions from a list in bulk we can simply tag
// them with the pending_removal tag and call the
// remove_all_pending() method on the list.
bool pending_removal() { return _pending_removal; }
void set_pending_removal(bool pending_removal) {
if (pending_removal) {
assert(!_pending_removal && containing_set() != NULL,
"can only set pending removal to true if it's false and "
"the region belongs to a region set");
} else {
assert( _pending_removal && containing_set() == NULL,
"can only set pending removal to false if it's true and "
"the region does not belong to a region set");
}
_pending_removal = pending_removal;
}
HeapRegion* get_next_young_region() { return _next_young_region; }
void set_next_young_region(HeapRegion* hr) {
_next_young_region = hr;
......
......@@ -26,6 +26,7 @@
#define SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSEQ_HPP
#include "gc_implementation/g1/g1BiasedArray.hpp"
#include "gc_implementation/g1/heapRegionSet.hpp"
class HeapRegion;
class HeapRegionClosure;
......@@ -33,7 +34,7 @@ class FreeRegionList;
class G1HeapRegionTable : public G1BiasedMappedArray<HeapRegion*> {
protected:
virtual HeapRegion* default_value() const { return NULL; }
virtual HeapRegion* default_value() const { return NULL; }
};
// This class keeps track of the region metadata (i.e., HeapRegion
......@@ -64,89 +65,151 @@ class HeapRegionSeq: public CHeapObj<mtGC> {
G1HeapRegionTable _regions;
// The number of regions committed in the heap.
uint _committed_length;
ReservedSpace _reserved;
VirtualSpace _storage;
FreeRegionList _free_list;
// The number of regions committed in the heap.
uint _num_committed;
// Internal only. The highest heap region +1 we allocated a HeapRegion instance for.
uint _allocated_heapregions_length;
HeapWord* heap_bottom() const { return _regions.bottom_address_mapped(); }
HeapWord* heap_top() const { return heap_bottom() + _num_committed * HeapRegion::GrainWords; }
HeapWord* heap_end() const {return _regions.end_address_mapped(); }
void make_regions_available(uint index, uint num_regions = 1);
// Pass down commit calls to the VirtualSpace.
void commit_regions(uint index, size_t num_regions = 1);
void uncommit_regions(uint index, size_t num_regions = 1);
// Notify other data structures about change in the heap layout.
void update_committed_space(HeapWord* old_end, HeapWord* new_end);
// Calculate the starting region for each worker during parallel iteration so
// that they do not all start from the same region.
uint start_region_for_worker(uint worker_i, uint num_workers, uint num_regions) const;
// Finds the next sequence of unavailable regions starting from start_idx. Returns the
// length of the sequence found. If this result is zero, no such sequence could be found,
// otherwise res_idx indicates the start index of these regions.
uint find_unavailable_from_idx(uint start_idx, uint* res_idx) const;
// Finds the next sequence of empty regions starting from start_idx, going backwards in
// the heap. Returns the length of the sequence found. If this value is zero, no
// sequence could be found, otherwise res_idx contains the start index of this range.
uint find_empty_from_idx_reverse(uint start_idx, uint* res_idx) const;
#ifdef ASSERT
public:
bool is_free(HeapRegion* hr) const;
#endif
// Returns whether the given region is available for allocation.
bool is_available(uint region) const;
// Allocate a new HeapRegion for the given index.
HeapRegion* new_heap_region(uint hrs_index);
public:
// Empty constructor, we'll initialize it with the initialize() method.
HeapRegionSeq() : _regions(), _reserved(), _storage(), _num_committed(0),
_free_list("Master Free List", new MasterFreeRegionListMtSafeChecker()),
_allocated_heapregions_length(0)
{ }
void initialize(ReservedSpace reserved);
// Return the "dummy" region used for G1AllocRegion. This is currently a hardwired
// new HeapRegion that owns HeapRegion at index 0. Since at the moment we commit
// the heap from the lowest address, this region (and its associated data
// structures) are available and we do not need to check further.
HeapRegion* get_dummy_region() { return new_heap_region(0); }
// A hint for which index to start searching from for humongous
// allocations.
uint _next_search_index;
// Return the HeapRegion at the given index. Assume that the index
// is valid.
inline HeapRegion* at(uint index) const;
// The number of regions for which we have allocated HeapRegions for.
uint _allocated_length;
// If addr is within the committed space return its corresponding
// HeapRegion, otherwise return NULL.
inline HeapRegion* addr_to_region(HeapWord* addr) const;
// Find a contiguous set of empty regions of length num, starting
// from the given index.
uint find_contiguous_from(uint from, uint num);
// Insert the given region into the free region list.
inline void insert_into_free_list(HeapRegion* hr);
void increment_allocated_length() {
assert(_allocated_length < max_length(), "pre-condition");
_allocated_length++;
// Insert the given region list into the global free region list.
void insert_list_into_free_list(FreeRegionList* list) {
_free_list.add_ordered(list);
}
void increment_length() {
assert(length() < max_length(), "pre-condition");
_committed_length++;
}
HeapRegion* allocate_free_region(bool is_old) {
HeapRegion* hr = _free_list.remove_region(is_old);
void decrement_length() {
assert(length() > 0, "pre-condition");
_committed_length--;
if (hr != NULL) {
assert(hr->next() == NULL, "Single region should not have next");
assert(is_available(hr->hrs_index()), "Must be committed");
}
return hr;
}
HeapWord* heap_bottom() const { return _regions.bottom_address_mapped(); }
HeapWord* heap_end() const {return _regions.end_address_mapped(); }
inline void allocate_free_regions_starting_at(uint first, uint num_regions);
public:
// Empty contructor, we'll initialize it with the initialize() method.
HeapRegionSeq() : _regions(), _committed_length(0), _next_search_index(0), _allocated_length(0) { }
// Remove all regions from the free list.
void remove_all_free_regions() {
_free_list.remove_all();
}
void initialize(HeapWord* bottom, HeapWord* end);
// Return the number of committed free regions in the heap.
uint num_free_regions() const {
return _free_list.length();
}
// Return the HeapRegion at the given index. Assume that the index
// is valid.
inline HeapRegion* at(uint index) const;
size_t total_capacity_bytes() const {
return num_free_regions() * HeapRegion::GrainBytes;
}
// If addr is within the committed space return its corresponding
// HeapRegion, otherwise return NULL.
inline HeapRegion* addr_to_region(HeapWord* addr) const;
// Return the number of available (uncommitted) regions.
uint available() const { return max_length() - length(); }
// Return the number of regions that have been committed in the heap.
uint length() const { return _committed_length; }
uint length() const { return _num_committed; }
// Return the maximum number of regions in the heap.
uint max_length() const { return (uint)_regions.length(); }
// Expand the sequence to reflect that the heap has grown from
// old_end to new_end. Either create new HeapRegions, or re-use
// existing ones, and return them in the given list. Returns the
// memory region that covers the newly-created regions. If a
// HeapRegion allocation fails, the result memory region might be
// smaller than the desired one.
MemRegion expand_by(HeapWord* old_end, HeapWord* new_end,
FreeRegionList* list);
MemRegion committed() const { return MemRegion(heap_bottom(), heap_top()); }
MemRegion reserved() const { return MemRegion(heap_bottom(), heap_end()); }
// Expand the sequence to reflect that the heap has grown. Either create new
// HeapRegions, or re-use existing ones. Returns the number of regions the
// sequence was expanded by. If a HeapRegion allocation fails, the resulting
// number of regions might be smaller than what's desired.
uint expand_by(uint num_regions);
// Return the number of contiguous regions at the end of the sequence
// that are available for allocation.
uint free_suffix();
// Makes sure that the regions from start to start+num_regions-1 are available
// for allocation. Returns the number of regions that were committed to achieve
// this.
uint expand_at(uint start, uint num_regions);
// Find a contiguous set of empty regions of length num and return
// the index of the first region or G1_NULL_HRS_INDEX if the
// search was unsuccessful.
uint find_contiguous(uint num);
// Find a contiguous set of empty or uncommitted regions of length num and return
// the index of the first region or G1_NO_HRS_INDEX if the search was unsuccessful.
// If only_empty is true, only empty regions are considered.
// Searches from bottom to top of the heap, doing a first-fit.
uint find_contiguous(size_t num, bool only_empty);
HeapRegion* next_region_in_heap(const HeapRegion* r) const;
// Apply blk->doHeapRegion() on all committed regions in address order,
// terminating the iteration early if doHeapRegion() returns true.
void iterate(HeapRegionClosure* blk) const;
// As above, but start the iteration from hr and loop around. If hr
// is NULL, we start from the first region in the heap.
void iterate_from(HeapRegion* hr, HeapRegionClosure* blk) const;
void par_iterate(HeapRegionClosure* blk, uint worker_id, uint no_of_par_workers, jint claim_value) const;
// Tag as uncommitted as many regions that are completely free as
// possible, up to num_regions_to_remove, from the suffix of the committed
// sequence. Return the actual number of removed regions.
// Uncommit up to num_regions_to_remove regions that are completely free.
// Return the actual number of uncommitted regions.
uint shrink_by(uint num_regions_to_remove);
void verify();
// Do some sanity checking.
void verify_optional() PRODUCT_RETURN;
};
......
......@@ -40,11 +40,19 @@ inline HeapRegion* HeapRegionSeq::addr_to_region(HeapWord* addr) const {
}
inline HeapRegion* HeapRegionSeq::at(uint index) const {
assert(index < length(), "pre-condition");
assert(is_available(index), "pre-condition");
HeapRegion* hr = _regions.get_by_index(index);
assert(hr != NULL, "sanity");
assert(hr->hrs_index() == index, "sanity");
return hr;
}
inline void HeapRegionSeq::insert_into_free_list(HeapRegion* hr) {
_free_list.add_ordered(hr);
}
inline void HeapRegionSeq::allocate_free_regions_starting_at(uint first, uint num_regions) {
_free_list.remove_starting_at(at(first), num_regions);
}
#endif // SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSEQ_INLINE_HPP
......@@ -23,6 +23,7 @@
*/
#include "precompiled.hpp"
#include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
#include "gc_implementation/g1/heapRegionRemSet.hpp"
#include "gc_implementation/g1/heapRegionSet.inline.hpp"
......@@ -67,7 +68,7 @@ void HeapRegionSetBase::verify_start() {
// Do the basic verification first before we do the checks over the regions.
HeapRegionSetBase::verify();
_verify_in_progress = true;
_verify_in_progress = true;
}
void HeapRegionSetBase::verify_end() {
......@@ -103,62 +104,7 @@ void FreeRegionList::set_unrealistically_long_length(uint len) {
}
void FreeRegionList::fill_in_ext_msg_extra(hrs_ext_msg* msg) {
msg->append(" hd: "PTR_FORMAT" tl: "PTR_FORMAT, head(), tail());
}
void FreeRegionList::add_as_head_or_tail(FreeRegionList* from_list, bool as_head) {
check_mt_safety();
from_list->check_mt_safety();
verify_optional();
from_list->verify_optional();
if (from_list->is_empty()) {
return;
}
#ifdef ASSERT
FreeRegionListIterator iter(from_list);
while (iter.more_available()) {
HeapRegion* hr = iter.get_next();
// In set_containing_set() we check that we either set the value
// from NULL to non-NULL or vice versa to catch bugs. So, we have
// to NULL it first before setting it to the value.
hr->set_containing_set(NULL);
hr->set_containing_set(this);
}
#endif // ASSERT
if (_head == NULL) {
assert(length() == 0 && _tail == NULL, hrs_ext_msg(this, "invariant"));
_head = from_list->_head;
_tail = from_list->_tail;
} else {
assert(length() > 0 && _tail != NULL, hrs_ext_msg(this, "invariant"));
if (as_head) {
from_list->_tail->set_next(_head);
_head->set_prev(from_list->_tail);
_head = from_list->_head;
} else {
_tail->set_next(from_list->_head);
from_list->_head->set_prev(_tail);
_tail = from_list->_tail;
}
}
_count.increment(from_list->length(), from_list->total_capacity_bytes());
from_list->clear();
verify_optional();
from_list->verify_optional();
}
void FreeRegionList::add_as_head(FreeRegionList* from_list) {
add_as_head_or_tail(from_list, true /* as_head */);
}
void FreeRegionList::add_as_tail(FreeRegionList* from_list) {
add_as_head_or_tail(from_list, false /* as_head */);
msg->append(" hd: "PTR_FORMAT" tl: "PTR_FORMAT, _head, _tail);
}
void FreeRegionList::remove_all() {
......@@ -191,11 +137,6 @@ void FreeRegionList::add_ordered(FreeRegionList* from_list) {
return;
}
if (is_empty()) {
add_as_head(from_list);
return;
}
#ifdef ASSERT
FreeRegionListIterator iter(from_list);
while (iter.more_available()) {
......@@ -208,37 +149,43 @@ void FreeRegionList::add_ordered(FreeRegionList* from_list) {
}
#endif // ASSERT
HeapRegion* curr_to = _head;
HeapRegion* curr_from = from_list->_head;
while (curr_from != NULL) {
while (curr_to != NULL && curr_to->hrs_index() < curr_from->hrs_index()) {
curr_to = curr_to->next();
}
if (is_empty()) {
assert(length() == 0 && _tail == NULL, hrs_ext_msg(this, "invariant"));
_head = from_list->_head;
_tail = from_list->_tail;
} else {
HeapRegion* curr_to = _head;
HeapRegion* curr_from = from_list->_head;
if (curr_to == NULL) {
// The rest of the from list should be added as tail
_tail->set_next(curr_from);
curr_from->set_prev(_tail);
curr_from = NULL;
} else {
HeapRegion* next_from = curr_from->next();
while (curr_from != NULL) {
while (curr_to != NULL && curr_to->hrs_index() < curr_from->hrs_index()) {
curr_to = curr_to->next();
}
curr_from->set_next(curr_to);
curr_from->set_prev(curr_to->prev());
if (curr_to->prev() == NULL) {
_head = curr_from;
if (curr_to == NULL) {
// The rest of the from list should be added as tail
_tail->set_next(curr_from);
curr_from->set_prev(_tail);
curr_from = NULL;
} else {
curr_to->prev()->set_next(curr_from);
HeapRegion* next_from = curr_from->next();
curr_from->set_next(curr_to);
curr_from->set_prev(curr_to->prev());
if (curr_to->prev() == NULL) {
_head = curr_from;
} else {
curr_to->prev()->set_next(curr_from);
}
curr_to->set_prev(curr_from);
curr_from = next_from;
}
curr_to->set_prev(curr_from);
curr_from = next_from;
}
}
if (_tail->hrs_index() < from_list->_tail->hrs_index()) {
_tail = from_list->_tail;
if (_tail->hrs_index() < from_list->_tail->hrs_index()) {
_tail = from_list->_tail;
}
}
_count.increment(from_list->length(), from_list->total_capacity_bytes());
......@@ -248,68 +195,59 @@ void FreeRegionList::add_ordered(FreeRegionList* from_list) {
from_list->verify_optional();
}
void FreeRegionList::remove_all_pending(uint target_count) {
void FreeRegionList::remove_starting_at(HeapRegion* first, uint num_regions) {
check_mt_safety();
assert(target_count > 1, hrs_ext_msg(this, "pre-condition"));
assert(num_regions >= 1, hrs_ext_msg(this, "pre-condition"));
assert(!is_empty(), hrs_ext_msg(this, "pre-condition"));
verify_optional();
DEBUG_ONLY(uint old_length = length();)
HeapRegion* curr = _head;
HeapRegion* curr = first;
uint count = 0;
while (curr != NULL) {
while (count < num_regions) {
verify_region(curr);
HeapRegion* next = curr->next();
HeapRegion* prev = curr->prev();
if (curr->pending_removal()) {
assert(count < target_count,
hrs_err_msg("[%s] should not come across more regions "
"pending for removal than target_count: %u",
name(), target_count));
if (prev == NULL) {
assert(_head == curr, hrs_ext_msg(this, "invariant"));
_head = next;
} else {
assert(_head != curr, hrs_ext_msg(this, "invariant"));
prev->set_next(next);
}
if (next == NULL) {
assert(_tail == curr, hrs_ext_msg(this, "invariant"));
_tail = prev;
} else {
assert(_tail != curr, hrs_ext_msg(this, "invariant"));
next->set_prev(prev);
}
if (_last = curr) {
_last = NULL;
}
assert(count < num_regions,
hrs_err_msg("[%s] should not come across more regions "
"pending for removal than num_regions: %u",
name(), num_regions));
curr->set_next(NULL);
curr->set_prev(NULL);
remove(curr);
curr->set_pending_removal(false);
if (prev == NULL) {
assert(_head == curr, hrs_ext_msg(this, "invariant"));
_head = next;
} else {
assert(_head != curr, hrs_ext_msg(this, "invariant"));
prev->set_next(next);
}
if (next == NULL) {
assert(_tail == curr, hrs_ext_msg(this, "invariant"));
_tail = prev;
} else {
assert(_tail != curr, hrs_ext_msg(this, "invariant"));
next->set_prev(prev);
}
if (_last = curr) {
_last = NULL;
}
count += 1;
curr->set_next(NULL);
curr->set_prev(NULL);
remove(curr);
// If we have come across the target number of regions we can
// just bail out. However, for debugging purposes, we can just
// carry on iterating to make sure there are not more regions
// tagged with pending removal.
DEBUG_ONLY(if (count == target_count) break;)
}
count++;
curr = next;
}
assert(count == target_count,
hrs_err_msg("[%s] count: %u should be == target_count: %u",
name(), count, target_count));
assert(length() + target_count == old_length,
assert(count == num_regions,
hrs_err_msg("[%s] count: %u should be == num_regions: %u",
name(), count, num_regions));
assert(length() + num_regions == old_length,
hrs_err_msg("[%s] new length should be consistent "
"new length: %u old length: %u target_count: %u",
name(), length(), old_length, target_count));
"new length: %u old length: %u num_regions: %u",
name(), length(), old_length, num_regions));
verify_optional();
}
......@@ -348,10 +286,12 @@ void FreeRegionList::print_on(outputStream* out, bool print_contents) {
hr->print_on(out);
}
}
out->cr();
}
void FreeRegionList::verify_list() {
HeapRegion* curr = head();
HeapRegion* curr = _head;
HeapRegion* prev1 = NULL;
HeapRegion* prev0 = NULL;
uint count = 0;
......@@ -379,7 +319,7 @@ void FreeRegionList::verify_list() {
curr = curr->next();
}
guarantee(tail() == prev0, err_msg("Expected %s to end with %u but it ended with %u.", name(), tail()->hrs_index(), prev0->hrs_index()));
guarantee(_tail == prev0, err_msg("Expected %s to end with %u but it ended with %u.", name(), _tail->hrs_index(), prev0->hrs_index()));
guarantee(_tail == NULL || _tail->next() == NULL, "_tail should not have a next");
guarantee(length() == count, err_msg("%s count mismatch. Expected %u, actual %u.", name(), length(), count));
guarantee(total_capacity_bytes() == capacity, err_msg("%s capacity mismatch. Expected " SIZE_FORMAT ", actual " SIZE_FORMAT,
......
......@@ -162,7 +162,7 @@ public:
// diagnosing failures.
class hrs_ext_msg : public hrs_err_msg {
public:
hrs_ext_msg(HeapRegionSetBase* set, const char* message) : hrs_err_msg("%s","") {
hrs_ext_msg(HeapRegionSetBase* set, const char* message) : hrs_err_msg("%s", "") {
set->fill_in_ext_msg(this, message);
}
};
......@@ -192,13 +192,9 @@ public:
};
// A set that links all the regions added to it in a doubly-linked
// list. We should try to avoid doing operations that iterate over
// sorted list. We should try to avoid doing operations that iterate over
// such lists in performance critical paths. Typically we should
// add / remove one region at a time or concatenate two lists. There are
// two ways to treat your lists, ordered and un-ordered. All un-ordered
// operations are done in constant time. To keep a list ordered only use
// add_ordered() to add elements to the list. If a list is not ordered
// from start, there is no way to sort it later.
// add / remove one region at a time or concatenate two lists.
class FreeRegionListIterator;
......@@ -210,13 +206,13 @@ private:
HeapRegion* _tail;
// _last is used to keep track of where we added an element the last
// time in ordered lists. It helps to improve performance when adding
// several ordered items in a row.
// time. It helps to improve performance when adding several ordered items in a row.
HeapRegion* _last;
static uint _unrealistically_long_length;
void add_as_head_or_tail(FreeRegionList* from_list, bool as_head);
inline HeapRegion* remove_from_head_impl();
inline HeapRegion* remove_from_tail_impl();
protected:
virtual void fill_in_ext_msg_extra(hrs_ext_msg* msg);
......@@ -232,8 +228,11 @@ public:
void verify_list();
HeapRegion* head() { return _head; }
HeapRegion* tail() { return _tail; }
#ifdef ASSERT
bool contains(HeapRegion* hr) const {
return hr->containing_set() == this;
}
#endif
static void set_unrealistically_long_length(uint len);
......@@ -242,55 +241,20 @@ public:
// is determined by hrs_index.
inline void add_ordered(HeapRegion* hr);
// It adds hr to the list as the new head. The region should not be
// a member of another set.
inline void add_as_head(HeapRegion* hr);
// It adds hr to the list as the new tail. The region should not be
// a member of another set.
inline void add_as_tail(HeapRegion* hr);
// It removes and returns the head of the list. It assumes that the
// list is not empty so it will return a non-NULL value.
inline HeapRegion* remove_head();
// Convenience method.
inline HeapRegion* remove_head_or_null();
// Removes and returns the last element (_tail) of the list. It assumes
// that the list isn't empty so that it can return a non-NULL value.
inline HeapRegion* remove_tail();
// Convenience method
inline HeapRegion* remove_tail_or_null();
// Removes from head or tail based on the given argument.
inline HeapRegion* remove_region(bool from_head);
HeapRegion* remove_region(bool from_head);
// Merge two ordered lists. The result is also ordered. The order is
// determined by hrs_index.
void add_ordered(FreeRegionList* from_list);
// It moves the regions from from_list to this list and empties
// from_list. The new regions will appear in the same order as they
// were in from_list and be linked in the beginning of this list.
void add_as_head(FreeRegionList* from_list);
// It moves the regions from from_list to this list and empties
// from_list. The new regions will appear in the same order as they
// were in from_list and be linked in the end of this list.
void add_as_tail(FreeRegionList* from_list);
// It empties the list by removing all regions from it.
void remove_all();
// It removes all regions in the list that are pending for removal
// (i.e., they have been tagged with "pending_removal"). The list
// must not be empty, target_count should reflect the exact number
// of regions that are pending for removal in the list, and
// target_count should be > 1 (currently, we never need to remove a
// single region using this).
void remove_all_pending(uint target_count);
// Remove all (contiguous) regions from first to first + num_regions -1 from
// this list.
// Num_regions must be > 1.
void remove_starting_at(HeapRegion* first, uint num_regions);
virtual void verify();
......@@ -298,7 +262,7 @@ public:
};
// Iterator class that provides a convenient way to iterate over the
// regions of a HeapRegionLinkedList instance.
// regions of a FreeRegionList.
class FreeRegionListIterator : public StackObj {
private:
......@@ -324,7 +288,7 @@ public:
}
FreeRegionListIterator(FreeRegionList* list) : _curr(NULL), _list(list) {
_curr = list->head();
_curr = list->_head;
}
};
......
......@@ -30,7 +30,8 @@
inline void HeapRegionSetBase::add(HeapRegion* hr) {
check_mt_safety();
assert(hr->containing_set() == NULL, hrs_ext_msg(this, "should not already have a containing set %u"));
assert(hr->next() == NULL && hr->prev() == NULL, hrs_ext_msg(this, "should not already be linked"));
assert(hr->next() == NULL, hrs_ext_msg(this, "should not already be linked"));
assert(hr->prev() == NULL, hrs_ext_msg(this, "should not already be linked"));
_count.increment(1u, hr->capacity());
hr->set_containing_set(this);
......@@ -40,7 +41,8 @@ inline void HeapRegionSetBase::add(HeapRegion* hr) {
inline void HeapRegionSetBase::remove(HeapRegion* hr) {
check_mt_safety();
verify_region(hr);
assert(hr->next() == NULL && hr->prev() == NULL, hrs_ext_msg(this, "should already be unlinked"));
assert(hr->next() == NULL, hrs_ext_msg(this, "should already be unlinked"));
assert(hr->prev() == NULL, hrs_ext_msg(this, "should already be unlinked"));
hr->set_containing_set(NULL);
assert(_count.length() > 0, hrs_ext_msg(this, "pre-condition"));
......@@ -48,8 +50,7 @@ inline void HeapRegionSetBase::remove(HeapRegion* hr) {
}
inline void FreeRegionList::add_ordered(HeapRegion* hr) {
check_mt_safety();
assert((length() == 0 && _head == NULL && _tail == NULL) ||
assert((length() == 0 && _head == NULL && _tail == NULL && _last == NULL) ||
(length() > 0 && _head != NULL && _tail != NULL),
hrs_ext_msg(this, "invariant"));
// add() will verify the region and check mt safety.
......@@ -95,89 +96,48 @@ inline void FreeRegionList::add_ordered(HeapRegion* hr) {
_last = hr;
}
inline void FreeRegionList::add_as_head(HeapRegion* hr) {
assert((length() == 0 && _head == NULL && _tail == NULL) ||
(length() > 0 && _head != NULL && _tail != NULL),
hrs_ext_msg(this, "invariant"));
// add() will verify the region and check mt safety.
add(hr);
// Now link the region.
if (_head != NULL) {
hr->set_next(_head);
_head->set_prev(hr);
} else {
_tail = hr;
}
_head = hr;
}
inline void FreeRegionList::add_as_tail(HeapRegion* hr) {
check_mt_safety();
assert((length() == 0 && _head == NULL && _tail == NULL) ||
(length() > 0 && _head != NULL && _tail != NULL),
hrs_ext_msg(this, "invariant"));
// add() will verify the region and check mt safety.
add(hr);
// Now link the region.
if (_tail != NULL) {
_tail->set_next(hr);
hr->set_prev(_tail);
} else {
_head = hr;
}
_tail = hr;
}
inline HeapRegion* FreeRegionList::remove_head() {
assert(!is_empty(), hrs_ext_msg(this, "the list should not be empty"));
assert(length() > 0 && _head != NULL && _tail != NULL,
hrs_ext_msg(this, "invariant"));
// We need to unlink it first.
HeapRegion* hr = _head;
_head = hr->next();
inline HeapRegion* FreeRegionList::remove_from_head_impl() {
HeapRegion* result = _head;
_head = result->next();
if (_head == NULL) {
_tail = NULL;
} else {
_head->set_prev(NULL);
}
hr->set_next(NULL);
result->set_next(NULL);
return result;
}
if (_last == hr) {
_last = NULL;
}
inline HeapRegion* FreeRegionList::remove_from_tail_impl() {
HeapRegion* result = _tail;
// remove() will verify the region and check mt safety.
remove(hr);
return hr;
_tail = result->prev();
if (_tail == NULL) {
_head = NULL;
} else {
_tail->set_next(NULL);
}
result->set_prev(NULL);
return result;
}
inline HeapRegion* FreeRegionList::remove_head_or_null() {
inline HeapRegion* FreeRegionList::remove_region(bool from_head) {
check_mt_safety();
if (!is_empty()) {
return remove_head();
} else {
verify_optional();
if (is_empty()) {
return NULL;
}
}
inline HeapRegion* FreeRegionList::remove_tail() {
assert(!is_empty(), hrs_ext_msg(this, "The list should not be empty"));
assert(length() > 0 && _head != NULL && _tail != NULL,
hrs_ext_msg(this, "invariant"));
// We need to unlink it first
HeapRegion* hr = _tail;
HeapRegion* hr;
_tail = hr->prev();
if (_tail == NULL) {
_head = NULL;
if (from_head) {
hr = remove_from_head_impl();
} else {
_tail->set_next(NULL);
hr = remove_from_tail_impl();
}
hr->set_prev(NULL);
if (_last == hr) {
_last = NULL;
......@@ -188,22 +148,5 @@ inline HeapRegion* FreeRegionList::remove_tail() {
return hr;
}
inline HeapRegion* FreeRegionList::remove_tail_or_null() {
check_mt_safety();
if (!is_empty()) {
return remove_tail();
} else {
return NULL;
}
}
inline HeapRegion* FreeRegionList::remove_region(bool from_head) {
if (from_head) {
return remove_head_or_null();
} else {
return remove_tail_or_null();
}
}
#endif // SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSET_INLINE_HPP
......@@ -43,10 +43,9 @@
nonstatic_field(G1HeapRegionTable, _shift_by, uint) \
\
nonstatic_field(HeapRegionSeq, _regions, G1HeapRegionTable) \
nonstatic_field(HeapRegionSeq, _committed_length, uint) \
nonstatic_field(HeapRegionSeq, _num_committed, uint) \
\
nonstatic_field(G1CollectedHeap, _hrs, HeapRegionSeq) \
nonstatic_field(G1CollectedHeap, _g1_committed, MemRegion) \
nonstatic_field(G1CollectedHeap, _summary_bytes_used, size_t) \
nonstatic_field(G1CollectedHeap, _g1mm, G1MonitoringSupport*) \
nonstatic_field(G1CollectedHeap, _old_set, HeapRegionSetBase) \
......
......@@ -231,7 +231,7 @@ WB_END
WB_ENTRY(jlong, WB_G1NumFreeRegions(JNIEnv* env, jobject o))
G1CollectedHeap* g1 = G1CollectedHeap::heap();
size_t nr = g1->free_regions();
size_t nr = g1->num_free_regions();
return (jlong)nr;
WB_END
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册