< prev index next >

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

Print this page




  81 #include "gc/shared/referenceProcessor.inline.hpp"
  82 #include "gc/shared/taskqueue.inline.hpp"
  83 #include "gc/shared/weakProcessor.inline.hpp"
  84 #include "gc/shared/workerPolicy.hpp"
  85 #include "logging/log.hpp"
  86 #include "memory/allocation.hpp"
  87 #include "memory/iterator.hpp"
  88 #include "memory/resourceArea.hpp"
  89 #include "memory/universe.hpp"
  90 #include "oops/access.inline.hpp"
  91 #include "oops/compressedOops.inline.hpp"
  92 #include "oops/oop.inline.hpp"
  93 #include "runtime/atomic.hpp"
  94 #include "runtime/flags/flagSetting.hpp"
  95 #include "runtime/handles.inline.hpp"
  96 #include "runtime/init.hpp"
  97 #include "runtime/orderAccess.hpp"
  98 #include "runtime/threadSMR.hpp"
  99 #include "runtime/vmThread.hpp"
 100 #include "utilities/align.hpp"

 101 #include "utilities/globalDefinitions.hpp"
 102 #include "utilities/stack.inline.hpp"
 103 
 104 size_t G1CollectedHeap::_humongous_object_threshold_in_words = 0;
 105 
 106 // INVARIANTS/NOTES
 107 //
 108 // All allocation activity covered by the G1CollectedHeap interface is
 109 // serialized by acquiring the HeapLock.  This happens in mem_allocate
 110 // and allocate_new_tlab, which are the "entry" points to the
 111 // allocation code from the rest of the JVM.  (Note that this does not
 112 // apply to TLAB allocation, which is not part of this interface: it
 113 // is done by clients of this interface.)
 114 
 115 class RedirtyLoggedCardTableEntryClosure : public G1CardTableEntryClosure {
 116  private:
 117   size_t _num_dirtied;
 118   G1CollectedHeap* _g1h;
 119   G1CardTable* _g1_ct;
 120 


2349 
2350 void G1CollectedHeap::heap_region_iterate(HeapRegionClosure* cl) const {
2351   _hrm->iterate(cl);
2352 }
2353 
2354 void G1CollectedHeap::heap_region_par_iterate_from_worker_offset(HeapRegionClosure* cl,
2355                                                                  HeapRegionClaimer *hrclaimer,
2356                                                                  uint worker_id) const {
2357   _hrm->par_iterate(cl, hrclaimer, hrclaimer->offset_for_worker(worker_id));
2358 }
2359 
2360 void G1CollectedHeap::heap_region_par_iterate_from_start(HeapRegionClosure* cl,
2361                                                          HeapRegionClaimer *hrclaimer) const {
2362   _hrm->par_iterate(cl, hrclaimer, 0);
2363 }
2364 
2365 void G1CollectedHeap::collection_set_iterate_all(HeapRegionClosure* cl) {
2366   _collection_set.iterate(cl);
2367 }
2368 




2369 void G1CollectedHeap::collection_set_iterate_increment_from(HeapRegionClosure *cl, HeapRegionClaimer* hr_claimer, uint worker_id) {
2370   _collection_set.iterate_incremental_part_from(cl, hr_claimer, worker_id, workers()->active_workers());
2371 }
2372 
2373 HeapWord* G1CollectedHeap::block_start(const void* addr) const {
2374   HeapRegion* hr = heap_region_containing(addr);
2375   return hr->block_start(addr);
2376 }
2377 
2378 bool G1CollectedHeap::block_is_obj(const HeapWord* addr) const {
2379   HeapRegion* hr = heap_region_containing(addr);
2380   return hr->block_is_obj(addr);
2381 }
2382 
2383 bool G1CollectedHeap::supports_tlab_allocation() const {
2384   return true;
2385 }
2386 
2387 size_t G1CollectedHeap::tlab_capacity(Thread* ignored) const {
2388   return (_policy->young_list_target_length() - _survivor.length()) * HeapRegion::GrainBytes;


4062   phase_times()->record_derived_pointer_table_update_time((os::elapsedTime() - start) * 1000.0);
4063 #endif
4064   policy()->print_age_table();
4065 }
4066 
4067 void G1CollectedHeap::record_obj_copy_mem_stats() {
4068   policy()->add_bytes_allocated_in_old_since_last_gc(_old_evac_stats.allocated() * HeapWordSize);
4069 
4070   _gc_tracer_stw->report_evacuation_statistics(create_g1_evac_summary(&_survivor_evac_stats),
4071                                                create_g1_evac_summary(&_old_evac_stats));
4072 }
4073 
4074 void G1CollectedHeap::free_region(HeapRegion* hr,
4075                                   FreeRegionList* free_list,
4076                                   bool skip_remset,
4077                                   bool skip_hot_card_cache,
4078                                   bool locked) {
4079   assert(!hr->is_free(), "the region should not be free");
4080   assert(!hr->is_empty(), "the region should not be empty");
4081   assert(_hrm->is_available(hr->hrm_index()), "region should be committed");
4082   assert(free_list != NULL, "pre-condition");
4083 
4084   if (G1VerifyBitmaps) {
4085     MemRegion mr(hr->bottom(), hr->end());
4086     concurrent_mark()->clear_range_in_prev_bitmap(mr);
4087   }
4088 
4089   // Clear the card counts for this region.
4090   // Note: we only need to do this if the region is not young
4091   // (since we don't refine cards in young regions).
4092   if (!skip_hot_card_cache && !hr->is_young()) {
4093     _hot_card_cache->reset_card_counts(hr);
4094   }
4095   hr->hr_clear(skip_remset, true /* clear_space */, locked /* locked */);
4096   _policy->remset_tracker()->update_at_free(hr);

4097   free_list->add_ordered(hr);

4098 }
4099 
4100 void G1CollectedHeap::free_humongous_region(HeapRegion* hr,
4101                                             FreeRegionList* free_list) {
4102   assert(hr->is_humongous(), "this is only for humongous regions");
4103   assert(free_list != NULL, "pre-condition");
4104   hr->clear_humongous();
4105   free_region(hr, free_list, false /* skip_remset */, false /* skip_hcc */, true /* locked */);
4106 }
4107 
4108 void G1CollectedHeap::remove_from_old_sets(const uint old_regions_removed,
4109                                            const uint humongous_regions_removed) {
4110   if (old_regions_removed > 0 || humongous_regions_removed > 0) {
4111     MutexLocker x(OldSets_lock, Mutex::_no_safepoint_check_flag);
4112     _old_set.bulk_remove(old_regions_removed);
4113     _humongous_set.bulk_remove(humongous_regions_removed);
4114   }
4115 
4116 }
4117 





4118 void G1CollectedHeap::prepend_to_freelist(FreeRegionList* list) {
4119   assert(list != NULL, "list can't be null");
4120   if (!list->is_empty()) {
4121     MutexLocker x(FreeList_lock, Mutex::_no_safepoint_check_flag);
4122     _hrm->insert_list_into_free_list(list);
4123   }
4124 }
4125 








4126 void G1CollectedHeap::decrement_summary_bytes(size_t bytes) {
4127   decrease_used(bytes);
4128 }
4129 
4130 class G1FreeCollectionSetTask : public AbstractGangTask {
4131 private:
4132 
4133   // Closure applied to all regions in the collection set to do work that needs to
4134   // be done serially in a single thread.
4135   class G1SerialFreeCollectionSetClosure : public HeapRegionClosure {
4136   private:
4137     G1EvacuationInfo* _evacuation_info;
4138     const size_t* _surviving_young_words;
4139 
4140     // Bytes used in successfully evacuated regions before the evacuation.
4141     size_t _before_used_bytes;
4142     // Bytes used in unsucessfully evacuated regions before the evacuation
4143     size_t _after_used_bytes;
4144 
4145     size_t _bytes_allocated_in_old_since_last_gc;
4146 
4147     size_t _failure_used_words;
4148     size_t _failure_waste_words;
4149 
4150     FreeRegionList _local_free_list;
4151   public:
4152     G1SerialFreeCollectionSetClosure(G1EvacuationInfo* evacuation_info, const size_t* surviving_young_words) :
4153       HeapRegionClosure(),
4154       _evacuation_info(evacuation_info),
4155       _surviving_young_words(surviving_young_words),
4156       _before_used_bytes(0),
4157       _after_used_bytes(0),
4158       _bytes_allocated_in_old_since_last_gc(0),
4159       _failure_used_words(0),
4160       _failure_waste_words(0),
4161       _local_free_list("Local Region List for CSet Freeing") {
4162     }
4163 
4164     virtual bool do_heap_region(HeapRegion* r) {
4165       G1CollectedHeap* g1h = G1CollectedHeap::heap();
4166 
4167       assert(r->in_collection_set(), "Region %u should be in collection set.", r->hrm_index());
4168       g1h->clear_region_attr(r);












4169 
4170       if (r->is_young()) {
4171         assert(r->young_index_in_cset() != 0 && (uint)r->young_index_in_cset() <= g1h->collection_set()->young_region_length(),
4172                "Young index %u is wrong for region %u of type %s with %u young regions",
4173                r->young_index_in_cset(),
4174                r->hrm_index(),
4175                r->get_type_str(),
4176                g1h->collection_set()->young_region_length());
4177         size_t words_survived = _surviving_young_words[r->young_index_in_cset()];
4178         r->record_surv_words_in_group(words_survived);
4179       }
4180 
4181       if (!r->evacuation_failed()) {
4182         assert(!r->is_empty(), "Region %u is an empty region in the collection set.", r->hrm_index());
4183         _before_used_bytes += r->used();
4184         g1h->free_region(r,
4185                          &_local_free_list,
4186                          true, /* skip_remset */
4187                          true, /* skip_hot_card_cache */
4188                          true  /* locked */);
4189       } else {
4190         r->uninstall_surv_rate_group();
4191         r->clear_young_index_in_cset();
4192         r->set_evacuation_failed(false);
4193         // When moving a young gen region to old gen, we "allocate" that whole region
4194         // there. This is in addition to any already evacuated objects. Notify the
4195         // policy about that.
4196         // Old gen regions do not cause an additional allocation: both the objects
4197         // still in the region and the ones already moved are accounted for elsewhere.
4198         if (r->is_young()) {
4199           _bytes_allocated_in_old_since_last_gc += HeapRegion::GrainBytes;
4200         }
4201         // The region is now considered to be old.
4202         r->set_old();
4203         // Do some allocation statistics accounting. Regions that failed evacuation
4204         // are always made old, so there is no need to update anything in the young
4205         // gen statistics, but we need to update old gen statistics.
4206         size_t used_words = r->marked_bytes() / HeapWordSize;
4207 
4208         _failure_used_words += used_words;
4209         _failure_waste_words += HeapRegion::GrainWords - used_words;






4210 
4211         g1h->old_set_add(r);
4212         _after_used_bytes += r->used();











4213       }
4214       return false;
4215     }
4216 
4217     void complete_work() {
4218       G1CollectedHeap* g1h = G1CollectedHeap::heap();
4219 
4220       _evacuation_info->set_regions_freed(_local_free_list.length());
4221       _evacuation_info->increment_collectionset_used_after(_after_used_bytes);
4222 
4223       g1h->prepend_to_freelist(&_local_free_list);
4224       g1h->decrement_summary_bytes(_before_used_bytes);
4225 
4226       G1Policy* policy = g1h->policy();
4227       policy->add_bytes_allocated_in_old_since_last_gc(_bytes_allocated_in_old_since_last_gc);
4228 
4229       g1h->alloc_buffer_stats(G1HeapRegionAttr::Old)->add_failure_used_and_waste(_failure_used_words, _failure_waste_words);







4230     }
4231   };
4232 
4233   G1CollectionSet* _collection_set;
4234   G1SerialFreeCollectionSetClosure _cl;
4235   const size_t* _surviving_young_words;











4236 
4237   size_t _rs_length;




4238 
4239   volatile jint _serial_work_claim;



4240 
4241   struct WorkItem {
4242     uint region_idx;
4243     bool is_young;
4244     bool evacuation_failed;
4245 
4246     WorkItem(HeapRegion* r) {
4247       region_idx = r->hrm_index();
4248       is_young = r->is_young();
4249       evacuation_failed = r->evacuation_failed();






4250     }
4251   };
4252 
4253   volatile size_t _parallel_work_claim;
4254   size_t _num_work_items;
4255   WorkItem* _work_items;
4256 
4257   void do_serial_work() {
4258     // Need to grab the lock to be allowed to modify the old region list.
4259     MutexLocker x(OldSets_lock, Mutex::_no_safepoint_check_flag);
4260     _collection_set->iterate(&_cl);
4261   }
4262 
4263   void do_parallel_work_for_region(uint region_idx, bool is_young, bool evacuation_failed) {
4264     G1CollectedHeap* g1h = G1CollectedHeap::heap();












4265 
4266     HeapRegion* r = g1h->region_at(region_idx);
4267     assert(!g1h->is_on_master_free_list(r), "sanity");


4268 
4269     Atomic::add(&_rs_length, r->rem_set()->occupied_locked());

4270 
4271     if (!is_young) {
4272       g1h->hot_card_cache()->reset_card_counts(r);



4273     }
4274 
4275     if (!evacuation_failed) {
4276       r->rem_set()->clear_locked();


4277     }



4278   }
4279 
4280   class G1PrepareFreeCollectionSetClosure : public HeapRegionClosure {
4281   private:
4282     size_t _cur_idx;
4283     WorkItem* _work_items;
4284   public:
4285     G1PrepareFreeCollectionSetClosure(WorkItem* work_items) : HeapRegionClosure(), _cur_idx(0), _work_items(work_items) { }
4286 
4287     virtual bool do_heap_region(HeapRegion* r) {
4288       _work_items[_cur_idx++] = WorkItem(r);
4289       return false;






4290     }
4291   };
4292 
4293   void prepare_work() {
4294     G1PrepareFreeCollectionSetClosure cl(_work_items);
4295     _collection_set->iterate(&cl);
4296   }



4297 
4298   void complete_work() {
4299     _cl.complete_work();

4300 
4301     G1Policy* policy = G1CollectedHeap::heap()->policy();
4302     policy->record_rs_length(_rs_length);
4303     policy->cset_regions_freed();




4304   }

4305 public:
4306   G1FreeCollectionSetTask(G1CollectionSet* collection_set, G1EvacuationInfo* evacuation_info, const size_t* surviving_young_words) :
4307     AbstractGangTask("G1 Free Collection Set"),
4308     _collection_set(collection_set),
4309     _cl(evacuation_info, surviving_young_words),


4310     _surviving_young_words(surviving_young_words),
4311     _rs_length(0),
4312     _serial_work_claim(0),
4313     _parallel_work_claim(0),
4314     _num_work_items(collection_set->region_length()),
4315     _work_items(NEW_C_HEAP_ARRAY(WorkItem, _num_work_items, mtGC)) {
4316     prepare_work();
4317   }
4318 
4319   ~G1FreeCollectionSetTask() {
4320     complete_work();
4321     FREE_C_HEAP_ARRAY(WorkItem, _work_items);


4322   }
4323 
4324   // Chunk size for work distribution. The chosen value has been determined experimentally
4325   // to be a good tradeoff between overhead and achievable parallelism.
4326   static uint chunk_size() { return 32; }
4327 
4328   virtual void work(uint worker_id) {
4329     G1GCPhaseTimes* timer = G1CollectedHeap::heap()->phase_times();



4330 
4331     // Claim serial work.
4332     if (_serial_work_claim == 0) {
4333       jint value = Atomic::add(&_serial_work_claim, 1) - 1;
4334       if (value == 0) {
4335         double serial_time = os::elapsedTime();
4336         do_serial_work();
4337         timer->record_serial_free_cset_time_ms((os::elapsedTime() - serial_time) * 1000.0);
4338       }
4339     }

4340 
4341     // Start parallel work.
4342     double young_time = 0.0;
4343     bool has_young_time = false;
4344     double non_young_time = 0.0;
4345     bool has_non_young_time = false;
4346 
4347     while (true) {
4348       size_t end = Atomic::add(&_parallel_work_claim, chunk_size());
4349       size_t cur = end - chunk_size();
4350 
4351       if (cur >= _num_work_items) {
4352         break;

















4353       }
4354 
4355       EventGCPhaseParallel event;
4356       double start_time = os::elapsedTime();




4357 
4358       end = MIN2(end, _num_work_items);




4359 
4360       for (; cur < end; cur++) {
4361         bool is_young = _work_items[cur].is_young;
4362 
4363         do_parallel_work_for_region(_work_items[cur].region_idx, is_young, _work_items[cur].evacuation_failed);



4364 
4365         double end_time = os::elapsedTime();
4366         double time_taken = end_time - start_time;
4367         if (is_young) {
4368           young_time += time_taken;
4369           has_young_time = true;
4370           event.commit(GCId::current(), worker_id, G1GCPhaseTimes::phase_name(G1GCPhaseTimes::YoungFreeCSet));
4371         } else {
4372           non_young_time += time_taken;
4373           has_non_young_time = true;
4374           event.commit(GCId::current(), worker_id, G1GCPhaseTimes::phase_name(G1GCPhaseTimes::NonYoungFreeCSet));
4375         }
4376         start_time = end_time;



4377       }
4378     }
4379 
4380     if (has_young_time) {
4381       timer->record_time_secs(G1GCPhaseTimes::YoungFreeCSet, worker_id, young_time);
4382     }
4383     if (has_non_young_time) {
4384       timer->record_time_secs(G1GCPhaseTimes::NonYoungFreeCSet, worker_id, non_young_time);
4385     }
4386   }
4387 };
4388 
4389 void G1CollectedHeap::free_collection_set(G1CollectionSet* collection_set, G1EvacuationInfo& evacuation_info, const size_t* surviving_young_words) {
4390   _eden.clear();
4391 
4392   double free_cset_start_time = os::elapsedTime();



4393 

4394   {
4395     uint const num_regions = _collection_set.region_length();
4396     uint const num_chunks = MAX2(num_regions / G1FreeCollectionSetTask::chunk_size(), 1U);
4397     uint const num_workers = MIN2(workers()->active_workers(), num_chunks);
4398 
4399     G1FreeCollectionSetTask cl(collection_set, &evacuation_info, surviving_young_words);









4400 
4401     log_debug(gc, ergo)("Running %s using %u workers for collection set length %u",
4402                         cl.name(), num_workers, num_regions);
4403     workers()->run_task(&cl, num_workers);
4404   }
4405   phase_times()->record_total_free_cset_time_ms((os::elapsedTime() - free_cset_start_time) * 1000.0);
4406 
4407   collection_set->clear();
4408 }
4409 
4410 class G1FreeHumongousRegionClosure : public HeapRegionClosure {
4411  private:
4412   FreeRegionList* _free_region_list;
4413   HeapRegionSet* _proxy_set;
4414   uint _humongous_objects_reclaimed;
4415   uint _humongous_regions_reclaimed;
4416   size_t _freed_bytes;
4417  public:
4418 
4419   G1FreeHumongousRegionClosure(FreeRegionList* free_region_list) :
4420     _free_region_list(free_region_list), _proxy_set(NULL), _humongous_objects_reclaimed(0), _humongous_regions_reclaimed(0), _freed_bytes(0) {
4421   }
4422 
4423   virtual bool do_heap_region(HeapRegion* r) {
4424     if (!r->is_starts_humongous()) {
4425       return false;




  81 #include "gc/shared/referenceProcessor.inline.hpp"
  82 #include "gc/shared/taskqueue.inline.hpp"
  83 #include "gc/shared/weakProcessor.inline.hpp"
  84 #include "gc/shared/workerPolicy.hpp"
  85 #include "logging/log.hpp"
  86 #include "memory/allocation.hpp"
  87 #include "memory/iterator.hpp"
  88 #include "memory/resourceArea.hpp"
  89 #include "memory/universe.hpp"
  90 #include "oops/access.inline.hpp"
  91 #include "oops/compressedOops.inline.hpp"
  92 #include "oops/oop.inline.hpp"
  93 #include "runtime/atomic.hpp"
  94 #include "runtime/flags/flagSetting.hpp"
  95 #include "runtime/handles.inline.hpp"
  96 #include "runtime/init.hpp"
  97 #include "runtime/orderAccess.hpp"
  98 #include "runtime/threadSMR.hpp"
  99 #include "runtime/vmThread.hpp"
 100 #include "utilities/align.hpp"
 101 #include "utilities/bitMap.inline.hpp"
 102 #include "utilities/globalDefinitions.hpp"
 103 #include "utilities/stack.inline.hpp"
 104 
 105 size_t G1CollectedHeap::_humongous_object_threshold_in_words = 0;
 106 
 107 // INVARIANTS/NOTES
 108 //
 109 // All allocation activity covered by the G1CollectedHeap interface is
 110 // serialized by acquiring the HeapLock.  This happens in mem_allocate
 111 // and allocate_new_tlab, which are the "entry" points to the
 112 // allocation code from the rest of the JVM.  (Note that this does not
 113 // apply to TLAB allocation, which is not part of this interface: it
 114 // is done by clients of this interface.)
 115 
 116 class RedirtyLoggedCardTableEntryClosure : public G1CardTableEntryClosure {
 117  private:
 118   size_t _num_dirtied;
 119   G1CollectedHeap* _g1h;
 120   G1CardTable* _g1_ct;
 121 


2350 
2351 void G1CollectedHeap::heap_region_iterate(HeapRegionClosure* cl) const {
2352   _hrm->iterate(cl);
2353 }
2354 
2355 void G1CollectedHeap::heap_region_par_iterate_from_worker_offset(HeapRegionClosure* cl,
2356                                                                  HeapRegionClaimer *hrclaimer,
2357                                                                  uint worker_id) const {
2358   _hrm->par_iterate(cl, hrclaimer, hrclaimer->offset_for_worker(worker_id));
2359 }
2360 
2361 void G1CollectedHeap::heap_region_par_iterate_from_start(HeapRegionClosure* cl,
2362                                                          HeapRegionClaimer *hrclaimer) const {
2363   _hrm->par_iterate(cl, hrclaimer, 0);
2364 }
2365 
2366 void G1CollectedHeap::collection_set_iterate_all(HeapRegionClosure* cl) {
2367   _collection_set.iterate(cl);
2368 }
2369 
2370 void G1CollectedHeap::collection_set_par_iterate_all(HeapRegionClosure* cl, HeapRegionClaimer* hr_claimer, uint worker_id) {
2371   _collection_set.par_iterate(cl, hr_claimer, worker_id, workers()->active_workers());
2372 }
2373 
2374 void G1CollectedHeap::collection_set_iterate_increment_from(HeapRegionClosure *cl, HeapRegionClaimer* hr_claimer, uint worker_id) {
2375   _collection_set.iterate_incremental_part_from(cl, hr_claimer, worker_id, workers()->active_workers());
2376 }
2377 
2378 HeapWord* G1CollectedHeap::block_start(const void* addr) const {
2379   HeapRegion* hr = heap_region_containing(addr);
2380   return hr->block_start(addr);
2381 }
2382 
2383 bool G1CollectedHeap::block_is_obj(const HeapWord* addr) const {
2384   HeapRegion* hr = heap_region_containing(addr);
2385   return hr->block_is_obj(addr);
2386 }
2387 
2388 bool G1CollectedHeap::supports_tlab_allocation() const {
2389   return true;
2390 }
2391 
2392 size_t G1CollectedHeap::tlab_capacity(Thread* ignored) const {
2393   return (_policy->young_list_target_length() - _survivor.length()) * HeapRegion::GrainBytes;


4067   phase_times()->record_derived_pointer_table_update_time((os::elapsedTime() - start) * 1000.0);
4068 #endif
4069   policy()->print_age_table();
4070 }
4071 
4072 void G1CollectedHeap::record_obj_copy_mem_stats() {
4073   policy()->add_bytes_allocated_in_old_since_last_gc(_old_evac_stats.allocated() * HeapWordSize);
4074 
4075   _gc_tracer_stw->report_evacuation_statistics(create_g1_evac_summary(&_survivor_evac_stats),
4076                                                create_g1_evac_summary(&_old_evac_stats));
4077 }
4078 
4079 void G1CollectedHeap::free_region(HeapRegion* hr,
4080                                   FreeRegionList* free_list,
4081                                   bool skip_remset,
4082                                   bool skip_hot_card_cache,
4083                                   bool locked) {
4084   assert(!hr->is_free(), "the region should not be free");
4085   assert(!hr->is_empty(), "the region should not be empty");
4086   assert(_hrm->is_available(hr->hrm_index()), "region should be committed");

4087 
4088   if (G1VerifyBitmaps) {
4089     MemRegion mr(hr->bottom(), hr->end());
4090     concurrent_mark()->clear_range_in_prev_bitmap(mr);
4091   }
4092 
4093   // Clear the card counts for this region.
4094   // Note: we only need to do this if the region is not young
4095   // (since we don't refine cards in young regions).
4096   if (!skip_hot_card_cache && !hr->is_young()) {
4097     _hot_card_cache->reset_card_counts(hr);
4098   }
4099   hr->hr_clear(skip_remset, true /* clear_space */, locked /* locked */);
4100   _policy->remset_tracker()->update_at_free(hr);
4101   if (free_list != NULL) {
4102     free_list->add_ordered(hr);
4103   }
4104 }
4105 
4106 void G1CollectedHeap::free_humongous_region(HeapRegion* hr,
4107                                             FreeRegionList* free_list) {
4108   assert(hr->is_humongous(), "this is only for humongous regions");
4109   assert(free_list != NULL, "pre-condition");
4110   hr->clear_humongous();
4111   free_region(hr, free_list, false /* skip_remset */, false /* skip_hcc */, true /* locked */);
4112 }
4113 
4114 void G1CollectedHeap::remove_from_old_sets(const uint old_regions_removed,
4115                                            const uint humongous_regions_removed) {
4116   if (old_regions_removed > 0 || humongous_regions_removed > 0) {
4117     MutexLocker x(OldSets_lock, Mutex::_no_safepoint_check_flag);
4118     _old_set.bulk_remove(old_regions_removed);
4119     _humongous_set.bulk_remove(humongous_regions_removed);
4120   }
4121 
4122 }
4123 
4124 void G1CollectedHeap::abandon_freelist() {
4125   MutexLocker x(FreeList_lock, Mutex::_no_safepoint_check_flag);
4126   _hrm->abandon_free_list();
4127 }
4128 
4129 void G1CollectedHeap::prepend_to_freelist(FreeRegionList* list) {
4130   assert(list != NULL, "list can't be null");
4131   if (!list->is_empty()) {
4132     MutexLocker x(FreeList_lock, Mutex::_no_safepoint_check_flag);
4133     _hrm->insert_list_into_free_list(list);
4134   }
4135 }
4136 
4137 void G1CollectedHeap::append_to_freelist(FreeRegionList* list) {
4138   assert(list != NULL, "list can't be null");
4139   if (!list->is_empty()) {
4140     MutexLocker x(FreeList_lock, Mutex::_no_safepoint_check_flag);
4141     _hrm->append_to_free_list(list);
4142   }
4143 }
4144 
4145 void G1CollectedHeap::decrement_summary_bytes(size_t bytes) {
4146   decrease_used(bytes);
4147 }
4148 
4149 class G1FreeCollectionSetTask : public AbstractGangTask {
4150   // Helper class to keep statistics for the collection set freeing
4151   class FreeCSetStats {
4152     size_t _before_used_bytes;   // Usage in regions successfully evacutate
4153     size_t _after_used_bytes;    // Usage in regions failing evacuation
4154     size_t _bytes_allocated_in_old_since_last_gc; // Size of young regions turned into old
4155     size_t _failure_used_words;  // Live size in failed regions
4156     size_t _failure_waste_words; // Wasted size in failed regions
4157     size_t _rs_length;           // Remembered set size
4158     uint _regions_freed;         // Number of regions freed











4159   public:
4160     FreeCSetStats() :



4161         _before_used_bytes(0),
4162         _after_used_bytes(0),
4163         _bytes_allocated_in_old_since_last_gc(0),
4164         _failure_used_words(0),
4165         _failure_waste_words(0),
4166         _rs_length(0),
4167         _regions_freed(0) { }



4168 
4169     void merge_stats(FreeCSetStats* other) {
4170       assert(other != NULL, "invariant");
4171       _before_used_bytes += other->_before_used_bytes;
4172       _after_used_bytes += other->_after_used_bytes;
4173       _bytes_allocated_in_old_since_last_gc += other->_bytes_allocated_in_old_since_last_gc;
4174       _failure_used_words += other->_failure_used_words;
4175       _failure_waste_words += other->_failure_waste_words;
4176       _rs_length += other->_rs_length;
4177       _regions_freed += other->_regions_freed;
4178     }
4179 
4180     void report(G1CollectedHeap* g1h, G1EvacuationInfo* evacuation_info) {
4181       evacuation_info->set_regions_freed(_regions_freed);
4182       evacuation_info->increment_collectionset_used_after(_after_used_bytes);
4183 
4184       g1h->decrement_summary_bytes(_before_used_bytes);
4185       g1h->alloc_buffer_stats(G1HeapRegionAttr::Old)->add_failure_used_and_waste(_failure_used_words, _failure_waste_words);








4186 
4187       G1Policy *policy = g1h->policy();
4188       policy->add_bytes_allocated_in_old_since_last_gc(_bytes_allocated_in_old_since_last_gc);
4189       policy->record_rs_length(_rs_length);
4190       policy->cset_regions_freed();















4191     }






4192 
4193     void update_used_before(size_t value) { _before_used_bytes += value; }
4194     void update_used_after(size_t value) { _after_used_bytes += value; }
4195     void update_bytes_alloc_in_old(size_t value) { _bytes_allocated_in_old_since_last_gc += value; }
4196     void update_failure_used(size_t value) { _failure_used_words += value; }
4197     void update_failure_waste(size_t value) { _failure_waste_words += value; }
4198     void update_rs_length(size_t value) { _rs_length += value; }
4199     void increment_regions_freed() { _regions_freed += 1; }
4200   };
4201 
4202   // Closure applied to all regions in the collection set.
4203   class G1FreeCollectionSetClosure : public HeapRegionClosure {
4204     // Helper to send JFR events for regions.
4205     class EventForRegion {
4206       EventGCPhaseParallel _event;
4207     public:
4208       EventForRegion(HeapRegion* region, uint worker_id) : _event() {
4209         _event.set_gcId(GCId::current());
4210         _event.set_gcWorkerId(worker_id);
4211         if (region->is_young()) {
4212           _event.set_name(G1GCPhaseTimes::phase_name(G1GCPhaseTimes::YoungFreeCSet));
4213         } else {
4214           _event.set_name(G1GCPhaseTimes::phase_name(G1GCPhaseTimes::NonYoungFreeCSet));
4215         }

4216       }
4217 
4218       ~EventForRegion() {
4219         _event.commit();
4220       }
4221     };







4222 
4223     // Helper to do timing for region work.
4224     class TimerForRegion {
4225       double& _time;
4226       double  _start_time;
4227     public:
4228       TimerForRegion(double& time) : _time(time), _start_time(os::elapsedTime()) { }
4229       ~TimerForRegion() {
4230         _time += os::elapsedTime() - _start_time;
4231       }
4232     };
4233 
4234     // G1FreeCollectionSetClosure members
4235     G1CollectedHeap* _g1h;
4236     const size_t*    _surviving_young_words;
4237     uint             _worker_id;
4238     double           _young_time;
4239     double           _non_young_time;
4240     FreeCSetStats*   _stats;
4241 
4242     void assert_in_cset(HeapRegion* r) {
4243       assert(r->young_index_in_cset() != 0 &&
4244              (uint)r->young_index_in_cset() <= _g1h->collection_set()->young_region_length(),
4245              "Young index %u is wrong for region %u of type %s with %u young regions",
4246              r->young_index_in_cset(), r->hrm_index(), r->get_type_str(), _g1h->collection_set()->young_region_length());
4247     }
4248 
4249     void handle_evacuated_region(HeapRegion* r) {
4250       assert(!r->is_empty(), "Region %u is an empty region in the collection set.", r->hrm_index());
4251       // Update stats.
4252       stats()->update_used_before(r->used());
4253       stats()->increment_regions_freed();
4254 
4255       // Free the region and mark it in the bit map so it gets added to the new free list.
4256       r->rem_set()->clear_locked();
4257       _g1h->free_region(r, NULL, true /* skip_remset */, true /* skip_hot_card_cache */, true  /* locked */);
4258     }
4259 
4260     void handle_failed_region(HeapRegion* r) {
4261       // Do some allocation statistics accounting. Regions that failed evacuation
4262       // are always made old, so there is no need to update anything in the young
4263       // gen statistics, but we need to update old gen statistics.
4264       size_t used_words = r->marked_bytes() / HeapWordSize;
4265       stats()->update_failure_used(used_words);
4266       stats()->update_failure_waste(HeapRegion::GrainWords - used_words);
4267       stats()->update_used_after(r->used());
4268       // When moving a young gen region to old gen, we "allocate" that whole
4269       // region there. This is in addition to any already evacuated objects.
4270       // Notify the policy about that. Old gen regions do not cause an
4271       // additional allocation: both the objects still in the region and the
4272       // ones already moved are accounted for elsewhere.
4273       if (r->is_young()) {
4274         stats()->update_bytes_alloc_in_old(HeapRegion::GrainBytes);
4275       }

4276 
4277       // Update the region state due to the failed evacuation.
4278       r->handle_evacuation_failed();

4279 
4280       // Add region to old set, need to hold lock.

4281       MutexLocker x(OldSets_lock, Mutex::_no_safepoint_check_flag);
4282       _g1h->old_set_add(r);
4283     }
4284 
4285     double& timer_for_region(HeapRegion* r) {
4286       return r->is_young() ? _young_time : _non_young_time;
4287     }
4288   public:
4289     G1FreeCollectionSetClosure(const size_t* surviving_young_words,
4290                                uint worker_id,
4291                                FreeCSetStats* stats) :
4292         HeapRegionClosure(),
4293         _g1h(G1CollectedHeap::heap()),
4294         _surviving_young_words(surviving_young_words),
4295         _worker_id(worker_id),
4296         _young_time(0.0),
4297         _non_young_time(0.0),
4298         _stats(stats) { }
4299 
4300     virtual bool do_heap_region(HeapRegion* r) {
4301       assert(r->in_collection_set(), "Invariant: %u missing from CSet", r->hrm_index());
4302       EventForRegion event(r, _worker_id);
4303       TimerForRegion timer(timer_for_region(r));
4304 
4305       _g1h->clear_region_attr(r);
4306       stats()->update_rs_length(r->rem_set()->occupied_locked());
4307 
4308       if (r->is_young()) {
4309         assert_in_cset(r);
4310         r->record_surv_words_in_group(_surviving_young_words[r->young_index_in_cset()]);
4311       } else {
4312         _g1h->hot_card_cache()->reset_card_counts(r);
4313       }
4314 
4315       if (r->evacuation_failed()) {
4316         handle_failed_region(r);
4317       } else {
4318         handle_evacuated_region(r);
4319       }
4320       assert(!_g1h->is_on_master_free_list(r), "sanity");
4321 
4322       return false;
4323     }
4324 
4325     FreeCSetStats* stats() {
4326       return _stats;
4327     }



4328 
4329     void report_timing(double parallel_time) {
4330       G1GCPhaseTimes* pt = _g1h->phase_times();
4331       pt->record_time_secs(G1GCPhaseTimes::ParFreeCSet, _worker_id, parallel_time);
4332       if (_young_time > 0.0) {
4333         pt->record_time_secs(G1GCPhaseTimes::YoungFreeCSet, _worker_id, _young_time);
4334       }
4335       if (_non_young_time > 0.0) {
4336         pt->record_time_secs(G1GCPhaseTimes::NonYoungFreeCSet, _worker_id, _non_young_time);
4337       }
4338     }
4339   };
4340 
4341   // G1FreeCollectionSetTask members
4342   G1CollectedHeap*  _g1h;
4343   G1EvacuationInfo* _evacuation_info;
4344   FreeCSetStats*    _worker_stats;
4345   HeapRegionClaimer _claimer;
4346   const size_t*     _surviving_young_words;
4347   uint              _active_workers;
4348 
4349   FreeCSetStats* worker_stats(uint worker) {
4350     return &_worker_stats[worker];
4351   }
4352 
4353   void report_statistics() {
4354     // Merge the accounting
4355     FreeCSetStats total_stats;
4356     for (uint worker = 0; worker < _active_workers; worker++) {
4357       total_stats.merge_stats(worker_stats(worker));
4358     }
4359     total_stats.report(_g1h, _evacuation_info);
4360   }
4361 
4362 public:
4363   G1FreeCollectionSetTask(G1EvacuationInfo* evacuation_info, const size_t* surviving_young_words, uint active_workers) :
4364       AbstractGangTask("G1 Free Collection Set"),
4365       _g1h(G1CollectedHeap::heap()),
4366       _evacuation_info(evacuation_info),
4367       _worker_stats(NEW_C_HEAP_ARRAY(FreeCSetStats, active_workers, mtGC)),
4368       _claimer(active_workers),
4369       _surviving_young_words(surviving_young_words),
4370       _active_workers(active_workers) {
4371     for (uint worker = 0; worker < active_workers; worker++) {
4372       ::new (&_worker_stats[worker]) FreeCSetStats();
4373     }


4374   }
4375 
4376   ~G1FreeCollectionSetTask() {
4377     double serial_time = os::elapsedTime();
4378     report_statistics();
4379     FREE_C_HEAP_ARRAY(FreeCSetStats, _worker_stats);
4380     _g1h->phase_times()->record_serial_free_cset_time_ms((os::elapsedTime() - serial_time) * 1000.0);
4381   }
4382 




4383   virtual void work(uint worker_id) {
4384     EventGCPhaseParallel event;
4385     double start = os::elapsedTime();
4386     G1FreeCollectionSetClosure cl(_surviving_young_words, worker_id, worker_stats(worker_id));
4387     _g1h->collection_set_par_iterate_all(&cl, &_claimer, worker_id);
4388 
4389     // Report the total parallel time along with some more detailed metrics.
4390     cl.report_timing(os::elapsedTime() - start);
4391     event.commit(GCId::current(), worker_id, G1GCPhaseTimes::phase_name(G1GCPhaseTimes::ParFreeCSet));





4392   }
4393 };
4394 
4395 class G1RebuildFreeListTask : public AbstractGangTask {
4396   G1CollectedHeap* _g1h;
4397   FreeRegionList*  _worker_freelists;
4398   uint             _worker_chunk_size;
4399   uint             _num_workers;
4400 
4401   FreeRegionList* worker_freelist(uint worker) { return &_worker_freelists[worker]; }


4402 
4403   // Take the partial free lists and link them together.
4404   void rebuild_freelist() {
4405     // Abandon current free list to allow a rebuild using the
4406     // workers chunked free lists.
4407     _g1h->abandon_freelist();
4408     for (uint worker = 0; worker < _num_workers; worker++) {
4409      _g1h->append_to_freelist(worker_freelist(worker));
4410     }
4411   }
4412 public:
4413   G1RebuildFreeListTask(G1CollectedHeap* g1, uint num_workers) :
4414       AbstractGangTask("G1 Rebuild Free List Task"),
4415       _g1h(g1),
4416       _worker_freelists(NEW_C_HEAP_ARRAY(FreeRegionList, num_workers, mtGC)),
4417       _worker_chunk_size((_g1h->max_regions() + num_workers - 1) / num_workers),
4418       _num_workers(num_workers) {
4419     for (uint worker = 0; worker < _num_workers; worker++) {
4420      ::new (&_worker_freelists[worker]) FreeRegionList("Appendable Worker Free List");
4421     }
4422   }
4423 
4424   ~G1RebuildFreeListTask() {
4425     double serial_time = os::elapsedTime();
4426     rebuild_freelist();
4427     FREE_C_HEAP_ARRAY(FreeRegionList, _worker_freelists);
4428     _g1h->phase_times()->record_serial_rebuild_freelist_time_ms((os::elapsedTime() - serial_time) * 1000.0);
4429   }
4430 
4431   // Each worker creates a free list for a chunk of the heap. The chunks won't
4432   // be overlapping so we don't need to do any claiming.
4433   virtual void work(uint worker_id) {
4434     double start_time = os::elapsedTime();
4435     EventGCPhaseParallel event;
4436 
4437     uint start = worker_id * _worker_chunk_size;
4438     uint end = MIN2(start + _worker_chunk_size, _g1h->max_regions());
4439 
4440     // If start is outside the heap, this worker has nothing to do.
4441     if (start > end) {
4442       return;
4443     }
4444 
4445     FreeRegionList* free_list = worker_freelist(worker_id);
4446     for (uint i = start; i < end; i++) {
4447       if (!_g1h->hrm()->is_available(i)) {
4448         continue;






4449       }
4450       HeapRegion* region = _g1h->region_at(i);
4451       if (region->is_free()) {
4452         region->unlink_from_list();
4453         free_list->add_to_tail(region);
4454       }
4455     }
4456 
4457     event.commit(GCId::current(), worker_id, G1GCPhaseTimes::phase_name(G1GCPhaseTimes::RebuildFreeList));
4458     _g1h->phase_times()->record_time_secs(G1GCPhaseTimes::RebuildFreeList, worker_id, os::elapsedTime() - start_time);




4459   }
4460 };
4461 
4462 void G1CollectedHeap::free_collection_set(G1CollectionSet* collection_set, G1EvacuationInfo& evacuation_info, const size_t* surviving_young_words) {
4463   _eden.clear();
4464 
4465   // The free collections set is split up in two tasks, the first
4466   // frees the collection set and records what regions are free,
4467   // and the second one rebuilds the free list. This proved to be
4468   // more efficient than adding a sorted list to another.
4469 
4470   double free_cset_start_time = os::elapsedTime();
4471   {
4472     uint const num_cs_regions = _collection_set.region_length();
4473     uint const num_workers = clamp(num_cs_regions, 1u, workers()->active_workers());
4474     G1FreeCollectionSetTask cl(&evacuation_info, surviving_young_words, num_workers);
4475 
4476     log_debug(gc, ergo)("Running %s using %u workers for collection set length %u (%u)",
4477                         cl.name(), num_workers, num_cs_regions, num_regions());
4478     workers()->run_task(&cl, num_workers);
4479   }
4480 
4481   double free_cset_end_time = os::elapsedTime();
4482   phase_times()->record_total_free_cset_time_ms((free_cset_end_time - free_cset_start_time) * 1000.0);
4483   {
4484     uint const num_workers = workers()->active_workers();
4485     G1RebuildFreeListTask cl(this, num_workers);
4486 
4487     log_debug(gc, ergo)("Running %s using %u workers for rebuilding free list of %u (%u) regions",
4488                         cl.name(), num_workers, num_free_regions(), num_regions());
4489     workers()->run_task(&cl, num_workers);
4490   }
4491   phase_times()->record_total_rebuild_freelist_time_ms((os::elapsedTime() - free_cset_end_time) * 1000.0);
4492 
4493   collection_set->clear();
4494 }
4495 
4496 class G1FreeHumongousRegionClosure : public HeapRegionClosure {
4497  private:
4498   FreeRegionList* _free_region_list;
4499   HeapRegionSet* _proxy_set;
4500   uint _humongous_objects_reclaimed;
4501   uint _humongous_regions_reclaimed;
4502   size_t _freed_bytes;
4503  public:
4504 
4505   G1FreeHumongousRegionClosure(FreeRegionList* free_region_list) :
4506     _free_region_list(free_region_list), _proxy_set(NULL), _humongous_objects_reclaimed(0), _humongous_regions_reclaimed(0), _freed_bytes(0) {
4507   }
4508 
4509   virtual bool do_heap_region(HeapRegion* r) {
4510     if (!r->is_starts_humongous()) {
4511       return false;


< prev index next >