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

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
This commit is contained in:
John Cuthbertson 2012-01-12 00:06:47 -08:00
parent 8861552c07
commit 0fa6fc2279
10 changed files with 1022 additions and 268 deletions

File diff suppressed because it is too large Load diff

View file

@ -426,7 +426,6 @@ protected:
WorkGangBarrierSync _first_overflow_barrier_sync; WorkGangBarrierSync _first_overflow_barrier_sync;
WorkGangBarrierSync _second_overflow_barrier_sync; WorkGangBarrierSync _second_overflow_barrier_sync;
// this is set by any task, when an overflow on the global data // this is set by any task, when an overflow on the global data
// structures is detected. // structures is detected.
volatile bool _has_overflown; 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: public:
// Manipulation of the global mark stack. // Manipulation of the global mark stack.
// Notice that the first mark_stack_push is CAS-based, whereas the // Notice that the first mark_stack_push is CAS-based, whereas the
@ -703,6 +723,7 @@ public:
ConcurrentMark(ReservedSpace rs, int max_regions); ConcurrentMark(ReservedSpace rs, int max_regions);
~ConcurrentMark(); ~ConcurrentMark();
ConcurrentMarkThread* cmThread() { return _cmThread; } ConcurrentMarkThread* cmThread() { return _cmThread; }
CMBitMapRO* prevMarkBitMap() const { return _prevMarkBitMap; } CMBitMapRO* prevMarkBitMap() const { return _prevMarkBitMap; }
@ -721,7 +742,7 @@ public:
// This notifies CM that a root during initial-mark needs to be // This notifies CM that a root during initial-mark needs to be
// grayed. It is MT-safe. // 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 // It's used during evacuation pauses to gray a region, if
// necessary, and it's MT-safe. It assumes that the caller has // 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 checkpointRootsFinal(bool clear_all_soft_refs);
void checkpointRootsFinalWork(); void checkpointRootsFinalWork();
void calcDesiredRegions();
void cleanup(); void cleanup();
void completeCleanup(); void completeCleanup();
// Mark in the previous bitmap. NB: this is usually read-only, so use // Mark in the previous bitmap. NB: this is usually read-only, so use
// this carefully! // this carefully!
inline void markPrev(oop p); 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, // Clears marks for all objects in the given range, for the prev,
// next, or both bitmaps. NB: the previous bitmap is usually // next, or both bitmaps. NB: the previous bitmap is usually
// read-only, so use this carefully! // read-only, so use this carefully!
@ -913,6 +932,104 @@ public:
bool verbose_high() { bool verbose_high() {
return _MARKING_VERBOSE_ && _verbose_level >= high_verbose; 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. // A class representing a marking task.
@ -1031,6 +1148,12 @@ private:
TruncatedSeq _marking_step_diffs_ms; 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 // LOTS of statistics related with this task
#if _MARKING_STATS_ #if _MARKING_STATS_
NumberSeq _all_clock_intervals_ms; NumberSeq _all_clock_intervals_ms;
@ -1196,6 +1319,7 @@ public:
} }
CMTask(int task_num, ConcurrentMark *cm, CMTask(int task_num, ConcurrentMark *cm,
size_t* marked_bytes, BitMap* card_bm,
CMTaskQueue* task_queue, CMTaskQueueSet* task_queues); CMTaskQueue* task_queue, CMTaskQueueSet* task_queues);
// it prints statistics associated with this task // it prints statistics associated with this task

View file

