< prev index next >

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

Print this page
rev 53920 : imported patch 8218880-g1-crashes-periodic-gc-gclocker
rev 53921 : [mq]: 8218880-shade-review
rev 53924 : imported patch 8219747-remove-g1-prefix
rev 53925 : imported patch 8219747-shade-review
rev 53926 : imported patch 8219748-add-and-use-timing-object

@@ -1960,11 +1960,11 @@
   size_t n_completed_buffers = 0;
   while (dcqs.apply_closure_during_gc(cl, worker_i)) {
     n_completed_buffers++;
   }
   assert(dcqs.completed_buffers_num() == 0, "Completed buffers exist!");
-  policy()->phase_times()->record_thread_work_item(G1GCPhaseTimes::UpdateRS, worker_i, n_completed_buffers, G1GCPhaseTimes::UpdateRSProcessedBuffers);
+  phase_times()->record_thread_work_item(G1GCPhaseTimes::UpdateRS, worker_i, n_completed_buffers, G1GCPhaseTimes::UpdateRSProcessedBuffers);
 }
 
 // Computes the sum of the storage used by the various regions.
 size_t G1CollectedHeap::used() const {
   size_t result = _summary_bytes_used + _allocator->used_in_alloc_regions();

@@ -2548,11 +2548,11 @@
   }
 
   // Fill TLAB's and such
   double start = os::elapsedTime();
   ensure_parsability(true);
