< prev index next >

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

Print this page
rev 53988 : imported patch 8219100-cleanup-young-collection-prologue

@@ -2541,11 +2541,11 @@
   // This summary needs to be printed before incrementing total collections.
   rem_set()->print_periodic_summary_info("Before GC RS summary", total_collections());
 
   // Update common counters.
   increment_total_collections(full /* full gc */);
-  if (full) {
+  if (full || collector_state()->in_initial_mark_gc()) {
     increment_old_marking_cycles_started();
   }
 
   // Fill TLAB's and such
   double start = os::elapsedTime();

@@ -2863,37 +2863,113 @@
     return false;
   }
 };
 
 void G1CollectedHeap::start_new_collection_set() {
+  double start = os::elapsedTime();
+
   collection_set()->start_incremental_building();
 
   clear_cset_fast_test();
 
   guarantee(_eden.length() == 0, "eden should have been cleared");
   policy()->transfer_survivors_to_cset(survivor());
+
+  // We redo the verification but now wrt to the new CSet which
+  // has just got initialized after the previous CSet was freed.
+  _cm->verify_no_collection_set_oops_in_stacks();
+
+  phase_times()->record_start_new_cset_time_ms((os::elapsedTime() - start) * 1000.0);
+}
+
+void G1CollectedHeap::calculate_collection_set(G1EvacuationInfo& evacuation_info, double target_pause_time_ms){
+  policy()->finalize_collection_set(target_pause_time_ms, &_survivor);
+  evacuation_info.set_collectionset_regions(collection_set()->region_length());
+
+  _cm->verify_no_collection_set_oops_in_stacks();
+
+  if (_hr_printer.is_active()) {
+    G1PrintCollectionSetClosure cl(&_hr_printer);
+    _collection_set.iterate(&cl);
+  }
+}
+
+G1HeapVerifier::G1VerifyType G1CollectedHeap::young_collection_verify_type() const {
+  if (collector_state()->in_initial_mark_gc()) {
+    return G1HeapVerifier::G1VerifyConcurrentStart;
+  } else if (collector_state()->in_young_only_phase()) {
+    return G1HeapVerifier::G1VerifyYoungNormal;
+  } else {
+    return G1HeapVerifier::G1VerifyMixed;
+  }
+}
+
+void G1CollectedHeap::verify_before_young_collection(G1HeapVerifier::G1VerifyType type) {
+  if (VerifyRememberedSets) {
+    log_info(gc, verify)("[Verifying RemSets before GC]");
+    VerifyRegionRemSetClosure v_cl;
+    heap_region_iterate(&v_cl);
+  }
+  _verifier->verify_before_gc(type);
+  _verifier->check_bitmaps("GC Start");
+}
+
+void G1CollectedHeap::verify_after_young_collection(G1HeapVerifier::G1VerifyType type) {
+  if (VerifyRememberedSets) {
+    log_info(gc, verify)("[Verifying RemSets after GC]");
+    VerifyRegionRemSetClosure v_cl;
+    heap_region_iterate(&v_cl);
+  }
+  _verifier->verify_after_gc(type);
+  _verifier->check_bitmaps("GC End");
+}
+
+void G1CollectedHeap::expand_heap_after_young_collection(){
+  size_t expand_bytes = _heap_sizing_policy->expansion_amount();
+  if (expand_bytes > 0) {
+    // No need for an ergo logging here,
+    // 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.
+    }
+    phase_times()->record_expand_heap_time(expand_ms);
+  }
+}
+
+const char* G1CollectedHeap::young_gc_name() const {
+  if (collector_state()->in_initial_mark_gc()) {
+    return "Pause Young (Concurrent Start)";
+  } else if (collector_state()->in_young_only_phase()) {
+    if (collector_state()->in_young_gc_before_mixed()) {
+      return "Pause Young (Prepare Mixed)";
+    } else {
+      return "Pause Young (Normal)";
+    }
+  } else {
+    return "Pause Young (Mixed)";
+  }
 }
 
