7097567: G1: abstract and encapsulate collector phases and transitions between them

Reviewed-by: mgerdin, tschatzl
This commit is contained in:
Derek White 2015-06-05 10:27:41 +02:00
parent 5f5af54f95
commit 24c9ffed42
14 changed files with 252 additions and 203 deletions

View File

@ -30,6 +30,7 @@
#include "gc/g1/concurrentMarkThread.inline.hpp"
#include "gc/g1/g1CollectedHeap.inline.hpp"
#include "gc/g1/g1CollectorPolicy.hpp"
#include "gc/g1/g1CollectorState.hpp"
#include "gc/g1/g1ErgoVerbose.hpp"
#include "gc/g1/g1Log.hpp"
#include "gc/g1/g1OopClosures.inline.hpp"
@ -177,7 +178,7 @@ class ClearBitmapHRClosure : public HeapRegionClosure {
// will have them as guarantees at the beginning / end of the bitmap
// clearing to get some checking in the product.
assert(!_may_yield || _cm->cmThread()->during_cycle(), "invariant");
assert(!_may_yield || !G1CollectedHeap::heap()->mark_in_progress(), "invariant");
assert(!_may_yield || !G1CollectedHeap::heap()->collector_state()->mark_in_progress(), "invariant");
}
return false;
@ -830,7 +831,7 @@ void ConcurrentMark::clearNextBitmap() {
// marking bitmap and getting it ready for the next cycle. During
// this time no other cycle can start. So, let's make sure that this
// is the case.
guarantee(!g1h->mark_in_progress(), "invariant");
guarantee(!g1h->collector_state()->mark_in_progress(), "invariant");
ClearBitmapHRClosure cl(this, _nextMarkBitMap, true /* may_yield */);
ParClearNextMarkBitmapTask task(&cl, parallel_marking_threads(), true);
@ -844,7 +845,7 @@ void ConcurrentMark::clearNextBitmap() {
// Repeat the asserts from above.
guarantee(cmThread()->during_cycle(), "invariant");
guarantee(!g1h->mark_in_progress(), "invariant");
guarantee(!g1h->collector_state()->mark_in_progress(), "invariant");
}
class CheckBitmapClearHRClosure : public HeapRegionClosure {
@ -1254,7 +1255,7 @@ void ConcurrentMark::checkpointRootsFinal(bool clear_all_soft_refs) {
// If a full collection has happened, we shouldn't do this.
if (has_aborted()) {
g1h->set_marking_complete(); // So bitmap clearing isn't confused
g1h->collector_state()->set_mark_in_progress(false); // So bitmap clearing isn't confused
return;
}
@ -1888,7 +1889,7 @@ void ConcurrentMark::cleanup() {
// If a full collection has happened, we shouldn't do this.
if (has_aborted()) {
g1h->set_marking_complete(); // So bitmap clearing isn't confused
g1h->collector_state()->set_mark_in_progress(false); // So bitmap clearing isn't confused
return;
}
@ -1934,7 +1935,7 @@ void ConcurrentMark::cleanup() {
}
size_t start_used_bytes = g1h->used();
g1h->set_marking_complete();
g1h->collector_state()->set_mark_in_progress(false);
double count_end = os::elapsedTime();
double this_final_counting_time = (count_end - start);
@ -2756,7 +2757,7 @@ public:
void ConcurrentMark::verify_no_cset_oops() {
assert(SafepointSynchronize::is_at_safepoint(), "should be at a safepoint");
if (!G1CollectedHeap::heap()->mark_in_progress()) {
if (!G1CollectedHeap::heap()->collector_state()->mark_in_progress()) {
return;
}

View File

@ -194,7 +194,7 @@ void ConcurrentMarkThread::run() {
// We don't want to update the marking status if a GC pause
// is already underway.
SuspendibleThreadSetJoiner sts_join;
g1h->set_marking_complete();
g1h->collector_state()->set_mark_in_progress(false);
}
// Check if cleanup set the free_regions_coming flag. If it

View File

@ -65,7 +65,7 @@ void G1Allocator::reuse_retained_old_region(EvacuationInfo& evacuation_info,
// we allocate to in the region sets. We'll re-add it later, when
// it's retired again.
_g1h->_old_set.remove(retained_region);
bool during_im = _g1h->g1_policy()->during_initial_mark_pause();
bool during_im = _g1h->collector_state()->during_initial_mark_pause();
retained_region->note_start_of_copying(during_im);
old->set(retained_region);
_g1h->_hr_printer.reuse(retained_region);

View File

@ -34,6 +34,7 @@
#include "gc/g1/g1AllocRegion.inline.hpp"
#include "gc/g1/g1CollectedHeap.inline.hpp"
#include "gc/g1/g1CollectorPolicy.hpp"
#include "gc/g1/g1CollectorState.hpp"
#include "gc/g1/g1ErgoVerbose.hpp"
#include "gc/g1/g1EvacFailure.hpp"
#include "gc/g1/g1GCPhaseTimes.hpp"
@ -1039,7 +1040,7 @@ HeapWord* G1CollectedHeap::attempt_allocation_at_safepoint(size_t word_size,
} else {
HeapWord* result = humongous_obj_allocate(word_size, context);
if (result != NULL && g1_policy()->need_to_start_conc_mark("STW humongous allocation")) {
g1_policy()->set_initiate_conc_mark_if_possible();
collector_state()->set_initiate_conc_mark_if_possible(true);
}
return result;
}
@ -1250,7 +1251,7 @@ bool G1CollectedHeap::do_collection(bool explicit_gc,
g1_policy()->stop_incremental_cset_building();
tear_down_region_sets(false /* free_list_only */);
g1_policy()->set_gcs_are_young(true);
collector_state()->set_gcs_are_young(true);
// See the comments in g1CollectedHeap.hpp and
// G1CollectedHeap::ref_processing_init() about
@ -1714,11 +1715,9 @@ G1CollectedHeap::G1CollectedHeap(G1CollectorPolicy* policy_) :
_ref_processor_stw(NULL),
_bot_shared(NULL),
_evac_failure_scan_stack(NULL),
_mark_in_progress(false),
_cg1r(NULL),
_g1mm(NULL),
_refine_cte_cl(NULL),
_full_collection(false),
_secondary_free_list("Secondary Free List", new SecondaryFreeRegionListMtSafeChecker()),
_old_set("Old Set", false /* humongous */, new OldRegionSetMtSafeChecker()),
_humongous_set("Master Humongous Set", true /* humongous */, new HumongousRegionSetMtSafeChecker()),
@ -1733,7 +1732,6 @@ G1CollectedHeap::G1CollectedHeap(G1CollectorPolicy* policy_) :
_surviving_young_words(NULL),
_old_marking_cycles_started(0),
_old_marking_cycles_completed(0),
_concurrent_cycle_started(false),
_heap_summary_sent(false),
_in_cset_fast_test(),
_dirty_cards_region_list(NULL),
@ -2288,7 +2286,7 @@ void G1CollectedHeap::increment_old_marking_cycles_completed(bool concurrent) {
}
void G1CollectedHeap::register_concurrent_cycle_start(const Ticks& start_time) {
_concurrent_cycle_started = true;
collector_state()->set_concurrent_cycle_started(true);
_gc_timer_cm->register_gc_start(start_time);
_gc_tracer_cm->report_gc_start(gc_cause(), _gc_timer_cm->gc_start());
@ -2296,7 +2294,7 @@ void G1CollectedHeap::register_concurrent_cycle_start(const Ticks& start_time) {
}
void G1CollectedHeap::register_concurrent_cycle_end() {
if (_concurrent_cycle_started) {
if (collector_state()->concurrent_cycle_started()) {
if (_cm->has_aborted()) {
_gc_tracer_cm->report_concurrent_mode_failure();
}
@ -2305,13 +2303,13 @@ void G1CollectedHeap::register_concurrent_cycle_end() {
_gc_tracer_cm->report_gc_end(_gc_timer_cm->gc_end(), _gc_timer_cm->time_partitions());
// Clear state variables to prepare for the next concurrent cycle.
_concurrent_cycle_started = false;
collector_state()->set_concurrent_cycle_started(false);
_heap_summary_sent = false;
}
}
void G1CollectedHeap::trace_heap_after_concurrent_cycle() {
if (_concurrent_cycle_started) {
if (collector_state()->concurrent_cycle_started()) {
// This function can be called when:
// the cleanup pause is run
// the concurrent cycle is aborted before the cleanup pause.
@ -2325,22 +2323,6 @@ void G1CollectedHeap::trace_heap_after_concurrent_cycle() {
}
}
G1YCType G1CollectedHeap::yc_type() {
bool is_young = g1_policy()->gcs_are_young();
bool is_initial_mark = g1_policy()->during_initial_mark_pause();
bool is_during_mark = mark_in_progress();
if (is_initial_mark) {
return InitialMark;
} else if (is_during_mark) {
return DuringMark;
} else if (is_young) {
return Normal;
} else {
return Mixed;
}
}
void G1CollectedHeap::collect(GCCause::Cause cause) {
assert_heap_not_locked();
@ -3587,8 +3569,8 @@ void G1CollectedHeap::log_gc_header() {
gclog_or_tty->gclog_stamp(_gc_tracer_stw->gc_id());
GCCauseString gc_cause_str = GCCauseString("GC pause", gc_cause())
.append(g1_policy()->gcs_are_young() ? "(young)" : "(mixed)")
.append(g1_policy()->during_initial_mark_pause() ? " (initial-mark)" : "");
.append(collector_state()->gcs_are_young() ? "(young)" : "(mixed)")
.append(collector_state()->during_initial_mark_pause() ? " (initial-mark)" : "");
gclog_or_tty->print("[%s", (const char*)gc_cause_str);
}
@ -3645,29 +3627,29 @@ G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
g1_policy()->decide_on_conc_mark_initiation();
// We do not allow initial-mark to be piggy-backed on a mixed GC.
assert(!g1_policy()->during_initial_mark_pause() ||
g1_policy()->gcs_are_young(), "sanity");
assert(!collector_state()->during_initial_mark_pause() ||
collector_state()->gcs_are_young(), "sanity");
// We also do not allow mixed GCs during marking.
assert(!mark_in_progress() || g1_policy()->gcs_are_young(), "sanity");
assert(!collector_state()->mark_in_progress() || collector_state()->gcs_are_young(), "sanity");
// Record whether this pause is an initial mark. When the current
// thread has completed its logging output and it's safe to signal
// the CM thread, the flag's value in the policy has been reset.
bool should_start_conc_mark = g1_policy()->during_initial_mark_pause();
bool should_start_conc_mark = collector_state()->during_initial_mark_pause();
// Inner scope for scope based logging, timers, and stats collection
{
EvacuationInfo evacuation_info;
if (g1_policy()->during_initial_mark_pause()) {
if (collector_state()->during_initial_mark_pause()) {
// We are about to start a marking cycle, so we increment the
// full collection counter.
increment_old_marking_cycles_started();
register_concurrent_cycle_start(_gc_timer_stw->gc_start());
}
_gc_tracer_stw->report_yc_type(yc_type());
_gc_tracer_stw->report_yc_type(collector_state()->yc_type());
TraceCPUTime tcpu(G1Log::finer(), true, gclog_or_tty);
@ -3677,7 +3659,7 @@ G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
workers()->set_active_workers(active_workers);
double pause_start_sec = os::elapsedTime();
g1_policy()->phase_times()->note_gc_start(active_workers, mark_in_progress());
g1_policy()->phase_times()->note_gc_start(active_workers, collector_state()->mark_in_progress());
log_gc_header();
TraceCollectorStats tcs(g1mm()->incremental_collection_counters());
@ -3771,7 +3753,7 @@ G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
_young_list->print();
#endif // YOUNG_LIST_VERBOSE
if (g1_policy()->during_initial_mark_pause()) {
if (collector_state()->during_initial_mark_pause()) {
concurrent_mark()->checkpointRootsInitialPre();
}
@ -3859,12 +3841,12 @@ G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
_allocator->increase_used(g1_policy()->bytes_copied_during_gc());
}
if (g1_policy()->during_initial_mark_pause()) {
if (collector_state()->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();
set_marking_started();
collector_state()->set_mark_in_progress(true);
// Note that we don't actually trigger the CM thread at
// this point. We do that later when we're sure that
// the current thread has completed its logging output.
@ -4343,7 +4325,7 @@ public:
pss.set_evac_failure_closure(&evac_failure_cl);
bool only_young = _g1h->g1_policy()->gcs_are_young();
bool only_young = _g1h->collector_state()->gcs_are_young();
// Non-IM young GC.
G1ParCopyClosure<G1BarrierNone, G1MarkNone> scan_only_root_cl(_g1h, &pss, rp);
@ -4369,7 +4351,7 @@ public:
bool trace_metadata = false;
if (_g1h->g1_policy()->during_initial_mark_pause()) {
if (_g1h->collector_state()->during_initial_mark_pause()) {
// We also need to mark copied objects.
strong_root_cl = &scan_mark_root_cl;
strong_cld_cl = &scan_mark_cld_cl;
@ -5021,7 +5003,7 @@ public:
OopClosure* copy_non_heap_cl = &only_copy_non_heap_cl;
if (_g1h->g1_policy()->during_initial_mark_pause()) {
if (_g1h->collector_state()->during_initial_mark_pause()) {
// We also need to mark copied objects.
copy_non_heap_cl = &copy_mark_non_heap_cl;
}
@ -5122,7 +5104,7 @@ public:
OopClosure* copy_non_heap_cl = &only_copy_non_heap_cl;
if (_g1h->g1_policy()->during_initial_mark_pause()) {
if (_g1h->collector_state()->during_initial_mark_pause()) {
// We also need to mark copied objects.
copy_non_heap_cl = &copy_mark_non_heap_cl;
}
@ -5234,7 +5216,7 @@ void G1CollectedHeap::process_discovered_references() {
OopClosure* copy_non_heap_cl = &only_copy_non_heap_cl;
if (g1_policy()->during_initial_mark_pause()) {
if (collector_state()->during_initial_mark_pause()) {
// We also need to mark copied objects.
copy_non_heap_cl = &copy_mark_non_heap_cl;
}
@ -5342,7 +5324,7 @@ void G1CollectedHeap::evacuate_collection_set(EvacuationInfo& evacuation_info) {
G1RootProcessor root_processor(this, n_workers);
G1ParTask g1_par_task(this, _task_queues, &root_processor, n_workers);
// InitialMark needs claim bits to keep track of the marked-through CLDs.
if (g1_policy()->during_initial_mark_pause()) {
if (collector_state()->during_initial_mark_pause()) {
ClassLoaderDataGraph::clear_claimed_marks();
}
@ -5598,7 +5580,7 @@ bool G1CollectedHeap::verify_bitmaps(const char* caller, HeapRegion* hr) {
// We reset mark_in_progress() before we reset _cmThread->in_progress() and in this window
// we do the clearing of the next bitmap concurrently. Thus, we can not verify the bitmap
// if we happen to be in that state.
if (mark_in_progress() || !_cmThread->in_progress()) {
if (collector_state()->mark_in_progress() || !_cmThread->in_progress()) {
res_n = verify_no_bits_over_tams("next", next_bitmap, ntams, end);
}
if (!res_p || !res_n) {
@ -6279,7 +6261,7 @@ HeapRegion* G1CollectedHeap::new_gc_alloc_region(size_t word_size,
_hr_printer.alloc(new_alloc_region, G1HRPrinter::Old);
check_bitmaps("Old Region Allocation", new_alloc_region);
}
bool during_im = g1_policy()->during_initial_mark_pause();
bool during_im = collector_state()->during_initial_mark_pause();
new_alloc_region->note_start_of_copying(during_im);
return new_alloc_region;
}
@ -6290,7 +6272,7 @@ HeapRegion* G1CollectedHeap::new_gc_alloc_region(size_t word_size,
void G1CollectedHeap::retire_gc_alloc_region(HeapRegion* alloc_region,
size_t allocated_bytes,
InCSetState dest) {
bool during_im = g1_policy()->during_initial_mark_pause();
bool during_im = collector_state()->during_initial_mark_pause();
alloc_region->note_end_of_copying(during_im);
g1_policy()->record_bytes_copied_during_gc(allocated_bytes);
if (dest.is_young()) {

View File

@ -31,6 +31,7 @@
#include "gc/g1/g1AllocationContext.hpp"
#include "gc/g1/g1Allocator.hpp"
#include "gc/g1/g1BiasedArray.hpp"
#include "gc/g1/g1CollectorState.hpp"
#include "gc/g1/g1HRPrinter.hpp"
#include "gc/g1/g1InCSetState.hpp"
#include "gc/g1/g1MonitoringSupport.hpp"
@ -328,6 +329,9 @@ private:
// (d) cause == _g1_humongous_allocation
bool should_do_concurrent_full_gc(GCCause::Cause cause);
// indicates whether we are in young or mixed GC mode
G1CollectorState _collector_state;
// Keeps track of how many "old marking cycles" (i.e., Full GCs or
// concurrent cycles) we have started.
volatile uint _old_marking_cycles_started;
@ -336,7 +340,6 @@ private:
// concurrent cycles) we have completed.
volatile uint _old_marking_cycles_completed;
bool _concurrent_cycle_started;
bool _heap_summary_sent;
// This is a non-product method that is helpful for testing. It is
@ -701,8 +704,6 @@ public:
void register_concurrent_cycle_end();
void trace_heap_after_concurrent_cycle();
G1YCType yc_type();
G1HRPrinter* hr_printer() { return &_hr_printer; }
// Frees a non-humongous region by initializing its contents and
@ -791,7 +792,6 @@ protected:
// The concurrent marker (and the thread it runs in.)
ConcurrentMark* _cm;
ConcurrentMarkThread* _cmThread;
bool _mark_in_progress;
// The concurrent refiner.
ConcurrentG1Refine* _cg1r;
@ -1019,6 +1019,8 @@ public:
return CollectedHeap::G1CollectedHeap;
}
G1CollectorState* collector_state() { return &_collector_state; }
// The current policy object for the collector.
G1CollectorPolicy* g1_policy() const { return _g1_policy; }
@ -1399,17 +1401,6 @@ public:
// bits.
void markModUnionRange(MemRegion mr);
// Records the fact that a marking phase is no longer in progress.
void set_marking_complete() {
_mark_in_progress = false;
}
void set_marking_started() {
_mark_in_progress = true;
}
bool mark_in_progress() {
return _mark_in_progress;
}
// Print the maximum heap capacity.
virtual size_t max_capacity() const;
@ -1522,14 +1513,6 @@ public:
void redirty_logged_cards();
// Verification
// The following is just to alert the verification code
// that a full collection has occurred and that the
// remembered sets are no longer up to date.
bool _full_collection;
void set_full_collection() { _full_collection = true;}
void clear_full_collection() {_full_collection = false;}
bool full_collection() {return _full_collection;}
// Perform any cleanup actions necessary before allowing a verification.
virtual void prepare_for_verify();

View File

@ -29,6 +29,7 @@
#include "gc/g1/g1AllocRegion.inline.hpp"
#include "gc/g1/g1CollectedHeap.hpp"
#include "gc/g1/g1CollectorPolicy.hpp"
#include "gc/g1/g1CollectorState.hpp"
#include "gc/g1/g1SATBCardTableModRefBS.hpp"
#include "gc/g1/heapRegionManager.inline.hpp"
#include "gc/g1/heapRegionSet.inline.hpp"
@ -288,9 +289,9 @@ G1CollectedHeap::set_evacuation_failure_alot_for_current_gc() {
_evacuation_failure_alot_for_current_gc = (elapsed_gcs >= G1EvacuationFailureALotInterval);
// Now check if G1EvacuationFailureALot is enabled for the current GC type.
const bool gcs_are_young = g1_policy()->gcs_are_young();
const bool during_im = g1_policy()->during_initial_mark_pause();
const bool during_marking = mark_in_progress();
const bool gcs_are_young = collector_state()->gcs_are_young();
const bool during_im = collector_state()->during_initial_mark_pause();
const bool during_marking = collector_state()->mark_in_progress();
_evacuation_failure_alot_for_current_gc &=
evacuation_failure_alot_for_gc_type(gcs_are_young,

View File

@ -107,22 +107,11 @@ G1CollectorPolicy::G1CollectorPolicy() :
_pause_time_target_ms((double) MaxGCPauseMillis),
_gcs_are_young(true),
_during_marking(false),
_in_marking_window(false),
_in_marking_window_im(false),
_recent_prev_end_times_for_all_gcs_sec(
new TruncatedSeq(NumPrevPausesForHeuristics)),
_recent_avg_pause_time_ratio(0.0),
_initiate_conc_mark_if_possible(false),
_during_initial_mark_pause(false),
_last_young_gc(false),
_last_gc_was_young(false),
_eden_used_bytes_before_gc(0),
_survivor_used_bytes_before_gc(0),
_heap_used_bytes_before_gc(0),
@ -334,6 +323,8 @@ void G1CollectorPolicy::post_heap_initialize() {
}
}
G1CollectorState* G1CollectorPolicy::collector_state() { return _g1->collector_state(); }
G1YoungGenSizer::G1YoungGenSizer() : _sizer_kind(SizerDefaults), _adaptive_size(true),
_min_desired_young_length(0), _max_desired_young_length(0) {
if (FLAG_IS_CMDLINE(NewRatio)) {
@ -552,7 +543,7 @@ void G1CollectorPolicy::update_young_list_target_length(size_t rs_lengths) {
uint young_list_target_length = 0;
if (adaptive_young_list_length()) {
if (gcs_are_young()) {
if (collector_state()->gcs_are_young()) {
young_list_target_length =
calculate_young_list_target_length(rs_lengths,
base_min_length,
@ -594,7 +585,7 @@ G1CollectorPolicy::calculate_young_list_target_length(size_t rs_lengths,
uint desired_min_length,
uint desired_max_length) {
assert(adaptive_young_list_length(), "pre-condition");
assert(gcs_are_young(), "only call this for young GCs");
assert(collector_state()->gcs_are_young(), "only call this for young GCs");
// In case some edge-condition makes the desired max length too small...
if (desired_max_length <= desired_min_length) {
@ -697,7 +688,7 @@ double G1CollectorPolicy::predict_survivor_regions_evac_time() {
for (HeapRegion * r = _recorded_survivor_head;
r != NULL && r != _recorded_survivor_tail->get_next_young_region();
r = r->get_next_young_region()) {
survivor_regions_evac_time += predict_region_elapsed_time_ms(r, gcs_are_young());
survivor_regions_evac_time += predict_region_elapsed_time_ms(r, collector_state()->gcs_are_young());
}
return survivor_regions_evac_time;
}
@ -782,7 +773,7 @@ void G1CollectorPolicy::record_full_collection_start() {
_full_collection_start_sec = os::elapsedTime();
record_heap_size_info_at_start(true /* full */);
// Release the future to-space so that it is available for compaction into.
_g1->set_full_collection();
collector_state()->set_full_collection(true);
}
void G1CollectorPolicy::record_full_collection_end() {
@ -796,16 +787,16 @@ void G1CollectorPolicy::record_full_collection_end() {
update_recent_gc_times(end_sec, full_gc_time_ms);
_g1->clear_full_collection();
collector_state()->set_full_collection(false);
// "Nuke" the heuristics that control the young/mixed GC
// transitions and make sure we start with young GCs after the Full GC.
set_gcs_are_young(true);
_last_young_gc = false;
clear_initiate_conc_mark_if_possible();
clear_during_initial_mark_pause();
_in_marking_window = false;
_in_marking_window_im = false;
collector_state()->set_gcs_are_young(true);
collector_state()->set_last_young_gc(false);
collector_state()->set_initiate_conc_mark_if_possible(false);
collector_state()->set_during_initial_mark_pause(false);
collector_state()->set_in_marking_window(false);
collector_state()->set_in_marking_window_im(false);
_short_lived_surv_rate_group->start_adding_regions();
// also call this on any additional surv rate groups
@ -845,7 +836,7 @@ void G1CollectorPolicy::record_collection_pause_start(double start_time_sec) {
_collection_set_bytes_used_before = 0;
_bytes_copied_during_gc = 0;
_last_gc_was_young = false;
collector_state()->set_last_gc_was_young(false);
// do that for any other surv rate groups
_short_lived_surv_rate_group->stop_adding_regions();
@ -856,15 +847,15 @@ void G1CollectorPolicy::record_collection_pause_start(double start_time_sec) {
void G1CollectorPolicy::record_concurrent_mark_init_end(double
mark_init_elapsed_time_ms) {
_during_marking = true;
assert(!initiate_conc_mark_if_possible(), "we should have cleared it by now");
clear_during_initial_mark_pause();
collector_state()->set_during_marking(true);
assert(!collector_state()->initiate_conc_mark_if_possible(), "we should have cleared it by now");
collector_state()->set_during_initial_mark_pause(false);
_cur_mark_stop_world_time_ms = mark_init_elapsed_time_ms;
}
void G1CollectorPolicy::record_concurrent_mark_remark_start() {
_mark_remark_start_sec = os::elapsedTime();
_during_marking = false;
collector_state()->set_during_marking(false);
}
void G1CollectorPolicy::record_concurrent_mark_remark_end() {
@ -882,8 +873,8 @@ void G1CollectorPolicy::record_concurrent_mark_cleanup_start() {
}
void G1CollectorPolicy::record_concurrent_mark_cleanup_completed() {
_last_young_gc = true;
_in_marking_window = false;
collector_state()->set_last_young_gc(true);
collector_state()->set_in_marking_window(false);
}
void G1CollectorPolicy::record_concurrent_pause() {
@ -904,7 +895,7 @@ bool G1CollectorPolicy::need_to_start_conc_mark(const char* source, size_t alloc
size_t alloc_byte_size = alloc_word_size * HeapWordSize;
if ((cur_used_bytes + alloc_byte_size) > marking_initiating_used_threshold) {
if (gcs_are_young() && !_last_young_gc) {
if (collector_state()->gcs_are_young() && !collector_state()->last_young_gc()) {
ergo_verbose5(ErgoConcCycles,
"request concurrent cycle initiation",
ergo_format_reason("occupancy higher than threshold")
@ -959,14 +950,14 @@ void G1CollectorPolicy::record_collection_pause_end(double pause_time_ms, Evacua
}
#endif // PRODUCT
last_pause_included_initial_mark = during_initial_mark_pause();
last_pause_included_initial_mark = collector_state()->during_initial_mark_pause();
if (last_pause_included_initial_mark) {
record_concurrent_mark_init_end(0.0);
} else if (need_to_start_conc_mark("end of GC")) {
// Note: this might have already been set, if during the last
// pause we decided to start a cycle but at the beginning of
// this pause we decided to postpone it. That's OK.
set_initiate_conc_mark_if_possible();
collector_state()->set_initiate_conc_mark_if_possible(true);
}
_mmu_tracker->add_pause(end_time_sec - pause_time_ms/1000.0,
@ -1028,37 +1019,37 @@ void G1CollectorPolicy::record_collection_pause_end(double pause_time_ms, Evacua
}
}
bool new_in_marking_window = _in_marking_window;
bool new_in_marking_window = collector_state()->in_marking_window();
bool new_in_marking_window_im = false;
if (last_pause_included_initial_mark) {
new_in_marking_window = true;
new_in_marking_window_im = true;
}
if (_last_young_gc) {
if (collector_state()->last_young_gc()) {
// This is supposed to to be the "last young GC" before we start
// doing mixed GCs. Here we decide whether to start mixed GCs or not.
if (!last_pause_included_initial_mark) {
if (next_gc_should_be_mixed("start mixed GCs",
"do not start mixed GCs")) {
set_gcs_are_young(false);
collector_state()->set_gcs_are_young(false);
}
} else {
ergo_verbose0(ErgoMixedGCs,
"do not start mixed GCs",
ergo_format_reason("concurrent cycle is about to start"));
}
_last_young_gc = false;
collector_state()->set_last_young_gc(false);
}
if (!_last_gc_was_young) {
if (!collector_state()->last_gc_was_young()) {
// This is a mixed GC. Here we decide whether to continue doing
// mixed GCs or not.
if (!next_gc_should_be_mixed("continue mixed GCs",
"do not continue mixed GCs")) {
set_gcs_are_young(true);
collector_state()->set_gcs_are_young(true);
}
}
@ -1077,7 +1068,7 @@ void G1CollectorPolicy::record_collection_pause_end(double pause_time_ms, Evacua
double cost_per_entry_ms = 0.0;
if (cards_scanned > 10) {
cost_per_entry_ms = phase_times()->average_time_ms(G1GCPhaseTimes::ScanRS) / (double) cards_scanned;
if (_last_gc_was_young) {
if (collector_state()->last_gc_was_young()) {
_cost_per_entry_ms_seq->add(cost_per_entry_ms);
} else {
_mixed_cost_per_entry_ms_seq->add(cost_per_entry_ms);
@ -1087,7 +1078,7 @@ void G1CollectorPolicy::record_collection_pause_end(double pause_time_ms, Evacua
if (_max_rs_lengths > 0) {
double cards_per_entry_ratio =
(double) cards_scanned / (double) _max_rs_lengths;
if (_last_gc_was_young) {
if (collector_state()->last_gc_was_young()) {
_young_cards_per_entry_ratio_seq->add(cards_per_entry_ratio);
} else {
_mixed_cards_per_entry_ratio_seq->add(cards_per_entry_ratio);
@ -1119,7 +1110,7 @@ void G1CollectorPolicy::record_collection_pause_end(double pause_time_ms, Evacua
if (copied_bytes > 0) {
cost_per_byte_ms = phase_times()->average_time_ms(G1GCPhaseTimes::ObjCopy) / (double) copied_bytes;
if (_in_marking_window) {
if (collector_state()->in_marking_window()) {
_cost_per_byte_ms_during_cm_seq->add(cost_per_byte_ms);
} else {
_cost_per_byte_ms_seq->add(cost_per_byte_ms);
@ -1162,8 +1153,8 @@ void G1CollectorPolicy::record_collection_pause_end(double pause_time_ms, Evacua
_rs_lengths_seq->add((double) _max_rs_lengths);
}
_in_marking_window = new_in_marking_window;
_in_marking_window_im = new_in_marking_window_im;
collector_state()->set_in_marking_window(new_in_marking_window);
collector_state()->set_in_marking_window_im(new_in_marking_window_im);
_free_regions_at_end_of_collection = _g1->num_free_regions();
update_young_list_target_length();
@ -1301,7 +1292,7 @@ double
G1CollectorPolicy::predict_base_elapsed_time_ms(size_t pending_cards) {
size_t rs_length = predict_rs_length_diff();
size_t card_num;
if (gcs_are_young()) {
if (collector_state()->gcs_are_young()) {
card_num = predict_young_card_num(rs_length);
} else {
card_num = predict_non_young_card_num(rs_length);
@ -1467,7 +1458,7 @@ bool G1CollectorPolicy::force_initial_mark_if_outside_cycle(
ergo_format_reason("requested by GC cause")
ergo_format_str("GC cause"),
GCCause::to_string(gc_cause));
set_initiate_conc_mark_if_possible();
collector_state()->set_initiate_conc_mark_if_possible(true);
return true;
} else {
ergo_verbose1(ErgoConcCycles,
@ -1484,13 +1475,13 @@ G1CollectorPolicy::decide_on_conc_mark_initiation() {
// We are about to decide on whether this pause will be an
// initial-mark pause.
// First, during_initial_mark_pause() should not be already set. We
// First, collector_state()->during_initial_mark_pause() should not be already set. We
// will set it here if we have to. However, it should be cleared by
// the end of the pause (it's only set for the duration of an
// initial-mark pause).
assert(!during_initial_mark_pause(), "pre-condition");
assert(!collector_state()->during_initial_mark_pause(), "pre-condition");
if (initiate_conc_mark_if_possible()) {
if (collector_state()->initiate_conc_mark_if_possible()) {
// We had noticed on a previous pause that the heap occupancy has
// gone over the initiating threshold and we should start a
// concurrent marking cycle. So we might initiate one.
@ -1501,10 +1492,10 @@ G1CollectorPolicy::decide_on_conc_mark_initiation() {
// it has completed the last one. So we can go ahead and
// initiate a new cycle.
set_during_initial_mark_pause();
collector_state()->set_during_initial_mark_pause(true);
// We do not allow mixed GCs during marking.
if (!gcs_are_young()) {
set_gcs_are_young(true);
if (!collector_state()->gcs_are_young()) {
collector_state()->set_gcs_are_young(true);
ergo_verbose0(ErgoMixedGCs,
"end mixed GCs",
ergo_format_reason("concurrent cycle is about to start"));
@ -1512,7 +1503,7 @@ G1CollectorPolicy::decide_on_conc_mark_initiation() {
// And we can now clear initiate_conc_mark_if_possible() as
// we've already acted on it.
clear_initiate_conc_mark_if_possible();
collector_state()->set_initiate_conc_mark_if_possible(false);
ergo_verbose0(ErgoConcCycles,
"initiate concurrent cycle",
@ -1686,7 +1677,7 @@ void G1CollectorPolicy::add_to_incremental_cset_info(HeapRegion* hr, size_t rs_l
// retiring the current allocation region) or a concurrent
// refine thread (RSet sampling).
double region_elapsed_time_ms = predict_region_elapsed_time_ms(hr, gcs_are_young());
double region_elapsed_time_ms = predict_region_elapsed_time_ms(hr, collector_state()->gcs_are_young());
size_t used_bytes = hr->used();
_inc_cset_recorded_rs_lengths += rs_length;
_inc_cset_predicted_elapsed_time_ms += region_elapsed_time_ms;
@ -1721,7 +1712,7 @@ void G1CollectorPolicy::update_incremental_cset_info(HeapRegion* hr,
_inc_cset_recorded_rs_lengths_diffs += rs_lengths_diff;
double old_elapsed_time_ms = hr->predicted_elapsed_time_ms();
double new_region_elapsed_time_ms = predict_region_elapsed_time_ms(hr, gcs_are_young());
double new_region_elapsed_time_ms = predict_region_elapsed_time_ms(hr, collector_state()->gcs_are_young());
double elapsed_ms_diff = new_region_elapsed_time_ms - old_elapsed_time_ms;
_inc_cset_predicted_elapsed_time_ms_diffs += elapsed_ms_diff;
@ -1914,9 +1905,9 @@ void G1CollectorPolicy::finalize_cset(double target_pause_time_ms, EvacuationInf
ergo_format_ms("target pause time"),
_pending_cards, base_time_ms, time_remaining_ms, target_pause_time_ms);
_last_gc_was_young = gcs_are_young() ? true : false;
collector_state()->set_last_gc_was_young(collector_state()->gcs_are_young());
if (_last_gc_was_young) {
if (collector_state()->last_gc_was_young()) {
_trace_young_gen_time_data.increment_young_collection_count();
} else {
_trace_young_gen_time_data.increment_mixed_collection_count();
@ -1967,7 +1958,7 @@ void G1CollectorPolicy::finalize_cset(double target_pause_time_ms, EvacuationInf
// Set the start of the non-young choice time.
double non_young_start_time_sec = young_end_time_sec;
if (!gcs_are_young()) {
if (!collector_state()->gcs_are_young()) {
CollectionSetChooser* cset_chooser = _collectionSetChooser;
cset_chooser->verify();
const uint min_old_cset_length = calc_min_old_cset_length();
@ -2013,7 +2004,7 @@ void G1CollectorPolicy::finalize_cset(double target_pause_time_ms, EvacuationInf
break;
}
double predicted_time_ms = predict_region_elapsed_time_ms(hr, gcs_are_young());
double predicted_time_ms = predict_region_elapsed_time_ms(hr, collector_state()->gcs_are_young());
if (check_time_remaining) {
if (predicted_time_ms > time_remaining_ms) {
// Too expensive for the current CSet.

View File

@ -27,6 +27,7 @@
#include "gc/g1/collectionSetChooser.hpp"
#include "gc/g1/g1Allocator.hpp"
#include "gc/g1/g1CollectorState.hpp"
#include "gc/g1/g1MMUTracker.hpp"
#include "gc/shared/collectorPolicy.hpp"
@ -193,9 +194,6 @@ private:
double _stop_world_start;
// indicates whether we are in young or mixed GC mode
bool _gcs_are_young;
uint _young_list_target_length;
uint _young_list_fixed_length;
@ -203,12 +201,6 @@ private:
// locker is active. This should be >= _young_list_target_length;
uint _young_list_max_length;
bool _last_gc_was_young;
bool _during_marking;
bool _in_marking_window;
bool _in_marking_window_im;
SurvRateGroup* _short_lived_surv_rate_group;
SurvRateGroup* _survivor_surv_rate_group;
// add here any more surv rate groups
@ -218,10 +210,6 @@ private:
double _reserve_factor;
uint _reserve_regions;
bool during_marking() {
return _during_marking;
}
enum PredictionConstants {
TruncatedSeqLength = 10
};
@ -363,7 +351,7 @@ public:
}
double predict_rs_scan_time_ms(size_t card_num) {
if (gcs_are_young()) {
if (collector_state()->gcs_are_young()) {
return (double) card_num * get_new_prediction(_cost_per_entry_ms_seq);
} else {
return predict_mixed_rs_scan_time_ms(card_num);
@ -390,7 +378,7 @@ public:
}
double predict_object_copy_time_ms(size_t bytes_to_copy) {
if (_in_marking_window && !_in_marking_window_im) {
if (collector_state()->during_concurrent_mark()) {
return predict_object_copy_time_ms_during_cm(bytes_to_copy);
} else {
return (double) bytes_to_copy *
@ -428,7 +416,7 @@ public:
double predict_survivor_regions_evac_time();
void cset_regions_freed() {
bool propagate = _last_gc_was_young && !_in_marking_window;
bool propagate = collector_state()->should_propagate();
_short_lived_surv_rate_group->all_surviving_words_recorded(propagate);
_survivor_surv_rate_group->all_surviving_words_recorded(propagate);
// also call it on any more surv rate groups
@ -552,33 +540,6 @@ private:
return _recent_avg_pause_time_ratio;
}
// At the end of a pause we check the heap occupancy and we decide
// whether we will start a marking cycle during the next pause. If
// we decide that we want to do that, we will set this parameter to
// true. So, this parameter will stay true between the end of a
// pause and the beginning of a subsequent pause (not necessarily
// the next one, see the comments on the next field) when we decide
// that we will indeed start a marking cycle and do the initial-mark
// work.
volatile bool _initiate_conc_mark_if_possible;
// If initiate_conc_mark_if_possible() is set at the beginning of a
// pause, it is a suggestion that the pause should start a marking
// cycle by doing the initial-mark work. However, it is possible
// that the concurrent marking thread is still finishing up the
// previous marking cycle (e.g., clearing the next marking
// bitmap). If that is the case we cannot start a new cycle and
// we'll have to wait for the concurrent marking thread to finish
// what it is doing. In this case we will postpone the marking cycle
// initiation decision for the next pause. When we eventually decide
// to start a cycle, we will set _during_initial_mark_pause which
// will stay true until the end of the initial-mark pause and it's
// the condition that indicates that a pause is doing the
// initial-mark work.
volatile bool _during_initial_mark_pause;
bool _last_young_gc;
// This set of variables tracks the collector efficiency, in order to
// determine whether we should initiate a new marking.
double _cur_mark_stop_world_time_ms;
@ -647,6 +608,8 @@ public:
return CollectorPolicy::G1CollectorPolicyKind;
}
G1CollectorState* collector_state();
G1GCPhaseTimes* phase_times() const { return _phase_times; }
// Check the current value of the young list RSet lengths and
@ -786,14 +749,6 @@ public:
void print_collection_set(HeapRegion* list_head, outputStream* st);
#endif // !PRODUCT
bool initiate_conc_mark_if_possible() { return _initiate_conc_mark_if_possible; }
void set_initiate_conc_mark_if_possible() { _initiate_conc_mark_if_possible = true; }
void clear_initiate_conc_mark_if_possible() { _initiate_conc_mark_if_possible = false; }
bool during_initial_mark_pause() { return _during_initial_mark_pause; }
void set_during_initial_mark_pause() { _during_initial_mark_pause = true; }
void clear_during_initial_mark_pause(){ _during_initial_mark_pause = false; }
// This sets the initiate_conc_mark_if_possible() flag to start a
// new cycle, as long as we are not already in one. It's best if it
// is called during a safepoint when the test whether a cycle is in
@ -837,13 +792,6 @@ public:
return _young_list_max_length;
}
bool gcs_are_young() {
return _gcs_are_young;
}
void set_gcs_are_young(bool gcs_are_young) {
_gcs_are_young = gcs_are_young;
}
bool adaptive_young_list_length() {
return _young_gen_sizer->adaptive_young_list_length();
}

View File

@ -0,0 +1,141 @@
/*
* Copyright (c) 2015, 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
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*
* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
* or visit www.oracle.com if you need additional information or have any
* questions.
*
*/
#ifndef SHARE_VM_GC_G1_G1COLLECTORSTATE_HPP
#define SHARE_VM_GC_G1_G1COLLECTORSTATE_HPP
#include "utilities/globalDefinitions.hpp"
#include "gc/g1/g1YCTypes.hpp"
// Various state variables that indicate
// the phase of the G1 collection.
class G1CollectorState VALUE_OBJ_CLASS_SPEC {
// Indicates whether we are in "full young" or "mixed" GC mode.
bool _gcs_are_young;
// Was the last GC "young"?
bool _last_gc_was_young;
// Is this the "last young GC" before we start doing mixed GCs?
// Set after a concurrent mark has completed.
bool _last_young_gc;
// If initiate_conc_mark_if_possible() is set at the beginning of a
// pause, it is a suggestion that the pause should start a marking
// cycle by doing the initial-mark work. However, it is possible
// that the concurrent marking thread is still finishing up the
// previous marking cycle (e.g., clearing the next marking
// bitmap). If that is the case we cannot start a new cycle and
// we'll have to wait for the concurrent marking thread to finish
// what it is doing. In this case we will postpone the marking cycle
// initiation decision for the next pause. When we eventually decide
// to start a cycle, we will set _during_initial_mark_pause which
// will stay true until the end of the initial-mark pause and it's
// the condition that indicates that a pause is doing the
// initial-mark work.
volatile bool _during_initial_mark_pause;
// At the end of a pause we check the heap occupancy and we decide
// whether we will start a marking cycle during the next pause. If
// we decide that we want to do that, we will set this parameter to
// true. So, this parameter will stay true between the end of a
// pause and the beginning of a subsequent pause (not necessarily
// the next one, see the comments on the next field) when we decide
// that we will indeed start a marking cycle and do the initial-mark
// work.
volatile bool _initiate_conc_mark_if_possible;
// NOTE: if some of these are synonyms for others,
// the redundant fields should be eliminated. XXX
bool _during_marking;
bool _mark_in_progress;
bool _in_marking_window;
bool _in_marking_window_im;
bool _concurrent_cycle_started;
bool _full_collection;
public:
G1CollectorState() :
_gcs_are_young(true),
_last_gc_was_young(false),
_last_young_gc(false),
_during_initial_mark_pause(false),
_initiate_conc_mark_if_possible(false),
_during_marking(false),
_mark_in_progress(false),
_in_marking_window(false),
_in_marking_window_im(false),
_concurrent_cycle_started(false),
_full_collection(false) {}
// Setters
void set_gcs_are_young(bool v) { _gcs_are_young = v; }
void set_last_gc_was_young(bool v) { _last_gc_was_young = v; }
void set_last_young_gc(bool v) { _last_young_gc = v; }
void set_during_initial_mark_pause(bool v) { _during_initial_mark_pause = v; }
void set_initiate_conc_mark_if_possible(bool v) { _initiate_conc_mark_if_possible = v; }
void set_during_marking(bool v) { _during_marking = v; }
void set_mark_in_progress(bool v) { _mark_in_progress = v; }
void set_in_marking_window(bool v) { _in_marking_window = v; }
void set_in_marking_window_im(bool v) { _in_marking_window_im = v; }
void set_concurrent_cycle_started(bool v) { _concurrent_cycle_started = v; }
void set_full_collection(bool v) { _full_collection = v; }
// Getters
bool gcs_are_young() { return _gcs_are_young; }
bool last_gc_was_young() { return _last_gc_was_young; }
bool last_young_gc() { return _last_young_gc; }
bool during_initial_mark_pause() { return _during_initial_mark_pause; }
bool initiate_conc_mark_if_possible() { return _initiate_conc_mark_if_possible; }
bool during_marking() { return _during_marking; }
bool mark_in_progress() { return _mark_in_progress; }
bool in_marking_window() { return _in_marking_window; }
bool in_marking_window_im() { return _in_marking_window_im; }
bool concurrent_cycle_started() { return _concurrent_cycle_started; }
bool full_collection() { return _full_collection; }
// Composite booleans (clients worry about flickering)
bool during_concurrent_mark() {
return (_in_marking_window && !_in_marking_window_im);
}
bool should_propagate() { // XXX should have a more suitable state name or abstraction for this
return (_last_young_gc && !_in_marking_window);
}
G1YCType yc_type() {
if (during_initial_mark_pause()) {
return InitialMark;
} else if (mark_in_progress()) {
return DuringMark;
} else if (gcs_are_young()) {
return Normal;
} else {
return Mixed;
}
}
};
#endif /* SHARE_VM_GC_G1_G1COLLECTORSTATE_HPP */

View File

@ -26,6 +26,7 @@
#include "gc/g1/concurrentMark.inline.hpp"
#include "gc/g1/dirtyCardQueue.hpp"
#include "gc/g1/g1CollectedHeap.inline.hpp"
#include "gc/g1/g1CollectorState.hpp"
#include "gc/g1/g1EvacFailure.hpp"
#include "gc/g1/g1OopClosures.inline.hpp"
#include "gc/g1/g1_globals.hpp"
@ -186,8 +187,8 @@ public:
}
bool doHeapRegion(HeapRegion *hr) {
bool during_initial_mark = _g1h->g1_policy()->during_initial_mark_pause();
bool during_conc_mark = _g1h->mark_in_progress();
bool during_initial_mark = _g1h->collector_state()->during_initial_mark_pause();
bool during_conc_mark = _g1h->collector_state()->mark_in_progress();
assert(!hr->is_humongous(), "sanity");
assert(hr->in_collection_set(), "bad CS");

View File

@ -627,7 +627,7 @@ void G1RemSet::print_summary_info(G1RemSetSummary * summary, const char * header
void G1RemSet::prepare_for_verify() {
if (G1HRRSFlushLogBuffersOnVerify &&
(VerifyBeforeGC || VerifyAfterGC)
&& (!_g1->full_collection() || G1VerifyRSetsDuringFullGC)) {
&& (!_g1->collector_state()->full_collection() || G1VerifyRSetsDuringFullGC)) {
cleanupHRRS();
_g1->set_refine_cte_cl_concurrency(false);
if (SafepointSynchronize::is_at_safepoint()) {

View File

@ -30,6 +30,7 @@
#include "gc/g1/bufferingOopClosure.hpp"
#include "gc/g1/g1CollectedHeap.inline.hpp"
#include "gc/g1/g1CollectorPolicy.hpp"
#include "gc/g1/g1CollectorState.hpp"
#include "gc/g1/g1GCPhaseTimes.hpp"
#include "gc/g1/g1RemSet.inline.hpp"
#include "gc/g1/g1RootProcessor.hpp"
@ -199,7 +200,7 @@ void G1RootProcessor::evacuate_roots(OopClosure* scan_non_heap_roots,
// as implicitly live).
{
G1GCParPhaseTimesTracker x(phase_times, G1GCPhaseTimes::SATBFiltering, worker_i);
if (!_process_strong_tasks->is_task_claimed(G1RP_PS_filter_satb_buffers) && _g1h->mark_in_progress()) {
if (!_process_strong_tasks->is_task_claimed(G1RP_PS_filter_satb_buffers) && _g1h->collector_state()->mark_in_progress()) {
JavaThread::satb_mark_queue_set().filter_thread_buffers();
}
}

View File

@ -711,7 +711,7 @@ public:
_n_failures++;
}
if (!_g1h->full_collection() || G1VerifyRSetsDuringFullGC) {
if (!_g1h->collector_state()->full_collection() || G1VerifyRSetsDuringFullGC) {
HeapRegion* from = _g1h->heap_region_containing((HeapWord*)p);
HeapRegion* to = _g1h->heap_region_containing(obj);
if (from != NULL && to != NULL &&

View File

@ -197,7 +197,7 @@ bool VM_CollectForMetadataAllocation::initiate_concurrent_GC() {
if (UseG1GC && ClassUnloadingWithConcurrentMark) {
G1CollectedHeap* g1h = G1CollectedHeap::heap();
g1h->g1_policy()->set_initiate_conc_mark_if_possible();
g1h->g1_policy()->collector_state()->set_initiate_conc_mark_if_possible(true);
GCCauseSetter x(g1h, _gc_cause);