提交 b73e0e37 编写于 作者: T tonyp

7127706: G1: re-enable survivors during the initial-mark pause

Summary: Re-enable survivors during the initial-mark pause. Afterwards, the concurrent marking threads have to scan them and mark everything reachable from them. The next GC will have to wait for the survivors to be scanned.
Reviewed-by: brutisso, johnc
上级 f13c0dfe
...@@ -460,6 +460,84 @@ bool ConcurrentMark::not_yet_marked(oop obj) const { ...@@ -460,6 +460,84 @@ bool ConcurrentMark::not_yet_marked(oop obj) const {
&& !nextMarkBitMap()->isMarked((HeapWord*)obj))); && !nextMarkBitMap()->isMarked((HeapWord*)obj)));
} }
CMRootRegions::CMRootRegions() :
_young_list(NULL), _cm(NULL), _scan_in_progress(false),
_should_abort(false), _next_survivor(NULL) { }
void CMRootRegions::init(G1CollectedHeap* g1h, ConcurrentMark* cm) {
_young_list = g1h->young_list();
_cm = cm;
}
void CMRootRegions::prepare_for_scan() {
assert(!scan_in_progress(), "pre-condition");
// Currently, only survivors can be root regions.
assert(_next_survivor == NULL, "pre-condition");
_next_survivor = _young_list->first_survivor_region();
_scan_in_progress = (_next_survivor != NULL);
_should_abort = false;
}
HeapRegion* CMRootRegions::claim_next() {
if (_should_abort) {
// If someone has set the should_abort flag, we return NULL to
// force the caller to bail out of their loop.
return NULL;
}
// Currently, only survivors can be root regions.
HeapRegion* res = _next_survivor;
if (res != NULL) {
MutexLockerEx x(RootRegionScan_lock, Mutex::_no_safepoint_check_flag);
// Read it again in case it changed while we were waiting for the lock.
res = _next_survivor;
if (res != NULL) {
if (res == _young_list->last_survivor_region()) {
// We just claimed the last survivor so store NULL to indicate
// that we're done.
_next_survivor = NULL;
} else {
_next_survivor = res->get_next_young_region();
}
} else {
// Someone else claimed the last survivor while we were trying
// to take the lock so nothing else to do.
}
}
assert(res == NULL || res->is_survivor(), "post-condition");
return res;
}
void CMRootRegions::scan_finished() {
assert(scan_in_progress(), "pre-condition");
// Currently, only survivors can be root regions.
if (!_should_abort) {
assert(_next_survivor == NULL, "we should have claimed all survivors");
}
_next_survivor = NULL;
{
MutexLockerEx x(RootRegionScan_lock, Mutex::_no_safepoint_check_flag);
_scan_in_progress = false;
RootRegionScan_lock->notify_all();
}
}
bool CMRootRegions::wait_until_scan_finished() {
if (!scan_in_progress()) return false;
{
MutexLockerEx x(RootRegionScan_lock, Mutex::_no_safepoint_check_flag);
while (scan_in_progress()) {
RootRegionScan_lock->wait(Mutex::_no_safepoint_check_flag);
}
}
return true;
}
#ifdef _MSC_VER // the use of 'this' below gets a warning, make it go away #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 #pragma warning( disable:4355 ) // 'this' : used in base member initializer list
#endif // _MSC_VER #endif // _MSC_VER
...@@ -548,6 +626,8 @@ ConcurrentMark::ConcurrentMark(ReservedSpace rs, ...@@ -548,6 +626,8 @@ ConcurrentMark::ConcurrentMark(ReservedSpace rs,
SATBMarkQueueSet& satb_qs = JavaThread::satb_mark_queue_set(); SATBMarkQueueSet& satb_qs = JavaThread::satb_mark_queue_set();
satb_qs.set_buffer_size(G1SATBBufferSize); satb_qs.set_buffer_size(G1SATBBufferSize);
_root_regions.init(_g1h, this);
_tasks = NEW_C_HEAP_ARRAY(CMTask*, _max_task_num); _tasks = NEW_C_HEAP_ARRAY(CMTask*, _max_task_num);
_accum_task_vtime = NEW_C_HEAP_ARRAY(double, _max_task_num); _accum_task_vtime = NEW_C_HEAP_ARRAY(double, _max_task_num);
...@@ -864,6 +944,8 @@ void ConcurrentMark::checkpointRootsInitialPost() { ...@@ -864,6 +944,8 @@ void ConcurrentMark::checkpointRootsInitialPost() {
satb_mq_set.set_active_all_threads(true, /* new active value */ satb_mq_set.set_active_all_threads(true, /* new active value */
false /* expected_active */); false /* expected_active */);
_root_regions.prepare_for_scan();
// update_g1_committed() will be called at the end of an evac pause // update_g1_committed() will be called at the end of an evac pause
// when marking is on. So, it's also called at the end of the // when marking is on. So, it's also called at the end of the
// initial-mark pause to update the heap end, if the heap expands // initial-mark pause to update the heap end, if the heap expands
...@@ -1157,6 +1239,69 @@ uint ConcurrentMark::calc_parallel_marking_threads() { ...@@ -1157,6 +1239,69 @@ uint ConcurrentMark::calc_parallel_marking_threads() {
return 0; return 0;
} }
void ConcurrentMark::scanRootRegion(HeapRegion* hr, uint worker_id) {
// Currently, only survivors can be root regions.
assert(hr->next_top_at_mark_start() == hr->bottom(), "invariant");
G1RootRegionScanClosure cl(_g1h, this, worker_id);
const uintx interval = PrefetchScanIntervalInBytes;
HeapWord* curr = hr->bottom();
const HeapWord* end = hr->top();
while (curr < end) {
Prefetch::read(curr, interval);
oop obj = oop(curr);
int size = obj->oop_iterate(&cl);
assert(size == obj->size(), "sanity");
curr += size;
}
}
class CMRootRegionScanTask : public AbstractGangTask {
private:
ConcurrentMark* _cm;
public:
CMRootRegionScanTask(ConcurrentMark* cm) :
AbstractGangTask("Root Region Scan"), _cm(cm) { }
void work(uint worker_id) {
assert(Thread::current()->is_ConcurrentGC_thread(),
"this should only be done by a conc GC thread");
CMRootRegions* root_regions = _cm->root_regions();
HeapRegion* hr = root_regions->claim_next();
while (hr != NULL) {
_cm->scanRootRegion(hr, worker_id);
hr = root_regions->claim_next();
}
}
};
void ConcurrentMark::scanRootRegions() {
// scan_in_progress() will have been set to true only if there was
// at least one root region to scan. So, if it's false, we
// should not attempt to do any further work.
if (root_regions()->scan_in_progress()) {
_parallel_marking_threads = calc_parallel_marking_threads();
assert(parallel_marking_threads() <= max_parallel_marking_threads(),
"Maximum number of marking threads exceeded");
uint active_workers = MAX2(1U, parallel_marking_threads());
CMRootRegionScanTask task(this);
if (parallel_marking_threads() > 0) {
_parallel_workers->set_active_workers((int) active_workers);
_parallel_workers->run_task(&task);
} else {
task.work(0);
}
// It's possible that has_aborted() is true here without actually
// aborting the survivor scan earlier. This is OK as it's
// mainly used for sanity checking.
root_regions()->scan_finished();
}
}
void ConcurrentMark::markFromRoots() { void ConcurrentMark::markFromRoots() {
// we might be tempted to assert that: // we might be tempted to assert that:
// assert(asynch == !SafepointSynchronize::is_at_safepoint(), // assert(asynch == !SafepointSynchronize::is_at_safepoint(),
......
...@@ -349,10 +349,62 @@ typedef enum { ...@@ -349,10 +349,62 @@ typedef enum {
high_verbose // per object verbose high_verbose // per object verbose
} CMVerboseLevel; } CMVerboseLevel;
class YoungList;
// Root Regions are regions that are not empty at the beginning of a
// marking cycle and which we might collect during an evacuation pause
// while the cycle is active. Given that, during evacuation pauses, we
// do not copy objects that are explicitly marked, what we have to do
// for the root regions is to scan them and mark all objects reachable
// from them. According to the SATB assumptions, we only need to visit
// each object once during marking. So, as long as we finish this scan
// before the next evacuation pause, we can copy the objects from the
// root regions without having to mark them or do anything else to them.
//
// Currently, we only support root region scanning once (at the start
// of the marking cycle) and the root regions are all the survivor
// regions populated during the initial-mark pause.
class CMRootRegions VALUE_OBJ_CLASS_SPEC {
private:
YoungList* _young_list;
ConcurrentMark* _cm;
volatile bool _scan_in_progress;
volatile bool _should_abort;
HeapRegion* volatile _next_survivor;
public:
CMRootRegions();
// We actually do most of the initialization in this method.
void init(G1CollectedHeap* g1h, ConcurrentMark* cm);
// Reset the claiming / scanning of the root regions.
void prepare_for_scan();
// Forces get_next() to return NULL so that the iteration aborts early.
void abort() { _should_abort = true; }
// Return true if the CM thread are actively scanning root regions,
// false otherwise.
bool scan_in_progress() { return _scan_in_progress; }
// Claim the next root region to scan atomically, or return NULL if
// all have been claimed.
HeapRegion* claim_next();
// Flag that we're done with root region scanning and notify anyone
// who's waiting on it. If aborted is false, assume that all regions
// have been claimed.
void scan_finished();
// If CM threads are still scanning root regions, wait until they
// are done. Return true if we had to wait, false otherwise.
bool wait_until_scan_finished();
};
class ConcurrentMarkThread; class ConcurrentMarkThread;
class ConcurrentMark: public CHeapObj { class ConcurrentMark : public CHeapObj {
friend class ConcurrentMarkThread; friend class ConcurrentMarkThread;
friend class CMTask; friend class CMTask;
friend class CMBitMapClosure; friend class CMBitMapClosure;
...@@ -400,6 +452,9 @@ protected: ...@@ -400,6 +452,9 @@ protected:
HeapWord* _heap_start; HeapWord* _heap_start;
HeapWord* _heap_end; HeapWord* _heap_end;
// Root region tracking and claiming.
CMRootRegions _root_regions;
// For gray objects // For gray objects
CMMarkStack _markStack; // Grey objects behind global finger. CMMarkStack _markStack; // Grey objects behind global finger.
CMRegionStack _regionStack; // Grey regions behind global finger. CMRegionStack _regionStack; // Grey regions behind global finger.
...@@ -553,9 +608,9 @@ protected: ...@@ -553,9 +608,9 @@ protected:
bool has_overflown() { return _has_overflown; } bool has_overflown() { return _has_overflown; }
void set_has_overflown() { _has_overflown = true; } void set_has_overflown() { _has_overflown = true; }
void clear_has_overflown() { _has_overflown = false; } void clear_has_overflown() { _has_overflown = false; }
bool restart_for_overflow() { return _restart_for_overflow; }
bool has_aborted() { return _has_aborted; } bool has_aborted() { return _has_aborted; }
bool restart_for_overflow() { return _restart_for_overflow; }
// Methods to enter the two overflow sync barriers // Methods to enter the two overflow sync barriers
void enter_first_sync_barrier(int task_num); void enter_first_sync_barrier(int task_num);
...@@ -691,6 +746,8 @@ public: ...@@ -691,6 +746,8 @@ public:
// Returns true if there are any aborted memory regions. // Returns true if there are any aborted memory regions.
bool has_aborted_regions(); bool has_aborted_regions();
CMRootRegions* root_regions() { return &_root_regions; }
bool concurrent_marking_in_progress() { bool concurrent_marking_in_progress() {
return _concurrent_marking_in_progress; return _concurrent_marking_in_progress;
} }
...@@ -741,8 +798,17 @@ public: ...@@ -741,8 +798,17 @@ public:
// G1CollectedHeap // G1CollectedHeap
// 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. word_size is the size of the object in
inline void grayRoot(oop obj, size_t word_size, uint worker_id); // 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
// 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);
// 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
...@@ -793,6 +859,13 @@ public: ...@@ -793,6 +859,13 @@ public:
void checkpointRootsInitialPre(); void checkpointRootsInitialPre();
void checkpointRootsInitialPost(); void checkpointRootsInitialPost();
// Scan all the root regions and mark everything reachable from
// them.
void scanRootRegions();
// Scan a single root region and mark everything reachable from it.
void scanRootRegion(HeapRegion* hr, uint worker_id);
// Do concurrent phase of marking, to a tentative transitive closure. // Do concurrent phase of marking, to a tentative transitive closure.
void markFromRoots(); void markFromRoots();
...@@ -972,6 +1045,10 @@ public: ...@@ -972,6 +1045,10 @@ public:
size_t* marked_bytes_array, size_t* marked_bytes_array,
BitMap* task_card_bm); 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, HeapRegion* hr, uint worker_id);
// Counts the given memory region in the task/worker counting // Counts the given memory region in the task/worker counting
// data structures for the given worker id. // data structures for the given worker id.
inline void count_region(MemRegion mr, uint worker_id); inline void count_region(MemRegion mr, uint worker_id);
...@@ -992,6 +1069,12 @@ public: ...@@ -992,6 +1069,12 @@ public:
size_t* marked_bytes_array, size_t* marked_bytes_array,
BitMap* task_card_bm); 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 and, if successful, counts // Attempts to mark the given object and, if successful, counts
// the object in the task/worker counting structures for the // the object in the task/worker counting structures for the
// given worker id. // given worker id.
......
...@@ -81,14 +81,22 @@ inline void ConcurrentMark::count_region(MemRegion mr, HeapRegion* hr, ...@@ -81,14 +81,22 @@ inline void ConcurrentMark::count_region(MemRegion mr, HeapRegion* hr,
} }
} }
// Counts the given memory region in the task/worker counting
// data structures for the given worker id.
inline void ConcurrentMark::count_region(MemRegion mr,
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);
count_region(mr, hr, marked_bytes_array, task_card_bm);
}
// Counts the given memory region, which may be a single object, in the // Counts the given memory region, which may be a single object, in the
// task/worker counting data structures for the given worker id. // task/worker counting data structures for the given worker id.
inline void ConcurrentMark::count_region(MemRegion mr, uint 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(); HeapWord* addr = mr.start();
HeapRegion* hr = _g1h->heap_region_containing_raw(addr); HeapRegion* hr = _g1h->heap_region_containing_raw(addr);
count_region(mr, hr, marked_bytes_array, task_card_bm); count_region(mr, hr, worker_id);
} }
// Counts the given object in the given task/worker counting data structures. // Counts the given object in the given task/worker counting data structures.
...@@ -102,7 +110,9 @@ inline void ConcurrentMark::count_object(oop obj, ...@@ -102,7 +110,9 @@ inline void ConcurrentMark::count_object(oop obj,
// Counts the given object in the task/worker counting data // Counts the given object in the task/worker counting data
// structures for the given worker id. // structures for the given worker id.
inline void ConcurrentMark::count_object(oop obj, HeapRegion* hr, uint 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); size_t* marked_bytes_array = count_marked_bytes_array_for(worker_id);
BitMap* task_card_bm = count_card_bitmap_for(worker_id); BitMap* task_card_bm = count_card_bitmap_for(worker_id);
HeapWord* addr = (HeapWord*) obj; HeapWord* addr = (HeapWord*) obj;
...@@ -124,6 +134,22 @@ inline bool ConcurrentMark::par_mark_and_count(oop obj, ...@@ -124,6 +134,22 @@ inline bool ConcurrentMark::par_mark_and_count(oop obj,
return false; 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,
size_t word_size,
HeapRegion* hr,
uint worker_id) {
HeapWord* addr = (HeapWord*)obj;
if (_nextMarkBitMap->parMark(addr)) {
MemRegion mr(addr, word_size);
count_region(mr, hr, worker_id);
return true;
}
return false;
}
// Attempts to mark the given object and, if successful, counts // Attempts to mark the given object and, if successful, counts
// the object in the task/worker counting structures for the // the object in the task/worker counting structures for the
// given worker id. // given worker id.
...@@ -342,20 +368,20 @@ inline void ConcurrentMark::markPrev(oop p) { ...@@ -342,20 +368,20 @@ inline void ConcurrentMark::markPrev(oop p) {
((CMBitMap*)_prevMarkBitMap)->mark((HeapWord*) p); ((CMBitMap*)_prevMarkBitMap)->mark((HeapWord*) p);
} }
inline void ConcurrentMark::grayRoot(oop obj, size_t word_size, uint worker_id) { inline void ConcurrentMark::grayRoot(oop obj, size_t word_size,
uint worker_id, HeapRegion* hr) {
assert(obj != NULL, "pre-condition");
HeapWord* addr = (HeapWord*) obj; HeapWord* addr = (HeapWord*) obj;
if (hr == NULL) {
// Currently we don't do anything with word_size but we will use it hr = _g1h->heap_region_containing_raw(addr);
// in the very near future in the liveness calculation piggy-backing } else {
// changes. assert(hr->is_in(addr), "pre-condition");
}
#ifdef ASSERT
HeapRegion* hr = _g1h->heap_region_containing(addr);
assert(hr != NULL, "sanity"); assert(hr != NULL, "sanity");
assert(!hr->is_survivor(), "should not allocate survivors during IM"); // Given that we're looking for a region that contains an object
assert(addr < hr->next_top_at_mark_start(), // header it's impossible to get back a HC region.
err_msg("addr: "PTR_FORMAT" hr: "HR_FORMAT" NTAMS: "PTR_FORMAT, assert(!hr->continuesHumongous(), "sanity");
addr, HR_FORMAT_PARAMS(hr), hr->next_top_at_mark_start()));
// We cannot assert that word_size == obj->size() given that obj // We cannot assert that word_size == obj->size() given that obj
// might not be in a consistent state (another thread might be in // might not be in a consistent state (another thread might be in
// the process of copying it). So the best thing we can do is to // the process of copying it). So the best thing we can do is to
...@@ -365,10 +391,11 @@ inline void ConcurrentMark::grayRoot(oop obj, size_t word_size, uint worker_id) ...@@ -365,10 +391,11 @@ inline void ConcurrentMark::grayRoot(oop obj, size_t word_size, uint worker_id)
err_msg("size: "SIZE_FORMAT" capacity: "SIZE_FORMAT" "HR_FORMAT, err_msg("size: "SIZE_FORMAT" capacity: "SIZE_FORMAT" "HR_FORMAT,
word_size * HeapWordSize, hr->capacity(), word_size * HeapWordSize, hr->capacity(),
HR_FORMAT_PARAMS(hr))); HR_FORMAT_PARAMS(hr)));
#endif // ASSERT
if (!_nextMarkBitMap->isMarked(addr)) { if (addr < hr->next_top_at_mark_start()) {
par_mark_and_count(obj, word_size, worker_id); if (!_nextMarkBitMap->isMarked(addr)) {
par_mark_and_count(obj, word_size, hr, worker_id);
}
} }
} }
......
...@@ -92,9 +92,36 @@ void ConcurrentMarkThread::run() { ...@@ -92,9 +92,36 @@ void ConcurrentMarkThread::run() {
ResourceMark rm; ResourceMark rm;
HandleMark hm; HandleMark hm;
double cycle_start = os::elapsedVTime(); double cycle_start = os::elapsedVTime();
double mark_start_sec = os::elapsedTime();
char verbose_str[128]; char verbose_str[128];
// We have to ensure that we finish scanning the root regions
// before the next GC takes place. To ensure this we have to
// make sure that we do not join the STS until the root regions
// have been scanned. If we did then it's possible that a
// subsequent GC could block us from joining the STS and proceed
// without the root regions have been scanned which would be a
// correctness issue.
double scan_start = os::elapsedTime();
if (!cm()->has_aborted()) {
if (PrintGC) {
gclog_or_tty->date_stamp(PrintGCDateStamps);
gclog_or_tty->stamp(PrintGCTimeStamps);
gclog_or_tty->print_cr("[GC concurrent-root-region-scan-start]");
}
_cm->scanRootRegions();
double scan_end = os::elapsedTime();
if (PrintGC) {
gclog_or_tty->date_stamp(PrintGCDateStamps);
gclog_or_tty->stamp(PrintGCTimeStamps);
gclog_or_tty->print_cr("[GC concurrent-root-region-scan-end, %1.7lf]",
scan_end - scan_start);
}
}
double mark_start_sec = os::elapsedTime();
if (PrintGC) { if (PrintGC) {
gclog_or_tty->date_stamp(PrintGCDateStamps); gclog_or_tty->date_stamp(PrintGCDateStamps);
gclog_or_tty->stamp(PrintGCTimeStamps); gclog_or_tty->stamp(PrintGCTimeStamps);
......
...@@ -174,13 +174,10 @@ public: ...@@ -174,13 +174,10 @@ public:
} }
}; };
YoungList::YoungList(G1CollectedHeap* g1h) YoungList::YoungList(G1CollectedHeap* g1h) :
: _g1h(g1h), _head(NULL), _g1h(g1h), _head(NULL), _length(0), _last_sampled_rs_lengths(0),
_length(0), _survivor_head(NULL), _survivor_tail(NULL), _survivor_length(0) {
_last_sampled_rs_lengths(0), guarantee(check_list_empty(false), "just making sure...");
_survivor_head(NULL), _survivor_tail(NULL), _survivor_length(0)
{
guarantee( check_list_empty(false), "just making sure..." );
} }
void YoungList::push_region(HeapRegion *hr) { void YoungList::push_region(HeapRegion *hr) {
...@@ -1270,7 +1267,18 @@ bool G1CollectedHeap::do_collection(bool explicit_gc, ...@@ -1270,7 +1267,18 @@ bool G1CollectedHeap::do_collection(bool explicit_gc,
double start = os::elapsedTime(); double start = os::elapsedTime();
g1_policy()->record_full_collection_start(); g1_policy()->record_full_collection_start();
// Note: When we have a more flexible GC logging framework that
// allows us to add optional attributes to a GC log record we
// could consider timing and reporting how long we wait in the
// following two methods.
wait_while_free_regions_coming(); wait_while_free_regions_coming();
// If we start the compaction before the CM threads finish
// scanning the root regions we might trip them over as we'll
// be moving objects / updating references. So let's wait until
// they are done. By telling them to abort, they should complete
// early.
_cm->root_regions()->abort();
_cm->root_regions()->wait_until_scan_finished();
append_secondary_free_list_if_not_empty_with_lock(); append_secondary_free_list_if_not_empty_with_lock();
gc_prologue(true); gc_prologue(true);
...@@ -1299,7 +1307,8 @@ bool G1CollectedHeap::do_collection(bool explicit_gc, ...@@ -1299,7 +1307,8 @@ bool G1CollectedHeap::do_collection(bool explicit_gc,
ref_processor_cm()->verify_no_references_recorded(); ref_processor_cm()->verify_no_references_recorded();
// Abandon current iterations of concurrent marking and concurrent // Abandon current iterations of concurrent marking and concurrent
// refinement, if any are in progress. // refinement, if any are in progress. We have to do this before
// wait_until_scan_finished() below.
concurrent_mark()->abort(); concurrent_mark()->abort();
// Make sure we'll choose a new allocation region afterwards. // Make sure we'll choose a new allocation region afterwards.
...@@ -3675,6 +3684,18 @@ G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) { ...@@ -3675,6 +3684,18 @@ G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
g1_policy()->record_collection_pause_start(start_time_sec, g1_policy()->record_collection_pause_start(start_time_sec,
start_used_bytes); start_used_bytes);
double scan_wait_start = os::elapsedTime();
// We have to wait until the CM threads finish scanning the
// root regions as it's the only way to ensure that all the
// objects on them have been correctly scanned before we start
// moving them during the GC.
bool waited = _cm->root_regions()->wait_until_scan_finished();
if (waited) {
double scan_wait_end = os::elapsedTime();
double wait_time_ms = (scan_wait_end - scan_wait_start) * 1000.0;
g1_policy()->record_root_region_scan_wait_time(wait_time_ms);
}
#if YOUNG_LIST_VERBOSE #if YOUNG_LIST_VERBOSE
gclog_or_tty->print_cr("\nAfter recording pause start.\nYoung_list:"); gclog_or_tty->print_cr("\nAfter recording pause start.\nYoung_list:");
_young_list->print(); _young_list->print();
...@@ -3784,6 +3805,9 @@ G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) { ...@@ -3784,6 +3805,9 @@ G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
} }
if (g1_policy()->during_initial_mark_pause()) { if (g1_policy()->during_initial_mark_pause()) {
// We have to do this before we notify the CM threads that
// they can start working to make sure that all the
// appropriate initialization is done on the CM object.
concurrent_mark()->checkpointRootsInitialPost(); concurrent_mark()->checkpointRootsInitialPost();
set_marking_started(); set_marking_started();
// Note that we don't actually trigger the CM thread at // Note that we don't actually trigger the CM thread at
...@@ -5773,8 +5797,9 @@ void G1CollectedHeap::set_free_regions_coming() { ...@@ -5773,8 +5797,9 @@ void G1CollectedHeap::set_free_regions_coming() {
} }
void G1CollectedHeap::reset_free_regions_coming() { void G1CollectedHeap::reset_free_regions_coming() {
assert(free_regions_coming(), "pre-condition");
{ {
assert(free_regions_coming(), "pre-condition");
MutexLockerEx x(SecondaryFreeList_lock, Mutex::_no_safepoint_check_flag); MutexLockerEx x(SecondaryFreeList_lock, Mutex::_no_safepoint_check_flag);
_free_regions_coming = false; _free_regions_coming = false;
SecondaryFreeList_lock->notify_all(); SecondaryFreeList_lock->notify_all();
......
...@@ -141,6 +141,7 @@ G1CollectorPolicy::G1CollectorPolicy() : ...@@ -141,6 +141,7 @@ G1CollectorPolicy::G1CollectorPolicy() :
_cur_clear_ct_time_ms(0.0), _cur_clear_ct_time_ms(0.0),
_mark_closure_time_ms(0.0), _mark_closure_time_ms(0.0),
_root_region_scan_wait_time_ms(0.0),
_cur_ref_proc_time_ms(0.0), _cur_ref_proc_time_ms(0.0),
_cur_ref_enq_time_ms(0.0), _cur_ref_enq_time_ms(0.0),
...@@ -903,19 +904,10 @@ void G1CollectorPolicy::record_collection_pause_start(double start_time_sec, ...@@ -903,19 +904,10 @@ void G1CollectorPolicy::record_collection_pause_start(double start_time_sec,
gclog_or_tty->print(" (%s)", gcs_are_young() ? "young" : "mixed"); gclog_or_tty->print(" (%s)", gcs_are_young() ? "young" : "mixed");
} }
if (!during_initial_mark_pause()) { // We only need to do this here as the policy will only be applied
// We only need to do this here as the policy will only be applied // to the GC we're about to start. so, no point is calculating this
// to the GC we're about to start. so, no point is calculating this // every time we calculate / recalculate the target young length.
// every time we calculate / recalculate the target young length. update_survivors_policy();
update_survivors_policy();
} else {
// The marking phase has a "we only copy implicitly live
// objects during marking" invariant. The easiest way to ensure it
// holds is not to allocate any survivor regions and tenure all
// objects. In the future we might change this and handle survivor
// regions specially during marking.
tenure_all_objects();
}
assert(_g1->used() == _g1->recalculate_used(), assert(_g1->used() == _g1->recalculate_used(),
err_msg("sanity, used: "SIZE_FORMAT" recalculate_used: "SIZE_FORMAT, err_msg("sanity, used: "SIZE_FORMAT" recalculate_used: "SIZE_FORMAT,
...@@ -967,6 +959,9 @@ void G1CollectorPolicy::record_collection_pause_start(double start_time_sec, ...@@ -967,6 +959,9 @@ void G1CollectorPolicy::record_collection_pause_start(double start_time_sec,
// This is initialized to zero here and is set during // This is initialized to zero here and is set during
// the evacuation pause if marking is in progress. // the evacuation pause if marking is in progress.
_cur_satb_drain_time_ms = 0.0; _cur_satb_drain_time_ms = 0.0;
// This is initialized to zero here and is set during the evacuation
// pause if we actually waited for the root region scanning to finish.
_root_region_scan_wait_time_ms = 0.0;
_last_gc_was_young = false; _last_gc_was_young = false;
...@@ -1271,6 +1266,10 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) { ...@@ -1271,6 +1266,10 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) {
// is in progress. // is in progress.
other_time_ms -= _cur_satb_drain_time_ms; other_time_ms -= _cur_satb_drain_time_ms;
// Subtract the root region scanning wait time. It's initialized to
// zero at the start of the pause.
other_time_ms -= _root_region_scan_wait_time_ms;
if (parallel) { if (parallel) {
other_time_ms -= _cur_collection_par_time_ms; other_time_ms -= _cur_collection_par_time_ms;
} else { } else {
...@@ -1303,6 +1302,8 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) { ...@@ -1303,6 +1302,8 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) {
// each other. Therefore we unconditionally record the SATB drain // each other. Therefore we unconditionally record the SATB drain
// time - even if it's zero. // time - even if it's zero.
body_summary->record_satb_drain_time_ms(_cur_satb_drain_time_ms); body_summary->record_satb_drain_time_ms(_cur_satb_drain_time_ms);
body_summary->record_root_region_scan_wait_time_ms(
_root_region_scan_wait_time_ms);
body_summary->record_ext_root_scan_time_ms(ext_root_scan_time); body_summary->record_ext_root_scan_time_ms(ext_root_scan_time);
body_summary->record_satb_filtering_time_ms(satb_filtering_time); body_summary->record_satb_filtering_time_ms(satb_filtering_time);
...@@ -1399,6 +1400,9 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) { ...@@ -1399,6 +1400,9 @@ void G1CollectorPolicy::record_collection_pause_end(int no_of_gc_threads) {
(last_pause_included_initial_mark) ? " (initial-mark)" : "", (last_pause_included_initial_mark) ? " (initial-mark)" : "",
elapsed_ms / 1000.0); elapsed_ms / 1000.0);
if (_root_region_scan_wait_time_ms > 0.0) {
print_stats(1, "Root Region Scan Waiting", _root_region_scan_wait_time_ms);
}
if (parallel) { if (parallel) {
print_stats(1, "Parallel Time", _cur_collection_par_time_ms); print_stats(1, "Parallel Time", _cur_collection_par_time_ms);
print_par_stats(2, "GC Worker Start", _par_last_gc_worker_start_times_ms); print_par_stats(2, "GC Worker Start", _par_last_gc_worker_start_times_ms);
...@@ -2002,6 +2006,7 @@ void G1CollectorPolicy::print_summary(PauseSummary* summary) const { ...@@ -2002,6 +2006,7 @@ void G1CollectorPolicy::print_summary(PauseSummary* summary) const {
if (summary->get_total_seq()->num() > 0) { if (summary->get_total_seq()->num() > 0) {
print_summary_sd(0, "Evacuation Pauses", summary->get_total_seq()); print_summary_sd(0, "Evacuation Pauses", summary->get_total_seq());
if (body_summary != NULL) { if (body_summary != NULL) {
print_summary(1, "Root Region Scan Wait", body_summary->get_root_region_scan_wait_seq());
if (parallel) { if (parallel) {
print_summary(1, "Parallel Time", body_summary->get_parallel_seq()); print_summary(1, "Parallel Time", body_summary->get_parallel_seq());
print_summary(2, "Ext Root Scanning", body_summary->get_ext_root_scan_seq()); print_summary(2, "Ext Root Scanning", body_summary->get_ext_root_scan_seq());
...@@ -2043,15 +2048,17 @@ void G1CollectorPolicy::print_summary(PauseSummary* summary) const { ...@@ -2043,15 +2048,17 @@ void G1CollectorPolicy::print_summary(PauseSummary* summary) const {
// parallel // parallel
NumberSeq* other_parts[] = { NumberSeq* other_parts[] = {
body_summary->get_satb_drain_seq(), body_summary->get_satb_drain_seq(),
body_summary->get_root_region_scan_wait_seq(),
body_summary->get_parallel_seq(), body_summary->get_parallel_seq(),
body_summary->get_clear_ct_seq() body_summary->get_clear_ct_seq()
}; };
calc_other_times_ms = NumberSeq(summary->get_total_seq(), calc_other_times_ms = NumberSeq(summary->get_total_seq(),
3, other_parts); 4, other_parts);
} else { } else {
// serial // serial
NumberSeq* other_parts[] = { NumberSeq* other_parts[] = {
body_summary->get_satb_drain_seq(), body_summary->get_satb_drain_seq(),
body_summary->get_root_region_scan_wait_seq(),
body_summary->get_update_rs_seq(), body_summary->get_update_rs_seq(),
body_summary->get_ext_root_scan_seq(), body_summary->get_ext_root_scan_seq(),
body_summary->get_satb_filtering_seq(), body_summary->get_satb_filtering_seq(),
...@@ -2059,7 +2066,7 @@ void G1CollectorPolicy::print_summary(PauseSummary* summary) const { ...@@ -2059,7 +2066,7 @@ void G1CollectorPolicy::print_summary(PauseSummary* summary) const {
body_summary->get_obj_copy_seq() body_summary->get_obj_copy_seq()
}; };
calc_other_times_ms = NumberSeq(summary->get_total_seq(), calc_other_times_ms = NumberSeq(summary->get_total_seq(),
6, other_parts); 7, other_parts);
} }
check_other_times(1, summary->get_other_seq(), &calc_other_times_ms); check_other_times(1, summary->get_other_seq(), &calc_other_times_ms);
} }
......
...@@ -65,6 +65,7 @@ public: ...@@ -65,6 +65,7 @@ public:
class MainBodySummary: public CHeapObj { class MainBodySummary: public CHeapObj {
define_num_seq(satb_drain) // optional define_num_seq(satb_drain) // optional
define_num_seq(root_region_scan_wait)
define_num_seq(parallel) // parallel only define_num_seq(parallel) // parallel only
define_num_seq(ext_root_scan) define_num_seq(ext_root_scan)
define_num_seq(satb_filtering) define_num_seq(satb_filtering)
...@@ -715,6 +716,7 @@ private: ...@@ -715,6 +716,7 @@ private:
double _mark_remark_start_sec; double _mark_remark_start_sec;
double _mark_cleanup_start_sec; double _mark_cleanup_start_sec;
double _mark_closure_time_ms; double _mark_closure_time_ms;
double _root_region_scan_wait_time_ms;
// Update the young list target length either by setting it to the // Update the young list target length either by setting it to the
// desired fixed value or by calculating it using G1's pause // desired fixed value or by calculating it using G1's pause
...@@ -817,6 +819,10 @@ public: ...@@ -817,6 +819,10 @@ public:
_mark_closure_time_ms = mark_closure_time_ms; _mark_closure_time_ms = mark_closure_time_ms;
} }
void record_root_region_scan_wait_time(double time_ms) {
_root_region_scan_wait_time_ms = time_ms;
}
void record_concurrent_mark_remark_start(); void record_concurrent_mark_remark_start();
void record_concurrent_mark_remark_end(); void record_concurrent_mark_remark_end();
...@@ -1147,11 +1153,6 @@ public: ...@@ -1147,11 +1153,6 @@ public:
_survivor_surv_rate_group->stop_adding_regions(); _survivor_surv_rate_group->stop_adding_regions();
} }
void tenure_all_objects() {
_max_survivor_regions = 0;
_tenuring_threshold = 0;
}
void record_survivor_regions(size_t regions, void record_survivor_regions(size_t regions,
HeapRegion* head, HeapRegion* head,
HeapRegion* tail) { HeapRegion* tail) {
......
...@@ -127,7 +127,7 @@ public: ...@@ -127,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->grayRoot(obj, obj_size, _worker_id); _cm->grayRoot(obj, obj_size, _worker_id, _hr);
} }
_marked_bytes += (obj_size * HeapWordSize); _marked_bytes += (obj_size * HeapWordSize);
obj->set_mark(markOopDesc::prototype()); obj->set_mark(markOopDesc::prototype());
......
...@@ -220,6 +220,7 @@ public: ...@@ -220,6 +220,7 @@ public:
// Closure for iterating over object fields during concurrent marking // Closure for iterating over object fields during concurrent marking
class G1CMOopClosure : public OopClosure { class G1CMOopClosure : public OopClosure {
private:
G1CollectedHeap* _g1h; G1CollectedHeap* _g1h;
ConcurrentMark* _cm; ConcurrentMark* _cm;
CMTask* _task; CMTask* _task;
...@@ -230,4 +231,19 @@ public: ...@@ -230,4 +231,19 @@ public:
virtual void do_oop(narrowOop* p) { do_oop_nv(p); } virtual void do_oop(narrowOop* p) { do_oop_nv(p); }
}; };
// Closure to scan the root regions during concurrent marking
class G1RootRegionScanClosure : public OopClosure {
private:
G1CollectedHeap* _g1h;
ConcurrentMark* _cm;
uint _worker_id;
public:
G1RootRegionScanClosure(G1CollectedHeap* g1h, ConcurrentMark* cm,
uint worker_id) :
_g1h(g1h), _cm(cm), _worker_id(worker_id) { }
template <class T> void do_oop_nv(T* p);
virtual void do_oop( oop* p) { do_oop_nv(p); }
virtual void do_oop(narrowOop* p) { do_oop_nv(p); }
};
#endif // SHARE_VM_GC_IMPLEMENTATION_G1_G1OOPCLOSURES_HPP #endif // SHARE_VM_GC_IMPLEMENTATION_G1_G1OOPCLOSURES_HPP
/* /*
* 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
...@@ -39,7 +39,8 @@ ...@@ -39,7 +39,8 @@
// perf-critical inner loop. // perf-critical inner loop.
#define FILTERINTOCSCLOSURE_DOHISTOGRAMCOUNT 0 #define FILTERINTOCSCLOSURE_DOHISTOGRAMCOUNT 0
template <class T> inline void FilterIntoCSClosure::do_oop_nv(T* p) { template <class T>
inline void FilterIntoCSClosure::do_oop_nv(T* p) {
T heap_oop = oopDesc::load_heap_oop(p); T heap_oop = oopDesc::load_heap_oop(p);
if (!oopDesc::is_null(heap_oop) && if (!oopDesc::is_null(heap_oop) &&
_g1->obj_in_cs(oopDesc::decode_heap_oop_not_null(heap_oop))) { _g1->obj_in_cs(oopDesc::decode_heap_oop_not_null(heap_oop))) {
...@@ -53,7 +54,8 @@ template <class T> inline void FilterIntoCSClosure::do_oop_nv(T* p) { ...@@ -53,7 +54,8 @@ template <class T> inline void FilterIntoCSClosure::do_oop_nv(T* p) {
#define FILTEROUTOFREGIONCLOSURE_DOHISTOGRAMCOUNT 0 #define FILTEROUTOFREGIONCLOSURE_DOHISTOGRAMCOUNT 0
template <class T> inline void FilterOutOfRegionClosure::do_oop_nv(T* p) { template <class T>
inline void FilterOutOfRegionClosure::do_oop_nv(T* p) {
T heap_oop = oopDesc::load_heap_oop(p); T heap_oop = oopDesc::load_heap_oop(p);
if (!oopDesc::is_null(heap_oop)) { if (!oopDesc::is_null(heap_oop)) {
HeapWord* obj_hw = (HeapWord*)oopDesc::decode_heap_oop_not_null(heap_oop); HeapWord* obj_hw = (HeapWord*)oopDesc::decode_heap_oop_not_null(heap_oop);
...@@ -67,7 +69,8 @@ template <class T> inline void FilterOutOfRegionClosure::do_oop_nv(T* p) { ...@@ -67,7 +69,8 @@ template <class T> inline void FilterOutOfRegionClosure::do_oop_nv(T* p) {
} }
// This closure is applied to the fields of the objects that have just been copied. // This closure is applied to the fields of the objects that have just been copied.
template <class T> inline void G1ParScanClosure::do_oop_nv(T* p) { template <class T>
inline void G1ParScanClosure::do_oop_nv(T* p) {
T heap_oop = oopDesc::load_heap_oop(p); T heap_oop = oopDesc::load_heap_oop(p);
if (!oopDesc::is_null(heap_oop)) { if (!oopDesc::is_null(heap_oop)) {
...@@ -96,7 +99,8 @@ template <class T> inline void G1ParScanClosure::do_oop_nv(T* p) { ...@@ -96,7 +99,8 @@ template <class T> inline void G1ParScanClosure::do_oop_nv(T* p) {
} }
} }
template <class T> inline void G1ParPushHeapRSClosure::do_oop_nv(T* p) { template <class T>
inline void G1ParPushHeapRSClosure::do_oop_nv(T* p) {
T heap_oop = oopDesc::load_heap_oop(p); T heap_oop = oopDesc::load_heap_oop(p);
if (!oopDesc::is_null(heap_oop)) { if (!oopDesc::is_null(heap_oop)) {
...@@ -111,7 +115,8 @@ template <class T> inline void G1ParPushHeapRSClosure::do_oop_nv(T* p) { ...@@ -111,7 +115,8 @@ template <class T> inline void G1ParPushHeapRSClosure::do_oop_nv(T* p) {
} }
} }
template <class T> inline void G1CMOopClosure::do_oop_nv(T* p) { template <class T>
inline void G1CMOopClosure::do_oop_nv(T* p) {
assert(_g1h->is_in_g1_reserved((HeapWord*) p), "invariant"); assert(_g1h->is_in_g1_reserved((HeapWord*) p), "invariant");
assert(!_g1h->is_on_master_free_list( assert(!_g1h->is_on_master_free_list(
_g1h->heap_region_containing((HeapWord*) p)), "invariant"); _g1h->heap_region_containing((HeapWord*) p)), "invariant");
...@@ -125,4 +130,16 @@ template <class T> inline void G1CMOopClosure::do_oop_nv(T* p) { ...@@ -125,4 +130,16 @@ template <class T> inline void G1CMOopClosure::do_oop_nv(T* p) {
_task->deal_with_reference(obj); _task->deal_with_reference(obj);
} }
template <class T>
inline void G1RootRegionScanClosure::do_oop_nv(T* p) {
T heap_oop = oopDesc::load_heap_oop(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);
if (hr != NULL) {
_cm->grayRoot(obj, obj->size(), _worker_id, hr);
}
}
}
#endif // SHARE_VM_GC_IMPLEMENTATION_G1_G1OOPCLOSURES_INLINE_HPP #endif // SHARE_VM_GC_IMPLEMENTATION_G1_G1OOPCLOSURES_INLINE_HPP
...@@ -72,10 +72,11 @@ inline void HeapRegion::note_end_of_marking() { ...@@ -72,10 +72,11 @@ inline void HeapRegion::note_end_of_marking() {
} }
inline void HeapRegion::note_start_of_copying(bool during_initial_mark) { inline void HeapRegion::note_start_of_copying(bool during_initial_mark) {
if (during_initial_mark) { if (is_survivor()) {
if (is_survivor()) { // This is how we always allocate survivors.
assert(false, "should not allocate survivors during IM"); assert(_next_top_at_mark_start == bottom(), "invariant");
} else { } else {
if (during_initial_mark) {
// During initial-mark we'll explicitly mark any objects on old // During initial-mark we'll explicitly mark any objects on old
// regions that are pointed to by roots. Given that explicit // regions that are pointed to by roots. Given that explicit
// marks only make sense under NTAMS it'd be nice if we could // marks only make sense under NTAMS it'd be nice if we could
...@@ -84,11 +85,6 @@ inline void HeapRegion::note_start_of_copying(bool during_initial_mark) { ...@@ -84,11 +85,6 @@ inline void HeapRegion::note_start_of_copying(bool during_initial_mark) {
// NTAMS to the end of the region so all marks will be below // NTAMS to the end of the region so all marks will be below
// NTAMS. We'll set it to the actual top when we retire this region. // NTAMS. We'll set it to the actual top when we retire this region.
_next_top_at_mark_start = end(); _next_top_at_mark_start = end();
}
} else {
if (is_survivor()) {
// This is how we always allocate survivors.
assert(_next_top_at_mark_start == bottom(), "invariant");
} else { } else {
// We could have re-used this old region as to-space over a // We could have re-used this old region as to-space over a
// couple of GCs since the start of the concurrent marking // couple of GCs since the start of the concurrent marking
...@@ -101,19 +97,15 @@ inline void HeapRegion::note_start_of_copying(bool during_initial_mark) { ...@@ -101,19 +97,15 @@ inline void HeapRegion::note_start_of_copying(bool during_initial_mark) {
} }
inline void HeapRegion::note_end_of_copying(bool during_initial_mark) { inline void HeapRegion::note_end_of_copying(bool during_initial_mark) {
if (during_initial_mark) { if (is_survivor()) {
if (is_survivor()) { // This is how we always allocate survivors.
assert(false, "should not allocate survivors during IM"); assert(_next_top_at_mark_start == bottom(), "invariant");
} else { } else {
if (during_initial_mark) {
// See the comment for note_start_of_copying() for the details // See the comment for note_start_of_copying() for the details
// on this. // on this.
assert(_next_top_at_mark_start == end(), "pre-condition"); assert(_next_top_at_mark_start == end(), "pre-condition");
_next_top_at_mark_start = top(); _next_top_at_mark_start = top();
}
} else {
if (is_survivor()) {
// This is how we always allocate survivors.
assert(_next_top_at_mark_start == bottom(), "invariant");
} else { } else {
// See the comment for note_start_of_copying() for the details // See the comment for note_start_of_copying() for the details
// on this. // on this.
......
/* /*
* Copyright (c) 1997, 2011, Oracle and/or its affiliates. All rights reserved. * Copyright (c) 1997, 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
...@@ -126,13 +126,14 @@ Mutex* OopMapCacheAlloc_lock = NULL; ...@@ -126,13 +126,14 @@ Mutex* OopMapCacheAlloc_lock = NULL;
Mutex* FreeList_lock = NULL; Mutex* FreeList_lock = NULL;
Monitor* SecondaryFreeList_lock = NULL; Monitor* SecondaryFreeList_lock = NULL;
Mutex* OldSets_lock = NULL; Mutex* OldSets_lock = NULL;
Monitor* RootRegionScan_lock = NULL;
Mutex* MMUTracker_lock = NULL; Mutex* MMUTracker_lock = NULL;
Mutex* HotCardCache_lock = NULL; Mutex* HotCardCache_lock = NULL;
Monitor* GCTaskManager_lock = NULL; Monitor* GCTaskManager_lock = NULL;
Mutex* Management_lock = NULL; Mutex* Management_lock = NULL;
Monitor* Service_lock = NULL; Monitor* Service_lock = NULL;
#define MAX_NUM_MUTEX 128 #define MAX_NUM_MUTEX 128
static Monitor * _mutex_array[MAX_NUM_MUTEX]; static Monitor * _mutex_array[MAX_NUM_MUTEX];
...@@ -193,6 +194,7 @@ void mutex_init() { ...@@ -193,6 +194,7 @@ void mutex_init() {
def(FreeList_lock , Mutex, leaf , true ); def(FreeList_lock , Mutex, leaf , true );
def(SecondaryFreeList_lock , Monitor, leaf , true ); def(SecondaryFreeList_lock , Monitor, leaf , true );
def(OldSets_lock , Mutex , leaf , true ); def(OldSets_lock , Mutex , leaf , true );
def(RootRegionScan_lock , Monitor, leaf , true );
def(MMUTracker_lock , Mutex , leaf , true ); def(MMUTracker_lock , Mutex , leaf , true );
def(HotCardCache_lock , Mutex , special , true ); def(HotCardCache_lock , Mutex , special , true );
def(EvacFailureStack_lock , Mutex , nonleaf , true ); def(EvacFailureStack_lock , Mutex , nonleaf , true );
......
/* /*
* Copyright (c) 1997, 2011, Oracle and/or its affiliates. All rights reserved. * Copyright (c) 1997, 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
...@@ -115,7 +115,7 @@ extern Mutex* OsrList_lock; // a lock used to serialize acc ...@@ -115,7 +115,7 @@ extern Mutex* OsrList_lock; // a lock used to serialize acc
#ifndef PRODUCT #ifndef PRODUCT
extern Mutex* FullGCALot_lock; // a lock to make FullGCALot MT safe extern Mutex* FullGCALot_lock; // a lock to make FullGCALot MT safe
#endif #endif // PRODUCT
extern Mutex* Debug1_lock; // A bunch of pre-allocated locks that can be used for tracing extern Mutex* Debug1_lock; // A bunch of pre-allocated locks that can be used for tracing
extern Mutex* Debug2_lock; // down synchronization related bugs! extern Mutex* Debug2_lock; // down synchronization related bugs!
extern Mutex* Debug3_lock; extern Mutex* Debug3_lock;
...@@ -129,6 +129,7 @@ extern Mutex* OopMapCacheAlloc_lock; // protects allocation of oop_m ...@@ -129,6 +129,7 @@ extern Mutex* OopMapCacheAlloc_lock; // protects allocation of oop_m
extern Mutex* FreeList_lock; // protects the free region list during safepoints extern Mutex* FreeList_lock; // protects the free region list during safepoints
extern Monitor* SecondaryFreeList_lock; // protects the secondary free region list extern Monitor* SecondaryFreeList_lock; // protects the secondary free region list
extern Mutex* OldSets_lock; // protects the old region sets extern Mutex* OldSets_lock; // protects the old region sets
extern Monitor* RootRegionScan_lock; // used to notify that the CM threads have finished scanning the IM snapshot regions
extern Mutex* MMUTracker_lock; // protects the MMU extern Mutex* MMUTracker_lock; // protects the MMU
// tracker data structures // tracker data structures
extern Mutex* HotCardCache_lock; // protects the hot card cache extern Mutex* HotCardCache_lock; // protects the hot card cache
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册