< prev index next >

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

Print this page

        

@@ -44,10 +44,11 @@
 #include "gc_implementation/g1/g1MarkSweep.hpp"
 #include "gc_implementation/g1/g1OopClosures.inline.hpp"
 #include "gc_implementation/g1/g1ParScanThreadState.inline.hpp"
 #include "gc_implementation/g1/g1RegionToSpaceMapper.hpp"
 #include "gc_implementation/g1/g1RemSet.inline.hpp"
+#include "gc_implementation/g1/g1RootProcessor.hpp"
 #include "gc_implementation/g1/g1StringDedup.hpp"
 #include "gc_implementation/g1/g1YCTypes.hpp"
 #include "gc_implementation/g1/heapRegion.inline.hpp"
 #include "gc_implementation/g1/heapRegionRemSet.hpp"
 #include "gc_implementation/g1/heapRegionSet.inline.hpp"

@@ -83,22 +84,10 @@
 // and allocate_new_tlab, which are the "entry" points to the
 // allocation code from the rest of the JVM.  (Note that this does not
 // apply to TLAB allocation, which is not part of this interface: it
 // is done by clients of this interface.)
 
-// Notes on implementation of parallelism in different tasks.
-//
-// G1ParVerifyTask uses heap_region_par_iterate_chunked() for parallelism.
-// The number of GC workers is passed to heap_region_par_iterate_chunked().
-// It does use run_task() which sets _n_workers in the task.
-// G1ParTask executes g1_process_roots() ->
-// SharedHeap::process_roots() which calls eventually to
-// CardTableModRefBS::par_non_clean_card_iterate_work() which uses
-// SequentialSubTasksDone.  SharedHeap::process_roots() also
-// directly uses SubTasksDone (_process_strong_tasks field in SharedHeap).
-//
-
 // Local to this file.
 
 class RefineCardTableEntryClosure: public CardTableEntryClosure {
   bool _concurrent;
 public:

@@ -1852,11 +1841,10 @@
   _into_cset_dirty_card_queue_set(false),
   _is_alive_closure_cm(this),
   _is_alive_closure_stw(this),
   _ref_processor_cm(NULL),
   _ref_processor_stw(NULL),
-  _process_strong_tasks(new SubTasksDone(G1H_PS_NumElements)),
   _bot_shared(NULL),
   _evac_failure_scan_stack(NULL),
   _mark_in_progress(false),
   _cg1r(NULL),
   _g1mm(NULL),

@@ -1885,13 +1873,10 @@
   _gc_timer_cm(new (ResourceObj::C_HEAP, mtGC) ConcurrentGCTimer()),
   _gc_tracer_stw(new (ResourceObj::C_HEAP, mtGC) G1NewTracer()),
   _gc_tracer_cm(new (ResourceObj::C_HEAP, mtGC) G1OldTracer()) {
 
   _g1h = this;
-  if (_process_strong_tasks == NULL || !_process_strong_tasks->valid()) {
-    vm_exit_during_initialization("Failed necessary allocation.");
-  }
 
   _allocator = G1Allocator::create_allocator(_g1h);
   _humongous_object_threshold_in_words = HeapRegion::GrainWords / 2;
 
   int n_queues = MAX2((int)ParallelGCThreads, 1);

@@ -3289,15 +3274,16 @@
     // system dictionary, class loader data graph, the string table
     // and the nmethods in the code cache.
     G1VerifyCodeRootOopClosure codeRootsCl(this, &rootsCl, vo);
     G1VerifyCodeRootBlobClosure blobsCl(&codeRootsCl);
 
-    process_all_roots(true,            // activate StrongRootsScope
-                      SO_AllCodeCache, // roots scanning options
-                      &rootsCl,
+    {
+      G1RootProcessor root_processor(this);
+      root_processor.process_all_roots(&rootsCl,
                       &cldCl,
                       &blobsCl);
+    }
 
     bool failures = rootsCl.failures() || codeRootsCl.failures();
 
     if (vo != VerifyOption_G1UseMarkWord) {
       // If we're verifying during a full GC then the region sets

@@ -4580,75 +4566,27 @@
     }
     _count++;
   }
 };
 
-class G1CodeBlobClosure : public CodeBlobClosure {
-  class HeapRegionGatheringOopClosure : public OopClosure {
-    G1CollectedHeap* _g1h;
-    OopClosure* _work;
-    nmethod* _nm;
-
-    template <typename T>
-    void do_oop_work(T* p) {
-      _work->do_oop(p);
-      T oop_or_narrowoop = oopDesc::load_heap_oop(p);
-      if (!oopDesc::is_null(oop_or_narrowoop)) {
-        oop o = oopDesc::decode_heap_oop_not_null(oop_or_narrowoop);
-        HeapRegion* hr = _g1h->heap_region_containing_raw(o);
-        assert(!_g1h->obj_in_cs(o) || hr->rem_set()->strong_code_roots_list_contains(_nm), "if o still in CS then evacuation failed and nm must already be in the remset");
-        hr->add_strong_code_root(_nm);
-      }
-    }
-
-  public:
-    HeapRegionGatheringOopClosure(OopClosure* oc) : _g1h(G1CollectedHeap::heap()), _work(oc), _nm(NULL) {}
-
-    void do_oop(oop* o) {
-      do_oop_work(o);
-    }
-
-    void do_oop(narrowOop* o) {
-      do_oop_work(o);
-    }
-
-    void set_nm(nmethod* nm) {
-      _nm = nm;
-    }
-  };
-
-  HeapRegionGatheringOopClosure _oc;
-public:
-  G1CodeBlobClosure(OopClosure* oc) : _oc(oc) {}
-
-  void do_code_blob(CodeBlob* cb) {
-    nmethod* nm = cb->as_nmethod_or_null();
-    if (nm != NULL) {
-      if (!nm->test_set_oops_do_mark()) {
-        _oc.set_nm(nm);
-        nm->oops_do(&_oc);
-        nm->fix_oop_relocations();
-      }
-    }
-  }
-};
-
 class G1ParTask : public AbstractGangTask {
 protected:
   G1CollectedHeap*       _g1h;
   RefToScanQueueSet      *_queues;
+  G1RootProcessor*       _root_processor;
   ParallelTaskTerminator _terminator;
   uint _n_workers;
 
   Mutex _stats_lock;
   Mutex* stats_lock() { return &_stats_lock; }
 
 public:
-  G1ParTask(G1CollectedHeap* g1h, RefToScanQueueSet *task_queues)
+  G1ParTask(G1CollectedHeap* g1h, RefToScanQueueSet *task_queues, G1RootProcessor* root_processor)
     : AbstractGangTask("G1 collection"),
       _g1h(g1h),
       _queues(task_queues),
+      _root_processor(root_processor),
       _terminator(0, _queues),
       _stats_lock(Mutex::leaf, "parallel G1 stats lock", true)
   {}
 
   RefToScanQueueSet* queues() { return _queues; }

@@ -4658,17 +4596,11 @@
   }
 
   ParallelTaskTerminator* terminator() { return &_terminator; }
 
   virtual void set_for_termination(int active_workers) {
-    // This task calls set_n_termination() in par_non_clean_card_iterate_work()
-    // in the young space (_par_seq_tasks) in the G1 heap
-    // for SequentialSubTasksDone.
-    // This task also uses SubTasksDone in SharedHeap and G1CollectedHeap
-    // both of which need setting by set_n_termination().
-    _g1h->SharedHeap::set_n_termination(active_workers);
-    _g1h->set_n_termination(active_workers);
+    _root_processor->set_num_workers(active_workers);
     terminator()->reset_for_reuse(active_workers);
     _n_workers = active_workers;
   }
 
   // Helps out with CLD processing.

@@ -4733,52 +4665,49 @@
       G1ParCopyClosure<G1BarrierNone, G1MarkPromotedFromRoot> scan_mark_weak_root_cl(_g1h, &pss, rp);
       G1CLDClosure<G1MarkPromotedFromRoot>                    scan_mark_weak_cld_cl(&scan_mark_weak_root_cl,
                                                                                     false, // Process all klasses.
                                                                                     true); // Need to claim CLDs.
 
-      G1CodeBlobClosure scan_only_code_cl(&scan_only_root_cl);
-      G1CodeBlobClosure scan_mark_code_cl(&scan_mark_root_cl);
-      // IM Weak code roots are handled later.
-
       OopClosure* strong_root_cl;
       OopClosure* weak_root_cl;
       CLDClosure* strong_cld_cl;
       CLDClosure* weak_cld_cl;
-      CodeBlobClosure* strong_code_cl;
+
+      bool trace_metadata = false;
 
       if (_g1h->g1_policy()->during_initial_mark_pause()) {
         // We also need to mark copied objects.
         strong_root_cl = &scan_mark_root_cl;
         strong_cld_cl  = &scan_mark_cld_cl;
-        strong_code_cl = &scan_mark_code_cl;
         if (ClassUnloadingWithConcurrentMark) {
           weak_root_cl = &scan_mark_weak_root_cl;
           weak_cld_cl  = &scan_mark_weak_cld_cl;
+          trace_metadata = true;
         } else {
           weak_root_cl = &scan_mark_root_cl;
           weak_cld_cl  = &scan_mark_cld_cl;
         }
       } else {
         strong_root_cl = &scan_only_root_cl;
         weak_root_cl   = &scan_only_root_cl;
         strong_cld_cl  = &scan_only_cld_cl;
         weak_cld_cl    = &scan_only_cld_cl;
-        strong_code_cl = &scan_only_code_cl;
       }
 
-
-      G1ParPushHeapRSClosure  push_heap_rs_cl(_g1h, &pss);
-
       pss.start_strong_roots();
-      _g1h->g1_process_roots(strong_root_cl,
+
+      _root_processor->evacuate_roots(strong_root_cl,
                              weak_root_cl,
-                             &push_heap_rs_cl,
                              strong_cld_cl,
                              weak_cld_cl,
-                             strong_code_cl,
+                                      trace_metadata,
                              worker_id);
 
+      G1ParPushHeapRSClosure push_heap_rs_cl(_g1h, &pss);
+      _root_processor->scan_remembered_sets(&push_heap_rs_cl,
+                                            weak_root_cl,
+                                            worker_id);
       pss.end_strong_roots();
 
       {
         double start = os::elapsedTime();
         G1ParEvacuateFollowersClosure evac(_g1h, &pss, _queues, &_terminator);

@@ -4805,91 +4734,10 @@
     }
     _g1h->g1_policy()->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerEnd, worker_id, os::elapsedTime());
   }
 };
 
