changeset 8281:38d6febe66af

8075210: Refactor strong root processing in order to allow G1 to evolve separately from GenCollectedHeap Summary: Create a G1RootProcessor and move SharedHeap root processing to GenCollectedHeap Reviewed-by: brutisso, tschatzl, ehelin
author mgerdin
date Mon, 01 Dec 2014 15:24:56 +0100
parents c3fcc09c9239
children 3ca53859c3c7
files src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.cpp src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp src/share/vm/gc_implementation/g1/g1CollectedHeap.hpp src/share/vm/gc_implementation/g1/g1MarkSweep.cpp src/share/vm/gc_implementation/g1/g1RemSet.cpp src/share/vm/gc_implementation/g1/g1RemSet.hpp src/share/vm/gc_implementation/g1/g1RootProcessor.cpp src/share/vm/gc_implementation/g1/g1RootProcessor.hpp src/share/vm/gc_implementation/parNew/parNewGeneration.cpp src/share/vm/memory/defNewGeneration.cpp src/share/vm/memory/genCollectedHeap.cpp src/share/vm/memory/genCollectedHeap.hpp src/share/vm/memory/genMarkSweep.cpp src/share/vm/memory/sharedHeap.cpp src/share/vm/memory/sharedHeap.hpp
diffstat 15 files changed, 646 insertions(+), 597 deletions(-) [+]
line wrap: on
line diff
--- a/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.cpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.cpp	Mon Dec 01 15:24:56 2014 +0100
@@ -598,7 +598,7 @@
   _collector_policy(cp),
   _should_unload_classes(CMSClassUnloadingEnabled),
   _concurrent_cycles_since_last_unload(0),
-  _roots_scanning_options(SharedHeap::SO_None),
+  _roots_scanning_options(GenCollectedHeap::SO_None),
   _inter_sweep_estimate(CMS_SweepWeight, CMS_SweepPadding),
   _intra_sweep_estimate(CMS_SweepWeight, CMS_SweepPadding),
   _gc_tracer_cm(new (ResourceObj::C_HEAP, mtGC) CMSTracer()),
@@ -3068,7 +3068,7 @@
   gch->gen_process_roots(_cmsGen->level(),
                          true,   // younger gens are roots
                          true,   // activate StrongRootsScope
-                         SharedHeap::ScanningOption(roots_scanning_options()),
+                         GenCollectedHeap::ScanningOption(roots_scanning_options()),
                          should_unload_classes(),
                          &notOlder,
                          NULL,
@@ -3136,7 +3136,7 @@
   gch->gen_process_roots(_cmsGen->level(),
                          true,   // younger gens are roots
                          true,   // activate StrongRootsScope
-                         SharedHeap::ScanningOption(roots_scanning_options()),
+                         GenCollectedHeap::ScanningOption(roots_scanning_options()),
                          should_unload_classes(),
                          &notOlder,
                          NULL,
@@ -3327,7 +3327,7 @@
 void CMSCollector::setup_cms_unloading_and_verification_state() {
   const  bool should_verify =   VerifyBeforeGC || VerifyAfterGC || VerifyDuringGC
                              || VerifyBeforeExit;
-  const  int  rso           =   SharedHeap::SO_AllCodeCache;
+  const  int  rso           =   GenCollectedHeap::SO_AllCodeCache;
 
   // We set the proper root for this CMS cycle here.
   if (should_unload_classes()) {   // Should unload classes this cycle
@@ -3753,7 +3753,7 @@
       gch->gen_process_roots(_cmsGen->level(),
                              true,   // younger gens are roots
                              true,   // activate StrongRootsScope
-                             SharedHeap::ScanningOption(roots_scanning_options()),
+                             GenCollectedHeap::ScanningOption(roots_scanning_options()),
                              should_unload_classes(),
                              &notOlder,
                              NULL,
@@ -5254,13 +5254,13 @@
   gch->gen_process_roots(_collector->_cmsGen->level(),
                          false,     // yg was scanned above
                          false,     // this is parallel code
-                         SharedHeap::ScanningOption(_collector->CMSCollector::roots_scanning_options()),
+                         GenCollectedHeap::ScanningOption(_collector->CMSCollector::roots_scanning_options()),
                          _collector->should_unload_classes(),
                          &par_mri_cl,
                          NULL,
                          &cld_closure);
   assert(_collector->should_unload_classes()
-         || (_collector->CMSCollector::roots_scanning_options() & SharedHeap::SO_AllCodeCache),
+         || (_collector->CMSCollector::roots_scanning_options() & GenCollectedHeap::SO_AllCodeCache),
          "if we didn't scan the code cache, we have to be ready to drop nmethods with expired weak oops");
   _timer.stop();
   if (PrintCMSStatistics != 0) {
@@ -5390,14 +5390,14 @@
   gch->gen_process_roots(_collector->_cmsGen->level(),
                          false,     // yg was scanned above
                          false,     // this is parallel code
-                         SharedHeap::ScanningOption(_collector->CMSCollector::roots_scanning_options()),
+                         GenCollectedHeap::ScanningOption(_collector->CMSCollector::roots_scanning_options()),
                          _collector->should_unload_classes(),
                          &par_mrias_cl,
                          NULL,
                          NULL);     // The dirty klasses will be handled below
 
   assert(_collector->should_unload_classes()
-         || (_collector->CMSCollector::roots_scanning_options() & SharedHeap::SO_AllCodeCache),
+         || (_collector->CMSCollector::roots_scanning_options() & GenCollectedHeap::SO_AllCodeCache),
          "if we didn't scan the code cache, we have to be ready to drop nmethods with expired weak oops");
   _timer.stop();
   if (PrintCMSStatistics != 0) {
@@ -5982,14 +5982,14 @@
     gch->gen_process_roots(_cmsGen->level(),
                            true,  // younger gens as roots
                            false, // use the local StrongRootsScope
-                           SharedHeap::ScanningOption(roots_scanning_options()),
+                           GenCollectedHeap::ScanningOption(roots_scanning_options()),
                            should_unload_classes(),
                            &mrias_cl,
                            NULL,
                            NULL); // The dirty klasses will be handled below
 
     assert(should_unload_classes()
-           || (roots_scanning_options() & SharedHeap::SO_AllCodeCache),
+           || (roots_scanning_options() & GenCollectedHeap::SO_AllCodeCache),
            "if we didn't scan the code cache, we have to be ready to drop nmethods with expired weak oops");
   }
 
--- a/src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp	Mon Dec 01 15:24:56 2014 +0100
@@ -46,6 +46,7 @@
 #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"
@@ -85,18 +86,6 @@
 // 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 {
@@ -1854,7 +1843,6 @@
   _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),
@@ -1888,9 +1876,6 @@
   _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;
@@ -3303,11 +3288,12 @@
     G1VerifyCodeRootOopClosure codeRootsCl(this, &rootsCl, vo);
     G1VerifyCodeRootBlobClosure blobsCl(&codeRootsCl);
 
-    process_all_roots(true,            // activate StrongRootsScope
-                      SO_AllCodeCache, // roots scanning options
-                      &rootsCl,
-                      &cldCl,
-                      &blobsCl);
+    {
+      G1RootProcessor root_processor(this);
+      root_processor.process_all_roots(&rootsCl,
+                                       &cldCl,
+                                       &blobsCl);
+    }
 
     bool failures = rootsCl.failures() || codeRootsCl.failures();
 
@@ -4543,60 +4529,11 @@
   }
 };
 
-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;
 
@@ -4604,10 +4541,11 @@
   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)
   {}
@@ -4621,13 +4559,7 @@
   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;
   }
@@ -4696,24 +4628,21 @@
                                                                                     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;
@@ -4723,21 +4652,21 @@
         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,
-                             weak_root_cl,
-                             &push_heap_rs_cl,
-                             strong_cld_cl,
-                             weak_cld_cl,
-                             strong_code_cl,
-                             worker_id);
-
+
+      _root_processor->evacuate_roots(strong_root_cl,
+                                      weak_root_cl,
+                                      strong_cld_cl,
+                                      weak_cld_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();
 
       {
@@ -4768,87 +4697,6 @@
   }
 };
 
