< prev index next >

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

Print this page
rev 54081 : imported patch 8220301-remove-jbyte-use-cardtable
rev 54086 : imported patch 8219100-cleanup-young-collection-prologue
rev 54087 : imported patch 8218668-reorganize-collection-set


 137       _num_dirtied++;
 138     }
 139 
 140     return true;
 141   }
 142 
 143   size_t num_dirtied()   const { return _num_dirtied; }
 144 };
 145 
 146 
 147 void G1RegionMappingChangedListener::reset_from_card_cache(uint start_idx, size_t num_regions) {
 148   HeapRegionRemSet::invalidate_from_card_cache(start_idx, num_regions);
 149 }
 150 
 151 void G1RegionMappingChangedListener::on_commit(uint start_idx, size_t num_regions, bool zero_filled) {
 152   // The from card cache is not the memory that is actually committed. So we cannot
 153   // take advantage of the zero_filled parameter.
 154   reset_from_card_cache(start_idx, num_regions);
 155 }
 156 





 157 
 158 HeapRegion* G1CollectedHeap::new_heap_region(uint hrs_index,
 159                                              MemRegion mr) {
 160   return new HeapRegion(hrs_index, bot(), mr);
 161 }
 162 
 163 // Private methods.
 164 
 165 HeapRegion* G1CollectedHeap::new_region(size_t word_size, HeapRegionType type, bool do_expand) {
 166   assert(!is_humongous(word_size) || word_size <= HeapRegion::GrainWords,
 167          "the only time we use this to allocate a humongous region is "
 168          "when we are allocating a single humongous region");
 169 
 170   HeapRegion* res = _hrm->allocate_free_region(type);
 171 
 172   if (res == NULL && do_expand && _expand_heap_after_alloc_failure) {
 173     // Currently, only attempts to allocate GC alloc regions set
 174     // do_expand to true. So, we should only reach here during a
 175     // safepoint. If this assumption changes we might have to
 176     // reconsider the use of _expand_heap_after_alloc_failure.


2229 void G1CollectedHeap::heap_region_iterate(HeapRegionClosure* cl) const {
2230   _hrm->iterate(cl);
2231 }
2232 
2233 void G1CollectedHeap::heap_region_par_iterate_from_worker_offset(HeapRegionClosure* cl,
2234                                                                  HeapRegionClaimer *hrclaimer,
2235                                                                  uint worker_id) const {
2236   _hrm->par_iterate(cl, hrclaimer, hrclaimer->offset_for_worker(worker_id));
2237 }
2238 
2239 void G1CollectedHeap::heap_region_par_iterate_from_start(HeapRegionClosure* cl,
2240                                                          HeapRegionClaimer *hrclaimer) const {
2241   _hrm->par_iterate(cl, hrclaimer, 0);
2242 }
2243 
2244 void G1CollectedHeap::collection_set_iterate(HeapRegionClosure* cl) {
2245   _collection_set.iterate(cl);
2246 }
2247 
2248 void G1CollectedHeap::collection_set_iterate_from(HeapRegionClosure *cl, uint worker_id) {
2249   _collection_set.iterate_from(cl, worker_id, workers()->active_workers());
2250 }
2251 
2252 HeapWord* G1CollectedHeap::block_start(const void* addr) const {
2253   HeapRegion* hr = heap_region_containing(addr);
2254   return hr->block_start(addr);
2255 }
2256 
2257 size_t G1CollectedHeap::block_size(const HeapWord* addr) const {
2258   HeapRegion* hr = heap_region_containing(addr);
2259   return hr->block_size(addr);
2260 }
2261 
2262 bool G1CollectedHeap::block_is_obj(const HeapWord* addr) const {
2263   HeapRegion* hr = heap_region_containing(addr);
2264   return hr->block_is_obj(addr);
2265 }
2266 
2267 bool G1CollectedHeap::supports_tlab_allocation() const {
2268   return true;
2269 }


2867   }
2868 };
2869 
2870 void G1CollectedHeap::start_new_collection_set() {
2871   double start = os::elapsedTime();
2872 
2873   collection_set()->start_incremental_building();
2874 
2875   clear_cset_fast_test();
2876 
2877   guarantee(_eden.length() == 0, "eden should have been cleared");
2878   policy()->transfer_survivors_to_cset(survivor());
2879 
2880   // We redo the verification but now wrt to the new CSet which
2881   // has just got initialized after the previous CSet was freed.
2882   _cm->verify_no_collection_set_oops_in_stacks();
2883 
2884   phase_times()->record_start_new_cset_time_ms((os::elapsedTime() - start) * 1000.0);
2885 }
2886 
2887 void G1CollectedHeap::calculate_collection_set(G1EvacuationInfo& evacuation_info, double target_pause_time_ms){
2888   policy()->finalize_collection_set(target_pause_time_ms, &_survivor);
2889   evacuation_info.set_collectionset_regions(collection_set()->region_length());

2890 
2891   _cm->verify_no_collection_set_oops_in_stacks();
2892 
2893   if (_hr_printer.is_active()) {
2894     G1PrintCollectionSetClosure cl(&_hr_printer);
2895     _collection_set.iterate(&cl);

2896   }
2897 }
2898 
2899 G1HeapVerifier::G1VerifyType G1CollectedHeap::young_collection_verify_type() const {
2900   if (collector_state()->in_initial_mark_gc()) {
2901     return G1HeapVerifier::G1VerifyConcurrentStart;
2902   } else if (collector_state()->in_young_only_phase()) {
2903     return G1HeapVerifier::G1VerifyYoungNormal;
2904   } else {
2905     return G1HeapVerifier::G1VerifyMixed;
2906   }
2907 }
2908 
2909 void G1CollectedHeap::verify_before_young_collection(G1HeapVerifier::G1VerifyType type) {
2910   if (VerifyRememberedSets) {
2911     log_info(gc, verify)("[Verifying RemSets before GC]");
2912     VerifyRegionRemSetClosure v_cl;
2913     heap_region_iterate(&v_cl);
2914   }
2915   _verifier->verify_before_gc(type);


3047         // CM ref processor, if necessary, and turn it back on
3048         // on again later if we do. Using a scoped
3049         // NoRefDiscovery object will do this.
3050         NoRefDiscovery no_cm_discovery(_ref_processor_cm);
3051 
3052         policy()->record_collection_pause_start(sample_start_time_sec);
3053 
3054         // Forget the current allocation region (we might even choose it to be part
3055         // of the collection set!).
3056         _allocator->release_mutator_alloc_region();
3057 
3058         calculate_collection_set(evacuation_info, target_pause_time_ms);
3059 
3060         G1ParScanThreadStateSet per_thread_states(this,
3061                                                   workers()->active_workers(),
3062                                                   collection_set()->young_region_length(),
3063                                                   collection_set()->optional_region_length());
3064         pre_evacuate_collection_set(evacuation_info);
3065 
3066         // Actually do the work...
3067         evacuate_collection_set(&per_thread_states);

3068         evacuate_optional_collection_set(&per_thread_states);
3069 
3070         post_evacuate_collection_set(evacuation_info, &per_thread_states);
3071 
3072         start_new_collection_set();
3073 
3074         _survivor_evac_stats.adjust_desired_plab_sz();
3075         _old_evac_stats.adjust_desired_plab_sz();
3076 
3077         if (should_start_conc_mark) {
3078           // We have to do this before we notify the CM threads that
3079           // they can start working to make sure that all the
3080           // appropriate initialization is done on the CM object.
3081           concurrent_mark()->post_initial_mark();
3082           // Note that we don't actually trigger the CM thread at
3083           // this point. We do that later when we're sure that
3084           // the current thread has completed its logging output.
3085         }
3086 
3087         allocate_dummy_regions();
3088 
3089         _allocator->init_mutator_alloc_region();
3090 
3091         expand_heap_after_young_collection();
3092 
3093         double sample_end_time_sec = os::elapsedTime();
3094         double pause_time_ms = (sample_end_time_sec - sample_start_time_sec) * MILLIUNITS;
3095         size_t total_cards_scanned = phase_times()->sum_thread_work_items(G1GCPhaseTimes::ScanRS, G1GCPhaseTimes::ScanRSScannedCards);

3096         policy()->record_collection_pause_end(pause_time_ms, total_cards_scanned, heap_used_bytes_before_gc);
3097       }
3098 
3099       verify_after_young_collection(verify_type);
3100 
3101 #ifdef TRACESPINNING
3102       ParallelTaskTerminator::print_termination_counts();
3103 #endif
3104 
3105       gc_epilogue(false);
3106     }
3107 
3108     // Print the remainder of the GC log output.
3109     if (evacuation_failed()) {
3110       log_info(gc)("To-space exhausted");
3111     }
3112 
3113     policy()->print_phases();
3114     heap_transition.print();
3115 