-bool
-G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
+bool G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
   assert_at_safepoint_on_vm_thread();
   guarantee(!is_gc_active(), "collection is not reentrant");
 
   if (GCLocker::check_active_before_gc()) {
     return false;
   }
 
-  _gc_timer_stw->register_gc_start();
-
   GCIdMark gc_id_mark;
-  _gc_tracer_stw->report_gc_start(gc_cause(), _gc_timer_stw->gc_start());
 
   SvcGCMarker sgcm(SvcGCMarker::MINOR);
   ResourceMark rm;
 
   policy()->note_gc_start();
 
+  _gc_timer_stw->register_gc_start();
+  _gc_tracer_stw->report_gc_start(gc_cause(), _gc_timer_stw->gc_start());
+
   wait_for_root_region_scanning();
 
   print_heap_before_gc();
   print_heap_regions();
   trace_heap_before_gc(_gc_tracer_stw);

@@ -2910,51 +2986,30 @@
   }
 
   // We do not allow initial-mark to be piggy-backed on a mixed GC.
   assert(!collector_state()->in_initial_mark_gc() ||
           collector_state()->in_young_only_phase(), "sanity");
-
   // We also do not allow mixed GCs during marking.
   assert(!collector_state()->mark_or_rebuild_in_progress() || collector_state()->in_young_only_phase(), "sanity");
 
   // Record whether this pause is an initial mark. When the current
   // thread has completed its logging output and it's safe to signal
   // the CM thread, the flag's value in the policy has been reset.
   bool should_start_conc_mark = collector_state()->in_initial_mark_gc();
+  if (should_start_conc_mark) {
+    _cm->gc_tracer_cm()->set_gc_cause(gc_cause());
+  }
 
   // Inner scope for scope based logging, timers, and stats collection
   {
     G1EvacuationInfo evacuation_info;
 
-    if (collector_state()->in_initial_mark_gc()) {
-      // We are about to start a marking cycle, so we increment the
-      // full collection counter.
-      increment_old_marking_cycles_started();
-      _cm->gc_tracer_cm()->set_gc_cause(gc_cause());
-    }
-
     _gc_tracer_stw->report_yc_type(collector_state()->yc_type());
 
     GCTraceCPUTime tcpu;
 
-    G1HeapVerifier::G1VerifyType verify_type;
-    FormatBuffer<> gc_string("Pause Young ");
-    if (collector_state()->in_initial_mark_gc()) {
-      gc_string.append("(Concurrent Start)");
-      verify_type = G1HeapVerifier::G1VerifyConcurrentStart;
-    } else if (collector_state()->in_young_only_phase()) {
-      if (collector_state()->in_young_gc_before_mixed()) {
-        gc_string.append("(Prepare Mixed)");
-      } else {
-        gc_string.append("(Normal)");
-      }
-      verify_type = G1HeapVerifier::G1VerifyYoungNormal;
-    } else {
-      gc_string.append("(Mixed)");
-      verify_type = G1HeapVerifier::G1VerifyMixed;
-    }
-    GCTraceTime(Info, gc) tm(gc_string, NULL, gc_cause(), true);
+    GCTraceTime(Info, gc) tm(young_gc_name(), NULL, gc_cause(), true);
 
     uint active_workers = WorkerPolicy::calc_active_workers(workers()->total_workers(),
                                                             workers()->active_workers(),
                                                             Threads::number_of_non_daemon_threads());
     active_workers = workers()->update_active_workers(active_workers);

@@ -2965,132 +3020,60 @@
                          collector_state()->yc_type() == Mixed /* all_memory_pools_affected */);
 
     G1HeapTransition heap_transition(this);
     size_t heap_used_bytes_before_gc = used();
 