-// *** 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,
-                 G1ParPushHeapRSClosure* 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;
@@ -5836,7 +5684,6 @@
     n_workers = 1;
   }
 
-  G1ParTask g1_par_task(this, _task_queues);
 
   init_for_evac_failure(NULL);
 
@@ -5847,7 +5694,8 @@
   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();
@@ -5868,9 +5716,9 @@
     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.
   }
 
--- a/src/share/vm/gc_implementation/g1/g1CollectedHeap.hpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/gc_implementation/g1/g1CollectedHeap.hpp	Mon Dec 01 15:24:56 2014 +0100
@@ -796,22 +796,6 @@
   // statistics or updating free lists.
   void abandon_collection_set(HeapRegion* cs_head);
 
-  // Applies "scan_non_heap_roots" to roots outside the heap,
-  // "scan_rs" to roots inside the heap (having done "set_region" to
-  // indicate the region in which the root resides),
-  // and does "scan_metadata" If "scan_rs" is
-  // NULL, then this step is skipped.  The "worker_i"
-  // param is for use with parallel roots processing, and should be
-  // the "i" of the calling parallel worker thread's work(i) function.
-  // In the sequential case this param will be ignored.
-  void g1_process_roots(OopClosure* scan_non_heap_roots,
-                        OopClosure* scan_non_heap_weak_roots,
-                        G1ParPushHeapRSClosure* scan_rs,
-                        CLDClosure* scan_strong_clds,
-                        CLDClosure* scan_weak_clds,
-                        CodeBlobClosure* scan_strong_code,
-                        uint worker_i);
-
   // The concurrent marker (and the thread it runs in.)
   ConcurrentMark* _cm;
   ConcurrentMarkThread* _cmThread;
@@ -998,21 +982,10 @@
   // of G1CollectedHeap::_gc_time_stamp.
   unsigned int* _worker_cset_start_region_time_stamp;
 
-  enum G1H_process_roots_tasks {
-    G1H_PS_filter_satb_buffers,
-    G1H_PS_refProcessor_oops_do,
-    // Leave this one last.
-    G1H_PS_NumElements
-  };
-
-  SubTasksDone* _process_strong_tasks;
-
   volatile bool _free_regions_coming;
 
 public:
 
-  SubTasksDone* process_strong_tasks() { return _process_strong_tasks; }
-
   void set_refine_cte_cl_concurrency(bool concurrent);
 
   RefToScanQueue *task_queue(int i) const;
@@ -1045,21 +1018,11 @@
   // Initialize weak reference processing.
   virtual void ref_processing_init();
 
-  void set_par_threads(uint t) {
-    SharedHeap::set_par_threads(t);
-    // Done in SharedHeap but oddly there are
-    // two _process_strong_tasks's in a G1CollectedHeap
-    // so do it here too.
-    _process_strong_tasks->set_n_threads(t);
-  }
-
+  // Explicitly import set_par_threads into this scope
+  using SharedHeap::set_par_threads;
   // Set _n_par_threads according to a policy TBD.
   void set_par_threads();
 
-  void set_n_termination(int t) {
-    _process_strong_tasks->set_n_threads(t);
-  }
-
   virtual CollectedHeap::Name kind() const {
     return CollectedHeap::G1CollectedHeap;
   }
--- a/src/share/vm/gc_implementation/g1/g1MarkSweep.cpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/gc_implementation/g1/g1MarkSweep.cpp	Mon Dec 01 15:24:56 2014 +0100
@@ -31,6 +31,7 @@
 #include "code/icBuffer.hpp"
 #include "gc_implementation/g1/g1Log.hpp"
 #include "gc_implementation/g1/g1MarkSweep.hpp"
+#include "gc_implementation/g1/g1RootProcessor.hpp"
 #include "gc_implementation/g1/g1StringDedup.hpp"
 #include "gc_implementation/shared/gcHeapSummary.hpp"
 #include "gc_implementation/shared/gcTimer.hpp"
@@ -126,21 +127,22 @@
   GCTraceTime tm("phase 1", G1Log::fine() && Verbose, true, gc_timer(), gc_tracer()->gc_id());
   GenMarkSweep::trace(" 1");
 
-  SharedHeap* sh = SharedHeap::heap();
+  G1CollectedHeap* g1h = G1CollectedHeap::heap();
 
   // Need cleared claim bits for the roots processing
   ClassLoaderDataGraph::clear_claimed_marks();
 
   MarkingCodeBlobClosure follow_code_closure(&GenMarkSweep::follow_root_closure, !CodeBlobToOopClosure::FixRelocations);
-  sh->process_strong_roots(true,   // activate StrongRootsScope
-                           SharedHeap::SO_None,
-                           &GenMarkSweep::follow_root_closure,
-                           &GenMarkSweep::follow_cld_closure,
-                           &follow_code_closure);
+  {
+    G1RootProcessor root_processor(g1h);
+    root_processor.process_strong_roots(&GenMarkSweep::follow_root_closure,
+                                        &GenMarkSweep::follow_cld_closure,
+                                        &follow_code_closure);
+  }
 
   // Process reference objects found during marking
   ReferenceProcessor* rp = GenMarkSweep::ref_processor();
-  assert(rp == G1CollectedHeap::heap()->ref_processor_stw(), "Sanity");
+  assert(rp == g1h->ref_processor_stw(), "Sanity");
 
   rp->setup_policy(clear_all_softrefs);
   const ReferenceProcessorStats& stats =
@@ -226,6 +228,12 @@
   }
 };
 