@ -28,6 +28,159 @@
#include "gc_implementation/g1/concurrentMark.hpp" #include "gc_implementation/g1/concurrentMark.hpp"
#include "gc_implementation/g1/g1CollectedHeap.inline.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) { inline bool CMBitMapRO::iterate(BitMapClosure* cl, MemRegion mr) {
HeapWord* start_addr = MAX2(startWord(), mr.start()); HeapWord* start_addr = MAX2(startWord(), mr.start());
HeapWord* end_addr = MIN2(endWord(), mr.end()); HeapWord* end_addr = MIN2(endWord(), mr.end());
@ -113,7 +266,7 @@ inline void CMTask::deal_with_reference(oop obj) {
HeapWord* objAddr = (HeapWord*) obj; HeapWord* objAddr = (HeapWord*) obj;
assert(obj->is_oop_or_null(true /* ignore mark word */), "Error"); 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"); assert(obj != NULL, "null check is implicit");
if (!_nextMarkBitMap->isMarked(objAddr)) { if (!_nextMarkBitMap->isMarked(objAddr)) {
// Only get the containing region if the object is not marked on the // 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 // 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 // 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(); HeapWord* global_finger = _cm->finger();
#if _CHECK_BOTH_FINGERS_ #if _CHECK_BOTH_FINGERS_
@ -189,12 +342,7 @@ inline void ConcurrentMark::markPrev(oop p) {
((CMBitMap*)_prevMarkBitMap)->mark((HeapWord*) p); ((CMBitMap*)_prevMarkBitMap)->mark((HeapWord*) p);
} }
inline void ConcurrentMark::markNext(oop p) { inline void ConcurrentMark::grayRoot(oop obj, size_t word_size, uint worker_id) {
assert(!_nextMarkBitMap->isMarked((HeapWord*) p), "sanity");
_nextMarkBitMap->mark((HeapWord*) p);
}
inline void ConcurrentMark::grayRoot(oop obj, size_t word_size) {
HeapWord* addr = (HeapWord*) obj; HeapWord* addr = (HeapWord*) obj;
// Currently we don't do anything with word_size but we will use it // 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 #endif // ASSERT
if (!_nextMarkBitMap->isMarked(addr)) { if (!_nextMarkBitMap->isMarked(addr)) {
_nextMarkBitMap->parMark(addr); par_mark_and_count(obj, word_size, worker_id);
} }
} }

View file

@ -1,5 +1,5 @@
/* /*
* 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. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
* *
* This code is free software; you can redistribute it and/or modify it * This code is free software; you can redistribute it and/or modify it
@ -44,9 +44,7 @@ ConcurrentMarkThread::ConcurrentMarkThread(ConcurrentMark* cm) :
_started(false), _started(false),
_in_progress(false), _in_progress(false),
_vtime_accum(0.0), _vtime_accum(0.0),
_vtime_mark_accum(0.0), _vtime_mark_accum(0.0) {
_vtime_count_accum(0.0)
{
create_and_start(); create_and_start();
} }
@ -148,36 +146,12 @@ void ConcurrentMarkThread::run() {
} }
} while (cm()->restart_for_overflow()); } 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(); 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 // Update the total virtual time before doing this, since it will try
// to measure it to get the vtime for this marking. We purposely // to measure it to get the vtime for this marking. We purposely
// neglect the presumably-short "completeCleanup" phase here. // neglect the presumably-short "completeCleanup" phase here.
_vtime_accum = (end_time - _vtime_start); _vtime_accum = (end_time - _vtime_start);
if (!cm()->has_aborted()) { if (!cm()->has_aborted()) {
if (g1_policy->adaptive_young_list_length()) { if (g1_policy->adaptive_young_list_length()) {
double now = os::elapsedTime(); double now = os::elapsedTime();

View file

@ -1,5 +1,5 @@
/* /*
* 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. * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
* *
* This code is free software; you can redistribute it and/or modify it * 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_accum; // Accumulated virtual time.
double _vtime_mark_accum; double _vtime_mark_accum;
double _vtime_count_accum;
public: public:
virtual void run(); virtual void run();
@ -69,8 +68,6 @@ class ConcurrentMarkThread: public ConcurrentGCThread {
double vtime_accum(); double vtime_accum();
// Marking virtual time so far // Marking virtual time so far
double vtime_mark_accum(); double vtime_mark_accum();
// Counting virtual time so far.
double vtime_count_accum() { return _vtime_count_accum; }
ConcurrentMark* cm() { return _cm; } ConcurrentMark* cm() { return _cm; }

View file

@ -4200,7 +4200,7 @@ HeapWord* G1CollectedHeap::par_allocate_during_gc(GCAllocPurpose purpose,
G1ParGCAllocBuffer::G1ParGCAllocBuffer(size_t gclab_word_size) : G1ParGCAllocBuffer::G1ParGCAllocBuffer(size_t gclab_word_size) :
ParGCAllocBuffer(gclab_word_size), _retired(false) { } ParGCAllocBuffer(gclab_word_size), _retired(false) { }
G1ParScanThreadState::G1ParScanThreadState(G1CollectedHeap* g1h, int queue_num) G1ParScanThreadState::G1ParScanThreadState(G1CollectedHeap* g1h, uint queue_num)
: _g1h(g1h), : _g1h(g1h),
_refs(g1h->task_queue(queue_num)), _refs(g1h->task_queue(queue_num)),
_dcq(&g1h->dirty_card_queue_set()), _dcq(&g1h->dirty_card_queue_set()),
@ -4321,6 +4321,7 @@ G1ParClosureSuper::G1ParClosureSuper(G1CollectedHeap* g1,
G1ParScanThreadState* par_scan_state) : G1ParScanThreadState* par_scan_state) :
_g1(g1), _g1_rem(_g1->g1_rem_set()), _cm(_g1->concurrent_mark()), _g1(g1), _g1_rem(_g1->g1_rem_set()), _cm(_g1->concurrent_mark()),
_par_scan_state(par_scan_state), _par_scan_state(par_scan_state),
_worker_id(par_scan_state->queue_num()),
_during_initial_mark(_g1->g1_policy()->during_initial_mark_pause()), _during_initial_mark(_g1->g1_policy()->during_initial_mark_pause()),
_mark_in_progress(_g1->mark_in_progress()) { } _mark_in_progress(_g1->mark_in_progress()) { }
@ -4332,7 +4333,7 @@ void G1ParCopyHelper::mark_object(oop obj) {
#endif // ASSERT #endif // ASSERT
// We know that the object is not moving so it's safe to read its size. // 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) { 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 // worker so we cannot trust that its to-space image is
// well-formed. So we have to read its size from its from-space // well-formed. So we have to read its size from its from-space
// image which we know should not be changing. // 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) { 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, assert(barrier != G1BarrierRS || obj != NULL,
"Precondition: G1BarrierRS implies obj is non-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 // here the null check is implicit in the cset_fast_test() test
if (_g1->in_cset_fast_test(obj)) { if (_g1->in_cset_fast_test(obj)) {
oop forwardee; 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. // When scanning the RS, we only care about objs in CS.
if (barrier == G1BarrierRS) { 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 { } else {
// The object is not in collection set. If we're a root scanning // 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) { 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) { if (do_gen_barrier && obj != NULL) {
@ -5704,16 +5707,6 @@ void G1CollectedHeap::free_collection_set(HeapRegion* cs_head) {
// And the region is empty. // And the region is empty.
assert(!used_mr.is_empty(), "Should not have empty regions in a CS."); 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 */); free_region(cur, &pre_used, &local_free_list, false /* par */);
} else { } else {
cur->uninstall_surv_rate_group(); cur->uninstall_surv_rate_group();

View file

@ -1909,7 +1909,7 @@ protected:
G1ParScanPartialArrayClosure* _partial_scan_cl; G1ParScanPartialArrayClosure* _partial_scan_cl;
int _hash_seed; int _hash_seed;
int _queue_num; uint _queue_num;
size_t _term_attempts; size_t _term_attempts;
@ -1953,7 +1953,7 @@ protected:
} }
public: public:
G1ParScanThreadState(G1CollectedHeap* g1h, int queue_num); G1ParScanThreadState(G1CollectedHeap* g1h, uint queue_num);
~G1ParScanThreadState() { ~G1ParScanThreadState() {
FREE_C_HEAP_ARRAY(size_t, _surviving_young_words_base); FREE_C_HEAP_ARRAY(size_t, _surviving_young_words_base);
@ -2045,7 +2045,7 @@ public:
} }
int* hash_seed() { return &_hash_seed; } 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; } size_t term_attempts() const { return _term_attempts; }
void note_term_attempt() { _term_attempts++; } void note_term_attempt() { _term_attempts++; }