-// *** Common G1 Evacuation Stuff
-
-// This method is run in a GC worker.
-
-void
-G1CollectedHeap::
-g1_process_roots(OopClosure* scan_non_heap_roots,
-                 OopClosure* scan_non_heap_weak_roots,
-                 OopsInHeapRegionClosure* scan_rs,
-                 CLDClosure* scan_strong_clds,
-                 CLDClosure* scan_weak_clds,
-                 CodeBlobClosure* scan_strong_code,
-                 uint worker_i) {
-
-  // First scan the shared roots.
-  double ext_roots_start = os::elapsedTime();
-  double closure_app_time_sec = 0.0;
-
-  bool during_im = _g1h->g1_policy()->during_initial_mark_pause();
-  bool trace_metadata = during_im && ClassUnloadingWithConcurrentMark;
-
-  BufferingOopClosure buf_scan_non_heap_roots(scan_non_heap_roots);
-  BufferingOopClosure buf_scan_non_heap_weak_roots(scan_non_heap_weak_roots);
-
-  process_roots(false, // no scoping; this is parallel code
-                SharedHeap::SO_None,
-                &buf_scan_non_heap_roots,
-                &buf_scan_non_heap_weak_roots,
-                scan_strong_clds,
-                // Unloading Initial Marks handle the weak CLDs separately.
-                (trace_metadata ? NULL : scan_weak_clds),
-                scan_strong_code);
-
-  // Now the CM ref_processor roots.
-  if (!_process_strong_tasks->is_task_claimed(G1H_PS_refProcessor_oops_do)) {
-    // We need to treat the discovered reference lists of the
-    // concurrent mark ref processor as roots and keep entries
-    // (which are added by the marking threads) on them live
-    // until they can be processed at the end of marking.
-    ref_processor_cm()->weak_oops_do(&buf_scan_non_heap_roots);
-  }
-
-  if (trace_metadata) {
-    // Barrier to make sure all workers passed
-    // the strong CLD and strong nmethods phases.
-    active_strong_roots_scope()->wait_until_all_workers_done_with_threads(n_par_threads());
-
-    // Now take the complement of the strong CLDs.
-    ClassLoaderDataGraph::roots_cld_do(NULL, scan_weak_clds);
-  }
-
-  // Finish up any enqueued closure apps (attributed as object copy time).
-  buf_scan_non_heap_roots.done();
-  buf_scan_non_heap_weak_roots.done();
-
-  double obj_copy_time_sec = buf_scan_non_heap_roots.closure_app_seconds()
-      + buf_scan_non_heap_weak_roots.closure_app_seconds();
-
-  g1_policy()->phase_times()->record_time_secs(G1GCPhaseTimes::ObjCopy, worker_i, obj_copy_time_sec);
-
-  double ext_root_time_sec = os::elapsedTime() - ext_roots_start - obj_copy_time_sec;
-  g1_policy()->phase_times()->record_time_secs(G1GCPhaseTimes::ExtRootScan, worker_i, ext_root_time_sec);
-
-  // During conc marking we have to filter the per-thread SATB buffers
-  // to make sure we remove any oops into the CSet (which will show up
-  // as implicitly live).
-  {
-    G1GCParPhaseTimesTracker x(g1_policy()->phase_times(), G1GCPhaseTimes::SATBFiltering, worker_i);
-    if (!_process_strong_tasks->is_task_claimed(G1H_PS_filter_satb_buffers) && mark_in_progress()) {
-      JavaThread::satb_mark_queue_set().filter_thread_buffers();
-    }
-  }
-
-  // Now scan the complement of the collection set.
-  G1CodeBlobClosure scavenge_cs_nmethods(scan_non_heap_weak_roots);
-
-  g1_rem_set()->oops_into_collection_set_do(scan_rs, &scavenge_cs_nmethods, worker_i);
-
-  _process_strong_tasks->all_tasks_completed();
-}
-
 class G1StringSymbolTableUnlinkTask : public AbstractGangTask {
 private:
   BoolObjectClosure* _is_alive;
   int _initial_string_table_size;
   int _initial_symbol_table_size;

@@ -5873,22 +5721,22 @@
     assert(n_par_threads() == 0,
            "Should be the original non-parallel value");
     n_workers = 1;
   }
 
-  G1ParTask g1_par_task(this, _task_queues);
 
   init_for_evac_failure(NULL);
 
   rem_set()->prepare_for_younger_refs_iterate(true);
 
   assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty");
   double start_par_time_sec = os::elapsedTime();
   double end_par_time_sec;
 
   {
-    StrongRootsScope srs(this);
+    G1RootProcessor root_processor(this);
+    G1ParTask g1_par_task(this, _task_queues, &root_processor);
     // InitialMark needs claim bits to keep track of the marked-through CLDs.
     if (g1_policy()->during_initial_mark_pause()) {
       ClassLoaderDataGraph::clear_claimed_marks();
     }
 

@@ -5905,13 +5753,13 @@
       g1_par_task.work(0);
     }
     end_par_time_sec = os::elapsedTime();
 
     // Closing the inner scope will execute the destructor
-    // for the StrongRootsScope object. We record the current
+    // for the G1RootProcessor object. We record the current
     // elapsed time before closing the scope so that time
-    // taken for the SRS destructor is NOT included in the
+    // taken for the destructor is NOT included in the
     // reported parallel time.
   }
 
   G1GCPhaseTimes* phase_times = g1_policy()->phase_times();
 
< prev index next >