< prev index next >

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

Print this page
rev 13070 : imported patch 8177544-full-gc-scope

@@ -36,23 +36,24 @@
 #include "gc/g1/g1CollectedHeap.inline.hpp"
 #include "gc/g1/g1CollectionSet.hpp"
 #include "gc/g1/g1CollectorPolicy.hpp"
 #include "gc/g1/g1CollectorState.hpp"
 #include "gc/g1/g1EvacStats.inline.hpp"
+#include "gc/g1/g1FullGCScope.hpp"
 #include "gc/g1/g1GCPhaseTimes.hpp"
 #include "gc/g1/g1HeapSizingPolicy.hpp"
 #include "gc/g1/g1HeapTransition.hpp"
 #include "gc/g1/g1HeapVerifier.hpp"
 #include "gc/g1/g1HotCardCache.hpp"
-#include "gc/g1/g1MarkSweep.hpp"
 #include "gc/g1/g1OopClosures.inline.hpp"
 #include "gc/g1/g1ParScanThreadState.inline.hpp"
 #include "gc/g1/g1Policy.hpp"
 #include "gc/g1/g1RegionToSpaceMapper.hpp"
 #include "gc/g1/g1RemSet.inline.hpp"
 #include "gc/g1/g1RootClosures.hpp"
 #include "gc/g1/g1RootProcessor.hpp"
+#include "gc/g1/g1SerialCollector.hpp"
 #include "gc/g1/g1StringDedup.hpp"
 #include "gc/g1/g1YCTypes.hpp"
 #include "gc/g1/heapRegion.inline.hpp"
 #include "gc/g1/heapRegionRemSet.hpp"
 #include "gc/g1/heapRegionSet.inline.hpp"

@@ -1069,77 +1070,10 @@
   }
 
   ShouldNotReachHere();
 }
 
-class PostMCRemSetClearClosure: public HeapRegionClosure {
-  G1CollectedHeap* _g1h;
-  ModRefBarrierSet* _mr_bs;
-public:
-  PostMCRemSetClearClosure(G1CollectedHeap* g1h, ModRefBarrierSet* mr_bs) :
-    _g1h(g1h), _mr_bs(mr_bs) {}
-
-  bool doHeapRegion(HeapRegion* r) {
-    HeapRegionRemSet* hrrs = r->rem_set();
-
-    _g1h->reset_gc_time_stamps(r);
-
-    if (r->is_continues_humongous()) {
-      // We'll assert that the strong code root list and RSet is empty
-      assert(hrrs->strong_code_roots_list_length() == 0, "sanity");
-      assert(hrrs->occupied() == 0, "RSet should be empty");
-    } else {
-      hrrs->clear();
-    }
-    // You might think here that we could clear just the cards
-    // corresponding to the used region.  But no: if we leave a dirty card
-    // in a region we might allocate into, then it would prevent that card
-    // from being enqueued, and cause it to be missed.
-    // Re: the performance cost: we shouldn't be doing full GC anyway!
-    _mr_bs->clear(MemRegion(r->bottom(), r->end()));
-
-    return false;
-  }
-};
-
-void G1CollectedHeap::clear_rsets_post_compaction() {
-  PostMCRemSetClearClosure rs_clear(this, g1_barrier_set());
-  heap_region_iterate(&rs_clear);
-}
-
-class RebuildRSOutOfRegionClosure: public HeapRegionClosure {
-  G1CollectedHeap*   _g1h;
-  UpdateRSOopClosure _cl;
-public:
-  RebuildRSOutOfRegionClosure(G1CollectedHeap* g1, uint worker_i = 0) :
-    _cl(g1->g1_rem_set(), worker_i),
-    _g1h(g1)
-  { }
-
-  bool doHeapRegion(HeapRegion* r) {
-    if (!r->is_continues_humongous()) {
-      _cl.set_from(r);
-      r->oop_iterate(&_cl);
-    }
-    return false;
-  }
-};
-
-class ParRebuildRSTask: public AbstractGangTask {
-  G1CollectedHeap* _g1;
-  HeapRegionClaimer _hrclaimer;
-
-public:
-  ParRebuildRSTask(G1CollectedHeap* g1) :
-      AbstractGangTask("ParRebuildRSTask"), _g1(g1), _hrclaimer(g1->workers()->active_workers()) {}
-
-  void work(uint worker_id) {
-    RebuildRSOutOfRegionClosure rebuild_rs(_g1, worker_id);
-    _g1->heap_region_par_iterate(&rebuild_rs, worker_id, &_hrclaimer);
-  }
-};
-
 class PostCompactionPrinterClosure: public HeapRegionClosure {
 private:
   G1HRPrinter* _hr_printer;
 public:
   bool doHeapRegion(HeapRegion* hr) {

@@ -1158,56 +1092,11 @@
     heap_region_iterate(&cl);
   }
 
 }
 
