--- old/src/share/vm/gc_implementation/g1/concurrentMark.cpp 2011-06-23 14:02:49.322423701 -0700 +++ new/src/share/vm/gc_implementation/g1/concurrentMark.cpp 2011-06-23 14:02:49.108372683 -0700 @@ -839,8 +839,9 @@ NoteStartOfMarkHRClosure startcl; g1h->heap_region_iterate(&startcl); - // Start weak-reference discovery. - ReferenceProcessor* rp = g1h->ref_processor(); + // Start Concurrent Marking weak-reference discovery. + ReferenceProcessor* rp = g1h->ref_processor_cm(); + assert(!rp->discovery_enabled(), "Precondition"); rp->verify_no_references_recorded(); rp->enable_discovery(); // enable ("weak") refs discovery rp->setup_policy(false); // snapshot the soft ref policy to be used in this cycle @@ -1186,6 +1187,7 @@ // world is stopped at this checkpoint assert(SafepointSynchronize::is_at_safepoint(), "world should be stopped"); + G1CollectedHeap* g1h = G1CollectedHeap::heap(); // If a full collection has happened, we shouldn't do this. @@ -1867,6 +1869,10 @@ size_t cleaned_up_bytes = start_used_bytes - g1h->used(); g1p->decrease_known_garbage_bytes(cleaned_up_bytes); + // Clean up will have freed any regions completely full of garbage. + // Update the soft reference policy with the new heap occupancy. + Universe::update_heap_info_at_gc(); + // We need to make this be a "collection" so any collection pause that // races with it goes around and waits for completeCleanup to finish. g1h->increment_total_collections(); @@ -2097,8 +2103,10 @@ } }; -// Implementation of AbstractRefProcTaskExecutor for G1 -class G1RefProcTaskExecutor: public AbstractRefProcTaskExecutor { +// Implementation of AbstractRefProcTaskExecutor for parallel +// reference processing at the end of G1 concurrent marking + +class G1CMRefProcTaskExecutor: public AbstractRefProcTaskExecutor { private: G1CollectedHeap* _g1h; ConcurrentMark* _cm; @@ -2107,7 +2115,7 @@ int _active_workers; public: - G1RefProcTaskExecutor(G1CollectedHeap* g1h, + G1CMRefProcTaskExecutor(G1CollectedHeap* g1h, ConcurrentMark* cm, CMBitMap* bitmap, WorkGang* workers, @@ -2121,7 +2129,7 @@ virtual void execute(EnqueueTask& task); }; -class G1RefProcTaskProxy: public AbstractGangTask { +class G1CMRefProcTaskProxy: public AbstractGangTask { typedef AbstractRefProcTaskExecutor::ProcessTask ProcessTask; ProcessTask& _proc_task; G1CollectedHeap* _g1h; @@ -2129,7 +2137,7 @@ CMBitMap* _bitmap; public: - G1RefProcTaskProxy(ProcessTask& proc_task, + G1CMRefProcTaskProxy(ProcessTask& proc_task, G1CollectedHeap* g1h, ConcurrentMark* cm, CMBitMap* bitmap) : @@ -2147,10 +2155,10 @@ } }; -void G1RefProcTaskExecutor::execute(ProcessTask& proc_task) { +void G1CMRefProcTaskExecutor::execute(ProcessTask& proc_task) { assert(_workers != NULL, "Need parallel worker threads."); - G1RefProcTaskProxy proc_task_proxy(proc_task, _g1h, _cm, _bitmap); + G1CMRefProcTaskProxy proc_task_proxy(proc_task, _g1h, _cm, _bitmap); // We need to reset the phase for each task execution so that // the termination protocol of CMTask::do_marking_step works. @@ -2160,12 +2168,12 @@ _g1h->set_par_threads(0); } -class G1RefEnqueueTaskProxy: public AbstractGangTask { +class G1CMRefEnqueueTaskProxy: public AbstractGangTask { typedef AbstractRefProcTaskExecutor::EnqueueTask EnqueueTask; EnqueueTask& _enq_task; public: - G1RefEnqueueTaskProxy(EnqueueTask& enq_task) : + G1CMRefEnqueueTaskProxy(EnqueueTask& enq_task) : AbstractGangTask("Enqueue reference objects in parallel"), _enq_task(enq_task) { } @@ -2175,10 +2183,10 @@ } }; -void G1RefProcTaskExecutor::execute(EnqueueTask& enq_task) { +void G1CMRefProcTaskExecutor::execute(EnqueueTask& enq_task) { assert(_workers != NULL, "Need parallel worker threads."); - G1RefEnqueueTaskProxy enq_task_proxy(enq_task); + G1CMRefEnqueueTaskProxy enq_task_proxy(enq_task); _g1h->set_par_threads(_active_workers); _workers->run_task(&enq_task_proxy); @@ -2189,7 +2197,7 @@ ResourceMark rm; HandleMark hm; G1CollectedHeap* g1h = G1CollectedHeap::heap(); - ReferenceProcessor* rp = g1h->ref_processor(); + ReferenceProcessor* rp = g1h->ref_processor_cm(); // See the comment in G1CollectedHeap::ref_processing_init() // about how reference processing currently works in G1. @@ -2207,7 +2215,7 @@ int active_workers = g1h->workers() ? g1h->workers()->total_workers() : 1; active_workers = MAX2(MIN2(active_workers, (int)_max_task_num), 1); - G1RefProcTaskExecutor par_task_executor(g1h, this, nextMarkBitMap(), + G1CMRefProcTaskExecutor par_task_executor(g1h, this, nextMarkBitMap(), g1h->workers(), active_workers); @@ -2251,7 +2259,7 @@ } rp->verify_no_references_recorded(); - assert(!rp->discovery_enabled(), "should have been disabled"); + assert(!rp->discovery_enabled(), "Post condition"); // Now clean up stale oops in StringTable StringTable::unlink(&g1_is_alive); @@ -3340,7 +3348,7 @@ assert(_ref_processor == NULL, "should be initialized to NULL"); if (G1UseConcMarkReferenceProcessing) { - _ref_processor = g1h->ref_processor(); + _ref_processor = g1h->ref_processor_cm(); assert(_ref_processor != NULL, "should not be NULL"); } } --- old/src/share/vm/gc_implementation/g1/concurrentMark.hpp 2011-06-23 14:02:52.688897796 -0700 +++ new/src/share/vm/gc_implementation/g1/concurrentMark.hpp 2011-06-23 14:02:52.471012696 -0700 @@ -366,8 +366,8 @@ friend class CMConcurrentMarkingTask; friend class G1ParNoteEndTask; friend class CalcLiveObjectsClosure; - friend class G1RefProcTaskProxy; - friend class G1RefProcTaskExecutor; + friend class G1CMRefProcTaskProxy; + friend class G1CMRefProcTaskExecutor; friend class G1CMParKeepAliveAndDrainClosure; friend class G1CMParDrainMarkingStackClosure; --- old/src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp 2011-06-23 14:02:55.056369139 -0700 +++ new/src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp 2011-06-23 14:02:54.834494052 -0700 @@ -1225,9 +1225,12 @@ // generation does some collection work, or // instanceRefKlass::enqueue_discovered_references if the // generation returns without doing any work. - ref_processor()->disable_discovery(); - ref_processor()->abandon_partial_discovery(); - ref_processor()->verify_no_references_recorded(); + + // Disable discovery and empty the discovered lists + // for the CM ref processor. + ref_processor_cm()->disable_discovery(); + ref_processor_cm()->abandon_partial_discovery(); + ref_processor_cm()->verify_no_references_recorded(); // Abandon current iterations of concurrent marking and concurrent // refinement, if any are in progress. @@ -1252,31 +1255,36 @@ g1_policy()->set_full_young_gcs(true); } - // See the comment in G1CollectedHeap::ref_processing_init() about + // See the comments in g1CollectedHeap.hpp and + // G1CollectedHeap::ref_processing_init() about // how reference processing currently works in G1. - // Temporarily make reference _discovery_ single threaded (non-MT). - ReferenceProcessorMTDiscoveryMutator rp_disc_ser(ref_processor(), false); + assert(!ref_processor_stw()->discovery_enabled(), "Precondition"); + ref_processor_stw()->verify_no_references_recorded(); + + // Temporarily make discovery by the STW ref processor single threaded (non-MT). + ReferenceProcessorMTDiscoveryMutator stw_rp_disc_ser(ref_processor_stw(), false); - // Temporarily make refs discovery atomic - ReferenceProcessorAtomicMutator rp_disc_atomic(ref_processor(), true); + // Temporarily clear the STW ref processor's _is_alive_non_header field. + ReferenceProcessorIsAliveMutator stw_rp_is_alive_null(ref_processor_stw(), NULL); - // Temporarily clear _is_alive_non_header - ReferenceProcessorIsAliveMutator rp_is_alive_null(ref_processor(), NULL); + ref_processor_stw()->enable_discovery(); + ref_processor_stw()->setup_policy(do_clear_all_soft_refs); - ref_processor()->enable_discovery(); - ref_processor()->setup_policy(do_clear_all_soft_refs); // Do collection work { HandleMark hm; // Discard invalid handles created during gc - G1MarkSweep::invoke_at_safepoint(ref_processor(), do_clear_all_soft_refs); + G1MarkSweep::invoke_at_safepoint(ref_processor_stw(), do_clear_all_soft_refs); } + assert(free_regions() == 0, "we should not have added any free regions"); rebuild_region_lists(); _summary_bytes_used = recalculate_used(); - ref_processor()->enqueue_discovered_references(); + // Enqueue any discovered reference objects that have + // not been removed from the discovered lists. + ref_processor_stw()->enqueue_discovered_references(); COMPILER2_PRESENT(DerivedPointerTable::update_pointers()); @@ -1291,7 +1299,16 @@ /* option */ VerifyOption_G1UsePrevMarking); } - NOT_PRODUCT(ref_processor()->verify_no_references_recorded()); + + assert(!ref_processor_stw()->discovery_enabled(), "Postcondition"); + ref_processor_stw()->verify_no_references_recorded(); + + // Note: since we've just done a full GC, concurrent + // marking is no longer active. Therefore we need not + // re-enable reference discovery for the CM ref processor. + // That will be done at the start of the next marking cycle. + assert(!ref_processor_cm()->discovery_enabled(), "Postcondition"); + ref_processor_cm()->verify_no_references_recorded(); reset_gc_time_stamp(); // Since everything potentially moved, we will clear all remembered @@ -1732,8 +1749,10 @@ _g1_policy(policy_), _dirty_card_queue_set(false), _into_cset_dirty_card_queue_set(false), - _is_alive_closure(this), - _ref_processor(NULL), + _is_alive_closure_cm(this), + _is_alive_closure_stw(this), + _ref_processor_cm(NULL), + _ref_processor_stw(NULL), _process_strong_tasks(new SubTasksDone(G1H_PS_NumElements)), _bot_shared(NULL), _objs_with_preserved_marks(NULL), _preserved_marks_of_objs(NULL), @@ -2020,34 +2039,79 @@ void G1CollectedHeap::ref_processing_init() { // Reference processing in G1 currently works as follows: // - // * There is only one reference processor instance that - // 'spans' the entire heap. It is created by the code - // below. - // * Reference discovery is not enabled during an incremental - // pause (see 6484982). - // * Discoverered refs are not enqueued nor are they processed - // during an incremental pause (see 6484982). - // * Reference discovery is enabled at initial marking. - // * Reference discovery is disabled and the discovered - // references processed etc during remarking. - // * Reference discovery is MT (see below). - // * Reference discovery requires a barrier (see below). - // * Reference processing is currently not MT (see 6608385). - // * A full GC enables (non-MT) reference discovery and - // processes any discovered references. + // * There are two reference processor instances. One is + // used to record and process discovered references + // during concurrent marking; the other is used to + // record and process references during STW pauses + // (both full and incremental). + // * Both ref processors need to 'span' the entire heap as + // the regions in the collection set may be dotted around. + // + // * For the concurrent marking ref processor: + // * Reference discovery is enabled at initial marking. + // * Reference discovery is disabled and the discovered + // references processed etc during remarking. + // * Reference discovery is MT (see below). + // * Reference discovery requires a barrier (see below). + // * Reference processing may or may not be MT + // (depending on the value of ParallelRefProcEnabled + // and ParallelGCThreads). + // * A full GC disables reference discovery by the CM + // ref processor and abandons any entries on it's + // discovered lists. + // + // * For the STW processor: + // * Non MT discovery is enabled at the start of a full GC. + // * Processing and enqueueing during a full GC is non-MT. + // * During a full GC, references are processed after marking. + // + // * Discovery (may or may not be MT) is enabled at the start + // of an incremental evacuation pause. + // * References are processed near the end of a STW evacuation pause. + // * For both types of GC: + // * Discovery is atomic - i.e. not concurrent. + // * Reference discovery will not need a barrier. SharedHeap::ref_processing_init(); MemRegion mr = reserved_region(); - _ref_processor = + + // Concurrent Mark ref processor + _ref_processor_cm = new ReferenceProcessor(mr, // span - ParallelRefProcEnabled && (ParallelGCThreads > 1), // mt processing - (int) ParallelGCThreads, // degree of mt processing - ParallelGCThreads > 1 || ConcGCThreads > 1, // mt discovery - (int) MAX2(ParallelGCThreads, ConcGCThreads), // degree of mt discovery - false, // Reference discovery is not atomic - &_is_alive_closure, // is alive closure for efficiency - true); // Setting next fields of discovered - // lists requires a barrier. + ParallelRefProcEnabled && (ParallelGCThreads > 1), + // mt processing + (int) ParallelGCThreads, + // degree of mt processing + (ParallelGCThreads > 1) || (ConcGCThreads > 1), + // mt discovery + (int) MAX2(ParallelGCThreads, ConcGCThreads), + // degree of mt discovery + false, + // Reference discovery is not atomic + &_is_alive_closure_cm, + // is alive closure for efficiency + true); + // Setting next fields of discovered + // lists requires a barrier. + + // STW ref processor + _ref_processor_stw = + new ReferenceProcessor(mr, // span + ParallelRefProcEnabled && (ParallelGCThreads > 1), + // mt processing + MAX2((int)ParallelGCThreads, 1), + // degree of mt processing + (ParallelGCThreads > 1), + // mt discovery + MAX2((int)ParallelGCThreads, 1), + // degree of mt discovery + true, + // Reference discovery is atomic + &_is_alive_closure_stw, + // is alive closure for efficiency + false); + // Setting next fields of discovered + // lists requires a barrier. } size_t G1CollectedHeap::capacity() const { @@ -3046,6 +3110,10 @@ COMPILER2_PRESENT(assert(DerivedPointerTable::is_empty(), "derived pointer present")); // always_do_update_barrier = true; + + // We have just completed a GC. Update the soft reference + // policy with the new heap occupancy + Universe::update_heap_info_at_gc(); } HeapWord* G1CollectedHeap::do_collection_pause(size_t word_size, @@ -3283,13 +3351,24 @@ COMPILER2_PRESENT(DerivedPointerTable::clear()); - // Please see comment in G1CollectedHeap::ref_processing_init() - // to see how reference processing currently works in G1. - // - // We want to turn off ref discovery, if necessary, and turn it back on - // on again later if we do. XXX Dubious: why is discovery disabled? - bool was_enabled = ref_processor()->discovery_enabled(); - if (was_enabled) ref_processor()->disable_discovery(); + // Please see comment in g1CollectedHeap.hpp and + // G1CollectedHeap::ref_processing_init() to see how + // reference processing currently works in G1. + + assert(!ref_processor_stw()->discovery_enabled(), "Precondition"); + ref_processor_stw()->verify_no_references_recorded(); + + // Enable discovery in the STW reference processor + ref_processor_stw()->enable_discovery(); + + // We want to temporarily turn off discovery by the + // CM ref processor, if necessary, and turn it back on + // on again later if we do. + // We could use a NoRefDiscovery scoped object here. + bool cm_ref_disc_enabled = ref_processor_cm()->discovery_enabled(); + if (cm_ref_disc_enabled) { + ref_processor_cm()->disable_discovery(); + } // Forget the current alloc region (we might even choose it to be part // of the collection set!). @@ -3455,7 +3534,13 @@ /* option */ VerifyOption_G1UsePrevMarking); } - if (was_enabled) ref_processor()->enable_discovery(); + assert(!ref_processor_stw()->discovery_enabled(), "Postcondition"); + ref_processor_stw()->verify_no_references_recorded(); + + // Re-enable CM reference discovery if necessary. + if (cm_ref_disc_enabled) { + ref_processor_cm()->enable_discovery(); + } { size_t expand_bytes = g1_policy()->expansion_amount(); @@ -3801,54 +3886,6 @@ _evac_failure_scan_stack = NULL; } - - -// *** Sequential G1 Evacuation - -class G1IsAliveClosure: public BoolObjectClosure { - G1CollectedHeap* _g1; -public: - G1IsAliveClosure(G1CollectedHeap* g1) : _g1(g1) {} - void do_object(oop p) { assert(false, "Do not call."); } - bool do_object_b(oop p) { - // It is reachable if it is outside the collection set, or is inside - // and forwarded. - -#ifdef G1_DEBUG - gclog_or_tty->print_cr("is alive "PTR_FORMAT" in CS %d forwarded %d overall %d", - (void*) p, _g1->obj_in_cs(p), p->is_forwarded(), - !_g1->obj_in_cs(p) || p->is_forwarded()); -#endif // G1_DEBUG - - return !_g1->obj_in_cs(p) || p->is_forwarded(); - } -}; - -class G1KeepAliveClosure: public OopClosure { - G1CollectedHeap* _g1; -public: - G1KeepAliveClosure(G1CollectedHeap* g1) : _g1(g1) {} - void do_oop(narrowOop* p) { guarantee(false, "Not needed"); } - void do_oop( oop* p) { - oop obj = *p; -#ifdef G1_DEBUG - if (PrintGC && Verbose) { - gclog_or_tty->print_cr("keep alive *"PTR_FORMAT" = "PTR_FORMAT" "PTR_FORMAT, - p, (void*) obj, (void*) *p); - } -#endif // G1_DEBUG - - if (_g1->obj_in_cs(obj)) { - assert( obj->is_forwarded(), "invariant" ); - *p = obj->forwardee(); -#ifdef G1_DEBUG - gclog_or_tty->print_cr(" in CSet: moved "PTR_FORMAT" -> "PTR_FORMAT, - (void*) obj, (void*) *p); -#endif // G1_DEBUG - } - } -}; - class UpdateRSetDeferred : public OopsInHeapRegionClosure { private: G1CollectedHeap* _g1; @@ -4350,12 +4387,17 @@ #endif // ASSERT void G1ParScanThreadState::trim_queue() { + assert(_evac_cl != NULL, "not set"); + assert(_evac_failure_cl != NULL, "not set"); + assert(_partial_scan_cl != NULL, "not set"); + StarTask ref; do { // Drain the overflow stack first, so other threads can steal. while (refs()->pop_overflow(ref)) { deal_with_reference(ref); } + while (refs()->pop_local(ref)) { deal_with_reference(ref); } @@ -4475,8 +4517,7 @@ template template -void G1ParCopyClosure -::do_oop_work(T* p) { +void G1ParCopyClosure ::do_oop_work(T* p) { oop obj = oopDesc::load_decode_heap_oop(p); assert(barrier != G1BarrierRS || obj != NULL, "Precondition: G1BarrierRS implies obj is nonNull"); @@ -4640,36 +4681,43 @@ ResourceMark rm; HandleMark hm; + + ReferenceProcessor* rp = _g1h->ref_processor_stw(); G1ParScanThreadState pss(_g1h, i); - G1ParScanHeapEvacClosure scan_evac_cl(_g1h, &pss); - G1ParScanHeapEvacFailureClosure evac_failure_cl(_g1h, &pss); - G1ParScanPartialArrayClosure partial_scan_cl(_g1h, &pss); + G1ParScanHeapEvacClosure scan_evac_cl(_g1h, &pss, rp); + G1ParScanHeapEvacFailureClosure evac_failure_cl(_g1h, &pss, rp); + G1ParScanPartialArrayClosure partial_scan_cl(_g1h, &pss, rp); pss.set_evac_closure(&scan_evac_cl); pss.set_evac_failure_closure(&evac_failure_cl); pss.set_partial_scan_closure(&partial_scan_cl); - G1ParScanExtRootClosure only_scan_root_cl(_g1h, &pss); - G1ParScanPermClosure only_scan_perm_cl(_g1h, &pss); - G1ParScanHeapRSClosure only_scan_heap_rs_cl(_g1h, &pss); - G1ParPushHeapRSClosure push_heap_rs_cl(_g1h, &pss); - - G1ParScanAndMarkExtRootClosure scan_mark_root_cl(_g1h, &pss); - G1ParScanAndMarkPermClosure scan_mark_perm_cl(_g1h, &pss); - G1ParScanAndMarkHeapRSClosure scan_mark_heap_rs_cl(_g1h, &pss); + G1ParScanExtRootClosure only_scan_root_cl(_g1h, &pss, rp); + G1ParScanPermClosure only_scan_perm_cl(_g1h, &pss, rp); + + G1ParScanAndMarkExtRootClosure scan_mark_root_cl(_g1h, &pss, rp); + G1ParScanAndMarkPermClosure scan_mark_perm_cl(_g1h, &pss, rp); - OopsInHeapRegionClosure *scan_root_cl; - OopsInHeapRegionClosure *scan_perm_cl; + OopClosure* scan_root_cl = &only_scan_root_cl; + OopsInHeapRegionClosure* scan_perm_cl = &only_scan_perm_cl; if (_g1h->g1_policy()->during_initial_mark_pause()) { + // We also need to mark copied objects. scan_root_cl = &scan_mark_root_cl; scan_perm_cl = &scan_mark_perm_cl; - } else { - scan_root_cl = &only_scan_root_cl; - scan_perm_cl = &only_scan_perm_cl; } + // The following closure is used to scan RSets looking for reference + // fields that point into the collection set. The actual field iteration + // is performed by a FilterIntoCSClosure, whose do_oop method calls the + // do_oop method of the following closure. + // Therefore we want to record the reference processor in the + // FilterIntoCSClosure. To do so we record the STW reference + // processor into the following closure and pass it to the + // FilterIntoCSClosure in HeapRegionDCTOC::walk_mem_region_with_cl. + G1ParPushHeapRSClosure push_heap_rs_cl(_g1h, &pss, rp); + pss.start_strong_roots(); _g1h->g1_process_strong_roots(/* not collecting perm */ false, SharedHeap::SO_AllClasses, @@ -4716,6 +4764,7 @@ OopsInHeapRegionClosure* scan_rs, OopsInGenClosure* scan_perm, int worker_i) { + // First scan the strong roots, including the perm gen. double ext_roots_start = os::elapsedTime(); double closure_app_time_sec = 0.0; @@ -4738,6 +4787,7 @@ buf_scan_non_heap_roots.done(); buf_scan_perm.done(); double ext_roots_end = os::elapsedTime(); + g1_policy()->reset_obj_copy_time(worker_i); double obj_copy_time_sec = buf_scan_non_heap_roots.closure_app_seconds() + @@ -4760,14 +4810,17 @@ if (scan_rs != NULL) { g1_rem_set()->oops_into_collection_set_do(scan_rs, worker_i); } - // Finish with the ref_processor roots. + + // Finish with the CM ref_processor roots. if (!_process_strong_tasks->is_task_claimed(G1H_PS_refProcessor_oops_do)) { - // We need to treat the discovered reference lists as roots and - // keep entries (which are added by the marking threads) on them - // live until they can be processed at the end of marking. - ref_processor()->weak_oops_do(scan_non_heap_roots); - ref_processor()->oops_do(scan_non_heap_roots); + // We need to treat the discovered reference lists of the + // concurrent mark ref processor as roots and keep entries + // (which are added by the marking threads) on them live + // until they can be processed at the end of marking. + ref_processor_cm()->weak_oops_do(scan_non_heap_roots); + ref_processor_cm()->oops_do(scan_non_heap_roots); } + g1_policy()->record_collection_pause_end_G1_strong_roots(); _process_strong_tasks->all_tasks_completed(); } @@ -4797,6 +4850,547 @@ perm_gen()->save_marks(); } +// Weak Reference Processing support + +bool G1STWIsAliveClosure::do_object_b(oop p) { + // An object is reachable if it is outside the collection set, + // or is inside and copied. +#ifdef G1_DEBUG + gclog_or_tty->print_cr("is alive "PTR_FORMAT" in CS %d forwarded %d overall %d", + (void*) p, _g1->obj_in_cs(p), p->is_forwarded(), + !_g1->obj_in_cs(p) || p->is_forwarded()); +#endif // G1_DEBUG + return !_g1->obj_in_cs(p) || p->is_forwarded(); +} + +// Non Copying Keep Alive closure +class G1KeepAliveClosure: public OopClosure { + G1CollectedHeap* _g1; +public: + G1KeepAliveClosure(G1CollectedHeap* g1) : _g1(g1) {} + void do_oop(narrowOop* p) { guarantee(false, "Not needed"); } + void do_oop( oop* p) { + oop obj = *p; +#ifdef G1_DEBUG + gclog_or_tty->print_cr("Non-copy keep alive *"PTR_FORMAT" = "PTR_FORMAT" "PTR_FORMAT, + p, (void*) obj, (void*) *p); +#endif // G1_DEBUG + + if (_g1->obj_in_cs(obj)) { + assert( obj->is_forwarded(), "invariant" ); + *p = obj->forwardee(); +#ifdef G1_DEBUG + gclog_or_tty->print_cr(" in CSet: moved "PTR_FORMAT" -> "PTR_FORMAT, + (void*) obj, (void*) *p); +#endif // G1_DEBUG + } + } +}; + +// Copying Keep Alive closure - can be called from both +// serial and parallel code as long as different worker +// threads utilize different G1ParScanThreadState instances +// and different queues. + +class G1CopyingKeepAliveClosure: public OopClosure { + G1CollectedHeap* _g1h; + OopClosure* _copy_non_heap_obj_cl; + OopsInHeapRegionClosure* _copy_perm_obj_cl; + G1ParScanThreadState* _par_scan_state; + +public: + G1CopyingKeepAliveClosure(G1CollectedHeap* g1h, + OopClosure* non_heap_obj_cl, + OopsInHeapRegionClosure* perm_obj_cl, + G1ParScanThreadState* pss): + _g1h(g1h), + _copy_non_heap_obj_cl(non_heap_obj_cl), + _copy_perm_obj_cl(perm_obj_cl), + _par_scan_state(pss) + {} + + virtual void do_oop(narrowOop* p) { do_oop_work(p); } + virtual void do_oop( oop* p) { do_oop_work(p); } + + template void do_oop_work(T* p) { + oop obj = oopDesc::load_decode_heap_oop(p); +#ifdef G1_DEBUG + gclog_or_tty->print_cr("Copying keep alive *"PTR_FORMAT" = "PTR_FORMAT" "PTR_FORMAT, + p, (void*) obj, (void*) *p); +#endif // G1_DEBUG + + if (_g1h->obj_in_cs(obj)) { + // If the referent object has been forwarded (either copied + // to a new location or to itself in the event of an + // evacuation failure) then we need to update the reference + // field and, if both reference and referent are in the G1 + // heap, update the RSet for the referent. + // + // If the referent has not been forwarded then we have to keep + // it alive by policy. Therefore we have copy the referent. + // If the reference field is in the G1 heap then we can push + // on the PSS queue. When the queue is drained (after each + // phase of reference processing) the object and it's followers + // will be copied. If the reference field is not in the G1 heap + // then we need to use the the non-heap or perm closures to + // copy the object to avoid updating the RSet. +#ifdef G1_DEBUG + if (obj->is_forwarded()) { + gclog_or_tty->print_cr(" in CSet: forwarded "PTR_FORMAT" -> "PTR_FORMAT, + (void*) obj, (void*) *p); + } +#endif // G1_DEBUG + + if (_g1h->is_in_g1_reserved(p)) { +#ifdef G1_DEBUG + gclog_or_tty->print_cr(" in CSet: pushing "PTR_FORMAT, p); +#endif // G1_DEBUG + _par_scan_state->push_on_queue(p); + } else { +#ifdef G1_DEBUG + gclog_or_tty->print_cr(" in CSet: applying closure to "PTR_FORMAT, p); +#endif // G1_DEBUG + // The reference field is not in the G1 heap. + if (_g1h->perm_gen()->is_in(p)) { + _copy_perm_obj_cl->do_oop(p); + } else { + _copy_non_heap_obj_cl->do_oop(p); + } + } + } else { +#ifdef G1_DEBUG + gclog_or_tty->print_cr(" not in CSet"); +#endif // G1_DEBUG + } + } +}; + +// Serial drain queue closure. Called as the 'complete_gc' +// closure for each discovered list in some of the +// reference processing phases. + +class G1STWDrainQueueClosure: public VoidClosure { +protected: + G1CollectedHeap* _g1h; + G1ParScanThreadState* _par_scan_state; + + G1ParScanThreadState* par_scan_state() { return _par_scan_state; } + +public: + G1STWDrainQueueClosure(G1CollectedHeap* g1h, G1ParScanThreadState* pss) : + _g1h(g1h), + _par_scan_state(pss) + { } + + void do_void() { + G1ParScanThreadState* const pss = par_scan_state(); + pss->trim_queue(); + } +}; + +// Parallel Reference Processing closures + +// Implementation of AbstractRefProcTaskExecutor for parallel reference +// processing during G1 evacuation pauses. + +class G1STWRefProcTaskExecutor: public AbstractRefProcTaskExecutor { +private: + G1CollectedHeap* _g1h; + RefToScanQueueSet* _queues; + WorkGang* _workers; + int _active_workers; + +public: + G1STWRefProcTaskExecutor(G1CollectedHeap* g1h, + WorkGang* workers, + RefToScanQueueSet *task_queues, + int n_workers) : + _g1h(g1h), + _queues(task_queues), + _workers(workers), + _active_workers(n_workers) + { + assert(n_workers > 0, "shouldn't call this otherwise"); + } + + // Executes the given task using concurrent marking worker threads. + virtual void execute(ProcessTask& task); + virtual void execute(EnqueueTask& task); +}; + +// Gang task for possibly parallel reference processing + +class G1STWRefProcTaskProxy: public AbstractGangTask { + typedef AbstractRefProcTaskExecutor::ProcessTask ProcessTask; + ProcessTask& _proc_task; + G1CollectedHeap* _g1h; + RefToScanQueueSet *_task_queues; + ParallelTaskTerminator* _terminator; + +public: + G1STWRefProcTaskProxy(ProcessTask& proc_task, + G1CollectedHeap* g1h, + RefToScanQueueSet *task_queues, + ParallelTaskTerminator* terminator) : + AbstractGangTask("Process reference objects in parallel"), + _proc_task(proc_task), + _g1h(g1h), + _task_queues(task_queues), + _terminator(terminator) + {} + + virtual void work(int i) { + // The reference processing task executed by a single worker. + ResourceMark rm; + HandleMark hm; + + G1STWIsAliveClosure is_alive(_g1h); + + G1ParScanThreadState pss(_g1h, i); + + G1ParScanHeapEvacClosure scan_evac_cl(_g1h, &pss, NULL); + G1ParScanHeapEvacFailureClosure evac_failure_cl(_g1h, &pss, NULL); + G1ParScanPartialArrayClosure partial_scan_cl(_g1h, &pss, NULL); + + pss.set_evac_closure(&scan_evac_cl); + pss.set_evac_failure_closure(&evac_failure_cl); + pss.set_partial_scan_closure(&partial_scan_cl); + + G1ParScanExtRootClosure only_copy_non_heap_cl(_g1h, &pss, NULL); + G1ParScanPermClosure only_copy_perm_cl(_g1h, &pss, NULL); + + G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(_g1h, &pss, NULL); + G1ParScanAndMarkPermClosure copy_mark_perm_cl(_g1h, &pss, NULL); + + OopClosure* copy_non_heap_cl = &only_copy_non_heap_cl; + OopsInHeapRegionClosure* copy_perm_cl = &only_copy_perm_cl; + + if (_g1h->g1_policy()->during_initial_mark_pause()) { + // We also need to mark copied objects. + copy_non_heap_cl = ©_mark_non_heap_cl; + copy_perm_cl = ©_mark_perm_cl; + } + + // Keep alive closure. + G1CopyingKeepAliveClosure keep_alive(_g1h, copy_non_heap_cl, copy_perm_cl, &pss); + + // Complete GC closure + G1ParEvacuateFollowersClosure drain_queue(_g1h, &pss, _task_queues, _terminator); + + // Call the reference processing task's work routine. + _proc_task.work(i, is_alive, keep_alive, drain_queue); + + // Note we cannot assert that the refs array is empty here as not all + // of the processing tasks (specifically phase2 - pp2_work) do not execute + // the complete_gc closure (which ordinarily would drain the queue) so + // the queue may not be empty. + } +}; + +// Driver routine for parallel reference processing. +// Creates an instance of the ref processing gang +// task and has the worker threads execute it. +void G1STWRefProcTaskExecutor::execute(ProcessTask& proc_task) { + assert(_workers != NULL, "Need parallel worker threads."); + + ParallelTaskTerminator terminator(_active_workers, _queues); + G1STWRefProcTaskProxy proc_task_proxy(proc_task, _g1h, _queues, &terminator); + + _g1h->set_par_threads(_active_workers); + _workers->run_task(&proc_task_proxy); + _g1h->set_par_threads(0); +} + +// Gang task for parallel reference enqueueing. + +class G1STWRefEnqueueTaskProxy: public AbstractGangTask { + typedef AbstractRefProcTaskExecutor::EnqueueTask EnqueueTask; + EnqueueTask& _enq_task; + +public: + G1STWRefEnqueueTaskProxy(EnqueueTask& enq_task) : + AbstractGangTask("Enqueue reference objects in parallel"), + _enq_task(enq_task) + { } + + virtual void work(int i) { + _enq_task.work(i); + } +}; + +// Driver routine for parallel reference enqueing. +// Creates an instance of the ref enqueueing gang +// task and has the worker threads execute it. + +void G1STWRefProcTaskExecutor::execute(EnqueueTask& enq_task) { + assert(_workers != NULL, "Need parallel worker threads."); + + G1STWRefEnqueueTaskProxy enq_task_proxy(enq_task); + + _g1h->set_par_threads(_active_workers); + _workers->run_task(&enq_task_proxy); + _g1h->set_par_threads(0); +} + +// End of weak reference support closures + +// Abstract task used to preserve (i.e. copy) any referent objects +// that are in the collection set and are pointed to by reference +// objects discovered by the CM ref processor. + +class G1ParPreserveCMReferentsTask: public AbstractGangTask { +protected: + G1CollectedHeap* _g1h; + RefToScanQueueSet *_queues; + ParallelTaskTerminator _terminator; + int _n_workers; + +public: + G1ParPreserveCMReferentsTask(G1CollectedHeap* g1h,int workers, RefToScanQueueSet *task_queues) : + AbstractGangTask("ParPreserveCMReferents"), + _g1h(g1h), + _queues(task_queues), + _terminator(workers, _queues), + _n_workers(workers) + { } + + void work(int i) { + ResourceMark rm; + HandleMark hm; + + G1ParScanThreadState pss(_g1h, i); + G1ParScanHeapEvacClosure scan_evac_cl(_g1h, &pss, NULL); + G1ParScanHeapEvacFailureClosure evac_failure_cl(_g1h, &pss, NULL); + G1ParScanPartialArrayClosure partial_scan_cl(_g1h, &pss, NULL); + + pss.set_evac_closure(&scan_evac_cl); + pss.set_evac_failure_closure(&evac_failure_cl); + pss.set_partial_scan_closure(&partial_scan_cl); + + assert(pss.refs()->is_empty(), "both queue and overflow should be empty"); + + // Copying keep alive closure. Applied to referent objects that need + // to be copied. + + G1ParScanExtRootClosure only_copy_non_heap_cl(_g1h, &pss, NULL); + G1ParScanPermClosure only_copy_perm_cl(_g1h, &pss, NULL); + + G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(_g1h, &pss, NULL); + G1ParScanAndMarkPermClosure copy_mark_perm_cl(_g1h, &pss, NULL); + + OopClosure* copy_non_heap_cl = &only_copy_non_heap_cl; + OopsInHeapRegionClosure* copy_perm_cl = &only_copy_perm_cl; + + if (_g1h->g1_policy()->during_initial_mark_pause()) { + // We also need to mark copied objects. + copy_non_heap_cl = ©_mark_non_heap_cl; + copy_perm_cl = ©_mark_perm_cl; + } + + // Keep alive closure. + G1CopyingKeepAliveClosure keep_alive(_g1h, copy_non_heap_cl, copy_perm_cl, &pss); + + ReferenceProcessor* rp = _g1h->ref_processor_cm(); + + int limit = ReferenceProcessor::subclasses_of_ref() * rp->max_num_q(); + int stride = MIN2(MAX2(_n_workers, 1), limit); + +#ifdef G1_DEBUG + gclog_or_tty->print_cr("_n_workers = %d", _n_workers); + gclog_or_tty->print_cr("limit = %d", limit); + gclog_or_tty->print_cr("stride = %d", stride); +#endif // G1_DEBUG + + // limit is set using max_num_q() - which was set using ParallelGCThreads. + // So this must be true - but assert just in case someone decides to + // change the worker ids. + assert(0 <= i && i < limit, "sanity"); + + // Select discovered lists [i, i+stride, i+2*stride,...,limit) + for (int idx = i; idx < limit; idx += stride) { + DiscoveredList& ref_list = rp->discovered_soft_refs()[idx]; +#ifdef G1_DEBUG + gclog_or_tty->print_cr("Worker %d: Preserving referents on %s list(%d) - "PTR_FORMAT, + i, rp->list_name(idx), idx, (void*)&ref_list); +#endif // G1_DEBUG + + oop ref = ref_list.head(); + while (ref != ReferenceProcessor::sentinel_ref()) { +#ifdef G1_DEBUG + if (_g1h->obj_in_cs(ref)) { + gclog_or_tty->print_cr("Boom! Worker %d: Found a reference in CS. Forwarded? %s", + i, (ref->is_forwarded() ? "YES" : "NO")); + } +#endif // G1_DEBUG + + HeapWord* referent_addr = java_lang_ref_Reference::referent_addr(ref); + oop referent = java_lang_ref_Reference::referent(ref); + if (referent != NULL) { + if (UseCompressedOops) { + keep_alive.do_oop((narrowOop*)referent_addr); + } else { + keep_alive.do_oop((oop*)referent_addr); + } + } + + // get the next reference object on the discovered list. + ref = java_lang_ref_Reference::discovered(ref); + } + } + + // Drain the queue - which may cause stealing + G1ParEvacuateFollowersClosure drain_queue(_g1h, &pss, _queues, &_terminator); + drain_queue.do_void(); + + // Allocation buffers were retired at the end of G1ParEvacuateFollowersClosure + assert(pss.refs()->is_empty(), "should be"); + } +}; + +// Weak Reference processing during an evacuation pause (part 1). +void G1CollectedHeap::process_discovered_references() { + double ref_proc_start = os::elapsedTime(); + + ReferenceProcessor* rp = _ref_processor_stw; + assert(rp->discovery_enabled(), "should have been enabled"); + + // Closure to test whether a referent is alive. + G1STWIsAliveClosure is_alive(this); + + // Even when parallel reference processing is enabled, the processing + // of JNI refs is serial and performed serially by the current thread + // rather than by a worker. The following PSS will be used for processing + // JNI refs. + + // Use only a single queue for this PSS. + G1ParScanThreadState pss(this, 0); + + // We do not embed a reference processor in the copying closures + // while we're actually processing the discovered reference objects. + G1ParScanHeapEvacClosure scan_evac_cl(this, &pss, NULL); + G1ParScanHeapEvacFailureClosure evac_failure_cl(this, &pss, NULL); + G1ParScanPartialArrayClosure partial_scan_cl(this, &pss, NULL); + + pss.set_evac_closure(&scan_evac_cl); + pss.set_evac_failure_closure(&evac_failure_cl); + pss.set_partial_scan_closure(&partial_scan_cl); + + assert(pss.refs()->is_empty(), "pre-condition"); + + G1ParScanExtRootClosure only_copy_non_heap_cl(this, &pss, NULL); + G1ParScanPermClosure only_copy_perm_cl(this, &pss, NULL); + + G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(this, &pss, NULL); + G1ParScanAndMarkPermClosure copy_mark_perm_cl(this, &pss, NULL); + + OopClosure* copy_non_heap_cl = &only_copy_non_heap_cl; + OopsInHeapRegionClosure* copy_perm_cl = &only_copy_perm_cl; + + if (_g1h->g1_policy()->during_initial_mark_pause()) { + // We also need to mark copied objects. + copy_non_heap_cl = ©_mark_non_heap_cl; + copy_perm_cl = ©_mark_perm_cl; + } + + // Keep alive closure. + G1CopyingKeepAliveClosure keep_alive(this, copy_non_heap_cl, copy_perm_cl, &pss); + + // Serial Complete GC closure + G1STWDrainQueueClosure drain_queue(this, &pss); + + // Setup the soft refs policy... + rp->setup_policy(false); + + if (!rp->processing_is_mt()) { + // Serial reference processing... + rp->process_discovered_references(&is_alive, + &keep_alive, + &drain_queue, + NULL); + } else { + // Parallel reference processing + int active_workers = (ParallelGCThreads > 0 ? workers()->total_workers() : 1); + assert(rp->num_q() == active_workers, "sanity"); + assert(active_workers <= rp->max_num_q(), "sanity"); + + G1STWRefProcTaskExecutor par_task_executor(this, workers(), _task_queues, active_workers); + rp->process_discovered_references(&is_alive, &keep_alive, &drain_queue, &par_task_executor); + } + + // We have completed copying any necessary live referent objects + // (that were not copied during the actual pause) so we can + // retire any active alloc buffers + pss.retire_alloc_buffers(); + assert(pss.refs()->is_empty(), "both queue and overflow should be empty"); + + // Any reference objects, in the collection set, that were 'discovered' + // by the CM ref processor should have already been copied (either by + // applying the external root copy closure to the discovered lists, or + // by following an RSet entry). + // + // But some of the referents, that are in the collection set, that these + // reference objects point to may not have been copied: the STW ref + // processor would have seen that the reference object had already + // been 'discovered' and would have skipped discovering the reference, + // but would not have treated the reference object as a regular oop. + // As a reult the copy closure would not have been applied to the + // referent object. + // We need to to copy these referent objects - the references will be + // processed at the end of Remark. + + int n_workers = (G1CollectedHeap::use_parallel_gc_threads() ? + workers()->total_workers() : 1); + + set_par_threads(n_workers); + G1ParPreserveCMReferentsTask keep_cm_referents(this, n_workers, _task_queues); + + if (G1CollectedHeap::use_parallel_gc_threads()) { + workers()->run_task(&keep_cm_referents); + } else { + keep_cm_referents.work(0); + } + + set_par_threads(0); + + double ref_proc_time = os::elapsedTime() - ref_proc_start; + g1_policy()->record_ref_proc_time(ref_proc_time * 1000.0); +} + +// Weak Reference processing during an evacuation pause (part 2). +void G1CollectedHeap::enqueue_discovered_references() { + double ref_enq_start = os::elapsedTime(); + + ReferenceProcessor* rp = _ref_processor_stw; + assert(!rp->discovery_enabled(), "should have been disabled as part of processing"); + + // Now enqueue any remaining on the discovered lists on to + // the pending list. + if (!rp->processing_is_mt()) { + // Serial reference processing... + rp->enqueue_discovered_references(); + } else { + // Parallel reference enqueuing + + int active_workers = (ParallelGCThreads > 0 ? workers()->total_workers() : 1); + assert(rp->num_q() == active_workers, "sanity"); + assert(active_workers <= rp->max_num_q(), "sanity"); + + G1STWRefProcTaskExecutor par_task_executor(this, workers(), _task_queues, active_workers); + rp->enqueue_discovered_references(&par_task_executor); + } + + rp->verify_no_references_recorded(); + assert(!rp->discovery_enabled(), "should have been disabled"); + + // FIXME + // CM's referent processing also cleans up the + // string table and symbol tables. Should we + // do that here also? + + double ref_enq_time = os::elapsedTime() - ref_enq_start; + g1_policy()->record_ref_enq_time(ref_enq_time * 1000.0); +} + void G1CollectedHeap::evacuate_collection_set() { set_evacuation_failed(false); @@ -4814,6 +5408,7 @@ assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty"); double start_par = os::elapsedTime(); + if (G1CollectedHeap::use_parallel_gc_threads()) { // The individual threads will set their evac-failure closures. StrongRootsScope srs(this); @@ -4827,6 +5422,14 @@ double par_time = (os::elapsedTime() - start_par) * 1000.0; g1_policy()->record_par_time(par_time); set_par_threads(0); + + // Process any discovered reference objects - we have + // to do this _before_ we retire the GC alloc regions + // as we may have to copy some 'reachable' referent + // objects (and their reachable sub-graphs) that were + // not copied during the pause. + process_discovered_references(); + // Is this the right thing to do here? We don't save marks // on individual heap regions when we allocate from // them in parallel, so this seems like the correct place for this. @@ -4837,10 +5440,11 @@ // non-perm oops then we will need to process the code blobs // here too. { - G1IsAliveClosure is_alive(this); + G1STWIsAliveClosure is_alive(this); G1KeepAliveClosure keep_alive(this); JNIHandles::weak_oops_do(&is_alive, &keep_alive); } + release_gc_alloc_regions(false /* totally */); g1_rem_set()->cleanup_after_oops_into_collection_set_do(); @@ -4862,6 +5466,15 @@ } } + // Enqueue any remaining references remaining on the STW + // reference processor's discovered lists. We need to do + // this after the card table is cleaned (and verified) as + // the act of enqueuing entries on to the pending list + // will log these updates (and dirty their associated + // cards). We need these updates logged to update any + // RSets. + enqueue_discovered_references(); + if (G1DeferredRSUpdate) { RedirtyLoggedCardTableEntryFastClosure redirty; dirty_card_queue_set().set_closure(&redirty); @@ -5104,7 +5717,7 @@ } double elapsed = os::elapsedTime() - start; - g1_policy()->record_clear_ct_time( elapsed * 1000.0); + g1_policy()->record_clear_ct_time(elapsed * 1000.0); #ifndef PRODUCT if (G1VerifyCTCleanup || VerifyAfterGC) { G1VerifyCardTableCleanup cleanup_verifier(this, ct_bs); --- old/src/share/vm/gc_implementation/g1/g1CollectedHeap.hpp 2011-06-23 14:02:56.977999761 -0700 +++ new/src/share/vm/gc_implementation/g1/g1CollectedHeap.hpp 2011-06-23 14:02:56.769576550 -0700 @@ -154,7 +154,21 @@ : G1AllocRegion("Mutator Alloc Region", false /* bot_updates */) { } }; +// The G1 STW is alive closure. +// An instance is embedded into the G1CH and used as the +// _is_alive_non_header closure in the STW reference +// processor. It is also extensively used during refence +// processing during STW evacuation pauses. +class G1STWIsAliveClosure: public BoolObjectClosure { + G1CollectedHeap* _g1; +public: + G1STWIsAliveClosure(G1CollectedHeap* g1) : _g1(g1) {} + void do_object(oop p) { assert(false, "Do not call."); } + bool do_object_b(oop p); +}; + class RefineCardTableEntryClosure; + class G1CollectedHeap : public SharedHeap { friend class VM_G1CollectForAllocation; friend class VM_GenCollectForPermanentAllocation; @@ -567,6 +581,14 @@ // allocated block, or else "NULL". HeapWord* expand_and_allocate(size_t word_size); + // Process any reference objects discovered during + // an incremental evacuation pause. + void process_discovered_references(); + + // Enqueue any remaining discovered references + // after processing. + void enqueue_discovered_references(); + public: G1MonitoringSupport* g1mm() { return _g1mm; } @@ -838,14 +860,73 @@ void print_gc_alloc_regions(); #endif // !PRODUCT - // Instance of the concurrent mark is_alive closure for embedding - // into the reference processor as the is_alive_non_header. This - // prevents unnecessary additions to the discovered lists during - // concurrent discovery. - G1CMIsAliveClosure _is_alive_closure; + // ("Weak") Reference processing support. + // + // G1 has 2 instances of the referece processor class. One + // (_ref_processor_cm) handles reference object discovery + // and subsequent processing during concurrent marking cycles. + // + // The other (_ref_processor_stw) handles reference object + // discovery and processing during full GCs and incremental + // evacuation pauses. + // + // During an incremental pause, reference discovery will be + // temporarily disabled for _ref_processor_cm and will be + // enabled for _ref_processor_stw. At the end of the evacuation + // pause references discovered by _ref_processor_stw will be + // processed and discovery will be disabled. The previous + // setting for reference object discovery for _ref_processor_cm + // will be re-instated. + // + // At the start of marking: + // * Discovery by the CM ref processor is verified to be inactive + // and it's discovered lists are empty. + // * Discovery by the CM ref processor is then enabled. + // + // At the end of marking: + // * Any references on the CM ref processor's discovered + // lists are processed (possibly MT). + // + // At the start of full GC we: + // * Disable discovery by the CM ref processor and + // empty CM ref processor's discovered lists + // (without processing any entries). + // * Verify that the STW ref processor is inactive and it's + // discovered lists are empty. + // * Temporarily set STW ref processor discovery as single threaded. + // * Temporarily clear the STW ref processor's _is_alive_non_header + // field. + // * Finally enable discovery by the STW ref processor. + // + // The STW ref processor is used to record any discovered + // references during the full GC. + // + // At the end of a full GC we: + // * Will enqueue any non-live discovered references on the + // STW ref processor's discovered lists. This makes the + // STW ref processor inactive by disabling discovery. + // * Verify that the CM ref processor is still inactive + // and no references have been placed on it's discovered + // lists (also checked as a precondition during initial marking). + + // The (stw) reference processor... + ReferenceProcessor* _ref_processor_stw; + + // Instance of the is_alive closure for embedding into the + // STW reference processor as the _is_alive_non_header field. + // The _is_alive_non_header prevents unnecessary additions to + // the discovered lists during reference discovery. + G1STWIsAliveClosure _is_alive_closure_stw; - // ("Weak") Reference processing support - ReferenceProcessor* _ref_processor; + // The (concurrent marking) reference processor... + ReferenceProcessor* _ref_processor_cm; + + // Instance of the concurrent mark is_alive closure for embedding + // into the Concurrent Marking reference processor as the + // _is_alive_non_header field. The _is_alive_non_header + // prevents unnecessary additions to the discovered lists + // during concurrent discovery. + G1CMIsAliveClosure _is_alive_closure_cm; enum G1H_process_strong_roots_tasks { G1H_PS_mark_stack_oops_do, @@ -886,6 +967,7 @@ // specified by the policy object. jint initialize(); + // Initialize weak reference processing. virtual void ref_processing_init(); void set_par_threads(int t) { @@ -937,8 +1019,13 @@ // The shared block offset table array. G1BlockOffsetSharedArray* bot_shared() const { return _bot_shared; } - // Reference Processing accessor - ReferenceProcessor* ref_processor() { return _ref_processor; } + // Reference Processing accessors + + // The STW reference processor.... + ReferenceProcessor* ref_processor_stw() const { return _ref_processor_stw; } + + // The Concurent Marking reference processor... + ReferenceProcessor* ref_processor_cm() const { return _ref_processor_cm; } virtual size_t capacity() const; virtual size_t used() const; --- old/src/share/vm/gc_implementation/g1/g1CollectorPolicy.cpp 2011-06-23 14:02:58.153760276 -0700 +++ new/src/share/vm/gc_implementation/g1/g1CollectorPolicy.cpp 2011-06-23 14:02:57.947253128 -0700 @@ -153,8 +153,12 @@ _summary(new Summary()), -#ifndef PRODUCT _cur_clear_ct_time_ms(0.0), + + _cur_ref_proc_time_ms(0.0), + _cur_ref_enq_time_ms(0.0), + +#ifndef PRODUCT _min_clear_cc_time_ms(-1.0), _max_clear_cc_time_ms(-1.0), _cur_clear_cc_time_ms(0.0), @@ -1460,6 +1464,8 @@ #endif print_stats(1, "Other", other_time_ms); print_stats(2, "Choose CSet", _recorded_young_cset_choice_time_ms); + print_stats(2, "Ref Proc", _cur_ref_proc_time_ms); + print_stats(2, "Ref Enq", _cur_ref_enq_time_ms); for (int i = 0; i < _aux_num; ++i) { if (_cur_aux_times_set[i]) { --- old/src/share/vm/gc_implementation/g1/g1CollectorPolicy.hpp 2011-06-23 14:02:59.910687512 -0700 +++ new/src/share/vm/gc_implementation/g1/g1CollectorPolicy.hpp 2011-06-23 14:02:59.549078640 -0700 @@ -119,6 +119,8 @@ double _cur_satb_drain_time_ms; double _cur_clear_ct_time_ms; bool _satb_drain_time_set; + double _cur_ref_proc_time_ms; + double _cur_ref_enq_time_ms; #ifndef PRODUCT // Card Table Count Cache stats @@ -980,6 +982,14 @@ _cur_aux_times_ms[i] += ms; } + void record_ref_proc_time(double ms) { + _cur_ref_proc_time_ms = ms; + } + + void record_ref_enq_time(double ms) { + _cur_ref_enq_time_ms = ms; + } + #ifndef PRODUCT void record_cc_clear_time(double ms) { if (_min_clear_cc_time_ms < 0.0 || ms <= _min_clear_cc_time_ms) --- old/src/share/vm/gc_implementation/g1/g1MarkSweep.cpp 2011-06-23 14:03:02.591182035 -0700 +++ new/src/share/vm/gc_implementation/g1/g1MarkSweep.cpp 2011-06-23 14:03:02.234863923 -0700 @@ -62,6 +62,8 @@ // hook up weak ref data so it can be used during Mark-Sweep assert(GenMarkSweep::ref_processor() == NULL, "no stomping"); assert(rp != NULL, "should be non-NULL"); + assert(rp == G1CollectedHeap::heap()->ref_processor_stw(), "Precondition"); + GenMarkSweep::_ref_processor = rp; rp->setup_policy(clear_all_softrefs); @@ -139,6 +141,8 @@ // Process reference objects found during marking ReferenceProcessor* rp = GenMarkSweep::ref_processor(); + assert(rp == G1CollectedHeap::heap()->ref_processor_stw(), "Sanity"); + rp->setup_policy(clear_all_softrefs); rp->process_discovered_references(&GenMarkSweep::is_alive, &GenMarkSweep::keep_alive, @@ -166,7 +170,6 @@ GenMarkSweep::follow_mdo_weak_refs(); assert(GenMarkSweep::_marking_stack.is_empty(), "just drained"); - // Visit interned string tables and delete unmarked oops StringTable::unlink(&GenMarkSweep::is_alive); // Clean up unreferenced symbols in symbol table. @@ -346,7 +349,14 @@ NULL, // do not touch code cache here &GenMarkSweep::adjust_pointer_closure); - g1h->ref_processor()->weak_oops_do(&GenMarkSweep::adjust_root_pointer_closure); + assert(GenMarkSweep::ref_processor() == g1h->ref_processor_stw(), "Sanity"); + g1h->ref_processor_stw()->weak_oops_do(&GenMarkSweep::adjust_root_pointer_closure); + + // The discovered lists of the CM reference processor were emptied + // at the start of the Full GC but if the sentinel ref was moved + // as part of this Full GC then they need to be adjusted to point + // to the sentinel ref's new location. + g1h->ref_processor_cm()->weak_oops_do(&GenMarkSweep::adjust_root_pointer_closure); // Now adjust pointers in remaining weak roots. (All of which should // have been cleared if they pointed to non-surviving objects.) --- old/src/share/vm/gc_implementation/g1/g1OopClosures.hpp 2011-06-23 14:03:05.209669443 -0700 +++ new/src/share/vm/gc_implementation/g1/g1OopClosures.hpp 2011-06-23 14:03:04.665571847 -0700 @@ -34,6 +34,7 @@ class CMMarkStack; class G1ParScanThreadState; class CMTask; +class ReferenceProcessor; // A class that scans oops in a given heap region (much as OopsInGenClosure // scans oops in a generation.) @@ -57,8 +58,15 @@ class G1ParPushHeapRSClosure : public G1ParClosureSuper { public: - G1ParPushHeapRSClosure(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state) : - G1ParClosureSuper(g1, par_scan_state) { } + G1ParPushHeapRSClosure(G1CollectedHeap* g1, + G1ParScanThreadState* par_scan_state, + ReferenceProcessor* rp) : + G1ParClosureSuper(g1, par_scan_state) + { + assert(_ref_processor == NULL, "sanity"); + _ref_processor = rp; + } + template void do_oop_nv(T* p); virtual void do_oop(oop* p) { do_oop_nv(p); } virtual void do_oop(narrowOop* p) { do_oop_nv(p); } @@ -66,8 +74,13 @@ class G1ParScanClosure : public G1ParClosureSuper { public: - G1ParScanClosure(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state) : - G1ParClosureSuper(g1, par_scan_state) { } + G1ParScanClosure(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state, ReferenceProcessor* rp) : + G1ParClosureSuper(g1, par_scan_state) + { + assert(_ref_processor == NULL, "sanity"); + _ref_processor = rp; + } + template void do_oop_nv(T* p); virtual void do_oop(oop* p) { do_oop_nv(p); } virtual void do_oop(narrowOop* p) { do_oop_nv(p); } @@ -90,9 +103,18 @@ class G1ParScanPartialArrayClosure : public G1ParClosureSuper { G1ParScanClosure _scanner; + public: - G1ParScanPartialArrayClosure(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state) : - G1ParClosureSuper(g1, par_scan_state), _scanner(g1, par_scan_state) { } + G1ParScanPartialArrayClosure(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state, ReferenceProcessor* rp) : + G1ParClosureSuper(g1, par_scan_state), _scanner(g1, par_scan_state, rp) + { + assert(_ref_processor == NULL, "sanity"); + } + + G1ParScanClosure* scanner() { + return &_scanner; + } + template void do_oop_nv(T* p); virtual void do_oop(oop* p) { do_oop_nv(p); } virtual void do_oop(narrowOop* p) { do_oop_nv(p); } @@ -104,20 +126,30 @@ protected: template void mark_forwardee(T* p); oop copy_to_survivor_space(oop obj); + public: G1ParCopyHelper(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state, G1ParScanClosure *scanner) : G1ParClosureSuper(g1, par_scan_state), _scanner(scanner) { } }; -template +template class G1ParCopyClosure : public G1ParCopyHelper { G1ParScanClosure _scanner; + template void do_oop_work(T* p); + public: - G1ParCopyClosure(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state) : - _scanner(g1, par_scan_state), G1ParCopyHelper(g1, par_scan_state, &_scanner) { } + G1ParCopyClosure(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state, + ReferenceProcessor* rp) : + _scanner(g1, par_scan_state, rp), + G1ParCopyHelper(g1, par_scan_state, &_scanner) + { + assert(_ref_processor == NULL, "sanity"); + } + + G1ParScanClosure* scanner() { return &_scanner; } + template void do_oop_nv(T* p) { do_oop_work(p); if (do_mark_forwardee) @@ -129,21 +161,25 @@ typedef G1ParCopyClosure G1ParScanExtRootClosure; typedef G1ParCopyClosure G1ParScanPermClosure; -typedef G1ParCopyClosure G1ParScanHeapRSClosure; + typedef G1ParCopyClosure G1ParScanAndMarkExtRootClosure; typedef G1ParCopyClosure G1ParScanAndMarkPermClosure; -typedef G1ParCopyClosure G1ParScanAndMarkHeapRSClosure; -// This is the only case when we set skip_cset_test. Basically, this -// closure is (should?) only be called directly while we're draining -// the overflow and task queues. In that case we know that the -// reference in question points into the collection set, otherwise we -// would not have pushed it on the queue. The following is defined in -// g1_specialized_oop_closures.hpp. -// typedef G1ParCopyClosure G1ParScanHeapEvacClosure; -// We need a separate closure to handle references during evacuation -// failure processing, as we cannot asume that the reference already -// points into the collection set (like G1ParScanHeapEvacClosure does). +// The following closure types are no longer used but are retained +// for historical reasons: +// typedef G1ParCopyClosure G1ParScanHeapRSClosure; +// typedef G1ParCopyClosure G1ParScanAndMarkHeapRSClosure; + +// The following closure type is defined in g1_specialized_oop_closures.hpp: +// +// typedef G1ParCopyClosure G1ParScanHeapEvacClosure; + +// We use a separate closure to handle references during evacuation +// failure processing. +// We could have used another instance of G1ParScanHeapEvacClosure +// (since that closure no longer assumes that the references it +// handles point into the collection set). + typedef G1ParCopyClosure G1ParScanHeapEvacFailureClosure; class FilterIntoCSClosure: public OopClosure { @@ -152,9 +188,15 @@ DirtyCardToOopClosure* _dcto_cl; public: FilterIntoCSClosure( DirtyCardToOopClosure* dcto_cl, - G1CollectedHeap* g1, OopClosure* oc) : + G1CollectedHeap* g1, + OopClosure* oc, + ReferenceProcessor* rp) : _dcto_cl(dcto_cl), _g1(g1), _oc(oc) - {} + { + assert(_ref_processor == NULL, "sanity"); + _ref_processor = rp; + } + template void do_oop_nv(T* p); virtual void do_oop(oop* p) { do_oop_nv(p); } virtual void do_oop(narrowOop* p) { do_oop_nv(p); } --- old/src/share/vm/gc_implementation/g1/g1RemSet.cpp 2011-06-23 14:03:07.031989646 -0700 +++ new/src/share/vm/gc_implementation/g1/g1RemSet.cpp 2011-06-23 14:03:06.824180493 -0700 @@ -234,6 +234,7 @@ HeapRegion *startRegion = calculateStartRegion(worker_i); ScanRSClosure scanRScl(oc, worker_i); + _g1->collection_set_iterate_from(startRegion, &scanRScl); scanRScl.set_try_claimed(); _g1->collection_set_iterate_from(startRegion, &scanRScl); @@ -283,6 +284,7 @@ double start = os::elapsedTime(); // Apply the given closure to all remaining log entries. RefineRecordRefsIntoCSCardTableEntryClosure into_cset_update_rs_cl(_g1, into_cset_dcq); + _g1->iterate_dirty_card_closure(&into_cset_update_rs_cl, into_cset_dcq, false, worker_i); // Now there should be no dirty cards. @@ -503,7 +505,7 @@ MemRegion scanRegion(start, end); UpdateRSetImmediate update_rs_cl(_g1->g1_rem_set()); - FilterIntoCSClosure update_rs_cset_oop_cl(NULL, _g1, &update_rs_cl); + FilterIntoCSClosure update_rs_cset_oop_cl(NULL, _g1, &update_rs_cl, NULL /* rp */); FilterOutOfRegionClosure filter_then_update_rs_cset_oop_cl(r, &update_rs_cset_oop_cl); // We can pass false as the "filter_young" parameter here as: @@ -684,7 +686,7 @@ update_rs_oop_cl.set_from(r); TriggerClosure trigger_cl; - FilterIntoCSClosure into_cs_cl(NULL, _g1, &trigger_cl); + FilterIntoCSClosure into_cs_cl(NULL, _g1, &trigger_cl, NULL /* rp */); InvokeIfNotTriggeredClosure invoke_cl(&trigger_cl, &into_cs_cl); Mux2Closure mux(&invoke_cl, &update_rs_oop_cl); --- old/src/share/vm/gc_implementation/g1/heapRegion.cpp 2011-06-23 14:03:09.744690712 -0700 +++ new/src/share/vm/gc_implementation/g1/heapRegion.cpp 2011-06-23 14:03:09.327884316 -0700 @@ -45,7 +45,7 @@ FilterKind fk) : ContiguousSpaceDCTOC(hr, cl, precision, NULL), _hr(hr), _fk(fk), _g1(g1) -{} +{ } FilterOutOfRegionClosure::FilterOutOfRegionClosure(HeapRegion* r, OopClosure* oc) : @@ -214,8 +214,37 @@ int oop_size; OopClosure* cl2 = cl; - FilterIntoCSClosure intoCSFilt(this, g1h, cl); + + // If we are scanning the remembered sets looking for refs + // into the collection set during an evacuation pause then + // we will want to 'discover' reference objects that point + // to referents in the collection set. + // + // Unfortunately it is an instance of FilterIntoCSClosure + // that is iterated over the reference fields of oops in + // mr (and not the G1ParPushHeapRSClosure - which is the + // cl parameter). + // If we set the _ref_processor field in the FilterIntoCSClosure + // instance, all the reference objects that are walked + // (regardless of whether their referent object's are in + // the cset) will be 'discovered'. + // + // The G1STWIsAlive closure considers a referent object that + // is outside the cset as alive. The G1CopyingKeepAliveClosure + // skips referents that are not in the cset. + // + // Therefore reference objects in mr with a referent that is + // outside the cset should be OK. + + ReferenceProcessor* rp = _cl->_ref_processor; + if (rp != NULL) { + assert(rp == _g1->ref_processor_stw(), "should be stw"); + assert(_fk == IntoCSFilterKind, "should be looking for refs into CS"); + } + + FilterIntoCSClosure intoCSFilt(this, g1h, cl, rp); FilterOutOfRegionClosure outOfRegionFilt(_hr, cl); + switch (_fk) { case IntoCSFilterKind: cl2 = &intoCSFilt; break; case OutOfRegionFilterKind: cl2 = &outOfRegionFilt; break; @@ -239,16 +268,19 @@ case NoFilterKind: bottom = walk_mem_region_loop(cl, g1h, _hr, bottom, top); break; + case IntoCSFilterKind: { - FilterIntoCSClosure filt(this, g1h, cl); + FilterIntoCSClosure filt(this, g1h, cl, rp); bottom = walk_mem_region_loop(&filt, g1h, _hr, bottom, top); break; } + case OutOfRegionFilterKind: { FilterOutOfRegionClosure filt(_hr, cl); bottom = walk_mem_region_loop(&filt, g1h, _hr, bottom, top); break; } + default: ShouldNotReachHere(); } @@ -484,7 +516,7 @@ HeapRegion(size_t hrs_index, G1BlockOffsetSharedArray* sharedOffsetArray, MemRegion mr, bool is_zeroed) : G1OffsetTableContigSpace(sharedOffsetArray, mr, is_zeroed), - _next_fk(HeapRegionDCTOC::NoFilterKind), _hrs_index(hrs_index), + _hrs_index(hrs_index), _humongous_type(NotHumongous), _humongous_start_region(NULL), _in_collection_set(false), _is_gc_alloc_region(false), _next_in_special_set(NULL), _orig_end(NULL), --- old/src/share/vm/gc_implementation/g1/heapRegion.hpp 2011-06-23 14:03:12.144501604 -0700 +++ new/src/share/vm/gc_implementation/g1/heapRegion.hpp 2011-06-23 14:03:11.598352826 -0700 @@ -118,7 +118,6 @@ FilterKind fk); }; - // The complicating factor is that BlockOffsetTable diverged // significantly, and we need functionality that is only in the G1 version. // So I copied that code, which led to an alternate G1 version of @@ -223,10 +222,6 @@ ContinuesHumongous }; - // The next filter kind that should be used for a "new_dcto_cl" call with - // the "traditional" signature. - HeapRegionDCTOC::FilterKind _next_fk; - // Requires that the region "mr" be dense with objects, and begin and end // with an object. void oops_in_mr_iterate(MemRegion mr, OopClosure* cl); @@ -598,40 +593,14 @@ // allocated in the current region before the last call to "save_mark". void oop_before_save_marks_iterate(OopClosure* cl); - // This call determines the "filter kind" argument that will be used for - // the next call to "new_dcto_cl" on this region with the "traditional" - // signature (i.e., the call below.) The default, in the absence of a - // preceding call to this method, is "NoFilterKind", and a call to this - // method is necessary for each such call, or else it reverts to the - // default. - // (This is really ugly, but all other methods I could think of changed a - // lot of main-line code for G1.) - void set_next_filter_kind(HeapRegionDCTOC::FilterKind nfk) { - _next_fk = nfk; - } - DirtyCardToOopClosure* new_dcto_closure(OopClosure* cl, CardTableModRefBS::PrecisionStyle precision, HeapRegionDCTOC::FilterKind fk); -#if WHASSUP - DirtyCardToOopClosure* - new_dcto_closure(OopClosure* cl, - CardTableModRefBS::PrecisionStyle precision, - HeapWord* boundary) { - assert(boundary == NULL, "This arg doesn't make sense here."); - DirtyCardToOopClosure* res = new_dcto_closure(cl, precision, _next_fk); - _next_fk = HeapRegionDCTOC::NoFilterKind; - return res; - } -#endif - - // // Note the start or end of marking. This tells the heap region // that the collector is about to start or has finished (concurrently) // marking the heap. - // // Note the start of a marking phase. Record the // start of the unmarked area of the region here. --- old/src/share/vm/gc_implementation/g1/satbQueue.cpp 2011-06-23 14:03:13.956704725 -0700 +++ new/src/share/vm/gc_implementation/g1/satbQueue.cpp 2011-06-23 14:03:13.753799619 -0700 @@ -29,6 +29,7 @@ #include "memory/sharedHeap.hpp" #include "runtime/mutexLocker.hpp" #include "runtime/thread.hpp" +#include "runtime/vmThread.hpp" // This method removes entries from an SATB buffer that will not be // useful to the concurrent marking threads. An entry is removed if it @@ -252,9 +253,18 @@ t->satb_mark_queue().apply_closure(_par_closures[worker]); } } - // We'll have worker 0 do this one. - if (worker == 0) { - shared_satb_queue()->apply_closure(_par_closures[0]); + + // We also need to claim the VMThread so that it's parity is updated + // otherwise the next call to Thread::possibly_parallel_oops_do inside + // a StrongRootsScope might skip the VMThread because it has a stale + // parity that matches the parity set by the StrongRootsScope + // + // Whichever worker succeeds in claiming the VMThread gets to do + // the shared queue. + + VMThread* vmt = VMThread::vm_thread(); + if (vmt->claim_oops_do(true, parity)) { + shared_satb_queue()->apply_closure(_par_closures[worker]); } } --- old/src/share/vm/memory/referenceProcessor.cpp 2011-06-23 14:03:17.209799210 -0700 +++ new/src/share/vm/memory/referenceProcessor.cpp 2011-06-23 14:03:17.004808641 -0700 @@ -36,40 +36,10 @@ ReferencePolicy* ReferenceProcessor::_always_clear_soft_ref_policy = NULL; ReferencePolicy* ReferenceProcessor::_default_soft_ref_policy = NULL; oop ReferenceProcessor::_sentinelRef = NULL; -const int subclasses_of_ref = REF_PHANTOM - REF_OTHER; -// List of discovered references. -class DiscoveredList { -public: - DiscoveredList() : _len(0), _compressed_head(0), _oop_head(NULL) { } - oop head() const { - return UseCompressedOops ? oopDesc::decode_heap_oop_not_null(_compressed_head) : - _oop_head; - } - HeapWord* adr_head() { - return UseCompressedOops ? (HeapWord*)&_compressed_head : - (HeapWord*)&_oop_head; - } - void set_head(oop o) { - if (UseCompressedOops) { - // Must compress the head ptr. - _compressed_head = oopDesc::encode_heap_oop_not_null(o); - } else { - _oop_head = o; - } - } - bool empty() const { return head() == ReferenceProcessor::sentinel_ref(); } - size_t length() { return _len; } - void set_length(size_t len) { _len = len; } - void inc_length(size_t inc) { _len += inc; assert(_len > 0, "Error"); } - void dec_length(size_t dec) { _len -= dec; } -private: - // Set value depending on UseCompressedOops. This could be a template class - // but then we have to fix all the instantiations and declarations that use this class. - oop _oop_head; - narrowOop _compressed_head; - size_t _len; -}; +bool DiscoveredList::empty() const { + return head() == ReferenceProcessor::sentinel_ref(); +} void referenceProcessor_init() { ReferenceProcessor::init_statics(); @@ -123,7 +93,7 @@ _discovery_is_mt = mt_discovery; _num_q = MAX2(1, mt_processing_degree); _max_num_q = MAX2(_num_q, mt_discovery_degree); - _discoveredSoftRefs = NEW_C_HEAP_ARRAY(DiscoveredList, _max_num_q * subclasses_of_ref); + _discoveredSoftRefs = NEW_C_HEAP_ARRAY(DiscoveredList, _max_num_q * subclasses_of_ref()); if (_discoveredSoftRefs == NULL) { vm_exit_during_initialization("Could not allocated RefProc Array"); } @@ -132,11 +102,11 @@ _discoveredPhantomRefs = &_discoveredFinalRefs[_max_num_q]; assert(sentinel_ref() != NULL, "_sentinelRef is NULL"); // Initialized all entries to _sentinelRef - for (int i = 0; i < _max_num_q * subclasses_of_ref; i++) { + for (int i = 0; i < _max_num_q * subclasses_of_ref(); i++) { _discoveredSoftRefs[i].set_head(sentinel_ref()); _discoveredSoftRefs[i].set_length(0); } - // If we do barreirs, cache a copy of the barrier set. + // If we do barriers, cache a copy of the barrier set. if (discovered_list_needs_barrier) { _bs = Universe::heap()->barrier_set(); } @@ -146,7 +116,7 @@ #ifndef PRODUCT void ReferenceProcessor::verify_no_references_recorded() { guarantee(!_discovering_refs, "Discovering refs?"); - for (int i = 0; i < _max_num_q * subclasses_of_ref; i++) { + for (int i = 0; i < _max_num_q * subclasses_of_ref(); i++) { guarantee(_discoveredSoftRefs[i].empty(), "Found non-empty discovered list"); } @@ -155,10 +125,10 @@ void ReferenceProcessor::weak_oops_do(OopClosure* f) { // Should this instead be - // for (int i = 0; i < subclasses_of_ref; i++_ { + // for (int i = 0; i < subclasses_of_ref(); i++_ { // for (int j = 0; j < _num_q; j++) { // int index = i * _max_num_q + j; - for (int i = 0; i < _max_num_q * subclasses_of_ref; i++) { + for (int i = 0; i < _max_num_q * subclasses_of_ref(); i++) { if (UseCompressedOops) { f->do_oop((narrowOop*)_discoveredSoftRefs[i].adr_head()); } else { @@ -392,7 +362,7 @@ // allocated and are indexed into. assert(_n_queues == (int) _ref_processor.max_num_q(), "Different number not expected"); for (int j = 0; - j < subclasses_of_ref; + j < ReferenceProcessor::subclasses_of_ref(); j++, index += _n_queues) { _ref_processor.enqueue_discovered_reflist( _refs_lists[index], _pending_list_addr); @@ -412,7 +382,7 @@ task_executor->execute(tsk); } else { // Serial code: call the parent class's implementation - for (int i = 0; i < _max_num_q * subclasses_of_ref; i++) { + for (int i = 0; i < _max_num_q * subclasses_of_ref(); i++) { enqueue_discovered_reflist(_discoveredSoftRefs[i], pending_list_addr); _discoveredSoftRefs[i].set_head(sentinel_ref()); _discoveredSoftRefs[i].set_length(0); @@ -453,7 +423,25 @@ inline void remove(); // Make the Reference object active again. - inline void make_active() { java_lang_ref_Reference::set_next(_ref, NULL); } + inline void make_active() { + // For G1 we don't want to use set_next - it + // will dirty the card for the next field of + // the reference object and will fail + // CT verification. + if (UseG1GC) { + BarrierSet* bs = oopDesc::bs(); + HeapWord* next_addr = java_lang_ref_Reference::next_addr(_ref); + + if (UseCompressedOops) { + bs->write_ref_field_pre((narrowOop*)next_addr, NULL); + } else { + bs->write_ref_field_pre((oop*)next_addr, NULL); + } + java_lang_ref_Reference::set_next_raw(_ref, NULL); + } else { + java_lang_ref_Reference::set_next(_ref, NULL); + } + } // Make the referent alive. inline void make_referent_alive() { @@ -495,9 +483,11 @@ oop _referent; OopClosure* _keep_alive; BoolObjectClosure* _is_alive; + DEBUG_ONLY( oop _first_seen; // cyclic linked list check ) + NOT_PRODUCT( size_t _processed; size_t _removed; @@ -506,8 +496,8 @@ inline DiscoveredListIterator::DiscoveredListIterator(DiscoveredList& refs_list, OopClosure* keep_alive, - BoolObjectClosure* is_alive) - : _refs_list(refs_list), + BoolObjectClosure* is_alive) : + _refs_list(refs_list), _prev_next(refs_list.adr_head()), _ref(refs_list.head()), #ifdef ASSERT @@ -745,10 +735,9 @@ void ReferenceProcessor::abandon_partial_discovery() { // loop over the lists - for (int i = 0; i < _max_num_q * subclasses_of_ref; i++) { + for (int i = 0; i < _max_num_q * subclasses_of_ref(); i++) { if (TraceReferenceGC && PrintGCDetails && ((i % _max_num_q) == 0)) { - gclog_or_tty->print_cr("\nAbandoning %s discovered list", - list_name(i)); + gclog_or_tty->print_cr("\nAbandoning %s discovered list", list_name(i)); } abandon_partial_discovered_list(_discoveredSoftRefs[i]); } @@ -867,7 +856,14 @@ move_tail = new_head; new_head = java_lang_ref_Reference::discovered(new_head); } - java_lang_ref_Reference::set_discovered(move_tail, ref_lists[to_idx].head()); + + if (_discovered_list_needs_barrier) { + java_lang_ref_Reference::set_discovered(move_tail, ref_lists[to_idx].head()); + } else { + HeapWord* const discovered_addr = java_lang_ref_Reference::discovered_addr(move_tail); + oop_store_raw(discovered_addr, ref_lists[to_idx].head()); + } + ref_lists[to_idx].set_head(move_head); ref_lists[to_idx].inc_length(refs_to_move); ref_lists[from_idx].set_head(new_head); @@ -981,10 +977,10 @@ void ReferenceProcessor::clean_up_discovered_references() { // loop over the lists // Should this instead be - // for (int i = 0; i < subclasses_of_ref; i++_ { + // for (int i = 0; i < subclasses_of_ref(); i++) { // for (int j = 0; j < _num_q; j++) { // int index = i * _max_num_q + j; - for (int i = 0; i < _max_num_q * subclasses_of_ref; i++) { + for (int i = 0; i < _max_num_q * subclasses_of_ref(); i++) { if (TraceReferenceGC && PrintGCDetails && ((i % _max_num_q) == 0)) { gclog_or_tty->print_cr( "\nScrubbing %s discovered list of Null referents", @@ -1205,6 +1201,8 @@ } } + ResourceMark rm; // Needed for tracing. + HeapWord* const discovered_addr = java_lang_ref_Reference::discovered_addr(obj); const oop discovered = java_lang_ref_Reference::discovered(obj); assert(discovered->is_oop_or_null(), "bad discovered field"); @@ -1419,7 +1417,7 @@ } const char* ReferenceProcessor::list_name(int i) { - assert(i >= 0 && i <= _max_num_q * subclasses_of_ref, "Out of bounds index"); + assert(i >= 0 && i <= _max_num_q * subclasses_of_ref(), "Out of bounds index"); int j = i / _max_num_q; switch (j) { case 0: return "SoftRef"; @@ -1444,7 +1442,7 @@ #ifndef PRODUCT void ReferenceProcessor::clear_discovered_references() { guarantee(!_discovering_refs, "Discovering refs?"); - for (int i = 0; i < _max_num_q * subclasses_of_ref; i++) { + for (int i = 0; i < _max_num_q * subclasses_of_ref(); i++) { oop obj = _discoveredSoftRefs[i].head(); while (obj != sentinel_ref()) { oop next = java_lang_ref_Reference::discovered(obj); --- old/src/share/vm/memory/referenceProcessor.hpp 2011-06-23 14:03:19.931726289 -0700 +++ new/src/share/vm/memory/referenceProcessor.hpp 2011-06-23 14:03:19.599515588 -0700 @@ -48,18 +48,56 @@ // forward references class ReferencePolicy; class AbstractRefProcTaskExecutor; -class DiscoveredList; + +// List of discovered references. +class DiscoveredList { + public: + DiscoveredList() : _len(0), _compressed_head(0), _oop_head(NULL) { } + oop head() const { + return UseCompressedOops ? oopDesc::decode_heap_oop_not_null(_compressed_head) : + _oop_head; + } + HeapWord* adr_head() { + return UseCompressedOops ? (HeapWord*)&_compressed_head : + (HeapWord*)&_oop_head; + } + void set_head(oop o) { + if (UseCompressedOops) { + // Must compress the head ptr. + _compressed_head = oopDesc::encode_heap_oop_not_null(o); + } else { + _oop_head = o; + } + } + + bool empty() const; + + size_t length() { return _len; } + void set_length(size_t len) { _len = len; } + void inc_length(size_t inc) { _len += inc; assert(_len > 0, "Error"); } + void dec_length(size_t dec) { _len -= dec; } + + private: + // Set value depending on UseCompressedOops. This could be a template class + // but then we have to fix all the instantiations and declarations that use this class. + oop _oop_head; + narrowOop _compressed_head; + size_t _len; +}; class ReferenceProcessor : public CHeapObj { protected: // End of list marker static oop _sentinelRef; - MemRegion _span; // (right-open) interval of heap - // subject to wkref discovery - bool _discovering_refs; // true when discovery enabled - bool _discovery_is_atomic; // if discovery is atomic wrt - // other collectors in configuration - bool _discovery_is_mt; // true if reference discovery is MT. + + MemRegion _span; // (right-open) interval of heap + // subject to wkref discovery + + bool _discovering_refs; // true when discovery enabled + bool _discovery_is_atomic; // if discovery is atomic wrt + // other collectors in configuration + bool _discovery_is_mt; // true if reference discovery is MT. + // If true, setting "next" field of a discovered refs list requires // write barrier(s). (Must be true if used in a collector in which // elements of a discovered list may be moved during discovery: for @@ -67,18 +105,19 @@ // long-term concurrent marking phase that does weak reference // discovery.) bool _discovered_list_needs_barrier; - BarrierSet* _bs; // Cached copy of BarrierSet. - bool _enqueuing_is_done; // true if all weak references enqueued - bool _processing_is_mt; // true during phases when - // reference processing is MT. - int _next_id; // round-robin mod _num_q counter in - // support of work distribution - // For collectors that do not keep GC marking information + BarrierSet* _bs; // Cached copy of BarrierSet. + bool _enqueuing_is_done; // true if all weak references enqueued + bool _processing_is_mt; // true during phases when + // reference processing is MT. + int _next_id; // round-robin mod _num_q counter in + // support of work distribution + + // For collectors that do not keep GC liveness information // in the object header, this field holds a closure that // helps the reference processor determine the reachability // of an oop (the field is currently initialized to NULL for - // all collectors but the CMS collector). + // all collectors but the CMS G1 collectors). BoolObjectClosure* _is_alive_non_header; // Soft ref clearing policies @@ -102,6 +141,8 @@ DiscoveredList* _discoveredPhantomRefs; public: + static int subclasses_of_ref() { return (REF_PHANTOM - REF_OTHER); } + int num_q() { return _num_q; } int max_num_q() { return _max_num_q; } void set_active_mt_degree(int v) { _num_q = v; } --- old/src/share/vm/runtime/thread.cpp 2011-06-23 14:03:22.400832090 -0700 +++ new/src/share/vm/runtime/thread.cpp 2011-06-23 14:03:21.865728915 -0700 @@ -749,8 +749,9 @@ jint thread_parity = _oops_do_parity; if (thread_parity != strong_roots_parity) { jint res = Atomic::cmpxchg(strong_roots_parity, &_oops_do_parity, thread_parity); - if (res == thread_parity) return true; - else { + if (res == thread_parity) { + return true; + } else { guarantee(res == strong_roots_parity, "Or else what?"); assert(SharedHeap::heap()->n_par_threads() > 0, "Should only fail when parallel."); @@ -3866,8 +3867,9 @@ } } VMThread* vmt = VMThread::vm_thread(); - if (vmt->claim_oops_do(is_par, cp)) + if (vmt->claim_oops_do(is_par, cp)) { vmt->oops_do(f, cf); + } } #ifndef SERIALGC