提交 8c70e3ca 编写于 作者: J johnc

6484965: G1: piggy-back liveness accounting phase on marking

Summary: Remove the separate counting phase of concurrent marking by tracking the amount of marked bytes and the cards spanned by marked objects in marking task/worker thread local data structures, which are updated as individual objects are marked.
Reviewed-by: brutisso, tonyp
上级 3c651de6
......@@ -426,7 +426,6 @@ protected:
WorkGangBarrierSync _first_overflow_barrier_sync;
WorkGangBarrierSync _second_overflow_barrier_sync;
// this is set by any task, when an overflow on the global data
// structures is detected.
volatile bool _has_overflown;
......@@ -578,6 +577,27 @@ protected:
}
}
// 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;
public:
// Manipulation of the global mark stack.
// Notice that the first mark_stack_push is CAS-based, whereas the
......@@ -703,6 +723,7 @@ public:
ConcurrentMark(ReservedSpace rs, int max_regions);
~ConcurrentMark();
ConcurrentMarkThread* cmThread() { return _cmThread; }
CMBitMapRO* prevMarkBitMap() const { return _prevMarkBitMap; }
......@@ -721,7 +742,7 @@ public:
// This notifies CM that a root during initial-mark needs to be
// grayed. It is MT-safe.
inline void grayRoot(oop obj, size_t word_size);
inline void grayRoot(oop obj, size_t word_size, uint worker_id);
// It's used during evacuation pauses to gray a region, if
// necessary, and it's MT-safe. It assumes that the caller has
......@@ -781,15 +802,13 @@ public:
void checkpointRootsFinal(bool clear_all_soft_refs);
void checkpointRootsFinalWork();
void calcDesiredRegions();
void cleanup();
void completeCleanup();
// Mark in the previous bitmap. NB: this is usually read-only, so use
// this carefully!
inline void markPrev(oop p);
inline void markNext(oop p);
void clear(oop p);
// Clears marks for all objects in the given range, for the prev,
// next, or both bitmaps. NB: the previous bitmap is usually
// read-only, so use this carefully!
......@@ -913,6 +932,104 @@ public:
bool verbose_high() {
return _MARKING_VERBOSE_ && _verbose_level >= high_verbose;
}
// Counting data structure accessors
// 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(0 <= worker_id && worker_id < _max_task_num, "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(0 <= worker_id && worker_id < _max_task_num, "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 memory region in the task/worker counting
// data structures for the given worker id.
inline void count_region(MemRegion mr, uint worker_id);
// 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);
// Counts the given object in the task/worker counting data
// structures for the given worker id.
inline void count_object(oop obj, HeapRegion* hr, uint worker_id);
// 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, HeapRegion* hr, uint worker_id);
// Similar to the above routine but we don't know the heap region that
// contains the object to be marked/counted, which this routine looks up.
inline bool par_mark_and_count(oop obj, uint worker_id);
// Similar to the above routine but there are times when we cannot
// safely calculate the size of obj due to races and we, therefore,
// pass the size in as a parameter. It is the caller's reponsibility
// to ensure that the size passed in for obj is valid.
inline bool par_mark_and_count(oop obj, size_t word_size, uint worker_id);
// Unconditionally mark the given object, and unconditinally count
// the object in the counting structures for worker id 0.
// Should *not* be called from parallel code.
inline bool mark_and_count(oop obj, HeapRegion* hr);
// Similar to the above routine but we don't know the heap region that
// contains the object to be marked/counted, which this routine looks up.
// Should *not* be called from parallel code.
inline bool mark_and_count(oop obj);
protected:
// Clear all the per-task bitmaps and arrays used to store the
// counting data.
void clear_all_count_data();
// 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();
// Verification routine
void verify_count_data();
};
// A class representing a marking task.
......@@ -1031,6 +1148,12 @@ 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;
// LOTS of statistics related with this task
#if _MARKING_STATS_
NumberSeq _all_clock_intervals_ms;
......@@ -1196,6 +1319,7 @@ public:
}
CMTask(int task_num, ConcurrentMark *cm,
size_t* marked_bytes, BitMap* card_bm,
CMTaskQueue* task_queue, CMTaskQueueSet* task_queues);
// it prints statistics associated with this task
......
......@@ -28,6 +28,159 @@
#include "gc_implementation/g1/concurrentMark.hpp"
#include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
// Returns the index in the liveness accounting card bitmap
// for the given address
inline BitMap::idx_t ConcurrentMark::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 ConcurrentMark::count_region(MemRegion mr, HeapRegion* hr,
size_t* marked_bytes_array,
BitMap* task_card_bm) {
G1CollectedHeap* g1h = _g1h;
HeapWord* start = mr.start();
HeapWord* last = mr.last();
size_t region_size_bytes = mr.byte_size();
size_t index = hr->hrs_index();
assert(!hr->continuesHumongous(), "should not be HC region");
assert(hr == g1h->heap_region_containing(start), "sanity");
assert(hr == g1h->heap_region_containing(mr.last()), "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 last_idx = card_bitmap_index_for(last);
// The card bitmap is task/worker specific => no need to use 'par' routines.
// Set bits in the inclusive bit range [start_idx, last_idx].
//
// For small ranges use a simple loop; otherwise use set_range
// The range are the cards that are spanned by the object/region
// so 8 cards will allow objects/regions up to 4K to be handled
// using the loop.
if ((last_idx - start_idx) <= 8) {
for (BitMap::idx_t i = start_idx; i <= last_idx; i += 1) {
task_card_bm->set_bit(i);
}
} else {
assert(last_idx < task_card_bm->size(), "sanity");
// Note: BitMap::set_range() is exclusive.
task_card_bm->set_range(start_idx, last_idx+1);
}
}
// Counts the given memory region, which may be a single object, in the
// task/worker counting data structures for the given worker id.
inline void ConcurrentMark::count_region(MemRegion mr, uint worker_id) {
size_t* marked_bytes_array = count_marked_bytes_array_for(worker_id);
BitMap* task_card_bm = count_card_bitmap_for(worker_id);
HeapWord* addr = mr.start();
HeapRegion* hr = _g1h->heap_region_containing_raw(addr);
count_region(mr, hr, marked_bytes_array, task_card_bm);
}
// Counts the given object in the given task/worker counting data structures.
inline void ConcurrentMark::count_object(oop obj,
HeapRegion* hr,
size_t* marked_bytes_array,
BitMap* task_card_bm) {
MemRegion mr((HeapWord*)obj, obj->size());
count_region(mr, hr, marked_bytes_array, task_card_bm);
}
// Counts the given object in the task/worker counting data
// structures for the given worker id.
inline void ConcurrentMark::count_object(oop obj, HeapRegion* hr, uint worker_id) {
size_t* marked_bytes_array = count_marked_bytes_array_for(worker_id);
BitMap* task_card_bm = count_card_bitmap_for(worker_id);
HeapWord* addr = (HeapWord*) obj;
count_object(obj, hr, marked_bytes_array, task_card_bm);
}
// Attempts to mark the given object and, if successful, counts
// the object in the given task/worker counting structures.
inline bool ConcurrentMark::par_mark_and_count(oop obj,
HeapRegion* hr,
size_t* marked_bytes_array,
BitMap* task_card_bm) {
HeapWord* addr = (HeapWord*)obj;
if (_nextMarkBitMap->parMark(addr)) {
// Update the task specific count data for the object.
count_object(obj, hr, marked_bytes_array, task_card_bm);
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 ConcurrentMark::par_mark_and_count(oop obj,
HeapRegion* hr,
uint worker_id) {
HeapWord* addr = (HeapWord*)obj;
if (_nextMarkBitMap->parMark(addr)) {
// Update the task specific count data for the object.
count_object(obj, hr, worker_id);
return true;
}
return false;
}
// As above - but we don't know the heap region containing the
// object and so have to supply it.
inline bool ConcurrentMark::par_mark_and_count(oop obj, uint worker_id) {
HeapWord* addr = (HeapWord*)obj;
HeapRegion* hr = _g1h->heap_region_containing_raw(addr);
return par_mark_and_count(obj, hr, worker_id);
}
// Similar to the above routine but we already know the size, in words, of
// the object that we wish to mark/count
inline bool ConcurrentMark::par_mark_and_count(oop obj,
size_t word_size,
uint worker_id) {
HeapWord* addr = (HeapWord*)obj;
if (_nextMarkBitMap->parMark(addr)) {
// Update the task specific count data for the object.
MemRegion mr(addr, word_size);
count_region(mr, worker_id);
return true;
}
return false;
}
// Unconditionally mark the given object, and unconditinally count
// the object in the counting structures for worker id 0.
// Should *not* be called from parallel code.
inline bool ConcurrentMark::mark_and_count(oop obj, HeapRegion* hr) {
HeapWord* addr = (HeapWord*)obj;
_nextMarkBitMap->mark(addr);
// Update the task specific count data for the object.
count_object(obj, hr, 0 /* worker_id */);
return true;
}
// As above - but we don't have the heap region containing the
// object, so we have to supply it.
inline bool ConcurrentMark::mark_and_count(oop obj) {
HeapWord* addr = (HeapWord*)obj;
HeapRegion* hr = _g1h->heap_region_containing_raw(addr);
return mark_and_count(obj, hr);
}
inline bool CMBitMapRO::iterate(BitMapClosure* cl, MemRegion mr) {
HeapWord* start_addr = MAX2(startWord(), mr.start());
HeapWord* end_addr = MIN2(endWord(), mr.end());
......@@ -113,7 +266,7 @@ inline void CMTask::deal_with_reference(oop obj) {
HeapWord* objAddr = (HeapWord*) obj;
assert(obj->is_oop_or_null(true /* ignore mark word */), "Error");
if (_g1h->is_in_g1_reserved(objAddr)) {
if (_g1h->is_in_g1_reserved(objAddr)) {
assert(obj != NULL, "null check is implicit");
if (!_nextMarkBitMap->isMarked(objAddr)) {
// Only get the containing region if the object is not marked on the
......@@ -127,9 +280,9 @@ inline void CMTask::deal_with_reference(oop obj) {
}
// we need to mark it first
if (_nextMarkBitMap->parMark(objAddr)) {
if (_cm->par_mark_and_count(obj, hr, _marked_bytes_array, _card_bm)) {
// No OrderAccess:store_load() is needed. It is implicit in the
// CAS done in parMark(objAddr) above
// CAS done in CMBitMap::parMark() call in the routine above.
HeapWord* global_finger = _cm->finger();
#if _CHECK_BOTH_FINGERS_
......@@ -189,12 +342,7 @@ inline void ConcurrentMark::markPrev(oop p) {
((CMBitMap*)_prevMarkBitMap)->mark((HeapWord*) p);
}
inline void ConcurrentMark::markNext(oop p) {
assert(!_nextMarkBitMap->isMarked((HeapWord*) p), "sanity");
_nextMarkBitMap->mark((HeapWord*) p);
}
inline void ConcurrentMark::grayRoot(oop obj, size_t word_size) {
inline void ConcurrentMark::grayRoot(oop obj, size_t word_size, uint worker_id) {
HeapWord* addr = (HeapWord*) obj;
// Currently we don't do anything with word_size but we will use it
......@@ -220,7 +368,7 @@ inline void ConcurrentMark::grayRoot(oop obj, size_t word_size) {
#endif // ASSERT
if (!_nextMarkBitMap->isMarked(addr)) {
_nextMarkBitMap->parMark(addr);
par_mark_and_count(obj, word_size, worker_id);
}
}
......
/*
* Copyright (c) 2001, 2011, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2001, 2012, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
......@@ -44,9 +44,7 @@ ConcurrentMarkThread::ConcurrentMarkThread(ConcurrentMark* cm) :
_started(false),
_in_progress(false),
_vtime_accum(0.0),
_vtime_mark_accum(0.0),
_vtime_count_accum(0.0)
{
_vtime_mark_accum(0.0) {
create_and_start();
}
......@@ -148,36 +146,12 @@ void ConcurrentMarkThread::run() {
}
} while (cm()->restart_for_overflow());
double counting_start_time = os::elapsedVTime();
if (!cm()->has_aborted()) {
double count_start_sec = os::elapsedTime();
if (PrintGC) {
gclog_or_tty->date_stamp(PrintGCDateStamps);
gclog_or_tty->stamp(PrintGCTimeStamps);
gclog_or_tty->print_cr("[GC concurrent-count-start]");
}
_sts.join();
_cm->calcDesiredRegions();
_sts.leave();
if (!cm()->has_aborted()) {
double count_end_sec = os::elapsedTime();
if (PrintGC) {
gclog_or_tty->date_stamp(PrintGCDateStamps);
gclog_or_tty->stamp(PrintGCTimeStamps);
gclog_or_tty->print_cr("[GC concurrent-count-end, %1.7lf]",
count_end_sec - count_start_sec);
}
}
}
double end_time = os::elapsedVTime();
_vtime_count_accum += (end_time - counting_start_time);
// Update the total virtual time before doing this, since it will try
// to measure it to get the vtime for this marking. We purposely
// neglect the presumably-short "completeCleanup" phase here.
_vtime_accum = (end_time - _vtime_start);
if (!cm()->has_aborted()) {
if (g1_policy->adaptive_young_list_length()) {
double now = os::elapsedTime();
......
/*
* Copyright (c) 2001, 2010, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2001, 2012, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
......@@ -40,7 +40,6 @@ class ConcurrentMarkThread: public ConcurrentGCThread {
double _vtime_accum; // Accumulated virtual time.
double _vtime_mark_accum;
double _vtime_count_accum;
public:
virtual void run();
......@@ -69,8 +68,6 @@ class ConcurrentMarkThread: public ConcurrentGCThread {
double vtime_accum();
// Marking virtual time so far
double vtime_mark_accum();
// Counting virtual time so far.
double vtime_count_accum() { return _vtime_count_accum; }
ConcurrentMark* cm() { return _cm; }
......
......@@ -4200,7 +4200,7 @@ HeapWord* G1CollectedHeap::par_allocate_during_gc(GCAllocPurpose purpose,
G1ParGCAllocBuffer::G1ParGCAllocBuffer(size_t gclab_word_size) :
ParGCAllocBuffer(gclab_word_size), _retired(false) { }
G1ParScanThreadState::G1ParScanThreadState(G1CollectedHeap* g1h, int queue_num)
G1ParScanThreadState::G1ParScanThreadState(G1CollectedHeap* g1h, uint queue_num)
: _g1h(g1h),
_refs(g1h->task_queue(queue_num)),
_dcq(&g1h->dirty_card_queue_set()),
......@@ -4321,6 +4321,7 @@ G1ParClosureSuper::G1ParClosureSuper(G1CollectedHeap* g1,
G1ParScanThreadState* par_scan_state) :
_g1(g1), _g1_rem(_g1->g1_rem_set()), _cm(_g1->concurrent_mark()),
_par_scan_state(par_scan_state),
_worker_id(par_scan_state->queue_num()),
_during_initial_mark(_g1->g1_policy()->during_initial_mark_pause()),
_mark_in_progress(_g1->mark_in_progress()) { }
......@@ -4332,7 +4333,7 @@ void G1ParCopyHelper::mark_object(oop obj) {
#endif // ASSERT
// We know that the object is not moving so it's safe to read its size.
_cm->grayRoot(obj, (size_t) obj->size());
_cm->grayRoot(obj, (size_t) obj->size(), _worker_id);
}
void G1ParCopyHelper::mark_forwarded_object(oop from_obj, oop to_obj) {
......@@ -4354,7 +4355,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());
_cm->grayRoot(to_obj, (size_t) from_obj->size(), _worker_id);
}
oop G1ParCopyHelper::copy_to_survivor_space(oop old) {
......@@ -4444,6 +4445,8 @@ void G1ParCopyClosure<do_gen_barrier, barrier, do_mark_object>
assert(barrier != G1BarrierRS || obj != NULL,
"Precondition: G1BarrierRS implies obj is non-NULL");
assert(_worker_id == _par_scan_state->queue_num(), "sanity");
// here the null check is implicit in the cset_fast_test() test
if (_g1->in_cset_fast_test(obj)) {
oop forwardee;
......@@ -4462,7 +4465,7 @@ void G1ParCopyClosure<do_gen_barrier, barrier, do_mark_object>
// When scanning the RS, we only care about objs in CS.
if (barrier == G1BarrierRS) {
_par_scan_state->update_rs(_from, p, _par_scan_state->queue_num());
_par_scan_state->update_rs(_from, p, _worker_id);
}
} else {
// The object is not in collection set. If we're a root scanning
......@@ -4474,7 +4477,7 @@ void G1ParCopyClosure<do_gen_barrier, barrier, do_mark_object>
}
if (barrier == G1BarrierEvac && obj != NULL) {
_par_scan_state->update_rs(_from, p, _par_scan_state->queue_num());
_par_scan_state->update_rs(_from, p, _worker_id);
}
if (do_gen_barrier && obj != NULL) {
......@@ -5704,16 +5707,6 @@ void G1CollectedHeap::free_collection_set(HeapRegion* cs_head) {
// And the region is empty.
assert(!used_mr.is_empty(), "Should not have empty regions in a CS.");
// If marking is in progress then clear any objects marked in
// the current region. Note mark_in_progress() returns false,
// even during an initial mark pause, until the set_marking_started()
// call which takes place later in the pause.
if (mark_in_progress()) {
assert(!g1_policy()->during_initial_mark_pause(), "sanity");
_cm->nextMarkBitMap()->clearRange(used_mr);
}
free_region(cur, &pre_used, &local_free_list, false /* par */);
} else {
cur->uninstall_surv_rate_group();
......
......@@ -1909,7 +1909,7 @@ protected:
G1ParScanPartialArrayClosure* _partial_scan_cl;
int _hash_seed;
int _queue_num;
uint _queue_num;
size_t _term_attempts;
......@@ -1953,7 +1953,7 @@ protected:
}
public:
G1ParScanThreadState(G1CollectedHeap* g1h, int queue_num);
G1ParScanThreadState(G1CollectedHeap* g1h, uint queue_num);
~G1ParScanThreadState() {
FREE_C_HEAP_ARRAY(size_t, _surviving_young_words_base);
......@@ -2045,7 +2045,7 @@ public:
}
int* hash_seed() { return &_hash_seed; }
int queue_num() { return _queue_num; }
uint queue_num() { return _queue_num; }
size_t term_attempts() const { return _term_attempts; }
void note_term_attempt() { _term_attempts++; }
......
......@@ -70,16 +70,20 @@ private:
OopsInHeapRegionClosure *_update_rset_cl;
bool _during_initial_mark;
bool _during_conc_mark;
uint _worker_id;
public:
RemoveSelfForwardPtrObjClosure(G1CollectedHeap* g1, ConcurrentMark* cm,
HeapRegion* hr,
OopsInHeapRegionClosure* update_rset_cl,
bool during_initial_mark,
bool during_conc_mark) :
bool during_conc_mark,
uint worker_id) :
_g1(g1), _cm(cm), _hr(hr), _marked_bytes(0),
_update_rset_cl(update_rset_cl),
_during_initial_mark(during_initial_mark),
_during_conc_mark(during_conc_mark) { }
_during_conc_mark(during_conc_mark),
_worker_id(worker_id) { }
size_t marked_bytes() { return _marked_bytes; }
......@@ -123,7 +127,7 @@ 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->markNext(obj);
_cm->grayRoot(obj, obj_size, _worker_id);
}
_marked_bytes += (obj_size * HeapWordSize);
obj->set_mark(markOopDesc::prototype());
......@@ -155,12 +159,14 @@ class RemoveSelfForwardPtrHRClosure: public HeapRegionClosure {
G1CollectedHeap* _g1h;
ConcurrentMark* _cm;
OopsInHeapRegionClosure *_update_rset_cl;
uint _worker_id;
public:
RemoveSelfForwardPtrHRClosure(G1CollectedHeap* g1h,
OopsInHeapRegionClosure* update_rset_cl) :
OopsInHeapRegionClosure* update_rset_cl,
uint worker_id) :
_g1h(g1h), _update_rset_cl(update_rset_cl),
_cm(_g1h->concurrent_mark()) { }
_worker_id(worker_id), _cm(_g1h->concurrent_mark()) { }
bool doHeapRegion(HeapRegion *hr) {
bool during_initial_mark = _g1h->g1_policy()->during_initial_mark_pause();
......@@ -173,7 +179,8 @@ public:
if (hr->evacuation_failed()) {
RemoveSelfForwardPtrObjClosure rspc(_g1h, _cm, hr, _update_rset_cl,
during_initial_mark,
during_conc_mark);
during_conc_mark,
_worker_id);
MemRegion mr(hr->bottom(), hr->end());
// We'll recreate the prev marking info so we'll first clear
......@@ -226,7 +233,7 @@ public:
update_rset_cl = &immediate_update;
}
RemoveSelfForwardPtrHRClosure rsfp_cl(_g1h, update_rset_cl);
RemoveSelfForwardPtrHRClosure rsfp_cl(_g1h, update_rset_cl, worker_id);
HeapRegion* hr = _g1h->start_cset_region_for_worker(worker_id);
_g1h->collection_set_iterate_from(hr, &rsfp_cl);
......
......@@ -51,6 +51,7 @@ protected:
G1RemSet* _g1_rem;
ConcurrentMark* _cm;
G1ParScanThreadState* _par_scan_state;
uint _worker_id;
bool _during_initial_mark;
bool _mark_in_progress;
public:
......
......@@ -374,7 +374,9 @@ class HeapRegion: public G1OffsetTableContigSpace {
ParVerifyClaimValue = 4,
RebuildRSClaimValue = 5,
CompleteMarkCSetClaimValue = 6,
ParEvacFailureClaimValue = 7
ParEvacFailureClaimValue = 7,
AggregateCountClaimValue = 8,
VerifyCountClaimValue = 9
};
inline HeapWord* par_allocate_no_bot_updates(size_t word_size) {
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册