-bool G1CollectedHeap::do_full_collection(bool explicit_gc,
-                                         bool clear_all_soft_refs) {
-  assert_at_safepoint(true /* should_be_vm_thread */);
-
-  if (GCLocker::check_active_before_gc()) {
-    return false;
-  }
-
-  STWGCTimer* gc_timer = G1MarkSweep::gc_timer();
-  gc_timer->register_gc_start();
-
-  SerialOldTracer* gc_tracer = G1MarkSweep::gc_tracer();
-  GCIdMark gc_id_mark;
-  gc_tracer->report_gc_start(gc_cause(), gc_timer->gc_start());
-
-  SvcGCMarker sgcm(SvcGCMarker::FULL);
-  ResourceMark rm;
-
-  print_heap_before_gc();
-  print_heap_regions();
-  trace_heap_before_gc(gc_tracer);
-
-  size_t metadata_prev_used = MetaspaceAux::used_bytes();
-
-  _verifier->verify_region_sets_optional();
-
-  const bool do_clear_all_soft_refs = clear_all_soft_refs ||
-                           collector_policy()->should_clear_all_soft_refs();
-
-  ClearedAllSoftRefs casr(do_clear_all_soft_refs, collector_policy());
-
-  {
-    IsGCActiveMark x;
-
-    // Timing
-    assert(!GCCause::is_user_requested_gc(gc_cause()) || explicit_gc, "invariant");
-    GCTraceCPUTime tcpu;
-
-    {
-      GCTraceTime(Info, gc) tm("Pause Full", NULL, gc_cause(), true);
-      TraceCollectorStats tcs(g1mm()->full_collection_counters());
-      TraceMemoryManagerStats tms(true /* fullGC */, gc_cause());
-
-      G1HeapTransition heap_transition(this);
-      g1_policy()->record_full_collection_start();
-
+void G1CollectedHeap::abort_concurrent_cycle() {
       // Note: When we have a more flexible GC logging framework that
       // allows us to add optional attributes to a GC log record we
       // could consider timing and reporting how long we wait in the
       // following two methods.
       wait_while_free_regions_coming();

@@ -1218,35 +1107,22 @@
       // early.
       _cm->root_regions()->abort();
       _cm->root_regions()->wait_until_scan_finished();
       append_secondary_free_list_if_not_empty_with_lock();
 
-      gc_prologue(true);
-      increment_total_collections(true /* full gc */);
-      increment_old_marking_cycles_started();
-
-      assert(used() == recalculate_used(), "Should be equal");
-
-      _verifier->verify_before_gc();
-
-      _verifier->check_bitmaps("Full GC Start");
-      pre_full_gc_dump(gc_timer);
-
-#if defined(COMPILER2) || INCLUDE_JVMCI
-      DerivedPointerTable::clear();
-#endif
-
       // Disable discovery and empty the discovered lists
       // for the CM ref processor.
       ref_processor_cm()->disable_discovery();
       ref_processor_cm()->abandon_partial_discovery();
       ref_processor_cm()->verify_no_references_recorded();
 
       // Abandon current iterations of concurrent marking and concurrent
       // refinement, if any are in progress.
       concurrent_mark()->abort();
+}
 
+void G1CollectedHeap::prepare_heap_for_full_collection() {
       // Make sure we'll choose a new allocation region afterwards.
       _allocator->release_mutator_alloc_region();
       _allocator->abandon_gc_alloc_regions();
       g1_rem_set()->cleanupHRRS();
 

@@ -1256,114 +1132,36 @@
       // after this full GC.
       abandon_collection_set(collection_set());
 
       tear_down_region_sets(false /* free_list_only */);
       collector_state()->set_gcs_are_young(true);
+}
 
-      // See the comments in g1CollectedHeap.hpp and
-      // G1CollectedHeap::ref_processing_init() about
-      // how reference processing currently works in G1.
-
-      // Temporarily make discovery by the STW ref processor single threaded (non-MT).
-      ReferenceProcessorMTDiscoveryMutator stw_rp_disc_ser(ref_processor_stw(), false);
-
-      // Temporarily clear the STW ref processor's _is_alive_non_header field.
-      ReferenceProcessorIsAliveMutator stw_rp_is_alive_null(ref_processor_stw(), NULL);
-
-      ref_processor_stw()->enable_discovery();
-      ref_processor_stw()->setup_policy(do_clear_all_soft_refs);
-
-      // Do collection work
-      {
-        HandleMark hm;  // Discard invalid handles created during gc
-        G1MarkSweep::invoke_at_safepoint(ref_processor_stw(), do_clear_all_soft_refs);
-      }
-
-      assert(num_free_regions() == 0, "we should not have added any free regions");
-      rebuild_region_sets(false /* free_list_only */);
-
-      // Enqueue any discovered reference objects that have
-      // not been removed from the discovered lists.
-      ref_processor_stw()->enqueue_discovered_references();
-
-#if defined(COMPILER2) || INCLUDE_JVMCI
-      DerivedPointerTable::update_pointers();
-#endif
-
-      MemoryService::track_memory_usage();
-
-      assert(!ref_processor_stw()->discovery_enabled(), "Postcondition");
-      ref_processor_stw()->verify_no_references_recorded();
-
-      // Delete metaspaces for unloaded class loaders and clean up loader_data graph
-      ClassLoaderDataGraph::purge();
-      MetaspaceAux::verify_metrics();
-
-      // Note: since we've just done a full GC, concurrent
-      // marking is no longer active. Therefore we need not
-      // re-enable reference discovery for the CM ref processor.
-      // That will be done at the start of the next marking cycle.
-      assert(!ref_processor_cm()->discovery_enabled(), "Postcondition");
-      ref_processor_cm()->verify_no_references_recorded();
-
-      reset_gc_time_stamp();
-      // Since everything potentially moved, we will clear all remembered
-      // sets, and clear all cards.  Later we will rebuild remembered
-      // sets. We will also reset the GC time stamps of the regions.
-      clear_rsets_post_compaction();
-      check_gc_time_stamps();
-
-      resize_if_necessary_after_full_collection();
-
-      // We should do this after we potentially resize the heap so
-      // that all the COMMIT / UNCOMMIT events are generated before
-      // the compaction events.
-      print_hrm_post_compaction();
-
+void G1CollectedHeap::reset_card_cache_and_queue() {
       if (_hot_card_cache->use_cache()) {
         _hot_card_cache->reset_card_counts();
         _hot_card_cache->reset_hot_cache();
       }
 
-      // Rebuild remembered sets of all regions.
-      uint n_workers =
-        AdaptiveSizePolicy::calc_active_workers(workers()->total_workers(),
-                                                workers()->active_workers(),
-                                                Threads::number_of_non_daemon_threads());
-      workers()->update_active_workers(n_workers);
-      log_info(gc,task)("Using %u workers of %u to rebuild remembered set", n_workers, workers()->total_workers());
-
-      ParRebuildRSTask rebuild_rs_task(this);
-      workers()->run_task(&rebuild_rs_task);
-
-      // Rebuild the strong code root lists for each region
-      rebuild_strong_code_roots();
-
-      if (true) { // FIXME
-        MetaspaceGC::compute_new_size();
-      }
-
-#ifdef TRACESPINNING
-      ParallelTaskTerminator::print_termination_counts();
-#endif
-
-      // Discard all rset updates
+  // Discard all stale remembered set updates.
       JavaThread::dirty_card_queue_set().abandon_logs();
       assert(dirty_card_queue_set().completed_buffers_num() == 0, "DCQS should be empty");
+}
 
-      // At this point there should be no regions in the
-      // entire heap tagged as young.
-      assert(check_young_list_empty(), "young list should be empty at this point");
-
-      // Update the number of full collections that have been completed.
-      increment_old_marking_cycles_completed(false /* concurrent */);
+void G1CollectedHeap::verify_before_full_collection(bool explicit_gc) {
+  assert(!GCCause::is_user_requested_gc(gc_cause()) || explicit_gc, "invariant");
+  assert(used() == recalculate_used(), "Should be equal");
+  _verifier->verify_region_sets_optional();
+  _verifier->verify_before_gc();
+  _verifier->check_bitmaps("Full GC Start");
+}
 
+void G1CollectedHeap::verify_after_full_collection() {
+  check_gc_time_stamps();
       _hrm.verify_optional();
       _verifier->verify_region_sets_optional();
-
       _verifier->verify_after_gc();
-
       // Clear the previous marking bitmap, if needed for bitmap verification.
       // Note we cannot do this when we clear the next marking bitmap in
       // G1ConcurrentMark::abort() above since VerifyDuringGC verifies the
       // objects marked during a full GC against the previous bitmap.
       // But we need to clear it before calling check_bitmaps below since

@@ -1373,37 +1171,102 @@
         GCTraceTime(Debug, gc)("Clear Bitmap for Verification");
         _cm->clear_prev_bitmap(workers());
       }
       _verifier->check_bitmaps("Full GC End");
 
+  // At this point there should be no regions in the
+  // entire heap tagged as young.
+  assert(check_young_list_empty(), "young list should be empty at this point");
+
+  // Note: since we've just done a full GC, concurrent
+  // marking is no longer active. Therefore we need not
+  // re-enable reference discovery for the CM ref processor.
+  // That will be done at the start of the next marking cycle.
+  // We also know that the STW processor should no longer
+  // discover any new references.
+  assert(!ref_processor_stw()->discovery_enabled(), "Postcondition");
+  assert(!ref_processor_cm()->discovery_enabled(), "Postcondition");
+  ref_processor_stw()->verify_no_references_recorded();
+  ref_processor_cm()->verify_no_references_recorded();
+}
+
+void G1CollectedHeap::do_full_collection_inner(G1FullGCScope* scope) {
+  GCTraceTime(Info, gc) tm("Pause Full", NULL, gc_cause(), true);
+  G1HeapTransition heap_transition(this);
+  g1_policy()->record_full_collection_start();
+
+  print_heap_before_gc();
+  print_heap_regions();
+
+  abort_concurrent_cycle();
+  verify_before_full_collection(scope->is_explicit_gc());
+
+  gc_prologue(true);
+  prepare_heap_for_full_collection();
+
+  G1SerialCollector serial(scope, ref_processor_stw());
+  serial.prepare_collection();
+  serial.collect();
+  serial.complete_collection();
+
+  assert(num_free_regions() == 0, "we should not have added any free regions");
+  MemoryService::track_memory_usage();
+
+  // Delete metaspaces for unloaded class loaders and clean up loader_data graph
+  ClassLoaderDataGraph::purge();
+  MetaspaceAux::verify_metrics();
+
+  // Prepare heap for normal collections.
+  rebuild_region_sets(false /* free_list_only */);
+  reset_card_cache_and_queue();
+  resize_if_necessary_after_full_collection();
+
+  // Rebuild the strong code root lists for each region
+  rebuild_strong_code_roots();
+
+  // Start a new incremental collection set for the next pause
       start_new_collection_set();
 
       _allocator->init_mutator_alloc_region();
 
+  // Post collection state updates.
+  MetaspaceGC::compute_new_size();
+  gc_epilogue(true);
       g1_policy()->record_full_collection_end();
 
-      // We must call G1MonitoringSupport::update_sizes() in the same scoping level
-      // as an active TraceMemoryManagerStats object (i.e. before the destructor for the
-      // TraceMemoryManagerStats is called) so that the G1 memory pools are updated
-      // before any GC notifications are raised.
-      g1mm()->update_sizes();
-
-      gc_epilogue(true);
+  // Post collection verification.
+  verify_after_full_collection();
 
+  // Post collection logging.
+  // We should do this after we potentially resize the heap so
+  // that all the COMMIT / UNCOMMIT events are generated before
+  // the compaction events.
+  print_hrm_post_compaction();
       heap_transition.print();
-
       print_heap_after_gc();
       print_heap_regions();
-      trace_heap_after_gc(gc_tracer);
+#ifdef TRACESPINNING
+  ParallelTaskTerminator::print_termination_counts();
+#endif
+}
 
-      post_full_gc_dump(gc_timer);
-    }
+bool G1CollectedHeap::do_full_collection(bool explicit_gc,
+                                         bool clear_all_soft_refs) {
+  assert_at_safepoint(true /* should_be_vm_thread */);
 
-    gc_timer->register_gc_end();
-    gc_tracer->report_gc_end(gc_timer->gc_end(), gc_timer->time_partitions());
+  if (GCLocker::check_active_before_gc()) {
+    // Full GC was not completed.
+    return false;
   }
 
+  const bool do_clear_all_soft_refs = clear_all_soft_refs ||
+      collector_policy()->should_clear_all_soft_refs();
+
+  G1FullGCScope scope(explicit_gc, do_clear_all_soft_refs);
+  do_full_collection_inner(&scope);
+
+  // Full collection was successfully completed.
   return true;
 }
 
 void G1CollectedHeap::do_full_collection(bool clear_all_soft_refs) {
   // Currently, there is no facility in the do_full_collection(bool) API to notify

@@ -2695,25 +2558,41 @@
   assert(heap != NULL, "Uninitialized access to G1CollectedHeap::heap()");
   assert(heap->kind() == CollectedHeap::G1CollectedHeap, "Not a G1CollectedHeap");
   return (G1CollectedHeap*)heap;
 }
 
-void G1CollectedHeap::gc_prologue(bool full /* Ignored */) {
+void G1CollectedHeap::gc_prologue(bool full) {
   // always_do_update_barrier = false;
   assert(InlineCacheBuffer::is_empty(), "should have cleaned up ICBuffer");
 
-  double start = os::elapsedTime();
+  // This summary needs to be printed before incrementing total collections.
+  g1_rem_set()->print_periodic_summary_info("Before GC RS summary", total_collections());
+
+  // Update common counters.
+  increment_total_collections(full /* full gc */);
+  if (full) {
+    increment_old_marking_cycles_started();
+    reset_gc_time_stamp();
+  } else {
+    increment_gc_time_stamp();
+  }
+
   // Fill TLAB's and such
+  double start = os::elapsedTime();
   accumulate_statistics_all_tlabs();
   ensure_parsability(true);
   g1_policy()->phase_times()->record_prepare_tlab_time_ms((os::elapsedTime() - start) * 1000.0);
-
-  g1_rem_set()->print_periodic_summary_info("Before GC RS summary", total_collections());
 }
 
 void G1CollectedHeap::gc_epilogue(bool full) {
-  // we are at the end of the GC. Total collections has already been increased.
+  // Update common counters.
+  if (full) {
+    // Update the number of full collections that have been completed.
+    increment_old_marking_cycles_completed(false /* concurrent */);
+  }
+
+  // We are at the end of the GC. Total collections has already been increased.
   g1_rem_set()->print_periodic_summary_info("After GC RS summary", total_collections() - 1);
 
   // FIXME: what is this about?
   // I'm ignoring the "fill_newgen()" call if "alloc_event_enabled"
   // is set.

@@ -3115,12 +2994,10 @@
 
     { // Call to jvmpi::post_class_unload_events must occur outside of active GC
       IsGCActiveMark x;
 
       gc_prologue(false);
-      increment_total_collections(false /* full gc */);
-      increment_gc_time_stamp();
 
       if (VerifyRememberedSets) {
         log_info(gc, verify)("[Verifying RemSets before GC]");
         VerifyRegionRemSetClosure v_cl;
         heap_region_iterate(&v_cl);
< prev index next >