3179   G1ParScanThreadState* const pss = par_scan_state();
3180   start_term_time();
3181   const bool res = terminator()->offer_termination();
3182   end_term_time();
3183   event.commit(GCId::current(), pss->worker_id(), G1GCPhaseTimes::phase_name(G1GCPhaseTimes::Termination));
3184   return res;
3185 }
3186 
3187 void G1ParEvacuateFollowersClosure::do_void() {
3188   EventGCPhaseParallel event;
3189   G1ParScanThreadState* const pss = par_scan_state();
3190   pss->trim_queue();
3191   event.commit(GCId::current(), pss->worker_id(), G1GCPhaseTimes::phase_name(_phase));
3192   do {
3193     EventGCPhaseParallel event;
3194     pss->steal_and_trim_queue(queues());
3195     event.commit(GCId::current(), pss->worker_id(), G1GCPhaseTimes::phase_name(_phase));
3196   } while (!offer_termination());
3197 }
3198 
3199 class G1ParTask : public AbstractGangTask {
3200 protected:
3201   G1CollectedHeap*         _g1h;
3202   G1ParScanThreadStateSet* _pss;
3203   RefToScanQueueSet*       _queues;
3204   G1RootProcessor*         _root_processor;
3205   TaskTerminator           _terminator;
3206   uint                     _n_workers;
3207 
3208 public:
3209   G1ParTask(G1CollectedHeap* g1h, G1ParScanThreadStateSet* per_thread_states, RefToScanQueueSet *task_queues, G1RootProcessor* root_processor, uint n_workers)
3210     : AbstractGangTask("G1 collection"),
3211       _g1h(g1h),
3212       _pss(per_thread_states),
3213       _queues(task_queues),
3214       _root_processor(root_processor),
3215       _terminator(n_workers, _queues),
3216       _n_workers(n_workers)
3217   {}
3218 
3219   void work(uint worker_id) {
3220     if (worker_id >= _n_workers) return;  // no work needed this round
3221 
3222     double start_sec = os::elapsedTime();
3223     _g1h->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerStart, worker_id, start_sec);
3224 
3225     {
3226       ResourceMark rm;
3227       HandleMark   hm;
3228 
3229       ReferenceProcessor*             rp = _g1h->ref_processor_stw();
3230 
3231       G1ParScanThreadState*           pss = _pss->state_for_worker(worker_id);
3232       pss->set_ref_discoverer(rp);
3233 
3234       double start_strong_roots_sec = os::elapsedTime();
3235 
3236       _root_processor->evacuate_roots(pss, worker_id);
3237 
3238       _g1h->rem_set()->oops_into_collection_set_do(pss, worker_id);
3239 
3240       double strong_roots_sec = os::elapsedTime() - start_strong_roots_sec;
3241 
3242       double term_sec = 0.0;
3243       size_t evac_term_attempts = 0;
3244       {
3245         double start = os::elapsedTime();
3246         G1ParEvacuateFollowersClosure evac(_g1h, pss, _queues, _terminator.terminator(), G1GCPhaseTimes::ObjCopy);
3247         evac.do_void();
3248 
3249         evac_term_attempts = evac.term_attempts();
3250         term_sec = evac.term_time();
3251         double elapsed_sec = os::elapsedTime() - start;
3252 
3253         G1GCPhaseTimes* p = _g1h->phase_times();
3254         p->add_time_secs(G1GCPhaseTimes::ObjCopy, worker_id, elapsed_sec - term_sec);
3255 
3256         p->record_or_add_thread_work_item(G1GCPhaseTimes::ObjCopy,
3257                                           worker_id,
3258                                           pss->lab_waste_words() * HeapWordSize,
3259                                           G1GCPhaseTimes::ObjCopyLABWaste);
3260         p->record_or_add_thread_work_item(G1GCPhaseTimes::ObjCopy,
3261                                           worker_id,
3262                                           pss->lab_undo_waste_words() * HeapWordSize,
3263                                           G1GCPhaseTimes::ObjCopyLABUndoWaste);
3264 
3265         p->record_time_secs(G1GCPhaseTimes::Termination, worker_id, term_sec);
3266         p->record_thread_work_item(G1GCPhaseTimes::Termination, worker_id, evac_term_attempts);
3267       }
3268 
3269       assert(pss->queue_is_empty(), "should be empty");
3270 
3271       // Close the inner scope so that the ResourceMark and HandleMark
3272       // destructors are executed here and are included as part of the
3273       // "GC Worker Time".
3274     }
3275     _g1h->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerEnd, worker_id, os::elapsedTime());
3276   }
3277 };
3278 
3279 void G1CollectedHeap::complete_cleaning(BoolObjectClosure* is_alive,
3280                                         bool class_unloading_occurred) {
3281   uint num_workers = workers()->active_workers();
3282   ParallelCleaningTask unlink_task(is_alive, num_workers, class_unloading_occurred, false);
3283   workers()->run_task(&unlink_task);
3284 }
3285 
3286 // Clean string dedup data structures.
3287 // Ideally we would prefer to use a StringDedupCleaningTask here, but we want to
3288 // record the durations of the phases. Hence the almost-copy.
3289 class G1StringDedupCleaningTask : public AbstractGangTask {
3290   BoolObjectClosure* _is_alive;
3291   OopClosure* _keep_alive;
3292   G1GCPhaseTimes* _phase_times;
3293 
3294 public:
3295   G1StringDedupCleaningTask(BoolObjectClosure* is_alive,
3296                             OopClosure* keep_alive,
3297                             G1GCPhaseTimes* phase_times) :
3298     AbstractGangTask("Partial Cleaning Task"),


3662   assert(_verifier->check_cset_fast_test(), "Inconsistency in the InCSetState table.");
3663 
3664   rem_set()->prepare_for_oops_into_collection_set_do();
3665   _preserved_marks_set.assert_empty();
3666 
3667 #if COMPILER2_OR_JVMCI
3668   DerivedPointerTable::clear();
3669 #endif
3670 
3671   // InitialMark needs claim bits to keep track of the marked-through CLDs.
3672   if (collector_state()->in_initial_mark_gc()) {
3673     concurrent_mark()->pre_initial_mark();
3674 
3675     double start_clear_claimed_marks = os::elapsedTime();
3676 
3677     ClassLoaderDataGraph::clear_claimed_marks();
3678 
3679     double recorded_clear_claimed_marks_time_ms = (os::elapsedTime() - start_clear_claimed_marks) * 1000.0;
3680     phase_times()->record_clear_claimed_marks_time_ms(recorded_clear_claimed_marks_time_ms);
3681   }
3682 }
3683 
3684 void G1CollectedHeap::evacuate_collection_set(G1ParScanThreadStateSet* per_thread_states) {
3685   // Should G1EvacuationFailureALot be in effect for this GC?
3686   NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
3687 
3688   assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty");
3689 
3690   double start_par_time_sec = os::elapsedTime();
3691   double end_par_time_sec;
3692 
3693   {
3694     const uint n_workers = workers()->active_workers();
3695     G1RootProcessor root_processor(this, n_workers);
3696     G1ParTask g1_par_task(this, per_thread_states, _task_queues, &root_processor, n_workers);
3697 
3698     workers()->run_task(&g1_par_task);
3699     end_par_time_sec = os::elapsedTime();
3700 
3701     // Closing the inner scope will execute the destructor
3702     // for the G1RootProcessor object. We record the current
3703     // elapsed time before closing the scope so that time
3704     // taken for the destructor is NOT included in the
3705     // reported parallel time.
3706   }
3707 
3708   double par_time_ms = (end_par_time_sec - start_par_time_sec) * 1000.0;
3709   phase_times()->record_par_time(par_time_ms);
3710 
3711   double code_root_fixup_time_ms =
3712         (os::elapsedTime() - end_par_time_sec) * 1000.0;
3713   phase_times()->record_code_root_fixup_time(code_root_fixup_time_ms);
3714 }
3715 
3716 class G1EvacuateOptionalRegionTask : public AbstractGangTask {

3717   G1CollectedHeap* _g1h;
3718   G1ParScanThreadStateSet* _per_thread_states;
3719   G1OptionalCSet* _optional;
3720   RefToScanQueueSet* _queues;
3721   ParallelTaskTerminator _terminator;
3722 
3723   Tickspan trim_ticks(G1ParScanThreadState* pss) {
3724     Tickspan copy_time = pss->trim_ticks();
3725     pss->reset_trim_ticks();
3726     return copy_time;
3727   }
3728 
3729   void scan_roots(G1ParScanThreadState* pss, uint worker_id) {
3730     G1EvacuationRootClosures* root_cls = pss->closures();
3731     G1ScanObjsDuringScanRSClosure obj_cl(_g1h, pss);
3732 
3733     size_t scanned = 0;
3734     size_t claimed = 0;
3735     size_t skipped = 0;
3736     size_t used_memory = 0;
3737 
3738     Ticks    start = Ticks::now();
3739     Tickspan copy_time;
3740 
3741     for (uint i = _optional->current_index(); i < _optional->current_limit(); i++) {
3742       HeapRegion* hr = _optional->region_at(i);
3743       G1ScanRSForOptionalClosure scan_opt_cl(&obj_cl);
3744       pss->oops_into_optional_region(hr)->oops_do(&scan_opt_cl, root_cls->raw_strong_oops());
3745       copy_time += trim_ticks(pss);
3746 
3747       G1ScanRSForRegionClosure scan_rs_cl(_g1h->rem_set()->scan_state(), &obj_cl, pss, G1GCPhaseTimes::OptScanRS, worker_id);
3748       scan_rs_cl.do_heap_region(hr);
3749       copy_time += trim_ticks(pss);
3750       scanned += scan_rs_cl.cards_scanned();
3751       claimed += scan_rs_cl.cards_claimed();
3752       skipped += scan_rs_cl.cards_skipped();
3753 
3754       // Chunk lists for this region is no longer needed.
3755       used_memory += pss->oops_into_optional_region(hr)->used_memory();






3756     }
3757 
3758     Tickspan scan_time = (Ticks::now() - start) - copy_time;
3759     G1GCPhaseTimes* p = _g1h->phase_times();
3760     p->record_or_add_time_secs(G1GCPhaseTimes::OptScanRS, worker_id, scan_time.seconds());
3761     p->record_or_add_time_secs(G1GCPhaseTimes::OptObjCopy, worker_id, copy_time.seconds());
3762 
3763     p->record_or_add_thread_work_item(G1GCPhaseTimes::OptScanRS, worker_id, scanned, G1GCPhaseTimes::OptCSetScannedCards);
3764     p->record_or_add_thread_work_item(G1GCPhaseTimes::OptScanRS, worker_id, claimed, G1GCPhaseTimes::OptCSetClaimedCards);
3765     p->record_or_add_thread_work_item(G1GCPhaseTimes::OptScanRS, worker_id, skipped, G1GCPhaseTimes::OptCSetSkippedCards);
3766     p->record_or_add_thread_work_item(G1GCPhaseTimes::OptScanRS, worker_id, used_memory, G1GCPhaseTimes::OptCSetUsedMemory);
3767   }
3768 
3769   void evacuate_live_objects(G1ParScanThreadState* pss, uint worker_id) {
3770     Ticks start = Ticks::now();
3771     G1ParEvacuateFollowersClosure cl(_g1h, pss, _queues, &_terminator, G1GCPhaseTimes::OptObjCopy);
3772     cl.do_void();
3773 
3774     Tickspan evac_time = (Ticks::now() - start);
3775     G1GCPhaseTimes* p = _g1h->phase_times();
3776     p->record_or_add_time_secs(G1GCPhaseTimes::OptObjCopy, worker_id, evac_time.seconds());
3777     assert(pss->trim_ticks().seconds() == 0.0, "Unexpected partial trimming done during optional evacuation");
3778   }
3779 
3780  public:
3781   G1EvacuateOptionalRegionTask(G1CollectedHeap* g1h,
3782                                G1ParScanThreadStateSet* per_thread_states,
3783                                G1OptionalCSet* cset,
3784                                RefToScanQueueSet* queues,
3785                                uint n_workers) :
3786     AbstractGangTask("G1 Evacuation Optional Region Task"),
3787     _g1h(g1h),
3788     _per_thread_states(per_thread_states),
3789     _optional(cset),
3790     _queues(queues),
3791     _terminator(n_workers, _queues) {
3792   }
3793 
3794   void work(uint worker_id) {



3795     ResourceMark rm;
3796     HandleMark  hm;
3797 
3798     G1ParScanThreadState* pss = _per_thread_states->state_for_worker(worker_id);
3799     pss->set_ref_discoverer(_g1h->ref_processor_stw());
3800 
3801     scan_roots(pss, worker_id);
3802     evacuate_live_objects(pss, worker_id);
3803   }



3804 };
3805 
3806 void G1CollectedHeap::evacuate_optional_regions(G1ParScanThreadStateSet* per_thread_states, G1OptionalCSet* ocset) {
3807   class G1MarkScope : public MarkScope {};
3808   G1MarkScope code_mark_scope;
































3809 
3810   G1EvacuateOptionalRegionTask task(this, per_thread_states, ocset, _task_queues, workers()->active_workers());
3811   workers()->run_task(&task);












3812 }
3813 
3814 void G1CollectedHeap::evacuate_optional_collection_set(G1ParScanThreadStateSet* per_thread_states) {
3815   G1OptionalCSet optional_cset(&_collection_set, per_thread_states);
3816   if (optional_cset.is_empty()) {
3817     return;
3818   }
3819 
3820   if (evacuation_failed()) {
3821     return;
3822   }
3823 




























3824   const double gc_start_time_ms = phase_times()->cur_collection_start_sec() * 1000.0;
3825 
3826   double start_time_sec = os::elapsedTime();


3827 
3828   do {
3829     double time_used_ms = os::elapsedTime() * 1000.0 - gc_start_time_ms;
3830     double time_left_ms = MaxGCPauseMillis - time_used_ms;
3831 
3832     if (time_left_ms < 0) {
3833       log_trace(gc, ergo, cset)("Skipping %u optional regions, pause time exceeded %.3fms", optional_cset.size(), time_used_ms);


3834       break;
3835     }
3836 
3837     optional_cset.prepare_evacuation(time_left_ms * _policy->optional_evacuation_fraction());
3838     if (optional_cset.prepare_failed()) {
3839       log_trace(gc, ergo, cset)("Skipping %u optional regions, no regions can be evacuated in %.3fms", optional_cset.size(), time_left_ms);
3840       break;
3841     }
3842 
3843     evacuate_optional_regions(per_thread_states, &optional_cset);
3844 
3845     optional_cset.complete_evacuation();
3846     if (optional_cset.evacuation_failed()) {
3847       break;
3848     }
3849   } while (!optional_cset.is_empty());
3850 
3851   phase_times()->record_optional_evacuation((os::elapsedTime() - start_time_sec) * 1000.0);
3852 }
3853 
3854 void G1CollectedHeap::post_evacuate_collection_set(G1EvacuationInfo& evacuation_info, G1ParScanThreadStateSet* per_thread_states) {
3855   // Also cleans the card table from temporary duplicate detection information used
3856   // during UpdateRS/ScanRS.
3857   rem_set()->cleanup_after_oops_into_collection_set_do();
3858 
3859   // Process any discovered reference objects - we have
3860   // to do this _before_ we retire the GC alloc regions
3861   // as we may have to copy some 'reachable' referent
3862   // objects (and their reachable sub-graphs) that were
3863   // not copied during the pause.
3864   process_discovered_references(per_thread_states);
3865 
3866   G1STWIsAliveClosure is_alive(this);
3867   G1KeepAliveClosure keep_alive(this);
3868 
3869   WeakProcessor::weak_oops_do(workers(), &is_alive, &keep_alive,
3870                               phase_times()->weak_phase_times());
3871 


4246         }
4247         start_time = end_time;
4248       }
4249     }
4250 
4251     if (has_young_time) {
4252       timer->record_time_secs(G1GCPhaseTimes::YoungFreeCSet, worker_id, young_time);
4253     }
4254     if (has_non_young_time) {
4255       timer->record_time_secs(G1GCPhaseTimes::NonYoungFreeCSet, worker_id, non_young_time);
4256     }
4257   }
4258 };
4259 
4260 void G1CollectedHeap::free_collection_set(G1CollectionSet* collection_set, G1EvacuationInfo& evacuation_info, const size_t* surviving_young_words) {
4261   _eden.clear();
4262 
4263   double free_cset_start_time = os::elapsedTime();
4264 
4265   {
4266     uint const num_chunks = MAX2(_collection_set.region_length() / G1FreeCollectionSetTask::chunk_size(), 1U);

4267     uint const num_workers = MIN2(workers()->active_workers(), num_chunks);
4268 
4269     G1FreeCollectionSetTask cl(collection_set, &evacuation_info, surviving_young_words);
4270 
4271     log_debug(gc, ergo)("Running %s using %u workers for collection set length %u",
4272                         cl.name(),
4273                         num_workers,
4274                         _collection_set.region_length());
4275     workers()->run_task(&cl, num_workers);
4276   }
4277   phase_times()->record_total_free_cset_time_ms((os::elapsedTime() - free_cset_start_time) * 1000.0);
4278 
4279   collection_set->clear();
4280 }
4281 
4282 class G1FreeHumongousRegionClosure : public HeapRegionClosure {
4283  private:
4284   FreeRegionList* _free_region_list;
4285   HeapRegionSet* _proxy_set;
4286   uint _humongous_objects_reclaimed;
4287   uint _humongous_regions_reclaimed;
4288   size_t _freed_bytes;
4289  public:
4290 
4291   G1FreeHumongousRegionClosure(FreeRegionList* free_region_list) :
4292     _free_region_list(free_region_list), _proxy_set(NULL), _humongous_objects_reclaimed(0), _humongous_regions_reclaimed(0), _freed_bytes(0) {
4293   }
4294 


4423 
4424   prepend_to_freelist(&local_cleanup_list);
4425   decrement_summary_bytes(cl.bytes_freed());
4426 
4427   phase_times()->record_fast_reclaim_humongous_time_ms((os::elapsedTime() - start_time) * 1000.0,
4428                                                        cl.humongous_objects_reclaimed());
4429 }
4430 
4431 class G1AbandonCollectionSetClosure : public HeapRegionClosure {
4432 public:
4433   virtual bool do_heap_region(HeapRegion* r) {
4434     assert(r->in_collection_set(), "Region %u must have been in collection set", r->hrm_index());
4435     G1CollectedHeap::heap()->clear_in_cset(r);
4436     r->set_young_index_in_cset(-1);
4437     return false;
4438   }
4439 };
4440 
4441 void G1CollectedHeap::abandon_collection_set(G1CollectionSet* collection_set) {
4442   G1AbandonCollectionSetClosure cl;
4443   collection_set->iterate(&cl);
4444 
4445   collection_set->clear();
4446   collection_set->stop_incremental_building();
4447 }
4448 
4449 bool G1CollectedHeap::is_old_gc_alloc_region(HeapRegion* hr) {
4450   return _allocator->is_retained_old_region(hr);
4451 }
4452 
4453 void G1CollectedHeap::set_region_short_lived_locked(HeapRegion* hr) {
4454   _eden.add(hr);
4455   _policy->set_region_eden(hr);
4456 }
4457 
4458 #ifdef ASSERT
4459 
4460 class NoYoungRegionsClosure: public HeapRegionClosure {
4461 private:
4462   bool _success;
4463 public:




 137       _num_dirtied++;
 138     }
 139 
 140     return true;
 141   }
 142 
 143   size_t num_dirtied()   const { return _num_dirtied; }
 144 };
 145 
 146 
 147 void G1RegionMappingChangedListener::reset_from_card_cache(uint start_idx, size_t num_regions) {
 148   HeapRegionRemSet::invalidate_from_card_cache(start_idx, num_regions);
 149 }
 150 
 151 void G1RegionMappingChangedListener::on_commit(uint start_idx, size_t num_regions, bool zero_filled) {
 152   // The from card cache is not the memory that is actually committed. So we cannot
 153   // take advantage of the zero_filled parameter.
 154   reset_from_card_cache(start_idx, num_regions);
 155 }
 156 
 157 Tickspan G1CollectedHeap::run_task(AbstractGangTask* task, uint num_workers) {
 158   Ticks start = Ticks::now();
 159   workers()->run_task(task, num_workers == 0 ? workers()->active_workers() : num_workers);
 160   return Ticks::now() - start;
 161 }
 162 
 163 HeapRegion* G1CollectedHeap::new_heap_region(uint hrs_index,
 164                                              MemRegion mr) {
 165   return new HeapRegion(hrs_index, bot(), mr);
 166 }
 167 
 168 // Private methods.
 169 
 170 HeapRegion* G1CollectedHeap::new_region(size_t word_size, HeapRegionType type, bool do_expand) {
 171   assert(!is_humongous(word_size) || word_size <= HeapRegion::GrainWords,
 172          "the only time we use this to allocate a humongous region is "
 173          "when we are allocating a single humongous region");
 174 
 175   HeapRegion* res = _hrm->allocate_free_region(type);
 176 
 177   if (res == NULL && do_expand && _expand_heap_after_alloc_failure) {
 178     // Currently, only attempts to allocate GC alloc regions set
 179     // do_expand to true. So, we should only reach here during a
 180     // safepoint. If this assumption changes we might have to
 181     // reconsider the use of _expand_heap_after_alloc_failure.


2234 void G1CollectedHeap::heap_region_iterate(HeapRegionClosure* cl) const {
2235   _hrm->iterate(cl);
2236 }
2237 
2238 void G1CollectedHeap::heap_region_par_iterate_from_worker_offset(HeapRegionClosure* cl,
2239                                                                  HeapRegionClaimer *hrclaimer,
2240                                                                  uint worker_id) const {
2241   _hrm->par_iterate(cl, hrclaimer, hrclaimer->offset_for_worker(worker_id));
2242 }
2243 
2244 void G1CollectedHeap::heap_region_par_iterate_from_start(HeapRegionClosure* cl,
2245                                                          HeapRegionClaimer *hrclaimer) const {
2246   _hrm->par_iterate(cl, hrclaimer, 0);
2247 }
2248 
2249 void G1CollectedHeap::collection_set_iterate(HeapRegionClosure* cl) {
2250   _collection_set.iterate(cl);
2251 }
2252 
2253 void G1CollectedHeap::collection_set_iterate_from(HeapRegionClosure *cl, uint worker_id) {
2254   _collection_set.iterate_incremental_part_from(cl, worker_id, workers()->active_workers());
2255 }
2256 
2257 HeapWord* G1CollectedHeap::block_start(const void* addr) const {
2258   HeapRegion* hr = heap_region_containing(addr);
2259   return hr->block_start(addr);
2260 }
2261 
2262 size_t G1CollectedHeap::block_size(const HeapWord* addr) const {
2263   HeapRegion* hr = heap_region_containing(addr);
2264   return hr->block_size(addr);
2265 }
2266 
2267 bool G1CollectedHeap::block_is_obj(const HeapWord* addr) const {
2268   HeapRegion* hr = heap_region_containing(addr);
2269   return hr->block_is_obj(addr);
2270 }
2271 
2272 bool G1CollectedHeap::supports_tlab_allocation() const {
2273   return true;
2274 }


2872   }
2873 };
2874 
2875 void G1CollectedHeap::start_new_collection_set() {
2876   double start = os::elapsedTime();
2877 
2878   collection_set()->start_incremental_building();
2879 
2880   clear_cset_fast_test();
2881 
2882   guarantee(_eden.length() == 0, "eden should have been cleared");
2883   policy()->transfer_survivors_to_cset(survivor());
2884 
2885   // We redo the verification but now wrt to the new CSet which
2886   // has just got initialized after the previous CSet was freed.
2887   _cm->verify_no_collection_set_oops_in_stacks();
2888 
2889   phase_times()->record_start_new_cset_time_ms((os::elapsedTime() - start) * 1000.0);
2890 }
2891 
2892 void G1CollectedHeap::calculate_collection_set(G1EvacuationInfo& evacuation_info, double target_pause_time_ms) {
2893   _collection_set.finalize_initial_collection_set(target_pause_time_ms, &_survivor);
2894   evacuation_info.set_collectionset_regions(collection_set()->region_length() +
2895                                             collection_set()->optional_region_length());
2896 
2897   _cm->verify_no_collection_set_oops_in_stacks();
2898 
2899   if (_hr_printer.is_active()) {
2900     G1PrintCollectionSetClosure cl(&_hr_printer);
2901     _collection_set.iterate(&cl);
2902     _collection_set.iterate_optional(&cl);
2903   }
2904 }
2905 
2906 G1HeapVerifier::G1VerifyType G1CollectedHeap::young_collection_verify_type() const {
2907   if (collector_state()->in_initial_mark_gc()) {
2908     return G1HeapVerifier::G1VerifyConcurrentStart;
2909   } else if (collector_state()->in_young_only_phase()) {
2910     return G1HeapVerifier::G1VerifyYoungNormal;
2911   } else {
2912     return G1HeapVerifier::G1VerifyMixed;
2913   }
2914 }
2915 
2916 void G1CollectedHeap::verify_before_young_collection(G1HeapVerifier::G1VerifyType type) {
2917   if (VerifyRememberedSets) {
2918     log_info(gc, verify)("[Verifying RemSets before GC]");
2919     VerifyRegionRemSetClosure v_cl;
2920     heap_region_iterate(&v_cl);
2921   }
2922   _verifier->verify_before_gc(type);


3054         // CM ref processor, if necessary, and turn it back on
3055         // on again later if we do. Using a scoped
3056         // NoRefDiscovery object will do this.
3057         NoRefDiscovery no_cm_discovery(_ref_processor_cm);
3058 
3059         policy()->record_collection_pause_start(sample_start_time_sec);
3060 
3061         // Forget the current allocation region (we might even choose it to be part
3062         // of the collection set!).
3063         _allocator->release_mutator_alloc_region();
3064 
3065         calculate_collection_set(evacuation_info, target_pause_time_ms);
3066 
3067         G1ParScanThreadStateSet per_thread_states(this,
3068                                                   workers()->active_workers(),
3069                                                   collection_set()->young_region_length(),
3070                                                   collection_set()->optional_region_length());
3071         pre_evacuate_collection_set(evacuation_info);
3072 
3073         // Actually do the work...
3074         evacuate_initial_collection_set(&per_thread_states);
3075         if (_collection_set.optional_region_length() != 0) {
3076           evacuate_optional_collection_set(&per_thread_states);
3077         }
3078         post_evacuate_collection_set(evacuation_info, &per_thread_states);
3079 
3080         start_new_collection_set();
3081 
3082         _survivor_evac_stats.adjust_desired_plab_sz();
3083         _old_evac_stats.adjust_desired_plab_sz();
3084 
3085         if (should_start_conc_mark) {
3086           // We have to do this before we notify the CM threads that
3087           // they can start working to make sure that all the
3088           // appropriate initialization is done on the CM object.
3089           concurrent_mark()->post_initial_mark();
3090           // Note that we don't actually trigger the CM thread at
3091           // this point. We do that later when we're sure that
3092           // the current thread has completed its logging output.
3093         }
3094 
3095         allocate_dummy_regions();
3096 
3097         _allocator->init_mutator_alloc_region();
3098 
3099         expand_heap_after_young_collection();
3100 
3101         double sample_end_time_sec = os::elapsedTime();
3102         double pause_time_ms = (sample_end_time_sec - sample_start_time_sec) * MILLIUNITS;
3103         size_t total_cards_scanned = phase_times()->sum_thread_work_items(G1GCPhaseTimes::ScanRS, G1GCPhaseTimes::ScanRSScannedCards) +
3104                                      phase_times()->sum_thread_work_items(G1GCPhaseTimes::OptScanRS, G1GCPhaseTimes::ScanRSScannedCards);
3105         policy()->record_collection_pause_end(pause_time_ms, total_cards_scanned, heap_used_bytes_before_gc);
3106       }
3107 
3108       verify_after_young_collection(verify_type);
3109 
3110 #ifdef TRACESPINNING
3111       ParallelTaskTerminator::print_termination_counts();
3112 #endif
3113 
3114       gc_epilogue(false);
3115     }
3116 
3117     // Print the remainder of the GC log output.
3118     if (evacuation_failed()) {
3119       log_info(gc)("To-space exhausted");
3120     }
3121 
3122     policy()->print_phases();
3123     heap_transition.print();
3124 