-    // Don't dynamically change the number of GC threads this early.  A value of
-    // 0 is used to indicate serial work.  When parallel work is done,
-    // it will be set.
-
-    { // Call to jvmpi::post_class_unload_events must occur outside of active GC
+    {
       IsGCActiveMark x;
 
       gc_prologue(false);
 
-      if (VerifyRememberedSets) {
-        log_info(gc, verify)("[Verifying RemSets before GC]");
-        VerifyRegionRemSetClosure v_cl;
-        heap_region_iterate(&v_cl);
-      }
-
-      _verifier->verify_before_gc(verify_type);
+      G1HeapVerifier::G1VerifyType verify_type = young_collection_verify_type();
+      verify_before_young_collection(verify_type);
 
-      _verifier->check_bitmaps("GC Start");
-
-#if COMPILER2_OR_JVMCI
-      DerivedPointerTable::clear();
-#endif
+      {
+        // The elapsed time induced by the start time below deliberately elides
+        // the possible verification above.
+        double sample_start_time_sec = os::elapsedTime();
 
       // Please see comment in g1CollectedHeap.hpp and
       // G1CollectedHeap::ref_processing_init() to see how
       // reference processing currently works in G1.
-
-      // Enable discovery in the STW reference processor
       _ref_processor_stw->enable_discovery();
 
-      {
         // We want to temporarily turn off discovery by the
         // CM ref processor, if necessary, and turn it back on
         // on again later if we do. Using a scoped
         // NoRefDiscovery object will do this.
         NoRefDiscovery no_cm_discovery(_ref_processor_cm);
 
-        // Forget the current alloc region (we might even choose it to be part
-        // of the collection set!).
-        _allocator->release_mutator_alloc_region();
-
-        // 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.
-        //
-        // Preserving the old comment here if that helps the investigation:
-        //
-        // The elapsed time induced by the start time below deliberately elides
-        // the possible verification above.
-        double sample_start_time_sec = os::elapsedTime();
-
         policy()->record_collection_pause_start(sample_start_time_sec);
 
-        if (collector_state()->in_initial_mark_gc()) {
-          concurrent_mark()->pre_initial_mark();
-        }
-
-        policy()->finalize_collection_set(target_pause_time_ms, &_survivor);
-
-        evacuation_info.set_collectionset_regions(collection_set()->region_length());
-
-        register_humongous_regions_with_cset();
-
-        assert(_verifier->check_cset_fast_test(), "Inconsistency in the InCSetState table.");
-
-        // We call this after finalize_cset() to
-        // ensure that the CSet has been finalized.
-        _cm->verify_no_cset_oops();
+        // Forget the current allocation region (we might even choose it to be part
+        // of the collection set!).
+        _allocator->release_mutator_alloc_region();
 
-        if (_hr_printer.is_active()) {
-          G1PrintCollectionSetClosure cl(&_hr_printer);
-          _collection_set.iterate(&cl);
-        }
-
-        // Initialize the GC alloc regions.
-        _allocator->init_gc_alloc_regions(evacuation_info);
+        calculate_collection_set(evacuation_info, target_pause_time_ms);
 
         G1ParScanThreadStateSet per_thread_states(this,
                                                   workers()->active_workers(),
                                                   collection_set()->young_region_length(),
                                                   collection_set()->optional_region_length());
-        pre_evacuate_collection_set();
+        pre_evacuate_collection_set(evacuation_info);
 
         // Actually do the work...
         evacuate_collection_set(&per_thread_states);
         evacuate_optional_collection_set(&per_thread_states);
 
         post_evacuate_collection_set(evacuation_info, &per_thread_states);
 
-        const size_t* surviving_young_words = per_thread_states.surviving_young_words();
-        free_collection_set(&_collection_set, evacuation_info, surviving_young_words);
-
-        eagerly_reclaim_humongous_regions();
+        start_new_collection_set();
 
-        record_obj_copy_mem_stats();
         _survivor_evac_stats.adjust_desired_plab_sz();
         _old_evac_stats.adjust_desired_plab_sz();
 
-        double start = os::elapsedTime();
-        start_new_collection_set();
-        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());
-          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++) {
-            if (_evacuation_failed_info_array[i].has_failed()) {
-              _gc_tracer_stw->report_evacuation_failed(_evacuation_failed_info_array[i]);
-            }
-          }
-        } else {
-          // The "used" of the the collection set have already been subtracted
-          // when they were freed.  Add in the bytes evacuated.
-          increase_used(policy()->bytes_copied_during_gc());
-        }
-
-        if (collector_state()->in_initial_mark_gc()) {
+        if (should_start_conc_mark) {
           // We have to do this before we notify the CM threads that
           // they can start working to make sure that all the
           // appropriate initialization is done on the CM object.
           concurrent_mark()->post_initial_mark();
           // Note that we don't actually trigger the CM thread at

@@ -3100,53 +3083,19 @@
 
         allocate_dummy_regions();
 
         _allocator->init_mutator_alloc_region();
 
-        {
-          size_t expand_bytes = _heap_sizing_policy->expansion_amount();
-          if (expand_bytes > 0) {
-            size_t bytes_before = capacity();
-            // No need for an ergo logging here,
-            // 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.
-            }
-            phase_times()->record_expand_heap_time(expand_ms);
-          }
-        }
+        expand_heap_after_young_collection();
 
-        // We redo the verification but now wrt to the new CSet which
-        // has just got initialized after the previous CSet was freed.
-        _cm->verify_no_cset_oops();
-
-        // 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 = 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());
-
-        if (VerifyRememberedSets) {
-          log_info(gc, verify)("[Verifying RemSets after GC]");
-          VerifyRegionRemSetClosure v_cl;
-          heap_region_iterate(&v_cl);
         }
 
-        _verifier->verify_after_gc(verify_type);
-        _verifier->check_bitmaps("GC End");
-
-        assert(!_ref_processor_stw->discovery_enabled(), "Postcondition");
-        _ref_processor_stw->verify_no_references_recorded();
-
-        // CM reference discovery will be re-enabled if necessary.
-      }
+      verify_after_young_collection(verify_type);
 
 #ifdef TRACESPINNING
       ParallelTaskTerminator::print_termination_counts();
 #endif
 

@@ -3159,15 +3108,10 @@
     }
 
     policy()->print_phases();
     heap_transition.print();
 
