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

Print this page
rev 6626 : imported patch 8035400-move-g1parscanthreadstate-into-own-files
rev 6627 : imported patch 8035400-2-bengt-fixes
rev 6628 : imported patch 8035401-fix-visibility-of-g1parscanthreadstate


4652                                 RefToScanQueueSet* queues,
4653                                 ParallelTaskTerminator* terminator)
4654     : _g1h(g1h), _par_scan_state(par_scan_state),
4655       _queues(queues), _terminator(terminator) {}
4656 
4657   void do_void();
4658 
4659 private:
4660   inline bool offer_termination();
4661 };
4662 
4663 bool G1ParEvacuateFollowersClosure::offer_termination() {
4664   G1ParScanThreadState* const pss = par_scan_state();
4665   pss->start_term_time();
4666   const bool res = terminator()->offer_termination();
4667   pss->end_term_time();
4668   return res;
4669 }
4670 
4671 void G1ParEvacuateFollowersClosure::do_void() {
4672   StarTask stolen_task;
4673   G1ParScanThreadState* const pss = par_scan_state();
4674   pss->trim_queue();
4675 
4676   do {
4677     while (queues()->steal(pss->queue_num(), pss->hash_seed(), stolen_task)) {
4678       assert(pss->verify_task(stolen_task), "sanity");
4679       if (stolen_task.is_narrow()) {
4680         pss->deal_with_reference((narrowOop*) stolen_task);
4681       } else {
4682         pss->deal_with_reference((oop*) stolen_task);
4683       }
4684 
4685       // We've just processed a reference and we might have made
4686       // available new entries on the queues. So we have to make sure
4687       // we drain the queues as necessary.
4688       pss->trim_queue();
4689     }
4690   } while (!offer_termination());
4691 }
4692 
4693 class G1KlassScanClosure : public KlassClosure {
4694  G1ParCopyHelper* _closure;
4695  bool             _process_only_dirty;
4696  int              _count;
4697  public:
4698   G1KlassScanClosure(G1ParCopyHelper* closure, bool process_only_dirty)
4699       : _process_only_dirty(process_only_dirty), _closure(closure), _count(0) {}
4700   void do_klass(Klass* klass) {
4701     // If the klass has not been dirtied we know that there's
4702     // no references into  the young gen and we can skip it.
4703    if (!_process_only_dirty || klass->has_modified_oops()) {
4704       // Clean the klass since we're going to scavenge all the metadata.
4705       klass->clear_modified_oops();
4706 
4707       // Tell the closure that this klass is the Klass to scavenge
4708       // and is the one to dirty if oops are left pointing into the young gen.
4709       _closure->set_scanned_klass(klass);


4715     _count++;
4716   }
4717 };
4718 
4719 class G1ParTask : public AbstractGangTask {
4720 protected:
4721   G1CollectedHeap*       _g1h;
4722   RefToScanQueueSet      *_queues;
4723   ParallelTaskTerminator _terminator;
4724   uint _n_workers;
4725 
4726   Mutex _stats_lock;
4727   Mutex* stats_lock() { return &_stats_lock; }
4728 
4729   size_t getNCards() {
4730     return (_g1h->capacity() + G1BlockOffsetSharedArray::N_bytes - 1)
4731       / G1BlockOffsetSharedArray::N_bytes;
4732   }
4733 
4734 public:
4735   G1ParTask(G1CollectedHeap* g1h,
4736             RefToScanQueueSet *task_queues)
4737     : AbstractGangTask("G1 collection"),
4738       _g1h(g1h),
4739       _queues(task_queues),
4740       _terminator(0, _queues),
4741       _stats_lock(Mutex::leaf, "parallel G1 stats lock", true)
4742   {}
4743 
4744   RefToScanQueueSet* queues() { return _queues; }
4745 
4746   RefToScanQueue *work_queue(int i) {
4747     return queues()->queue(i);
4748   }
4749 
4750   ParallelTaskTerminator* terminator() { return &_terminator; }
4751 
4752   virtual void set_for_termination(int active_workers) {
4753     // This task calls set_n_termination() in par_non_clean_card_iterate_work()
4754     // in the young space (_par_seq_tasks) in the G1 heap
4755     // for SequentialSubTasksDone.
4756     // This task also uses SubTasksDone in SharedHeap and G1CollectedHeap


4814                                     worker_id);
4815       pss.end_strong_roots();
4816 
4817       {
4818         double start = os::elapsedTime();
4819         G1ParEvacuateFollowersClosure evac(_g1h, &pss, _queues, &_terminator);
4820         evac.do_void();
4821         double elapsed_ms = (os::elapsedTime()-start)*1000.0;
4822         double term_ms = pss.term_time()*1000.0;
4823         _g1h->g1_policy()->phase_times()->add_obj_copy_time(worker_id, elapsed_ms-term_ms);
4824         _g1h->g1_policy()->phase_times()->record_termination(worker_id, term_ms, pss.term_attempts());
4825       }
4826       _g1h->g1_policy()->record_thread_age_table(pss.age_table());
4827       _g1h->update_surviving_young_words(pss.surviving_young_words()+1);
4828 
4829       if (ParallelGCVerbose) {
4830         MutexLocker x(stats_lock());
4831         pss.print_termination_stats(worker_id);
4832       }
4833 
4834       assert(pss.refs()->is_empty(), "should be empty");
4835 
4836       // Close the inner scope so that the ResourceMark and HandleMark
4837       // destructors are executed here and are included as part of the
4838       // "GC Worker Time".
4839     }
4840 
4841     double end_time_ms = os::elapsedTime() * 1000.0;
4842     _g1h->g1_policy()->phase_times()->record_gc_worker_end_time(worker_id, end_time_ms);
4843   }
4844 };
4845 
4846 // *** Common G1 Evacuation Stuff
4847 
4848 // This method is run in a GC worker.
4849 
4850 void
4851 G1CollectedHeap::
4852 g1_process_strong_roots(bool is_scavenging,
4853                         ScanningOption so,
4854                         OopClosure* scan_non_heap_roots,


5338   uint _n_workers;
5339 
5340 public:
5341   G1ParPreserveCMReferentsTask(G1CollectedHeap* g1h,int workers, RefToScanQueueSet *task_queues) :
5342     AbstractGangTask("ParPreserveCMReferents"),
5343     _g1h(g1h),
5344     _queues(task_queues),
5345     _terminator(workers, _queues),
5346     _n_workers(workers)
5347   { }
5348 
5349   void work(uint worker_id) {
5350     ResourceMark rm;
5351     HandleMark   hm;
5352 
5353     G1ParScanThreadState            pss(_g1h, worker_id, NULL);
5354     G1ParScanHeapEvacFailureClosure evac_failure_cl(_g1h, &pss, NULL);
5355 
5356     pss.set_evac_failure_closure(&evac_failure_cl);
5357 
5358     assert(pss.refs()->is_empty(), "both queue and overflow should be empty");
5359 
5360 
5361     G1ParScanExtRootClosure        only_copy_non_heap_cl(_g1h, &pss, NULL);
5362 
5363     G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(_g1h, &pss, NULL);
5364 
5365     OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
5366 
5367     if (_g1h->g1_policy()->during_initial_mark_pause()) {
5368       // We also need to mark copied objects.
5369       copy_non_heap_cl = &copy_mark_non_heap_cl;
5370     }
5371 
5372     // Is alive closure
5373     G1AlwaysAliveClosure always_alive(_g1h);
5374 
5375     // Copying keep alive closure. Applied to referent objects that need
5376     // to be copied.
5377     G1CopyingKeepAliveClosure keep_alive(_g1h, copy_non_heap_cl, &pss);
5378 
5379     ReferenceProcessor* rp = _g1h->ref_processor_cm();


5393 
5394       DiscoveredListIterator iter(ref_list, &keep_alive, &always_alive);
5395       while (iter.has_next()) {
5396         // Since discovery is not atomic for the CM ref processor, we
5397         // can see some null referent objects.
5398         iter.load_ptrs(DEBUG_ONLY(true));
5399         oop ref = iter.obj();
5400 
5401         // This will filter nulls.
5402         if (iter.is_referent_alive()) {
5403           iter.make_referent_alive();
5404         }
5405         iter.move_to_next();
5406       }
5407     }
5408 
5409     // Drain the queue - which may cause stealing
5410     G1ParEvacuateFollowersClosure drain_queue(_g1h, &pss, _queues, &_terminator);
5411     drain_queue.do_void();
5412     // Allocation buffers were retired at the end of G1ParEvacuateFollowersClosure
5413     assert(pss.refs()->is_empty(), "should be");
5414   }
5415 };
5416 
5417 // Weak Reference processing during an evacuation pause (part 1).
5418 void G1CollectedHeap::process_discovered_references(uint no_of_gc_workers) {
5419   double ref_proc_start = os::elapsedTime();
5420 
5421   ReferenceProcessor* rp = _ref_processor_stw;
5422   assert(rp->discovery_enabled(), "should have been enabled");
5423 
5424   // Any reference objects, in the collection set, that were 'discovered'
5425   // by the CM ref processor should have already been copied (either by
5426   // applying the external root copy closure to the discovered lists, or
5427   // by following an RSet entry).
5428   //
5429   // But some of the referents, that are in the collection set, that these
5430   // reference objects point to may not have been copied: the STW ref
5431   // processor would have seen that the reference object had already
5432   // been 'discovered' and would have skipped discovering the reference,
5433   // but would not have treated the reference object as a regular oop.


5460   set_par_threads(0);
5461 
5462   // Closure to test whether a referent is alive.
5463   G1STWIsAliveClosure is_alive(this);
5464 
5465   // Even when parallel reference processing is enabled, the processing
5466   // of JNI refs is serial and performed serially by the current thread
5467   // rather than by a worker. The following PSS will be used for processing
5468   // JNI refs.
5469 
5470   // Use only a single queue for this PSS.
5471   G1ParScanThreadState            pss(this, 0, NULL);
5472 
5473   // We do not embed a reference processor in the copying/scanning
5474   // closures while we're actually processing the discovered
5475   // reference objects.
5476   G1ParScanHeapEvacFailureClosure evac_failure_cl(this, &pss, NULL);
5477 
5478   pss.set_evac_failure_closure(&evac_failure_cl);
5479 
5480   assert(pss.refs()->is_empty(), "pre-condition");
5481 
5482   G1ParScanExtRootClosure        only_copy_non_heap_cl(this, &pss, NULL);
5483 
5484   G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(this, &pss, NULL);
5485 
5486   OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
5487 
5488   if (_g1h->g1_policy()->during_initial_mark_pause()) {
5489     // We also need to mark copied objects.
5490     copy_non_heap_cl = &copy_mark_non_heap_cl;
5491   }
5492 
5493   // Keep alive closure.
5494   G1CopyingKeepAliveClosure keep_alive(this, copy_non_heap_cl, &pss);
5495 
5496   // Serial Complete GC closure
5497   G1STWDrainQueueClosure drain_queue(this, &pss);
5498 
5499   // Setup the soft refs policy...
5500   rp->setup_policy(false);


5508                                               NULL,
5509                                               _gc_timer_stw,
5510                                               _gc_tracer_stw->gc_id());
5511   } else {
5512     // Parallel reference processing
5513     assert(rp->num_q() == no_of_gc_workers, "sanity");
5514     assert(no_of_gc_workers <= rp->max_num_q(), "sanity");
5515 
5516     G1STWRefProcTaskExecutor par_task_executor(this, workers(), _task_queues, no_of_gc_workers);
5517     stats = rp->process_discovered_references(&is_alive,
5518                                               &keep_alive,
5519                                               &drain_queue,
5520                                               &par_task_executor,
5521                                               _gc_timer_stw,
5522                                               _gc_tracer_stw->gc_id());
5523   }
5524 
5525   _gc_tracer_stw->report_gc_reference_stats(stats);
5526 
5527   // We have completed copying any necessary live referent objects.
5528   assert(pss.refs()->is_empty(), "both queue and overflow should be empty");
5529 
5530   double ref_proc_time = os::elapsedTime() - ref_proc_start;
5531   g1_policy()->phase_times()->record_ref_proc_time(ref_proc_time * 1000.0);
5532 }
5533 
5534 // Weak Reference processing during an evacuation pause (part 2).
5535 void G1CollectedHeap::enqueue_discovered_references(uint no_of_gc_workers) {
5536   double ref_enq_start = os::elapsedTime();
5537 
5538   ReferenceProcessor* rp = _ref_processor_stw;
5539   assert(!rp->discovery_enabled(), "should have been disabled as part of processing");
5540 
5541   // Now enqueue any remaining on the discovered lists on to
5542   // the pending list.
5543   if (!rp->processing_is_mt()) {
5544     // Serial reference processing...
5545     rp->enqueue_discovered_references();
5546   } else {
5547     // Parallel reference enqueueing
5548 




4652                                 RefToScanQueueSet* queues,
4653                                 ParallelTaskTerminator* terminator)
4654     : _g1h(g1h), _par_scan_state(par_scan_state),
4655       _queues(queues), _terminator(terminator) {}
4656 
4657   void do_void();
4658 
4659 private:
4660   inline bool offer_termination();
4661 };
4662 
4663 bool G1ParEvacuateFollowersClosure::offer_termination() {
4664   G1ParScanThreadState* const pss = par_scan_state();
4665   pss->start_term_time();
4666   const bool res = terminator()->offer_termination();
4667   pss->end_term_time();
4668   return res;
4669 }
4670 
4671 void G1ParEvacuateFollowersClosure::do_void() {

4672   G1ParScanThreadState* const pss = par_scan_state();
4673   pss->trim_queue();

4674   do {
4675     pss->steal_and_trim_queue(queues());












4676   } while (!offer_termination());
4677 }
4678 
4679 class G1KlassScanClosure : public KlassClosure {
4680  G1ParCopyHelper* _closure;
4681  bool             _process_only_dirty;
4682  int              _count;
4683  public:
4684   G1KlassScanClosure(G1ParCopyHelper* closure, bool process_only_dirty)
4685       : _process_only_dirty(process_only_dirty), _closure(closure), _count(0) {}
4686   void do_klass(Klass* klass) {
4687     // If the klass has not been dirtied we know that there's
4688     // no references into  the young gen and we can skip it.
4689    if (!_process_only_dirty || klass->has_modified_oops()) {
4690       // Clean the klass since we're going to scavenge all the metadata.
4691       klass->clear_modified_oops();
4692 
4693       // Tell the closure that this klass is the Klass to scavenge
4694       // and is the one to dirty if oops are left pointing into the young gen.
4695       _closure->set_scanned_klass(klass);


4701     _count++;
4702   }
4703 };
4704 
4705 class G1ParTask : public AbstractGangTask {
4706 protected:
4707   G1CollectedHeap*       _g1h;
4708   RefToScanQueueSet      *_queues;
4709   ParallelTaskTerminator _terminator;
4710   uint _n_workers;
4711 
4712   Mutex _stats_lock;
4713   Mutex* stats_lock() { return &_stats_lock; }
4714 
4715   size_t getNCards() {
4716     return (_g1h->capacity() + G1BlockOffsetSharedArray::N_bytes - 1)
4717       / G1BlockOffsetSharedArray::N_bytes;
4718   }
4719 
4720 public:
4721   G1ParTask(G1CollectedHeap* g1h, RefToScanQueueSet *task_queues)

4722     : AbstractGangTask("G1 collection"),
4723       _g1h(g1h),
4724       _queues(task_queues),
4725       _terminator(0, _queues),
4726       _stats_lock(Mutex::leaf, "parallel G1 stats lock", true)
4727   {}
4728 
4729   RefToScanQueueSet* queues() { return _queues; }
4730 
4731   RefToScanQueue *work_queue(int i) {
4732     return queues()->queue(i);
4733   }
4734 
4735   ParallelTaskTerminator* terminator() { return &_terminator; }
4736 
4737   virtual void set_for_termination(int active_workers) {
4738     // This task calls set_n_termination() in par_non_clean_card_iterate_work()
4739     // in the young space (_par_seq_tasks) in the G1 heap
4740     // for SequentialSubTasksDone.
4741     // This task also uses SubTasksDone in SharedHeap and G1CollectedHeap


4799                                     worker_id);
4800       pss.end_strong_roots();
4801 
4802       {
4803         double start = os::elapsedTime();
4804         G1ParEvacuateFollowersClosure evac(_g1h, &pss, _queues, &_terminator);
4805         evac.do_void();
4806         double elapsed_ms = (os::elapsedTime()-start)*1000.0;
4807         double term_ms = pss.term_time()*1000.0;
4808         _g1h->g1_policy()->phase_times()->add_obj_copy_time(worker_id, elapsed_ms-term_ms);
4809         _g1h->g1_policy()->phase_times()->record_termination(worker_id, term_ms, pss.term_attempts());
4810       }
4811       _g1h->g1_policy()->record_thread_age_table(pss.age_table());
4812       _g1h->update_surviving_young_words(pss.surviving_young_words()+1);
4813 
4814       if (ParallelGCVerbose) {
4815         MutexLocker x(stats_lock());
4816         pss.print_termination_stats(worker_id);
4817       }
4818 
4819       assert(pss.queue_is_empty(), "should be empty");
4820 
4821       // Close the inner scope so that the ResourceMark and HandleMark
4822       // destructors are executed here and are included as part of the
4823       // "GC Worker Time".
4824     }
4825 
4826     double end_time_ms = os::elapsedTime() * 1000.0;
4827     _g1h->g1_policy()->phase_times()->record_gc_worker_end_time(worker_id, end_time_ms);
4828   }
4829 };
4830 
4831 // *** Common G1 Evacuation Stuff
4832 
4833 // This method is run in a GC worker.
4834 
4835 void
4836 G1CollectedHeap::
4837 g1_process_strong_roots(bool is_scavenging,
4838                         ScanningOption so,
4839                         OopClosure* scan_non_heap_roots,


5323   uint _n_workers;
5324 
5325 public:
5326   G1ParPreserveCMReferentsTask(G1CollectedHeap* g1h,int workers, RefToScanQueueSet *task_queues) :
5327     AbstractGangTask("ParPreserveCMReferents"),
5328     _g1h(g1h),
5329     _queues(task_queues),
5330     _terminator(workers, _queues),
5331     _n_workers(workers)
5332   { }
5333 
5334   void work(uint worker_id) {
5335     ResourceMark rm;
5336     HandleMark   hm;
5337 
5338     G1ParScanThreadState            pss(_g1h, worker_id, NULL);
5339     G1ParScanHeapEvacFailureClosure evac_failure_cl(_g1h, &pss, NULL);
5340 
5341     pss.set_evac_failure_closure(&evac_failure_cl);
5342 
5343     assert(pss.queue_is_empty(), "both queue and overflow should be empty");

5344 
5345     G1ParScanExtRootClosure        only_copy_non_heap_cl(_g1h, &pss, NULL);
5346 
5347     G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(_g1h, &pss, NULL);
5348 
5349     OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
5350 
5351     if (_g1h->g1_policy()->during_initial_mark_pause()) {
5352       // We also need to mark copied objects.
5353       copy_non_heap_cl = &copy_mark_non_heap_cl;
5354     }
5355 
5356     // Is alive closure
5357     G1AlwaysAliveClosure always_alive(_g1h);
5358 
5359     // Copying keep alive closure. Applied to referent objects that need
5360     // to be copied.
5361     G1CopyingKeepAliveClosure keep_alive(_g1h, copy_non_heap_cl, &pss);
5362 
5363     ReferenceProcessor* rp = _g1h->ref_processor_cm();


5377 
5378       DiscoveredListIterator iter(ref_list, &keep_alive, &always_alive);
5379       while (iter.has_next()) {
5380         // Since discovery is not atomic for the CM ref processor, we
5381         // can see some null referent objects.
5382         iter.load_ptrs(DEBUG_ONLY(true));
5383         oop ref = iter.obj();
5384 
5385         // This will filter nulls.
5386         if (iter.is_referent_alive()) {
5387           iter.make_referent_alive();
5388         }
5389         iter.move_to_next();
5390       }
5391     }
5392 
5393     // Drain the queue - which may cause stealing
5394     G1ParEvacuateFollowersClosure drain_queue(_g1h, &pss, _queues, &_terminator);
5395     drain_queue.do_void();
5396     // Allocation buffers were retired at the end of G1ParEvacuateFollowersClosure
5397     assert(pss.queue_is_empty(), "should be");
5398   }
5399 };
5400 
5401 // Weak Reference processing during an evacuation pause (part 1).
5402 void G1CollectedHeap::process_discovered_references(uint no_of_gc_workers) {
5403   double ref_proc_start = os::elapsedTime();
5404 
5405   ReferenceProcessor* rp = _ref_processor_stw;
5406   assert(rp->discovery_enabled(), "should have been enabled");
5407 
5408   // Any reference objects, in the collection set, that were 'discovered'
5409   // by the CM ref processor should have already been copied (either by
5410   // applying the external root copy closure to the discovered lists, or
5411   // by following an RSet entry).
5412   //
5413   // But some of the referents, that are in the collection set, that these
5414   // reference objects point to may not have been copied: the STW ref
5415   // processor would have seen that the reference object had already
5416   // been 'discovered' and would have skipped discovering the reference,
5417   // but would not have treated the reference object as a regular oop.


5444   set_par_threads(0);
5445 
5446   // Closure to test whether a referent is alive.
5447   G1STWIsAliveClosure is_alive(this);
5448 
5449   // Even when parallel reference processing is enabled, the processing
5450   // of JNI refs is serial and performed serially by the current thread
5451   // rather than by a worker. The following PSS will be used for processing
5452   // JNI refs.
5453 
5454   // Use only a single queue for this PSS.
5455   G1ParScanThreadState            pss(this, 0, NULL);
5456 
5457   // We do not embed a reference processor in the copying/scanning
5458   // closures while we're actually processing the discovered
5459   // reference objects.
5460   G1ParScanHeapEvacFailureClosure evac_failure_cl(this, &pss, NULL);
5461 
5462   pss.set_evac_failure_closure(&evac_failure_cl);
5463 
5464   assert(pss.queue_is_empty(), "pre-condition");
5465 
5466   G1ParScanExtRootClosure        only_copy_non_heap_cl(this, &pss, NULL);
5467 
5468   G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(this, &pss, NULL);
5469 
5470   OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
5471 
5472   if (_g1h->g1_policy()->during_initial_mark_pause()) {
5473     // We also need to mark copied objects.
5474     copy_non_heap_cl = &copy_mark_non_heap_cl;
5475   }
5476 
5477   // Keep alive closure.
5478   G1CopyingKeepAliveClosure keep_alive(this, copy_non_heap_cl, &pss);
5479 
5480   // Serial Complete GC closure
5481   G1STWDrainQueueClosure drain_queue(this, &pss);
5482 
5483   // Setup the soft refs policy...
5484   rp->setup_policy(false);


5492                                               NULL,
5493                                               _gc_timer_stw,
5494                                               _gc_tracer_stw->gc_id());
5495   } else {
5496     // Parallel reference processing
5497     assert(rp->num_q() == no_of_gc_workers, "sanity");
5498     assert(no_of_gc_workers <= rp->max_num_q(), "sanity");
5499 
5500     G1STWRefProcTaskExecutor par_task_executor(this, workers(), _task_queues, no_of_gc_workers);
5501     stats = rp->process_discovered_references(&is_alive,
5502                                               &keep_alive,
5503                                               &drain_queue,
5504                                               &par_task_executor,
5505                                               _gc_timer_stw,
5506                                               _gc_tracer_stw->gc_id());
5507   }
5508 
5509   _gc_tracer_stw->report_gc_reference_stats(stats);
5510 
5511   // We have completed copying any necessary live referent objects.
5512   assert(pss.queue_is_empty(), "both queue and overflow should be empty");
5513 
5514   double ref_proc_time = os::elapsedTime() - ref_proc_start;
5515   g1_policy()->phase_times()->record_ref_proc_time(ref_proc_time * 1000.0);
5516 }
5517 
5518 // Weak Reference processing during an evacuation pause (part 2).
5519 void G1CollectedHeap::enqueue_discovered_references(uint no_of_gc_workers) {
5520   double ref_enq_start = os::elapsedTime();
5521 
5522   ReferenceProcessor* rp = _ref_processor_stw;
5523   assert(!rp->discovery_enabled(), "should have been disabled as part of processing");
5524 
5525   // Now enqueue any remaining on the discovered lists on to
5526   // the pending list.
5527   if (!rp->processing_is_mt()) {
5528     // Serial reference processing...
5529     rp->enqueue_discovered_references();
5530   } else {
5531     // Parallel reference enqueueing
5532