< prev index next >

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

Print this page
rev 8816 : [mq]: 8133470-fix-plab-inline
rev 8817 : imported patch 8073013-add-detailed-information-about-plab-memory-usage
rev 8818 : imported patch jon-review-statistics
rev 8819 : imported patch dlindholm-changes
rev 8821 : imported patch move-jfr-event-to-extra-cr
rev 8822 : imported patch 8133530-add-jfr-event-for-evacuation
rev 8823 : imported patch 8040162-avoid-reallocating-plab-allocators
rev 8824 : imported patch parallel-ref-proc-fixes
rev 8825 : imported patch mikael-suggestions-pss-alloc

@@ -4380,10 +4380,17 @@
     }
   }
 }
 
 class G1ParEvacuateFollowersClosure : public VoidClosure {
+private:
+  double _start_term;
+  double _term_time;
+  size_t _term_attempts;
+
+  void start_term_time() { _term_attempts++; _start_term = os::elapsedTime(); }
+  void end_term_time() { _term_time += os::elapsedTime() - _start_term; }
 protected:
   G1CollectedHeap*              _g1h;
   G1ParScanThreadState*         _par_scan_state;
   RefToScanQueueSet*            _queues;
   ParallelTaskTerminator*       _terminator;

@@ -4396,23 +4403,27 @@
   G1ParEvacuateFollowersClosure(G1CollectedHeap* g1h,
                                 G1ParScanThreadState* par_scan_state,
                                 RefToScanQueueSet* queues,
                                 ParallelTaskTerminator* terminator)
     : _g1h(g1h), _par_scan_state(par_scan_state),
-      _queues(queues), _terminator(terminator) {}
+      _queues(queues), _terminator(terminator),
+      _start_term(0.0), _term_time(0.0), _term_attempts(0) {}
 
   void do_void();
 
+  double term_time() const { return _term_time; }
+  size_t term_attempts() const { return _term_attempts; }
+
 private:
   inline bool offer_termination();
 };
 
 bool G1ParEvacuateFollowersClosure::offer_termination() {
   G1ParScanThreadState* const pss = par_scan_state();
-  pss->start_term_time();
+  start_term_time();
   const bool res = terminator()->offer_termination();
-  pss->end_term_time();
+  end_term_time();
   return res;
 }
 
 void G1ParEvacuateFollowersClosure::do_void() {
   G1ParScanThreadState* const pss = par_scan_state();

@@ -4449,19 +4460,21 @@
 };
 
 class G1ParTask : public AbstractGangTask {
 protected:
   G1CollectedHeap*       _g1h;
-  RefToScanQueueSet      *_queues;
+  G1ParScanThreadState** _pss;
+  RefToScanQueueSet*     _queues;
   G1RootProcessor*       _root_processor;
   ParallelTaskTerminator _terminator;
   uint _n_workers;
 
 public:
-  G1ParTask(G1CollectedHeap* g1h, RefToScanQueueSet *task_queues, G1RootProcessor* root_processor, uint n_workers)
+  G1ParTask(G1CollectedHeap* g1h, G1ParScanThreadState** per_thread_states, RefToScanQueueSet *task_queues, G1RootProcessor* root_processor, uint n_workers)
     : AbstractGangTask("G1 collection"),
       _g1h(g1h),
+      _pss(per_thread_states),
       _queues(task_queues),
       _root_processor(root_processor),
       _terminator(n_workers, _queues),
       _n_workers(n_workers)
   {}

@@ -4504,35 +4517,37 @@
   };
 
   void work(uint worker_id) {
     if (worker_id >= _n_workers) return;  // no work needed this round
 
-    _g1h->g1_policy()->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerStart, worker_id, os::elapsedTime());
+    double start_sec = os::elapsedTime();
+    _g1h->g1_policy()->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerStart, worker_id, start_sec);
 
     {
       ResourceMark rm;
       HandleMark   hm;
 
       ReferenceProcessor*             rp = _g1h->ref_processor_stw();
 
-      G1ParScanThreadState            pss(_g1h, worker_id, rp);
+      G1ParScanThreadState*           pss = _pss[worker_id];
+      pss->set_ref_processor(rp);
 
       bool only_young = _g1h->collector_state()->gcs_are_young();
 
       // Non-IM young GC.
-      G1ParCopyClosure<G1BarrierNone, G1MarkNone>             scan_only_root_cl(_g1h, &pss, rp);
+      G1ParCopyClosure<G1BarrierNone, G1MarkNone>             scan_only_root_cl(_g1h, pss, rp);
       G1CLDClosure<G1MarkNone>                                scan_only_cld_cl(&scan_only_root_cl,
                                                                                only_young, // Only process dirty klasses.
                                                                                false);     // No need to claim CLDs.
       // IM young GC.
       //    Strong roots closures.
-      G1ParCopyClosure<G1BarrierNone, G1MarkFromRoot>         scan_mark_root_cl(_g1h, &pss, rp);
+      G1ParCopyClosure<G1BarrierNone, G1MarkFromRoot>         scan_mark_root_cl(_g1h, pss, rp);
       G1CLDClosure<G1MarkFromRoot>                            scan_mark_cld_cl(&scan_mark_root_cl,
                                                                                false, // Process all klasses.
                                                                                true); // Need to claim CLDs.
       //    Weak roots closures.
-      G1ParCopyClosure<G1BarrierNone, G1MarkPromotedFromRoot> scan_mark_weak_root_cl(_g1h, &pss, rp);
+      G1ParCopyClosure<G1BarrierNone, G1MarkPromotedFromRoot> scan_mark_weak_root_cl(_g1h, pss, rp);
       G1CLDClosure<G1MarkPromotedFromRoot>                    scan_mark_weak_cld_cl(&scan_mark_weak_root_cl,
                                                                                     false, // Process all klasses.
                                                                                     true); // Need to claim CLDs.
 
       OopClosure* strong_root_cl;

@@ -4559,53 +4574,94 @@
         weak_root_cl   = &scan_only_root_cl;
         strong_cld_cl  = &scan_only_cld_cl;
         weak_cld_cl    = &scan_only_cld_cl;
       }
 
