--- old/src/hotspot/share/gc/g1/g1CollectedHeap.cpp 2019-03-13 14:02:57.269925863 +0100 +++ new/src/hotspot/share/gc/g1/g1CollectedHeap.cpp 2019-03-13 14:02:57.045919779 +0100 @@ -154,6 +154,11 @@ reset_from_card_cache(start_idx, num_regions); } +Tickspan G1CollectedHeap::run_task(AbstractGangTask* task, uint num_workers) { + Ticks start = Ticks::now(); + workers()->run_task(task, num_workers == 0 ? workers()->active_workers() : num_workers); + return Ticks::now() - start; +} HeapRegion* G1CollectedHeap::new_heap_region(uint hrs_index, MemRegion mr) { @@ -2246,7 +2251,7 @@ } void G1CollectedHeap::collection_set_iterate_from(HeapRegionClosure *cl, uint worker_id) { - _collection_set.iterate_from(cl, worker_id, workers()->active_workers()); + _collection_set.iterate_incremental_part_from(cl, worker_id, workers()->active_workers()); } HeapWord* G1CollectedHeap::block_start(const void* addr) const { @@ -2884,15 +2889,17 @@ phase_times()->record_start_new_cset_time_ms((os::elapsedTime() - start) * 1000.0); } -void G1CollectedHeap::calculate_collection_set(G1EvacuationInfo& evacuation_info, double target_pause_time_ms){ - policy()->finalize_collection_set(target_pause_time_ms, &_survivor); - evacuation_info.set_collectionset_regions(collection_set()->region_length()); +void G1CollectedHeap::calculate_collection_set(G1EvacuationInfo& evacuation_info, double target_pause_time_ms) { + _collection_set.finalize_initial_collection_set(target_pause_time_ms, &_survivor); + evacuation_info.set_collectionset_regions(collection_set()->region_length() + + collection_set()->optional_region_length()); _cm->verify_no_collection_set_oops_in_stacks(); if (_hr_printer.is_active()) { G1PrintCollectionSetClosure cl(&_hr_printer); _collection_set.iterate(&cl); + _collection_set.iterate_optional(&cl); } } @@ -3064,9 +3071,10 @@ pre_evacuate_collection_set(evacuation_info); // Actually do the work... - evacuate_collection_set(&per_thread_states); - evacuate_optional_collection_set(&per_thread_states); - + evacuate_initial_collection_set(&per_thread_states); + if (_collection_set.optional_region_length() != 0) { + evacuate_optional_collection_set(&per_thread_states); + } post_evacuate_collection_set(evacuation_info, &per_thread_states); start_new_collection_set(); @@ -3092,7 +3100,8 @@ double sample_end_time_sec = os::elapsedTime(); double pause_time_ms = (sample_end_time_sec - sample_start_time_sec) * MILLIUNITS; - size_t total_cards_scanned = phase_times()->sum_thread_work_items(G1GCPhaseTimes::ScanRS, G1GCPhaseTimes::ScanRSScannedCards); + size_t total_cards_scanned = phase_times()->sum_thread_work_items(G1GCPhaseTimes::ScanRS, G1GCPhaseTimes::ScanRSScannedCards) + + phase_times()->sum_thread_work_items(G1GCPhaseTimes::OptScanRS, G1GCPhaseTimes::ScanRSScannedCards); policy()->record_collection_pause_end(pause_time_ms, total_cards_scanned, heap_used_bytes_before_gc); } @@ -3196,86 +3205,6 @@ } while (!offer_termination()); } -class G1ParTask : public AbstractGangTask { -protected: - G1CollectedHeap* _g1h; - G1ParScanThreadStateSet* _pss; - RefToScanQueueSet* _queues; - G1RootProcessor* _root_processor; - TaskTerminator _terminator; - uint _n_workers; - -public: - G1ParTask(G1CollectedHeap* g1h, G1ParScanThreadStateSet* per_thread_states, RefToScanQueueSet *task_queues, G1RootProcessor* root_processor, uint n_workers) - : AbstractGangTask("G1 collection"), - _g1h(g1h), - _pss(per_thread_states), - _queues(task_queues), - _root_processor(root_processor), - _terminator(n_workers, _queues), - _n_workers(n_workers) - {} - - void work(uint worker_id) { - if (worker_id >= _n_workers) return; // no work needed this round - - double start_sec = os::elapsedTime(); - _g1h->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerStart, worker_id, start_sec); - - { - ResourceMark rm; - HandleMark hm; - - ReferenceProcessor* rp = _g1h->ref_processor_stw(); - - G1ParScanThreadState* pss = _pss->state_for_worker(worker_id); - pss->set_ref_discoverer(rp); - - double start_strong_roots_sec = os::elapsedTime(); - - _root_processor->evacuate_roots(pss, worker_id); - - _g1h->rem_set()->oops_into_collection_set_do(pss, worker_id); - - double strong_roots_sec = os::elapsedTime() - start_strong_roots_sec; - - double term_sec = 0.0; - size_t evac_term_attempts = 0; - { - double start = os::elapsedTime(); - G1ParEvacuateFollowersClosure evac(_g1h, pss, _queues, _terminator.terminator(), G1GCPhaseTimes::ObjCopy); - evac.do_void(); - - evac_term_attempts = evac.term_attempts(); - term_sec = evac.term_time(); - double elapsed_sec = os::elapsedTime() - start; - - G1GCPhaseTimes* p = _g1h->phase_times(); - p->add_time_secs(G1GCPhaseTimes::ObjCopy, worker_id, elapsed_sec - term_sec); - - p->record_or_add_thread_work_item(G1GCPhaseTimes::ObjCopy, - worker_id, - pss->lab_waste_words() * HeapWordSize, - G1GCPhaseTimes::ObjCopyLABWaste); - p->record_or_add_thread_work_item(G1GCPhaseTimes::ObjCopy, - worker_id, - pss->lab_undo_waste_words() * HeapWordSize, - G1GCPhaseTimes::ObjCopyLABUndoWaste); - - p->record_time_secs(G1GCPhaseTimes::Termination, worker_id, term_sec); - p->record_thread_work_item(G1GCPhaseTimes::Termination, worker_id, evac_term_attempts); - } - - assert(pss->queue_is_empty(), "should be empty"); - - // Close the inner scope so that the ResourceMark and HandleMark - // destructors are executed here and are included as part of the - // "GC Worker Time". - } - _g1h->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerEnd, worker_id, os::elapsedTime()); - } -}; - void G1CollectedHeap::complete_cleaning(BoolObjectClosure* is_alive, bool class_unloading_occurred) { uint num_workers = workers()->active_workers(); @@ -3679,176 +3608,197 @@ double recorded_clear_claimed_marks_time_ms = (os::elapsedTime() - start_clear_claimed_marks) * 1000.0; phase_times()->record_clear_claimed_marks_time_ms(recorded_clear_claimed_marks_time_ms); } -} -void G1CollectedHeap::evacuate_collection_set(G1ParScanThreadStateSet* per_thread_states) { // Should G1EvacuationFailureALot be in effect for this GC? NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();) assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty"); +} - double start_par_time_sec = os::elapsedTime(); - double end_par_time_sec; +class G1EvacuateRegionsBaseTask : public AbstractGangTask { +protected: + G1CollectedHeap* _g1h; + G1ParScanThreadStateSet* _per_thread_states; + RefToScanQueueSet* _task_queues; + TaskTerminator _terminator; + uint _num_workers; - { - const uint n_workers = workers()->active_workers(); - G1RootProcessor root_processor(this, n_workers); - G1ParTask g1_par_task(this, per_thread_states, _task_queues, &root_processor, n_workers); + void evacuate_live_objects(G1ParScanThreadState* pss, + uint worker_id, + G1GCPhaseTimes::GCParPhases objcopy_phase, + G1GCPhaseTimes::GCParPhases termination_phase) { + G1GCPhaseTimes* p = _g1h->phase_times(); - workers()->run_task(&g1_par_task); - end_par_time_sec = os::elapsedTime(); + Ticks start = Ticks::now(); + G1ParEvacuateFollowersClosure cl(_g1h, pss, _task_queues, _terminator.terminator(), objcopy_phase); + cl.do_void(); - // Closing the inner scope will execute the destructor - // for the G1RootProcessor object. We record the current - // elapsed time before closing the scope so that time - // taken for the destructor is NOT included in the - // reported parallel time. + assert(pss->queue_is_empty(), "should be empty"); + + Tickspan evac_time = (Ticks::now() - start); + p->record_or_add_time_secs(objcopy_phase, worker_id, evac_time.seconds() - cl.term_time()); + + p->record_or_add_thread_work_item(objcopy_phase, worker_id, pss->lab_waste_words() * HeapWordSize, G1GCPhaseTimes::ObjCopyLABWaste); + p->record_or_add_thread_work_item(objcopy_phase, worker_id, pss->lab_undo_waste_words() * HeapWordSize, G1GCPhaseTimes::ObjCopyLABUndoWaste); + + if (termination_phase == G1GCPhaseTimes::Termination) { + p->record_time_secs(termination_phase, worker_id, cl.term_time()); + p->record_thread_work_item(termination_phase, worker_id, cl.term_attempts()); + } else { + p->record_or_add_time_secs(termination_phase, worker_id, cl.term_time()); + p->record_or_add_thread_work_item(termination_phase, worker_id, cl.term_attempts()); + } + assert(pss->trim_ticks().seconds() == 0.0, "Unexpected partial trimming during evacuation"); } - double par_time_ms = (end_par_time_sec - start_par_time_sec) * 1000.0; - phase_times()->record_par_time(par_time_ms); + virtual void start_work(uint worker_id) { } - double code_root_fixup_time_ms = - (os::elapsedTime() - end_par_time_sec) * 1000.0; - phase_times()->record_code_root_fixup_time(code_root_fixup_time_ms); -} + virtual void end_work(uint worker_id) { } -class G1EvacuateOptionalRegionTask : public AbstractGangTask { - G1CollectedHeap* _g1h; - G1ParScanThreadStateSet* _per_thread_states; - G1OptionalCSet* _optional; - RefToScanQueueSet* _queues; - ParallelTaskTerminator _terminator; - - Tickspan trim_ticks(G1ParScanThreadState* pss) { - Tickspan copy_time = pss->trim_ticks(); - pss->reset_trim_ticks(); - return copy_time; - } + virtual void scan_roots(G1ParScanThreadState* pss, uint worker_id) = 0; - void scan_roots(G1ParScanThreadState* pss, uint worker_id) { - G1EvacuationRootClosures* root_cls = pss->closures(); - G1ScanObjsDuringScanRSClosure obj_cl(_g1h, pss); + virtual void evacuate_live_objects(G1ParScanThreadState* pss, uint worker_id) = 0; + +public: + G1EvacuateRegionsBaseTask(const char* name, G1ParScanThreadStateSet* per_thread_states, RefToScanQueueSet* task_queues, uint num_workers) : + AbstractGangTask(name), + _g1h(G1CollectedHeap::heap()), + _per_thread_states(per_thread_states), + _task_queues(task_queues), + _terminator(num_workers, _task_queues), + _num_workers(num_workers) + { } + + void work(uint worker_id) { + start_work(worker_id); + + { + ResourceMark rm; + HandleMark hm; - size_t scanned = 0; - size_t claimed = 0; - size_t skipped = 0; - size_t used_memory = 0; - - Ticks start = Ticks::now(); - Tickspan copy_time; - - for (uint i = _optional->current_index(); i < _optional->current_limit(); i++) { - HeapRegion* hr = _optional->region_at(i); - G1ScanRSForOptionalClosure scan_opt_cl(&obj_cl); - pss->oops_into_optional_region(hr)->oops_do(&scan_opt_cl, root_cls->raw_strong_oops()); - copy_time += trim_ticks(pss); - - G1ScanRSForRegionClosure scan_rs_cl(_g1h->rem_set()->scan_state(), &obj_cl, pss, G1GCPhaseTimes::OptScanRS, worker_id); - scan_rs_cl.do_heap_region(hr); - copy_time += trim_ticks(pss); - scanned += scan_rs_cl.cards_scanned(); - claimed += scan_rs_cl.cards_claimed(); - skipped += scan_rs_cl.cards_skipped(); + G1ParScanThreadState* pss = _per_thread_states->state_for_worker(worker_id); + pss->set_ref_discoverer(_g1h->ref_processor_stw()); - // Chunk lists for this region is no longer needed. - used_memory += pss->oops_into_optional_region(hr)->used_memory(); + scan_roots(pss, worker_id); + evacuate_live_objects(pss, worker_id); } - Tickspan scan_time = (Ticks::now() - start) - copy_time; - G1GCPhaseTimes* p = _g1h->phase_times(); - p->record_or_add_time_secs(G1GCPhaseTimes::OptScanRS, worker_id, scan_time.seconds()); - p->record_or_add_time_secs(G1GCPhaseTimes::OptObjCopy, worker_id, copy_time.seconds()); + end_work(worker_id); + } +}; + +class G1EvacuateRegionsTask : public G1EvacuateRegionsBaseTask { + G1RootProcessor* _root_processor; - p->record_or_add_thread_work_item(G1GCPhaseTimes::OptScanRS, worker_id, scanned, G1GCPhaseTimes::OptCSetScannedCards); - p->record_or_add_thread_work_item(G1GCPhaseTimes::OptScanRS, worker_id, claimed, G1GCPhaseTimes::OptCSetClaimedCards); - p->record_or_add_thread_work_item(G1GCPhaseTimes::OptScanRS, worker_id, skipped, G1GCPhaseTimes::OptCSetSkippedCards); - p->record_or_add_thread_work_item(G1GCPhaseTimes::OptScanRS, worker_id, used_memory, G1GCPhaseTimes::OptCSetUsedMemory); + void scan_roots(G1ParScanThreadState* pss, uint worker_id) { + _root_processor->evacuate_roots(pss, worker_id); + _g1h->rem_set()->update_rem_set(pss, worker_id); + _g1h->rem_set()->scan_rem_set(pss, worker_id, G1GCPhaseTimes::ScanRS, G1GCPhaseTimes::ObjCopy, G1GCPhaseTimes::CodeRoots); } void evacuate_live_objects(G1ParScanThreadState* pss, uint worker_id) { - Ticks start = Ticks::now(); - G1ParEvacuateFollowersClosure cl(_g1h, pss, _queues, &_terminator, G1GCPhaseTimes::OptObjCopy); - cl.do_void(); + G1EvacuateRegionsBaseTask::evacuate_live_objects(pss, worker_id, G1GCPhaseTimes::ObjCopy, G1GCPhaseTimes::Termination); + } - Tickspan evac_time = (Ticks::now() - start); - G1GCPhaseTimes* p = _g1h->phase_times(); - p->record_or_add_time_secs(G1GCPhaseTimes::OptObjCopy, worker_id, evac_time.seconds()); - assert(pss->trim_ticks().seconds() == 0.0, "Unexpected partial trimming done during optional evacuation"); + void start_work(uint worker_id) { + _g1h->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerStart, worker_id, Ticks::now().seconds()); } - public: - G1EvacuateOptionalRegionTask(G1CollectedHeap* g1h, - G1ParScanThreadStateSet* per_thread_states, - G1OptionalCSet* cset, - RefToScanQueueSet* queues, - uint n_workers) : - AbstractGangTask("G1 Evacuation Optional Region Task"), - _g1h(g1h), - _per_thread_states(per_thread_states), - _optional(cset), - _queues(queues), - _terminator(n_workers, _queues) { + void end_work(uint worker_id) { + _g1h->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerEnd, worker_id, Ticks::now().seconds()); } - void work(uint worker_id) { - ResourceMark rm; - HandleMark hm; +public: + G1EvacuateRegionsTask(G1CollectedHeap* g1h, + G1ParScanThreadStateSet* per_thread_states, + RefToScanQueueSet* task_queues, + G1RootProcessor* root_processor, + uint num_workers) : + G1EvacuateRegionsBaseTask("G1 Evacuate Regions", per_thread_states, task_queues, num_workers), + _root_processor(root_processor) + { } +}; - G1ParScanThreadState* pss = _per_thread_states->state_for_worker(worker_id); - pss->set_ref_discoverer(_g1h->ref_processor_stw()); +void G1CollectedHeap::evacuate_initial_collection_set(G1ParScanThreadStateSet* per_thread_states) { + Tickspan task_time; + const uint num_workers = workers()->active_workers(); - scan_roots(pss, worker_id); - evacuate_live_objects(pss, worker_id); + Ticks start_processing = Ticks::now(); + { + G1RootProcessor root_processor(this, num_workers); + G1EvacuateRegionsTask g1_par_task(this, per_thread_states, _task_queues, &root_processor, num_workers); + task_time = run_task(&g1_par_task, num_workers); + // Closing the inner scope will execute the destructor for the G1RootProcessor object. + // To extract its code root fixup time we measure total time of this scope and + // subtract from the time the WorkGang task took. } -}; + Tickspan total_processing = Ticks::now() - start_processing; -void G1CollectedHeap::evacuate_optional_regions(G1ParScanThreadStateSet* per_thread_states, G1OptionalCSet* ocset) { - class G1MarkScope : public MarkScope {}; - G1MarkScope code_mark_scope; - - G1EvacuateOptionalRegionTask task(this, per_thread_states, ocset, _task_queues, workers()->active_workers()); - workers()->run_task(&task); + G1GCPhaseTimes* p = phase_times(); + p->record_or_add_initial_evac_time(task_time.milliseconds()); + p->record_or_add_code_root_fixup_time((total_processing - task_time).milliseconds()); } -void G1CollectedHeap::evacuate_optional_collection_set(G1ParScanThreadStateSet* per_thread_states) { - G1OptionalCSet optional_cset(&_collection_set, per_thread_states); - if (optional_cset.is_empty()) { - return; +class G1EvacuateOptionalRegionsTask : public G1EvacuateRegionsBaseTask { + + void scan_roots(G1ParScanThreadState* pss, uint worker_id) { + _g1h->rem_set()->scan_rem_set(pss, worker_id, G1GCPhaseTimes::OptScanRS, G1GCPhaseTimes::OptObjCopy, G1GCPhaseTimes::OptCodeRoots); } - if (evacuation_failed()) { - return; + void evacuate_live_objects(G1ParScanThreadState* pss, uint worker_id) { + G1EvacuateRegionsBaseTask::evacuate_live_objects(pss, worker_id, G1GCPhaseTimes::OptObjCopy, G1GCPhaseTimes::OptTermination); + } + +public: + G1EvacuateOptionalRegionsTask(G1ParScanThreadStateSet* per_thread_states, + RefToScanQueueSet* queues, + uint num_workers) : + G1EvacuateRegionsBaseTask("G1 Evacuate Optional Regions", per_thread_states, queues, num_workers) { } +}; + +void G1CollectedHeap::evacuate_next_optional_regions(G1ParScanThreadStateSet* per_thread_states) { + class G1MarkScope : public MarkScope { }; + + Tickspan task_time; + Ticks start_processing = Ticks::now(); + { + G1MarkScope code_mark_scope; + G1EvacuateOptionalRegionsTask task(per_thread_states, _task_queues, workers()->active_workers()); + task_time = run_task(&task); + // See comment in evacuate_collection_set() for the reason of the scope. + } + Tickspan total_processing = Ticks::now() - start_processing; + + G1GCPhaseTimes* p = phase_times(); + p->record_or_add_initial_evac_time(task_time.milliseconds()); + p->record_or_add_code_root_fixup_time((total_processing - task_time).milliseconds()); +} + +void G1CollectedHeap::evacuate_optional_collection_set(G1ParScanThreadStateSet* per_thread_states) { const double gc_start_time_ms = phase_times()->cur_collection_start_sec() * 1000.0; - double start_time_sec = os::elapsedTime(); + Ticks start = Ticks::now(); + + while (!evacuation_failed() && _collection_set.optional_region_length() > 0) { - do { double time_used_ms = os::elapsedTime() * 1000.0 - gc_start_time_ms; double time_left_ms = MaxGCPauseMillis - time_used_ms; - if (time_left_ms < 0) { - log_trace(gc, ergo, cset)("Skipping %u optional regions, pause time exceeded %.3fms", optional_cset.size(), time_used_ms); + if (time_left_ms < 0 || + !_collection_set.finalize_optional_for_evacuation(time_left_ms * policy()->optional_evacuation_fraction())) { + log_trace(gc, ergo, cset)("Skipping evacuation of %u optional regions, no more regions can be evacuated in %.3fms", + _collection_set.optional_region_length(), time_left_ms); break; } - optional_cset.prepare_evacuation(time_left_ms * _policy->optional_evacuation_fraction()); - if (optional_cset.prepare_failed()) { - log_trace(gc, ergo, cset)("Skipping %u optional regions, no regions can be evacuated in %.3fms", optional_cset.size(), time_left_ms); - break; - } - - evacuate_optional_regions(per_thread_states, &optional_cset); + evacuate_next_optional_regions(per_thread_states); + } - optional_cset.complete_evacuation(); - if (optional_cset.evacuation_failed()) { - break; - } - } while (!optional_cset.is_empty()); + _collection_set.abandon_optional_collection_set(per_thread_states); - phase_times()->record_optional_evacuation((os::elapsedTime() - start_time_sec) * 1000.0); + phase_times()->record_optional_evacuation((Ticks::now() - start).milliseconds()); } void G1CollectedHeap::post_evacuate_collection_set(G1EvacuationInfo& evacuation_info, G1ParScanThreadStateSet* per_thread_states) { @@ -4263,15 +4213,14 @@ double free_cset_start_time = os::elapsedTime(); { - uint const num_chunks = MAX2(_collection_set.region_length() / G1FreeCollectionSetTask::chunk_size(), 1U); + uint const num_regions = _collection_set.region_length(); + uint const num_chunks = MAX2(num_regions / G1FreeCollectionSetTask::chunk_size(), 1U); uint const num_workers = MIN2(workers()->active_workers(), num_chunks); G1FreeCollectionSetTask cl(collection_set, &evacuation_info, surviving_young_words); log_debug(gc, ergo)("Running %s using %u workers for collection set length %u", - cl.name(), - num_workers, - _collection_set.region_length()); + cl.name(), num_workers, num_regions); workers()->run_task(&cl, num_workers); } phase_times()->record_total_free_cset_time_ms((os::elapsedTime() - free_cset_start_time) * 1000.0); @@ -4440,7 +4389,7 @@ void G1CollectedHeap::abandon_collection_set(G1CollectionSet* collection_set) { G1AbandonCollectionSetClosure cl; - collection_set->iterate(&cl); + collection_set_iterate(&cl); collection_set->clear(); collection_set->stop_incremental_building(); --- old/src/hotspot/share/gc/g1/g1CollectedHeap.hpp 2019-03-13 14:02:58.382956089 +0100 +++ new/src/hotspot/share/gc/g1/g1CollectedHeap.hpp 2019-03-13 14:02:58.169950305 +0100 @@ -143,9 +143,8 @@ // Closures used in implementation. friend class G1ParScanThreadState; friend class G1ParScanThreadStateSet; - friend class G1ParTask; + friend class G1EvacuateRegionsTask; friend class G1PLABAllocator; - friend class G1PrepareCompactClosure; // Other related classes. friend class HeapRegionClaimer; @@ -519,6 +518,8 @@ WorkGang* workers() const { return _workers; } + Tickspan run_task(AbstractGangTask* task, uint num_workers = 0); + G1Allocator* allocator() { return _allocator; } @@ -738,10 +739,11 @@ void calculate_collection_set(G1EvacuationInfo& evacuation_info, double target_pause_time_ms); - // Actually do the work of evacuating the collection set. - void evacuate_collection_set(G1ParScanThreadStateSet* per_thread_states); + // Actually do the work of evacuating the parts of the collection set. + void evacuate_initial_collection_set(G1ParScanThreadStateSet* per_thread_states); void evacuate_optional_collection_set(G1ParScanThreadStateSet* per_thread_states); - void evacuate_optional_regions(G1ParScanThreadStateSet* per_thread_states, G1OptionalCSet* ocset); + // Evacuate the next set of optional regions. + void evacuate_next_optional_regions(G1ParScanThreadStateSet* per_thread_states); void pre_evacuate_collection_set(G1EvacuationInfo& evacuation_info); void post_evacuate_collection_set(G1EvacuationInfo& evacuation_info, G1ParScanThreadStateSet* pss); @@ -1424,7 +1426,7 @@ size_t _term_attempts; void start_term_time() { _term_attempts++; _start_term = os::elapsedTime(); } - void end_term_time() { _term_time += os::elapsedTime() - _start_term; } + void end_term_time() { _term_time += (os::elapsedTime() - _start_term); } protected: G1CollectedHeap* _g1h; G1ParScanThreadState* _par_scan_state; --- old/src/hotspot/share/gc/g1/g1CollectionSet.cpp 2019-03-13 14:02:59.297980938 +0100 +++ new/src/hotspot/share/gc/g1/g1CollectionSet.cpp 2019-03-13 14:02:59.086975208 +0100 @@ -59,9 +59,7 @@ _collection_set_regions(NULL), _collection_set_cur_length(0), _collection_set_max_length(0), - _optional_regions(NULL), - _optional_region_length(0), - _optional_region_max_length(0), + _num_optional_regions(0), _bytes_used_before(0), _recorded_rs_lengths(0), _inc_build_state(Inactive), @@ -91,7 +89,7 @@ "Young region length %u should match collection set length " SIZE_FORMAT, young_region_length(), _collection_set_cur_length); _old_region_length = 0; - _optional_region_length = 0; + _num_optional_regions = 0; } void G1CollectionSet::initialize(uint max_region_length) { @@ -100,21 +98,8 @@ _collection_set_regions = NEW_C_HEAP_ARRAY(uint, max_region_length, mtGC); } -void G1CollectionSet::initialize_optional(uint max_length) { - assert(_optional_regions == NULL, "Already initialized"); - assert(_optional_region_length == 0, "Already initialized"); - assert(_optional_region_max_length == 0, "Already initialized"); - _optional_region_max_length = max_length; - _optional_regions = NEW_C_HEAP_ARRAY(HeapRegion*, _optional_region_max_length, mtGC); -} - void G1CollectionSet::free_optional_regions() { - _optional_region_length = 0; - _optional_region_max_length = 0; - if (_optional_regions != NULL) { - FREE_C_HEAP_ARRAY(HeapRegion*, _optional_regions); - _optional_regions = NULL; - } + _num_optional_regions = 0; } void G1CollectionSet::clear_candidates() { @@ -130,39 +115,32 @@ void G1CollectionSet::add_old_region(HeapRegion* hr) { assert_at_safepoint_on_vm_thread(); - assert(_inc_build_state == Active || hr->index_in_opt_cset() != G1OptionalCSet::InvalidCSetIndex, + assert(_inc_build_state == Active, "Precondition, actively building cset or adding optional later on"); assert(hr->is_old(), "the region should be old"); - assert(!hr->in_collection_set(), "should not already be in the CSet"); + assert(!hr->in_collection_set(), "should not already be in the collection set"); _g1h->register_old_region_with_cset(hr); _collection_set_regions[_collection_set_cur_length++] = hr->hrm_index(); assert(_collection_set_cur_length <= _collection_set_max_length, "Collection set now larger than maximum size."); _bytes_used_before += hr->used(); - size_t rs_length = hr->rem_set()->occupied(); - _recorded_rs_lengths += rs_length; - _old_region_length += 1; + _recorded_rs_lengths += hr->rem_set()->occupied(); + _old_region_length++; - log_trace(gc, cset)("Added old region %d to collection set", hr->hrm_index()); + _g1h->old_set_remove(hr); } void G1CollectionSet::add_optional_region(HeapRegion* hr) { - assert(!optional_is_full(), "Precondition, must have room left for this region"); assert(hr->is_old(), "the region should be old"); assert(!hr->in_collection_set(), "should not already be in the CSet"); _g1h->register_optional_region_with_cset(hr); - _optional_regions[_optional_region_length] = hr; - uint index = _optional_region_length++; - hr->set_index_in_opt_cset(index); - - log_trace(gc, cset)("Added region %d to optional collection set (%u)", hr->hrm_index(), _optional_region_length); + hr->set_index_in_opt_cset(_num_optional_regions++); } -// Initialize the per-collection-set information void G1CollectionSet::start_incremental_building() { assert(_collection_set_cur_length == 0, "Collection set must be empty before starting a new collection set."); assert(_inc_build_state == Inactive, "Precondition"); @@ -173,10 +151,11 @@ _inc_recorded_rs_lengths_diffs = 0; _inc_predicted_elapsed_time_ms = 0.0; _inc_predicted_elapsed_time_ms_diffs = 0.0; - _inc_build_state = Active; + + update_incremental_marker(); } -void G1CollectionSet::finalize_incremental_building() { +void G1CollectionSet::finalize_young_increment() { assert(_inc_build_state == Active, "Precondition"); assert(SafepointSynchronize::is_at_safepoint(), "should be at a safepoint"); @@ -211,29 +190,48 @@ void G1CollectionSet::clear() { assert_at_safepoint_on_vm_thread(); _collection_set_cur_length = 0; - _optional_region_length = 0; } void G1CollectionSet::iterate(HeapRegionClosure* cl) const { - iterate_from(cl, 0, 1); -} - -void G1CollectionSet::iterate_from(HeapRegionClosure* cl, uint worker_id, uint total_workers) const { size_t len = _collection_set_cur_length; OrderAccess::loadload(); + + for (uint i = 0; i < len; i++) { + HeapRegion* r = _g1h->region_at(_collection_set_regions[i]); + bool result = cl->do_heap_region(r); + if (result) { + cl->set_incomplete(); + return; + } + } +} + +void G1CollectionSet::iterate_optional(HeapRegionClosure* cl) const { + assert_at_safepoint(); + + for (uint i = 0; i < _num_optional_regions; i++) { + HeapRegion* r = _candidates->at(i); + bool result = cl->do_heap_region(r); + guarantee(!result, "Must not cancel iteration"); + } +} + +void G1CollectionSet::iterate_incremental_part_from(HeapRegionClosure* cl, uint worker_id, uint total_workers) const { + assert_at_safepoint(); + + size_t len = _collection_set_cur_length - _inc_part_start; if (len == 0) { return; } + size_t start_pos = (worker_id * len) / total_workers; size_t cur_pos = start_pos; do { - HeapRegion* r = _g1h->region_at(_collection_set_regions[cur_pos]); + HeapRegion* r = _g1h->region_at(_collection_set_regions[cur_pos + _inc_part_start]); bool result = cl->do_heap_region(r); - if (result) { - cl->set_incomplete(); - return; - } + guarantee(!result, "Must not cancel iteration"); + cur_pos++; if (cur_pos == len) { cur_pos = 0; @@ -399,7 +397,7 @@ double G1CollectionSet::finalize_young_part(double target_pause_time_ms, G1SurvivorRegions* survivors) { double young_start_time_sec = os::elapsedTime(); - finalize_incremental_building(); + finalize_young_increment(); guarantee(target_pause_time_ms > 0.0, "target_pause_time_ms = %1.6lf should be positive", target_pause_time_ms); @@ -440,30 +438,6 @@ return time_remaining_ms; } -void G1CollectionSet::add_as_old(HeapRegion* hr) { - candidates()->pop_front(); // already have region via peek() - _g1h->old_set_remove(hr); - add_old_region(hr); -} - -void G1CollectionSet::add_as_optional(HeapRegion* hr) { - assert(_optional_regions != NULL, "Must not be called before array is allocated"); - candidates()->pop_front(); // already have region via peek() - _g1h->old_set_remove(hr); - add_optional_region(hr); -} - -bool G1CollectionSet::optional_is_full() { - assert(_optional_region_length <= _optional_region_max_length, "Invariant"); - return _optional_region_length == _optional_region_max_length; -} - -void G1CollectionSet::clear_optional_region(const HeapRegion* hr) { - assert(_optional_regions != NULL, "Must not be called before array is allocated"); - uint index = hr->index_in_opt_cset(); - _optional_regions[index] = NULL; -} - static int compare_region_idx(const uint a, const uint b) { if (a > b) { return 1; @@ -476,87 +450,28 @@ void G1CollectionSet::finalize_old_part(double time_remaining_ms) { double non_young_start_time_sec = os::elapsedTime(); - double predicted_old_time_ms = 0.0; - double predicted_optional_time_ms = 0.0; - double optional_threshold_ms = time_remaining_ms * _policy->optional_prediction_fraction(); - uint expensive_region_num = 0; + + uint num_expensive_regions = 0; if (collector_state()->in_mixed_phase()) { candidates()->verify(); - const uint min_old_cset_length = _policy->calc_min_old_cset_length(); - const uint max_old_cset_length = MAX2(min_old_cset_length, _policy->calc_max_old_cset_length()); - bool check_time_remaining = _policy->adaptive_young_list_length(); - - initialize_optional(max_old_cset_length - min_old_cset_length); - log_debug(gc, ergo, cset)("Start adding old regions for mixed gc. min %u regions, max %u regions, " - "time remaining %1.2fms, optional threshold %1.2fms", - min_old_cset_length, max_old_cset_length, time_remaining_ms, optional_threshold_ms); - - HeapRegion* hr = candidates()->peek_front(); - while (hr != NULL) { - if (old_region_length() + optional_region_length() >= max_old_cset_length) { - // Added maximum number of old regions to the CSet. - log_debug(gc, ergo, cset)("Finish adding old regions to CSet (old CSet region num reached max). " - "old %u regions, optional %u regions", - old_region_length(), optional_region_length()); - break; - } - - // Stop adding regions if the remaining reclaimable space is - // not above G1HeapWastePercent. - size_t reclaimable_bytes = candidates()->remaining_reclaimable_bytes(); - double reclaimable_percent = _policy->reclaimable_bytes_percent(reclaimable_bytes); - double threshold = (double) G1HeapWastePercent; - if (reclaimable_percent <= threshold) { - // We've added enough old regions that the amount of uncollected - // reclaimable space is at or below the waste threshold. Stop - // adding old regions to the CSet. - log_debug(gc, ergo, cset)("Finish adding old regions to CSet (reclaimable percentage not over threshold). " - "reclaimable: " SIZE_FORMAT "%s (%1.2f%%) threshold: " UINTX_FORMAT "%%", - byte_size_in_proper_unit(reclaimable_bytes), proper_unit_for_byte_size(reclaimable_bytes), - reclaimable_percent, G1HeapWastePercent); - break; - } - - double predicted_time_ms = predict_region_elapsed_time_ms(hr); - time_remaining_ms = MAX2(time_remaining_ms - predicted_time_ms, 0.0); - // Add regions to old set until we reach minimum amount - if (old_region_length() < min_old_cset_length) { - predicted_old_time_ms += predicted_time_ms; - add_as_old(hr); - // Record the number of regions added when no time remaining - if (time_remaining_ms == 0.0) { - expensive_region_num++; - } - } else { - // In the non-auto-tuning case, we'll finish adding regions - // to the CSet if we reach the minimum. - if (!check_time_remaining) { - log_debug(gc, ergo, cset)("Finish adding old regions to CSet (old CSet region num reached min)."); - break; - } - // Keep adding regions to old set until we reach optional threshold - if (time_remaining_ms > optional_threshold_ms) { - predicted_old_time_ms += predicted_time_ms; - add_as_old(hr); - } else if (time_remaining_ms > 0) { - // Keep adding optional regions until time is up - if (!optional_is_full()) { - predicted_optional_time_ms += predicted_time_ms; - add_as_optional(hr); - } else { - log_debug(gc, ergo, cset)("Finish adding old regions to CSet (optional set full)."); - break; - } - } else { - log_debug(gc, ergo, cset)("Finish adding old regions to CSet (predicted time is too high)."); - break; - } - } - hr = candidates()->peek_front(); - } - if (hr == NULL) { - log_debug(gc, ergo, cset)("Finish adding old regions to CSet (candidate old regions not available)"); + + uint num_initial_regions; + uint num_optional_regions; + + _policy->select_old_collection_set_regions(candidates(), + time_remaining_ms, + num_expensive_regions, + num_initial_regions, + num_optional_regions); + + // Prepare initial old regions. + move_candidates_to_collection_set(num_initial_regions); + + // Prepare optional regions for evacuation. + uint candidate_idx = candidates()->cur_idx(); + for (uint i = 0; i < num_optional_regions; i++) { + add_optional_region(candidates()->at(candidate_idx + i)); } candidates()->verify(); @@ -564,99 +479,59 @@ stop_incremental_building(); - log_debug(gc, ergo, cset)("Finish choosing CSet regions old: %u, optional: %u, " - "predicted old time: %1.2fms, predicted optional time: %1.2fms, time remaining: %1.2f", - old_region_length(), optional_region_length(), - predicted_old_time_ms, predicted_optional_time_ms, time_remaining_ms); - if (expensive_region_num > 0) { - log_debug(gc, ergo, cset)("CSet contains %u old regions that were added although the predicted time was too high.", - expensive_region_num); - } - double non_young_end_time_sec = os::elapsedTime(); phase_times()->record_non_young_cset_choice_time_ms((non_young_end_time_sec - non_young_start_time_sec) * 1000.0); QuickSort::sort(_collection_set_regions, _collection_set_cur_length, compare_region_idx, true); } -HeapRegion* G1OptionalCSet::region_at(uint index) { - return _cset->optional_region_at(index); -} - -void G1OptionalCSet::prepare_evacuation(double time_limit) { - assert(_current_index == _current_limit, "Before prepare no regions should be ready for evac"); - - uint prepared_regions = 0; - double prediction_ms = 0; - - _prepare_failed = true; - for (uint i = _current_index; i < _cset->optional_region_length(); i++) { - HeapRegion* hr = region_at(i); - prediction_ms += _cset->predict_region_elapsed_time_ms(hr); - if (prediction_ms > time_limit) { - log_debug(gc, cset)("Prepared %u regions for optional evacuation. Predicted time: %.3fms", prepared_regions, prediction_ms); - return; - } - - // This region will be included in the next optional evacuation. - prepare_to_evacuate_optional_region(hr); - prepared_regions++; - _current_limit++; - _prepare_failed = false; +void G1CollectionSet::move_candidates_to_collection_set(uint num_old_candidate_regions) { + if (num_old_candidate_regions == 0) { + return; + } + uint candidate_idx = candidates()->cur_idx(); + for (uint i = 0; i < num_old_candidate_regions; i++) { + HeapRegion* r = candidates()->at(candidate_idx + i); + // This potentially optional candidate region is going to be an actual collection + // set region. Clear cset marker. + _g1h->clear_in_cset(r); + add_old_region(r); } + candidates()->remove(num_old_candidate_regions); - log_debug(gc, cset)("Prepared all %u regions for optional evacuation. Predicted time: %.3fms", - prepared_regions, prediction_ms); + candidates()->verify(); } -bool G1OptionalCSet::prepare_failed() { - return _prepare_failed; +void G1CollectionSet::finalize_initial_collection_set(double target_pause_time_ms, G1SurvivorRegions* survivor) { + double time_remaining_ms = finalize_young_part(target_pause_time_ms, survivor); + finalize_old_part(time_remaining_ms); } -void G1OptionalCSet::complete_evacuation() { - _evacuation_failed = false; - for (uint i = _current_index; i < _current_limit; i++) { - HeapRegion* hr = region_at(i); - _cset->clear_optional_region(hr); - if (hr->evacuation_failed()){ - _evacuation_failed = true; - } - } - _current_index = _current_limit; -} +bool G1CollectionSet::finalize_optional_for_evacuation(double remaining_pause_time) { + update_incremental_marker(); -bool G1OptionalCSet::evacuation_failed() { - return _evacuation_failed; -} + uint num_selected_regions; + _policy->select_optional_collection_set_regions(candidates(), + _num_optional_regions, + remaining_pause_time, + num_selected_regions); -G1OptionalCSet::~G1OptionalCSet() { - G1CollectedHeap* g1h = G1CollectedHeap::heap(); - while (!is_empty()) { - // We want to return regions not evacuated to the collection set candidates - // in reverse order to maintain the old order. - HeapRegion* hr = _cset->remove_last_optional_region(); - assert(hr != NULL, "Should be valid region left"); - _pset->record_unused_optional_region(hr); - g1h->old_set_add(hr); - g1h->clear_in_cset(hr); - hr->set_index_in_opt_cset(InvalidCSetIndex); - _cset->candidates()->push_front(hr); - } - _cset->free_optional_regions(); -} + move_candidates_to_collection_set(num_selected_regions); -uint G1OptionalCSet::size() { - return _cset->optional_region_length() - _current_index; -} + _num_optional_regions -= num_selected_regions; -bool G1OptionalCSet::is_empty() { - return size() == 0; + stop_incremental_building(); + return num_selected_regions > 0; } -void G1OptionalCSet::prepare_to_evacuate_optional_region(HeapRegion* hr) { - log_trace(gc, cset)("Adding region %u for optional evacuation", hr->hrm_index()); - G1CollectedHeap::heap()->clear_in_cset(hr); - _cset->add_old_region(hr); +void G1CollectionSet::abandon_optional_collection_set(G1ParScanThreadStateSet* pss) { + for (uint i = 0; i < _num_optional_regions; i++) { + HeapRegion* r = candidates()->at(candidates()->cur_idx() + i); + pss->record_unused_optional_region(r); + _g1h->clear_in_cset(r); + r->clear_index_in_opt_cset(); + } + _num_optional_regions = 0; } #ifdef ASSERT --- old/src/hotspot/share/gc/g1/g1CollectionSet.hpp 2019-03-13 14:03:00.215005842 +0100 +++ new/src/hotspot/share/gc/g1/g1CollectionSet.hpp 2019-03-13 14:03:00.004000112 +0100 @@ -38,11 +38,32 @@ class HeapRegion; class HeapRegionClosure; +// The collection set. +// +// The collection set is built incrementally: it starts off with the set of +// survivor regions, and at mutator time G1 adds retired up eden regions to it. +// +// For non-mixed collections this is all the collection set consists of and its +// regions are evacuated in one pass. +// +// For mixed collections we not only determine a few old gen regions for an initial +// collection set, but also a set of optional collection set regions from the +// collection set candidates. +// +// After evacuating the initial collection set, G1 incrementally selects more +// regions from the optional collection set regions as time prediction permit. +// +// Support for incremental building is implemented by keeping an index into the +// collection set set; during evacuation only the part from that index to the +// end is used for evacuation. +// +// This results in having a single complete collection set of the evacuation phases +// for cleaning up. class G1CollectionSet { G1CollectedHeap* _g1h; G1Policy* _policy; - // All old gen collection set candidate regions for the current mixed gc phase. + // All old gen collection set candidate regions for the current mixed phase. G1CollectionSetCandidates* _candidates; uint _eden_region_length; @@ -51,7 +72,7 @@ // The actual collection set as a set of region indices. // All entries in _collection_set_regions below _collection_set_cur_length are - // assumed to be valid entries. + // assumed to be part of the collection set. // We assume that at any time there is at most only one writer and (one or more) // concurrent readers. This means we are good with using storestore and loadload // barriers on the writer and reader respectively only. @@ -59,31 +80,33 @@ volatile size_t _collection_set_cur_length; size_t _collection_set_max_length; - // When doing mixed collections we can add old regions to the collection, which - // can be collected if there is enough time. We call these optional regions and - // the pointer to these regions are stored in the array below. - HeapRegion** _optional_regions; - uint _optional_region_length; - uint _optional_region_max_length; + // When doing mixed collections we can add old regions to the collection set, which + // will be collected only if there is enough time. We call these optional regions. + // This member records the current number of regions that are of that type that + // correspond to the first x entries in the collection set candidates. + uint _num_optional_regions; // The number of bytes in the collection set before the pause. Set from // the incrementally built collection set at the start of an evacuation - // pause, and incremented in finalize_old_part() when adding old regions - // (if any) to the collection set. + // pause, and updated as more regions are added to the collection set. size_t _bytes_used_before; + // The number of cards in the remembered set in the collection set. Set from + // the incrementally built collection set at the start of an evacuation + // pause, and updated as more regions are added to the collection set. size_t _recorded_rs_lengths; - // The associated information that is maintained while the incremental - // collection set is being built with young regions. Used to populate - // the recorded info for the evacuation pause. - enum CSetBuildType { Active, // We are actively building the collection set Inactive // We are not actively building the collection set }; CSetBuildType _inc_build_state; + size_t _inc_part_start; + + // The associated information that is maintained while the incremental + // collection set is being built with *young* regions. Used to populate + // the recorded info for the evacuation pause. // The number of bytes in the incrementally built collection set. // Used to set _collection_set_bytes_used_before at the start of @@ -117,10 +140,28 @@ G1GCPhaseTimes* phase_times(); void verify_young_cset_indices() const NOT_DEBUG_RETURN; - void add_as_optional(HeapRegion* hr); - void add_as_old(HeapRegion* hr); - bool optional_is_full(); + double predict_region_elapsed_time_ms(HeapRegion* hr); + + // Update the incremental collection set information when adding a region. + void add_young_region_common(HeapRegion* hr); + + // Add old region "hr" to the collection set. + void add_old_region(HeapRegion* hr); + + // Add old region "hr" to optional collection set. + void add_optional_region(HeapRegion* hr); + + void move_candidates_to_collection_set(uint num_regions); + + // Choose a new collection set. Marks the chosen regions as being + // "in_collection_set". + double finalize_young_part(double target_pause_time_ms, G1SurvivorRegions* survivors); + // Perform any final calculations on the incremental collection set fields + // before we can use them. + void finalize_young_increment(); + + void finalize_old_part(double time_remaining_ms); public: G1CollectionSet(G1CollectedHeap* g1h, G1Policy* policy); ~G1CollectionSet(); @@ -151,32 +192,29 @@ uint eden_region_length() const { return _eden_region_length; } uint survivor_region_length() const { return _survivor_region_length; } uint old_region_length() const { return _old_region_length; } - uint optional_region_length() const { return _optional_region_length; } + uint optional_region_length() const { return _num_optional_regions; } + + // Reset the contents of the collection set. + void clear(); // Incremental collection set support // Initialize incremental collection set info. void start_incremental_building(); + // Start a new collection set increment. + void update_incremental_marker() { _inc_build_state = Active; _inc_part_start = _collection_set_cur_length; } + // Stop adding regions to the current collection set increment. + void stop_incremental_building() { _inc_build_state = Inactive; } - // Perform any final calculations on the incremental collection set fields - // before we can use them. - void finalize_incremental_building(); - - // Reset the contents of the collection set. - void clear(); + // Iterate over the current collection set increment applying the given HeapRegionClosure + // from a starting position determined by the given worker id. + void iterate_incremental_part_from(HeapRegionClosure* cl, uint worker_id, uint total_workers) const; - // Iterate over the collection set, applying the given HeapRegionClosure on all of them. - // If may_be_aborted is true, iteration may be aborted using the return value of the - // called closure method. + // Iterate over the entire collection set (all increments calculated so far), applying + // the given HeapRegionClosure on all of them. void iterate(HeapRegionClosure* cl) const; - // Iterate over the collection set, applying the given HeapRegionClosure on all of them, - // trying to optimally spread out starting position of total_workers workers given the - // caller's worker_id. - void iterate_from(HeapRegionClosure* cl, uint worker_id, uint total_workers) const; - - // Stop adding regions to the incremental collection set. - void stop_incremental_building() { _inc_build_state = Inactive; } + void iterate_optional(HeapRegionClosure* cl) const; size_t recorded_rs_lengths() { return _recorded_rs_lengths; } @@ -188,16 +226,14 @@ _bytes_used_before = 0; } - // Choose a new collection set. Marks the chosen regions as being - // "in_collection_set". - double finalize_young_part(double target_pause_time_ms, G1SurvivorRegions* survivors); - void finalize_old_part(double time_remaining_ms); - - // Add old region "hr" to the collection set. - void add_old_region(HeapRegion* hr); - - // Add old region "hr" to optional collection set. - void add_optional_region(HeapRegion* hr); + // Finalize the initial (first) collection set consisting of all young regions and a + // few old gen regions. + void finalize_initial_collection_set(double target_pause_time_ms, G1SurvivorRegions* survivor); + // Finalize the next collection set from the set of available optional old gen regions. + bool finalize_optional_for_evacuation(double remaining_pause_time); + // Abandon (clean up) optional collection set regions that were not evacuated in this + // pause. + void abandon_optional_collection_set(G1ParScanThreadStateSet* pss); // Update information about hr in the aggregated information for // the incrementally built collection set. @@ -214,73 +250,6 @@ void print(outputStream* st); #endif // !PRODUCT - - double predict_region_elapsed_time_ms(HeapRegion* hr); - - void clear_optional_region(const HeapRegion* hr); - - HeapRegion* optional_region_at(uint i) const { - assert(_optional_regions != NULL, "Not yet initialized"); - assert(i < _optional_region_length, "index %u out of bounds (%u)", i, _optional_region_length); - return _optional_regions[i]; - } - - HeapRegion* remove_last_optional_region() { - assert(_optional_regions != NULL, "Not yet initialized"); - assert(_optional_region_length != 0, "No region to remove"); - _optional_region_length--; - HeapRegion* removed = _optional_regions[_optional_region_length]; - _optional_regions[_optional_region_length] = NULL; - return removed; - } - -private: - // Update the incremental collection set information when adding a region. - void add_young_region_common(HeapRegion* hr); -}; - -// Helper class to manage the optional regions in a Mixed collection. -class G1OptionalCSet : public StackObj { -private: - G1CollectionSet* _cset; - G1ParScanThreadStateSet* _pset; - uint _current_index; - uint _current_limit; - bool _prepare_failed; - bool _evacuation_failed; - - void prepare_to_evacuate_optional_region(HeapRegion* hr); - -public: - static const uint InvalidCSetIndex = UINT_MAX; - - G1OptionalCSet(G1CollectionSet* cset, G1ParScanThreadStateSet* pset) : - _cset(cset), - _pset(pset), - _current_index(0), - _current_limit(0), - _prepare_failed(false), - _evacuation_failed(false) { } - // The destructor returns regions to the collection set candidates set and - // frees the optional structure in the collection set. - ~G1OptionalCSet(); - - uint current_index() { return _current_index; } - uint current_limit() { return _current_limit; } - - uint size(); - bool is_empty(); - - HeapRegion* region_at(uint index); - - // Prepare a set of regions for optional evacuation. - void prepare_evacuation(double time_left_ms); - bool prepare_failed(); - - // Complete the evacuation of the previously prepared - // regions by updating their state and check for failures. - void complete_evacuation(); - bool evacuation_failed(); }; #endif // SHARE_GC_G1_G1COLLECTIONSET_HPP --- old/src/hotspot/share/gc/g1/g1CollectionSetCandidates.cpp 2019-03-13 14:03:01.134030800 +0100 +++ new/src/hotspot/share/gc/g1/g1CollectionSetCandidates.cpp 2019-03-13 14:03:00.913024798 +0100 @@ -27,26 +27,12 @@ #include "gc/g1/g1CollectionSetChooser.hpp" #include "gc/g1/heapRegion.inline.hpp" -HeapRegion* G1CollectionSetCandidates::pop_front() { - assert(_front_idx < _num_regions, "pre-condition"); - HeapRegion* hr = _regions[_front_idx]; - assert(hr != NULL, "pre-condition"); - _regions[_front_idx] = NULL; - assert(hr->reclaimable_bytes() <= _remaining_reclaimable_bytes, - "Remaining reclaimable bytes inconsistent " - "from region: " SIZE_FORMAT " remaining: " SIZE_FORMAT, - hr->reclaimable_bytes(), _remaining_reclaimable_bytes); - _remaining_reclaimable_bytes -= hr->reclaimable_bytes(); - _front_idx++; - return hr; -} - -void G1CollectionSetCandidates::push_front(HeapRegion* hr) { - assert(hr != NULL, "Can't put back a NULL region"); - assert(_front_idx >= 1, "Too many regions have been put back."); - _front_idx--; - _regions[_front_idx] = hr; - _remaining_reclaimable_bytes += hr->reclaimable_bytes(); +void G1CollectionSetCandidates::remove(uint num_regions) { + assert(num_regions <= num_remaining(), "Trying to remove more regions (%u) than available (%u)", num_regions, num_remaining()); + for (uint i = 0; i < num_regions; i++) { + _remaining_reclaimable_bytes -= at(_front_idx)->reclaimable_bytes(); + _front_idx++; + } } void G1CollectionSetCandidates::iterate(HeapRegionClosure* cl) { @@ -62,13 +48,8 @@ #ifndef PRODUCT void G1CollectionSetCandidates::verify() const { guarantee(_front_idx <= _num_regions, "Index: %u Num_regions: %u", _front_idx, _num_regions); - uint idx = 0; + uint idx = _front_idx; size_t sum_of_reclaimable_bytes = 0; - while (idx < _front_idx) { - guarantee(_regions[idx] == NULL, "All entries before _front_idx %u should be NULL, but %u is not", - _front_idx, idx); - idx++; - } HeapRegion *prev = NULL; for (; idx < _num_regions; idx++) { HeapRegion *cur = _regions[idx]; --- old/src/hotspot/share/gc/g1/g1CollectionSetCandidates.hpp 2019-03-13 14:03:02.040055405 +0100 +++ new/src/hotspot/share/gc/g1/g1CollectionSetCandidates.hpp 2019-03-13 14:03:01.828049647 +0100 @@ -63,22 +63,18 @@ // Returns the total number of collection set candidate old regions added. uint num_regions() { return _num_regions; } - // Return the candidate region at the cursor position to be considered for collection without - // removing it. - HeapRegion* peek_front() { + uint cur_idx() const { return _front_idx; } + + HeapRegion* at(uint idx) const { HeapRegion* res = NULL; - if (_front_idx < _num_regions) { - res = _regions[_front_idx]; - assert(res != NULL, "Unexpected NULL HeapRegion at index %u", _front_idx); + if (idx < _num_regions) { + res = _regions[idx]; + assert(res != NULL, "Unexpected NULL HeapRegion at index %u", idx); } return res; } - // Remove the given region from the candidates set and move the cursor to the next one. - HeapRegion* pop_front(); - - // Add the given HeapRegion to the front of the collection set candidate set again. - void push_front(HeapRegion* hr); + void remove(uint num_regions); // Iterate over all remaining collection set candidate regions. void iterate(HeapRegionClosure* cl); --- old/src/hotspot/share/gc/g1/g1ConcurrentMark.cpp 2019-03-13 14:03:02.941079874 +0100 +++ new/src/hotspot/share/gc/g1/g1ConcurrentMark.cpp 2019-03-13 14:03:02.729074116 +0100 @@ -1940,9 +1940,10 @@ guarantee(oopDesc::is_oop(task_entry.obj()), "Non-oop " PTR_FORMAT ", phase: %s, info: %d", p2i(task_entry.obj()), _phase, _info); - guarantee(!_g1h->is_in_cset(task_entry.obj()), - "obj: " PTR_FORMAT " in CSet, phase: %s, info: %d", - p2i(task_entry.obj()), _phase, _info); + HeapRegion* r = _g1h->heap_region_containing(task_entry.obj()); + guarantee(!(r->in_collection_set() || r->has_index_in_opt_cset()), + "obj " PTR_FORMAT " from %s (%d) in region %u in (optional) collection set", + p2i(task_entry.obj()), _phase, _info, r->hrm_index()); } }; @@ -1979,11 +1980,11 @@ HeapWord* task_finger = task->finger(); if (task_finger != NULL && task_finger < _heap.end()) { // See above note on the global finger verification. - HeapRegion* task_hr = _g1h->heap_region_containing(task_finger); - guarantee(task_hr == NULL || task_finger == task_hr->bottom() || - !task_hr->in_collection_set(), + HeapRegion* r = _g1h->heap_region_containing(task_finger); + guarantee(r == NULL || task_finger == r->bottom() || + !r->in_collection_set() || !r->has_index_in_opt_cset(), "task finger: " PTR_FORMAT " region: " HR_FORMAT, - p2i(task_finger), HR_FORMAT_PARAMS(task_hr)); + p2i(task_finger), HR_FORMAT_PARAMS(r)); } } } --- old/src/hotspot/share/gc/g1/g1EvacFailure.cpp 2019-03-13 14:03:03.858104778 +0100 +++ new/src/hotspot/share/gc/g1/g1EvacFailure.cpp 2019-03-13 14:03:03.647099047 +0100 @@ -228,6 +228,8 @@ if (_hrclaimer->claim_region(hr->hrm_index())) { if (hr->evacuation_failed()) { + hr->clear_index_in_opt_cset(); + bool during_initial_mark = _g1h->collector_state()->in_initial_mark_gc(); bool during_conc_mark = _g1h->collector_state()->mark_or_rebuild_in_progress(); --- old/src/hotspot/share/gc/g1/g1GCPhaseTimes.cpp 2019-03-13 14:03:04.758129219 +0100 +++ new/src/hotspot/share/gc/g1/g1GCPhaseTimes.cpp 2019-03-13 14:03:04.545123435 +0100 @@ -76,10 +76,12 @@ } _gc_par_phases[ScanRS] = new WorkerDataArray(max_gc_threads, "Scan RS (ms):"); _gc_par_phases[OptScanRS] = new WorkerDataArray(max_gc_threads, "Optional Scan RS (ms):"); - _gc_par_phases[CodeRoots] = new WorkerDataArray(max_gc_threads, "Code Root Scanning (ms):"); + _gc_par_phases[CodeRoots] = new WorkerDataArray(max_gc_threads, "Code Root Scan (ms):"); + _gc_par_phases[OptCodeRoots] = new WorkerDataArray(max_gc_threads, "Optional Code Root Scan (ms):"); _gc_par_phases[ObjCopy] = new WorkerDataArray(max_gc_threads, "Object Copy (ms):"); _gc_par_phases[OptObjCopy] = new WorkerDataArray(max_gc_threads, "Optional Object Copy (ms):"); _gc_par_phases[Termination] = new WorkerDataArray(max_gc_threads, "Termination (ms):"); + _gc_par_phases[OptTermination] = new WorkerDataArray(max_gc_threads, "Optional Termination (ms):"); _gc_par_phases[GCWorkerTotal] = new WorkerDataArray(max_gc_threads, "GC Worker Total (ms):"); _gc_par_phases[GCWorkerEnd] = new WorkerDataArray(max_gc_threads, "GC Worker End (ms):"); _gc_par_phases[Other] = new WorkerDataArray(max_gc_threads, "GC Worker Other (ms):"); @@ -91,14 +93,16 @@ _scan_rs_skipped_cards = new WorkerDataArray(max_gc_threads, "Skipped Cards:"); _gc_par_phases[ScanRS]->link_thread_work_items(_scan_rs_skipped_cards, ScanRSSkippedCards); - _opt_cset_scanned_cards = new WorkerDataArray(max_gc_threads, "Scanned Cards:"); - _gc_par_phases[OptScanRS]->link_thread_work_items(_opt_cset_scanned_cards, OptCSetScannedCards); - _opt_cset_claimed_cards = new WorkerDataArray(max_gc_threads, "Claimed Cards:"); - _gc_par_phases[OptScanRS]->link_thread_work_items(_opt_cset_claimed_cards, OptCSetClaimedCards); - _opt_cset_skipped_cards = new WorkerDataArray(max_gc_threads, "Skipped Cards:"); - _gc_par_phases[OptScanRS]->link_thread_work_items(_opt_cset_skipped_cards, OptCSetSkippedCards); - _opt_cset_used_memory = new WorkerDataArray(max_gc_threads, "Used Memory:"); - _gc_par_phases[OptScanRS]->link_thread_work_items(_opt_cset_used_memory, OptCSetUsedMemory); + _opt_scan_rs_scanned_cards = new WorkerDataArray(max_gc_threads, "Scanned Cards:"); + _gc_par_phases[OptScanRS]->link_thread_work_items(_opt_scan_rs_scanned_cards, ScanRSScannedCards); + _opt_scan_rs_claimed_cards = new WorkerDataArray(max_gc_threads, "Claimed Cards:"); + _gc_par_phases[OptScanRS]->link_thread_work_items(_opt_scan_rs_claimed_cards, ScanRSClaimedCards); + _opt_scan_rs_skipped_cards = new WorkerDataArray(max_gc_threads, "Skipped Cards:"); + _gc_par_phases[OptScanRS]->link_thread_work_items(_opt_scan_rs_skipped_cards, ScanRSSkippedCards); + _opt_scan_rs_scanned_opt_refs = new WorkerDataArray(max_gc_threads, "Scanned Refs:"); + _gc_par_phases[OptScanRS]->link_thread_work_items(_opt_scan_rs_scanned_opt_refs, ScanRSScannedOptRefs); + _opt_scan_rs_used_memory = new WorkerDataArray(max_gc_threads, "Used Memory:"); + _gc_par_phases[OptScanRS]->link_thread_work_items(_opt_scan_rs_used_memory, ScanRSUsedMemory); _update_rs_processed_buffers = new WorkerDataArray(max_gc_threads, "Processed Buffers:"); _gc_par_phases[UpdateRS]->link_thread_work_items(_update_rs_processed_buffers, UpdateRSProcessedBuffers); @@ -112,9 +116,17 @@ _obj_copy_lab_undo_waste = new WorkerDataArray(max_gc_threads, "LAB Undo Waste"); _gc_par_phases[ObjCopy]->link_thread_work_items(_obj_copy_lab_undo_waste, ObjCopyLABUndoWaste); + _opt_obj_copy_lab_waste = new WorkerDataArray(max_gc_threads, "LAB Waste"); + _gc_par_phases[OptObjCopy]->link_thread_work_items(_obj_copy_lab_waste, ObjCopyLABWaste); + _opt_obj_copy_lab_undo_waste = new WorkerDataArray(max_gc_threads, "LAB Undo Waste"); + _gc_par_phases[OptObjCopy]->link_thread_work_items(_obj_copy_lab_undo_waste, ObjCopyLABUndoWaste); + _termination_attempts = new WorkerDataArray(max_gc_threads, "Termination Attempts:"); _gc_par_phases[Termination]->link_thread_work_items(_termination_attempts); + _opt_termination_attempts = new WorkerDataArray(max_gc_threads, "Optional Termination Attempts:"); + _gc_par_phases[OptTermination]->link_thread_work_items(_opt_termination_attempts); + if (UseStringDeduplication) { _gc_par_phases[StringDedupQueueFixup] = new WorkerDataArray(max_gc_threads, "Queue Fixup (ms):"); _gc_par_phases[StringDedupTableFixup] = new WorkerDataArray(max_gc_threads, "Table Fixup (ms):"); @@ -134,7 +146,7 @@ } void G1GCPhaseTimes::reset() { - _cur_collection_par_time_ms = 0.0; + _cur_collection_initial_evac_time_ms = 0.0; _cur_optional_evac_ms = 0.0; _cur_collection_code_root_fixup_time_ms = 0.0; _cur_strong_code_root_purge_time_ms = 0.0; @@ -251,6 +263,10 @@ } } +double G1GCPhaseTimes::get_time_secs(GCParPhases phase, uint worker_i) { + return _gc_par_phases[phase]->get(worker_i); +} + void G1GCPhaseTimes::record_thread_work_item(GCParPhases phase, uint worker_i, size_t count, uint index) { _gc_par_phases[phase]->set_thread_work_item(worker_i, count, index); } @@ -259,6 +275,10 @@ _gc_par_phases[phase]->set_or_add_thread_work_item(worker_i, count, index); } +size_t G1GCPhaseTimes::get_thread_work_item(GCParPhases phase, uint worker_i, uint index) { + return _gc_par_phases[phase]->get_thread_work_item(worker_i, index); +} + // return the average time for a phase in milliseconds double G1GCPhaseTimes::average_time_ms(GCParPhases phase) { return _gc_par_phases[phase]->average() * 1000.0; @@ -374,12 +394,14 @@ info_time("Evacuate Optional Collection Set", sum_ms); debug_phase(_gc_par_phases[OptScanRS]); debug_phase(_gc_par_phases[OptObjCopy]); + debug_phase(_gc_par_phases[OptCodeRoots]); + debug_phase(_gc_par_phases[OptTermination]); } return sum_ms; } double G1GCPhaseTimes::print_evacuate_collection_set() const { - const double sum_ms = _cur_collection_par_time_ms; + const double sum_ms = _cur_collection_initial_evac_time_ms; info_time("Evacuate Collection Set", sum_ms); @@ -517,9 +539,11 @@ "ScanRS", "OptScanRS", "CodeRoots", + "OptCodeRoots", "ObjCopy", "OptObjCopy", "Termination", + "OptTermination", "Other", "GCWorkerTotal", "GCWorkerEnd", --- old/src/hotspot/share/gc/g1/g1GCPhaseTimes.hpp 2019-03-13 14:03:05.663153797 +0100 +++ new/src/hotspot/share/gc/g1/g1GCPhaseTimes.hpp 2019-03-13 14:03:05.454148121 +0100 @@ -67,9 +67,11 @@ ScanRS, OptScanRS, CodeRoots, + OptCodeRoots, ObjCopy, OptObjCopy, Termination, + OptTermination, Other, GCWorkerTotal, GCWorkerEnd, @@ -87,7 +89,9 @@ enum GCScanRSWorkItems { ScanRSScannedCards, ScanRSClaimedCards, - ScanRSSkippedCards + ScanRSSkippedCards, + ScanRSScannedOptRefs, + ScanRSUsedMemory }; enum GCUpdateRSWorkItems { @@ -101,13 +105,6 @@ ObjCopyLABUndoWaste }; - enum GCOptCSetWorkItems { - OptCSetScannedCards, - OptCSetClaimedCards, - OptCSetSkippedCards, - OptCSetUsedMemory - }; - private: // Markers for grouping the phases in the GCPhases enum above static const int GCMainParPhasesLast = GCWorkerEnd; @@ -122,19 +119,25 @@ WorkerDataArray* _scan_rs_claimed_cards; WorkerDataArray* _scan_rs_skipped_cards; + WorkerDataArray* _opt_scan_rs_scanned_cards; + WorkerDataArray* _opt_scan_rs_claimed_cards; + WorkerDataArray* _opt_scan_rs_skipped_cards; + WorkerDataArray* _opt_scan_rs_scanned_opt_refs; + WorkerDataArray* _opt_scan_rs_used_memory; + WorkerDataArray* _obj_copy_lab_waste; WorkerDataArray* _obj_copy_lab_undo_waste; - WorkerDataArray* _opt_cset_scanned_cards; - WorkerDataArray* _opt_cset_claimed_cards; - WorkerDataArray* _opt_cset_skipped_cards; - WorkerDataArray* _opt_cset_used_memory; + WorkerDataArray* _opt_obj_copy_lab_waste; + WorkerDataArray* _opt_obj_copy_lab_undo_waste; WorkerDataArray* _termination_attempts; + WorkerDataArray* _opt_termination_attempts; + WorkerDataArray* _redirtied_cards; - double _cur_collection_par_time_ms; + double _cur_collection_initial_evac_time_ms; double _cur_optional_evac_ms; double _cur_collection_code_root_fixup_time_ms; double _cur_strong_code_root_purge_time_ms; @@ -225,10 +228,14 @@ void record_or_add_time_secs(GCParPhases phase, uint worker_i, double secs); + double get_time_secs(GCParPhases phase, uint worker_i); + void record_thread_work_item(GCParPhases phase, uint worker_i, size_t count, uint index = 0); void record_or_add_thread_work_item(GCParPhases phase, uint worker_i, size_t count, uint index = 0); + size_t get_thread_work_item(GCParPhases phase, uint worker_i, uint index = 0); + // return the average time for a phase in milliseconds double average_time_ms(GCParPhases phase); @@ -256,16 +263,16 @@ _cur_expand_heap_time_ms = ms; } - void record_par_time(double ms) { - _cur_collection_par_time_ms = ms; + void record_or_add_initial_evac_time(double ms) { + _cur_collection_initial_evac_time_ms += ms; } void record_optional_evacuation(double ms) { _cur_optional_evac_ms = ms; } - void record_code_root_fixup_time(double ms) { - _cur_collection_code_root_fixup_time_ms = ms; + void record_or_add_code_root_fixup_time(double ms) { + _cur_collection_code_root_fixup_time_ms += ms; } void record_strong_code_root_purge_time(double ms) { @@ -360,7 +367,7 @@ } double cur_collection_par_time_ms() { - return _cur_collection_par_time_ms; + return _cur_collection_initial_evac_time_ms; } double cur_clear_ct_time_ms() { --- old/src/hotspot/share/gc/g1/g1HeapVerifier.cpp 2019-03-13 14:03:06.568178375 +0100 +++ new/src/hotspot/share/gc/g1/g1HeapVerifier.cpp 2019-03-13 14:03:06.356172617 +0100 @@ -371,6 +371,7 @@ } bool do_heap_region(HeapRegion* r) { + guarantee(!r->has_index_in_opt_cset(), "Region %u still has opt collection set index %u", r->hrm_index(), r->index_in_opt_cset()); guarantee(!r->is_young() || r->rem_set()->is_complete(), "Remembered set for Young region %u must be complete, is %s", r->hrm_index(), r->rem_set()->get_state_str()); // Humongous and old regions regions might be of any state, so can't check here. guarantee(!r->is_free() || !r->rem_set()->is_tracked(), "Remembered set for free region %u must be untracked, is %s", r->hrm_index(), r->rem_set()->get_state_str()); --- old/src/hotspot/share/gc/g1/g1OopStarChunkedList.cpp 2019-03-13 14:03:07.475203007 +0100 +++ new/src/hotspot/share/gc/g1/g1OopStarChunkedList.cpp 2019-03-13 14:03:07.265197304 +0100 @@ -32,9 +32,11 @@ delete_list(_coops); } -void G1OopStarChunkedList::oops_do(OopClosure* obj_cl, OopClosure* root_cl) { - chunks_do(_roots, root_cl); - chunks_do(_croots, root_cl); - chunks_do(_oops, obj_cl); - chunks_do(_coops, obj_cl); +size_t G1OopStarChunkedList::oops_do(OopClosure* obj_cl, OopClosure* root_cl) { + size_t result = 0; + result += chunks_do(_roots, root_cl); + result += chunks_do(_croots, root_cl); + result += chunks_do(_oops, obj_cl); + result += chunks_do(_coops, obj_cl); + return result; } --- old/src/hotspot/share/gc/g1/g1OopStarChunkedList.hpp 2019-03-13 14:03:08.377227503 +0100 +++ new/src/hotspot/share/gc/g1/g1OopStarChunkedList.hpp 2019-03-13 14:03:08.160221610 +0100 @@ -41,8 +41,8 @@ template void delete_list(ChunkedList* c); template - void chunks_do(ChunkedList* head, - OopClosure* cl); + size_t chunks_do(ChunkedList* head, + OopClosure* cl); template inline void push(ChunkedList** field, T* p); @@ -53,7 +53,7 @@ size_t used_memory() { return _used_memory; } - void oops_do(OopClosure* obj_cl, OopClosure* root_cl); + size_t oops_do(OopClosure* obj_cl, OopClosure* root_cl); inline void push_oop(oop* p); inline void push_oop(narrowOop* p); --- old/src/hotspot/share/gc/g1/g1OopStarChunkedList.inline.hpp 2019-03-13 14:03:09.279251999 +0100 +++ new/src/hotspot/share/gc/g1/g1OopStarChunkedList.inline.hpp 2019-03-13 14:03:09.069246296 +0100 @@ -72,13 +72,16 @@ } template -void G1OopStarChunkedList::chunks_do(ChunkedList* head, OopClosure* cl) { +size_t G1OopStarChunkedList::chunks_do(ChunkedList* head, OopClosure* cl) { + size_t result = 0; for (ChunkedList* c = head; c != NULL; c = c->next_used()) { + result += c->size(); for (size_t i = 0; i < c->size(); i++) { T* p = c->at(i); cl->do_oop(p); } } + return result; } #endif // SHARE_GC_G1_G1OOPSTARCHUNKEDLIST_INLINE_HPP --- old/src/hotspot/share/gc/g1/g1ParScanThreadState.cpp 2019-03-13 14:03:10.181276496 +0100 +++ new/src/hotspot/share/gc/g1/g1ParScanThreadState.cpp 2019-03-13 14:03:09.971270792 +0100 @@ -372,7 +372,7 @@ } size_t used_memory = pss->oops_into_optional_region(hr)->used_memory(); - _g1h->phase_times()->record_or_add_thread_work_item(G1GCPhaseTimes::OptScanRS, worker_index, used_memory, G1GCPhaseTimes::OptCSetUsedMemory); + _g1h->phase_times()->record_or_add_thread_work_item(G1GCPhaseTimes::OptScanRS, worker_index, used_memory, G1GCPhaseTimes::ScanRSUsedMemory); } } --- old/src/hotspot/share/gc/g1/g1Policy.cpp 2019-03-13 14:03:11.084301019 +0100 +++ new/src/hotspot/share/gc/g1/g1Policy.cpp 2019-03-13 14:03:10.873295289 +0100 @@ -659,7 +659,7 @@ double cost_per_entry_ms = 0.0; if (cards_scanned > 10) { - cost_per_entry_ms = average_time_ms(G1GCPhaseTimes::ScanRS) / (double) cards_scanned; + cost_per_entry_ms = (average_time_ms(G1GCPhaseTimes::ScanRS) + average_time_ms(G1GCPhaseTimes::OptScanRS)) / (double) cards_scanned; _analytics->report_cost_per_entry_ms(cost_per_entry_ms, this_pause_was_young_only); } @@ -694,7 +694,7 @@ double cost_per_byte_ms = 0.0; if (copied_bytes > 0) { - cost_per_byte_ms = average_time_ms(G1GCPhaseTimes::ObjCopy) / (double) copied_bytes; + cost_per_byte_ms = (average_time_ms(G1GCPhaseTimes::ObjCopy) + average_time_ms(G1GCPhaseTimes::OptObjCopy)) / (double) copied_bytes; _analytics->report_cost_per_byte_ms(cost_per_byte_ms, collector_state()->mark_or_rebuild_in_progress()); } @@ -1188,11 +1188,136 @@ return (uint) result; } -uint G1Policy::finalize_collection_set(double target_pause_time_ms, G1SurvivorRegions* survivor) { - double time_remaining_ms = _collection_set->finalize_young_part(target_pause_time_ms, survivor); - _collection_set->finalize_old_part(time_remaining_ms); +void G1Policy::select_old_collection_set_regions(G1CollectionSetCandidates* candidates, + double time_remaining_ms, + uint& num_expensive_regions, + uint& num_initial_regions, + uint& num_optional_regions) { + assert(candidates != NULL, "Must be"); + + num_initial_regions = 0; + num_optional_regions = 0; + num_expensive_regions = 0; + + double predicted_old_time_ms = 0.0; + double predicted_initial_time_ms = 0.0; + double predicted_optional_time_ms = 0.0; + + double optional_threshold_ms = time_remaining_ms * optional_prediction_fraction(); + + const uint min_old_cset_length = calc_min_old_cset_length(); + const uint max_old_cset_length = MAX2(min_old_cset_length, calc_max_old_cset_length()); + const uint max_optional_regions = max_old_cset_length - min_old_cset_length; + bool check_time_remaining = adaptive_young_list_length(); + + uint candidate_idx = candidates->cur_idx(); + + log_debug(gc, ergo, cset)("Start adding old regions to collection set. Min %u regions, max %u regions, " + "time remaining %1.2fms, optional threshold %1.2fms", + min_old_cset_length, max_old_cset_length, time_remaining_ms, optional_threshold_ms); + + HeapRegion* hr = candidates->at(candidate_idx); + while (hr != NULL) { + if (num_initial_regions + num_optional_regions >= max_old_cset_length) { + // Added maximum number of old regions to the CSet. + log_debug(gc, ergo, cset)("Finish adding old regions to collection set (Maximum number of regions). " + "Initial %u regions, optional %u regions", + num_initial_regions, num_optional_regions); + break; + } + + // Stop adding regions if the remaining reclaimable space is + // not above G1HeapWastePercent. + size_t reclaimable_bytes = candidates->remaining_reclaimable_bytes(); + double reclaimable_percent = reclaimable_bytes_percent(reclaimable_bytes); + double threshold = (double) G1HeapWastePercent; + if (reclaimable_percent <= threshold) { + // We've added enough old regions that the amount of uncollected + // reclaimable space is at or below the waste threshold. Stop + // adding old regions to the CSet. + log_debug(gc, ergo, cset)("Finish adding old regions to collection set (Reclaimable percentage below threshold). " + "Reclaimable: " SIZE_FORMAT "%s (%1.2f%%) threshold: " UINTX_FORMAT "%%", + byte_size_in_proper_unit(reclaimable_bytes), proper_unit_for_byte_size(reclaimable_bytes), + reclaimable_percent, G1HeapWastePercent); + break; + } + + double predicted_time_ms = predict_region_elapsed_time_ms(hr, false); + time_remaining_ms = MAX2(time_remaining_ms - predicted_time_ms, 0.0); + // Add regions to old set until we reach the minimum amount + if (num_initial_regions < min_old_cset_length) { + predicted_old_time_ms += predicted_time_ms; + num_initial_regions++; + // Record the number of regions added with no time remaining + if (time_remaining_ms == 0.0) { + num_expensive_regions++; + } + } else if (!check_time_remaining) { + // In the non-auto-tuning case, we'll finish adding regions + // to the CSet if we reach the minimum. + log_debug(gc, ergo, cset)("Finish adding old regions to collection set (Region amount reached min)."); + break; + } else { + // Keep adding regions to old set until we reach the optional threshold + if (time_remaining_ms > optional_threshold_ms) { + predicted_old_time_ms += predicted_time_ms; + num_initial_regions++; + } else if (time_remaining_ms > 0) { + // Keep adding optional regions until time is up. + assert(num_optional_regions < max_optional_regions, "Should not be possible."); + predicted_optional_time_ms += predicted_time_ms; + num_optional_regions++; + } else { + log_debug(gc, ergo, cset)("Finish adding old regions to collection set (Predicted time too high)."); + break; + } + } + hr = candidates->at(++candidate_idx); + } + if (hr == NULL) { + log_debug(gc, ergo, cset)("Old candidate collection set empty."); + } + + if (num_expensive_regions > 0) { + log_debug(gc, ergo, cset)("Added %u initial old regions to collection set although the predicted time was too high.", + num_expensive_regions); + } + + log_debug(gc, ergo, cset)("Finish choosing collection set old regions. Initial: %u, optional: %u, " + "predicted old time: %1.2fms, predicted optional time: %1.2fms, time remaining: %1.2f", + num_initial_regions, num_optional_regions, + predicted_initial_time_ms, predicted_optional_time_ms, time_remaining_ms); +} + +void G1Policy::select_optional_collection_set_regions(G1CollectionSetCandidates* candidates, + uint const max_optional_regions, + double time_remaining_ms, + uint& num_optional_regions) { + assert(_g1h->collector_state()->in_mixed_phase(), "Should only be called in mixed phase"); + + num_optional_regions = 0; + double prediction_ms = 0; + uint candidate_idx = candidates->cur_idx(); + + HeapRegion* r = candidates->at(candidate_idx); + while (num_optional_regions < max_optional_regions) { + assert(r != NULL, "Region must exist"); + prediction_ms += predict_region_elapsed_time_ms(r, false); + + if (prediction_ms > time_remaining_ms) { + log_debug(gc, ergo, cset)("Prediction %.3fms for region %u does not fit remaining time: %.3fms.", + prediction_ms, r->hrm_index(), time_remaining_ms); + break; + } + // This region will be included in the next optional evacuation. + + time_remaining_ms -= prediction_ms; + num_optional_regions++; + r = candidates->at(++candidate_idx); + } - return _collection_set->region_length(); + log_debug(gc, ergo, cset)("Prepared %u regions out of %u for optional evacuation. Predicted time: %.3fms", + num_optional_regions, max_optional_regions, prediction_ms); } void G1Policy::transfer_survivors_to_cset(const G1SurvivorRegions* survivors) { --- old/src/hotspot/share/gc/g1/g1Policy.hpp 2019-03-13 14:03:12.000325895 +0100 +++ new/src/hotspot/share/gc/g1/g1Policy.hpp 2019-03-13 14:03:11.789320165 +0100 @@ -44,6 +44,7 @@ class HeapRegion; class G1CollectionSet; +class G1CollectionSetCandidates; class G1CollectionSetChooser; class G1IHOPControl; class G1Analytics; @@ -344,7 +345,24 @@ bool next_gc_should_be_mixed(const char* true_action_str, const char* false_action_str) const; - uint finalize_collection_set(double target_pause_time_ms, G1SurvivorRegions* survivor); + // Select and return the number of initial and optional old gen regions from + // the given collection set candidates and the remaining time. Num_expensive_regions + // returns the number of regions taken within the initial region count because of + // the minimum old gen region requirement. + void select_old_collection_set_regions(G1CollectionSetCandidates* candidates, + double time_remaining_ms, + uint& num_expensive_regions, + uint& num_initial_regions, + uint& num_optional_regions); + + // Select the number of optional regions from the given collection set candidates, + // the remaining time and the maximum number of these regions and return in + // num_optional_regions. + void select_optional_collection_set_regions(G1CollectionSetCandidates* candidates, + uint const max_optional_regions, + double time_remaining_ms, + uint& num_optional_regions); + private: // Set the state to start a concurrent marking cycle and clear // _initiate_conc_mark_if_possible because it has now been @@ -403,11 +421,13 @@ AgeTable _survivors_age_table; size_t desired_survivor_size(uint max_regions) const; -public: + // Fraction used when predicting how many optional regions to include in // the CSet. This fraction of the available time is used for optional regions, // the rest is used to add old regions to the normal CSet. double optional_prediction_fraction() { return 0.2; } + +public: // Fraction used when evacuating the optional regions. This fraction of the // remaining time is used to choose what regions to include in the evacuation. double optional_evacuation_fraction() { return 0.75; } --- old/src/hotspot/share/gc/g1/g1RemSet.cpp 2019-03-13 14:03:12.904350446 +0100 +++ new/src/hotspot/share/gc/g1/g1RemSet.cpp 2019-03-13 14:03:12.693344716 +0100 @@ -315,6 +315,8 @@ _scan_state(scan_state), _phase(phase), _worker_i(worker_i), + _opt_refs_scanned(0), + _opt_refs_memory_used(0), _cards_scanned(0), _cards_claimed(0), _cards_skipped(0), @@ -337,6 +339,19 @@ _cards_scanned++; } +void G1ScanRSForRegionClosure::scan_opt_rem_set_roots(HeapRegion* r){ + EventGCPhaseParallel event; + + G1OopStarChunkedList* opt_rem_set_list = _pss->oops_into_optional_region(r); + + G1ScanObjsDuringScanRSClosure scan_cl(_g1h, _pss); + G1ScanRSForOptionalClosure cl(&scan_cl); + _opt_refs_scanned += opt_rem_set_list->oops_do(&cl, _pss->closures()->raw_strong_oops()); + _opt_refs_memory_used += opt_rem_set_list->used_memory(); + + event.commit(GCId::current(), _worker_i, G1GCPhaseTimes::phase_name(_phase)); +} + void G1ScanRSForRegionClosure::scan_rem_set_roots(HeapRegion* r) { EventGCPhaseParallel event; uint const region_idx = r->hrm_index(); @@ -413,11 +428,16 @@ } bool G1ScanRSForRegionClosure::do_heap_region(HeapRegion* r) { - assert(r->in_collection_set(), - "Should only be called on elements of the collection set but region %u is not.", - r->hrm_index()); + assert(r->in_collection_set(), "Region %u is not in the collection set.", r->hrm_index()); uint const region_idx = r->hrm_index(); + // The individual references for the optional remembered set are per-worker, so we + // always need to scan them. + if (r->has_index_in_opt_cset()) { + G1EvacPhaseWithTrimTimeTracker timer(_pss, _rem_set_root_scan_time, _rem_set_trim_partially_time); + scan_opt_rem_set_roots(r); + } + // Do an early out if we know we are complete. if (_scan_state->iter_is_complete(region_idx)) { return false; @@ -436,22 +456,33 @@ return false; } -void G1RemSet::scan_rem_set(G1ParScanThreadState* pss, uint worker_i) { +void G1RemSet::scan_rem_set(G1ParScanThreadState* pss, + uint worker_i, + G1GCPhaseTimes::GCParPhases scan_phase, + G1GCPhaseTimes::GCParPhases objcopy_phase, + G1GCPhaseTimes::GCParPhases coderoots_phase) { + assert(pss->trim_ticks().value() == 0, "Queues must have been trimmed before entering."); + G1ScanObjsDuringScanRSClosure scan_cl(_g1h, pss); - G1ScanRSForRegionClosure cl(_scan_state, &scan_cl, pss, G1GCPhaseTimes::ScanRS, worker_i); + G1ScanRSForRegionClosure cl(_scan_state, &scan_cl, pss, scan_phase, worker_i); _g1h->collection_set_iterate_from(&cl, worker_i); G1GCPhaseTimes* p = _g1p->phase_times(); - p->record_time_secs(G1GCPhaseTimes::ScanRS, worker_i, cl.rem_set_root_scan_time().seconds()); - p->add_time_secs(G1GCPhaseTimes::ObjCopy, worker_i, cl.rem_set_trim_partially_time().seconds()); + p->record_or_add_time_secs(objcopy_phase, worker_i, cl.rem_set_trim_partially_time().seconds()); - p->record_thread_work_item(G1GCPhaseTimes::ScanRS, worker_i, cl.cards_scanned(), G1GCPhaseTimes::ScanRSScannedCards); - p->record_thread_work_item(G1GCPhaseTimes::ScanRS, worker_i, cl.cards_claimed(), G1GCPhaseTimes::ScanRSClaimedCards); - p->record_thread_work_item(G1GCPhaseTimes::ScanRS, worker_i, cl.cards_skipped(), G1GCPhaseTimes::ScanRSSkippedCards); + p->record_or_add_time_secs(scan_phase, worker_i, cl.rem_set_root_scan_time().seconds()); + p->record_or_add_thread_work_item(scan_phase, worker_i, cl.cards_scanned(), G1GCPhaseTimes::ScanRSScannedCards); + p->record_or_add_thread_work_item(scan_phase, worker_i, cl.cards_claimed(), G1GCPhaseTimes::ScanRSClaimedCards); + p->record_or_add_thread_work_item(scan_phase, worker_i, cl.cards_skipped(), G1GCPhaseTimes::ScanRSSkippedCards); + // At this time we only record some metrics for the optional remembered set. + if (scan_phase == G1GCPhaseTimes::OptScanRS) { + p->record_or_add_thread_work_item(scan_phase, worker_i, cl.opt_refs_scanned(), G1GCPhaseTimes::ScanRSScannedOptRefs); + p->record_or_add_thread_work_item(scan_phase, worker_i, cl.opt_refs_memory_used(), G1GCPhaseTimes::ScanRSUsedMemory); + } - p->record_time_secs(G1GCPhaseTimes::CodeRoots, worker_i, cl.strong_code_root_scan_time().seconds()); - p->add_time_secs(G1GCPhaseTimes::ObjCopy, worker_i, cl.strong_code_root_trim_partially_time().seconds()); + p->record_or_add_time_secs(coderoots_phase, worker_i, cl.strong_code_root_scan_time().seconds()); + p->add_time_secs(objcopy_phase, worker_i, cl.strong_code_root_trim_partially_time().seconds()); } // Closure used for updating rem sets. Only called during an evacuation pause. @@ -513,11 +544,6 @@ } } -void G1RemSet::oops_into_collection_set_do(G1ParScanThreadState* pss, uint worker_i) { - update_rem_set(pss, worker_i); - scan_rem_set(pss, worker_i);; -} - void G1RemSet::prepare_for_oops_into_collection_set_do() { G1DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set(); dcqs.concatenate_logs(); --- old/src/hotspot/share/gc/g1/g1RemSet.hpp 2019-03-13 14:03:13.814375159 +0100 +++ new/src/hotspot/share/gc/g1/g1RemSet.hpp 2019-03-13 14:03:13.604369456 +0100 @@ -60,14 +60,6 @@ G1RemSetSummary _prev_period_summary; - // Scan all remembered sets of the collection set for references into the collection - // set. - void scan_rem_set(G1ParScanThreadState* pss, uint worker_i); - - // Flush remaining refinement buffers for cross-region references to either evacuate references - // into the collection set or update the remembered set. - void update_rem_set(G1ParScanThreadState* pss, uint worker_i); - G1CollectedHeap* _g1h; size_t _num_conc_refined_cards; // Number of cards refined concurrently to the mutator. @@ -93,12 +85,19 @@ G1HotCardCache* hot_card_cache); ~G1RemSet(); - // Process all oops in the collection set from the cards in the refinement buffers and - // remembered sets using pss. - // + // Scan all remembered sets of the collection set for references into the collection + // set. // Further applies heap_region_codeblobs on the oops of the unmarked nmethods on the strong code // roots list for each region in the collection set. - void oops_into_collection_set_do(G1ParScanThreadState* pss, uint worker_i); + void scan_rem_set(G1ParScanThreadState* pss, + uint worker_i, + G1GCPhaseTimes::GCParPhases scan_phase, + G1GCPhaseTimes::GCParPhases objcopy_phase, + G1GCPhaseTimes::GCParPhases coderoots_phase); + + // Flush remaining refinement buffers for cross-region references to either evacuate references + // into the collection set or update the remembered set. + void update_rem_set(G1ParScanThreadState* pss, uint worker_i); // Prepare for and cleanup after an oops_into_collection_set_do // call. Must call each of these once before and after (in sequential @@ -144,6 +143,9 @@ uint _worker_i; + size_t _opt_refs_scanned; + size_t _opt_refs_memory_used; + size_t _cards_scanned; size_t _cards_claimed; size_t _cards_skipped; @@ -157,6 +159,7 @@ void claim_card(size_t card_index, const uint region_idx_for_card); void scan_card(MemRegion mr, uint region_idx_for_card); + void scan_opt_rem_set_roots(HeapRegion* r); void scan_rem_set_roots(HeapRegion* r); void scan_strong_code_roots(HeapRegion* r); public: @@ -177,6 +180,9 @@ size_t cards_scanned() const { return _cards_scanned; } size_t cards_claimed() const { return _cards_claimed; } size_t cards_skipped() const { return _cards_skipped; } + + size_t opt_refs_scanned() const { return _opt_refs_scanned; } + size_t opt_refs_memory_used() const { return _opt_refs_memory_used; } }; #endif // SHARE_GC_G1_G1REMSET_HPP --- old/src/hotspot/share/gc/g1/heapRegion.cpp 2019-03-13 14:03:14.713399574 +0100 +++ new/src/hotspot/share/gc/g1/heapRegion.cpp 2019-03-13 14:03:14.502393844 +0100 @@ -117,6 +117,7 @@ "Should not clear heap region %u in the collection set", hrm_index()); set_young_index_in_cset(-1); + clear_index_in_opt_cset(); uninstall_surv_rate_group(); set_free(); reset_pre_dummy_top(); @@ -241,7 +242,7 @@ _containing_set(NULL), #endif _prev_marked_bytes(0), _next_marked_bytes(0), _gc_efficiency(0.0), - _index_in_opt_cset(G1OptionalCSet::InvalidCSetIndex), _young_index_in_cset(-1), + _index_in_opt_cset(InvalidCSetIndex), _young_index_in_cset(-1), _surv_rate_group(NULL), _age_index(-1), _prev_top_at_mark_start(NULL), _next_top_at_mark_start(NULL), _recorded_rs_length(0), _predicted_elapsed_time_ms(0) --- old/src/hotspot/share/gc/g1/heapRegion.hpp 2019-03-13 14:03:15.623424288 +0100 +++ new/src/hotspot/share/gc/g1/heapRegion.hpp 2019-03-13 14:03:15.410418503 +0100 @@ -250,6 +250,8 @@ // The calculated GC efficiency of the region. double _gc_efficiency; + static const uint InvalidCSetIndex = UINT_MAX; + // The index in the optional regions array, if this region // is considered optional during a mixed collections. uint _index_in_opt_cset; @@ -549,8 +551,13 @@ void calc_gc_efficiency(void); double gc_efficiency() const { return _gc_efficiency;} - uint index_in_opt_cset() const { return _index_in_opt_cset; } + uint index_in_opt_cset() const { + assert(has_index_in_opt_cset(), "Opt cset index not set."); + return _index_in_opt_cset; + } + bool has_index_in_opt_cset() const { return _index_in_opt_cset != InvalidCSetIndex; } void set_index_in_opt_cset(uint index) { _index_in_opt_cset = index; } + void clear_index_in_opt_cset() { _index_in_opt_cset = InvalidCSetIndex; } int young_index_in_cset() const { return _young_index_in_cset; } void set_young_index_in_cset(int index) { --- old/src/hotspot/share/gc/shared/workerDataArray.hpp 2019-03-13 14:03:16.524448757 +0100 +++ new/src/hotspot/share/gc/shared/workerDataArray.hpp 2019-03-13 14:03:16.312442999 +0100 @@ -34,7 +34,7 @@ class WorkerDataArray : public CHeapObj { friend class WDAPrinter; public: - static const uint MaxThreadWorkItems = 4; + static const uint MaxThreadWorkItems = 5; private: T* _data; uint _length; @@ -50,6 +50,7 @@ void set_thread_work_item(uint worker_i, size_t value, uint index = 0); void add_thread_work_item(uint worker_i, size_t value, uint index = 0); void set_or_add_thread_work_item(uint worker_i, size_t value, uint index = 0); + size_t get_thread_work_item(uint worker_i, uint index = 0); WorkerDataArray* thread_work_items(uint index = 0) const { assert(index < MaxThreadWorkItems, "Tried to access thread work item %u max %u", index, MaxThreadWorkItems); --- old/src/hotspot/share/gc/shared/workerDataArray.inline.hpp 2019-03-13 14:03:17.490474991 +0100 +++ new/src/hotspot/share/gc/shared/workerDataArray.inline.hpp 2019-03-13 14:03:17.266468908 +0100 @@ -92,6 +92,13 @@ } template +size_t WorkerDataArray::get_thread_work_item(uint worker_i, uint index) { + assert(index < MaxThreadWorkItems, "Tried to access thread work item %u (max %u)", index, MaxThreadWorkItems); + assert(_thread_work_items[index] != NULL, "No sub count"); + return _thread_work_items[index]->get(worker_i); +} + +template void WorkerDataArray::add(uint worker_i, T value) { assert(worker_i < _length, "Worker %d is greater than max: %d", worker_i, _length); assert(_data[worker_i] != uninitialized(), "No data to add to for worker %d", worker_i);