< prev index next >

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

Print this page




4381 
4382 class G1ParPreserveCMReferentsTask: public AbstractGangTask {
4383 protected:
4384   G1CollectedHeap*         _g1h;
4385   G1ParScanThreadStateSet* _pss;
4386   RefToScanQueueSet*       _queues;
4387   ParallelTaskTerminator   _terminator;
4388   uint                     _n_workers;
4389 
4390 public:
4391   G1ParPreserveCMReferentsTask(G1CollectedHeap* g1h, G1ParScanThreadStateSet* per_thread_states, int workers, RefToScanQueueSet *task_queues) :
4392     AbstractGangTask("ParPreserveCMReferents"),
4393     _g1h(g1h),
4394     _pss(per_thread_states),
4395     _queues(task_queues),
4396     _terminator(workers, _queues),
4397     _n_workers(workers)
4398   { }
4399 
4400   void work(uint worker_id) {


4401     ResourceMark rm;
4402     HandleMark   hm;
4403 
4404     G1ParScanThreadState*          pss = _pss->state_for_worker(worker_id);
4405     pss->set_ref_processor(NULL);
4406     assert(pss->queue_is_empty(), "both queue and overflow should be empty");
4407 
4408     // Is alive closure
4409     G1AlwaysAliveClosure always_alive(_g1h);
4410 
4411     // Copying keep alive closure. Applied to referent objects that need
4412     // to be copied.
4413     G1CopyingKeepAliveClosure keep_alive(_g1h, pss->closures()->raw_strong_oops(), pss);
4414 
4415     ReferenceProcessor* rp = _g1h->ref_processor_cm();
4416 
4417     uint limit = ReferenceProcessor::number_of_subclasses_of_ref() * rp->max_num_q();
4418     uint stride = MIN2(MAX2(_n_workers, 1U), limit);
4419 
4420     // limit is set using max_num_q() - which was set using ParallelGCThreads.


4444 
4445     // Drain the queue - which may cause stealing
4446     G1ParEvacuateFollowersClosure drain_queue(_g1h, pss, _queues, &_terminator);
4447     drain_queue.do_void();
4448     // Allocation buffers were retired at the end of G1ParEvacuateFollowersClosure
4449     assert(pss->queue_is_empty(), "should be");
4450   }
4451 };
4452 
4453 void G1CollectedHeap::process_weak_jni_handles() {
4454   double ref_proc_start = os::elapsedTime();
4455 
4456   G1STWIsAliveClosure is_alive(this);
4457   G1KeepAliveClosure keep_alive(this);
4458   JNIHandles::weak_oops_do(&is_alive, &keep_alive);
4459 
4460   double ref_proc_time = os::elapsedTime() - ref_proc_start;
4461   g1_policy()->phase_times()->record_ref_proc_time(ref_proc_time * 1000.0);
4462 }
4463 
4464 // Weak Reference processing during an evacuation pause (part 1).
4465 void G1CollectedHeap::process_discovered_references(G1ParScanThreadStateSet* per_thread_states) {
4466   double ref_proc_start = os::elapsedTime();
4467 
4468   ReferenceProcessor* rp = _ref_processor_stw;
4469   assert(rp->discovery_enabled(), "should have been enabled");
4470 
4471   // Any reference objects, in the collection set, that were 'discovered'
4472   // by the CM ref processor should have already been copied (either by
4473   // applying the external root copy closure to the discovered lists, or
4474   // by following an RSet entry).
4475   //
4476   // But some of the referents, that are in the collection set, that these
4477   // reference objects point to may not have been copied: the STW ref
4478   // processor would have seen that the reference object had already
4479   // been 'discovered' and would have skipped discovering the reference,
4480   // but would not have treated the reference object as a regular oop.
4481   // As a result the copy closure would not have been applied to the
4482   // referent object.
4483   //
4484   // We need to explicitly copy these referent objects - the references
4485   // will be processed at the end of remarking.
4486   //
4487   // We also need to do this copying before we process the reference
4488   // objects discovered by the STW ref processor in case one of these
4489   // referents points to another object which is also referenced by an
4490   // object discovered by the STW ref processor.
4491 
4492   uint no_of_gc_workers = workers()->active_workers();
4493 
4494   G1ParPreserveCMReferentsTask keep_cm_referents(this,
4495                                                  per_thread_states,
4496                                                  no_of_gc_workers,
4497                                                  _task_queues);
4498 
4499   workers()->run_task(&keep_cm_referents);
4500 










4501   // Closure to test whether a referent is alive.
4502   G1STWIsAliveClosure is_alive(this);
4503 
4504   // Even when parallel reference processing is enabled, the processing
4505   // of JNI refs is serial and performed serially by the current thread
4506   // rather than by a worker. The following PSS will be used for processing
4507   // JNI refs.
4508 
4509   // Use only a single queue for this PSS.
4510   G1ParScanThreadState*          pss = per_thread_states->state_for_worker(0);
4511   pss->set_ref_processor(NULL);
4512   assert(pss->queue_is_empty(), "pre-condition");
4513 
4514   // Keep alive closure.
4515   G1CopyingKeepAliveClosure keep_alive(this, pss->closures()->raw_strong_oops(), pss);
4516 
4517   // Serial Complete GC closure
4518   G1STWDrainQueueClosure drain_queue(this, pss);
4519 
4520   // Setup the soft refs policy...
4521   rp->setup_policy(false);
4522 
4523   ReferenceProcessorStats stats;
4524   if (!rp->processing_is_mt()) {
4525     // Serial reference processing...
4526     stats = rp->process_discovered_references(&is_alive,
4527                                               &keep_alive,
4528                                               &drain_queue,
4529                                               NULL,
4530                                               _gc_timer_stw);
4531   } else {


4532     // Parallel reference processing
4533     assert(rp->num_q() == no_of_gc_workers, "sanity");
4534     assert(no_of_gc_workers <= rp->max_num_q(), "sanity");
4535 
4536     G1STWRefProcTaskExecutor par_task_executor(this, per_thread_states, workers(), _task_queues, no_of_gc_workers);
4537     stats = rp->process_discovered_references(&is_alive,
4538                                               &keep_alive,
4539                                               &drain_queue,
4540                                               &par_task_executor,
4541                                               _gc_timer_stw);
4542   }
4543 
4544   _gc_tracer_stw->report_gc_reference_stats(stats);
4545 
4546   // We have completed copying any necessary live referent objects.
4547   assert(pss->queue_is_empty(), "both queue and overflow should be empty");
4548 
4549   double ref_proc_time = os::elapsedTime() - ref_proc_start;
4550   g1_policy()->phase_times()->record_ref_proc_time(ref_proc_time * 1000.0);
4551 }


