# HG changeset patch # User tschatzl # Date 1525349340 -7200 # Node ID 99e698e94cc79289f795b82596d1cc683a9024c3 # Parent 93a7cbd3f8383090582662ce1a34ef2d59c1c318 8201492: Properly implement non-contiguous generations for Reference discovery Summary: Collectors like G1 implementing non-contiguous generations previously used an inexact but conservative area for discovery. Concurrent and STW reference processing could discover the same reference multiple times, potentially missing referents during evacuation. So these collectors had to take extra measures while concurrent marking/reference discovery has been running. This change makes discovery exact for G1 (and any collector using non-contiguous generations) so that concurrent discovery and STW discovery discover on strictly disjoint memory areas. This means that the mentioned situation can not occur any more, and extra work is not required any more too. Reviewed-by: kbarrett, sjohanss diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.cpp --- a/src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.cpp Thu May 03 14:09:00 2018 +0200 @@ -290,13 +290,13 @@ if (_ref_processor == NULL) { // Allocate and initialize a reference processor _ref_processor = - new ReferenceProcessor(_span, // span + new ReferenceProcessor(&_span_based_discoverer, (ParallelGCThreads > 1) && ParallelRefProcEnabled, // mt processing - ParallelGCThreads, // mt processing degree - _cmsGen->refs_discovery_is_mt(), // mt discovery + ParallelGCThreads, // mt processing degree + _cmsGen->refs_discovery_is_mt(), // mt discovery MAX2(ConcGCThreads, ParallelGCThreads), // mt discovery degree - _cmsGen->refs_discovery_is_atomic(), // discovery is not atomic - &_is_alive_closure); // closure for liveness info + _cmsGen->refs_discovery_is_atomic(), // discovery is not atomic + &_is_alive_closure); // closure for liveness info // Initialize the _ref_processor field of CMSGen _cmsGen->set_ref_processor(_ref_processor); @@ -445,7 +445,10 @@ CardTableRS* ct, ConcurrentMarkSweepPolicy* cp): _cmsGen(cmsGen), + // Adjust span to cover old (cms) gen + _span(cmsGen->reserved()), _ct(ct), + _span_based_discoverer(_span), _ref_processor(NULL), // will be set later _conc_workers(NULL), // may be set later _abort_preclean(false), @@ -455,8 +458,6 @@ _modUnionTable((CardTable::card_shift - LogHeapWordSize), -1 /* lock-free */, "No_lock" /* dummy */), _modUnionClosurePar(&_modUnionTable), - // Adjust my span to cover old (cms) gen - _span(cmsGen->reserved()), // Construct the is_alive_closure with _span & markBitMap _is_alive_closure(_span, &_markBitMap), _restart_addr(NULL), @@ -3744,7 +3745,6 @@ } } - size_t CMSCollector::preclean_work(bool clean_refs, bool clean_survivor) { assert(_collectorState == Precleaning || _collectorState == AbortablePreclean, "incorrect state"); @@ -3761,7 +3761,7 @@ // referents. if (clean_refs) { CMSPrecleanRefsYieldClosure yield_cl(this); - assert(rp->span().equals(_span), "Spans should be equal"); + assert(_span_based_discoverer.span().equals(_span), "Spans should be equal"); CMSKeepAliveClosure keep_alive(this, _span, &_markBitMap, &_markStack, true /* preclean */); CMSDrainMarkingStackClosure complete_trace(this, @@ -5153,7 +5153,7 @@ WorkGang* workers = heap->workers(); assert(workers != NULL, "Need parallel worker threads."); CMSRefProcTaskProxy rp_task(task, &_collector, - _collector.ref_processor()->span(), + _collector.ref_processor_span(), _collector.markBitMap(), workers, _collector.task_queues()); workers->run_task(&rp_task); @@ -5174,7 +5174,7 @@ HandleMark hm; ReferenceProcessor* rp = ref_processor(); - assert(rp->span().equals(_span), "Spans should be equal"); + assert(_span_based_discoverer.span().equals(_span), "Spans should be equal"); assert(!rp->enqueuing_is_done(), "Enqueuing should not be complete"); // Process weak references. rp->setup_policy(false); diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.hpp --- a/src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.hpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.hpp Thu May 03 14:09:00 2018 +0200 @@ -617,7 +617,7 @@ protected: ConcurrentMarkSweepGeneration* _cmsGen; // Old gen (CMS) - MemRegion _span; // Span covering above two + MemRegion _span; // Span covering above CardTableRS* _ct; // Card table // CMS marking support structures @@ -641,8 +641,9 @@ NOT_PRODUCT(ssize_t _num_par_pushes;) // ("Weak") Reference processing support. - ReferenceProcessor* _ref_processor; - CMSIsAliveClosure _is_alive_closure; + SpanSubjectToDiscoveryClosure _span_based_discoverer; + ReferenceProcessor* _ref_processor; + CMSIsAliveClosure _is_alive_closure; // Keep this textually after _markBitMap and _span; c'tor dependency. ConcurrentMarkSweepThread* _cmsThread; // The thread doing the work @@ -841,6 +842,7 @@ ConcurrentMarkSweepPolicy* cp); ConcurrentMarkSweepThread* cmsThread() { return _cmsThread; } + MemRegion ref_processor_span() const { return _span_based_discoverer.span(); } ReferenceProcessor* ref_processor() { return _ref_processor; } void ref_processor_init(); diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/cms/parNewGeneration.cpp --- a/src/hotspot/share/gc/cms/parNewGeneration.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/cms/parNewGeneration.cpp Thu May 03 14:09:00 2018 +0200 @@ -1471,8 +1471,9 @@ void ParNewGeneration::ref_processor_init() { if (_ref_processor == NULL) { // Allocate and initialize a reference processor + _span_based_discoverer.set_span(_reserved); _ref_processor = - new ReferenceProcessor(_reserved, // span + new ReferenceProcessor(&_span_based_discoverer, // span ParallelRefProcEnabled && (ParallelGCThreads > 1), // mt processing ParallelGCThreads, // mt processing degree refs_discovery_is_mt(), // mt discovery diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/g1/g1CollectedHeap.cpp --- a/src/hotspot/share/gc/g1/g1CollectedHeap.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/g1/g1CollectedHeap.cpp Thu May 03 14:09:00 2018 +0200 @@ -1410,10 +1410,12 @@ _g1_policy(new G1Policy(_gc_timer_stw)), _collection_set(this, _g1_policy), _dirty_card_queue_set(false), - _is_alive_closure_cm(this), + _ref_processor_stw(NULL), _is_alive_closure_stw(this), + _is_subject_to_discovery_stw(this), _ref_processor_cm(NULL), - _ref_processor_stw(NULL), + _is_alive_closure_cm(this), + _is_subject_to_discovery_cm(this), _bot(NULL), _hot_card_cache(NULL), _g1_rem_set(NULL), @@ -1786,43 +1788,27 @@ // * Discovery is atomic - i.e. not concurrent. // * Reference discovery will not need a barrier. - MemRegion mr = reserved_region(); - bool mt_processing = ParallelRefProcEnabled && (ParallelGCThreads > 1); // Concurrent Mark ref processor _ref_processor_cm = - new ReferenceProcessor(mr, // span - mt_processing, - // mt processing - ParallelGCThreads, - // degree of mt processing - (ParallelGCThreads > 1) || (ConcGCThreads > 1), - // mt discovery - MAX2(ParallelGCThreads, ConcGCThreads), - // degree of mt discovery - false, - // Reference discovery is not atomic - &_is_alive_closure_cm); - // is alive closure - // (for efficiency/performance) + new ReferenceProcessor(&_is_subject_to_discovery_cm, + mt_processing, // mt processing + ParallelGCThreads, // degree of mt processing + (ParallelGCThreads > 1) || (ConcGCThreads > 1), // mt discovery + MAX2(ParallelGCThreads, ConcGCThreads), // degree of mt discovery + false, // Reference discovery is not atomic + &_is_alive_closure_cm); // is alive closure // STW ref processor _ref_processor_stw = - new ReferenceProcessor(mr, // span - mt_processing, - // mt processing - ParallelGCThreads, - // degree of mt processing - (ParallelGCThreads > 1), - // mt discovery - ParallelGCThreads, - // degree of mt discovery - true, - // Reference discovery is atomic - &_is_alive_closure_stw); - // is alive closure - // (for efficiency/performance) + new ReferenceProcessor(&_is_subject_to_discovery_stw, + mt_processing, // mt processing + ParallelGCThreads, // degree of mt processing + (ParallelGCThreads > 1), // mt discovery + ParallelGCThreads, // degree of mt discovery + true, // Reference discovery is atomic + &_is_alive_closure_stw); // is alive closure } CollectorPolicy* G1CollectedHeap::collector_policy() const { @@ -3642,26 +3628,21 @@ // Weak Reference Processing support -// An always "is_alive" closure that is used to preserve referents. -// If the object is non-null then it's alive. Used in the preservation -// of referent objects that are pointed to by reference objects -// discovered by the CM ref processor. -class G1AlwaysAliveClosure: public BoolObjectClosure { -public: - bool do_object_b(oop p) { - if (p != NULL) { - return true; - } - return false; - } -}; - bool G1STWIsAliveClosure::do_object_b(oop p) { // An object is reachable if it is outside the collection set, // or is inside and copied. return !_g1h->is_in_cset(p) || p->is_forwarded(); } +bool G1STWSubjectToDiscoveryClosure::do_object_b(oop obj) { + assert(obj != NULL, "must not be NULL"); + assert(_g1h->is_in_reserved(obj), "Trying to discover obj " PTR_FORMAT " not in heap", p2i(obj)); + // The areas the CM and STW ref processor manage must be disjoint. The is_in_cset() below + // may falsely indicate that this is not the case here: however the collection set only + // contains old regions when concurrent mark is not running. + return _g1h->is_in_cset(obj) || _g1h->heap_region_containing(obj)->is_survivor(); +} + // Non Copying Keep Alive closure class G1KeepAliveClosure: public OopClosure { G1CollectedHeap*_g1h; @@ -3892,126 +3873,6 @@ // 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; - G1ParScanThreadStateSet* _pss; - RefToScanQueueSet* _queues; - ParallelTaskTerminator _terminator; - uint _n_workers; - -public: - G1ParPreserveCMReferentsTask(G1CollectedHeap* g1h, G1ParScanThreadStateSet* per_thread_states, int workers, RefToScanQueueSet *task_queues) : - AbstractGangTask("ParPreserveCMReferents"), - _g1h(g1h), - _pss(per_thread_states), - _queues(task_queues), - _terminator(workers, _queues), - _n_workers(workers) - { - g1h->ref_processor_cm()->set_active_mt_degree(workers); - } - - void work(uint worker_id) { - G1GCParPhaseTimesTracker x(_g1h->g1_policy()->phase_times(), G1GCPhaseTimes::PreserveCMReferents, worker_id); - - ResourceMark rm; - HandleMark hm; - - G1ParScanThreadState* pss = _pss->state_for_worker(worker_id); - pss->set_ref_discoverer(NULL); - assert(pss->queue_is_empty(), "both queue and overflow should be empty"); - - // Is alive closure - G1AlwaysAliveClosure always_alive; - - // Copying keep alive closure. Applied to referent objects that need - // to be copied. - G1CopyingKeepAliveClosure keep_alive(_g1h, pss->closures()->raw_strong_oops(), pss); - - ReferenceProcessor* rp = _g1h->ref_processor_cm(); - - uint limit = ReferenceProcessor::number_of_subclasses_of_ref() * rp->max_num_q(); - uint stride = MIN2(MAX2(_n_workers, 1U), limit); - - // 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(worker_id < limit, "sanity"); - assert(!rp->discovery_is_atomic(), "check this code"); - - // Select discovered lists [i, i+stride, i+2*stride,...,limit) - for (uint idx = worker_id; idx < limit; idx += stride) { - DiscoveredList& ref_list = rp->discovered_refs()[idx]; - - DiscoveredListIterator iter(ref_list, &keep_alive, &always_alive); - while (iter.has_next()) { - // Since discovery is not atomic for the CM ref processor, we - // can see some null referent objects. - iter.load_ptrs(DEBUG_ONLY(true)); - oop ref = iter.obj(); - - // This will filter nulls. - if (iter.is_referent_alive()) { - iter.make_referent_alive(); - } - iter.move_to_next(); - } - } - - // 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->queue_is_empty(), "should be"); - } -}; - -void G1CollectedHeap::preserve_cm_referents(G1ParScanThreadStateSet* per_thread_states) { - // 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 result the copy closure would not have been applied to the - // referent object. - // - // We need to explicitly copy these referent objects - the references - // will be processed at the end of remarking. - // - // We also need to do this copying before we process the reference - // objects discovered by the STW ref processor in case one of these - // referents points to another object which is also referenced by an - // object discovered by the STW ref processor. - double preserve_cm_referents_time = 0.0; - - // To avoid spawning task when there is no work to do, check that - // a concurrent cycle is active and that some references have been - // discovered. - if (concurrent_mark()->cm_thread()->during_cycle() && - ref_processor_cm()->has_discovered_references()) { - double preserve_cm_referents_start = os::elapsedTime(); - uint no_of_gc_workers = workers()->active_workers(); - G1ParPreserveCMReferentsTask keep_cm_referents(this, - per_thread_states, - no_of_gc_workers, - _task_queues); - workers()->run_task(&keep_cm_referents); - preserve_cm_referents_time = os::elapsedTime() - preserve_cm_referents_start; - } - - g1_policy()->phase_times()->record_preserve_cm_referents_time_ms(preserve_cm_referents_time * 1000.0); -} - // Weak Reference processing during an evacuation pause (part 1). void G1CollectedHeap::process_discovered_references(G1ParScanThreadStateSet* per_thread_states) { double ref_proc_start = os::elapsedTime(); @@ -4197,7 +4058,6 @@ // as we may have to copy some 'reachable' referent // objects (and their reachable sub-graphs) that were // not copied during the pause. - preserve_cm_referents(per_thread_states); process_discovered_references(per_thread_states); G1STWIsAliveClosure is_alive(this); diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/g1/g1CollectedHeap.hpp --- a/src/hotspot/share/gc/g1/g1CollectedHeap.hpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/g1/g1CollectedHeap.hpp Thu May 03 14:09:00 2018 +0200 @@ -107,13 +107,20 @@ // (optional) _is_alive_non_header closure in the STW // reference processor. It is also extensively used during // reference processing during STW evacuation pauses. -class G1STWIsAliveClosure: public BoolObjectClosure { +class G1STWIsAliveClosure : public BoolObjectClosure { G1CollectedHeap* _g1h; public: G1STWIsAliveClosure(G1CollectedHeap* g1h) : _g1h(g1h) {} bool do_object_b(oop p); }; +class G1STWSubjectToDiscoveryClosure : public BoolObjectClosure { + G1CollectedHeap* _g1h; +public: + G1STWSubjectToDiscoveryClosure(G1CollectedHeap* g1h) : _g1h(g1h) {} + bool do_object_b(oop p); +}; + class G1RegionMappingChangedListener : public G1MappingChangedListener { private: void reset_from_card_cache(uint start_idx, size_t num_regions); @@ -506,9 +513,6 @@ // allocated block, or else "NULL". HeapWord* expand_and_allocate(size_t word_size); - // Preserve any referents discovered by concurrent marking that have not yet been - // copied by the STW pause. - void preserve_cm_referents(G1ParScanThreadStateSet* per_thread_states); // Process any reference objects discovered during // an incremental evacuation pause. void process_discovered_references(G1ParScanThreadStateSet* per_thread_states); @@ -897,6 +901,8 @@ // the discovered lists during reference discovery. G1STWIsAliveClosure _is_alive_closure_stw; + G1STWSubjectToDiscoveryClosure _is_subject_to_discovery_stw; + // The (concurrent marking) reference processor... ReferenceProcessor* _ref_processor_cm; @@ -908,6 +914,7 @@ // discovery. G1CMIsAliveClosure _is_alive_closure_cm; + G1CMSubjectToDiscoveryClosure _is_subject_to_discovery_cm; public: RefToScanQueue *task_queue(uint i) const; diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/g1/g1ConcurrentMark.cpp --- a/src/hotspot/share/gc/g1/g1ConcurrentMark.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/g1/g1ConcurrentMark.cpp Thu May 03 14:09:00 2018 +0200 @@ -1389,15 +1389,6 @@ } } -// Supporting Object and Oop closures for reference discovery -// and processing in during marking - -bool G1CMIsAliveClosure::do_object_b(oop obj) { - HeapWord* addr = (HeapWord*)obj; - return addr != NULL && - (!_g1h->is_in_g1_reserved(addr) || !_g1h->is_obj_ill(obj)); -} - // 'Keep Alive' oop closure used by both serial parallel reference processing. // Uses the G1CMTask associated with a worker thread (for serial reference // processing the G1CMTask for worker 0 is used) to preserve (mark) and diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/g1/g1ConcurrentMark.hpp --- a/src/hotspot/share/gc/g1/g1ConcurrentMark.hpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/g1/g1ConcurrentMark.hpp Thu May 03 14:09:00 2018 +0200 @@ -109,7 +109,13 @@ G1CollectedHeap* _g1h; public: G1CMIsAliveClosure(G1CollectedHeap* g1h) : _g1h(g1h) { } + bool do_object_b(oop obj); +}; +class G1CMSubjectToDiscoveryClosure : public BoolObjectClosure { + G1CollectedHeap* _g1h; +public: + G1CMSubjectToDiscoveryClosure(G1CollectedHeap* g1h) : _g1h(g1h) { } bool do_object_b(oop obj); }; diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/g1/g1ConcurrentMark.inline.hpp --- a/src/hotspot/share/gc/g1/g1ConcurrentMark.inline.hpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/g1/g1ConcurrentMark.inline.hpp Thu May 03 14:09:00 2018 +0200 @@ -38,6 +38,22 @@ #include "gc/shared/taskqueue.inline.hpp" #include "utilities/bitMap.inline.hpp" +inline bool G1CMIsAliveClosure::do_object_b(oop obj) { + return !_g1h->is_obj_ill(obj); +} + +inline bool G1CMSubjectToDiscoveryClosure::do_object_b(oop obj) { + // Re-check whether the passed object is null. With ReferentBasedDiscovery the + // mutator may have changed the referent's value (i.e. cleared it) between the + // time the referent was determined to be potentially alive and calling this + // method. + if (obj == NULL) { + return false; + } + assert(_g1h->is_in_reserved(obj), "Trying to discover obj " PTR_FORMAT " not in heap", p2i(obj)); + return _g1h->heap_region_containing(obj)->is_old_or_humongous(); +} + inline bool G1ConcurrentMark::mark_in_next_bitmap(uint const worker_id, oop const obj, size_t const obj_size) { HeapRegion* const hr = _g1h->heap_region_containing(obj); return mark_in_next_bitmap(worker_id, hr, obj, obj_size); diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/g1/g1FullCollector.cpp --- a/src/hotspot/share/gc/g1/g1FullCollector.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/g1/g1FullCollector.cpp Thu May 03 14:09:00 2018 +0200 @@ -112,7 +112,9 @@ _preserved_marks_set(true), _serial_compaction_point(), _is_alive(heap->concurrent_mark()->next_mark_bitmap()), - _is_alive_mutator(heap->ref_processor_stw(), &_is_alive) { + _is_alive_mutator(heap->ref_processor_stw(), &_is_alive), + _always_subject_to_discovery(), + _is_subject_mutator(heap->ref_processor_stw(), &_always_subject_to_discovery) { assert(SafepointSynchronize::is_at_safepoint(), "must be at a safepoint"); _preserved_marks_set.init(_num_workers); diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/g1/g1FullCollector.hpp --- a/src/hotspot/share/gc/g1/g1FullCollector.hpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/g1/g1FullCollector.hpp Thu May 03 14:09:00 2018 +0200 @@ -42,6 +42,16 @@ class GCMemoryManager; class ReferenceProcessor; +// Subject-to-discovery closure for reference processing during Full GC. During +// Full GC the whole heap is subject to discovery. +class G1FullGCSubjectToDiscoveryClosure: public BoolObjectClosure { +public: + bool do_object_b(oop p) { + assert(p != NULL, "must be"); + return true; + } +}; + // The G1FullCollector holds data associated with the current Full GC. class G1FullCollector : StackObj { G1CollectedHeap* _heap; @@ -58,6 +68,9 @@ static uint calc_active_workers(); + G1FullGCSubjectToDiscoveryClosure _always_subject_to_discovery; + ReferenceProcessorSubjectToDiscoveryMutator _is_subject_mutator; + public: G1FullCollector(G1CollectedHeap* heap, GCMemoryManager* memory_manager, bool explicit_gc, bool clear_soft_refs); ~G1FullCollector(); diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/g1/g1GCPhaseTimes.cpp --- a/src/hotspot/share/gc/g1/g1GCPhaseTimes.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/g1/g1GCPhaseTimes.cpp Thu May 03 14:09:00 2018 +0200 @@ -113,8 +113,6 @@ _gc_par_phases[YoungFreeCSet] = new WorkerDataArray(max_gc_threads, "Young Free Collection Set (ms):"); _gc_par_phases[NonYoungFreeCSet] = new WorkerDataArray(max_gc_threads, "Non-Young Free Collection Set (ms):"); - _gc_par_phases[PreserveCMReferents] = new WorkerDataArray(max_gc_threads, "Parallel Preserve CM Refs (ms):"); - reset(); } @@ -399,9 +397,6 @@ debug_time("Code Roots Fixup", _cur_collection_code_root_fixup_time_ms); - debug_time("Preserve CM Refs", _recorded_preserve_cm_referents_time_ms); - trace_phase(_gc_par_phases[PreserveCMReferents]); - debug_time_for_reference("Reference Processing", _cur_ref_proc_time_ms); _ref_phase_times.print_all_references(2, false); diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/g1/g1GCPhaseTimes.hpp --- a/src/hotspot/share/gc/g1/g1GCPhaseTimes.hpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/g1/g1GCPhaseTimes.hpp Thu May 03 14:09:00 2018 +0200 @@ -73,7 +73,6 @@ StringDedupQueueFixup, StringDedupTableFixup, RedirtyCards, - PreserveCMReferents, YoungFreeCSet, NonYoungFreeCSet, GCParPhasesSentinel diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/parallel/asPSYoungGen.cpp --- a/src/hotspot/share/gc/parallel/asPSYoungGen.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/parallel/asPSYoungGen.cpp Thu May 03 14:09:00 2018 +0200 @@ -496,7 +496,7 @@ _reserved = MemRegion((HeapWord*)virtual_space()->low_boundary(), (HeapWord*)virtual_space()->high_boundary()); - PSScavenge::reference_processor()->set_span(_reserved); + PSScavenge::set_subject_to_discovery_span(_reserved); HeapWord* new_eden_bottom = (HeapWord*)virtual_space()->low(); HeapWord* eden_bottom = eden_space()->bottom(); diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/parallel/psMarkSweep.cpp --- a/src/hotspot/share/gc/parallel/psMarkSweep.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/parallel/psMarkSweep.cpp Thu May 03 14:09:00 2018 +0200 @@ -65,9 +65,11 @@ jlong PSMarkSweep::_time_of_last_gc = 0; CollectorCounters* PSMarkSweep::_counters = NULL; +SpanSubjectToDiscoveryClosure PSMarkSweep::_span_based_discoverer; + void PSMarkSweep::initialize() { - MemRegion mr = ParallelScavengeHeap::heap()->reserved_region(); - set_ref_processor(new ReferenceProcessor(mr)); // a vanilla ref proc + _span_based_discoverer.set_span(ParallelScavengeHeap::heap()->reserved_region()); + set_ref_processor(new ReferenceProcessor(&_span_based_discoverer)); // a vanilla ref proc _counters = new CollectorCounters("PSMarkSweep", 1); } diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/parallel/psMarkSweep.hpp --- a/src/hotspot/share/gc/parallel/psMarkSweep.hpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/parallel/psMarkSweep.hpp Thu May 03 14:09:00 2018 +0200 @@ -27,6 +27,7 @@ #include "gc/serial/markSweep.hpp" #include "gc/shared/collectorCounters.hpp" +#include "gc/shared/referenceProcessor.hpp" #include "utilities/stack.hpp" class PSAdaptiveSizePolicy; @@ -39,6 +40,8 @@ static jlong _time_of_last_gc; // ms static CollectorCounters* _counters; + static SpanSubjectToDiscoveryClosure _span_based_discoverer; + // Closure accessors static OopClosure* mark_and_push_closure() { return &MarkSweep::mark_and_push_closure; } static VoidClosure* follow_stack_closure() { return &MarkSweep::follow_stack_closure; } diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/parallel/psParallelCompact.cpp --- a/src/hotspot/share/gc/parallel/psParallelCompact.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/parallel/psParallelCompact.cpp Thu May 03 14:09:00 2018 +0200 @@ -117,6 +117,7 @@ SpaceInfo PSParallelCompact::_space_info[PSParallelCompact::last_space_id]; +SpanSubjectToDiscoveryClosure PSParallelCompact::_span_based_discoverer; ReferenceProcessor* PSParallelCompact::_ref_processor = NULL; double PSParallelCompact::_dwl_mean; @@ -843,14 +844,14 @@ void PSParallelCompact::post_initialize() { ParallelScavengeHeap* heap = ParallelScavengeHeap::heap(); - MemRegion mr = heap->reserved_region(); + _span_based_discoverer.set_span(heap->reserved_region()); _ref_processor = - new ReferenceProcessor(mr, // span + new ReferenceProcessor(&_span_based_discoverer, ParallelRefProcEnabled && (ParallelGCThreads > 1), // mt processing - ParallelGCThreads, // mt processing degree - true, // mt discovery - ParallelGCThreads, // mt discovery degree - true, // atomic_discovery + ParallelGCThreads, // mt processing degree + true, // mt discovery + ParallelGCThreads, // mt discovery degree + true, // atomic_discovery &_is_alive_closure); // non-header is alive closure _counters = new CollectorCounters("PSParallelCompact", 1); diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/parallel/psParallelCompact.hpp --- a/src/hotspot/share/gc/parallel/psParallelCompact.hpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/parallel/psParallelCompact.hpp Thu May 03 14:09:00 2018 +0200 @@ -968,6 +968,7 @@ static SpaceInfo _space_info[last_space_id]; // Reference processing (used in ...follow_contents) + static SpanSubjectToDiscoveryClosure _span_based_discoverer; static ReferenceProcessor* _ref_processor; // Values computed at initialization and used by dead_wood_limiter(). diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/parallel/psScavenge.cpp --- a/src/hotspot/share/gc/parallel/psScavenge.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/parallel/psScavenge.cpp Thu May 03 14:09:00 2018 +0200 @@ -58,18 +58,19 @@ #include "services/memoryService.hpp" #include "utilities/stack.inline.hpp" -HeapWord* PSScavenge::_to_space_top_before_gc = NULL; -int PSScavenge::_consecutive_skipped_scavenges = 0; -ReferenceProcessor* PSScavenge::_ref_processor = NULL; -PSCardTable* PSScavenge::_card_table = NULL; -bool PSScavenge::_survivor_overflow = false; -uint PSScavenge::_tenuring_threshold = 0; -HeapWord* PSScavenge::_young_generation_boundary = NULL; -uintptr_t PSScavenge::_young_generation_boundary_compressed = 0; -elapsedTimer PSScavenge::_accumulated_time; -STWGCTimer PSScavenge::_gc_timer; -ParallelScavengeTracer PSScavenge::_gc_tracer; -CollectorCounters* PSScavenge::_counters = NULL; +HeapWord* PSScavenge::_to_space_top_before_gc = NULL; +int PSScavenge::_consecutive_skipped_scavenges = 0; +SpanSubjectToDiscoveryClosure PSScavenge::_span_based_discoverer; +ReferenceProcessor* PSScavenge::_ref_processor = NULL; +PSCardTable* PSScavenge::_card_table = NULL; +bool PSScavenge::_survivor_overflow = false; +uint PSScavenge::_tenuring_threshold = 0; +HeapWord* PSScavenge::_young_generation_boundary = NULL; +uintptr_t PSScavenge::_young_generation_boundary_compressed = 0; +elapsedTimer PSScavenge::_accumulated_time; +STWGCTimer PSScavenge::_gc_timer; +ParallelScavengeTracer PSScavenge::_gc_tracer; +CollectorCounters* PSScavenge::_counters = NULL; // Define before use class PSIsAliveClosure: public BoolObjectClosure { @@ -766,10 +767,9 @@ set_young_generation_boundary(young_gen->eden_space()->bottom()); // Initialize ref handling object for scavenging. - MemRegion mr = young_gen->reserved(); - + _span_based_discoverer.set_span(young_gen->reserved()); _ref_processor = - new ReferenceProcessor(mr, // span + new ReferenceProcessor(&_span_based_discoverer, ParallelRefProcEnabled && (ParallelGCThreads > 1), // mt processing ParallelGCThreads, // mt processing degree true, // mt discovery diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/parallel/psScavenge.hpp --- a/src/hotspot/share/gc/parallel/psScavenge.hpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/parallel/psScavenge.hpp Thu May 03 14:09:00 2018 +0200 @@ -65,14 +65,15 @@ protected: // Flags/counters - static ReferenceProcessor* _ref_processor; // Reference processor for scavenging. - static PSIsAliveClosure _is_alive_closure; // Closure used for reference processing - static PSCardTable* _card_table; // We cache the card table for fast access. - static bool _survivor_overflow; // Overflow this collection - static uint _tenuring_threshold; // tenuring threshold for next scavenge - static elapsedTimer _accumulated_time; // total time spent on scavenge - static STWGCTimer _gc_timer; // GC time book keeper - static ParallelScavengeTracer _gc_tracer; // GC tracing + static SpanSubjectToDiscoveryClosure _span_based_discoverer; + static ReferenceProcessor* _ref_processor; // Reference processor for scavenging. + static PSIsAliveClosure _is_alive_closure; // Closure used for reference processing + static PSCardTable* _card_table; // We cache the card table for fast access. + static bool _survivor_overflow; // Overflow this collection + static uint _tenuring_threshold; // tenuring threshold for next scavenge + static elapsedTimer _accumulated_time; // total time spent on scavenge + static STWGCTimer _gc_timer; // GC time book keeper + static ParallelScavengeTracer _gc_tracer; // GC tracing // The lowest address possible for the young_gen. // This is used to decide if an oop should be scavenged, // cards should be marked, etc. @@ -102,6 +103,9 @@ // Performance Counters static CollectorCounters* counters() { return _counters; } + static void set_subject_to_discovery_span(MemRegion mr) { + _span_based_discoverer.set_span(mr); + } // Used by scavenge_contents && psMarkSweep static ReferenceProcessor* const reference_processor() { assert(_ref_processor != NULL, "Sanity"); diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/parallel/psYoungGen.cpp --- a/src/hotspot/share/gc/parallel/psYoungGen.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/parallel/psYoungGen.cpp Thu May 03 14:09:00 2018 +0200 @@ -839,7 +839,7 @@ void PSYoungGen::reset_survivors_after_shrink() { _reserved = MemRegion((HeapWord*)virtual_space()->low_boundary(), (HeapWord*)virtual_space()->high_boundary()); - PSScavenge::reference_processor()->set_span(_reserved); + PSScavenge::set_subject_to_discovery_span(_reserved); MutableSpace* space_shrinking = NULL; if (from_space()->end() > to_space()->end()) { diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/shared/generation.cpp --- a/src/hotspot/share/gc/shared/generation.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/shared/generation.cpp Thu May 03 14:09:00 2018 +0200 @@ -77,7 +77,8 @@ void Generation::ref_processor_init() { assert(_ref_processor == NULL, "a reference processor already exists"); assert(!_reserved.is_empty(), "empty generation?"); - _ref_processor = new ReferenceProcessor(_reserved); // a vanilla reference processor + _span_based_discoverer.set_span(_reserved); + _ref_processor = new ReferenceProcessor(&_span_based_discoverer); // a vanilla reference processor if (_ref_processor == NULL) { vm_exit_during_initialization("Could not allocate ReferenceProcessor object"); } diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/shared/generation.hpp --- a/src/hotspot/share/gc/shared/generation.hpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/shared/generation.hpp Thu May 03 14:09:00 2018 +0200 @@ -100,6 +100,7 @@ VirtualSpace _virtual_space; // ("Weak") Reference processing support + SpanSubjectToDiscoveryClosure _span_based_discoverer; ReferenceProcessor* _ref_processor; // Performance Counters diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/shared/referenceProcessor.cpp --- a/src/hotspot/share/gc/shared/referenceProcessor.cpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/shared/referenceProcessor.cpp Thu May 03 14:09:00 2018 +0200 @@ -92,20 +92,22 @@ _discovering_refs = true; } -ReferenceProcessor::ReferenceProcessor(MemRegion span, +ReferenceProcessor::ReferenceProcessor(BoolObjectClosure* is_subject_to_discovery, bool mt_processing, uint mt_processing_degree, bool mt_discovery, uint mt_discovery_degree, bool atomic_discovery, BoolObjectClosure* is_alive_non_header) : + _is_subject_to_discovery(is_subject_to_discovery), _discovering_refs(false), _enqueuing_is_done(false), _is_alive_non_header(is_alive_non_header), _processing_is_mt(mt_processing), _next_id(0) { - _span = span; + assert(is_subject_to_discovery != NULL, "must be set"); + _discovery_is_atomic = atomic_discovery; _discovery_is_mt = mt_discovery; _num_q = MAX2(1U, mt_processing_degree); @@ -449,6 +451,19 @@ iter.removed(), iter.processed(), p2i(&refs_list)); } +void ReferenceProcessor::process_phase2(DiscoveredList& refs_list, + BoolObjectClosure* is_alive, + OopClosure* keep_alive, + VoidClosure* complete_gc) { + if (discovery_is_atomic()) { + // complete_gc is ignored in this case for this phase + pp2_work(refs_list, is_alive, keep_alive); + } else { + assert(complete_gc != NULL, "Error"); + pp2_work_concurrent_discovery(refs_list, is_alive, + keep_alive, complete_gc); + } +} // Traverse the list and remove any Refs that are not active, or // whose referents are either alive or NULL. void @@ -941,6 +956,10 @@ } #endif +bool ReferenceProcessor::is_subject_to_discovery(oop const obj) const { + return _is_subject_to_discovery->do_object_b(obj); +} + // We mention two of several possible choices here: // #0: if the reference object is not in the "originating generation" // (or part of the heap being collected, indicated by our "span" @@ -978,9 +997,8 @@ return false; } - HeapWord* obj_addr = (HeapWord*)obj; if (RefDiscoveryPolicy == ReferenceBasedDiscovery && - !_span.contains(obj_addr)) { + !is_subject_to_discovery(obj)) { // Reference is not in the originating generation; // don't treat it specially (i.e. we want to scan it as a normal // object with strong references). @@ -1039,16 +1057,15 @@ // Discover if and only if EITHER: // .. reference is in our span, OR // .. we are an atomic collector and referent is in our span - if (_span.contains(obj_addr) || + if (is_subject_to_discovery(obj) || (discovery_is_atomic() && - _span.contains(java_lang_ref_Reference::referent(obj)))) { - // should_enqueue = true; + is_subject_to_discovery(java_lang_ref_Reference::referent(obj)))) { } else { return false; } } else { assert(RefDiscoveryPolicy == ReferenceBasedDiscovery && - _span.contains(obj_addr), "code inconsistency"); + is_subject_to_discovery(obj), "code inconsistency"); } // Get the right type of discovered queue head. diff -r 93a7cbd3f838 -r 99e698e94cc7 src/hotspot/share/gc/shared/referenceProcessor.hpp --- a/src/hotspot/share/gc/shared/referenceProcessor.hpp Thu Apr 26 12:54:53 2018 +0200 +++ b/src/hotspot/share/gc/shared/referenceProcessor.hpp Thu May 03 14:09:00 2018 +0200 @@ -38,18 +38,13 @@ // of java.lang.Reference objects for GC. The interface is useful for supporting // a generational abstraction, in particular when there are multiple // generations that are being independently collected -- possibly -// concurrently and/or incrementally. Note, however, that the +// concurrently and/or incrementally. // ReferenceProcessor class abstracts away from a generational setting -// by using only a heap interval (called "span" below), thus allowing -// its use in a straightforward manner in a general, non-generational -// setting. +// by using a closure that determines whether a given reference or referent are +// subject to this ReferenceProcessor's discovery, thus allowing its use in a +// straightforward manner in a general, non-generational, non-contiguous generation +// (or heap) setting. // -// The basic idea is that each ReferenceProcessor object concerns -// itself with ("weak") reference processing in a specific "span" -// of the heap of interest to a specific collector. Currently, -// the span is a convex interval of the heap, but, efficiency -// apart, there seems to be no reason it couldn't be extended -// (with appropriate modifications) to any "non-convex interval". // forward references class ReferencePolicy; @@ -168,16 +163,14 @@ }; class ReferenceProcessor : public ReferenceDiscoverer { - - private: size_t total_count(DiscoveredList lists[]) const; - protected: // The SoftReference master timestamp clock static jlong _soft_ref_timestamp_clock; - MemRegion _span; // (right-open) interval of heap - // subject to wkref discovery + BoolObjectClosure* _is_subject_to_discovery; // determines whether a given oop is subject + // to this ReferenceProcessor's discovery + // (and further processing). bool _discovering_refs; // true when discovery enabled bool _discovery_is_atomic; // if discovery is atomic wrt @@ -257,19 +250,10 @@ VoidClosure* complete_gc); // Phase2: remove all those references whose referents are // reachable. - inline void process_phase2(DiscoveredList& refs_list, - BoolObjectClosure* is_alive, - OopClosure* keep_alive, - VoidClosure* complete_gc) { - if (discovery_is_atomic()) { - // complete_gc is ignored in this case for this phase - pp2_work(refs_list, is_alive, keep_alive); - } else { - assert(complete_gc != NULL, "Error"); - pp2_work_concurrent_discovery(refs_list, is_alive, - keep_alive, complete_gc); - } - } + void process_phase2(DiscoveredList& refs_list, + BoolObjectClosure* is_alive, + OopClosure* keep_alive, + VoidClosure* complete_gc); // Work methods in support of process_phase2 void pp2_work(DiscoveredList& refs_list, BoolObjectClosure* is_alive, @@ -312,7 +296,6 @@ void enqueue_discovered_reflists(AbstractRefProcTaskExecutor* task_executor, ReferenceProcessorPhaseTimes* phase_times); - protected: // "Preclean" the given discovered reference list // by removing references with strongly reachable referents. // Currently used in support of CMS only. @@ -321,7 +304,7 @@ OopClosure* keep_alive, VoidClosure* complete_gc, YieldClosure* yield); - +private: // round-robin mod _num_q (not: _not_ mode _max_num_q) uint next_id() { uint id = _next_id; @@ -346,9 +329,11 @@ // Update (advance) the soft ref master clock field. void update_soft_ref_master_clock(); - public: + bool is_subject_to_discovery(oop const obj) const; + +public: // Default parameters give you a vanilla reference processor. - ReferenceProcessor(MemRegion span, + ReferenceProcessor(BoolObjectClosure* is_subject_to_discovery, bool mt_processing = false, uint mt_processing_degree = 1, bool mt_discovery = false, uint mt_discovery_degree = 1, bool atomic_discovery = true, @@ -373,9 +358,8 @@ _is_alive_non_header = is_alive_non_header; } - // get and set span - MemRegion span() { return _span; } - void set_span(MemRegion span) { _span = span; } + BoolObjectClosure* is_subject_to_discovery_closure() const { return _is_subject_to_discovery; } + void set_is_subject_to_discovery_closure(BoolObjectClosure* cl) { _is_subject_to_discovery = cl; } // start and stop weak ref discovery void enable_discovery(bool check_no_refs = true); @@ -435,6 +419,26 @@ void verify_referent(oop obj) PRODUCT_RETURN; }; +// A subject-to-discovery closure that uses a single memory span to determine the area that +// is subject to discovery. Useful for collectors which have contiguous generations. +class SpanSubjectToDiscoveryClosure : public BoolObjectClosure { + MemRegion _span; + +public: + SpanSubjectToDiscoveryClosure() : BoolObjectClosure(), _span() { } + SpanSubjectToDiscoveryClosure(MemRegion span) : BoolObjectClosure(), _span(span) { } + + MemRegion span() const { return _span; } + + void set_span(MemRegion mr) { + _span = mr; + } + + virtual bool do_object_b(oop obj) { + return _span.contains(obj); + } +}; + // A utility class to disable reference discovery in // the scope which contains it, for given ReferenceProcessor. class NoRefDiscovery: StackObj { @@ -456,24 +460,43 @@ } }; +// A utility class to temporarily mutate the subject discovery closure of the +// given ReferenceProcessor in the scope that contains it. +class ReferenceProcessorSubjectToDiscoveryMutator : StackObj { + ReferenceProcessor* _rp; + BoolObjectClosure* _saved_cl; + +public: + ReferenceProcessorSubjectToDiscoveryMutator(ReferenceProcessor* rp, BoolObjectClosure* cl): + _rp(rp) { + _saved_cl = _rp->is_subject_to_discovery_closure(); + _rp->set_is_subject_to_discovery_closure(cl); + } + + ~ReferenceProcessorSubjectToDiscoveryMutator() { + _rp->set_is_subject_to_discovery_closure(_saved_cl); + } +}; // A utility class to temporarily mutate the span of the // given ReferenceProcessor in the scope that contains it. -class ReferenceProcessorSpanMutator: StackObj { - private: +class ReferenceProcessorSpanMutator : StackObj { ReferenceProcessor* _rp; - MemRegion _saved_span; + SpanSubjectToDiscoveryClosure _discoverer; + BoolObjectClosure* _old_discoverer; - public: +public: ReferenceProcessorSpanMutator(ReferenceProcessor* rp, MemRegion span): - _rp(rp) { - _saved_span = _rp->span(); - _rp->set_span(span); + _rp(rp), + _discoverer(span), + _old_discoverer(rp->is_subject_to_discovery_closure()) { + + rp->set_is_subject_to_discovery_closure(&_discoverer); } ~ReferenceProcessorSpanMutator() { - _rp->set_span(_saved_span); + _rp->set_is_subject_to_discovery_closure(_old_discoverer); } }; @@ -498,7 +521,6 @@ } }; - // A utility class to temporarily change the disposition // of the "is_alive_non_header" closure field of the // given ReferenceProcessor in the scope that contains it. diff -r 93a7cbd3f838 -r 99e698e94cc7 test/hotspot/jtreg/gc/g1/TestGCLogMessages.java --- a/test/hotspot/jtreg/gc/g1/TestGCLogMessages.java Thu Apr 26 12:54:53 2018 +0200 +++ b/test/hotspot/jtreg/gc/g1/TestGCLogMessages.java Thu May 03 14:09:00 2018 +0200 @@ -134,8 +134,6 @@ // Humongous Eager Reclaim new LogMessageWithLevel("Humongous Reclaim", Level.DEBUG), new LogMessageWithLevel("Humongous Register", Level.DEBUG), - // Preserve CM Referents - new LogMessageWithLevel("Preserve CM Refs", Level.DEBUG), // Merge PSS new LogMessageWithLevel("Merge Per-Thread State", Level.DEBUG), // TLAB handling