< prev index next >

src/hotspot/share/gc/g1/g1CollectedHeap.cpp

Print this page

        

*** 96,105 **** --- 96,106 ---- #include "runtime/init.hpp" #include "runtime/orderAccess.hpp" #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" size_t G1CollectedHeap::_humongous_object_threshold_in_words = 0;
*** 2364,2373 **** --- 2365,2378 ---- void G1CollectedHeap::collection_set_iterate_all(HeapRegionClosure* cl) { _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()); } HeapWord* G1CollectedHeap::block_start(const void* addr) const {
*** 4077,4087 **** bool skip_hot_card_cache, bool locked) { 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()); concurrent_mark()->clear_range_in_prev_bitmap(mr); } --- 4082,4091 ----
*** 4092,4102 **** --- 4096,4108 ---- if (!skip_hot_card_cache && !hr->is_young()) { _hot_card_cache->reset_card_counts(hr); } hr->hr_clear(skip_remset, true /* clear_space */, locked /* locked */); _policy->remset_tracker()->update_at_free(hr); + if (free_list != NULL) { free_list->add_ordered(hr); + } } void G1CollectedHeap::free_humongous_region(HeapRegion* hr, FreeRegionList* free_list) { assert(hr->is_humongous(), "this is only for humongous regions");
*** 4113,4410 **** _humongous_set.bulk_remove(humongous_regions_removed); } } void G1CollectedHeap::prepend_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->insert_list_into_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; 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") { ! } ! ! virtual bool do_heap_region(HeapRegion* r) { ! G1CollectedHeap* g1h = G1CollectedHeap::heap(); ! assert(r->in_collection_set(), "Region %u should be in collection set.", r->hrm_index()); ! g1h->clear_region_attr(r); ! 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); ! } ! 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; } - // 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; ! g1h->old_set_add(r); ! _after_used_bytes += r->used(); } - return false; } ! void complete_work() { ! G1CollectedHeap* g1h = G1CollectedHeap::heap(); ! ! _evacuation_info->set_regions_freed(_local_free_list.length()); ! _evacuation_info->increment_collectionset_used_after(_after_used_bytes); ! ! g1h->prepend_to_freelist(&_local_free_list); ! g1h->decrement_summary_bytes(_before_used_bytes); ! ! G1Policy* policy = g1h->policy(); ! policy->add_bytes_allocated_in_old_since_last_gc(_bytes_allocated_in_old_since_last_gc); ! g1h->alloc_buffer_stats(G1HeapRegionAttr::Old)->add_failure_used_and_waste(_failure_used_words, _failure_waste_words); } }; ! 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(); } - }; ! volatile size_t _parallel_work_claim; ! size_t _num_work_items; ! WorkItem* _work_items; ! 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); } ! void do_parallel_work_for_region(uint region_idx, bool is_young, bool evacuation_failed) { ! G1CollectedHeap* g1h = G1CollectedHeap::heap(); ! HeapRegion* r = g1h->region_at(region_idx); ! assert(!g1h->is_on_master_free_list(r), "sanity"); ! Atomic::add(&_rs_length, r->rem_set()->occupied_locked()); ! if (!is_young) { ! g1h->hot_card_cache()->reset_card_counts(r); } ! if (!evacuation_failed) { ! r->rem_set()->clear_locked(); } } ! 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 prepare_work() { ! G1PrepareFreeCollectionSetClosure cl(_work_items); ! _collection_set->iterate(&cl); ! } ! void complete_work() { ! _cl.complete_work(); ! G1Policy* policy = G1CollectedHeap::heap()->policy(); ! policy->record_rs_length(_rs_length); ! policy->cset_regions_freed(); } 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() { ! complete_work(); ! FREE_C_HEAP_ARRAY(WorkItem, _work_items); } - // 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(); ! // 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); ! } } ! // 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; } ! EventGCPhaseParallel event; ! double start_time = os::elapsedTime(); ! end = MIN2(end, _num_work_items); ! for (; cur < end; cur++) { ! bool is_young = _work_items[cur].is_young; ! 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; } } ! 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); ! } } }; 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(); { ! 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, num_regions); workers()->run_task(&cl, num_workers); } ! phase_times()->record_total_free_cset_time_ms((os::elapsedTime() - free_cset_start_time) * 1000.0); collection_set->clear(); } class G1FreeHumongousRegionClosure : public HeapRegionClosure { --- 4119,4496 ---- _humongous_set.bulk_remove(humongous_regions_removed); } } + 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()) { MutexLocker x(FreeList_lock, Mutex::_no_safepoint_check_flag); _hrm->insert_list_into_free_list(list); } } + 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 { ! // 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: ! 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); ! g1h->decrement_summary_bytes(_before_used_bytes); ! g1h->alloc_buffer_stats(G1HeapRegionAttr::Old)->add_failure_used_and_waste(_failure_used_words, _failure_waste_words); ! 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(); } ! 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; } ! }; ! // 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)); } } ! ~EventForRegion() { ! _event.commit(); ! } ! }; ! // 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; } }; ! // 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(); ! // 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 */); ! } ! 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); } ! // Update the region state due to the failed evacuation. ! r->handle_evacuation_failed(); ! // Add region to old set, need to hold lock. MutexLocker x(OldSets_lock, Mutex::_no_safepoint_check_flag); ! _g1h->old_set_add(r); } ! double& timer_for_region(HeapRegion* r) { ! return r->is_young() ? _young_time : _non_young_time; ! } ! 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) { } ! 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)); ! _g1h->clear_region_attr(r); ! stats()->update_rs_length(r->rem_set()->occupied_locked()); ! 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); } ! if (r->evacuation_failed()) { ! handle_failed_region(r); ! } else { ! handle_evacuated_region(r); } + assert(!_g1h->is_on_master_free_list(r), "sanity"); + + return false; } ! FreeCSetStats* stats() { ! return _stats; ! } ! 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); ! } } }; ! // G1FreeCollectionSetTask members ! G1CollectedHeap* _g1h; ! G1EvacuationInfo* _evacuation_info; ! FreeCSetStats* _worker_stats; ! HeapRegionClaimer _claimer; ! const size_t* _surviving_young_words; ! uint _active_workers; ! FreeCSetStats* worker_stats(uint worker) { ! return &_worker_stats[worker]; ! } ! 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(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() { ! 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); } virtual void work(uint worker_id) { ! 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)); } + }; ! 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"); ! } } ! ~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); ! } ! // 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; ! uint start = worker_id * _worker_chunk_size; ! uint end = MIN2(start + _worker_chunk_size, _g1h->max_regions()); ! // If start is outside the heap, this worker has nothing to do. ! if (start > end) { ! return; ! } ! 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); } } ! 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(); ! // 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_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); ! } ! ! 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 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_rebuild_freelist_time_ms((os::elapsedTime() - free_cset_end_time) * 1000.0); collection_set->clear(); } class G1FreeHumongousRegionClosure : public HeapRegionClosure {
< prev index next >