4569 
4570     assert(rp->num_q() == n_workers, "sanity");
4571     assert(n_workers <= rp->max_num_q(), "sanity");
4572 
4573     G1STWRefProcTaskExecutor par_task_executor(this, per_thread_states, workers(), _task_queues, n_workers);
4574     rp->enqueue_discovered_references(&par_task_executor);
4575   }
4576 
4577   rp->verify_no_references_recorded();
4578   assert(!rp->discovery_enabled(), "should have been disabled");
4579 
4580   // FIXME
4581   // CM's reference processing also cleans up the string and symbol tables.
4582   // Should we do that here also? We could, but it is a serial operation
4583   // and could significantly increase the pause time.
4584 
4585   double ref_enq_time = os::elapsedTime() - ref_enq_start;
4586   g1_policy()->phase_times()->record_ref_enq_time(ref_enq_time * 1000.0);
4587 }
4588 






4589 void G1CollectedHeap::pre_evacuate_collection_set() {
4590   _expand_heap_after_alloc_failure = true;
4591   _evacuation_failed = false;
4592 
4593   // Disable the hot card cache.
4594   G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
4595   hot_card_cache->reset_hot_cache_claimed_index();
4596   hot_card_cache->set_use_cache(false);
4597 
4598   g1_rem_set()->prepare_for_oops_into_collection_set_do();
4599 }
4600 
4601 void G1CollectedHeap::evacuate_collection_set(EvacuationInfo& evacuation_info, G1ParScanThreadStateSet* per_thread_states) {
4602   // Should G1EvacuationFailureALot be in effect for this GC?
4603   NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
4604 
4605   assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty");
4606   double start_par_time_sec = os::elapsedTime();
4607   double end_par_time_sec;
4608 


4627     // reported parallel time.
4628   }
4629 
4630   G1GCPhaseTimes* phase_times = g1_policy()->phase_times();
4631 
4632   double par_time_ms = (end_par_time_sec - start_par_time_sec) * 1000.0;
4633   phase_times->record_par_time(par_time_ms);
4634 
4635   double code_root_fixup_time_ms =
4636         (os::elapsedTime() - end_par_time_sec) * 1000.0;
4637   phase_times->record_code_root_fixup_time(code_root_fixup_time_ms);
4638 }
4639 
4640 void G1CollectedHeap::post_evacuate_collection_set(EvacuationInfo& evacuation_info, G1ParScanThreadStateSet* per_thread_states) {
4641   // Process any discovered reference objects - we have
4642   // to do this _before_ we retire the GC alloc regions
4643   // as we may have to copy some 'reachable' referent
4644   // objects (and their reachable sub-graphs) that were
4645   // not copied during the pause.
4646   if (g1_policy()->should_process_references()) {

4647     process_discovered_references(per_thread_states);
4648   } else {
4649     ref_processor_stw()->verify_no_references_recorded();
4650     process_weak_jni_handles();
4651   }
4652 
4653   if (G1StringDedup::is_enabled()) {
4654     double fixup_start = os::elapsedTime();
4655 
4656     G1STWIsAliveClosure is_alive(this);
4657     G1KeepAliveClosure keep_alive(this);
4658     G1StringDedup::unlink_or_oops_do(&is_alive, &keep_alive, true, g1_policy()->phase_times());
4659 
4660     double fixup_time_ms = (os::elapsedTime() - fixup_start) * 1000.0;
4661     g1_policy()->phase_times()->record_string_dedup_fixup_time(fixup_time_ms);
4662   }
4663 
4664   g1_rem_set()->cleanup_after_oops_into_collection_set_do();
4665 
4666   if (evacuation_failed()) {


4670     // Note: the values are reset only when an actual
4671     // evacuation failure occurs.
4672     NOT_PRODUCT(reset_evacuation_should_fail();)
4673   }
4674 
4675   // Enqueue any remaining references remaining on the STW
4676   // reference processor's discovered lists. We need to do
4677   // this after the card table is cleaned (and verified) as
4678   // the act of enqueueing entries on to the pending list
4679   // will log these updates (and dirty their associated
4680   // cards). We need these updates logged to update any
4681   // RSets.
4682   if (g1_policy()->should_process_references()) {
4683     enqueue_discovered_references(per_thread_states);
4684   } else {
4685     g1_policy()->phase_times()->record_ref_enq_time(0);
4686   }
4687 
4688   _allocator->release_gc_alloc_regions(evacuation_info);
4689 
4690   per_thread_states->flush();
4691 
4692   record_obj_copy_mem_stats();
4693 
4694   _survivor_evac_stats.adjust_desired_plab_sz();
4695   _old_evac_stats.adjust_desired_plab_sz();
4696 
4697   // Reset and re-enable the hot card cache.
4698   // Note the counts for the cards in the regions in the
4699   // collection set are reset when the collection set is freed.
4700   G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
4701   hot_card_cache->reset_hot_cache();
4702   hot_card_cache->set_use_cache(true);
4703 
4704   purge_code_root_memory();
4705 
4706   redirty_logged_cards();
4707 #if defined(COMPILER2) || INCLUDE_JVMCI
4708   DerivedPointerTable::update_pointers();
4709 #endif
4710 }