+class G1AlwaysTrueClosure: public BoolObjectClosure {
+public:
+  bool do_object_b(oop p) { return true; }
+};
+static G1AlwaysTrueClosure always_true;
+
 void G1MarkSweep::mark_sweep_phase3() {
   G1CollectedHeap* g1h = G1CollectedHeap::heap();
 
@@ -233,24 +241,23 @@
   GCTraceTime tm("phase 3", G1Log::fine() && Verbose, true, gc_timer(), gc_tracer()->gc_id());
   GenMarkSweep::trace("3");
 
-  SharedHeap* sh = SharedHeap::heap();
-
   // Need cleared claim bits for the roots processing
   ClassLoaderDataGraph::clear_claimed_marks();
 
   CodeBlobToOopClosure adjust_code_closure(&GenMarkSweep::adjust_pointer_closure, CodeBlobToOopClosure::FixRelocations);
-  sh->process_all_roots(true,  // activate StrongRootsScope
-                        SharedHeap::SO_AllCodeCache,
-                        &GenMarkSweep::adjust_pointer_closure,
-                        &GenMarkSweep::adjust_cld_closure,
-                        &adjust_code_closure);
+  {
+    G1RootProcessor root_processor(g1h);
+    root_processor.process_all_roots(&GenMarkSweep::adjust_pointer_closure,
+                                     &GenMarkSweep::adjust_cld_closure,
+                                     &adjust_code_closure);
+  }
 
   assert(GenMarkSweep::ref_processor() == g1h->ref_processor_stw(), "Sanity");
   g1h->ref_processor_stw()->weak_oops_do(&GenMarkSweep::adjust_pointer_closure);
 
   // Now adjust pointers in remaining weak roots.  (All of which should
   // have been cleared if they pointed to non-surviving objects.)
-  sh->process_weak_roots(&GenMarkSweep::adjust_pointer_closure);
+  JNIHandles::weak_oops_do(&always_true, &GenMarkSweep::adjust_pointer_closure);
 
   if (G1StringDedup::is_enabled()) {
     G1StringDedup::oops_do(&GenMarkSweep::adjust_pointer_closure);
--- a/src/share/vm/gc_implementation/g1/g1RemSet.cpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/gc_implementation/g1/g1RemSet.cpp	Mon Dec 01 15:24:56 2014 +0100
@@ -78,7 +78,6 @@
     _cards_scanned(NULL), _total_cards_scanned(0),
     _prev_period_summary()
 {
-  _seq_task = new SubTasksDone(NumSeqTasks);
   guarantee(n_workers() > 0, "There should be some workers");
   _cset_rs_update_cl = NEW_C_HEAP_ARRAY(G1ParPushHeapRSClosure*, n_workers(), mtGC);
   for (uint i = 0; i < n_workers(); i++) {
@@ -90,7 +89,6 @@
 }
 
 G1RemSet::~G1RemSet() {
-  delete _seq_task;
   for (uint i = 0; i < n_workers(); i++) {
     assert(_cset_rs_update_cl[i] == NULL, "it should be");
   }
--- a/src/share/vm/gc_implementation/g1/g1RemSet.hpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/gc_implementation/g1/g1RemSet.hpp	Mon Dec 01 15:24:56 2014 +0100
@@ -59,7 +59,6 @@
   };
 
   CardTableModRefBS*     _ct_bs;
-  SubTasksDone*          _seq_task;
   G1CollectorPolicy*     _g1p;
 
   ConcurrentG1Refine*    _cg1r;
--- /dev/null	Thu Jan 01 00:00:00 1970 +0000
+++ b/src/share/vm/gc_implementation/g1/g1RootProcessor.cpp	Mon Dec 01 15:24:56 2014 +0100
@@ -0,0 +1,290 @@
+/*
+ * Copyright (c) 2015, Oracle and/or its affiliates. All rights reserved.
+ * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
+ *
+ * This code is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This code is distributed in the hope that it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License
+ * version 2 for more details (a copy is included in the LICENSE file that
+ * accompanied this code).
+ *
+ * You should have received a copy of the GNU General Public License version
+ * 2 along with this work; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
+ *
+ * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
+ * or visit www.oracle.com if you need additional information or have any
+ * questions.
+ *
+ */
+
+#include "precompiled.hpp"
+
+#include "classfile/symbolTable.hpp"
+#include "classfile/systemDictionary.hpp"
+#include "code/codeCache.hpp"
+#include "gc_implementation/g1/bufferingOopClosure.hpp"
+#include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
+#include "gc_implementation/g1/g1CollectorPolicy.hpp"
+#include "gc_implementation/g1/g1GCPhaseTimes.hpp"
+#include "gc_implementation/g1/g1RemSet.inline.hpp"
+#include "gc_implementation/g1/g1RootProcessor.hpp"
+#include "memory/allocation.inline.hpp"
+#include "runtime/fprofiler.hpp"
+#include "runtime/mutex.hpp"
+#include "services/management.hpp"
+
+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();
+      }
+    }
+  }
+};
+
+
+void G1RootProcessor::worker_has_discovered_all_strong_classes() {
+  uint n_workers = _g1h->n_par_threads();
+  assert(ClassUnloadingWithConcurrentMark, "Currently only needed when doing G1 Class Unloading");
+
+  uint new_value = (uint)Atomic::add(1, &_n_workers_discovered_strong_classes);
+  if (new_value == n_workers) {
+    // This thread is last. Notify the others.
+    MonitorLockerEx ml(&_lock, Mutex::_no_safepoint_check_flag);
+    _lock.notify_all();
+  }
+}
+
+void G1RootProcessor::wait_until_all_strong_classes_discovered() {
+  uint n_workers = _g1h->n_par_threads();
+  assert(ClassUnloadingWithConcurrentMark, "Currently only needed when doing G1 Class Unloading");
+
+  if ((uint)_n_workers_discovered_strong_classes != n_workers) {
+    MonitorLockerEx ml(&_lock, Mutex::_no_safepoint_check_flag);
+    while ((uint)_n_workers_discovered_strong_classes != n_workers) {
+      _lock.wait(Mutex::_no_safepoint_check_flag, 0, false);
+    }
+  }
+}
+
+G1RootProcessor::G1RootProcessor(G1CollectedHeap* g1h) :
+    _g1h(g1h),
+    _process_strong_tasks(new SubTasksDone(G1RP_PS_NumElements)),
+    _srs(g1h),
+    _lock(Mutex::leaf, "G1 Root Scanning barrier lock", false),
+    _n_workers_discovered_strong_classes(0) {}
+
+void G1RootProcessor::evacuate_roots(OopClosure* scan_non_heap_roots,
+                                     OopClosure* scan_non_heap_weak_roots,
+                                     CLDClosure* scan_strong_clds,
+                                     CLDClosure* scan_weak_clds,
+                                     bool trace_metadata,
+                                     uint worker_i) {
+  // First scan the shared roots.
+  double ext_roots_start = os::elapsedTime();
+
+  BufferingOopClosure buf_scan_non_heap_roots(scan_non_heap_roots);
+  BufferingOopClosure buf_scan_non_heap_weak_roots(scan_non_heap_weak_roots);
+
+  OopClosure* const weak_roots = &buf_scan_non_heap_weak_roots;
+  OopClosure* const strong_roots = &buf_scan_non_heap_roots;
+
+  // CodeBlobClosures are not interoperable with BufferingOopClosures
+  G1CodeBlobClosure root_code_blobs(scan_non_heap_roots);
+
+  process_java_roots(strong_roots,
+                     trace_metadata ? scan_strong_clds : NULL,
+                     scan_strong_clds,
+                     trace_metadata ? NULL : scan_weak_clds,
+                     &root_code_blobs);
+
+  // This is the point where this worker thread will not find more strong CLDs/nmethods.
+  // Report this so G1 can synchronize the strong and weak CLDs/nmethods processing.
+  if (trace_metadata) {
+    worker_has_discovered_all_strong_classes();
+  }
+
+  process_vm_roots(strong_roots, weak_roots);
+
+  // Now the CM ref_processor roots.
+  if (!_process_strong_tasks->is_task_claimed(G1RP_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.
+    _g1h->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.
+    wait_until_all_strong_classes_discovered();
+
+    // 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();
+
+  G1GCPhaseTimes* phase_times = _g1h->g1_policy()->phase_times();
+  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;
+
+  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(phase_times, G1GCPhaseTimes::SATBFiltering, worker_i);
+    if (!_process_strong_tasks->is_task_claimed(G1RP_PS_filter_satb_buffers) && _g1h->mark_in_progress()) {
+      JavaThread::satb_mark_queue_set().filter_thread_buffers();
+    }
+  }
+
+  _process_strong_tasks->all_tasks_completed();
+}
+
+void G1RootProcessor::process_strong_roots(OopClosure* oops,
+                                           CLDClosure* clds,
+                                           CodeBlobClosure* blobs) {
+
+  process_java_roots(oops, clds, clds, NULL, blobs);
+  process_vm_roots(oops, NULL);
+
+  _process_strong_tasks->all_tasks_completed();
+}
+
+void G1RootProcessor::process_all_roots(OopClosure* oops,
+                                        CLDClosure* clds,
+                                        CodeBlobClosure* blobs) {
+
+  process_java_roots(oops, NULL, clds, clds, NULL);
+  process_vm_roots(oops, oops);
+
+  if (!_process_strong_tasks->is_task_claimed(G1RP_PS_CodeCache_oops_do)) {
+    CodeCache::blobs_do(blobs);
+  }
+
+  _process_strong_tasks->all_tasks_completed();
+}
+
+void G1RootProcessor::process_java_roots(OopClosure* strong_roots,
+                                         CLDClosure* thread_stack_clds,
+                                         CLDClosure* strong_clds,
+                                         CLDClosure* weak_clds,
+                                         CodeBlobClosure* strong_code) {
+  assert(thread_stack_clds == NULL || weak_clds == NULL, "There is overlap between those, only one may be set");
+  // Iterating over the CLDG and the Threads are done early to allow us to
+  // first process the strong CLDs and nmethods and then, after a barrier,
+  // let the thread process the weak CLDs and nmethods.
+
+  if (!_process_strong_tasks->is_task_claimed(G1RP_PS_ClassLoaderDataGraph_oops_do)) {
+    ClassLoaderDataGraph::roots_cld_do(strong_clds, weak_clds);
+  }
+
+  Threads::possibly_parallel_oops_do(strong_roots, thread_stack_clds, strong_code);
+}
+
+void G1RootProcessor::process_vm_roots(OopClosure* strong_roots,
+                                       OopClosure* weak_roots) {
+
+  if (!_process_strong_tasks->is_task_claimed(G1RP_PS_Universe_oops_do)) {
+    Universe::oops_do(strong_roots);
+  }
+
+  if (!_process_strong_tasks->is_task_claimed(G1RP_PS_JNIHandles_oops_do)) {
+    JNIHandles::oops_do(strong_roots);
+  }
+
+  if (!_process_strong_tasks-> is_task_claimed(G1RP_PS_ObjectSynchronizer_oops_do)) {
+    ObjectSynchronizer::oops_do(strong_roots);
+  }
+
+  if (!_process_strong_tasks->is_task_claimed(G1RP_PS_FlatProfiler_oops_do)) {
+    FlatProfiler::oops_do(strong_roots);
+  }
+
+  if (!_process_strong_tasks->is_task_claimed(G1RP_PS_Management_oops_do)) {
+    Management::oops_do(strong_roots);
+  }
+
+  if (!_process_strong_tasks->is_task_claimed(G1RP_PS_jvmti_oops_do)) {
+    JvmtiExport::oops_do(strong_roots);
+  }
+
+  if (!_process_strong_tasks->is_task_claimed(G1RP_PS_SystemDictionary_oops_do)) {
+    SystemDictionary::roots_oops_do(strong_roots, weak_roots);
+  }
+
+  // All threads execute the following. A specific chunk of buckets
+  // from the StringTable are the individual tasks.
+  if (weak_roots != NULL) {
+    StringTable::possibly_parallel_oops_do(weak_roots);
+  }
+}
+
+void G1RootProcessor::scan_remembered_sets(G1ParPushHeapRSClosure* scan_rs,
+                                           OopClosure* scan_non_heap_weak_roots,
+                                           uint worker_i) {
+  // Now scan the complement of the collection set.
+  G1CodeBlobClosure scavenge_cs_nmethods(scan_non_heap_weak_roots);
+
+  _g1h->g1_rem_set()->oops_into_collection_set_do(scan_rs, &scavenge_cs_nmethods, worker_i);
+}
+
+void G1RootProcessor::set_num_workers(int active_workers) {
+  _process_strong_tasks->set_n_threads(active_workers);
+}
--- /dev/null	Thu Jan 01 00:00:00 1970 +0000
+++ b/src/share/vm/gc_implementation/g1/g1RootProcessor.hpp	Mon Dec 01 15:24:56 2014 +0100
@@ -0,0 +1,116 @@
+/*
+ * Copyright (c) 2015, Oracle and/or its affiliates. All rights reserved.
+ * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
+ *
+ * This code is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This code is distributed in the hope that it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License
+ * version 2 for more details (a copy is included in the LICENSE file that
+ * accompanied this code).
+ *
+ * You should have received a copy of the GNU General Public License version
+ * 2 along with this work; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
+ *
+ * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
+ * or visit www.oracle.com if you need additional information or have any
+ * questions.
+ *
+ */
+
+#ifndef SHARE_VM_GC_IMPLEMENTATION_G1_ROOTPROCESSOR_HPP
+#define SHARE_VM_GC_IMPLEMENTATION_G1_ROOTPROCESSOR_HPP
+
+#include "memory/allocation.hpp"
+#include "memory/sharedHeap.hpp"
+#include "runtime/mutex.hpp"
+
+class CLDClosure;
+class CodeBlobClosure;
+class G1CollectedHeap;
+class G1ParPushHeapRSClosure;
+class Monitor;
+class OopClosure;
+class SubTasksDone;
+
+// Scoped object to assist in applying oop, CLD and code blob closures to
+// root locations. Handles claiming of different root scanning tasks
+// and takes care of global state for root scanning via a StrongRootsScope.
+// In the parallel case there is a shared G1RootProcessor object where all
+// worker thread call the process_roots methods.
+class G1RootProcessor : public StackObj {
+  G1CollectedHeap* _g1h;
+  SubTasksDone* _process_strong_tasks;
+  SharedHeap::StrongRootsScope _srs;
+
+  // Used to implement the Thread work barrier.
+  Monitor _lock;
+  volatile jint _n_workers_discovered_strong_classes;
+
+  enum G1H_process_roots_tasks {
+    G1RP_PS_Universe_oops_do,
+    G1RP_PS_JNIHandles_oops_do,
+    G1RP_PS_ObjectSynchronizer_oops_do,
+    G1RP_PS_FlatProfiler_oops_do,
+    G1RP_PS_Management_oops_do,
+    G1RP_PS_SystemDictionary_oops_do,
+    G1RP_PS_ClassLoaderDataGraph_oops_do,
+    G1RP_PS_jvmti_oops_do,
+    G1RP_PS_CodeCache_oops_do,
+    G1RP_PS_filter_satb_buffers,
+    G1RP_PS_refProcessor_oops_do,
+    // Leave this one last.
+    G1RP_PS_NumElements
+  };
+
+  void worker_has_discovered_all_strong_classes();
+  void wait_until_all_strong_classes_discovered();
+
+  void process_java_roots(OopClosure* scan_non_heap_roots,
+                          CLDClosure* thread_stack_clds,
+                          CLDClosure* scan_strong_clds,
+                          CLDClosure* scan_weak_clds,
+                          CodeBlobClosure* scan_strong_code);
+
+  void process_vm_roots(OopClosure* scan_non_heap_roots,
+                        OopClosure* scan_non_heap_weak_roots);
+
+public:
+  G1RootProcessor(G1CollectedHeap* g1h);
+
+  // Apply closures to the strongly and weakly reachable roots in the system
+  // in a single pass.
+  // Record and report timing measurements for sub phases using the worker_i
+  void evacuate_roots(OopClosure* scan_non_heap_roots,
+                      OopClosure* scan_non_heap_weak_roots,
+                      CLDClosure* scan_strong_clds,
+                      CLDClosure* scan_weak_clds,
+                      bool trace_metadata,
+                      uint worker_i);
+
+  // Apply oops, clds and blobs to all strongly reachable roots in the system
+  void process_strong_roots(OopClosure* oops,
+                            CLDClosure* clds,
+                            CodeBlobClosure* blobs);
+
+  // Apply oops, clds and blobs to strongly and weakly reachable roots in the system
+  void process_all_roots(OopClosure* oops,
+                         CLDClosure* clds,
+                         CodeBlobClosure* blobs);
+
+  // Apply scan_rs to all locations in the union of the remembered sets for all
+  // regions in the collection set
+  // (having done "set_region" to indicate the region in which the root resides),
+  void scan_remembered_sets(G1ParPushHeapRSClosure* scan_rs,
+                            OopClosure* scan_non_heap_weak_roots,
+                            uint worker_i);
+
+  // Inform the root processor about the number of worker threads
+  void set_num_workers(int active_workers);
+};
+
+#endif // SHARE_VM_GC_IMPLEMENTATION_G1_ROOTPROCESSOR_HPP
--- a/src/share/vm/gc_implementation/parNew/parNewGeneration.cpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/gc_implementation/parNew/parNewGeneration.cpp	Mon Dec 01 15:24:56 2014 +0100
@@ -622,7 +622,7 @@
                          true,  // Process younger gens, if any,
                                 // as strong roots.
                          false, // no scope; this is parallel code
-                         SharedHeap::SO_ScavengeCodeCache,
+                         GenCollectedHeap::SO_ScavengeCodeCache,
                          GenCollectedHeap::StrongAndWeakRoots,
                          &par_scan_state.to_space_root_closure(),
                          &par_scan_state.older_gen_closure(),
--- a/src/share/vm/memory/defNewGeneration.cpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/memory/defNewGeneration.cpp	Mon Dec 01 15:24:56 2014 +0100
@@ -629,7 +629,7 @@
                          true,  // Process younger gens, if any,
                                 // as strong roots.
                          true,  // activate StrongRootsScope
-                         SharedHeap::SO_ScavengeCodeCache,
+                         GenCollectedHeap::SO_ScavengeCodeCache,
                          GenCollectedHeap::StrongAndWeakRoots,
                          &fsc_with_no_gc_barrier,
                          &fsc_with_gc_barrier,
--- a/src/share/vm/memory/genCollectedHeap.cpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/memory/genCollectedHeap.cpp	Mon Dec 01 15:24:56 2014 +0100
@@ -26,6 +26,7 @@
 #include "classfile/symbolTable.hpp"
 #include "classfile/systemDictionary.hpp"
 #include "classfile/vmSymbols.hpp"
+#include "code/codeCache.hpp"
 #include "code/icBuffer.hpp"
 #include "gc_implementation/shared/collectorCounters.hpp"
 #include "gc_implementation/shared/gcTrace.hpp"
@@ -49,6 +50,7 @@
 #include "runtime/handles.inline.hpp"
 #include "runtime/java.hpp"
 #include "runtime/vmThread.hpp"
+#include "services/management.hpp"
 #include "services/memoryService.hpp"
 #include "utilities/vmError.hpp"
 #include "utilities/workgroup.hpp"
@@ -63,7 +65,15 @@
 
 // The set of potentially parallel tasks in root scanning.
 enum GCH_strong_roots_tasks {
-  // We probably want to parallelize both of these internally, but for now...
+  GCH_PS_Universe_oops_do,
+  GCH_PS_JNIHandles_oops_do,
+  GCH_PS_ObjectSynchronizer_oops_do,
+  GCH_PS_FlatProfiler_oops_do,
+  GCH_PS_Management_oops_do,
+  GCH_PS_SystemDictionary_oops_do,
+  GCH_PS_ClassLoaderDataGraph_oops_do,
+  GCH_PS_jvmti_oops_do,
+  GCH_PS_CodeCache_oops_do,
   GCH_PS_younger_gens,
   // Leave this one last.
   GCH_PS_NumElements
@@ -72,13 +82,9 @@
 GenCollectedHeap::GenCollectedHeap(GenCollectorPolicy *policy) :
   SharedHeap(policy),
   _gen_policy(policy),
-  _gen_process_roots_tasks(new SubTasksDone(GCH_PS_NumElements)),
+  _process_strong_tasks(new SubTasksDone(GCH_PS_NumElements)),
   _full_collections_completed(0)
 {
-  if (_gen_process_roots_tasks == NULL ||
-      !_gen_process_roots_tasks->valid()) {
-    vm_exit_during_initialization("Failed necessary allocation.");
-  }
   assert(policy != NULL, "Sanity check");
 }
 
@@ -589,29 +595,137 @@
 
 void GenCollectedHeap::set_par_threads(uint t) {
   SharedHeap::set_par_threads(t);
-  _gen_process_roots_tasks->set_n_threads(t);
+  set_n_termination(t);
+}
+
+void GenCollectedHeap::set_n_termination(uint t) {
+  _process_strong_tasks->set_n_threads(t);
 }
 
-void GenCollectedHeap::
-gen_process_roots(int level,
-                  bool younger_gens_as_roots,
-                  bool activate_scope,
-                  SharedHeap::ScanningOption so,
-                  OopsInGenClosure* not_older_gens,
-                  OopsInGenClosure* weak_roots,
-                  OopsInGenClosure* older_gens,
-                  CLDClosure* cld_closure,
-                  CLDClosure* weak_cld_closure,
-                  CodeBlobClosure* code_closure) {
+#ifdef ASSERT
+class AssertNonScavengableClosure: public OopClosure {
+public:
+  virtual void do_oop(oop* p) {
+    assert(!Universe::heap()->is_in_partial_collection(*p),
+      "Referent should not be scavengable.");  }
+  virtual void do_oop(narrowOop* p) { ShouldNotReachHere(); }
+};
+static AssertNonScavengableClosure assert_is_non_scavengable_closure;
+#endif
+
+void GenCollectedHeap::process_roots(bool activate_scope,
+                                     ScanningOption so,
+                                     OopClosure* strong_roots,
+                                     OopClosure* weak_roots,
+                                     CLDClosure* strong_cld_closure,
+                                     CLDClosure* weak_cld_closure,
+                                     CodeBlobClosure* code_roots) {
+  StrongRootsScope srs(this, activate_scope);
 
   // General roots.
-  SharedHeap::process_roots(activate_scope, so,
-                            not_older_gens, weak_roots,
-                            cld_closure, weak_cld_closure,
-                            code_closure);
+  assert(_strong_roots_parity != 0, "must have called prologue code");
+  assert(code_roots != NULL, "code root closure should always be set");
+  // _n_termination for _process_strong_tasks should be set up stream
+  // in a method not running in a GC worker.  Otherwise the GC worker
+  // could be trying to change the termination condition while the task
+  // is executing in another GC worker.
+
+  if (!_process_strong_tasks->is_task_claimed(GCH_PS_ClassLoaderDataGraph_oops_do)) {
+    ClassLoaderDataGraph::roots_cld_do(strong_cld_closure, weak_cld_closure);
+  }
+
+  // Some CLDs contained in the thread frames should be considered strong.
+  // Don't process them if they will be processed during the ClassLoaderDataGraph phase.
+  CLDClosure* roots_from_clds_p = (strong_cld_closure != weak_cld_closure) ? strong_cld_closure : NULL;
+  // Only process code roots from thread stacks if we aren't visiting the entire CodeCache anyway
+  CodeBlobClosure* roots_from_code_p = (so & SO_AllCodeCache) ? NULL : code_roots;
+
+  Threads::possibly_parallel_oops_do(strong_roots, roots_from_clds_p, roots_from_code_p);
+
+  if (!_process_strong_tasks->is_task_claimed(GCH_PS_Universe_oops_do)) {
+    Universe::oops_do(strong_roots);
+  }
+  // Global (strong) JNI handles
+  if (!_process_strong_tasks->is_task_claimed(GCH_PS_JNIHandles_oops_do)) {
+    JNIHandles::oops_do(strong_roots);
+  }
+
+  if (!_process_strong_tasks->is_task_claimed(GCH_PS_ObjectSynchronizer_oops_do)) {
+    ObjectSynchronizer::oops_do(strong_roots);
+  }
+  if (!_process_strong_tasks->is_task_claimed(GCH_PS_FlatProfiler_oops_do)) {
+    FlatProfiler::oops_do(strong_roots);
+  }
+  if (!_process_strong_tasks->is_task_claimed(GCH_PS_Management_oops_do)) {
+    Management::oops_do(strong_roots);
+  }
+  if (!_process_strong_tasks->is_task_claimed(GCH_PS_jvmti_oops_do)) {
+    JvmtiExport::oops_do(strong_roots);
+  }
+
+  if (!_process_strong_tasks->is_task_claimed(GCH_PS_SystemDictionary_oops_do)) {
+    SystemDictionary::roots_oops_do(strong_roots, weak_roots);
+  }
+
+  // All threads execute the following. A specific chunk of buckets
+  // from the StringTable are the individual tasks.
+  if (weak_roots != NULL) {
+    if (CollectedHeap::use_parallel_gc_threads()) {
+      StringTable::possibly_parallel_oops_do(weak_roots);
+    } else {
+      StringTable::oops_do(weak_roots);
+    }
+  }
+
+  if (!_process_strong_tasks->is_task_claimed(GCH_PS_CodeCache_oops_do)) {
+    if (so & SO_ScavengeCodeCache) {
+      assert(code_roots != NULL, "must supply closure for code cache");
+
+      // We only visit parts of the CodeCache when scavenging.
+      CodeCache::scavenge_root_nmethods_do(code_roots);
+    }
+    if (so & SO_AllCodeCache) {
+      assert(code_roots != NULL, "must supply closure for code cache");
+
+      // CMSCollector uses this to do intermediate-strength collections.
+      // We scan the entire code cache, since CodeCache::do_unloading is not called.
+      CodeCache::blobs_do(code_roots);
+    }
+    // Verify that the code cache contents are not subject to
+    // movement by a scavenging collection.
+    DEBUG_ONLY(CodeBlobToOopClosure assert_code_is_non_scavengable(&assert_is_non_scavengable_closure, !CodeBlobToOopClosure::FixRelocations));
+    DEBUG_ONLY(CodeCache::asserted_non_scavengable_nmethods_do(&assert_code_is_non_scavengable));
+  }
+
+}
+
+void GenCollectedHeap::gen_process_roots(int level,
+                                         bool younger_gens_as_roots,
+                                         bool activate_scope,
+                                         ScanningOption so,
+                                         bool only_strong_roots,
+                                         OopsInGenClosure* not_older_gens,
+                                         OopsInGenClosure* older_gens,
+                                         CLDClosure* cld_closure) {
+  const bool is_adjust_phase = !only_strong_roots && !younger_gens_as_roots;
+
+  bool is_moving_collection = false;
+  if (level == 0 || is_adjust_phase) {
+    // young collections are always moving
+    is_moving_collection = true;
+  }
+
+  MarkingCodeBlobClosure mark_code_closure(not_older_gens, is_moving_collection);
+  OopsInGenClosure* weak_roots = only_strong_roots ? NULL : not_older_gens;
+  CLDClosure* weak_cld_closure = only_strong_roots ? NULL : cld_closure;
+
+  process_roots(activate_scope, so,
+                not_older_gens, weak_roots,
+                cld_closure, weak_cld_closure,
+                &mark_code_closure);
 
   if (younger_gens_as_roots) {
-    if (!_gen_process_roots_tasks->is_task_claimed(GCH_PS_younger_gens)) {
+    if (!_process_strong_tasks->is_task_claimed(GCH_PS_younger_gens)) {
       for (int i = 0; i < level; i++) {
         not_older_gens->set_generation(_gens[i]);
         _gens[i]->oop_iterate(not_older_gens);
@@ -627,43 +741,18 @@
     older_gens->reset_generation();
   }
 
-  _gen_process_roots_tasks->all_tasks_completed();
+  _process_strong_tasks->all_tasks_completed();
 }
 
-void GenCollectedHeap::
-gen_process_roots(int level,
-                  bool younger_gens_as_roots,
-                  bool activate_scope,
-                  SharedHeap::ScanningOption so,
-                  bool only_strong_roots,
-                  OopsInGenClosure* not_older_gens,
-                  OopsInGenClosure* older_gens,
-                  CLDClosure* cld_closure) {
 
-  const bool is_adjust_phase = !only_strong_roots && !younger_gens_as_roots;
-
-  bool is_moving_collection = false;
-  if (level == 0 || is_adjust_phase) {
-    // young collections are always moving
-    is_moving_collection = true;
-  }
-
-  MarkingCodeBlobClosure mark_code_closure(not_older_gens, is_moving_collection);
-  CodeBlobClosure* code_closure = &mark_code_closure;
-
-  gen_process_roots(level,
-                    younger_gens_as_roots,
-                    activate_scope, so,
-                    not_older_gens, only_strong_roots ? NULL : not_older_gens,
-                    older_gens,
-                    cld_closure, only_strong_roots ? NULL : cld_closure,
-                    code_closure);
-
-}
+class AlwaysTrueClosure: public BoolObjectClosure {
+public:
+  bool do_object_b(oop p) { return true; }
+};
+static AlwaysTrueClosure always_true;
 
 void GenCollectedHeap::gen_process_weak_roots(OopClosure* root_closure) {
-  SharedHeap::process_weak_roots(root_closure);
-  // "Local" "weak" refs
+  JNIHandles::weak_oops_do(&always_true, root_closure);
   for (int i = 0; i < _n_gens; i++) {
     _gens[i]->ref_processor()->weak_oops_do(root_closure);
   }
--- a/src/share/vm/memory/genCollectedHeap.hpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/memory/genCollectedHeap.hpp	Mon Dec 01 15:24:56 2014 +0100
@@ -79,8 +79,7 @@
 
   // Data structure for claiming the (potentially) parallel tasks in
   // (gen-specific) roots processing.
-  SubTasksDone* _gen_process_roots_tasks;
-  SubTasksDone* gen_process_roots_tasks() { return _gen_process_roots_tasks; }
+  SubTasksDone* _process_strong_tasks;
 
   // In block contents verification, the number of header words to skip
   NOT_PRODUCT(static size_t _skip_header_HeapWords;)
@@ -390,6 +389,7 @@
   static GenCollectedHeap* heap();
 
   void set_par_threads(uint t);
+  void set_n_termination(uint t);
 
   // Invoke the "do_oop" method of one of the closures "not_older_gens"
   // or "older_gens" on root locations for the generation at
@@ -403,11 +403,25 @@
   // The "so" argument determines which of the roots
   // the closure is applied to:
   // "SO_None" does none;
+  enum ScanningOption {
+    SO_None                =  0x0,
+    SO_AllCodeCache        =  0x8,
+    SO_ScavengeCodeCache   = 0x10
+  };
+
  private:
+  void process_roots(bool activate_scope,
+                     ScanningOption so,
+                     OopClosure* strong_roots,
+                     OopClosure* weak_roots,
+                     CLDClosure* strong_cld_closure,
+                     CLDClosure* weak_cld_closure,
+                     CodeBlobClosure* code_roots);
+
   void gen_process_roots(int level,
                          bool younger_gens_as_roots,
                          bool activate_scope,
-                         SharedHeap::ScanningOption so,
+                         ScanningOption so,
                          OopsInGenClosure* not_older_gens,
                          OopsInGenClosure* weak_roots,
                          OopsInGenClosure* older_gens,
@@ -422,7 +436,7 @@
   void gen_process_roots(int level,
                          bool younger_gens_as_roots,
                          bool activate_scope,
-                         SharedHeap::ScanningOption so,
+                         ScanningOption so,
                          bool only_strong_roots,
                          OopsInGenClosure* not_older_gens,
                          OopsInGenClosure* older_gens,
--- a/src/share/vm/memory/genMarkSweep.cpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/memory/genMarkSweep.cpp	Mon Dec 01 15:24:56 2014 +0100
@@ -210,7 +210,7 @@
   gch->gen_process_roots(level,
                          false, // Younger gens are not roots.
                          true,  // activate StrongRootsScope
-                         SharedHeap::SO_None,
+                         GenCollectedHeap::SO_None,
                          GenCollectedHeap::StrongRootsOnly,
                          &follow_root_closure,
                          &follow_root_closure,
@@ -295,7 +295,7 @@
   gch->gen_process_roots(level,
                          false, // Younger gens are not roots.
                          true,  // activate StrongRootsScope
-                         SharedHeap::SO_AllCodeCache,
+                         GenCollectedHeap::SO_AllCodeCache,
                          GenCollectedHeap::StrongAndWeakRoots,
                          &adjust_pointer_closure,
                          &adjust_pointer_closure,
--- a/src/share/vm/memory/sharedHeap.cpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/memory/sharedHeap.cpp	Mon Dec 01 15:24:56 2014 +0100
@@ -32,7 +32,6 @@
 #include "runtime/atomic.inline.hpp"
 #include "runtime/fprofiler.hpp"
 #include "runtime/java.hpp"
-#include "services/management.hpp"
 #include "utilities/copy.hpp"
 #include "utilities/workgroup.hpp"
 
@@ -40,33 +39,13 @@
 
 SharedHeap* SharedHeap::_sh;
 
-// The set of potentially parallel tasks in root scanning.
-enum SH_process_roots_tasks {
-  SH_PS_Universe_oops_do,
-  SH_PS_JNIHandles_oops_do,
-  SH_PS_ObjectSynchronizer_oops_do,
-  SH_PS_FlatProfiler_oops_do,
-  SH_PS_Management_oops_do,
-  SH_PS_SystemDictionary_oops_do,
-  SH_PS_ClassLoaderDataGraph_oops_do,
-  SH_PS_jvmti_oops_do,
-  SH_PS_CodeCache_oops_do,
-  // Leave this one last.
-  SH_PS_NumElements
-};
-
 SharedHeap::SharedHeap(CollectorPolicy* policy_) :
   CollectedHeap(),
   _collector_policy(policy_),
   _rem_set(NULL),
-  _strong_roots_scope(NULL),
   _strong_roots_parity(0),
-  _process_strong_tasks(new SubTasksDone(SH_PS_NumElements)),
   _workers(NULL)
 {
-  if (_process_strong_tasks == NULL || !_process_strong_tasks->valid()) {
-    vm_exit_during_initialization("Failed necessary allocation.");
-  }
   _sh = this;  // ch is static, should be set only once.
   if ((UseParNewGC ||
       (UseConcMarkSweepGC && (CMSParallelInitialMarkEnabled ||
@@ -84,14 +63,6 @@
   }
 }
 
-int SharedHeap::n_termination() {
-  return _process_strong_tasks->n_threads();
-}
-
-void SharedHeap::set_n_termination(int t) {
-  _process_strong_tasks->set_n_threads(t);
-}
-
 bool SharedHeap::heap_lock_held_for_gc() {
   Thread* t = Thread::current();
   return    Heap_lock->owned_by_self()
@@ -102,31 +73,6 @@
 void SharedHeap::set_par_threads(uint t) {
   assert(t == 0 || !UseSerialGC, "Cannot have parallel threads");
   _n_par_threads = t;
-  _process_strong_tasks->set_n_threads(t);
-}
-
-#ifdef ASSERT
-class AssertNonScavengableClosure: public OopClosure {
-public:
-  virtual void do_oop(oop* p) {
-    assert(!Universe::heap()->is_in_partial_collection(*p),
-      "Referent should not be scavengable.");  }
-  virtual void do_oop(narrowOop* p) { ShouldNotReachHere(); }
-};
-static AssertNonScavengableClosure assert_is_non_scavengable_closure;
-#endif
-
-SharedHeap::StrongRootsScope* SharedHeap::active_strong_roots_scope() const {
-  return _strong_roots_scope;
-}
-void SharedHeap::register_strong_roots_scope(SharedHeap::StrongRootsScope* scope) {
-  assert(_strong_roots_scope == NULL, "Should only have one StrongRootsScope active");
-  assert(scope != NULL, "Illegal argument");
-  _strong_roots_scope = scope;
-}
-void SharedHeap::unregister_strong_roots_scope(SharedHeap::StrongRootsScope* scope) {
-  assert(_strong_roots_scope == scope, "Wrong scope unregistered");
-  _strong_roots_scope = NULL;
 }
 
 void SharedHeap::change_strong_roots_parity() {
@@ -140,174 +86,15 @@
 }
 
 SharedHeap::StrongRootsScope::StrongRootsScope(SharedHeap* heap, bool activate)
-  : MarkScope(activate), _sh(heap), _n_workers_done_with_threads(0)
+  : MarkScope(activate), _sh(heap)
 {
   if (_active) {
-    _sh->register_strong_roots_scope(this);
     _sh->change_strong_roots_parity();
     // Zero the claimed high water mark in the StringTable
     StringTable::clear_parallel_claimed_index();
   }
 }
 
-SharedHeap::StrongRootsScope::~StrongRootsScope() {
-  if (_active) {
-    _sh->unregister_strong_roots_scope(this);
-  }
-}
-
-Monitor* SharedHeap::StrongRootsScope::_lock = new Monitor(Mutex::leaf, "StrongRootsScope lock", false);
-
-void SharedHeap::StrongRootsScope::mark_worker_done_with_threads(uint n_workers) {
-  // The Thread work barrier is only needed by G1 Class Unloading.
-  // No need to use the barrier if this is single-threaded code.
-  if (UseG1GC && ClassUnloadingWithConcurrentMark && n_workers > 0) {
-    uint new_value = (uint)Atomic::add(1, &_n_workers_done_with_threads);
-    if (new_value == n_workers) {
-      // This thread is last. Notify the others.
-      MonitorLockerEx ml(_lock, Mutex::_no_safepoint_check_flag);
-      _lock->notify_all();
-    }
-  }
-}
-
-void SharedHeap::StrongRootsScope::wait_until_all_workers_done_with_threads(uint n_workers) {
-  assert(UseG1GC,                          "Currently only used by G1");
-  assert(ClassUnloadingWithConcurrentMark, "Currently only needed when doing G1 Class Unloading");
-
-  // No need to use the barrier if this is single-threaded code.
-  if (n_workers > 0 && (uint)_n_workers_done_with_threads != n_workers) {
-    MonitorLockerEx ml(_lock, Mutex::_no_safepoint_check_flag);
-    while ((uint)_n_workers_done_with_threads != n_workers) {
-      _lock->wait(Mutex::_no_safepoint_check_flag, 0, false);
-    }
-  }
-}
-
-void SharedHeap::process_roots(bool activate_scope,
-                               ScanningOption so,
-                               OopClosure* strong_roots,
-                               OopClosure* weak_roots,
-                               CLDClosure* strong_cld_closure,
-                               CLDClosure* weak_cld_closure,
-                               CodeBlobClosure* code_roots) {
-  StrongRootsScope srs(this, activate_scope);
-
-  // General roots.
-  assert(_strong_roots_parity != 0, "must have called prologue code");
-  assert(code_roots != NULL, "code root closure should always be set");
-  // _n_termination for _process_strong_tasks should be set up stream
-  // in a method not running in a GC worker.  Otherwise the GC worker
-  // could be trying to change the termination condition while the task
-  // is executing in another GC worker.
-
-  // Iterating over the CLDG and the Threads are done early to allow G1 to
-  // first process the strong CLDs and nmethods and then, after a barrier,
-  // let the thread process the weak CLDs and nmethods.
-
-  if (!_process_strong_tasks->is_task_claimed(SH_PS_ClassLoaderDataGraph_oops_do)) {
-    ClassLoaderDataGraph::roots_cld_do(strong_cld_closure, weak_cld_closure);
-  }
-
-  // Some CLDs contained in the thread frames should be considered strong.
-  // Don't process them if they will be processed during the ClassLoaderDataGraph phase.
-  CLDClosure* roots_from_clds_p = (strong_cld_closure != weak_cld_closure) ? strong_cld_closure : NULL;
-  // Only process code roots from thread stacks if we aren't visiting the entire CodeCache anyway
-  CodeBlobClosure* roots_from_code_p = (so & SO_AllCodeCache) ? NULL : code_roots;
-
-  Threads::possibly_parallel_oops_do(strong_roots, roots_from_clds_p, roots_from_code_p);
-
-  // This is the point where this worker thread will not find more strong CLDs/nmethods.
-  // Report this so G1 can synchronize the strong and weak CLDs/nmethods processing.
-  active_strong_roots_scope()->mark_worker_done_with_threads(n_par_threads());
-
-  if (!_process_strong_tasks->is_task_claimed(SH_PS_Universe_oops_do)) {
-    Universe::oops_do(strong_roots);
-  }
-  // Global (strong) JNI handles
-  if (!_process_strong_tasks->is_task_claimed(SH_PS_JNIHandles_oops_do))
-    JNIHandles::oops_do(strong_roots);
-
-  if (!_process_strong_tasks-> is_task_claimed(SH_PS_ObjectSynchronizer_oops_do))
-    ObjectSynchronizer::oops_do(strong_roots);
-  if (!_process_strong_tasks->is_task_claimed(SH_PS_FlatProfiler_oops_do))
-    FlatProfiler::oops_do(strong_roots);
-  if (!_process_strong_tasks->is_task_claimed(SH_PS_Management_oops_do))
-    Management::oops_do(strong_roots);
-  if (!_process_strong_tasks->is_task_claimed(SH_PS_jvmti_oops_do))
-    JvmtiExport::oops_do(strong_roots);
-
-  if (!_process_strong_tasks->is_task_claimed(SH_PS_SystemDictionary_oops_do)) {
-    SystemDictionary::roots_oops_do(strong_roots, weak_roots);
-  }
-
-  // All threads execute the following. A specific chunk of buckets
-  // from the StringTable are the individual tasks.
-  if (weak_roots != NULL) {
-    if (CollectedHeap::use_parallel_gc_threads()) {
-      StringTable::possibly_parallel_oops_do(weak_roots);
-    } else {
-      StringTable::oops_do(weak_roots);
-    }
-  }
-
-  if (!_process_strong_tasks->is_task_claimed(SH_PS_CodeCache_oops_do)) {
-    if (so & SO_ScavengeCodeCache) {
-      assert(code_roots != NULL, "must supply closure for code cache");
-
-      // We only visit parts of the CodeCache when scavenging.
-      CodeCache::scavenge_root_nmethods_do(code_roots);
-    }
-    if (so & SO_AllCodeCache) {
-      assert(code_roots != NULL, "must supply closure for code cache");
-
-      // CMSCollector uses this to do intermediate-strength collections.
-      // We scan the entire code cache, since CodeCache::do_unloading is not called.
-      CodeCache::blobs_do(code_roots);
-    }
-    // Verify that the code cache contents are not subject to
-    // movement by a scavenging collection.
-    DEBUG_ONLY(CodeBlobToOopClosure assert_code_is_non_scavengable(&assert_is_non_scavengable_closure, !CodeBlobToOopClosure::FixRelocations));
-    DEBUG_ONLY(CodeCache::asserted_non_scavengable_nmethods_do(&assert_code_is_non_scavengable));
-  }
-
-  _process_strong_tasks->all_tasks_completed();
-}
-
-void SharedHeap::process_all_roots(bool activate_scope,
-                                   ScanningOption so,
-                                   OopClosure* roots,
-                                   CLDClosure* cld_closure,
-                                   CodeBlobClosure* code_closure) {
-  process_roots(activate_scope, so,
-                roots, roots,
-                cld_closure, cld_closure,
-                code_closure);
-}
-
-void SharedHeap::process_strong_roots(bool activate_scope,
-                                      ScanningOption so,
-                                      OopClosure* roots,
-                                      CLDClosure* cld_closure,
-                                      CodeBlobClosure* code_closure) {
-  process_roots(activate_scope, so,
-                roots, NULL,
-                cld_closure, NULL,
-                code_closure);
-}
-
-
-class AlwaysTrueClosure: public BoolObjectClosure {
-public:
-  bool do_object_b(oop p) { return true; }
-};
-static AlwaysTrueClosure always_true;
-
-void SharedHeap::process_weak_roots(OopClosure* root_closure) {
-  // Global (weak) JNI handles
-  JNIHandles::weak_oops_do(&always_true, root_closure);
-}
-
 void SharedHeap::set_barrier_set(BarrierSet* bs) {
   _barrier_set = bs;
   // Cached barrier set for fast access in oops
--- a/src/share/vm/memory/sharedHeap.hpp	Thu Mar 26 13:19:32 2015 +0100
+++ b/src/share/vm/memory/sharedHeap.hpp	Mon Dec 01 15:24:56 2014 +0100
@@ -61,18 +61,18 @@
 //    counts the number of tasks that have been done and then reset
 //    the SubTasksDone so that it can be used again.  When the number of
 //    tasks is set to the number of GC workers, then _n_threads must
-//    be set to the number of active GC workers. G1CollectedHeap,
-//    HRInto_G1RemSet, GenCollectedHeap and SharedHeap have SubTasksDone.
-//    This seems too many.
+//    be set to the number of active GC workers. G1RootProcessor and
+//    GenCollectedHeap have SubTasksDone.
 //    3) SequentialSubTasksDone has an _n_threads that is used in
 //    a way similar to SubTasksDone and has the same dependency on the
 //    number of active GC workers.  CompactibleFreeListSpace and Space
 //    have SequentialSubTasksDone's.
-// Example of using SubTasksDone and SequentialSubTasksDone
-// G1CollectedHeap::g1_process_roots()
-//  to SharedHeap::process_roots() and uses
-//  SubTasksDone* _process_strong_tasks to claim tasks.
-//  process_roots() calls
+//
+// Examples of using SubTasksDone and SequentialSubTasksDone:
+//  G1RootProcessor and GenCollectedHeap::process_roots() use
+//  SubTasksDone* _process_strong_tasks to claim tasks for workers
+//
+//  GenCollectedHeap::gen_process_roots() calls
 //      rem_set()->younger_refs_iterate()
 //  to scan the card table and which eventually calls down into
 //  CardTableModRefBS::par_non_clean_card_iterate_work().  This method
@@ -104,10 +104,6 @@
   friend class VM_GC_Operation;
   friend class VM_CGC_Operation;
 
-private:
-  // For claiming strong_roots tasks.
-  SubTasksDone* _process_strong_tasks;
-
 protected:
   // There should be only a single instance of "SharedHeap" in a program.
   // This is enforced with the protected constructor below, which will also
@@ -144,7 +140,6 @@
   static SharedHeap* heap() { return _sh; }
 
   void set_barrier_set(BarrierSet* bs);
-  SubTasksDone* process_strong_tasks() { return _process_strong_tasks; }
 
   // Does operations required after initialization has been done.
   virtual void post_initialize();
@@ -201,69 +196,19 @@
   // strong_roots_prologue calls change_strong_roots_parity, if
   // parallel tasks are enabled.
   class StrongRootsScope : public MarkingCodeBlobClosure::MarkScope {
-    // Used to implement the Thread work barrier.
-    static Monitor* _lock;
-
     SharedHeap*   _sh;
-    volatile jint _n_workers_done_with_threads;
 
    public:
     StrongRootsScope(SharedHeap* heap, bool activate = true);
-    ~StrongRootsScope();
-
-    // Mark that this thread is done with the Threads work.
-    void mark_worker_done_with_threads(uint n_workers);
-    // Wait until all n_workers are done with the Threads work.
-    void wait_until_all_workers_done_with_threads(uint n_workers);
   };
   friend class StrongRootsScope;
 
-  // The current active StrongRootScope
-  StrongRootsScope* _strong_roots_scope;
-
-  StrongRootsScope* active_strong_roots_scope() const;
-
  private:
-  void register_strong_roots_scope(StrongRootsScope* scope);
-  void unregister_strong_roots_scope(StrongRootsScope* scope);
   void change_strong_roots_parity();
 
  public:
-  enum ScanningOption {
-    SO_None                =  0x0,
-    SO_AllCodeCache        =  0x8,
-    SO_ScavengeCodeCache   = 0x10
-  };
-
   FlexibleWorkGang* workers() const { return _workers; }
 
-  // Invoke the "do_oop" method the closure "roots" on all root locations.
-  // The "so" argument determines which roots the closure is applied to:
-  // "SO_None" does none;
-  // "SO_AllCodeCache" applies the closure to all elements of the CodeCache.
-  // "SO_ScavengeCodeCache" applies the closure to elements on the scavenge root list in the CodeCache.
-  void process_roots(bool activate_scope,
-                     ScanningOption so,
-                     OopClosure* strong_roots,
-                     OopClosure* weak_roots,
-                     CLDClosure* strong_cld_closure,
-                     CLDClosure* weak_cld_closure,
-                     CodeBlobClosure* code_roots);
-  void process_all_roots(bool activate_scope,
-                         ScanningOption so,
-                         OopClosure* roots,
-                         CLDClosure* cld_closure,
-                         CodeBlobClosure* code_roots);
-  void process_strong_roots(bool activate_scope,
-                            ScanningOption so,
-                            OopClosure* roots,
-                            CLDClosure* cld_closure,
-                            CodeBlobClosure* code_roots);
-
-
-  // Apply "root_closure" to the JNI weak roots..
-  void process_weak_roots(OopClosure* root_closure);
-
   // The functions below are helper functions that a subclass of
   // "SharedHeap" can use in the implementation of its virtual
   // functions.
@@ -278,9 +223,6 @@
   // (such as process roots) subsequently.
   virtual void set_par_threads(uint t);
 
-  int n_termination();
-  void set_n_termination(int t);
-
   //
   // New methods from CollectedHeap
   //
@@ -292,8 +234,4 @@
                              size_t capacity);
 };
 
-inline SharedHeap::ScanningOption operator|(SharedHeap::ScanningOption so0, SharedHeap::ScanningOption so1) {
-  return static_cast<SharedHeap::ScanningOption>(static_cast<int>(so0) | static_cast<int>(so1));
-}
-
 #endif // SHARE_VM_MEMORY_SHAREDHEAP_HPP