提交 f13c0dfe 编写于 作者: 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
上级 648c7c49
......@@ -484,6 +484,7 @@ ConcurrentMark::ConcurrentMark(ReservedSpace rs,
_card_bm((rs.size() + CardTableModRefBS::card_size - 1) >>
CardTableModRefBS::card_shift,
false /* in_resource_area*/),
_prevMarkBitMap(&_markBitMap1),
_nextMarkBitMap(&_markBitMap2),
_at_least_one_mark_complete(false),
......@@ -512,7 +513,11 @@ ConcurrentMark::ConcurrentMark(ReservedSpace rs,
_cleanup_times(),
_total_counting_time(0.0),
_total_rs_scrub_time(0.0),
_parallel_workers(NULL) {
_parallel_workers(NULL),
_count_card_bitmaps(NULL),
_count_marked_bytes(NULL) {
CMVerboseLevel verbose_level = (CMVerboseLevel) G1MarkingVerboseLevel;
if (verbose_level < no_verbose) {
verbose_level = no_verbose;
......@@ -546,6 +551,11 @@ ConcurrentMark::ConcurrentMark(ReservedSpace rs,
_tasks = NEW_C_HEAP_ARRAY(CMTask*, _max_task_num);
_accum_task_vtime = NEW_C_HEAP_ARRAY(double, _max_task_num);
_count_card_bitmaps = NEW_C_HEAP_ARRAY(BitMap, _max_task_num);
_count_marked_bytes = NEW_C_HEAP_ARRAY(size_t*, _max_task_num);
BitMap::idx_t card_bm_size = _card_bm.size();
// so that the assertion in MarkingTaskQueue::task_queue doesn't fail
_active_tasks = _max_task_num;
for (int i = 0; i < (int) _max_task_num; ++i) {
......@@ -553,10 +563,26 @@ ConcurrentMark::ConcurrentMark(ReservedSpace rs,
task_queue->initialize();
_task_queues->register_queue(i, task_queue);
_tasks[i] = new CMTask(i, this, task_queue, _task_queues);
_count_card_bitmaps[i] = BitMap(card_bm_size, false);
_count_marked_bytes[i] = NEW_C_HEAP_ARRAY(size_t, max_regions);
_tasks[i] = new CMTask(i, this,
_count_marked_bytes[i],
&_count_card_bitmaps[i],
task_queue, _task_queues);
_accum_task_vtime[i] = 0.0;
}
// Calculate the card number for the bottom of the heap. Used
// in biasing indexes into the accounting card bitmaps.
_heap_bottom_card_num =
intptr_t(uintptr_t(_g1h->reserved_region().start()) >>
CardTableModRefBS::card_shift);
// Clear all the liveness counting data
clear_all_count_data();
if (ConcGCThreads > ParallelGCThreads) {
vm_exit_during_initialization("Can't have more ConcGCThreads "
"than ParallelGCThreads.");
......@@ -775,6 +801,9 @@ void ConcurrentMark::clearNextBitmap() {
assert(!g1h->mark_in_progress(), "invariant");
}
// Clear the liveness counting data
clear_all_count_data();
// Repeat the asserts from above.
guarantee(cmThread()->during_cycle(), "invariant");
guarantee(!g1h->mark_in_progress(), "invariant");
......@@ -1206,6 +1235,10 @@ void ConcurrentMark::checkpointRootsFinal(bool clear_all_soft_refs) {
gclog_or_tty->print_cr("\nRemark led to restart for overflow.");
}
} else {
// Aggregate the per-task counting data that we have accumulated
// while marking.
aggregate_count_data();
SATBMarkQueueSet& satb_mq_set = JavaThread::satb_mark_queue_set();
// We're done with marking.
// This is the end of the marking cycle, we're expected all
......@@ -1243,48 +1276,41 @@ void ConcurrentMark::checkpointRootsFinal(bool clear_all_soft_refs) {
g1p->record_concurrent_mark_remark_end();
}
#define CARD_BM_TEST_MODE 0
// Used to calculate the # live objects per region
// for verification purposes
class CalcLiveObjectsClosure: public HeapRegionClosure {
CMBitMapRO* _bm;
ConcurrentMark* _cm;
bool _changed;
bool _yield;
size_t _words_done;
BitMap* _region_bm;
BitMap* _card_bm;
// Debugging
size_t _tot_words_done;
size_t _tot_live;
size_t _tot_used;
size_t _regions_done;
double _start_vtime_sec;
BitMap* _region_bm;
BitMap* _card_bm;
size_t _region_marked_bytes;
intptr_t _bottom_card_num;
bool _final;
void mark_card_num_range(intptr_t start_card_num, intptr_t last_card_num) {
for (intptr_t i = start_card_num; i <= last_card_num; i++) {
#if CARD_BM_TEST_MODE
guarantee(_card_bm->at(i - _bottom_card_num), "Should already be set.");
#else
_card_bm->par_at_put(i - _bottom_card_num, 1);
#endif
assert(start_card_num <= last_card_num, "sanity");
BitMap::idx_t start_idx = start_card_num - _bottom_card_num;
BitMap::idx_t last_idx = last_card_num - _bottom_card_num;
for (BitMap::idx_t i = start_idx; i <= last_idx; i += 1) {
_card_bm->par_at_put(i, 1);
}
}
public:
CalcLiveObjectsClosure(bool final,
CMBitMapRO *bm, ConcurrentMark *cm,
CalcLiveObjectsClosure(CMBitMapRO *bm, ConcurrentMark *cm,
BitMap* region_bm, BitMap* card_bm) :
_bm(bm), _cm(cm), _changed(false), _yield(true),
_words_done(0), _tot_live(0), _tot_used(0),
_region_bm(region_bm), _card_bm(card_bm),_final(final),
_regions_done(0), _start_vtime_sec(0.0)
{
_bottom_card_num =
intptr_t(uintptr_t(G1CollectedHeap::heap()->reserved_region().start()) >>
CardTableModRefBS::card_shift);
}
_bm(bm), _cm(cm), _region_bm(region_bm), _card_bm(card_bm),
_region_marked_bytes(0), _tot_words_done(0),
_tot_live(0), _tot_used(0),
_bottom_card_num(cm->heap_bottom_card_num()) { }
// It takes a region that's not empty (i.e., it has at least one
// live object in it and sets its corresponding bit on the region
......@@ -1300,29 +1326,16 @@ public:
_region_bm->par_at_put((BitMap::idx_t) index, true);
} else {
// Starts humongous case: calculate how many regions are part of
// this humongous region and then set the bit range. It might
// have been a bit more efficient to look at the object that
// spans these humongous regions to calculate their number from
// the object's size. However, it's a good idea to calculate
// this based on the metadata itself, and not the region
// contents, so that this code is not aware of what goes into
// the humongous regions (in case this changes in the future).
// this humongous region and then set the bit range.
G1CollectedHeap* g1h = G1CollectedHeap::heap();
size_t end_index = index + 1;
while (end_index < g1h->n_regions()) {
HeapRegion* chr = g1h->region_at(end_index);
if (!chr->continuesHumongous()) break;
end_index += 1;
}
HeapRegion *last_hr = g1h->heap_region_containing_raw(hr->end() - 1);
size_t end_index = last_hr->hrs_index() + 1;
_region_bm->par_at_put_range((BitMap::idx_t) index,
(BitMap::idx_t) end_index, true);
}
}
bool doHeapRegion(HeapRegion* hr) {
if (!_final && _regions_done == 0) {
_start_vtime_sec = os::elapsedVTime();
}
if (hr->continuesHumongous()) {
// We will ignore these here and process them when their
......@@ -1336,48 +1349,41 @@ public:
}
HeapWord* nextTop = hr->next_top_at_mark_start();
HeapWord* start = hr->top_at_conc_mark_count();
assert(hr->bottom() <= start && start <= hr->end() &&
hr->bottom() <= nextTop && nextTop <= hr->end() &&
start <= nextTop,
"Preconditions.");
// Otherwise, record the number of word's we'll examine.
HeapWord* start = hr->bottom();
assert(start <= hr->end() && start <= nextTop && nextTop <= hr->end(),
err_msg("Preconditions not met - "
"start: "PTR_FORMAT", nextTop: "PTR_FORMAT", end: "PTR_FORMAT,
start, nextTop, hr->end()));
// Record the number of word's we'll examine.
size_t words_done = (nextTop - start);
// Find the first marked object at or after "start".
start = _bm->getNextMarkedWordAddress(start, nextTop);
size_t marked_bytes = 0;
// 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.
// The first card num of the sequence of live cards currently being
// constructed. -1 ==> no sequence.
intptr_t start_card_num = -1;
// The last card num of the sequence of live cards currently being
// constructed. -1 ==> no sequence.
intptr_t last_card_num = -1;
while (start < nextTop) {
if (_yield && _cm->do_yield_check()) {
// We yielded. It might be for a full collection, in which case
// all bets are off; terminate the traversal.
if (_cm->has_aborted()) {
_changed = false;
return true;
} else {
// Otherwise, it might be a collection pause, and the region
// we're looking at might be in the collection set. We'll
// abandon this region.
return false;
}
}
oop obj = oop(start);
int obj_sz = obj->size();
// The card num of the start of the current object.
intptr_t obj_card_num =
intptr_t(uintptr_t(start) >> CardTableModRefBS::card_shift);
HeapWord* obj_last = start + obj_sz - 1;
intptr_t obj_last_card_num =
intptr_t(uintptr_t(obj_last) >> CardTableModRefBS::card_shift);
......@@ -1395,110 +1401,404 @@ public:
start_card_num = obj_card_num;
}
}
#if CARD_BM_TEST_MODE
/*
gclog_or_tty->print_cr("Setting bits from %d/%d.",
obj_card_num - _bottom_card_num,
obj_last_card_num - _bottom_card_num);
*/
for (intptr_t j = obj_card_num; j <= obj_last_card_num; j++) {
_card_bm->par_at_put(j - _bottom_card_num, 1);
}
#endif
}
// In any case, we set the last card num.
last_card_num = obj_last_card_num;
marked_bytes += (size_t)obj_sz * HeapWordSize;
// Find the next marked object after this one.
start = _bm->getNextMarkedWordAddress(start + 1, nextTop);
_changed = true;
}
// Handle the last range, if any.
if (start_card_num != -1) {
mark_card_num_range(start_card_num, last_card_num);
}
if (_final) {
// Mark the allocated-since-marking portion...
HeapWord* tp = hr->top();
if (nextTop < tp) {
start_card_num =
intptr_t(uintptr_t(nextTop) >> CardTableModRefBS::card_shift);
last_card_num =
intptr_t(uintptr_t(tp) >> CardTableModRefBS::card_shift);
mark_card_num_range(start_card_num, last_card_num);
// This definitely means the region has live objects.
set_bit_for_region(hr);
}
// Mark the allocated-since-marking portion...
HeapWord* top = hr->top();
if (nextTop < top) {
start_card_num = intptr_t(uintptr_t(nextTop) >> CardTableModRefBS::card_shift);
last_card_num = intptr_t(uintptr_t(top) >> CardTableModRefBS::card_shift);
mark_card_num_range(start_card_num, last_card_num);
// This definitely means the region has live objects.
set_bit_for_region(hr);
}
hr->add_to_marked_bytes(marked_bytes);
// Update the live region bitmap.
if (marked_bytes > 0) {
set_bit_for_region(hr);
}
hr->set_top_at_conc_mark_count(nextTop);
// Set the marked bytes for the current region so that
// it can be queried by a calling verificiation routine
_region_marked_bytes = marked_bytes;
_tot_live += hr->next_live_bytes();
_tot_used += hr->used();
_words_done = words_done;
_tot_words_done = words_done;
if (!_final) {
++_regions_done;
if (_regions_done % 10 == 0) {
double end_vtime_sec = os::elapsedVTime();
double elapsed_vtime_sec = end_vtime_sec - _start_vtime_sec;
if (elapsed_vtime_sec > (10.0 / 1000.0)) {
jlong sleep_time_ms =
(jlong) (elapsed_vtime_sec * _cm->cleanup_sleep_factor() * 1000.0);
os::sleep(Thread::current(), sleep_time_ms, false);
_start_vtime_sec = end_vtime_sec;
return false;
}
size_t region_marked_bytes() const { return _region_marked_bytes; }
// Debugging
size_t tot_words_done() const { return _tot_words_done; }
size_t tot_live() const { return _tot_live; }
size_t tot_used() const { return _tot_used; }
};
// Heap region closure used for verifying the counting data
// that was accumulated concurrently and aggregated during
// the remark pause. This closure is applied to the heap
// regions during the STW cleanup pause.
class VerifyLiveObjectDataHRClosure: public HeapRegionClosure {
ConcurrentMark* _cm;
CalcLiveObjectsClosure _calc_cl;
BitMap* _region_bm; // Region BM to be verified
BitMap* _card_bm; // Card BM to be verified
bool _verbose; // verbose output?
BitMap* _exp_region_bm; // Expected Region BM values
BitMap* _exp_card_bm; // Expected card BM values
int _failures;
public:
VerifyLiveObjectDataHRClosure(ConcurrentMark* cm,
BitMap* region_bm,
BitMap* card_bm,
BitMap* exp_region_bm,
BitMap* exp_card_bm,
bool verbose) :
_cm(cm),
_calc_cl(_cm->nextMarkBitMap(), _cm, exp_region_bm, exp_card_bm),
_region_bm(region_bm), _card_bm(card_bm), _verbose(verbose),
_exp_region_bm(exp_region_bm), _exp_card_bm(exp_card_bm),
_failures(0) { }
int failures() const { return _failures; }
bool doHeapRegion(HeapRegion* hr) {
if (hr->continuesHumongous()) {
// We will ignore these here and process them when their
// associated "starts humongous" region is processed (see
// set_bit_for_heap_region()). Note that we cannot rely on their
// associated "starts humongous" region to have their bit set to
// 1 since, due to the region chunking in the parallel region
// iteration, a "continues humongous" region might be visited
// before its associated "starts humongous".
return false;
}
int failures = 0;
// Call the CalcLiveObjectsClosure to walk the marking bitmap for
// this region and set the corresponding bits in the expected region
// and card bitmaps.
bool res = _calc_cl.doHeapRegion(hr);
assert(res == false, "should be continuing");
MutexLockerEx x((_verbose ? ParGCRareEvent_lock : NULL),
Mutex::_no_safepoint_check_flag);
// Verify that _top_at_conc_count == ntams
if (hr->top_at_conc_mark_count() != hr->next_top_at_mark_start()) {
if (_verbose) {
gclog_or_tty->print_cr("Region " SIZE_FORMAT ": top at conc count incorrect: "
"expected " PTR_FORMAT ", actual: " PTR_FORMAT,
hr->hrs_index(), hr->next_top_at_mark_start(),
hr->top_at_conc_mark_count());
}
failures += 1;
}
// Verify the marked bytes for this region.
size_t exp_marked_bytes = _calc_cl.region_marked_bytes();
size_t act_marked_bytes = hr->next_marked_bytes();
// We're not OK if expected marked bytes > actual marked bytes. It means
// we have missed accounting some objects during the actual marking.
if (exp_marked_bytes > act_marked_bytes) {
if (_verbose) {
gclog_or_tty->print_cr("Region " SIZE_FORMAT ": marked bytes mismatch: "
"expected: " SIZE_FORMAT ", actual: " SIZE_FORMAT,
hr->hrs_index(), exp_marked_bytes, act_marked_bytes);
}
failures += 1;
}
// Verify the bit, for this region, in the actual and expected
// (which was just calculated) region bit maps.
// We're not OK if the bit in the calculated expected region
// bitmap is set and the bit in the actual region bitmap is not.
BitMap::idx_t index = (BitMap::idx_t)hr->hrs_index();
bool expected = _exp_region_bm->at(index);
bool actual = _region_bm->at(index);
if (expected && !actual) {
if (_verbose) {
gclog_or_tty->print_cr("Region " SIZE_FORMAT ": region bitmap mismatch: "
"expected: %d, actual: %d",
hr->hrs_index(), expected, actual);
}
failures += 1;
}
// Verify that the card bit maps for the cards spanned by the current
// region match. We have an error if we have a set bit in the expected
// bit map and the corresponding bit in the actual bitmap is not set.
BitMap::idx_t start_idx = _cm->card_bitmap_index_for(hr->bottom());
BitMap::idx_t end_idx = _cm->card_bitmap_index_for(hr->top());
for (BitMap::idx_t i = start_idx; i < end_idx; i+=1) {
expected = _exp_card_bm->at(i);
actual = _card_bm->at(i);
if (expected && !actual) {
if (_verbose) {
gclog_or_tty->print_cr("Region " SIZE_FORMAT ": card bitmap mismatch at " SIZE_FORMAT ": "
"expected: %d, actual: %d",
hr->hrs_index(), i, expected, actual);
}
failures += 1;
}
}
if (failures > 0 && _verbose) {
gclog_or_tty->print_cr("Region " HR_FORMAT ", ntams: " PTR_FORMAT ", "
"marked_bytes: calc/actual " SIZE_FORMAT "/" SIZE_FORMAT,
HR_FORMAT_PARAMS(hr), hr->next_top_at_mark_start(),
_calc_cl.region_marked_bytes(), hr->next_marked_bytes());
}
_failures += failures;
// We could stop iteration over the heap when we
// find the first voilating region by returning true.
return false;
}
};
class G1ParVerifyFinalCountTask: public AbstractGangTask {
protected:
G1CollectedHeap* _g1h;
ConcurrentMark* _cm;
BitMap* _actual_region_bm;
BitMap* _actual_card_bm;
uint _n_workers;
BitMap* _expected_region_bm;
BitMap* _expected_card_bm;
int _failures;
bool _verbose;
public:
G1ParVerifyFinalCountTask(G1CollectedHeap* g1h,
BitMap* region_bm, BitMap* card_bm,
BitMap* expected_region_bm, BitMap* expected_card_bm)
: AbstractGangTask("G1 verify final counting"),
_g1h(g1h), _cm(_g1h->concurrent_mark()),
_actual_region_bm(region_bm), _actual_card_bm(card_bm),
_expected_region_bm(expected_region_bm), _expected_card_bm(expected_card_bm),
_failures(0), _verbose(false),
_n_workers(0) {
assert(VerifyDuringGC, "don't call this otherwise");
// Use the value already set as the number of active threads
// in the call to run_task().
if (G1CollectedHeap::use_parallel_gc_threads()) {
assert( _g1h->workers()->active_workers() > 0,
"Should have been previously set");
_n_workers = _g1h->workers()->active_workers();
} else {
_n_workers = 1;
}
assert(_expected_card_bm->size() == _actual_card_bm->size(), "sanity");
assert(_expected_region_bm->size() == _actual_region_bm->size(), "sanity");
_verbose = _cm->verbose_medium();
}
void work(uint worker_id) {
assert(worker_id < _n_workers, "invariant");
VerifyLiveObjectDataHRClosure verify_cl(_cm,
_actual_region_bm, _actual_card_bm,
_expected_region_bm,
_expected_card_bm,
_verbose);
if (G1CollectedHeap::use_parallel_gc_threads()) {
_g1h->heap_region_par_iterate_chunked(&verify_cl,
worker_id,
_n_workers,
HeapRegion::VerifyCountClaimValue);
} else {
_g1h->heap_region_iterate(&verify_cl);
}
Atomic::add(verify_cl.failures(), &_failures);
}
bool changed() { return _changed; }
void reset() { _changed = false; _words_done = 0; }
void no_yield() { _yield = false; }
size_t words_done() { return _words_done; }
size_t tot_live() { return _tot_live; }
size_t tot_used() { return _tot_used; }
int failures() const { return _failures; }
};
// Final update of count data (during cleanup).
// Adds [top_at_count, NTAMS) to the marked bytes for each
// region. Sets the bits in the card bitmap corresponding
// to the interval [top_at_count, top], and sets the
// liveness bit for each region containing live data
// in the region bitmap.
void ConcurrentMark::calcDesiredRegions() {
_region_bm.clear();
_card_bm.clear();
CalcLiveObjectsClosure calccl(false /*final*/,
nextMarkBitMap(), this,
&_region_bm, &_card_bm);
G1CollectedHeap *g1h = G1CollectedHeap::heap();
g1h->heap_region_iterate(&calccl);
class FinalCountDataUpdateClosure: public HeapRegionClosure {
ConcurrentMark* _cm;
BitMap* _region_bm;
BitMap* _card_bm;
do {
calccl.reset();
g1h->heap_region_iterate(&calccl);
} while (calccl.changed());
}
size_t _total_live_bytes;
size_t _total_used_bytes;
size_t _total_words_done;
void set_card_bitmap_range(BitMap::idx_t start_idx, BitMap::idx_t last_idx) {
assert(start_idx <= last_idx, "sanity");
// Set the inclusive bit range [start_idx, last_idx].
// For small ranges (up to 8 cards) use a simple loop; otherwise
// use par_at_put_range.
if ((last_idx - start_idx) <= 8) {
for (BitMap::idx_t i = start_idx; i <= last_idx; i += 1) {
_card_bm->par_set_bit(i);
}
} else {
assert(last_idx < _card_bm->size(), "sanity");
// Note BitMap::par_at_put_range() is exclusive.
_card_bm->par_at_put_range(start_idx, last_idx+1, true);
}
}
// It takes a region that's not empty (i.e., it has at least one
// live object in it and sets its corresponding bit on the region
// bitmap to 1. If the region is "starts humongous" it will also set
// to 1 the bits on the region bitmap that correspond to its
// associated "continues humongous" regions.
void set_bit_for_region(HeapRegion* hr) {
assert(!hr->continuesHumongous(), "should have filtered those out");
size_t index = hr->hrs_index();
if (!hr->startsHumongous()) {
// Normal (non-humongous) case: just set the bit.
_region_bm->par_set_bit((BitMap::idx_t) index);
} else {
// Starts humongous case: calculate how many regions are part of
// this humongous region and then set the bit range.
G1CollectedHeap* g1h = G1CollectedHeap::heap();
HeapRegion *last_hr = g1h->heap_region_containing_raw(hr->end() - 1);
size_t end_index = last_hr->hrs_index() + 1;
_region_bm->par_at_put_range((BitMap::idx_t) index,
(BitMap::idx_t) end_index, true);
}
}
public:
FinalCountDataUpdateClosure(ConcurrentMark* cm,
BitMap* region_bm,
BitMap* card_bm) :
_cm(cm), _region_bm(region_bm), _card_bm(card_bm),
_total_words_done(0), _total_live_bytes(0), _total_used_bytes(0) { }
bool doHeapRegion(HeapRegion* hr) {
if (hr->continuesHumongous()) {
// We will ignore these here and process them when their
// associated "starts humongous" region is processed (see
// set_bit_for_heap_region()). Note that we cannot rely on their
// associated "starts humongous" region to have their bit set to
// 1 since, due to the region chunking in the parallel region
// iteration, a "continues humongous" region might be visited
// before its associated "starts humongous".
return false;
}
HeapWord* start = hr->top_at_conc_mark_count();
HeapWord* ntams = hr->next_top_at_mark_start();
HeapWord* top = hr->top();
assert(hr->bottom() <= start && start <= hr->end() &&
hr->bottom() <= ntams && ntams <= hr->end(), "Preconditions.");
size_t words_done = ntams - hr->bottom();
if (start < ntams) {
// Region was changed between remark and cleanup pauses
// We need to add (ntams - start) to the marked bytes
// for this region, and set bits for the range
// [ card_idx(start), card_idx(ntams) ) in the card bitmap.
size_t live_bytes = (ntams - start) * HeapWordSize;
hr->add_to_marked_bytes(live_bytes);
// Record the new top at conc count
hr->set_top_at_conc_mark_count(ntams);
// The setting of the bits in the card bitmap takes place below
}
// Mark the allocated-since-marking portion...
if (ntams < top) {
// This definitely means the region has live objects.
set_bit_for_region(hr);
}
// Now set the bits for [start, top]
BitMap::idx_t start_idx = _cm->card_bitmap_index_for(start);
BitMap::idx_t last_idx = _cm->card_bitmap_index_for(top);
set_card_bitmap_range(start_idx, last_idx);
// Set the bit for the region if it contains live data
if (hr->next_marked_bytes() > 0) {
set_bit_for_region(hr);
}
_total_words_done += words_done;
_total_used_bytes += hr->used();
_total_live_bytes += hr->next_marked_bytes();
return false;
}
size_t total_words_done() const { return _total_words_done; }
size_t total_live_bytes() const { return _total_live_bytes; }
size_t total_used_bytes() const { return _total_used_bytes; }
};
class G1ParFinalCountTask: public AbstractGangTask {
protected:
G1CollectedHeap* _g1h;
CMBitMap* _bm;
ConcurrentMark* _cm;
BitMap* _actual_region_bm;
BitMap* _actual_card_bm;
uint _n_workers;
size_t *_live_bytes;
size_t *_used_bytes;
BitMap* _region_bm;
BitMap* _card_bm;
public:
G1ParFinalCountTask(G1CollectedHeap* g1h, CMBitMap* bm,
BitMap* region_bm, BitMap* card_bm)
: AbstractGangTask("G1 final counting"), _g1h(g1h),
_bm(bm), _region_bm(region_bm), _card_bm(card_bm),
_n_workers(0)
{
G1ParFinalCountTask(G1CollectedHeap* g1h, BitMap* region_bm, BitMap* card_bm)
: AbstractGangTask("G1 final counting"),
_g1h(g1h), _cm(_g1h->concurrent_mark()),
_actual_region_bm(region_bm), _actual_card_bm(card_bm),
_n_workers(0) {
// Use the value already set as the number of active threads
// in the call to run_task(). Needed for the allocation of
// _live_bytes and _used_bytes.
......@@ -1520,29 +1820,32 @@ public:
}
void work(uint worker_id) {
CalcLiveObjectsClosure calccl(true /*final*/,
_bm, _g1h->concurrent_mark(),
_region_bm, _card_bm);
calccl.no_yield();
assert(worker_id < _n_workers, "invariant");
FinalCountDataUpdateClosure final_update_cl(_cm,
_actual_region_bm,
_actual_card_bm);
if (G1CollectedHeap::use_parallel_gc_threads()) {
_g1h->heap_region_par_iterate_chunked(&calccl, worker_id,
(int) _n_workers,
_g1h->heap_region_par_iterate_chunked(&final_update_cl,
worker_id,
_n_workers,
HeapRegion::FinalCountClaimValue);
} else {
_g1h->heap_region_iterate(&calccl);
_g1h->heap_region_iterate(&final_update_cl);
}
assert(calccl.complete(), "Shouldn't have yielded!");
assert(worker_id < _n_workers, "invariant");
_live_bytes[worker_id] = calccl.tot_live();
_used_bytes[worker_id] = calccl.tot_used();
_live_bytes[worker_id] = final_update_cl.total_live_bytes();
_used_bytes[worker_id] = final_update_cl.total_used_bytes();
}
size_t live_bytes() {
size_t live_bytes = 0;
for (uint i = 0; i < _n_workers; ++i)
live_bytes += _live_bytes[i];
return live_bytes;
}
size_t used_bytes() {
size_t used_bytes = 0;
for (uint i = 0; i < _n_workers; ++i)
......@@ -1705,8 +2008,7 @@ public:
G1ParScrubRemSetTask(G1CollectedHeap* g1h,
BitMap* region_bm, BitMap* card_bm) :
AbstractGangTask("G1 ScrubRS"), _g1rs(g1h->g1_rem_set()),
_region_bm(region_bm), _card_bm(card_bm)
{}
_region_bm(region_bm), _card_bm(card_bm) { }
void work(uint worker_id) {
if (G1CollectedHeap::use_parallel_gc_threads()) {
......@@ -1753,11 +2055,10 @@ void ConcurrentMark::cleanup() {
uint n_workers;
// Do counting once more with the world stopped for good measure.
G1ParFinalCountTask g1_par_count_task(g1h, nextMarkBitMap(),
&_region_bm, &_card_bm);
G1ParFinalCountTask g1_par_count_task(g1h, &_region_bm, &_card_bm);
if (G1CollectedHeap::use_parallel_gc_threads()) {
assert(g1h->check_heap_region_claim_values(
HeapRegion::InitialClaimValue),
assert(g1h->check_heap_region_claim_values(HeapRegion::InitialClaimValue),
"sanity check");
g1h->set_par_threads();
......@@ -1768,14 +2069,42 @@ void ConcurrentMark::cleanup() {
// Done with the parallel phase so reset to 0.
g1h->set_par_threads(0);
assert(g1h->check_heap_region_claim_values(
HeapRegion::FinalCountClaimValue),
assert(g1h->check_heap_region_claim_values(HeapRegion::FinalCountClaimValue),
"sanity check");
} else {
n_workers = 1;
g1_par_count_task.work(0);
}
if (VerifyDuringGC) {
// Verify that the counting data accumulated during marking matches
// that calculated by walking the marking bitmap.
// Bitmaps to hold expected values
BitMap expected_region_bm(_region_bm.size(), false);
BitMap expected_card_bm(_card_bm.size(), false);
G1ParVerifyFinalCountTask g1_par_verify_task(g1h,
&_region_bm,
&_card_bm,
&expected_region_bm,
&expected_card_bm);
if (G1CollectedHeap::use_parallel_gc_threads()) {
g1h->set_par_threads((int)n_workers);
g1h->workers()->run_task(&g1_par_verify_task);
// Done with the parallel phase so reset to 0.
g1h->set_par_threads(0);
assert(g1h->check_heap_region_claim_values(HeapRegion::VerifyCountClaimValue),
"sanity check");
} else {
g1_par_verify_task.work(0);
}
guarantee(g1_par_verify_task.failures() == 0, "Unexpected accounting failures");
}
size_t known_garbage_bytes =
g1_par_count_task.used_bytes() - g1_par_count_task.live_bytes();
g1p->set_known_garbage_bytes(known_garbage_bytes);
......@@ -1968,12 +2297,11 @@ bool G1CMIsAliveClosure::do_object_b(oop obj) {
class G1CMKeepAliveClosure: public OopClosure {
G1CollectedHeap* _g1;
ConcurrentMark* _cm;
CMBitMap* _bitMap;
public:
G1CMKeepAliveClosure(G1CollectedHeap* g1, ConcurrentMark* cm,
CMBitMap* bitMap) :
_g1(g1), _cm(cm),
_bitMap(bitMap) {}
G1CMKeepAliveClosure(G1CollectedHeap* g1, ConcurrentMark* cm) :
_g1(g1), _cm(cm) {
assert(Thread::current()->is_VM_thread(), "otherwise fix worker id");
}
virtual void do_oop(narrowOop* p) { do_oop_work(p); }
virtual void do_oop( oop* p) { do_oop_work(p); }
......@@ -1989,26 +2317,25 @@ class G1CMKeepAliveClosure: public OopClosure {
}
if (_g1->is_in_g1_reserved(addr) && _g1->is_obj_ill(obj)) {
_bitMap->mark(addr);
_cm->mark_and_count(obj);
_cm->mark_stack_push(obj);
}
}
};
class G1CMDrainMarkingStackClosure: public VoidClosure {
ConcurrentMark* _cm;
CMMarkStack* _markStack;
CMBitMap* _bitMap;
G1CMKeepAliveClosure* _oopClosure;
public:
G1CMDrainMarkingStackClosure(CMBitMap* bitMap, CMMarkStack* markStack,
G1CMDrainMarkingStackClosure(ConcurrentMark* cm, CMMarkStack* markStack,
G1CMKeepAliveClosure* oopClosure) :
_bitMap(bitMap),
_cm(cm),
_markStack(markStack),
_oopClosure(oopClosure)
{}
_oopClosure(oopClosure) { }
void do_void() {
_markStack->drain((OopClosure*)_oopClosure, _bitMap, false);
_markStack->drain((OopClosure*)_oopClosure, _cm->nextMarkBitMap(), false);
}
};
......@@ -2087,8 +2414,7 @@ class G1CMParDrainMarkingStackClosure: public VoidClosure {
CMTask* _task;
public:
G1CMParDrainMarkingStackClosure(ConcurrentMark* cm, CMTask* task) :
_cm(cm), _task(task)
{}
_cm(cm), _task(task) { }
void do_void() {
do {
......@@ -2227,9 +2553,9 @@ void ConcurrentMark::weakRefsWork(bool clear_all_soft_refs) {
rp->setup_policy(clear_all_soft_refs);
assert(_markStack.isEmpty(), "mark stack should be empty");
G1CMKeepAliveClosure g1_keep_alive(g1h, this, nextMarkBitMap());
G1CMKeepAliveClosure g1_keep_alive(g1h, this);
G1CMDrainMarkingStackClosure
g1_drain_mark_stack(nextMarkBitMap(), &_markStack, &g1_keep_alive);
g1_drain_mark_stack(this, &_markStack, &g1_keep_alive);
// We use the work gang from the G1CollectedHeap and we utilize all
// the worker threads.
......@@ -2601,18 +2927,6 @@ void ConcurrentMark::print_reachable(const char* str,
// during an evacuation pause). This was a late change to the code and
// is currently not being taken advantage of.
class CMGlobalObjectClosure : public ObjectClosure {
private:
ConcurrentMark* _cm;
public:
void do_object(oop obj) {
_cm->deal_with_reference(obj);
}
CMGlobalObjectClosure(ConcurrentMark* cm) : _cm(cm) { }
};
void ConcurrentMark::deal_with_reference(oop obj) {
if (verbose_high()) {
gclog_or_tty->print_cr("[global] we're dealing with reference "PTR_FORMAT,
......@@ -2657,6 +2971,18 @@ void ConcurrentMark::deal_with_reference(oop obj) {
}
}
class CMGlobalObjectClosure : public ObjectClosure {
private:
ConcurrentMark* _cm;
public:
void do_object(oop obj) {
_cm->deal_with_reference(obj);
}
CMGlobalObjectClosure(ConcurrentMark* cm) : _cm(cm) { }
};
void ConcurrentMark::drainAllSATBBuffers() {
guarantee(false, "drainAllSATBBuffers(): don't call this any more");
......@@ -2678,15 +3004,6 @@ void ConcurrentMark::drainAllSATBBuffers() {
assert(satb_mq_set.completed_buffers_num() == 0, "invariant");
}
void ConcurrentMark::clear(oop p) {
assert(p != NULL && p->is_oop(), "expected an oop");
HeapWord* addr = (HeapWord*)p;
assert(addr >= _nextMarkBitMap->startWord() ||
addr < _nextMarkBitMap->endWord(), "in a region");
_nextMarkBitMap->clear(addr);
}
void ConcurrentMark::clearRangePrevBitmap(MemRegion mr) {
// Note we are overriding the read-only view of the prev map here, via
// the cast.
......@@ -3000,6 +3317,192 @@ void ConcurrentMark::clear_marking_state(bool clear_overflow) {
}
}
// Aggregate the counting data that was constructed concurrently
// with marking.
class AggregateCountDataHRClosure: public HeapRegionClosure {
ConcurrentMark* _cm;
BitMap* _cm_card_bm;
size_t _max_task_num;
public:
AggregateCountDataHRClosure(ConcurrentMark *cm,
BitMap* cm_card_bm,
size_t max_task_num) :
_cm(cm), _cm_card_bm(cm_card_bm),
_max_task_num(max_task_num) { }
bool is_card_aligned(HeapWord* p) {
return ((uintptr_t(p) & (CardTableModRefBS::card_size - 1)) == 0);
}
bool doHeapRegion(HeapRegion* hr) {
if (hr->continuesHumongous()) {
// We will ignore these here and process them when their
// associated "starts humongous" region is processed.
// Note that we cannot rely on their associated
// "starts humongous" region to have their bit set to 1
// since, due to the region chunking in the parallel region
// iteration, a "continues humongous" region might be visited
// before its associated "starts humongous".
return false;
}
HeapWord* start = hr->bottom();
HeapWord* limit = hr->next_top_at_mark_start();
HeapWord* end = hr->end();
assert(start <= limit && limit <= hr->top() && hr->top() <= hr->end(),
err_msg("Preconditions not met - "
"start: "PTR_FORMAT", limit: "PTR_FORMAT", "
"top: "PTR_FORMAT", end: "PTR_FORMAT,
start, limit, hr->top(), hr->end()));
assert(hr->next_marked_bytes() == 0, "Precondition");
if (start == limit) {
// NTAMS of this region has not been set so nothing to do.
return false;
}
assert(is_card_aligned(start), "sanity");
assert(is_card_aligned(end), "sanity");
BitMap::idx_t start_idx = _cm->card_bitmap_index_for(start);
BitMap::idx_t limit_idx = _cm->card_bitmap_index_for(limit);
BitMap::idx_t end_idx = _cm->card_bitmap_index_for(end);
// If ntams is not card aligned then we bump the index for
// limit so that we get the card spanning ntams.
if (!is_card_aligned(limit)) {
limit_idx += 1;
}
assert(limit_idx <= end_idx, "or else use atomics");
// Aggregate the "stripe" in the count data associated with hr.
size_t hrs_index = hr->hrs_index();
size_t marked_bytes = 0;
for (int i = 0; (size_t)i < _max_task_num; i += 1) {
size_t* marked_bytes_array = _cm->count_marked_bytes_array_for(i);
BitMap* task_card_bm = _cm->count_card_bitmap_for(i);
// Fetch the marked_bytes in this region for task i and
// add it to the running total for this region.
marked_bytes += marked_bytes_array[hrs_index];
// Now union the bitmaps[0,max_task_num)[start_idx..limit_idx)
// into the global card bitmap.
BitMap::idx_t scan_idx = task_card_bm->get_next_one_offset(start_idx, limit_idx);
while (scan_idx < limit_idx) {
assert(task_card_bm->at(scan_idx) == true, "should be");
_cm_card_bm->set_bit(scan_idx);
assert(_cm_card_bm->at(scan_idx) == true, "should be");
// BitMap::get_next_one_offset() can handle the case when
// its left_offset parameter is greater than its right_offset
// parameter. If does, however, have an early exit if
// left_offset == right_offset. So let's limit the value
// passed in for left offset here.
BitMap::idx_t next_idx = MIN2(scan_idx + 1, limit_idx);
scan_idx = task_card_bm->get_next_one_offset(next_idx, limit_idx);
}
}
// Update the marked bytes for this region.
hr->add_to_marked_bytes(marked_bytes);
// Now set the top at count to NTAMS.
hr->set_top_at_conc_mark_count(limit);
// Next heap region
return false;
}
};
class G1AggregateCountDataTask: public AbstractGangTask {
protected:
G1CollectedHeap* _g1h;
ConcurrentMark* _cm;
BitMap* _cm_card_bm;
size_t _max_task_num;
int _active_workers;
public:
G1AggregateCountDataTask(G1CollectedHeap* g1h,
ConcurrentMark* cm,
BitMap* cm_card_bm,
size_t max_task_num,
int n_workers) :
AbstractGangTask("Count Aggregation"),
_g1h(g1h), _cm(cm), _cm_card_bm(cm_card_bm),
_max_task_num(max_task_num),
_active_workers(n_workers) { }
void work(uint worker_id) {
AggregateCountDataHRClosure cl(_cm, _cm_card_bm, _max_task_num);
if (G1CollectedHeap::use_parallel_gc_threads()) {
_g1h->heap_region_par_iterate_chunked(&cl, worker_id,
_active_workers,
HeapRegion::AggregateCountClaimValue);
} else {
_g1h->heap_region_iterate(&cl);
}
}
};
void ConcurrentMark::aggregate_count_data() {
int n_workers = (G1CollectedHeap::use_parallel_gc_threads() ?
_g1h->workers()->active_workers() :
1);
G1AggregateCountDataTask g1_par_agg_task(_g1h, this, &_card_bm,
_max_task_num, n_workers);
if (G1CollectedHeap::use_parallel_gc_threads()) {
assert(_g1h->check_heap_region_claim_values(HeapRegion::InitialClaimValue),
"sanity check");
_g1h->set_par_threads(n_workers);
_g1h->workers()->run_task(&g1_par_agg_task);
_g1h->set_par_threads(0);
assert(_g1h->check_heap_region_claim_values(HeapRegion::AggregateCountClaimValue),
"sanity check");
_g1h->reset_heap_region_claim_values();
} else {
g1_par_agg_task.work(0);
}
}
// Clear the per-worker arrays used to store the per-region counting data
void ConcurrentMark::clear_all_count_data() {
// Clear the global card bitmap - it will be filled during
// liveness count aggregation (during remark) and the
// final counting task.
_card_bm.clear();
// Clear the global region bitmap - it will be filled as part
// of the final counting task.
_region_bm.clear();
size_t max_regions = _g1h->max_regions();
assert(_max_task_num != 0, "unitialized");
for (int i = 0; (size_t) i < _max_task_num; i += 1) {
BitMap* task_card_bm = count_card_bitmap_for(i);
size_t* marked_bytes_array = count_marked_bytes_array_for(i);
assert(task_card_bm->size() == _card_bm.size(), "size mismatch");
assert(marked_bytes_array != NULL, "uninitialized");
memset(marked_bytes_array, 0, (max_regions * sizeof(size_t)));
task_card_bm->clear();
}
}
void ConcurrentMark::print_stats() {
if (verbose_stats()) {
gclog_or_tty->print_cr("---------------------------------------------------------------------");
......@@ -3335,6 +3838,8 @@ void ConcurrentMark::reset_active_task_region_fields_in_cset() {
void ConcurrentMark::abort() {
// Clear all marks to force marking thread to do nothing
_nextMarkBitMap->clearAll();
// Clear the liveness counting data
clear_all_count_data();
// Empty mark stack
clear_marking_state();
for (int i = 0; i < (int)_max_task_num; ++i) {
......@@ -3387,10 +3892,9 @@ void ConcurrentMark::print_summary_info() {
(_init_times.sum() + _remark_times.sum() +
_cleanup_times.sum())/1000.0);
gclog_or_tty->print_cr(" Total concurrent time = %8.2f s "
"(%8.2f s marking, %8.2f s counting).",
"(%8.2f s marking).",
cmThread()->vtime_accum(),
cmThread()->vtime_mark_accum(),
cmThread()->vtime_count_accum());
cmThread()->vtime_mark_accum());
}
void ConcurrentMark::print_worker_threads_on(outputStream* st) const {
......@@ -4682,6 +5186,8 @@ void CMTask::do_marking_step(double time_target_ms,
CMTask::CMTask(int task_id,
ConcurrentMark* cm,
size_t* marked_bytes,
BitMap* card_bm,
CMTaskQueue* task_queue,
CMTaskQueueSet* task_queues)
: _g1h(G1CollectedHeap::heap()),
......@@ -4691,7 +5197,9 @@ CMTask::CMTask(int task_id,
_task_queue(task_queue),
_task_queues(task_queues),
_cm_oop_closure(NULL),
_aborted_region(MemRegion()) {
_aborted_region(MemRegion()),
_marked_bytes_array(marked_bytes),
_card_bm(card_bm) {
guarantee(task_queue != NULL, "invariant");
guarantee(task_queues != NULL, "invariant");
......
......@@ -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.
先完成此消息的编辑!
想要评论请 注册