-      pss.start_strong_roots();
-
+      double start_strong_roots_sec = os::elapsedTime();
       _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);
+      G1ParPushHeapRSClosure push_heap_rs_cl(_g1h, pss);
       _root_processor->scan_remembered_sets(&push_heap_rs_cl,
                                             weak_root_cl,
                                             worker_id);
-      pss.end_strong_roots();
+      double strong_roots_sec = os::elapsedTime() - start_strong_roots_sec;
 
+      double term_sec = 0.0;
+      size_t evac_term_attempts = 0;
       {
         double start = os::elapsedTime();
-        G1ParEvacuateFollowersClosure evac(_g1h, &pss, _queues, &_terminator);
+        G1ParEvacuateFollowersClosure evac(_g1h, pss, _queues, &_terminator);
         evac.do_void();
+
+        evac_term_attempts = evac.term_attempts();
+        term_sec = evac.term_time();
         double elapsed_sec = os::elapsedTime() - start;
-        double term_sec = pss.term_time();
         _g1h->g1_policy()->phase_times()->add_time_secs(G1GCPhaseTimes::ObjCopy, worker_id, elapsed_sec - term_sec);
         _g1h->g1_policy()->phase_times()->record_time_secs(G1GCPhaseTimes::Termination, worker_id, term_sec);
-        _g1h->g1_policy()->phase_times()->record_thread_work_item(G1GCPhaseTimes::Termination, worker_id, pss.term_attempts());
+        _g1h->g1_policy()->phase_times()->record_thread_work_item(G1GCPhaseTimes::Termination, worker_id, evac_term_attempts);
       }
-      _g1h->g1_policy()->record_thread_age_table(pss.age_table());
-      _g1h->update_surviving_young_words(pss.surviving_young_words()+1);
+
+      // Flush any statistics.
+      _g1h->g1_policy()->record_thread_age_table(pss->age_table());
+      _g1h->update_surviving_young_words(pss->surviving_young_words());
+
+      assert(pss->queue_is_empty(), "should be empty");
 
       if (PrintTerminationStats) {
         MutexLockerEx x(ParGCRareEvent_lock, Mutex::_no_safepoint_check_flag);
-        pss.print_termination_stats();
+        size_t lab_waste;
+        size_t lab_undo_waste;
+        pss->waste(lab_waste, lab_undo_waste);
+        _g1h->print_termination_stats(gclog_or_tty,
+                                      worker_id,
+                                      (os::elapsedTime() - start_sec) * 1000.0,   /* elapsed time */
+                                      strong_roots_sec * 1000.0,                  /* strong roots time */
+                                      term_sec * 1000.0,                          /* evac term time */
+                                      evac_term_attempts,                         /* evac term attempts */
+                                      lab_waste,                                  /* alloc buffer waste */
+                                      lab_undo_waste                              /* undo waste */
+                                      );
       }
 