View file

@ -70,16 +70,20 @@ private:
OopsInHeapRegionClosure *_update_rset_cl; OopsInHeapRegionClosure *_update_rset_cl;
bool _during_initial_mark; bool _during_initial_mark;
bool _during_conc_mark; bool _during_conc_mark;
uint _worker_id;
public: public:
RemoveSelfForwardPtrObjClosure(G1CollectedHeap* g1, ConcurrentMark* cm, RemoveSelfForwardPtrObjClosure(G1CollectedHeap* g1, ConcurrentMark* cm,
HeapRegion* hr, HeapRegion* hr,
OopsInHeapRegionClosure* update_rset_cl, OopsInHeapRegionClosure* update_rset_cl,
bool during_initial_mark, bool during_initial_mark,
bool during_conc_mark) : bool during_conc_mark,
uint worker_id) :
_g1(g1), _cm(cm), _hr(hr), _marked_bytes(0), _g1(g1), _cm(cm), _hr(hr), _marked_bytes(0),
_update_rset_cl(update_rset_cl), _update_rset_cl(update_rset_cl),
_during_initial_mark(during_initial_mark), _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; } size_t marked_bytes() { return _marked_bytes; }
@ -123,7 +127,7 @@ public:
// explicitly and all objects in the CSet are considered // explicitly and all objects in the CSet are considered
// (implicitly) live. So, we won't mark them explicitly and // (implicitly) live. So, we won't mark them explicitly and
// we'll leave them over NTAMS. // we'll leave them over NTAMS.
_cm->markNext(obj); _cm->grayRoot(obj, obj_size, _worker_id);
} }
_marked_bytes += (obj_size * HeapWordSize); _marked_bytes += (obj_size * HeapWordSize);
obj->set_mark(markOopDesc::prototype()); obj->set_mark(markOopDesc::prototype());
@ -155,12 +159,14 @@ class RemoveSelfForwardPtrHRClosure: public HeapRegionClosure {
G1CollectedHeap* _g1h; G1CollectedHeap* _g1h;
ConcurrentMark* _cm; ConcurrentMark* _cm;
OopsInHeapRegionClosure *_update_rset_cl; OopsInHeapRegionClosure *_update_rset_cl;
uint _worker_id;
public: public:
RemoveSelfForwardPtrHRClosure(G1CollectedHeap* g1h, RemoveSelfForwardPtrHRClosure(G1CollectedHeap* g1h,
OopsInHeapRegionClosure* update_rset_cl) : OopsInHeapRegionClosure* update_rset_cl,
uint worker_id) :
_g1h(g1h), _update_rset_cl(update_rset_cl), _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 doHeapRegion(HeapRegion *hr) {
bool during_initial_mark = _g1h->g1_policy()->during_initial_mark_pause(); bool during_initial_mark = _g1h->g1_policy()->during_initial_mark_pause();
@ -173,7 +179,8 @@ public:
if (hr->evacuation_failed()) { if (hr->evacuation_failed()) {
RemoveSelfForwardPtrObjClosure rspc(_g1h, _cm, hr, _update_rset_cl, RemoveSelfForwardPtrObjClosure rspc(_g1h, _cm, hr, _update_rset_cl,
during_initial_mark, during_initial_mark,
during_conc_mark); during_conc_mark,
_worker_id);
MemRegion mr(hr->bottom(), hr->end()); MemRegion mr(hr->bottom(), hr->end());
// We'll recreate the prev marking info so we'll first clear // We'll recreate the prev marking info so we'll first clear
@ -226,7 +233,7 @@ public:
update_rset_cl = &immediate_update; 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); HeapRegion* hr = _g1h->start_cset_region_for_worker(worker_id);
_g1h->collection_set_iterate_from(hr, &rsfp_cl); _g1h->collection_set_iterate_from(hr, &rsfp_cl);

View file

@ -51,6 +51,7 @@ protected:
G1RemSet* _g1_rem; G1RemSet* _g1_rem;
ConcurrentMark* _cm; ConcurrentMark* _cm;
G1ParScanThreadState* _par_scan_state; G1ParScanThreadState* _par_scan_state;
uint _worker_id;
bool _during_initial_mark; bool _during_initial_mark;
bool _mark_in_progress; bool _mark_in_progress;
public: public:

View file

@ -374,7 +374,9 @@ class HeapRegion: public G1OffsetTableContigSpace {
ParVerifyClaimValue = 4, ParVerifyClaimValue = 4,
RebuildRSClaimValue = 5, RebuildRSClaimValue = 5,
CompleteMarkCSetClaimValue = 6, CompleteMarkCSetClaimValue = 6,
ParEvacFailureClaimValue = 7 ParEvacFailureClaimValue = 7,
AggregateCountClaimValue = 8,
VerifyCountClaimValue = 9
}; };
inline HeapWord* par_allocate_no_bot_updates(size_t word_size) { inline HeapWord* par_allocate_no_bot_updates(size_t word_size) {