-    // It is not yet to safe to tell the concurrent mark to
-    // start as we have some optional output below. We don't want the
-    // output from the concurrent mark thread interfering with this
-    // logging output either.
-
     _hrm->verify_optional();
     _verifier->verify_region_sets_optional();
 
     TASKQUEUE_STATS_ONLY(print_taskqueue_stats());
     TASKQUEUE_STATS_ONLY(reset_taskqueue_stats());

@@ -3190,17 +3134,15 @@
   // It should now be safe to tell the concurrent mark thread to start
   // without its logging output interfering with the logging output
   // that came from the pause.
 
   if (should_start_conc_mark) {
-    // CAUTION: after the doConcurrentMark() call below,
-    // the concurrent marking thread(s) could be running
-    // concurrently with us. Make sure that anything after
-    // this point does not assume that we are the only GC thread
-    // running. Note: of course, the actual marking work will
-    // not start until the safepoint itself is released in
-    // SuspendibleThreadSet::desynchronize().
+    // CAUTION: after the doConcurrentMark() call below, the concurrent marking
+    // thread(s) could be running concurrently with us. Make sure that anything
+    // after this point does not assume that we are the only GC thread running.
+    // Note: of course, the actual marking work will not start until the safepoint
+    // itself is released in SuspendibleThreadSet::desynchronize().
     do_concurrent_mark();
   }
 
   return true;
 }

@@ -3677,10 +3619,11 @@
   // We have completed copying any necessary live referent objects.
   assert(pss->queue_is_empty(), "both queue and overflow should be empty");
 
   make_pending_list_reachable();
 