4381 
4382 class G1ParPreserveCMReferentsTask: public AbstractGangTask {
4383 protected:
4384   G1CollectedHeap*         _g1h;
4385   G1ParScanThreadStateSet* _pss;
4386   RefToScanQueueSet*       _queues;
4387   ParallelTaskTerminator   _terminator;
4388   uint                     _n_workers;
4389 
4390 public:
4391   G1ParPreserveCMReferentsTask(G1CollectedHeap* g1h, G1ParScanThreadStateSet* per_thread_states, int workers, RefToScanQueueSet *task_queues) :
4392     AbstractGangTask("ParPreserveCMReferents"),
4393     _g1h(g1h),
4394     _pss(per_thread_states),
4395     _queues(task_queues),
4396     _terminator(workers, _queues),
4397     _n_workers(workers)
4398   { }
4399 
4400   void work(uint worker_id) {
4401     G1GCParPhaseTimesTracker x(_g1h->g1_policy()->phase_times(), G1GCPhaseTimes::PreserveCMReferents, worker_id);
4402 
4403     ResourceMark rm;
4404     HandleMark   hm;
4405 
4406     G1ParScanThreadState*          pss = _pss->state_for_worker(worker_id);
4407     pss->set_ref_processor(NULL);
4408     assert(pss->queue_is_empty(), "both queue and overflow should be empty");
4409 
4410     // Is alive closure
4411     G1AlwaysAliveClosure always_alive(_g1h);
4412 
4413     // Copying keep alive closure. Applied to referent objects that need
4414     // to be copied.
4415     G1CopyingKeepAliveClosure keep_alive(_g1h, pss->closures()->raw_strong_oops(), pss);
4416 
4417     ReferenceProcessor* rp = _g1h->ref_processor_cm();
4418 
4419     uint limit = ReferenceProcessor::number_of_subclasses_of_ref() * rp->max_num_q();
4420     uint stride = MIN2(MAX2(_n_workers, 1U), limit);
4421 
4422     // limit is set using max_num_q() - which was set using ParallelGCThreads.


4446 
4447     // Drain the queue - which may cause stealing
4448     G1ParEvacuateFollowersClosure drain_queue(_g1h, pss, _queues, &_terminator);
4449     drain_queue.do_void();
4450     // Allocation buffers were retired at the end of G1ParEvacuateFollowersClosure
4451     assert(pss->queue_is_empty(), "should be");
4452   }
4453 };
4454 
4455 void G1CollectedHeap::process_weak_jni_handles() {
4456   double ref_proc_start = os::elapsedTime();
4457 
4458   G1STWIsAliveClosure is_alive(this);
4459   G1KeepAliveClosure keep_alive(this);
4460   JNIHandles::weak_oops_do(&is_alive, &keep_alive);
4461 
4462   double ref_proc_time = os::elapsedTime() - ref_proc_start;
4463   g1_policy()->phase_times()->record_ref_proc_time(ref_proc_time * 1000.0);
4464 }
4465 
4466 void G1CollectedHeap::preserve_cm_referents(G1ParScanThreadStateSet* per_thread_states) {
4467   double preserve_cm_referents_start = os::elapsedTime();





4468   // Any reference objects, in the collection set, that were 'discovered'
4469   // by the CM ref processor should have already been copied (either by
4470   // applying the external root copy closure to the discovered lists, or
4471   // by following an RSet entry).
4472   //
4473   // But some of the referents, that are in the collection set, that these
4474   // reference objects point to may not have been copied: the STW ref
4475   // processor would have seen that the reference object had already
4476   // been 'discovered' and would have skipped discovering the reference,
4477   // but would not have treated the reference object as a regular oop.
4478   // As a result the copy closure would not have been applied to the
4479   // referent object.
4480   //
4481   // We need to explicitly copy these referent objects - the references
4482   // will be processed at the end of remarking.
4483   //
4484   // We also need to do this copying before we process the reference
4485   // objects discovered by the STW ref processor in case one of these
4486   // referents points to another object which is also referenced by an
4487   // object discovered by the STW ref processor.
4488 
4489   uint no_of_gc_workers = workers()->active_workers();
4490 
4491   G1ParPreserveCMReferentsTask keep_cm_referents(this,
4492                                                  per_thread_states,
4493                                                  no_of_gc_workers,
4494                                                  _task_queues);

4495   workers()->run_task(&keep_cm_referents);
4496 
4497   g1_policy()->phase_times()->record_preserve_cm_referents_time_ms((os::elapsedTime() - preserve_cm_referents_start) * 1000.0);
4498 }
4499 
4500 // Weak Reference processing during an evacuation pause (part 1).
4501 void G1CollectedHeap::process_discovered_references(G1ParScanThreadStateSet* per_thread_states) {
4502   double ref_proc_start = os::elapsedTime();
4503 
4504   ReferenceProcessor* rp = _ref_processor_stw;
4505   assert(rp->discovery_enabled(), "should have been enabled");
4506 
4507   // Closure to test whether a referent is alive.
4508   G1STWIsAliveClosure is_alive(this);
4509 
4510   // Even when parallel reference processing is enabled, the processing
4511   // of JNI refs is serial and performed serially by the current thread
4512   // rather than by a worker. The following PSS will be used for processing
4513   // JNI refs.
4514 
4515   // Use only a single queue for this PSS.
4516   G1ParScanThreadState*          pss = per_thread_states->state_for_worker(0);
4517   pss->set_ref_processor(NULL);
4518   assert(pss->queue_is_empty(), "pre-condition");
4519 
4520   // Keep alive closure.
4521   G1CopyingKeepAliveClosure keep_alive(this, pss->closures()->raw_strong_oops(), pss);
4522 
4523   // Serial Complete GC closure
4524   G1STWDrainQueueClosure drain_queue(this, pss);
4525 
4526   // Setup the soft refs policy...
4527   rp->setup_policy(false);
4528 
4529   ReferenceProcessorStats stats;
4530   if (!rp->processing_is_mt()) {
4531     // Serial reference processing...
4532     stats = rp->process_discovered_references(&is_alive,
4533                                               &keep_alive,
4534                                               &drain_queue,
4535                                               NULL,
4536                                               _gc_timer_stw);
4537   } else {
4538     uint no_of_gc_workers = workers()->active_workers();
4539 
4540     // Parallel reference processing
4541     assert(rp->num_q() == no_of_gc_workers, "sanity");
4542     assert(no_of_gc_workers <= rp->max_num_q(), "sanity");
4543 
4544     G1STWRefProcTaskExecutor par_task_executor(this, per_thread_states, workers(), _task_queues, no_of_gc_workers);
4545     stats = rp->process_discovered_references(&is_alive,
4546                                               &keep_alive,
4547                                               &drain_queue,
4548                                               &par_task_executor,
4549                                               _gc_timer_stw);
4550   }
4551 
4552   _gc_tracer_stw->report_gc_reference_stats(stats);
4553 
4554   // We have completed copying any necessary live referent objects.
4555   assert(pss->queue_is_empty(), "both queue and overflow should be empty");
4556 
4557   double ref_proc_time = os::elapsedTime() - ref_proc_start;
4558   g1_policy()->phase_times()->record_ref_proc_time(ref_proc_time * 1000.0);
4559 }


