8077144: Concurrent mark initialization takes too long

Remove per-marking thread liveness bitmaps and recreate liveness bitmap concurrently after the cleanup pause.

Reviewed-by: mgerdin, ehelin, kbarrett
This commit is contained in:
Thomas Schatzl 2016-04-06 13:32:48 +02:00
parent b51a55a78c
commit 0c06163b35
10 changed files with 452 additions and 711 deletions

View file

@ -183,6 +183,11 @@ void ConcurrentMarkThread::run_service() {
}
} while (cm()->restart_for_overflow());
if (!cm()->has_aborted()) {
G1ConcPhaseTimer t(_cm, "Concurrent Create Live Data");
cm()->create_live_data();
}
double end_time = os::elapsedVTime();
// Update the total virtual time before doing this, since it will try
// to measure it to get the vtime for this marking. We purposely

File diff suppressed because it is too large Load diff

View file

@ -266,7 +266,7 @@ class ConcurrentMarkThread;
class G1ConcurrentMark: public CHeapObj<mtGC> {
friend class ConcurrentMarkThread;
friend class G1ParNoteEndTask;
friend class CalcLiveObjectsClosure;
friend class G1VerifyLiveDataClosure;
friend class G1CMRefProcTaskProxy;
friend class G1CMRefProcTaskExecutor;
friend class G1CMKeepAliveAndDrainClosure;
@ -298,8 +298,14 @@ protected:
G1CMBitMapRO* _prevMarkBitMap; // Completed mark bitmap
G1CMBitMap* _nextMarkBitMap; // Under-construction mark bitmap
BitMap _region_bm;
BitMap _card_bm;
// Liveness count data. After marking G1 iterates over the recently gathered mark
// bitmap and records rough information about liveness on card and region basis.
// This information can be used for e.g. remembered set scrubbing.
// A set bit indicates whether the given region contains any live object.
BitMap _region_live_bm;
// A set bit indicates that the given card contains a live object.
BitMap _card_live_bm;
// Heap bounds
HeapWord* _heap_start;
@ -373,6 +379,14 @@ protected:
void swapMarkBitMaps();
// Allocates and returns a zero-ed out "large" bitmap of the given size in bits.
// It is always allocated using virtual memory.
BitMap allocate_large_bitmap(BitMap::idx_t size_in_bits);
// Allocates the memory for all bitmaps used by the concurrent marking.
void allocate_internal_bitmaps();
// Pre-touches the internal bitmaps.
void pretouch_internal_bitmaps();
// It resets the global marking data structures, as well as the
// task local ones; should be called during initial mark.
void reset();
@ -461,23 +475,6 @@ protected:
void enter_first_sync_barrier(uint worker_id);
void enter_second_sync_barrier(uint worker_id);
// Live Data Counting data structures...
// These data structures are initialized at the start of
// marking. They are written to while marking is active.
// They are aggregated during remark; the aggregated values
// are then used to populate the _region_bm, _card_bm, and
// the total live bytes, which are then subsequently updated
// during cleanup.
// An array of bitmaps (one bit map per task). Each bitmap
// is used to record the cards spanned by the live objects
// marked by that task/worker.
BitMap* _count_card_bitmaps;
// Used to record the number of marked live bytes
// (for each region, by worker thread).
size_t** _count_marked_bytes;
// Card index of the bottom of the G1 heap. Used for biasing indices into
// the card bitmaps.
intptr_t _heap_bottom_card_num;
@ -563,18 +560,10 @@ public:
// G1CollectedHeap
// This notifies CM that a root during initial-mark needs to be
// grayed. It is MT-safe. word_size is the size of the object in
// words. It is passed explicitly as sometimes we cannot calculate
// it from the given object because it might be in an inconsistent
// state (e.g., in to-space and being copied). So the caller is
// responsible for dealing with this issue (e.g., get the size from
// the from-space image when the to-space image might be
// inconsistent) and always passing the size. hr is the region that
// grayed. It is MT-safe. hr is the region that
// contains the object and it's passed optionally from callers who
// might already have it (no point in recalculating it).
inline void grayRoot(oop obj,
size_t word_size,
uint worker_id,
HeapRegion* hr = NULL);
// Prepare internal data structures for the next mark cycle. This includes clearing
@ -641,7 +630,7 @@ public:
inline bool do_yield_check(uint worker_i = 0);
// Called to abort the marking cycle after a Full GC takes place.
// Abandon current marking iteration due to a Full GC.
void abort();
bool has_aborted() { return _has_aborted; }
@ -652,75 +641,8 @@ public:
void print_on_error(outputStream* st) const;
// Liveness counting
// Utility routine to set an exclusive range of cards on the given
// card liveness bitmap
inline void set_card_bitmap_range(BitMap* card_bm,
BitMap::idx_t start_idx,
BitMap::idx_t end_idx,
bool is_par);
// Returns the card number of the bottom of the G1 heap.
// Used in biasing indices into accounting card bitmaps.
intptr_t heap_bottom_card_num() const {
return _heap_bottom_card_num;
}
// Returns the card bitmap for a given task or worker id.
BitMap* count_card_bitmap_for(uint worker_id) {
assert(worker_id < _max_worker_id, "oob");
assert(_count_card_bitmaps != NULL, "uninitialized");
BitMap* task_card_bm = &_count_card_bitmaps[worker_id];
assert(task_card_bm->size() == _card_bm.size(), "size mismatch");
return task_card_bm;
}
// Returns the array containing the marked bytes for each region,
// for the given worker or task id.
size_t* count_marked_bytes_array_for(uint worker_id) {
assert(worker_id < _max_worker_id, "oob");
assert(_count_marked_bytes != NULL, "uninitialized");
size_t* marked_bytes_array = _count_marked_bytes[worker_id];
assert(marked_bytes_array != NULL, "uninitialized");
return marked_bytes_array;
}
// Returns the index in the liveness accounting card table bitmap
// for the given address
inline BitMap::idx_t card_bitmap_index_for(HeapWord* addr);
// Counts the size of the given memory region in the the given
// marked_bytes array slot for the given HeapRegion.
// Sets the bits in the given card bitmap that are associated with the
// cards that are spanned by the memory region.
inline void count_region(MemRegion mr,
HeapRegion* hr,
size_t* marked_bytes_array,
BitMap* task_card_bm);
// Counts the given object in the given task/worker counting
// data structures.
inline void count_object(oop obj,
HeapRegion* hr,
size_t* marked_bytes_array,
BitMap* task_card_bm,
size_t word_size);
// Attempts to mark the given object and, if successful, counts
// the object in the given task/worker counting structures.
inline bool par_mark_and_count(oop obj,
HeapRegion* hr,
size_t* marked_bytes_array,
BitMap* task_card_bm);
// Attempts to mark the given object and, if successful, counts
// the object in the task/worker counting structures for the
// given worker id.
inline bool par_mark_and_count(oop obj,
size_t word_size,
HeapRegion* hr,
uint worker_id);
// Attempts to mark the given object on the next mark bitmap.
inline bool par_mark(oop obj);
// Returns true if initialization was successfully completed.
bool completed_initialization() const {
@ -730,19 +652,19 @@ public:
ConcurrentGCTimer* gc_timer_cm() const { return _gc_timer_cm; }
G1OldTracer* gc_tracer_cm() const { return _gc_tracer_cm; }
protected:
// Clear all the per-task bitmaps and arrays used to store the
// counting data.
void clear_all_count_data();
private:
// Clear (Reset) all liveness count data.
void clear_all_live_data(WorkGang* workers);
// Aggregates the counting data for each worker/task
// that was constructed while marking. Also sets
// the amount of marked bytes for each region and
// the top at concurrent mark count.
void aggregate_count_data();
// Verify all of the above data structures that they are in initial state.
void verify_all_live_data();
// Aggregates the per-card liveness data based on the current marking. Also sets
// the amount of marked bytes for each region.
void create_live_data();
// Verification routine
void verify_count_data();
void verify_live_data();
};
// A class representing a marking task.
@ -844,12 +766,6 @@ private:
TruncatedSeq _marking_step_diffs_ms;
// Counting data structures. Embedding the task's marked_bytes_array
// and card bitmap into the actual task saves having to go through
// the ConcurrentMark object.
size_t* _marked_bytes_array;
BitMap* _card_bm;
// it updates the local fields after this task has claimed
// a new region to scan
void setup_for_region(HeapRegion* hr);
@ -936,9 +852,8 @@ public:
// Grey the object by marking it. If not already marked, push it on
// the local queue if below the finger.
// Precondition: obj is in region.
// Precondition: obj is below region's NTAMS.
inline void make_reference_grey(oop obj, HeapRegion* region);
inline void make_reference_grey(oop obj);
// Grey the object (by calling make_grey_reference) if required,
// e.g. obj is below its containing region's NTAMS.
@ -976,8 +891,6 @@ public:
G1CMTask(uint worker_id,
G1ConcurrentMark *cm,
size_t* marked_bytes,
BitMap* card_bm,
G1CMTaskQueue* task_queue,
G1CMTaskQueueSet* task_queues);

View file

@ -29,138 +29,8 @@
#include "gc/g1/g1ConcurrentMark.hpp"
#include "gc/shared/taskqueue.inline.hpp"
// Utility routine to set an exclusive range of cards on the given
// card liveness bitmap
inline void G1ConcurrentMark::set_card_bitmap_range(BitMap* card_bm,
BitMap::idx_t start_idx,
BitMap::idx_t end_idx,
bool is_par) {
// Set the exclusive bit range [start_idx, end_idx).
assert((end_idx - start_idx) > 0, "at least one card");
assert(end_idx <= card_bm->size(), "sanity");
// Silently clip the end index
end_idx = MIN2(end_idx, card_bm->size());
// For small ranges use a simple loop; otherwise use set_range or
// use par_at_put_range (if parallel). The range is made up of the
// cards that are spanned by an object/mem region so 8 cards will
// allow up to object sizes up to 4K to be handled using the loop.
if ((end_idx - start_idx) <= 8) {
for (BitMap::idx_t i = start_idx; i < end_idx; i += 1) {
if (is_par) {
card_bm->par_set_bit(i);
} else {
card_bm->set_bit(i);
}
}
} else {
// Note BitMap::par_at_put_range() and BitMap::set_range() are exclusive.
if (is_par) {
card_bm->par_at_put_range(start_idx, end_idx, true);
} else {
card_bm->set_range(start_idx, end_idx);
}
}
}
// Returns the index in the liveness accounting card bitmap
// for the given address
inline BitMap::idx_t G1ConcurrentMark::card_bitmap_index_for(HeapWord* addr) {
// Below, the term "card num" means the result of shifting an address
// by the card shift -- address 0 corresponds to card number 0. One
// must subtract the card num of the bottom of the heap to obtain a
// card table index.
intptr_t card_num = intptr_t(uintptr_t(addr) >> CardTableModRefBS::card_shift);
return card_num - heap_bottom_card_num();
}
// Counts the given memory region in the given task/worker
// counting data structures.
inline void G1ConcurrentMark::count_region(MemRegion mr, HeapRegion* hr,
size_t* marked_bytes_array,
BitMap* task_card_bm) {
G1CollectedHeap* g1h = _g1h;
CardTableModRefBS* ct_bs = g1h->g1_barrier_set();
HeapWord* start = mr.start();
HeapWord* end = mr.end();
size_t region_size_bytes = mr.byte_size();
uint index = hr->hrm_index();
assert(hr == g1h->heap_region_containing(start), "sanity");
assert(marked_bytes_array != NULL, "pre-condition");
assert(task_card_bm != NULL, "pre-condition");
// Add to the task local marked bytes for this region.
marked_bytes_array[index] += region_size_bytes;
BitMap::idx_t start_idx = card_bitmap_index_for(start);
BitMap::idx_t end_idx = card_bitmap_index_for(end);
// Note: if we're looking at the last region in heap - end
// could be actually just beyond the end of the heap; end_idx
// will then correspond to a (non-existent) card that is also
// just beyond the heap.
if (g1h->is_in_g1_reserved(end) && !ct_bs->is_card_aligned(end)) {
// end of region is not card aligned - increment to cover
// all the cards spanned by the region.
end_idx += 1;
}
// The card bitmap is task/worker specific => no need to use
// the 'par' BitMap routines.
// Set bits in the exclusive bit range [start_idx, end_idx).
set_card_bitmap_range(task_card_bm, start_idx, end_idx, false /* is_par */);
}
// Counts the given object in the given task/worker counting data structures.
inline void G1ConcurrentMark::count_object(oop obj,
HeapRegion* hr,
size_t* marked_bytes_array,
BitMap* task_card_bm,
size_t word_size) {
assert(!hr->is_continues_humongous(), "Cannot enter count_object with continues humongous");
if (!hr->is_starts_humongous()) {
MemRegion mr((HeapWord*)obj, word_size);
count_region(mr, hr, marked_bytes_array, task_card_bm);
} else {
do {
MemRegion mr(hr->bottom(), hr->top());
count_region(mr, hr, marked_bytes_array, task_card_bm);
hr = _g1h->next_region_in_humongous(hr);
} while (hr != NULL);
}
}
// Attempts to mark the given object and, if successful, counts
// the object in the given task/worker counting structures.
inline bool G1ConcurrentMark::par_mark_and_count(oop obj,
HeapRegion* hr,
size_t* marked_bytes_array,
BitMap* task_card_bm) {
if (_nextMarkBitMap->parMark((HeapWord*)obj)) {
// Update the task specific count data for the object.
count_object(obj, hr, marked_bytes_array, task_card_bm, obj->size());
return true;
}
return false;
}
// Attempts to mark the given object and, if successful, counts
// the object in the task/worker counting structures for the
// given worker id.
inline bool G1ConcurrentMark::par_mark_and_count(oop obj,
size_t word_size,
HeapRegion* hr,
uint worker_id) {
if (_nextMarkBitMap->parMark((HeapWord*)obj)) {
size_t* marked_bytes_array = count_marked_bytes_array_for(worker_id);
BitMap* task_card_bm = count_card_bitmap_for(worker_id);
count_object(obj, hr, marked_bytes_array, task_card_bm, word_size);
return true;
}
return false;
inline bool G1ConcurrentMark::par_mark(oop obj) {
return _nextMarkBitMap->parMark((HeapWord*)obj);
}
inline bool G1CMBitMapRO::iterate(BitMapClosure* cl, MemRegion mr) {
@ -294,10 +164,8 @@ inline void G1CMTask::process_grey_object(oop obj) {
check_limits();
}
inline void G1CMTask::make_reference_grey(oop obj, HeapRegion* hr) {
if (_cm->par_mark_and_count(obj, hr, _marked_bytes_array, _card_bm)) {
inline void G1CMTask::make_reference_grey(oop obj) {
if (_cm->par_mark(obj)) {
// No OrderAccess:store_load() is needed. It is implicit in the
// CAS done in G1CMBitMap::parMark() call in the routine above.
HeapWord* global_finger = _cm->finger();
@ -348,7 +216,7 @@ inline void G1CMTask::deal_with_reference(oop obj) {
// anything with it).
HeapRegion* hr = _g1h->heap_region_containing(obj);
if (!hr->obj_allocated_since_next_marking(obj)) {
make_reference_grey(obj, hr);
make_reference_grey(obj);
}
}
}
@ -370,8 +238,7 @@ bool G1ConcurrentMark::isPrevMarked(oop p) const {
return _prevMarkBitMap->isMarked(addr);
}
inline void G1ConcurrentMark::grayRoot(oop obj, size_t word_size,
uint worker_id, HeapRegion* hr) {
inline void G1ConcurrentMark::grayRoot(oop obj, HeapRegion* hr) {
assert(obj != NULL, "pre-condition");
HeapWord* addr = (HeapWord*) obj;
if (hr == NULL) {
@ -386,7 +253,7 @@ inline void G1ConcurrentMark::grayRoot(oop obj, size_t word_size,
if (addr < hr->next_top_at_mark_start()) {
if (!_nextMarkBitMap->isMarked(addr)) {
par_mark_and_count(obj, word_size, hr, worker_id);
par_mark(obj);
}
}
}

View file

@ -95,8 +95,6 @@ public:
void do_object(oop obj) {
HeapWord* obj_addr = (HeapWord*) obj;
assert(_hr->is_in(obj_addr), "sanity");
size_t obj_size = obj->size();
HeapWord* obj_end = obj_addr + obj_size;
if (obj->is_forwarded() && obj->forwardee() == obj) {
// The object failed to move.
@ -119,8 +117,10 @@ public:
// explicitly and all objects in the CSet are considered
// (implicitly) live. So, we won't mark them explicitly and
// we'll leave them over NTAMS.
_cm->grayRoot(obj, obj_size, _worker_id, _hr);
_cm->grayRoot(obj, _hr);
}
size_t obj_size = obj->size();
_marked_bytes += (obj_size * HeapWordSize);
obj->set_mark(markOopDesc::prototype());
@ -138,6 +138,7 @@ public:
// the collection set. So, we'll recreate such entries now.
obj->oop_iterate(_update_rset_cl);
HeapWord* obj_end = obj_addr + obj_size;
_last_forwarded_object_end = obj_end;
_hr->cross_threshold(obj_addr, obj_end);
}

View file

@ -131,7 +131,7 @@ inline void G1RootRegionScanClosure::do_oop_nv(T* p) {
if (!oopDesc::is_null(heap_oop)) {
oop obj = oopDesc::decode_heap_oop_not_null(heap_oop);
HeapRegion* hr = _g1h->heap_region_containing((HeapWord*) obj);
_cm->grayRoot(obj, obj->size(), _worker_id, hr);
_cm->grayRoot(obj, hr);
}
}
@ -246,7 +246,7 @@ void G1ParCopyHelper::mark_object(oop obj) {
assert(!_g1->heap_region_containing(obj)->in_collection_set(), "should not mark objects in the CSet");
// We know that the object is not moving so it's safe to read its size.
_cm->grayRoot(obj, (size_t) obj->size(), _worker_id);
_cm->grayRoot(obj);
}
void G1ParCopyHelper::mark_forwarded_object(oop from_obj, oop to_obj) {
@ -261,7 +261,7 @@ void G1ParCopyHelper::mark_forwarded_object(oop from_obj, oop to_obj) {
// worker so we cannot trust that its to-space image is
// well-formed. So we have to read its size from its from-space
// image which we know should not be changing.
_cm->grayRoot(to_obj, (size_t) from_obj->size(), _worker_id);
_cm->grayRoot(to_obj);
}
template <G1Barrier barrier, G1Mark do_mark_object, bool use_ext>

View file

@ -260,6 +260,9 @@
"The target number of mixed GCs after a marking cycle.") \
range(0, max_uintx) \
\
experimental(bool, G1PretouchAuxiliaryMemory, false, \
"Pre-touch large auxiliary data structures used by the GC.") \
\
experimental(bool, G1EagerReclaimHumongousObjects, true, \
"Try to reclaim dead large objects at every young GC.") \
\

View file

@ -68,6 +68,10 @@ void BitMap::resize(idx_t size_in_bits, bool in_resource_area) {
}
}
void BitMap::pretouch() {
os::pretouch_memory((char*)word_addr(0), (char*)word_addr(size()));
}
void BitMap::set_range_within_word(idx_t beg, idx_t end) {
// With a valid range (beg <= end), this test ensures that end != 0, as
// required by inverted_bit_mask_for_range. Also avoids an unnecessary write.

View file

@ -135,12 +135,19 @@ class BitMap VALUE_OBJ_CLASS_SPEC {
// use the same value for "in_resource_area".)
void resize(idx_t size_in_bits, bool in_resource_area = true);
// Pretouch the entire range of memory this BitMap covers.
void pretouch();
// Accessing
idx_t size() const { return _size; }
idx_t size_in_words() const {
return word_index(size() + BitsPerWord - 1);
}
static idx_t size_in_words(size_t size_in_bits) {
return word_index(size_in_bits + BitsPerWord - 1);
}
bool at(idx_t index) const {
verify_index(index);
return (*word_addr(index) & bit_mask(index)) != 0;

View file

@ -25,6 +25,9 @@
* @test Test2GbHeap
* @bug 8031686
* @summary Regression test to ensure we can start G1 with 2gb heap.
* Skip test on 32 bit Windows: it typically does not support the many and large virtual memory reservations needed.
* @requires (vm.gc == "G1" | vm.gc == "null")
* @requires !((sun.arch.data.model == "32") & (os.family == "windows"))
* @key gc
* @key regression
* @library /testlibrary
@ -48,17 +51,6 @@ public class Test2GbHeap {
ProcessBuilder pb = ProcessTools.createJavaProcessBuilder(testArguments.toArray(new String[0]));
OutputAnalyzer output = new OutputAnalyzer(pb.start());
// Avoid failing test for setups not supported.
if (output.getOutput().contains("Could not reserve enough space for 2097152KB object heap")) {
// Will fail on machines with too little memory (and Windows 32-bit VM), ignore such failures.
output.shouldHaveExitValue(1);
} else if (output.getOutput().contains("-XX:+UseG1GC not supported in this VM")) {
// G1 is not supported on embedded, ignore such failures.
output.shouldHaveExitValue(1);
} else {
// Normally everything should be fine.
output.shouldHaveExitValue(0);
}
output.shouldHaveExitValue(0);
}
}