+  assert(!rp->discovery_enabled(), "Postcondition");
   rp->verify_no_references_recorded();
 
   double ref_proc_time = os::elapsedTime() - ref_proc_start;
   phase_times()->record_ref_proc_time(ref_proc_time * 1000.0);
 }

@@ -3699,23 +3642,35 @@
   double merge_pss_time_start = os::elapsedTime();
   per_thread_states->flush();
   phase_times()->record_merge_pss_time_ms((os::elapsedTime() - merge_pss_time_start) * 1000.0);
 }
 
-void G1CollectedHeap::pre_evacuate_collection_set() {
+void G1CollectedHeap::pre_evacuate_collection_set(G1EvacuationInfo& evacuation_info) {
   _expand_heap_after_alloc_failure = true;
   _evacuation_failed = false;
 
   // Disable the hot card cache.
   _hot_card_cache->reset_hot_cache_claimed_index();
   _hot_card_cache->set_use_cache(false);
 
+  // Initialize the GC alloc regions.
+  _allocator->init_gc_alloc_regions(evacuation_info);
+
+  register_humongous_regions_with_cset();
+  assert(_verifier->check_cset_fast_test(), "Inconsistency in the InCSetState table.");
+
   rem_set()->prepare_for_oops_into_collection_set_do();
   _preserved_marks_set.assert_empty();
 
+#if COMPILER2_OR_JVMCI
+  DerivedPointerTable::clear();
+#endif
+
   // InitialMark needs claim bits to keep track of the marked-through CLDs.
   if (collector_state()->in_initial_mark_gc()) {
+    concurrent_mark()->pre_initial_mark();
+
     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;

@@ -3918,23 +3873,38 @@
 
     double string_cleanup_time_ms = (os::elapsedTime() - string_dedup_time_ms) * 1000.0;
     phase_times()->record_string_deduplication_time(string_cleanup_time_ms);
   }
 
+  _allocator->release_gc_alloc_regions(evacuation_info);
+
   if (evacuation_failed()) {
     restore_after_evac_failure();
 
     // Reset the G1EvacuationFailureALot counters and flags
-    // Note: the values are reset only when an actual
-    // evacuation failure occurs.
     NOT_PRODUCT(reset_evacuation_should_fail();)
+
+    double recalculate_used_start = os::elapsedTime();
+    set_used(recalculate_used());
+    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++) {
+      if (_evacuation_failed_info_array[i].has_failed()) {
+        _gc_tracer_stw->report_evacuation_failed(_evacuation_failed_info_array[i]);
+      }
+    }
+  } else {
+    // The "used" of the the collection set have already been subtracted
+    // when they were freed.  Add in the bytes evacuated.
+    increase_used(policy()->bytes_copied_during_gc());
   }
 
   _preserved_marks_set.assert_empty();
 
-  _allocator->release_gc_alloc_regions(evacuation_info);
-
   merge_per_thread_state_info(per_thread_states);
 
   // Reset and re-enable the hot card cache.
   // Note the counts for the cards in the regions in the
   // collection set are reset when the collection set is freed.

@@ -3942,10 +3912,20 @@
   _hot_card_cache->set_use_cache(true);
 
   purge_code_root_memory();
 
   redirty_logged_cards();
+
+  free_collection_set(&_collection_set, evacuation_info, per_thread_states->surviving_young_words());
+
+  eagerly_reclaim_humongous_regions();
+
+  record_obj_copy_mem_stats();
+
+  evacuation_info.set_collectionset_used_before(collection_set()->bytes_used_before());
+  evacuation_info.set_bytes_copied(policy()->bytes_copied_during_gc());
+
 #if COMPILER2_OR_JVMCI
   double start = os::elapsedTime();
   DerivedPointerTable::update_pointers();
   phase_times()->record_derived_pointer_table_update_time((os::elapsedTime() - start) * 1000.0);
 #endif
< prev index next >