4577 
4578     assert(rp->num_q() == n_workers, "sanity");
4579     assert(n_workers <= rp->max_num_q(), "sanity");
4580 
4581     G1STWRefProcTaskExecutor par_task_executor(this, per_thread_states, workers(), _task_queues, n_workers);
4582     rp->enqueue_discovered_references(&par_task_executor);
4583   }
4584 
4585   rp->verify_no_references_recorded();
4586   assert(!rp->discovery_enabled(), "should have been disabled");
4587 
4588   // FIXME
4589   // CM's reference processing also cleans up the string and symbol tables.
4590   // Should we do that here also? We could, but it is a serial operation
4591   // and could significantly increase the pause time.
4592 
4593   double ref_enq_time = os::elapsedTime() - ref_enq_start;
4594   g1_policy()->phase_times()->record_ref_enq_time(ref_enq_time * 1000.0);
4595 }
4596 
4597 void G1CollectedHeap::merge_per_thread_state_info(G1ParScanThreadStateSet* per_thread_states) {
4598   double merge_pss_time_start = os::elapsedTime();
4599   per_thread_states->flush();
4600   g1_policy()->phase_times()->record_merge_pss_time_ms((os::elapsedTime() - merge_pss_time_start) * 1000.0);
4601 }
4602 
4603 void G1CollectedHeap::pre_evacuate_collection_set() {
4604   _expand_heap_after_alloc_failure = true;
4605   _evacuation_failed = false;
4606 
4607   // Disable the hot card cache.
4608   G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
4609   hot_card_cache->reset_hot_cache_claimed_index();
4610   hot_card_cache->set_use_cache(false);
4611 
4612   g1_rem_set()->prepare_for_oops_into_collection_set_do();
4613 }
4614 
4615 void G1CollectedHeap::evacuate_collection_set(EvacuationInfo& evacuation_info, G1ParScanThreadStateSet* per_thread_states) {
4616   // Should G1EvacuationFailureALot be in effect for this GC?
4617   NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
4618 
4619   assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty");
4620   double start_par_time_sec = os::elapsedTime();
4621   double end_par_time_sec;
4622 


4641     // reported parallel time.
4642   }
4643 
4644   G1GCPhaseTimes* phase_times = g1_policy()->phase_times();
4645 
4646   double par_time_ms = (end_par_time_sec - start_par_time_sec) * 1000.0;
4647   phase_times->record_par_time(par_time_ms);
4648 
4649   double code_root_fixup_time_ms =
4650         (os::elapsedTime() - end_par_time_sec) * 1000.0;
4651   phase_times->record_code_root_fixup_time(code_root_fixup_time_ms);
4652 }
4653 
4654 void G1CollectedHeap::post_evacuate_collection_set(EvacuationInfo& evacuation_info, G1ParScanThreadStateSet* per_thread_states) {
4655   // Process any discovered reference objects - we have
4656   // to do this _before_ we retire the GC alloc regions
4657   // as we may have to copy some 'reachable' referent
4658   // objects (and their reachable sub-graphs) that were
4659   // not copied during the pause.
4660   if (g1_policy()->should_process_references()) {
4661     preserve_cm_referents(per_thread_states);
4662     process_discovered_references(per_thread_states);
4663   } else {
4664     ref_processor_stw()->verify_no_references_recorded();
4665     process_weak_jni_handles();
4666   }
4667 
4668   if (G1StringDedup::is_enabled()) {
4669     double fixup_start = os::elapsedTime();
4670 
4671     G1STWIsAliveClosure is_alive(this);
4672     G1KeepAliveClosure keep_alive(this);
4673     G1StringDedup::unlink_or_oops_do(&is_alive, &keep_alive, true, g1_policy()->phase_times());
4674 
4675     double fixup_time_ms = (os::elapsedTime() - fixup_start) * 1000.0;
4676     g1_policy()->phase_times()->record_string_dedup_fixup_time(fixup_time_ms);
4677   }
4678 
4679   g1_rem_set()->cleanup_after_oops_into_collection_set_do();
4680 
4681   if (evacuation_failed()) {


4685     // Note: the values are reset only when an actual
4686     // evacuation failure occurs.
4687     NOT_PRODUCT(reset_evacuation_should_fail();)
4688   }
4689 
4690   // Enqueue any remaining references remaining on the STW
4691   // reference processor's discovered lists. We need to do
4692   // this after the card table is cleaned (and verified) as
4693   // the act of enqueueing entries on to the pending list
4694   // will log these updates (and dirty their associated
4695   // cards). We need these updates logged to update any
4696   // RSets.
4697   if (g1_policy()->should_process_references()) {
4698     enqueue_discovered_references(per_thread_states);
4699   } else {
4700     g1_policy()->phase_times()->record_ref_enq_time(0);
4701   }
4702 
4703   _allocator->release_gc_alloc_regions(evacuation_info);
4704 
4705   merge_per_thread_state_info(per_thread_states);
4706 
4707   record_obj_copy_mem_stats();
4708 
4709   _survivor_evac_stats.adjust_desired_plab_sz();
4710   _old_evac_stats.adjust_desired_plab_sz();
4711 
4712   // Reset and re-enable the hot card cache.
4713   // Note the counts for the cards in the regions in the
4714   // collection set are reset when the collection set is freed.
4715   G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
4716   hot_card_cache->reset_hot_cache();
4717   hot_card_cache->set_use_cache(true);
4718 
4719   purge_code_root_memory();
4720 
4721   redirty_logged_cards();
4722 #if defined(COMPILER2) || INCLUDE_JVMCI
4723   DerivedPointerTable::update_pointers();
4724 #endif
4725 }


< prev index next >