-  policy()->phase_times()->record_prepare_tlab_time_ms((os::elapsedTime() - start) * 1000.0);
+  phase_times()->record_prepare_tlab_time_ms((os::elapsedTime() - start) * 1000.0);
 }
 
 void G1CollectedHeap::gc_epilogue(bool full) {
   // Update common counters.
   if (full) {

@@ -2571,11 +2571,11 @@
 #endif
   // always_do_update_barrier = true;
 
   double start = os::elapsedTime();
   resize_all_tlabs();
-  policy()->phase_times()->record_resize_tlab_time_ms((os::elapsedTime() - start) * 1000.0);
+  phase_times()->record_resize_tlab_time_ms((os::elapsedTime() - start) * 1000.0);
 
   MemoryService::track_memory_usage();
   // We have just completed a GC. Update the soft reference
   // policy with the new heap occupancy
   Universe::update_heap_info_at_gc();

@@ -2765,21 +2765,21 @@
   void flush_rem_set_entries() { _dcq.flush(); }
 };
 
 void G1CollectedHeap::register_humongous_regions_with_cset() {
   if (!G1EagerReclaimHumongousObjects) {
-    policy()->phase_times()->record_fast_reclaim_humongous_stats(0.0, 0, 0);
+    phase_times()->record_fast_reclaim_humongous_stats(0.0, 0, 0);
     return;
   }
   double time = os::elapsed_counter();
 
   // Collect reclaim candidate information and register candidates with cset.
   RegisterHumongousWithInCSetFastTestClosure cl;
   heap_region_iterate(&cl);
 
   time = ((double)(os::elapsed_counter() - time) / os::elapsed_frequency()) * 1000.0;
-  policy()->phase_times()->record_fast_reclaim_humongous_stats(time,
+  phase_times()->record_fast_reclaim_humongous_stats(time,
                                                                cl.total_humongous(),
                                                                cl.candidate_humongous());
   _has_humongous_reclaim_candidates = cl.candidate_humongous() > 0;
 
   // Finally flush all remembered set entries to re-check into the global DCQS.

@@ -2847,11 +2847,11 @@
   double wait_time_ms = 0.0;
   if (waited) {
     double scan_wait_end = os::elapsedTime();
     wait_time_ms = (scan_wait_end - scan_wait_start) * 1000.0;
   }
-  policy()->phase_times()->record_root_region_scan_wait_time(wait_time_ms);
+  phase_times()->record_root_region_scan_wait_time(wait_time_ms);
 }
 
 class G1PrintCollectionSetClosure : public HeapRegionClosure {
 private:
   G1HRPrinter* _hr_printer;

@@ -3065,16 +3065,16 @@
         _survivor_evac_stats.adjust_desired_plab_sz();
         _old_evac_stats.adjust_desired_plab_sz();
 
         double start = os::elapsedTime();
         start_new_collection_set();
-        policy()->phase_times()->record_start_new_cset_time_ms((os::elapsedTime() - start) * 1000.0);
+        phase_times()->record_start_new_cset_time_ms((os::elapsedTime() - start) * 1000.0);
 
         if (evacuation_failed()) {
           double recalculate_used_start = os::elapsedTime();
           set_used(recalculate_used());
-          policy()->phase_times()->record_evac_fail_recalc_used_time((os::elapsedTime() - recalculate_used_start) * 1000.0);
+          phase_times()->record_evac_fail_recalc_used_time((os::elapsedTime() - recalculate_used_start) * 1000.0);
 
           if (_archive_allocator != NULL) {
             _archive_allocator->clear_used();
           }
           for (uint i = 0; i < ParallelGCThreads; i++) {

@@ -3110,11 +3110,11 @@
             // expansion_amount() does this when it returns a value > 0.
             double expand_ms;
             if (!expand(expand_bytes, _workers, &expand_ms)) {
               // We failed to expand the heap. Cannot do anything about it.
             }
-            policy()->phase_times()->record_expand_heap_time(expand_ms);
+            phase_times()->record_expand_heap_time(expand_ms);
           }
         }
 
         // We redo the verification but now wrt to the new CSet which
         // has just got initialized after the previous CSet was freed.

@@ -3123,11 +3123,11 @@
         // This timing is only used by the ergonomics to handle our pause target.
         // It is unclear why this should not include the full pause. We will
         // investigate this in CR 7178365.
         double sample_end_time_sec = os::elapsedTime();
         double pause_time_ms = (sample_end_time_sec - sample_start_time_sec) * MILLIUNITS;
-        size_t total_cards_scanned = policy()->phase_times()->sum_thread_work_items(G1GCPhaseTimes::ScanRS, G1GCPhaseTimes::ScanRSScannedCards);
+        size_t total_cards_scanned = phase_times()->sum_thread_work_items(G1GCPhaseTimes::ScanRS, G1GCPhaseTimes::ScanRSScannedCards);
         policy()->record_collection_pause_end(pause_time_ms, total_cards_scanned, heap_used_bytes_before_gc);
 
         evacuation_info.set_collectionset_used_before(collection_set()->bytes_used_before());
         evacuation_info.set_bytes_copied(policy()->bytes_copied_during_gc());
 

@@ -3215,11 +3215,11 @@
 
   remove_self_forwarding_pointers();
   SharedRestorePreservedMarksTaskExecutor task_executor(workers());
   _preserved_marks_set.restore(&task_executor);
 
-  policy()->phase_times()->record_evac_fail_remove_self_forwards((os::elapsedTime() - remove_self_forwards_start) * 1000.0);
+  phase_times()->record_evac_fail_remove_self_forwards((os::elapsedTime() - remove_self_forwards_start) * 1000.0);
 }
 
 void G1CollectedHeap::preserve_mark_during_evac_failure(uint worker_id, oop obj, markOop m) {
   if (!_evacuation_failed) {
     _evacuation_failed = true;

@@ -3273,11 +3273,11 @@
 
   void work(uint worker_id) {
     if (worker_id >= _n_workers) return;  // no work needed this round
 
     double start_sec = os::elapsedTime();
-    _g1h->policy()->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerStart, worker_id, start_sec);
+    _g1h->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerStart, worker_id, start_sec);
 
     {
       ResourceMark rm;
       HandleMark   hm;
 

@@ -3303,11 +3303,11 @@
 
         evac_term_attempts = evac.term_attempts();
         term_sec = evac.term_time();
         double elapsed_sec = os::elapsedTime() - start;
 
-        G1GCPhaseTimes* p = _g1h->policy()->phase_times();
+        G1GCPhaseTimes* p = _g1h->phase_times();
         p->add_time_secs(G1GCPhaseTimes::ObjCopy, worker_id, elapsed_sec - term_sec);
 
         p->record_or_add_thread_work_item(G1GCPhaseTimes::ObjCopy,
                                           worker_id,
                                           pss->lab_waste_words() * HeapWordSize,

@@ -3325,11 +3325,11 @@
 
       // Close the inner scope so that the ResourceMark and HandleMark
       // destructors are executed here and are included as part of the
       // "GC Worker Time".
     }
-    _g1h->policy()->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerEnd, worker_id, os::elapsedTime());
+    _g1h->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerEnd, worker_id, os::elapsedTime());
   }
 };
 
 void G1CollectedHeap::complete_cleaning(BoolObjectClosure* is_alive,
                                         bool class_unloading_occurred) {

@@ -3390,17 +3390,17 @@
  public:
   G1RedirtyLoggedCardsTask(G1DirtyCardQueueSet* queue, G1CollectedHeap* g1h) : AbstractGangTask("Redirty Cards"),
     _queue(queue), _g1h(g1h) { }
 
   virtual void work(uint worker_id) {
-    G1GCPhaseTimes* phase_times = _g1h->policy()->phase_times();
-    G1GCParPhaseTimesTracker x(phase_times, G1GCPhaseTimes::RedirtyCards, worker_id);
+    G1GCPhaseTimes* p = _g1h->phase_times();
+    G1GCParPhaseTimesTracker x(p, G1GCPhaseTimes::RedirtyCards, worker_id);
 
     RedirtyLoggedCardTableEntryClosure cl(_g1h);
     _queue->par_apply_closure_to_all_completed_buffers(&cl);
 
-    phase_times->record_thread_work_item(G1GCPhaseTimes::RedirtyCards, worker_id, cl.num_dirtied());
+    p->record_thread_work_item(G1GCPhaseTimes::RedirtyCards, worker_id, cl.num_dirtied());
   }
 };
 
 void G1CollectedHeap::redirty_logged_cards() {
   double redirty_logged_cards_start = os::elapsedTime();

@@ -3411,11 +3411,11 @@
 
   G1DirtyCardQueueSet& dcq = G1BarrierSet::dirty_card_queue_set();
   dcq.merge_bufferlists(&dirty_card_queue_set());
   assert(dirty_card_queue_set().completed_buffers_num() == 0, "All should be consumed");
 
-  policy()->phase_times()->record_redirty_logged_cards_time_ms((os::elapsedTime() - redirty_logged_cards_start) * 1000.0);
+  phase_times()->record_redirty_logged_cards_time_ms((os::elapsedTime() - redirty_logged_cards_start) * 1000.0);
 }
 
 // Weak Reference Processing support
 
 bool G1STWIsAliveClosure::do_object_b(oop p) {

@@ -3644,11 +3644,11 @@
   G1STWDrainQueueClosure drain_queue(this, pss);
 
   // Setup the soft refs policy...
   rp->setup_policy(false);
 
-  ReferenceProcessorPhaseTimes* pt = policy()->phase_times()->ref_phase_times();
+  ReferenceProcessorPhaseTimes* pt = phase_times()->ref_phase_times();
 
   ReferenceProcessorStats stats;
   if (!rp->processing_is_mt()) {
     // Serial reference processing...
     stats = rp->process_discovered_references(&is_alive,

@@ -3680,11 +3680,11 @@
   make_pending_list_reachable();
 
   rp->verify_no_references_recorded();
 
   double ref_proc_time = os::elapsedTime() - ref_proc_start;
-  policy()->phase_times()->record_ref_proc_time(ref_proc_time * 1000.0);
+  phase_times()->record_ref_proc_time(ref_proc_time * 1000.0);
 }
 
 void G1CollectedHeap::make_pending_list_reachable() {
   if (collector_state()->in_initial_mark_gc()) {
     oop pll_head = Universe::reference_pending_list();

@@ -3696,11 +3696,11 @@
 }
 
 void G1CollectedHeap::merge_per_thread_state_info(G1ParScanThreadStateSet* per_thread_states) {
   double merge_pss_time_start = os::elapsedTime();
   per_thread_states->flush();
-  policy()->phase_times()->record_merge_pss_time_ms((os::elapsedTime() - merge_pss_time_start) * 1000.0);
+  phase_times()->record_merge_pss_time_ms((os::elapsedTime() - merge_pss_time_start) * 1000.0);
 }
 
 void G1CollectedHeap::pre_evacuate_collection_set() {
   _expand_heap_after_alloc_failure = true;
   _evacuation_failed = false;

@@ -3710,31 +3710,27 @@
   _hot_card_cache->set_use_cache(false);
 
   rem_set()->prepare_for_oops_into_collection_set_do();
   _preserved_marks_set.assert_empty();
 
-  G1GCPhaseTimes* phase_times = policy()->phase_times();
-
   // InitialMark needs claim bits to keep track of the marked-through CLDs.
   if (collector_state()->in_initial_mark_gc()) {
     double start_clear_claimed_marks = os::elapsedTime();
 
     ClassLoaderDataGraph::clear_claimed_marks();
 
     double recorded_clear_claimed_marks_time_ms = (os::elapsedTime() - start_clear_claimed_marks) * 1000.0;
-    phase_times->record_clear_claimed_marks_time_ms(recorded_clear_claimed_marks_time_ms);
+    phase_times()->record_clear_claimed_marks_time_ms(recorded_clear_claimed_marks_time_ms);
   }
 }
 
 void G1CollectedHeap::evacuate_collection_set(G1ParScanThreadStateSet* per_thread_states) {
   // Should G1EvacuationFailureALot be in effect for this GC?
   NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
 
   assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty");
 
-  G1GCPhaseTimes* phase_times = policy()->phase_times();
-
   double start_par_time_sec = os::elapsedTime();
   double end_par_time_sec;
 
   {
     const uint n_workers = workers()->active_workers();

@@ -3750,15 +3746,15 @@
     // taken for the destructor is NOT included in the
     // reported parallel time.
   }
 
   double par_time_ms = (end_par_time_sec - start_par_time_sec) * 1000.0;
-  phase_times->record_par_time(par_time_ms);
+  phase_times()->record_par_time(par_time_ms);
 
   double code_root_fixup_time_ms =
         (os::elapsedTime() - end_par_time_sec) * 1000.0;
-  phase_times->record_code_root_fixup_time(code_root_fixup_time_ms);
+  phase_times()->record_code_root_fixup_time(code_root_fixup_time_ms);
 }
 
 class G1EvacuateOptionalRegionTask : public AbstractGangTask {
   G1CollectedHeap* _g1h;
   G1ParScanThreadStateSet* _per_thread_states;

@@ -3800,11 +3796,11 @@
       // Chunk lists for this region is no longer needed.
       used_memory += pss->oops_into_optional_region(hr)->used_memory();
     }
 
     Tickspan scan_time = (Ticks::now() - start) - copy_time;
-    G1GCPhaseTimes* p = _g1h->policy()->phase_times();
+    G1GCPhaseTimes* p = _g1h->phase_times();
     p->record_or_add_time_secs(G1GCPhaseTimes::OptScanRS, worker_id, scan_time.seconds());
     p->record_or_add_time_secs(G1GCPhaseTimes::OptObjCopy, worker_id, copy_time.seconds());
 
     p->record_or_add_thread_work_item(G1GCPhaseTimes::OptScanRS, worker_id, scanned, G1GCPhaseTimes::OptCSetScannedCards);
     p->record_or_add_thread_work_item(G1GCPhaseTimes::OptScanRS, worker_id, claimed, G1GCPhaseTimes::OptCSetClaimedCards);

@@ -3816,11 +3812,11 @@
     Ticks start = Ticks::now();
     G1ParEvacuateFollowersClosure cl(_g1h, pss, _queues, &_terminator, G1GCPhaseTimes::OptObjCopy);
     cl.do_void();
 
     Tickspan evac_time = (Ticks::now() - start);
-    G1GCPhaseTimes* p = _g1h->policy()->phase_times();
+    G1GCPhaseTimes* p = _g1h->phase_times();
     p->record_or_add_time_secs(G1GCPhaseTimes::OptObjCopy, worker_id, evac_time.seconds());
     assert(pss->trim_ticks().seconds() == 0.0, "Unexpected partial trimming done during optional evacuation");
   }
 
  public:

@@ -3865,12 +3861,11 @@
 
   if (evacuation_failed()) {
     return;
   }
 
-  G1GCPhaseTimes* phase_times = policy()->phase_times();
-  const double gc_start_time_ms = phase_times->cur_collection_start_sec() * 1000.0;
+  const double gc_start_time_ms = phase_times()->cur_collection_start_sec() * 1000.0;
 
   double start_time_sec = os::elapsedTime();
 
   do {
     double time_used_ms = os::elapsedTime() * 1000.0 - gc_start_time_ms;

@@ -3893,11 +3888,11 @@
     if (optional_cset.evacuation_failed()) {
       break;
     }
   } while (!optional_cset.is_empty());
 
-  phase_times->record_optional_evacuation((os::elapsedTime() - start_time_sec) * 1000.0);
+  phase_times()->record_optional_evacuation((os::elapsedTime() - start_time_sec) * 1000.0);
 }
 
 void G1CollectedHeap::post_evacuate_collection_set(G1EvacuationInfo& evacuation_info, G1ParScanThreadStateSet* per_thread_states) {
   // Also cleans the card table from temporary duplicate detection information used
   // during UpdateRS/ScanRS.

@@ -3912,19 +3907,19 @@
 
   G1STWIsAliveClosure is_alive(this);
   G1KeepAliveClosure keep_alive(this);
 
   WeakProcessor::weak_oops_do(workers(), &is_alive, &keep_alive,
-                              policy()->phase_times()->weak_phase_times());
+                              phase_times()->weak_phase_times());
 
   if (G1StringDedup::is_enabled()) {
     double string_dedup_time_ms = os::elapsedTime();
 
-    string_dedup_cleaning(&is_alive, &keep_alive, policy()->phase_times());
+    string_dedup_cleaning(&is_alive, &keep_alive, phase_times());
 
     double string_cleanup_time_ms = (os::elapsedTime() - string_dedup_time_ms) * 1000.0;
-    policy()->phase_times()->record_string_deduplication_time(string_cleanup_time_ms);
+    phase_times()->record_string_deduplication_time(string_cleanup_time_ms);
   }
 
   if (evacuation_failed()) {
     restore_after_evac_failure();
 

@@ -3950,11 +3945,11 @@
 
   redirty_logged_cards();
 #if COMPILER2_OR_JVMCI
   double start = os::elapsedTime();
   DerivedPointerTable::update_pointers();
-  policy()->phase_times()->record_derived_pointer_table_update_time((os::elapsedTime() - start) * 1000.0);
+  phase_times()->record_derived_pointer_table_update_time((os::elapsedTime() - start) * 1000.0);
 #endif
   policy()->print_age_table();
 }
 
 void G1CollectedHeap::record_obj_copy_mem_stats() {

@@ -4217,11 +4212,11 @@
   // Chunk size for work distribution. The chosen value has been determined experimentally
   // to be a good tradeoff between overhead and achievable parallelism.
   static uint chunk_size() { return 32; }
 
   virtual void work(uint worker_id) {
-    G1GCPhaseTimes* timer = G1CollectedHeap::heap()->policy()->phase_times();
+    G1GCPhaseTimes* timer = G1CollectedHeap::heap()->phase_times();
 
     // Claim serial work.
     if (_serial_work_claim == 0) {
       jint value = Atomic::add(1, &_serial_work_claim) - 1;
       if (value == 0) {

@@ -4294,11 +4289,11 @@
                         cl.name(),
                         num_workers,
                         _collection_set.region_length());
     workers()->run_task(&cl, num_workers);
   }
-  policy()->phase_times()->record_total_free_cset_time_ms((os::elapsedTime() - free_cset_start_time) * 1000.0);
+  phase_times()->record_total_free_cset_time_ms((os::elapsedTime() - free_cset_start_time) * 1000.0);
 
   collection_set->clear();
 }
 
 class G1FreeHumongousRegionClosure : public HeapRegionClosure {

@@ -4419,11 +4414,11 @@
 void G1CollectedHeap::eagerly_reclaim_humongous_regions() {
   assert_at_safepoint_on_vm_thread();
 
   if (!G1EagerReclaimHumongousObjects ||
       (!_has_humongous_reclaim_candidates && !log_is_enabled(Debug, gc, humongous))) {
-    policy()->phase_times()->record_fast_reclaim_humongous_time_ms(0.0, 0);
+    phase_times()->record_fast_reclaim_humongous_time_ms(0.0, 0);
     return;
   }
 
   double start_time = os::elapsedTime();
 

@@ -4444,11 +4439,11 @@
   }
 
   prepend_to_freelist(&local_cleanup_list);
   decrement_summary_bytes(cl.bytes_freed());
 
-  policy()->phase_times()->record_fast_reclaim_humongous_time_ms((os::elapsedTime() - start_time) * 1000.0,
+  phase_times()->record_fast_reclaim_humongous_time_ms((os::elapsedTime() - start_time) * 1000.0,
                                                                  cl.humongous_objects_reclaimed());
 }
 
 class G1AbandonCollectionSetClosure : public HeapRegionClosure {
 public:

@@ -4824,11 +4819,11 @@
 
 void G1CollectedHeap::purge_code_root_memory() {
   double purge_start = os::elapsedTime();
   G1CodeRootSet::purge();
   double purge_time_ms = (os::elapsedTime() - purge_start) * 1000.0;
-  policy()->phase_times()->record_strong_code_root_purge_time(purge_time_ms);
+  phase_times()->record_strong_code_root_purge_time(purge_time_ms);
 }
 
 class RebuildStrongCodeRootClosure: public CodeBlobClosure {
   G1CollectedHeap* _g1h;
 
< prev index next >