-      assert(pss.queue_is_empty(), "should be empty");
-
       // Close the inner scope so that the ResourceMark and HandleMark
       // destructors are executed here and are included as part of the
       // "GC Worker Time".
     }
     _g1h->g1_policy()->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerEnd, worker_id, os::elapsedTime());
   }
 };
 
+void G1CollectedHeap::print_termination_stats_hdr(outputStream* const st) {
+  st->print_raw_cr("GC Termination Stats");
+  st->print_raw_cr("     elapsed  --strong roots-- -------termination------- ------waste (KiB)------");
+  st->print_raw_cr("thr     ms        ms      %        ms      %    attempts  total   alloc    undo");
+  st->print_raw_cr("--- --------- --------- ------ --------- ------ -------- ------- ------- -------");
+}
+
+void G1CollectedHeap::print_termination_stats(outputStream* const st,
+                                              uint worker_id,
+                                              double elapsed_ms,
+                                              double strong_roots_ms,
+                                              double term_ms,
+                                              size_t term_attempts,
+                                              size_t alloc_buffer_waste,
+                                              size_t undo_waste) const {
+  st->print_cr("%3d %9.2f %9.2f %6.2f "
+               "%9.2f %6.2f " SIZE_FORMAT_W(8) " "
+               SIZE_FORMAT_W(7) " " SIZE_FORMAT_W(7) " " SIZE_FORMAT_W(7),
+               worker_id, elapsed_ms, strong_roots_ms, strong_roots_ms * 100 / elapsed_ms,
+               term_ms, term_ms * 100 / elapsed_ms, term_attempts,
+               (alloc_buffer_waste + undo_waste) * HeapWordSize / K,
+               alloc_buffer_waste * HeapWordSize / K,
+               undo_waste * HeapWordSize / K);
+}
+
 class G1StringSymbolTableUnlinkTask : public AbstractGangTask {
 private:
   BoolObjectClosure* _is_alive;
   int _initial_string_table_size;
   int _initial_symbol_table_size;

@@ -5131,20 +5187,23 @@
 // processing during G1 evacuation pauses.
 
 class G1STWRefProcTaskExecutor: public AbstractRefProcTaskExecutor {
 private:
   G1CollectedHeap*   _g1h;
+  G1ParScanThreadState**  _pss;
   RefToScanQueueSet* _queues;
   FlexibleWorkGang*  _workers;
   uint               _active_workers;
 
 public:
   G1STWRefProcTaskExecutor(G1CollectedHeap* g1h,
+                           G1ParScanThreadState** per_thread_states,
                            FlexibleWorkGang* workers,
                            RefToScanQueueSet *task_queues,
                            uint n_workers) :
     _g1h(g1h),
+    _pss(per_thread_states),
     _queues(task_queues),
     _workers(workers),
     _active_workers(n_workers)
   {
     assert(n_workers > 0, "shouldn't call this otherwise");

@@ -5159,21 +5218,24 @@
 
 class G1STWRefProcTaskProxy: public AbstractGangTask {
   typedef AbstractRefProcTaskExecutor::ProcessTask ProcessTask;
   ProcessTask&     _proc_task;
   G1CollectedHeap* _g1h;
-  RefToScanQueueSet *_task_queues;
+  G1ParScanThreadState** _pss;
+  RefToScanQueueSet* _task_queues;
   ParallelTaskTerminator* _terminator;
 
 public:
   G1STWRefProcTaskProxy(ProcessTask& proc_task,
                      G1CollectedHeap* g1h,
+                        G1ParScanThreadState** per_thread_states,
                      RefToScanQueueSet *task_queues,
                      ParallelTaskTerminator* terminator) :
     AbstractGangTask("Process reference objects in parallel"),
     _proc_task(proc_task),
     _g1h(g1h),
+    _pss(per_thread_states),
     _task_queues(task_queues),
     _terminator(terminator)
   {}
 
   virtual void work(uint worker_id) {

@@ -5181,28 +5243,29 @@
     ResourceMark rm;
     HandleMark   hm;
 
     G1STWIsAliveClosure is_alive(_g1h);
 
-    G1ParScanThreadState            pss(_g1h, worker_id, NULL);
+    G1ParScanThreadState*           pss = _pss[worker_id];
+    pss->set_ref_processor(NULL);
 
-    G1ParScanExtRootClosure        only_copy_non_heap_cl(_g1h, &pss, NULL);
+    G1ParScanExtRootClosure        only_copy_non_heap_cl(_g1h, pss, NULL);
 
-    G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(_g1h, &pss, NULL);
+    G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(_g1h, pss, NULL);
 
     OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
 
     if (_g1h->collector_state()->during_initial_mark_pause()) {
       // We also need to mark copied objects.
       copy_non_heap_cl = &copy_mark_non_heap_cl;
     }
 
     // Keep alive closure.
-    G1CopyingKeepAliveClosure keep_alive(_g1h, copy_non_heap_cl, &pss);
+    G1CopyingKeepAliveClosure keep_alive(_g1h, copy_non_heap_cl, pss);
 
     // Complete GC closure
-    G1ParEvacuateFollowersClosure drain_queue(_g1h, &pss, _task_queues, _terminator);
+    G1ParEvacuateFollowersClosure drain_queue(_g1h, pss, _task_queues, _terminator);
 
     // Call the reference processing task's work routine.
     _proc_task.work(worker_id, is_alive, keep_alive, drain_queue);
 
     // Note we cannot assert that the refs array is empty here as not all

@@ -5217,11 +5280,11 @@
 // task and has the worker threads execute it.
 void G1STWRefProcTaskExecutor::execute(ProcessTask& proc_task) {
   assert(_workers != NULL, "Need parallel worker threads.");
 
   ParallelTaskTerminator terminator(_active_workers, _queues);
-  G1STWRefProcTaskProxy proc_task_proxy(proc_task, _g1h, _queues, &terminator);
+  G1STWRefProcTaskProxy proc_task_proxy(proc_task, _g1h, _pss, _queues, &terminator);
 
   _workers->run_task(&proc_task_proxy);
 }
 
 // Gang task for parallel reference enqueueing.

@@ -5260,33 +5323,36 @@
 // objects discovered by the CM ref processor.
 
 class G1ParPreserveCMReferentsTask: public AbstractGangTask {
 protected:
   G1CollectedHeap* _g1h;
-  RefToScanQueueSet      *_queues;
+  G1ParScanThreadState** _pss;
+  RefToScanQueueSet*     _queues;
   ParallelTaskTerminator _terminator;
   uint _n_workers;
 
 public:
-  G1ParPreserveCMReferentsTask(G1CollectedHeap* g1h, uint workers, RefToScanQueueSet *task_queues) :
+  G1ParPreserveCMReferentsTask(G1CollectedHeap* g1h, G1ParScanThreadState** per_thread_states, int workers, RefToScanQueueSet *task_queues) :
     AbstractGangTask("ParPreserveCMReferents"),
     _g1h(g1h),
+    _pss(per_thread_states),
     _queues(task_queues),
     _terminator(workers, _queues),
     _n_workers(workers)
   { }
 
   void work(uint worker_id) {
     ResourceMark rm;
     HandleMark   hm;
 
-    G1ParScanThreadState            pss(_g1h, worker_id, NULL);
-    assert(pss.queue_is_empty(), "both queue and overflow should be empty");
+    G1ParScanThreadState*          pss = _pss[worker_id];
+    pss->set_ref_processor(NULL);
+    assert(pss->queue_is_empty(), "both queue and overflow should be empty");
 
-    G1ParScanExtRootClosure        only_copy_non_heap_cl(_g1h, &pss, NULL);
+    G1ParScanExtRootClosure        only_copy_non_heap_cl(_g1h, pss, NULL);
 
-    G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(_g1h, &pss, NULL);
+    G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(_g1h, pss, NULL);
 
     OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
 
     if (_g1h->collector_state()->during_initial_mark_pause()) {
       // We also need to mark copied objects.

@@ -5296,11 +5362,11 @@
     // Is alive closure
     G1AlwaysAliveClosure always_alive(_g1h);
 
     // Copying keep alive closure. Applied to referent objects that need
     // to be copied.
-    G1CopyingKeepAliveClosure keep_alive(_g1h, copy_non_heap_cl, &pss);
+    G1CopyingKeepAliveClosure keep_alive(_g1h, copy_non_heap_cl, pss);
 
     ReferenceProcessor* rp = _g1h->ref_processor_cm();
 
     uint limit = ReferenceProcessor::number_of_subclasses_of_ref() * rp->max_num_q();
     uint stride = MIN2(MAX2(_n_workers, 1U), limit);

@@ -5329,19 +5395,19 @@
         iter.move_to_next();
       }
     }
 
     // Drain the queue - which may cause stealing
-    G1ParEvacuateFollowersClosure drain_queue(_g1h, &pss, _queues, &_terminator);
+    G1ParEvacuateFollowersClosure drain_queue(_g1h, pss, _queues, &_terminator);
     drain_queue.do_void();
     // Allocation buffers were retired at the end of G1ParEvacuateFollowersClosure
-    assert(pss.queue_is_empty(), "should be");
+    assert(pss->queue_is_empty(), "should be");
   }
 };
 
 // Weak Reference processing during an evacuation pause (part 1).
-void G1CollectedHeap::process_discovered_references() {
+void G1CollectedHeap::process_discovered_references(G1ParScanThreadState** per_thread_states) {
   double ref_proc_start = os::elapsedTime();
 
   ReferenceProcessor* rp = _ref_processor_stw;
   assert(rp->discovery_enabled(), "should have been enabled");
 

@@ -5367,10 +5433,11 @@
   // object discovered by the STW ref processor.
 
   uint no_of_gc_workers = workers()->active_workers();
 
   G1ParPreserveCMReferentsTask keep_cm_referents(this,
+                                                 per_thread_states,
                                                  no_of_gc_workers,
                                                  _task_queues);
 
   workers()->run_task(&keep_cm_referents);
 

@@ -5381,33 +5448,34 @@
   // of JNI refs is serial and performed serially by the current thread
   // rather than by a worker. The following PSS will be used for processing
   // JNI refs.
 
   // Use only a single queue for this PSS.
-  G1ParScanThreadState            pss(this, 0, NULL);
-  assert(pss.queue_is_empty(), "pre-condition");
+  G1ParScanThreadState*           pss = per_thread_states[0];
+  pss->set_ref_processor(NULL);
+  assert(pss->queue_is_empty(), "pre-condition");
 
   // We do not embed a reference processor in the copying/scanning
   // closures while we're actually processing the discovered
   // reference objects.
 
-  G1ParScanExtRootClosure        only_copy_non_heap_cl(this, &pss, NULL);
+  G1ParScanExtRootClosure        only_copy_non_heap_cl(this, pss, NULL);
 
-  G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(this, &pss, NULL);
+  G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(this, pss, NULL);
 
   OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
 
   if (collector_state()->during_initial_mark_pause()) {
     // We also need to mark copied objects.
     copy_non_heap_cl = &copy_mark_non_heap_cl;
   }
 
   // Keep alive closure.
-  G1CopyingKeepAliveClosure keep_alive(this, copy_non_heap_cl, &pss);
+  G1CopyingKeepAliveClosure keep_alive(this, copy_non_heap_cl, pss);
 
   // Serial Complete GC closure
-  G1STWDrainQueueClosure drain_queue(this, &pss);
+  G1STWDrainQueueClosure drain_queue(this, pss);
 
   // Setup the soft refs policy...
   rp->setup_policy(false);
 
   ReferenceProcessorStats stats;

@@ -5422,11 +5490,11 @@
   } else {
     // Parallel reference processing
     assert(rp->num_q() == no_of_gc_workers, "sanity");
     assert(no_of_gc_workers <= rp->max_num_q(), "sanity");
 
-    G1STWRefProcTaskExecutor par_task_executor(this, workers(), _task_queues, no_of_gc_workers);
+    G1STWRefProcTaskExecutor par_task_executor(this, per_thread_states, workers(), _task_queues, no_of_gc_workers);
     stats = rp->process_discovered_references(&is_alive,
                                               &keep_alive,
                                               &drain_queue,
                                               &par_task_executor,
                                               _gc_timer_stw,

@@ -5434,18 +5502,18 @@
   }
 
   _gc_tracer_stw->report_gc_reference_stats(stats);
 
   // We have completed copying any necessary live referent objects.
-  assert(pss.queue_is_empty(), "both queue and overflow should be empty");
+  assert(pss->queue_is_empty(), "both queue and overflow should be empty");
 
   double ref_proc_time = os::elapsedTime() - ref_proc_start;
   g1_policy()->phase_times()->record_ref_proc_time(ref_proc_time * 1000.0);
 }
 
 // Weak Reference processing during an evacuation pause (part 2).
-void G1CollectedHeap::enqueue_discovered_references() {
+void G1CollectedHeap::enqueue_discovered_references(G1ParScanThreadState** per_thread_states) {
   double ref_enq_start = os::elapsedTime();
 
   ReferenceProcessor* rp = _ref_processor_stw;
   assert(!rp->discovery_enabled(), "should have been disabled as part of processing");
 

@@ -5460,11 +5528,11 @@
     uint n_workers = workers()->active_workers();
 
     assert(rp->num_q() == n_workers, "sanity");
     assert(n_workers <= rp->max_num_q(), "sanity");
 
-    G1STWRefProcTaskExecutor par_task_executor(this, workers(), _task_queues, n_workers);
+    G1STWRefProcTaskExecutor par_task_executor(this, per_thread_states, workers(), _task_queues, n_workers);
     rp->enqueue_discovered_references(&par_task_executor);
   }
 
   rp->verify_no_references_recorded();
   assert(!rp->discovery_enabled(), "should have been disabled");

@@ -5496,21 +5564,26 @@
 
   assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty");
   double start_par_time_sec = os::elapsedTime();
   double end_par_time_sec;
 
+  G1ParScanThreadState** per_thread_states = NEW_C_HEAP_ARRAY(G1ParScanThreadState*, n_workers, mtGC);
+  for (uint i = 0; i < n_workers; i++) {
+    per_thread_states[i] = new_par_scan_state(i);
+  }
+
   {
     G1RootProcessor root_processor(this, n_workers);
-    G1ParTask g1_par_task(this, _task_queues, &root_processor, n_workers);
+    G1ParTask g1_par_task(this, per_thread_states, _task_queues, &root_processor, n_workers);
     // InitialMark needs claim bits to keep track of the marked-through CLDs.
     if (collector_state()->during_initial_mark_pause()) {
       ClassLoaderDataGraph::clear_claimed_marks();
     }
 
     // The individual threads will set their evac-failure closures.
     if (PrintTerminationStats) {
-      G1ParScanThreadState::print_termination_stats_hdr();
+      print_termination_stats_hdr(gclog_or_tty);
     }
 
     workers()->run_task(&g1_par_task);
     end_par_time_sec = os::elapsedTime();
 

@@ -5533,11 +5606,11 @@
   // Process any discovered reference objects - we have
   // to do this _before_ we retire the GC alloc regions
   // as we may have to copy some 'reachable' referent
   // objects (and their reachable sub-graphs) that were
   // not copied during the pause.
-  process_discovered_references();
+  process_discovered_references(per_thread_states);
 
   if (G1StringDedup::is_enabled()) {
     double fixup_start = os::elapsedTime();
 
     G1STWIsAliveClosure is_alive(this);

@@ -5549,10 +5622,16 @@
   }
 
   _allocator->release_gc_alloc_regions(evacuation_info);
   g1_rem_set()->cleanup_after_oops_into_collection_set_do();
 
+  for (uint i = 0; i < n_workers; i++) {
+    G1ParScanThreadState* pss = per_thread_states[i];
+    delete pss;
+  }
+  FREE_C_HEAP_ARRAY(G1ParScanThreadState*, per_thread_states);
+
   record_obj_copy_mem_stats();
 
   // Reset and re-enable the hot card cache.
   // Note the counts for the cards in the regions in the
   // collection set are reset when the collection set is freed.

@@ -5575,11 +5654,11 @@
   // this after the card table is cleaned (and verified) as
   // the act of enqueueing entries on to the pending list
   // will log these updates (and dirty their associated
   // cards). We need these updates logged to update any
   // RSets.
-  enqueue_discovered_references();
+  enqueue_discovered_references(per_thread_states);
 
   redirty_logged_cards();
   COMPILER2_PRESENT(DerivedPointerTable::update_pointers());
 }
 
< prev index next >