3188   G1ParScanThreadState* const pss = par_scan_state();
3189   start_term_time();
3190   const bool res = terminator()->offer_termination();
3191   end_term_time();
3192   event.commit(GCId::current(), pss->worker_id(), G1GCPhaseTimes::phase_name(G1GCPhaseTimes::Termination));
3193   return res;
3194 }
3195 
3196 void G1ParEvacuateFollowersClosure::do_void() {
3197   EventGCPhaseParallel event;
3198   G1ParScanThreadState* const pss = par_scan_state();
3199   pss->trim_queue();
3200   event.commit(GCId::current(), pss->worker_id(), G1GCPhaseTimes::phase_name(_phase));
3201   do {
3202     EventGCPhaseParallel event;
3203     pss->steal_and_trim_queue(queues());
3204     event.commit(GCId::current(), pss->worker_id(), G1GCPhaseTimes::phase_name(_phase));
3205   } while (!offer_termination());
3206 }
3207 
















































































3208 void G1CollectedHeap::complete_cleaning(BoolObjectClosure* is_alive,
3209                                         bool class_unloading_occurred) {
3210   uint num_workers = workers()->active_workers();
3211   ParallelCleaningTask unlink_task(is_alive, num_workers, class_unloading_occurred, false);
3212   workers()->run_task(&unlink_task);
3213 }
3214 
3215 // Clean string dedup data structures.
3216 // Ideally we would prefer to use a StringDedupCleaningTask here, but we want to
3217 // record the durations of the phases. Hence the almost-copy.
3218 class G1StringDedupCleaningTask : public AbstractGangTask {
3219   BoolObjectClosure* _is_alive;
3220   OopClosure* _keep_alive;
3221   G1GCPhaseTimes* _phase_times;
3222 
3223 public:
3224   G1StringDedupCleaningTask(BoolObjectClosure* is_alive,
3225                             OopClosure* keep_alive,
3226                             G1GCPhaseTimes* phase_times) :
3227     AbstractGangTask("Partial Cleaning Task"),


3591   assert(_verifier->check_cset_fast_test(), "Inconsistency in the InCSetState table.");
3592 
3593   rem_set()->prepare_for_oops_into_collection_set_do();
3594   _preserved_marks_set.assert_empty();
3595 
3596 #if COMPILER2_OR_JVMCI
3597   DerivedPointerTable::clear();
3598 #endif
3599 
3600   // InitialMark needs claim bits to keep track of the marked-through CLDs.
3601   if (collector_state()->in_initial_mark_gc()) {
3602     concurrent_mark()->pre_initial_mark();
3603 
3604     double start_clear_claimed_marks = os::elapsedTime();
3605 
3606     ClassLoaderDataGraph::clear_claimed_marks();
3607 
3608     double recorded_clear_claimed_marks_time_ms = (os::elapsedTime() - start_clear_claimed_marks) * 1000.0;
3609     phase_times()->record_clear_claimed_marks_time_ms(recorded_clear_claimed_marks_time_ms);
3610   }

3611 

3612   // Should G1EvacuationFailureALot be in effect for this GC?
3613   NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
3614 
3615   assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty");

























3616 }
3617 
3618 class G1EvacuateRegionsBaseTask : public AbstractGangTask {
3619 protected:
3620   G1CollectedHeap* _g1h;
3621   G1ParScanThreadStateSet* _per_thread_states;
3622   RefToScanQueueSet* _task_queues;
3623   TaskTerminator _terminator;
3624   uint _num_workers;
3625 
3626   void evacuate_live_objects(G1ParScanThreadState* pss,
3627                              uint worker_id,
3628                              G1GCPhaseTimes::GCParPhases objcopy_phase,
3629                              G1GCPhaseTimes::GCParPhases termination_phase) {
3630     G1GCPhaseTimes* p = _g1h->phase_times();
3631 
3632     Ticks start = Ticks::now();
3633     G1ParEvacuateFollowersClosure cl(_g1h, pss, _task_queues, _terminator.terminator(), objcopy_phase);
3634     cl.do_void();
3635 
3636     assert(pss->queue_is_empty(), "should be empty");



3637 
3638     Tickspan evac_time = (Ticks::now() - start);
3639     p->record_or_add_time_secs(objcopy_phase, worker_id, evac_time.seconds() - cl.term_time());
3640     
3641     p->record_or_add_thread_work_item(objcopy_phase, worker_id, pss->lab_waste_words() * HeapWordSize, G1GCPhaseTimes::ObjCopyLABWaste);
3642     p->record_or_add_thread_work_item(objcopy_phase, worker_id, pss->lab_undo_waste_words() * HeapWordSize, G1GCPhaseTimes::ObjCopyLABUndoWaste);










3643 
3644     if (termination_phase == G1GCPhaseTimes::Termination) {
3645       p->record_time_secs(termination_phase, worker_id, cl.term_time());
3646       p->record_thread_work_item(termination_phase, worker_id, cl.term_attempts());
3647     } else {
3648       p->record_or_add_time_secs(termination_phase, worker_id, cl.term_time());
3649       p->record_or_add_thread_work_item(termination_phase, worker_id, cl.term_attempts());
3650     }
3651     assert(pss->trim_ticks().seconds() == 0.0, "Unexpected partial trimming during evacuation");
3652   }
3653 
3654   virtual void start_work(uint worker_id) { }



3655 
3656   virtual void end_work(uint worker_id) { }




3657 
3658   virtual void scan_roots(G1ParScanThreadState* pss, uint worker_id) = 0;



3659 
3660   virtual void evacuate_live_objects(G1ParScanThreadState* pss, uint worker_id) = 0;




3661 
3662 public:
3663   G1EvacuateRegionsBaseTask(const char* name, G1ParScanThreadStateSet* per_thread_states, RefToScanQueueSet* task_queues, uint num_workers) :
3664     AbstractGangTask(name),
3665     _g1h(G1CollectedHeap::heap()),




3666     _per_thread_states(per_thread_states),
3667     _task_queues(task_queues),
3668     _terminator(num_workers, _task_queues),
3669     _num_workers(num_workers)
3670   { }
3671 
3672   void work(uint worker_id) {
3673     start_work(worker_id);
3674 
3675     {
3676       ResourceMark rm;
3677       HandleMark   hm;
3678 
3679       G1ParScanThreadState* pss = _per_thread_states->state_for_worker(worker_id);
3680       pss->set_ref_discoverer(_g1h->ref_processor_stw());
3681 
3682       scan_roots(pss, worker_id);
3683       evacuate_live_objects(pss, worker_id);
3684     }
3685 
3686     end_work(worker_id);
3687   }
3688 };
3689 
3690 class G1EvacuateRegionsTask : public G1EvacuateRegionsBaseTask {
3691   G1RootProcessor* _root_processor;
3692 
3693   void scan_roots(G1ParScanThreadState* pss, uint worker_id) {
3694     _root_processor->evacuate_roots(pss, worker_id);
3695     _g1h->rem_set()->update_rem_set(pss, worker_id);
3696     _g1h->rem_set()->scan_rem_set(pss, worker_id, G1GCPhaseTimes::ScanRS, G1GCPhaseTimes::ObjCopy, G1GCPhaseTimes::CodeRoots);
3697   }
3698 
3699   void evacuate_live_objects(G1ParScanThreadState* pss, uint worker_id) {
3700     G1EvacuateRegionsBaseTask::evacuate_live_objects(pss, worker_id, G1GCPhaseTimes::ObjCopy, G1GCPhaseTimes::Termination);
3701   }
3702 
3703   void start_work(uint worker_id) {
3704     _g1h->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerStart, worker_id, Ticks::now().seconds());
3705   }
3706 
3707   void end_work(uint worker_id) {
3708     _g1h->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerEnd, worker_id, Ticks::now().seconds());
3709   }
3710 
3711 public:
3712   G1EvacuateRegionsTask(G1CollectedHeap* g1h,
3713                         G1ParScanThreadStateSet* per_thread_states,
3714                         RefToScanQueueSet* task_queues,
3715                         G1RootProcessor* root_processor,
3716                         uint num_workers) :
3717     G1EvacuateRegionsBaseTask("G1 Evacuate Regions", per_thread_states, task_queues, num_workers),
3718     _root_processor(root_processor)
3719   { }
3720 };
3721 
3722 void G1CollectedHeap::evacuate_initial_collection_set(G1ParScanThreadStateSet* per_thread_states) {
3723   Tickspan task_time;
3724   const uint num_workers = workers()->active_workers();
3725 
3726   Ticks start_processing = Ticks::now();
3727   {
3728     G1RootProcessor root_processor(this, num_workers);
3729     G1EvacuateRegionsTask g1_par_task(this, per_thread_states, _task_queues, &root_processor, num_workers);
3730     task_time = run_task(&g1_par_task, num_workers);
3731     // Closing the inner scope will execute the destructor for the G1RootProcessor object.
3732     // To extract its code root fixup time we measure total time of this scope and
3733     // subtract from the time the WorkGang task took.
3734   }
3735   Tickspan total_processing = Ticks::now() - start_processing;
3736 
3737   G1GCPhaseTimes* p = phase_times();
3738   p->record_or_add_initial_evac_time(task_time.milliseconds());
3739   p->record_or_add_code_root_fixup_time((total_processing - task_time).milliseconds());
3740 }
3741 
3742 class G1EvacuateOptionalRegionsTask : public G1EvacuateRegionsBaseTask {
3743 
3744   void scan_roots(G1ParScanThreadState* pss, uint worker_id) {
3745     _g1h->rem_set()->scan_rem_set(pss, worker_id, G1GCPhaseTimes::OptScanRS, G1GCPhaseTimes::OptObjCopy, G1GCPhaseTimes::OptCodeRoots);
3746   }
3747 
3748   void evacuate_live_objects(G1ParScanThreadState* pss, uint worker_id) {
3749     G1EvacuateRegionsBaseTask::evacuate_live_objects(pss, worker_id, G1GCPhaseTimes::OptObjCopy, G1GCPhaseTimes::OptTermination);
3750   }
3751 
3752 public:
3753   G1EvacuateOptionalRegionsTask(G1ParScanThreadStateSet* per_thread_states,
3754                                 RefToScanQueueSet* queues,
3755                                 uint num_workers) :
3756     G1EvacuateRegionsBaseTask("G1 Evacuate Optional Regions", per_thread_states, queues, num_workers) {
3757   }
3758 };
3759 
3760 void G1CollectedHeap::evacuate_next_optional_regions(G1ParScanThreadStateSet* per_thread_states) {
3761   class G1MarkScope : public MarkScope { };
3762 
3763   Tickspan task_time;
3764 
3765   Ticks start_processing = Ticks::now();
3766   {
3767     G1MarkScope code_mark_scope;
3768     G1EvacuateOptionalRegionsTask task(per_thread_states, _task_queues, workers()->active_workers());
3769     task_time = run_task(&task);
3770     // See comment in evacuate_collection_set() for the reason of the scope.
3771   }
3772   Tickspan total_processing = Ticks::now() - start_processing;
3773 
3774   G1GCPhaseTimes* p = phase_times();
3775   p->record_or_add_initial_evac_time(task_time.milliseconds());
3776   p->record_or_add_code_root_fixup_time((total_processing - task_time).milliseconds());
3777 }
3778 
3779 void G1CollectedHeap::evacuate_optional_collection_set(G1ParScanThreadStateSet* per_thread_states) {
3780   const double gc_start_time_ms = phase_times()->cur_collection_start_sec() * 1000.0;
3781 
3782   Ticks start = Ticks::now();
3783 
3784   while (!evacuation_failed() && _collection_set.optional_region_length() > 0) {
3785 

3786     double time_used_ms = os::elapsedTime() * 1000.0 - gc_start_time_ms;
3787     double time_left_ms = MaxGCPauseMillis - time_used_ms;
3788 
3789     if (time_left_ms < 0 ||
3790         !_collection_set.finalize_optional_for_evacuation(time_left_ms * policy()->optional_evacuation_fraction())) {
3791       log_trace(gc, ergo, cset)("Skipping evacuation of %u optional regions, no more regions can be evacuated in %.3fms",
3792                                 _collection_set.optional_region_length(), time_left_ms);
3793       break;
3794     }
3795 
3796     evacuate_next_optional_regions(per_thread_states);



3797   }
3798 
3799   _collection_set.abandon_optional_collection_set(per_thread_states);
3800 
3801   phase_times()->record_optional_evacuation((Ticks::now() - start).milliseconds());






3802 }
3803 
3804 void G1CollectedHeap::post_evacuate_collection_set(G1EvacuationInfo& evacuation_info, G1ParScanThreadStateSet* per_thread_states) {
3805   // Also cleans the card table from temporary duplicate detection information used
3806   // during UpdateRS/ScanRS.
3807   rem_set()->cleanup_after_oops_into_collection_set_do();
3808 
3809   // Process any discovered reference objects - we have
3810   // to do this _before_ we retire the GC alloc regions
3811   // as we may have to copy some 'reachable' referent
3812   // objects (and their reachable sub-graphs) that were
3813   // not copied during the pause.
3814   process_discovered_references(per_thread_states);
3815 
3816   G1STWIsAliveClosure is_alive(this);
3817   G1KeepAliveClosure keep_alive(this);
3818 
3819   WeakProcessor::weak_oops_do(workers(), &is_alive, &keep_alive,
3820                               phase_times()->weak_phase_times());
3821 


4196         }
4197         start_time = end_time;
4198       }
4199     }
4200 
4201     if (has_young_time) {
4202       timer->record_time_secs(G1GCPhaseTimes::YoungFreeCSet, worker_id, young_time);
4203     }
4204     if (has_non_young_time) {
4205       timer->record_time_secs(G1GCPhaseTimes::NonYoungFreeCSet, worker_id, non_young_time);
4206     }
4207   }
4208 };
4209 
4210 void G1CollectedHeap::free_collection_set(G1CollectionSet* collection_set, G1EvacuationInfo& evacuation_info, const size_t* surviving_young_words) {
4211   _eden.clear();
4212 
4213   double free_cset_start_time = os::elapsedTime();
4214 
4215   {
4216     uint const num_regions = _collection_set.region_length();
4217     uint const num_chunks = MAX2(num_regions / G1FreeCollectionSetTask::chunk_size(), 1U);
4218     uint const num_workers = MIN2(workers()->active_workers(), num_chunks);
4219 
4220     G1FreeCollectionSetTask cl(collection_set, &evacuation_info, surviving_young_words);
4221 
4222     log_debug(gc, ergo)("Running %s using %u workers for collection set length %u",
4223                         cl.name(), num_workers, num_regions);


4224     workers()->run_task(&cl, num_workers);
4225   }
4226   phase_times()->record_total_free_cset_time_ms((os::elapsedTime() - free_cset_start_time) * 1000.0);
4227 
4228   collection_set->clear();
4229 }
4230 
4231 class G1FreeHumongousRegionClosure : public HeapRegionClosure {
4232  private:
4233   FreeRegionList* _free_region_list;
4234   HeapRegionSet* _proxy_set;
4235   uint _humongous_objects_reclaimed;
4236   uint _humongous_regions_reclaimed;
4237   size_t _freed_bytes;
4238  public:
4239 
4240   G1FreeHumongousRegionClosure(FreeRegionList* free_region_list) :
4241     _free_region_list(free_region_list), _proxy_set(NULL), _humongous_objects_reclaimed(0), _humongous_regions_reclaimed(0), _freed_bytes(0) {
4242   }
4243 


4372 
4373   prepend_to_freelist(&local_cleanup_list);
4374   decrement_summary_bytes(cl.bytes_freed());
4375 
4376   phase_times()->record_fast_reclaim_humongous_time_ms((os::elapsedTime() - start_time) * 1000.0,
4377                                                        cl.humongous_objects_reclaimed());
4378 }
4379 
4380 class G1AbandonCollectionSetClosure : public HeapRegionClosure {
4381 public:
4382   virtual bool do_heap_region(HeapRegion* r) {
4383     assert(r->in_collection_set(), "Region %u must have been in collection set", r->hrm_index());
4384     G1CollectedHeap::heap()->clear_in_cset(r);
4385     r->set_young_index_in_cset(-1);
4386     return false;
4387   }
4388 };
4389 
4390 void G1CollectedHeap::abandon_collection_set(G1CollectionSet* collection_set) {
4391   G1AbandonCollectionSetClosure cl;
4392   collection_set_iterate(&cl);
4393 
4394   collection_set->clear();
4395   collection_set->stop_incremental_building();
4396 }
4397 
4398 bool G1CollectedHeap::is_old_gc_alloc_region(HeapRegion* hr) {
4399   return _allocator->is_retained_old_region(hr);
4400 }
4401 
4402 void G1CollectedHeap::set_region_short_lived_locked(HeapRegion* hr) {
4403   _eden.add(hr);
4404   _policy->set_region_eden(hr);
4405 }
4406 
4407 #ifdef ASSERT
4408 
4409 class NoYoungRegionsClosure: public HeapRegionClosure {
4410 private:
4411   bool _success;
4412 public:


< prev index next >