--- old/src/hotspot/share/gc/g1/g1CollectedHeap.cpp 2019-11-26 16:18:11.074846434 +0100 +++ new/src/hotspot/share/gc/g1/g1CollectedHeap.cpp 2019-11-26 16:18:10.678844471 +0100 @@ -98,6 +98,7 @@ #include "runtime/threadSMR.hpp" #include "runtime/vmThread.hpp" #include "utilities/align.hpp" +#include "utilities/bitMap.inline.hpp" #include "utilities/globalDefinitions.hpp" #include "utilities/stack.inline.hpp" @@ -2366,6 +2367,10 @@ _collection_set.iterate(cl); } +void G1CollectedHeap::collection_set_par_iterate_all(HeapRegionClosure* cl, HeapRegionClaimer* hr_claimer, uint worker_id) { + _collection_set.par_iterate(cl, hr_claimer, worker_id, workers()->active_workers()); +} + void G1CollectedHeap::collection_set_iterate_increment_from(HeapRegionClosure *cl, HeapRegionClaimer* hr_claimer, uint worker_id) { _collection_set.iterate_incremental_part_from(cl, hr_claimer, worker_id, workers()->active_workers()); } @@ -4079,7 +4084,6 @@ assert(!hr->is_free(), "the region should not be free"); assert(!hr->is_empty(), "the region should not be empty"); assert(_hrm->is_available(hr->hrm_index()), "region should be committed"); - assert(free_list != NULL, "pre-condition"); if (G1VerifyBitmaps) { MemRegion mr(hr->bottom(), hr->end()); @@ -4094,7 +4098,9 @@ } hr->hr_clear(skip_remset, true /* clear_space */, locked /* locked */); _policy->remset_tracker()->update_at_free(hr); - free_list->add_ordered(hr); + if (free_list != NULL) { + free_list->add_ordered(hr); + } } void G1CollectedHeap::free_humongous_region(HeapRegion* hr, @@ -4115,6 +4121,11 @@ } +void G1CollectedHeap::abandon_freelist() { + MutexLocker x(FreeList_lock, Mutex::_no_safepoint_check_flag); + _hrm->abandon_free_list(); +} + void G1CollectedHeap::prepend_to_freelist(FreeRegionList* list) { assert(list != NULL, "list can't be null"); if (!list->is_empty()) { @@ -4123,286 +4134,361 @@ } } +void G1CollectedHeap::append_to_freelist(FreeRegionList* list) { + assert(list != NULL, "list can't be null"); + if (!list->is_empty()) { + MutexLocker x(FreeList_lock, Mutex::_no_safepoint_check_flag); + _hrm->append_to_free_list(list); + } +} + void G1CollectedHeap::decrement_summary_bytes(size_t bytes) { decrease_used(bytes); } class G1FreeCollectionSetTask : public AbstractGangTask { -private: - - // Closure applied to all regions in the collection set to do work that needs to - // be done serially in a single thread. - class G1SerialFreeCollectionSetClosure : public HeapRegionClosure { - private: - G1EvacuationInfo* _evacuation_info; - const size_t* _surviving_young_words; - - // Bytes used in successfully evacuated regions before the evacuation. - size_t _before_used_bytes; - // Bytes used in unsucessfully evacuated regions before the evacuation - size_t _after_used_bytes; - - size_t _bytes_allocated_in_old_since_last_gc; - - size_t _failure_used_words; - size_t _failure_waste_words; - - FreeRegionList _local_free_list; + // Helper class to keep statistics for the collection set freeing + class FreeCSetStats { + size_t _before_used_bytes; // Usage in regions successfully evacutate + size_t _after_used_bytes; // Usage in regions failing evacuation + size_t _bytes_allocated_in_old_since_last_gc; // Size of young regions turned into old + size_t _failure_used_words; // Live size in failed regions + size_t _failure_waste_words; // Wasted size in failed regions + size_t _rs_length; // Remembered set size + uint _regions_freed; // Number of regions freed public: - G1SerialFreeCollectionSetClosure(G1EvacuationInfo* evacuation_info, const size_t* surviving_young_words) : - HeapRegionClosure(), - _evacuation_info(evacuation_info), - _surviving_young_words(surviving_young_words), - _before_used_bytes(0), - _after_used_bytes(0), - _bytes_allocated_in_old_since_last_gc(0), - _failure_used_words(0), - _failure_waste_words(0), - _local_free_list("Local Region List for CSet Freeing") { - } + FreeCSetStats() : + _before_used_bytes(0), + _after_used_bytes(0), + _bytes_allocated_in_old_since_last_gc(0), + _failure_used_words(0), + _failure_waste_words(0), + _rs_length(0), + _regions_freed(0) { } + + void merge_stats(FreeCSetStats* other) { + assert(other != NULL, "invariant"); + _before_used_bytes += other->_before_used_bytes; + _after_used_bytes += other->_after_used_bytes; + _bytes_allocated_in_old_since_last_gc += other->_bytes_allocated_in_old_since_last_gc; + _failure_used_words += other->_failure_used_words; + _failure_waste_words += other->_failure_waste_words; + _rs_length += other->_rs_length; + _regions_freed += other->_regions_freed; + } + + void report(G1CollectedHeap* g1h, G1EvacuationInfo* evacuation_info) { + evacuation_info->set_regions_freed(_regions_freed); + evacuation_info->increment_collectionset_used_after(_after_used_bytes); - virtual bool do_heap_region(HeapRegion* r) { - G1CollectedHeap* g1h = G1CollectedHeap::heap(); + g1h->decrement_summary_bytes(_before_used_bytes); + g1h->alloc_buffer_stats(G1HeapRegionAttr::Old)->add_failure_used_and_waste(_failure_used_words, _failure_waste_words); - assert(r->in_collection_set(), "Region %u should be in collection set.", r->hrm_index()); - g1h->clear_region_attr(r); + G1Policy *policy = g1h->policy(); + policy->add_bytes_allocated_in_old_since_last_gc(_bytes_allocated_in_old_since_last_gc); + policy->record_rs_length(_rs_length); + policy->cset_regions_freed(); + } - if (r->is_young()) { - assert(r->young_index_in_cset() != 0 && (uint)r->young_index_in_cset() <= g1h->collection_set()->young_region_length(), - "Young index %u is wrong for region %u of type %s with %u young regions", - r->young_index_in_cset(), - r->hrm_index(), - r->get_type_str(), - g1h->collection_set()->young_region_length()); - size_t words_survived = _surviving_young_words[r->young_index_in_cset()]; - r->record_surv_words_in_group(words_survived); - } + void update_used_before(size_t value) { _before_used_bytes += value; } + void update_used_after(size_t value) { _after_used_bytes += value; } + void update_bytes_alloc_in_old(size_t value) { _bytes_allocated_in_old_since_last_gc += value; } + void update_failure_used(size_t value) { _failure_used_words += value; } + void update_failure_waste(size_t value) { _failure_waste_words += value; } + void update_rs_length(size_t value) { _rs_length += value; } + void increment_regions_freed() { _regions_freed += 1; } + }; - if (!r->evacuation_failed()) { - assert(!r->is_empty(), "Region %u is an empty region in the collection set.", r->hrm_index()); - _before_used_bytes += r->used(); - g1h->free_region(r, - &_local_free_list, - true, /* skip_remset */ - true, /* skip_hot_card_cache */ - true /* locked */); - } else { - r->uninstall_surv_rate_group(); - r->clear_young_index_in_cset(); - r->set_evacuation_failed(false); - // When moving a young gen region to old gen, we "allocate" that whole region - // there. This is in addition to any already evacuated objects. Notify the - // policy about that. - // Old gen regions do not cause an additional allocation: both the objects - // still in the region and the ones already moved are accounted for elsewhere. - if (r->is_young()) { - _bytes_allocated_in_old_since_last_gc += HeapRegion::GrainBytes; + // Closure applied to all regions in the collection set. + class G1FreeCollectionSetClosure : public HeapRegionClosure { + // Helper to send JFR events for regions. + class EventForRegion { + EventGCPhaseParallel _event; + public: + EventForRegion(HeapRegion* region, uint worker_id) : _event() { + _event.set_gcId(GCId::current()); + _event.set_gcWorkerId(worker_id); + if (region->is_young()) { + _event.set_name(G1GCPhaseTimes::phase_name(G1GCPhaseTimes::YoungFreeCSet)); + } else { + _event.set_name(G1GCPhaseTimes::phase_name(G1GCPhaseTimes::NonYoungFreeCSet)); } - // The region is now considered to be old. - r->set_old(); - // Do some allocation statistics accounting. Regions that failed evacuation - // are always made old, so there is no need to update anything in the young - // gen statistics, but we need to update old gen statistics. - size_t used_words = r->marked_bytes() / HeapWordSize; + } - _failure_used_words += used_words; - _failure_waste_words += HeapRegion::GrainWords - used_words; + ~EventForRegion() { + _event.commit(); + } + }; - g1h->old_set_add(r); - _after_used_bytes += r->used(); + // Helper to do timing for region work. + class TimerForRegion { + double& _time; + double _start_time; + public: + TimerForRegion(double& time) : _time(time), _start_time(os::elapsedTime()) { } + ~TimerForRegion() { + _time += os::elapsedTime() - _start_time; } - return false; - } + }; - void complete_work() { - G1CollectedHeap* g1h = G1CollectedHeap::heap(); + // G1FreeCollectionSetClosure members + G1CollectedHeap* _g1h; + const size_t* _surviving_young_words; + uint _worker_id; + double _young_time; + double _non_young_time; + FreeCSetStats* _stats; + + void assert_in_cset(HeapRegion* r) { + assert(r->young_index_in_cset() != 0 && + (uint)r->young_index_in_cset() <= _g1h->collection_set()->young_region_length(), + "Young index %u is wrong for region %u of type %s with %u young regions", + r->young_index_in_cset(), r->hrm_index(), r->get_type_str(), _g1h->collection_set()->young_region_length()); + } + + void handle_evacuated_region(HeapRegion* r) { + assert(!r->is_empty(), "Region %u is an empty region in the collection set.", r->hrm_index()); + // Update stats. + stats()->update_used_before(r->used()); + stats()->increment_regions_freed(); - _evacuation_info->set_regions_freed(_local_free_list.length()); - _evacuation_info->increment_collectionset_used_after(_after_used_bytes); + // Free the region and mark it in the bit map so it gets added to the new free list. + r->rem_set()->clear_locked(); + _g1h->free_region(r, NULL, true /* skip_remset */, true /* skip_hot_card_cache */, true /* locked */); + } - g1h->prepend_to_freelist(&_local_free_list); - g1h->decrement_summary_bytes(_before_used_bytes); + void handle_failed_region(HeapRegion* r) { + // Do some allocation statistics accounting. Regions that failed evacuation + // are always made old, so there is no need to update anything in the young + // gen statistics, but we need to update old gen statistics. + size_t used_words = r->marked_bytes() / HeapWordSize; + stats()->update_failure_used(used_words); + stats()->update_failure_waste(HeapRegion::GrainWords - used_words); + stats()->update_used_after(r->used()); + // When moving a young gen region to old gen, we "allocate" that whole + // region there. This is in addition to any already evacuated objects. + // Notify the policy about that. Old gen regions do not cause an + // additional allocation: both the objects still in the region and the + // ones already moved are accounted for elsewhere. + if (r->is_young()) { + stats()->update_bytes_alloc_in_old(HeapRegion::GrainBytes); + } - G1Policy* policy = g1h->policy(); - policy->add_bytes_allocated_in_old_since_last_gc(_bytes_allocated_in_old_since_last_gc); + // Update the region state due to the failed evacuation. + r->handle_evacuation_failed(); - g1h->alloc_buffer_stats(G1HeapRegionAttr::Old)->add_failure_used_and_waste(_failure_used_words, _failure_waste_words); + // Add region to old set, need to hold lock. + MutexLocker x(OldSets_lock, Mutex::_no_safepoint_check_flag); + _g1h->old_set_add(r); } - }; - G1CollectionSet* _collection_set; - G1SerialFreeCollectionSetClosure _cl; - const size_t* _surviving_young_words; - - size_t _rs_length; - - volatile jint _serial_work_claim; - - struct WorkItem { - uint region_idx; - bool is_young; - bool evacuation_failed; - - WorkItem(HeapRegion* r) { - region_idx = r->hrm_index(); - is_young = r->is_young(); - evacuation_failed = r->evacuation_failed(); + double& timer_for_region(HeapRegion* r) { + return r->is_young() ? _young_time : _non_young_time; } - }; - - volatile size_t _parallel_work_claim; - size_t _num_work_items; - WorkItem* _work_items; + public: + G1FreeCollectionSetClosure(const size_t* surviving_young_words, + uint worker_id, + FreeCSetStats* stats) : + HeapRegionClosure(), + _g1h(G1CollectedHeap::heap()), + _surviving_young_words(surviving_young_words), + _worker_id(worker_id), + _young_time(0.0), + _non_young_time(0.0), + _stats(stats) { } - void do_serial_work() { - // Need to grab the lock to be allowed to modify the old region list. - MutexLocker x(OldSets_lock, Mutex::_no_safepoint_check_flag); - _collection_set->iterate(&_cl); - } + virtual bool do_heap_region(HeapRegion* r) { + assert(r->in_collection_set(), "Invariant: %u missing from CSet", r->hrm_index()); + EventForRegion event(r, _worker_id); + TimerForRegion timer(timer_for_region(r)); - void do_parallel_work_for_region(uint region_idx, bool is_young, bool evacuation_failed) { - G1CollectedHeap* g1h = G1CollectedHeap::heap(); + _g1h->clear_region_attr(r); + stats()->update_rs_length(r->rem_set()->occupied_locked()); - HeapRegion* r = g1h->region_at(region_idx); - assert(!g1h->is_on_master_free_list(r), "sanity"); + if (r->is_young()) { + assert_in_cset(r); + r->record_surv_words_in_group(_surviving_young_words[r->young_index_in_cset()]); + } else { + _g1h->hot_card_cache()->reset_card_counts(r); + } - Atomic::add(&_rs_length, r->rem_set()->occupied_locked()); + if (r->evacuation_failed()) { + handle_failed_region(r); + } else { + handle_evacuated_region(r); + } + assert(!_g1h->is_on_master_free_list(r), "sanity"); - if (!is_young) { - g1h->hot_card_cache()->reset_card_counts(r); + return false; } - if (!evacuation_failed) { - r->rem_set()->clear_locked(); + FreeCSetStats* stats() { + return _stats; } - } - - class G1PrepareFreeCollectionSetClosure : public HeapRegionClosure { - private: - size_t _cur_idx; - WorkItem* _work_items; - public: - G1PrepareFreeCollectionSetClosure(WorkItem* work_items) : HeapRegionClosure(), _cur_idx(0), _work_items(work_items) { } - virtual bool do_heap_region(HeapRegion* r) { - _work_items[_cur_idx++] = WorkItem(r); - return false; + void report_timing(double parallel_time) { + G1GCPhaseTimes* pt = _g1h->phase_times(); + pt->record_time_secs(G1GCPhaseTimes::ParFreeCSet, _worker_id, parallel_time); + if (_young_time > 0.0) { + pt->record_time_secs(G1GCPhaseTimes::YoungFreeCSet, _worker_id, _young_time); + } + if (_non_young_time > 0.0) { + pt->record_time_secs(G1GCPhaseTimes::NonYoungFreeCSet, _worker_id, _non_young_time); + } } }; - void prepare_work() { - G1PrepareFreeCollectionSetClosure cl(_work_items); - _collection_set->iterate(&cl); - } + // G1FreeCollectionSetTask members + G1CollectedHeap* _g1h; + G1EvacuationInfo* _evacuation_info; + FreeCSetStats* _worker_stats; + HeapRegionClaimer _claimer; + const size_t* _surviving_young_words; + uint _active_workers; - void complete_work() { - _cl.complete_work(); + FreeCSetStats* worker_stats(uint worker) { + return &_worker_stats[worker]; + } - G1Policy* policy = G1CollectedHeap::heap()->policy(); - policy->record_rs_length(_rs_length); - policy->cset_regions_freed(); + void report_statistics() { + // Merge the accounting + FreeCSetStats total_stats; + for (uint worker = 0; worker < _active_workers; worker++) { + total_stats.merge_stats(worker_stats(worker)); + } + total_stats.report(_g1h, _evacuation_info); } + public: - G1FreeCollectionSetTask(G1CollectionSet* collection_set, G1EvacuationInfo* evacuation_info, const size_t* surviving_young_words) : - AbstractGangTask("G1 Free Collection Set"), - _collection_set(collection_set), - _cl(evacuation_info, surviving_young_words), - _surviving_young_words(surviving_young_words), - _rs_length(0), - _serial_work_claim(0), - _parallel_work_claim(0), - _num_work_items(collection_set->region_length()), - _work_items(NEW_C_HEAP_ARRAY(WorkItem, _num_work_items, mtGC)) { - prepare_work(); + G1FreeCollectionSetTask(G1EvacuationInfo* evacuation_info, const size_t* surviving_young_words, uint active_workers) : + AbstractGangTask("G1 Free Collection Set"), + _g1h(G1CollectedHeap::heap()), + _evacuation_info(evacuation_info), + _worker_stats(NEW_C_HEAP_ARRAY(FreeCSetStats, active_workers, mtGC)), + _claimer(active_workers), + _surviving_young_words(surviving_young_words), + _active_workers(active_workers) { + for (uint worker = 0; worker < active_workers; worker++) { + ::new (&_worker_stats[worker]) FreeCSetStats(); + } } ~G1FreeCollectionSetTask() { - complete_work(); - FREE_C_HEAP_ARRAY(WorkItem, _work_items); + double serial_time = os::elapsedTime(); + report_statistics(); + FREE_C_HEAP_ARRAY(FreeCSetStats, _worker_stats); + _g1h->phase_times()->record_serial_free_cset_time_ms((os::elapsedTime() - serial_time) * 1000.0); } - // Chunk size for work distribution. The chosen value has been determined experimentally - // to be a good tradeoff between overhead and achievable parallelism. - static uint chunk_size() { return 32; } - virtual void work(uint worker_id) { - G1GCPhaseTimes* timer = G1CollectedHeap::heap()->phase_times(); + EventGCPhaseParallel event; + double start = os::elapsedTime(); + G1FreeCollectionSetClosure cl(_surviving_young_words, worker_id, worker_stats(worker_id)); + _g1h->collection_set_par_iterate_all(&cl, &_claimer, worker_id); + + // Report the total parallel time along with some more detailed metrics. + cl.report_timing(os::elapsedTime() - start); + event.commit(GCId::current(), worker_id, G1GCPhaseTimes::phase_name(G1GCPhaseTimes::ParFreeCSet)); + } +}; - // Claim serial work. - if (_serial_work_claim == 0) { - jint value = Atomic::add(&_serial_work_claim, 1) - 1; - if (value == 0) { - double serial_time = os::elapsedTime(); - do_serial_work(); - timer->record_serial_free_cset_time_ms((os::elapsedTime() - serial_time) * 1000.0); - } +class G1RebuildFreeListTask : public AbstractGangTask { + G1CollectedHeap* _g1h; + FreeRegionList* _worker_freelists; + uint _worker_chunk_size; + uint _num_workers; + + FreeRegionList* worker_freelist(uint worker) { return &_worker_freelists[worker]; } + + // Take the partial free lists and link them together. + void rebuild_freelist() { + // Abandon current free list to allow a rebuild using the + // workers chunked free lists. + _g1h->abandon_freelist(); + for (uint worker = 0; worker < _num_workers; worker++) { + _g1h->append_to_freelist(worker_freelist(worker)); } + } +public: + G1RebuildFreeListTask(G1CollectedHeap* g1, uint num_workers) : + AbstractGangTask("G1 Rebuild Free List Task"), + _g1h(g1), + _worker_freelists(NEW_C_HEAP_ARRAY(FreeRegionList, num_workers, mtGC)), + _worker_chunk_size((_g1h->max_regions() + num_workers - 1) / num_workers), + _num_workers(num_workers) { + for (uint worker = 0; worker < _num_workers; worker++) { + ::new (&_worker_freelists[worker]) FreeRegionList("Appendable Worker Free List"); + } + } - // Start parallel work. - double young_time = 0.0; - bool has_young_time = false; - double non_young_time = 0.0; - bool has_non_young_time = false; - - while (true) { - size_t end = Atomic::add(&_parallel_work_claim, chunk_size()); - size_t cur = end - chunk_size(); - - if (cur >= _num_work_items) { - break; - } + ~G1RebuildFreeListTask() { + double serial_time = os::elapsedTime(); + rebuild_freelist(); + FREE_C_HEAP_ARRAY(FreeRegionList, _worker_freelists); + _g1h->phase_times()->record_serial_rebuild_freelist_time_ms((os::elapsedTime() - serial_time) * 1000.0); + } - EventGCPhaseParallel event; - double start_time = os::elapsedTime(); + // Each worker creates a free list for a chunk of the heap. The chunks won't + // be overlapping so we don't need to do any claiming. + virtual void work(uint worker_id) { + double start_time = os::elapsedTime(); + EventGCPhaseParallel event; - end = MIN2(end, _num_work_items); + uint start = worker_id * _worker_chunk_size; + uint end = MIN2(start + _worker_chunk_size, _g1h->max_regions()); - for (; cur < end; cur++) { - bool is_young = _work_items[cur].is_young; + // If start is outside the heap, this worker has nothing to do. + if (start > end) { + return; + } - do_parallel_work_for_region(_work_items[cur].region_idx, is_young, _work_items[cur].evacuation_failed); - - double end_time = os::elapsedTime(); - double time_taken = end_time - start_time; - if (is_young) { - young_time += time_taken; - has_young_time = true; - event.commit(GCId::current(), worker_id, G1GCPhaseTimes::phase_name(G1GCPhaseTimes::YoungFreeCSet)); - } else { - non_young_time += time_taken; - has_non_young_time = true; - event.commit(GCId::current(), worker_id, G1GCPhaseTimes::phase_name(G1GCPhaseTimes::NonYoungFreeCSet)); - } - start_time = end_time; + FreeRegionList* free_list = worker_freelist(worker_id); + for (uint i = start; i < end; i++) { + if (!_g1h->hrm()->is_available(i)) { + continue; + } + HeapRegion* region = _g1h->region_at(i); + if (region->is_free()) { + region->unlink_from_list(); + free_list->add_to_tail(region); } } - if (has_young_time) { - timer->record_time_secs(G1GCPhaseTimes::YoungFreeCSet, worker_id, young_time); - } - if (has_non_young_time) { - timer->record_time_secs(G1GCPhaseTimes::NonYoungFreeCSet, worker_id, non_young_time); - } + event.commit(GCId::current(), worker_id, G1GCPhaseTimes::phase_name(G1GCPhaseTimes::RebuildFreeList)); + _g1h->phase_times()->record_time_secs(G1GCPhaseTimes::RebuildFreeList, worker_id, os::elapsedTime() - start_time); } }; void G1CollectedHeap::free_collection_set(G1CollectionSet* collection_set, G1EvacuationInfo& evacuation_info, const size_t* surviving_young_words) { _eden.clear(); - double free_cset_start_time = os::elapsedTime(); + // The free collections set is split up in two tasks, the first + // frees the collection set and records what regions are free, + // and the second one rebuilds the free list. This proved to be + // more efficient than adding a sorted list to another. + double free_cset_start_time = os::elapsedTime(); { - 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); + uint const num_cs_regions = _collection_set.region_length(); + uint const num_workers = clamp(num_cs_regions, 1u, workers()->active_workers()); + G1FreeCollectionSetTask cl(&evacuation_info, surviving_young_words, num_workers); + + log_debug(gc, ergo)("Running %s using %u workers for collection set length %u (%u)", + cl.name(), num_workers, num_cs_regions, num_regions()); + workers()->run_task(&cl, num_workers); + } - G1FreeCollectionSetTask cl(collection_set, &evacuation_info, surviving_young_words); + double free_cset_end_time = os::elapsedTime(); + phase_times()->record_total_free_cset_time_ms((free_cset_end_time - free_cset_start_time) * 1000.0); + { + uint const num_workers = workers()->active_workers(); + G1RebuildFreeListTask cl(this, num_workers); - log_debug(gc, ergo)("Running %s using %u workers for collection set length %u", - cl.name(), num_workers, num_regions); + log_debug(gc, ergo)("Running %s using %u workers for rebuilding free list of %u (%u) regions", + cl.name(), num_workers, num_free_regions(), num_regions()); workers()->run_task(&cl, num_workers); } - phase_times()->record_total_free_cset_time_ms((os::elapsedTime() - free_cset_start_time) * 1000.0); + phase_times()->record_total_rebuild_freelist_time_ms((os::elapsedTime() - free_cset_end_time) * 1000.0); collection_set->clear(); } --- old/src/hotspot/share/gc/g1/g1CollectedHeap.hpp 2019-11-26 16:18:11.690849489 +0100 +++ new/src/hotspot/share/gc/g1/g1CollectedHeap.hpp 2019-11-26 16:18:11.262847367 +0100 @@ -1106,7 +1106,9 @@ bool evacuation_failed() { return _evacuation_failed; } void remove_from_old_sets(const uint old_regions_removed, const uint humongous_regions_removed); + void abandon_freelist(); void prepend_to_freelist(FreeRegionList* list); + void append_to_freelist(FreeRegionList* list); void decrement_summary_bytes(size_t bytes); virtual bool is_in(const void* p) const; @@ -1201,6 +1203,11 @@ void heap_region_par_iterate_from_start(HeapRegionClosure* cl, HeapRegionClaimer* hrclaimer) const; + // Iterate over all regions in the collection set in parallel. + void collection_set_par_iterate_all(HeapRegionClosure* cl, + HeapRegionClaimer* hr_claimer, + uint worker_id); + // Iterate over all regions currently in the current collection set. void collection_set_iterate_all(HeapRegionClosure* blk); --- old/src/hotspot/share/gc/g1/g1CollectionSet.cpp 2019-11-26 16:18:12.202852028 +0100 +++ new/src/hotspot/share/gc/g1/g1CollectionSet.cpp 2019-11-26 16:18:11.778849925 +0100 @@ -205,6 +205,13 @@ } } +void G1CollectionSet::par_iterate(HeapRegionClosure* cl, + HeapRegionClaimer* hr_claimer, + uint worker_id, + uint total_workers) const { + iterate_part_from(cl, hr_claimer, 0, cur_length(), worker_id, total_workers); +} + void G1CollectionSet::iterate_optional(HeapRegionClosure* cl) const { assert_at_safepoint(); @@ -219,18 +226,25 @@ HeapRegionClaimer* hr_claimer, uint worker_id, uint total_workers) const { - assert_at_safepoint(); + iterate_part_from(cl, hr_claimer, _inc_part_start, increment_length(), worker_id, total_workers); +} - size_t len = increment_length(); - if (len == 0) { +void G1CollectionSet::iterate_part_from(HeapRegionClosure* cl, + HeapRegionClaimer* hr_claimer, + size_t offset, + size_t lenght, + uint worker_id, + uint total_workers) const { + assert_at_safepoint(); + if (lenght == 0) { return; } - size_t start_pos = (worker_id * len) / total_workers; + size_t start_pos = (worker_id * lenght) / total_workers; size_t cur_pos = start_pos; do { - uint region_idx = _collection_set_regions[cur_pos + _inc_part_start]; + uint region_idx = _collection_set_regions[cur_pos + offset]; if (hr_claimer == NULL || hr_claimer->claim_region(region_idx)) { HeapRegion* r = _g1h->region_at(region_idx); bool result = cl->do_heap_region(r); @@ -238,7 +252,7 @@ } cur_pos++; - if (cur_pos == len) { + if (cur_pos == lenght) { cur_pos = 0; } } while (cur_pos != start_pos); --- old/src/hotspot/share/gc/g1/g1CollectionSet.hpp 2019-11-26 16:18:12.710854547 +0100 +++ new/src/hotspot/share/gc/g1/g1CollectionSet.hpp 2019-11-26 16:18:12.286852445 +0100 @@ -278,6 +278,16 @@ // Stop adding regions to the current collection set increment. void stop_incremental_building() { _inc_build_state = Inactive; } + // Iterate the part of the collection set given by the offset and length applying the given + // HeapRegionClosure. The worker_id will determine where in the part to start the iteration + // to allow for more efficient parallel iteration. + void iterate_part_from(HeapRegionClosure* cl, + HeapRegionClaimer* hr_claimer, + size_t offset, + size_t lenght, + uint worker_id, + uint total_workers) const; + // 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, HeapRegionClaimer* hr_claimer, uint worker_id, uint total_workers) const; @@ -290,6 +300,10 @@ // Iterate over the entire collection set (all increments calculated so far), applying // the given HeapRegionClosure on all of them. void iterate(HeapRegionClosure* cl) const; + void par_iterate(HeapRegionClosure* cl, + HeapRegionClaimer* hr_claimer, + uint worker_id, + uint total_workers) const; void iterate_optional(HeapRegionClosure* cl) const; --- old/src/hotspot/share/gc/g1/g1GCPhaseTimes.cpp 2019-11-26 16:18:13.210857026 +0100 +++ new/src/hotspot/share/gc/g1/g1GCPhaseTimes.cpp 2019-11-26 16:18:12.790854944 +0100 @@ -155,8 +155,10 @@ _redirtied_cards = new WorkerDataArray(max_gc_threads, "Redirtied Cards:"); _gc_par_phases[RedirtyCards]->link_thread_work_items(_redirtied_cards); + _gc_par_phases[ParFreeCSet] = new WorkerDataArray(max_gc_threads, "Parallel Free Collection Set (ms):"); _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[RebuildFreeList] = new WorkerDataArray(max_gc_threads, "Parallel Rebuild Free List (ms):"); reset(); } @@ -191,6 +193,8 @@ _recorded_start_new_cset_time_ms = 0.0; _recorded_total_free_cset_time_ms = 0.0; _recorded_serial_free_cset_time_ms = 0.0; + _recorded_total_rebuild_freelist_time_ms = 0.0; + _recorded_serial_rebuild_freelist_time_ms = 0.0; _cur_fast_reclaim_humongous_time_ms = 0.0; _cur_region_register_time = 0.0; _cur_fast_reclaim_humongous_total = 0; @@ -474,6 +478,7 @@ _cur_strong_code_root_purge_time_ms + _recorded_redirty_logged_cards_time_ms + _recorded_total_free_cset_time_ms + + _recorded_total_rebuild_freelist_time_ms + _cur_fast_reclaim_humongous_time_ms + _cur_expand_heap_time_ms + _cur_string_deduplication_time_ms; @@ -510,10 +515,15 @@ #endif debug_time("Free Collection Set", _recorded_total_free_cset_time_ms); - trace_time("Free Collection Set Serial", _recorded_serial_free_cset_time_ms); + trace_time("Serial Free Collection Set", _recorded_serial_free_cset_time_ms); + trace_phase(_gc_par_phases[ParFreeCSet]); trace_phase(_gc_par_phases[YoungFreeCSet]); trace_phase(_gc_par_phases[NonYoungFreeCSet]); + debug_time("Rebuild Free List", _recorded_total_rebuild_freelist_time_ms); + trace_time("Serial Rebuild Free List ", _recorded_serial_rebuild_freelist_time_ms); + trace_phase(_gc_par_phases[RebuildFreeList]); + if (G1EagerReclaimHumongousObjects) { debug_time("Humongous Reclaim", _cur_fast_reclaim_humongous_time_ms); trace_count("Humongous Reclaimed", _cur_fast_reclaim_humongous_reclaimed); @@ -584,8 +594,10 @@ "StringDedupQueueFixup", "StringDedupTableFixup", "RedirtyCards", + "ParFreeCSet", "YoungFreeCSet", "NonYoungFreeCSet", + "RebuildFreeList", "MergePSS" //GCParPhasesSentinel only used to tell end of enum }; --- old/src/hotspot/share/gc/g1/g1GCPhaseTimes.hpp 2019-11-26 16:18:13.682859367 +0100 +++ new/src/hotspot/share/gc/g1/g1GCPhaseTimes.hpp 2019-11-26 16:18:13.294857443 +0100 @@ -76,8 +76,10 @@ StringDedupQueueFixup, StringDedupTableFixup, RedirtyCards, + ParFreeCSet, YoungFreeCSet, NonYoungFreeCSet, + RebuildFreeList, MergePSS, GCParPhasesSentinel }; @@ -204,6 +206,10 @@ double _recorded_serial_free_cset_time_ms; + double _recorded_total_rebuild_freelist_time_ms; + + double _recorded_serial_rebuild_freelist_time_ms; + double _cur_region_register_time; double _cur_fast_reclaim_humongous_time_ms; @@ -351,6 +357,14 @@ _recorded_serial_free_cset_time_ms = time_ms; } + void record_total_rebuild_freelist_time_ms(double time_ms) { + _recorded_total_rebuild_freelist_time_ms = time_ms; + } + + void record_serial_rebuild_freelist_time_ms(double time_ms) { + _recorded_serial_rebuild_freelist_time_ms = time_ms; + } + void record_register_regions(double time_ms, size_t total, size_t candidates) { _cur_region_register_time = time_ms; _cur_fast_reclaim_humongous_total = total; @@ -434,6 +448,10 @@ return _recorded_total_free_cset_time_ms; } + double total_rebuild_freelist_time_ms() { + return _recorded_total_rebuild_freelist_time_ms; + } + double non_young_cset_choice_time_ms() { return _recorded_non_young_cset_choice_time_ms; } --- old/src/hotspot/share/gc/g1/g1Policy.cpp 2019-11-26 16:18:14.182861846 +0100 +++ new/src/hotspot/share/gc/g1/g1Policy.cpp 2019-11-26 16:18:13.762859764 +0100 @@ -601,7 +601,7 @@ } double G1Policy::constant_other_time_ms(double pause_time_ms) const { - return other_time_ms(pause_time_ms) - phase_times()->total_free_cset_time_ms(); + return other_time_ms(pause_time_ms) - phase_times()->total_free_cset_time_ms() - phase_times()->total_rebuild_freelist_time_ms(); } bool G1Policy::about_to_start_mixed_phase() const { --- old/src/hotspot/share/gc/g1/heapRegion.cpp 2019-11-26 16:18:14.662864227 +0100 +++ new/src/hotspot/share/gc/g1/heapRegion.cpp 2019-11-26 16:18:14.274862303 +0100 @@ -112,6 +112,19 @@ } } +void HeapRegion::handle_evacuation_failed() { + uninstall_surv_rate_group(); + clear_young_index_in_cset(); + set_evacuation_failed(false); + set_old(); +} + +void HeapRegion::unlink_from_list() { + set_next(NULL); + set_prev(NULL); + set_containing_set(NULL); +} + void HeapRegion::hr_clear(bool keep_remset, bool clear_space, bool locked) { assert(_humongous_start_region == NULL, "we should have already filtered out humongous regions"); --- old/src/hotspot/share/gc/g1/heapRegion.hpp 2019-11-26 16:18:15.126866527 +0100 +++ new/src/hotspot/share/gc/g1/heapRegion.hpp 2019-11-26 16:18:14.746864643 +0100 @@ -471,14 +471,16 @@ void set_prev(HeapRegion* prev) { _prev = prev; } HeapRegion* prev() { return _prev; } + void unlink_from_list(); + // Every region added to a set is tagged with a reference to that // set. This is used for doing consistency checking to make sure that // the contents of a set are as they should be and it's only // available in non-product builds. #ifdef ASSERT void set_containing_set(HeapRegionSetBase* containing_set) { - assert((containing_set == NULL && _containing_set != NULL) || - (containing_set != NULL && _containing_set == NULL), + assert((containing_set != NULL && _containing_set == NULL) || + containing_set == NULL, "containing_set: " PTR_FORMAT " " "_containing_set: " PTR_FORMAT, p2i(containing_set), p2i(_containing_set)); @@ -599,6 +601,9 @@ return (HeapWord *) obj >= next_top_at_mark_start(); } + // Update the region state after a failed evacuation. + void handle_evacuation_failed(); + // Iterate over the objects overlapping the given memory region, applying cl // to all references in the region. This is a helper for // G1RemSet::refine_card*, and is tightly coupled with them. --- old/src/hotspot/share/gc/g1/heapRegionManager.hpp 2019-11-26 16:18:15.630869027 +0100 +++ new/src/hotspot/share/gc/g1/heapRegionManager.hpp 2019-11-26 16:18:15.206866924 +0100 @@ -172,11 +172,22 @@ // Insert the given region into the free region list. inline void insert_into_free_list(HeapRegion* hr); + // Abandon the current free list to allow for a rebuild. The actual + // region in the free list must be handled separately. + void abandon_free_list() { + _free_list.abandon(); + } + // Insert the given region list into the global free region list. void insert_list_into_free_list(FreeRegionList* list) { _free_list.add_ordered(list); } + // Append the list to the current list, requires the lists to be sorted. + void append_to_free_list(FreeRegionList* list) { + _free_list.append_ordered(list); + } + // Allocate a free region with specific node index. If fails allocate with next node index. virtual HeapRegion* allocate_free_region(HeapRegionType type, uint requested_node_index); --- old/src/hotspot/share/gc/g1/heapRegionSet.cpp 2019-11-26 16:18:16.126871486 +0100 +++ new/src/hotspot/share/gc/g1/heapRegionSet.cpp 2019-11-26 16:18:15.706869403 +0100 @@ -90,6 +90,12 @@ _unrealistically_long_length = len; } +void FreeRegionList::abandon() { + check_mt_safety(); + clear(); + verify_optional(); +} + void FreeRegionList::remove_all() { check_mt_safety(); verify_optional(); @@ -112,10 +118,9 @@ verify_optional(); } -void FreeRegionList::add_ordered(FreeRegionList* from_list) { +void FreeRegionList::add_list_common_start(FreeRegionList* from_list) { check_mt_safety(); from_list->check_mt_safety(); - verify_optional(); from_list->verify_optional(); @@ -138,6 +143,47 @@ hr->set_containing_set(this); } #endif // ASSERT +} + +void FreeRegionList::add_list_common_end(FreeRegionList* from_list) { + _length += from_list->length(); + from_list->clear(); + + verify_optional(); + from_list->verify_optional(); +} + +void FreeRegionList::append_ordered(FreeRegionList* from_list) { + add_list_common_start(from_list); + + if (from_list->is_empty()) { + return; + } + + if (is_empty()) { + // Make from_list the current list. + assert_free_region_list(length() == 0 && _tail == NULL, "invariant"); + _head = from_list->_head; + _tail = from_list->_tail; + } else { + // Add the from_list to the end of the current list. + assert(_tail->hrm_index() < from_list->_head->hrm_index(), "Should be sorted %u < %u", + _tail->hrm_index(), from_list->_head->hrm_index()); + + _tail->set_next(from_list->_head); + from_list->_head->set_prev(_tail); + _tail = from_list->_tail; + } + + add_list_common_end(from_list); +} + +void FreeRegionList::add_ordered(FreeRegionList* from_list) { + add_list_common_start(from_list); + + if (from_list->is_empty()) { + return; + } if (is_empty()) { assert_free_region_list(length() == 0 && _tail == NULL, "invariant"); @@ -178,11 +224,7 @@ } } - _length += from_list->length(); - from_list->clear(); - - verify_optional(); - from_list->verify_optional(); + add_list_common_end(from_list); } void FreeRegionList::remove_starting_at(HeapRegion* first, uint num_regions) { --- old/src/hotspot/share/gc/g1/heapRegionSet.hpp 2019-11-26 16:18:16.598873827 +0100 +++ new/src/hotspot/share/gc/g1/heapRegionSet.hpp 2019-11-26 16:18:16.206871883 +0100 @@ -180,6 +180,10 @@ inline void increase_length(uint node_index); inline void decrease_length(uint node_index); + // Common checks for adding a list. + void add_list_common_start(FreeRegionList* from_list); + void add_list_common_end(FreeRegionList* from_list); + protected: // See the comment for HeapRegionSetBase::clear() virtual void clear(); @@ -202,6 +206,8 @@ // Assumes that the list is ordered and will preserve that order. The order // is determined by hrm_index. inline void add_ordered(HeapRegion* hr); + // Same restrictions as above, but adds the region last in the list. + inline void add_to_tail(HeapRegion* region_to_add); // Removes from head or tail based on the given argument. HeapRegion* remove_region(bool from_head); @@ -212,10 +218,15 @@ // Merge two ordered lists. The result is also ordered. The order is // determined by hrm_index. void add_ordered(FreeRegionList* from_list); + void append_ordered(FreeRegionList* from_list); // It empties the list by removing all regions from it. void remove_all(); + // Abandon current free list. Requires that all regions in the current list + // are taken care of separately, to allow a rebuild. + void abandon(); + // Remove all (contiguous) regions from first to first + num_regions -1 from // this list. // Num_regions must be > 1. --- old/src/hotspot/share/gc/g1/heapRegionSet.inline.hpp 2019-11-26 16:18:17.122876425 +0100 +++ new/src/hotspot/share/gc/g1/heapRegionSet.inline.hpp 2019-11-26 16:18:16.690874283 +0100 @@ -50,6 +50,26 @@ _length--; } +inline void FreeRegionList::add_to_tail(HeapRegion* region_to_add) { + assert_free_region_list((length() == 0 && _head == NULL && _tail == NULL && _last == NULL) || + (length() > 0 && _head != NULL && _tail != NULL && _tail->hrm_index() < region_to_add->hrm_index()), + "invariant"); + // add() will verify the region and check mt safety. + add(region_to_add); + + if (_head != NULL) { + // Link into list, next is already NULL, no need to set. + region_to_add->set_prev(_tail); + _tail->set_next(region_to_add); + _tail = region_to_add; + } else { + // Empty list, this region is now the list. + _head = region_to_add; + _tail = region_to_add; + } + increase_length(region_to_add->node_index()); +} + inline void FreeRegionList::add_ordered(HeapRegion* hr) { assert_free_region_list((length() == 0 && _head == NULL && _tail == NULL && _last == NULL) || (length() > 0 && _head != NULL && _tail != NULL), --- old/test/hotspot/jtreg/gc/g1/TestGCLogMessages.java 2019-11-26 16:18:17.638878984 +0100 +++ new/test/hotspot/jtreg/gc/g1/TestGCLogMessages.java 2019-11-26 16:18:17.210876861 +0100 @@ -145,9 +145,15 @@ new LogMessageWithLevel("Prepare Heap Roots", Level.DEBUG), // Free CSet new LogMessageWithLevel("Free Collection Set", Level.DEBUG), - new LogMessageWithLevel("Free Collection Set Serial", Level.TRACE), + new LogMessageWithLevel("Serial Free Collection Set", Level.TRACE), + new LogMessageWithLevel("Parallel Free Collection Set", Level.TRACE), new LogMessageWithLevel("Young Free Collection Set", Level.TRACE), new LogMessageWithLevel("Non-Young Free Collection Set", Level.TRACE), + // Rebuild Free List + new LogMessageWithLevel("Rebuild Free List", Level.DEBUG), + new LogMessageWithLevel("Serial Rebuild Free List", Level.TRACE), + new LogMessageWithLevel("Parallel Rebuild Free List", Level.TRACE), + // Humongous Eager Reclaim new LogMessageWithLevel("Humongous Reclaim", Level.DEBUG), // Merge PSS --- old/test/jdk/jdk/jfr/event/gc/collection/TestG1ParallelPhases.java 2019-11-26 16:18:18.162881582 +0100 +++ new/test/jdk/jdk/jfr/event/gc/collection/TestG1ParallelPhases.java 2019-11-26 16:18:17.726879420 +0100 @@ -110,8 +110,10 @@ "StringDedupQueueFixup", "StringDedupTableFixup", "RedirtyCards", + "ParFreeCSet", "NonYoungFreeCSet", - "YoungFreeCSet" + "YoungFreeCSet", + "RebuildFreeList" ); // Some GC phases may or may not occur depending on environment. Filter them out