diff src/share/vm/gc_implementation/g1/g1RemSet.cpp @ 1705:2d160770d2e5

6814437: G1: remove the _new_refs array Summary: The per-worker _new_refs array is used to hold references that point into the collection set. It is populated during RSet updating and subsequently processed. In the event of an evacuation failure it processed again to recreate the RSets of regions in the collection set. Remove the per-worker _new_refs array by processing the references directly. Use a DirtyCardQueue to hold the cards containing the references so that the RSets of regions in the collection set can be recreated when handling an evacuation failure. Reviewed-by: iveresov, jmasa, tonyp
author johnc
date Mon, 02 Aug 2010 12:51:43 -0700
parents 5cbac8938c4c
children a03ae377b2e8
line wrap: on
line diff
--- a/src/share/vm/gc_implementation/g1/g1RemSet.cpp	Sat Jul 31 15:10:59 2010 +0100
+++ b/src/share/vm/gc_implementation/g1/g1RemSet.cpp	Mon Aug 02 12:51:43 2010 -0700
@@ -122,23 +122,24 @@
 HRInto_G1RemSet::HRInto_G1RemSet(G1CollectedHeap* g1, CardTableModRefBS* ct_bs)
   : G1RemSet(g1), _ct_bs(ct_bs), _g1p(_g1->g1_policy()),
     _cg1r(g1->concurrent_g1_refine()),
-    _par_traversal_in_progress(false), _new_refs(NULL),
+    _par_traversal_in_progress(false),
+    _cset_rs_update_cl(NULL),
     _cards_scanned(NULL), _total_cards_scanned(0)
 {
   _seq_task = new SubTasksDone(NumSeqTasks);
   guarantee(n_workers() > 0, "There should be some workers");
-  _new_refs = NEW_C_HEAP_ARRAY(GrowableArray<OopOrNarrowOopStar>*, n_workers());
+  _cset_rs_update_cl = NEW_C_HEAP_ARRAY(OopsInHeapRegionClosure*, n_workers());
   for (uint i = 0; i < n_workers(); i++) {
-    _new_refs[i] = new (ResourceObj::C_HEAP) GrowableArray<OopOrNarrowOopStar>(8192,true);
+    _cset_rs_update_cl[i] = NULL;
   }
 }
 
 HRInto_G1RemSet::~HRInto_G1RemSet() {
   delete _seq_task;
   for (uint i = 0; i < n_workers(); i++) {
-    delete _new_refs[i];
+    assert(_cset_rs_update_cl[i] == NULL, "it should be");
   }
-  FREE_C_HEAP_ARRAY(GrowableArray<OopOrNarrowOopStar>*, _new_refs);
+  FREE_C_HEAP_ARRAY(OopsInHeapRegionClosure*, _cset_rs_update_cl);
 }
 
 void CountNonCleanMemRegionClosure::do_MemRegion(MemRegion mr) {
@@ -306,12 +307,45 @@
   _g1p->record_scan_rs_time(worker_i, scan_rs_time_sec * 1000.0);
 }
 
-void HRInto_G1RemSet::updateRS(int worker_i) {
-  ConcurrentG1Refine* cg1r = _g1->concurrent_g1_refine();
+// Closure used for updating RSets and recording references that
+// point into the collection set. Only called during an
+// evacuation pause.
+
+class RefineRecordRefsIntoCSCardTableEntryClosure: public CardTableEntryClosure {
+  G1RemSet* _g1rs;
+  DirtyCardQueue* _into_cset_dcq;
+public:
+  RefineRecordRefsIntoCSCardTableEntryClosure(G1CollectedHeap* g1h,
+                                              DirtyCardQueue* into_cset_dcq) :
+    _g1rs(g1h->g1_rem_set()), _into_cset_dcq(into_cset_dcq)
+  {}
+  bool do_card_ptr(jbyte* card_ptr, int worker_i) {
+    // The only time we care about recording cards that
+    // contain references that point into the collection set
+    // is during RSet updating within an evacuation pause.
+    // In this case worker_i should be the id of a GC worker thread.
+    assert(SafepointSynchronize::is_at_safepoint(), "not during an evacuation pause");
+    assert(worker_i < (int) DirtyCardQueueSet::num_par_ids(), "should be a GC worker");
 
+    if (_g1rs->concurrentRefineOneCard(card_ptr, worker_i, true)) {
+      // 'card_ptr' contains references that point into the collection
+      // set. We need to record the card in the DCQS
+      // (G1CollectedHeap::into_cset_dirty_card_queue_set())
+      // that's used for that purpose.
+      //
+      // Enqueue the card
+      _into_cset_dcq->enqueue(card_ptr);
+    }
+    return true;
+  }
+};
+
+void HRInto_G1RemSet::updateRS(DirtyCardQueue* into_cset_dcq, int worker_i) {
   double start = os::elapsedTime();
-  // Apply the appropriate closure to all remaining log entries.
-  _g1->iterate_dirty_card_closure(false, worker_i);
+  // Apply the given closure to all remaining log entries.
+  RefineRecordRefsIntoCSCardTableEntryClosure into_cset_update_rs_cl(_g1, into_cset_dcq);
+  _g1->iterate_dirty_card_closure(&into_cset_update_rs_cl, into_cset_dcq, false, worker_i);
+
   // Now there should be no dirty cards.
   if (G1RSLogCheckCardTable) {
     CountNonCleanMemRegionClosure cl(_g1);
@@ -405,33 +439,6 @@
   }
 };
 
-template <class T> void
-HRInto_G1RemSet::scanNewRefsRS_work(OopsInHeapRegionClosure* oc,
-                                    int worker_i) {
-  double scan_new_refs_start_sec = os::elapsedTime();
-  G1CollectedHeap* g1h = G1CollectedHeap::heap();
-  CardTableModRefBS* ct_bs = (CardTableModRefBS*) (g1h->barrier_set());
-  for (int i = 0; i < _new_refs[worker_i]->length(); i++) {
-    T* p = (T*) _new_refs[worker_i]->at(i);
-    oop obj = oopDesc::load_decode_heap_oop(p);
-    // *p was in the collection set when p was pushed on "_new_refs", but
-    // another thread may have processed this location from an RS, so it
-    // might not point into the CS any longer.  If so, it's obviously been
-    // processed, and we don't need to do anything further.
-    if (g1h->obj_in_cs(obj)) {
-      HeapRegion* r = g1h->heap_region_containing(p);
-
-      DEBUG_ONLY(HeapRegion* to = g1h->heap_region_containing(obj));
-      oc->set_region(r);
-      // If "p" has already been processed concurrently, this is
-      // idempotent.
-      oc->do_oop(p);
-    }
-  }
-  double scan_new_refs_time_ms = (os::elapsedTime() - scan_new_refs_start_sec) * 1000.0;
-  _g1p->record_scan_new_refs_time(worker_i, scan_new_refs_time_ms);
-}
-
 void HRInto_G1RemSet::cleanupHRRS() {
   HeapRegionRemSet::cleanup();
 }
@@ -457,6 +464,26 @@
     count_cl.print_histo();
   }
 
+  // We cache the value of 'oc' closure into the appropriate slot in the
+  // _cset_rs_update_cl for this worker
+  assert(worker_i < (int)n_workers(), "sanity");
+  _cset_rs_update_cl[worker_i] = oc;
+
+  // A DirtyCardQueue that is used to hold cards containing references
+  // that point into the collection set. This DCQ is associated with a
+  // special DirtyCardQueueSet (see g1CollectedHeap.hpp).  Under normal
+  // circumstances (i.e. the pause successfully completes), these cards
+  // are just discarded (there's no need to update the RSets of regions
+  // that were in the collection set - after the pause these regions
+  // are wholly 'free' of live objects. In the event of an evacuation
+  // failure the cards/buffers in this queue set are:
+  // * passed to the DirtyCardQueueSet that is used to manage deferred
+  //   RSet updates, or
+  // * scanned for references that point into the collection set
+  //   and the RSet of the corresponding region in the collection set
+  //   is updated immediately.
+  DirtyCardQueue into_cset_dcq(&_g1->into_cset_dirty_card_queue_set());
+
   if (ParallelGCThreads > 0) {
     // The two flags below were introduced temporarily to serialize
     // the updating and scanning of remembered sets. There are some
@@ -465,12 +492,10 @@
     // conditions, we'll revert back to parallel remembered set
     // updating and scanning. See CRs 6677707 and 6677708.
     if (G1UseParallelRSetUpdating || (worker_i == 0)) {
-      updateRS(worker_i);
-      scanNewRefsRS(oc, worker_i);
+      updateRS(&into_cset_dcq, worker_i);
     } else {
       _g1p->record_update_rs_processed_buffers(worker_i, 0.0);
       _g1p->record_update_rs_time(worker_i, 0.0);
-      _g1p->record_scan_new_refs_time(worker_i, 0.0);
     }
     if (G1UseParallelRSetScanning || (worker_i == 0)) {
       scanRS(oc, worker_i);
@@ -479,10 +504,12 @@
     }
   } else {
     assert(worker_i == 0, "invariant");
-    updateRS(0);
-    scanNewRefsRS(oc, 0);
+    updateRS(&into_cset_dcq, 0);
     scanRS(oc, 0);
   }
+
+  // We now clear the cached values of _cset_rs_update_cl for this worker
+  _cset_rs_update_cl[worker_i] = NULL;
 }
 
 void HRInto_G1RemSet::
@@ -519,49 +546,65 @@
   }
 };
 
-class UpdateRSetOopsIntoCSImmediate : public OopClosure {
-  G1CollectedHeap* _g1;
-public:
-  UpdateRSetOopsIntoCSImmediate(G1CollectedHeap* g1) : _g1(g1) { }
-  virtual void do_oop(narrowOop* p) { do_oop_work(p); }
-  virtual void do_oop(      oop* p) { do_oop_work(p); }
-  template <class T> void do_oop_work(T* p) {
-    HeapRegion* to = _g1->heap_region_containing(oopDesc::load_decode_heap_oop(p));
-    if (to->in_collection_set()) {
-      to->rem_set()->add_reference(p, 0);
-    }
-  }
-};
-
-class UpdateRSetOopsIntoCSDeferred : public OopClosure {
+// This closure, applied to a DirtyCardQueueSet, is used to immediately
+// update the RSets for the regions in the CSet. For each card it iterates
+// through the oops which coincide with that card. It scans the reference
+// fields in each oop; when it finds an oop that points into the collection
+// set, the RSet for the region containing the referenced object is updated.
+// Note: _par_traversal_in_progress in the G1RemSet must be FALSE; otherwise
+// the UpdateRSetImmediate closure will cause cards to be enqueued on to
+// the DCQS that we're iterating over, causing an infinite loop.
+class UpdateRSetCardTableEntryIntoCSetClosure: public CardTableEntryClosure {
   G1CollectedHeap* _g1;
   CardTableModRefBS* _ct_bs;
-  DirtyCardQueue* _dcq;
 public:
-  UpdateRSetOopsIntoCSDeferred(G1CollectedHeap* g1, DirtyCardQueue* dcq) :
-    _g1(g1), _ct_bs((CardTableModRefBS*)_g1->barrier_set()), _dcq(dcq) { }
-  virtual void do_oop(narrowOop* p) { do_oop_work(p); }
-  virtual void do_oop(      oop* p) { do_oop_work(p); }
-  template <class T> void do_oop_work(T* p) {
-    oop obj = oopDesc::load_decode_heap_oop(p);
-    if (_g1->obj_in_cs(obj)) {
-      size_t card_index = _ct_bs->index_for(p);
-      if (_ct_bs->mark_card_deferred(card_index)) {
-        _dcq->enqueue((jbyte*)_ct_bs->byte_for_index(card_index));
-      }
-    }
+  UpdateRSetCardTableEntryIntoCSetClosure(G1CollectedHeap* g1,
+                                          CardTableModRefBS* bs):
+    _g1(g1), _ct_bs(bs)
+  { }
+
+  bool do_card_ptr(jbyte* card_ptr, int worker_i) {
+    // Construct the region representing the card.
+    HeapWord* start = _ct_bs->addr_for(card_ptr);
+    // And find the region containing it.
+    HeapRegion* r = _g1->heap_region_containing(start);
+    assert(r != NULL, "unexpected null");
+
+    // Scan oops in the card looking for references into the collection set
+    HeapWord* end   = _ct_bs->addr_for(card_ptr + 1);
+    MemRegion scanRegion(start, end);
+
+    UpdateRSetImmediate update_rs_cl(_g1->g1_rem_set());
+    FilterIntoCSClosure update_rs_cset_oop_cl(NULL, _g1, &update_rs_cl);
+    FilterOutOfRegionClosure filter_then_update_rs_cset_oop_cl(r, &update_rs_cset_oop_cl);
+
+    // We can pass false as the "filter_young" parameter here as:
+    // * we should be in a STW pause,
+    // * the DCQS to which this closure is applied is used to hold
+    //   references that point into the collection set from the prior
+    //   RSet updating,
+    // * the post-write barrier shouldn't be logging updates to young
+    //   regions (but there is a situation where this can happen - see
+    //   the comment in HRInto_G1RemSet::concurrentRefineOneCard below -
+    //   that should not be applicable here), and
+    // * during actual RSet updating, the filtering of cards in young
+    //   regions in HeapRegion::oops_on_card_seq_iterate_careful is
+    //   employed.
+    // As a result, when this closure is applied to "refs into cset"
+    // DCQS, we shouldn't see any cards in young regions.
+    update_rs_cl.set_region(r);
+    HeapWord* stop_point =
+      r->oops_on_card_seq_iterate_careful(scanRegion,
+                                        &filter_then_update_rs_cset_oop_cl,
+                                        false /* filter_young */);
+
+    // Since this is performed in the event of an evacuation failure, we
+    // we shouldn't see a non-null stop point
+    assert(stop_point == NULL, "saw an unallocated region");
+    return true;
   }
 };
 
-template <class T> void HRInto_G1RemSet::new_refs_iterate_work(OopClosure* cl) {
-  for (size_t i = 0; i < n_workers(); i++) {
-    for (int j = 0; j < _new_refs[i]->length(); j++) {
-      T* p = (T*) _new_refs[i]->at(j);
-      cl->do_oop(p);
-    }
-  }
-}
-
 void HRInto_G1RemSet::cleanup_after_oops_into_collection_set_do() {
   guarantee( _cards_scanned != NULL, "invariant" );
   _total_cards_scanned = 0;
@@ -584,21 +627,38 @@
     set_par_traversal(false);
   }
 
+  DirtyCardQueueSet& into_cset_dcqs = _g1->into_cset_dirty_card_queue_set();
+  int into_cset_n_buffers = into_cset_dcqs.completed_buffers_num();
+
   if (_g1->evacuation_failed()) {
-    // Restore remembered sets for the regions pointing into
-    // the collection set.
+    // Restore remembered sets for the regions pointing into the collection set.
+
     if (G1DeferredRSUpdate) {
-      DirtyCardQueue dcq(&_g1->dirty_card_queue_set());
-      UpdateRSetOopsIntoCSDeferred deferred_update(_g1, &dcq);
-      new_refs_iterate(&deferred_update);
+      // If deferred RS updates are enabled then we just need to transfer
+      // the completed buffers from (a) the DirtyCardQueueSet used to hold
+      // cards that contain references that point into the collection set
+      // to (b) the DCQS used to hold the deferred RS updates
+      _g1->dirty_card_queue_set().merge_bufferlists(&into_cset_dcqs);
     } else {
-      UpdateRSetOopsIntoCSImmediate immediate_update(_g1);
-      new_refs_iterate(&immediate_update);
+
+      CardTableModRefBS* bs = (CardTableModRefBS*)_g1->barrier_set();
+      UpdateRSetCardTableEntryIntoCSetClosure update_rs_cset_immediate(_g1, bs);
+
+      int n_completed_buffers = 0;
+      while (into_cset_dcqs.apply_closure_to_completed_buffer(&update_rs_cset_immediate,
+                                                    0, 0, true)) {
+        n_completed_buffers++;
+      }
+      assert(n_completed_buffers == into_cset_n_buffers, "missed some buffers");
     }
   }
-  for (uint i = 0; i < n_workers(); i++) {
-    _new_refs[i]->clear();
-  }
+
+  // Free any completed buffers in the DirtyCardQueueSet used to hold cards
+  // which contain references that point into the collection.
+  _g1->into_cset_dirty_card_queue_set().clear();
+  assert(_g1->into_cset_dirty_card_queue_set().completed_buffers_num() == 0,
+         "all buffers should be freed");
+  _g1->into_cset_dirty_card_queue_set().clear_n_completed_buffers();
 
   assert(!_par_traversal_in_progress, "Invariant between iterations.");
 }
@@ -652,7 +712,43 @@
 
 static IntHistogram out_of_histo(50, 50);
 
-void HRInto_G1RemSet::concurrentRefineOneCard_impl(jbyte* card_ptr, int worker_i) {
+class TriggerClosure : public OopClosure {
+  bool _trigger;
+public:
+  TriggerClosure() : _trigger(false) { }
+  bool value() const { return _trigger; }
+  template <class T> void do_oop_nv(T* p) { _trigger = true; }
+  virtual void do_oop(oop* p)        { do_oop_nv(p); }
+  virtual void do_oop(narrowOop* p)  { do_oop_nv(p); }
+};
+
+class InvokeIfNotTriggeredClosure: public OopClosure {
+  TriggerClosure* _t;
+  OopClosure* _oc;
+public:
+  InvokeIfNotTriggeredClosure(TriggerClosure* t, OopClosure* oc):
+    _t(t), _oc(oc) { }
+  template <class T> void do_oop_nv(T* p) {
+    if (!_t->value()) _oc->do_oop(p);
+  }
+  virtual void do_oop(oop* p)        { do_oop_nv(p); }
+  virtual void do_oop(narrowOop* p)  { do_oop_nv(p); }
+};
+
+class Mux2Closure : public OopClosure {
+  OopClosure* _c1;
+  OopClosure* _c2;
+public:
+  Mux2Closure(OopClosure *c1, OopClosure *c2) : _c1(c1), _c2(c2) { }
+  template <class T> void do_oop_nv(T* p) {
+    _c1->do_oop(p); _c2->do_oop(p);
+  }
+  virtual void do_oop(oop* p)        { do_oop_nv(p); }
+  virtual void do_oop(narrowOop* p)  { do_oop_nv(p); }
+};
+
+bool HRInto_G1RemSet::concurrentRefineOneCard_impl(jbyte* card_ptr, int worker_i,
+                                                   bool check_for_refs_into_cset) {
   // Construct the region representing the card.
   HeapWord* start = _ct_bs->addr_for(card_ptr);
   // And find the region containing it.
@@ -669,7 +765,16 @@
 
   UpdateRSOopClosure update_rs_oop_cl(this, worker_i);
   update_rs_oop_cl.set_from(r);
-  FilterOutOfRegionClosure filter_then_update_rs_oop_cl(r, &update_rs_oop_cl);
+
+  TriggerClosure trigger_cl;
+  FilterIntoCSClosure into_cs_cl(NULL, _g1, &trigger_cl);
+  InvokeIfNotTriggeredClosure invoke_cl(&trigger_cl, &into_cs_cl);
+  Mux2Closure mux(&invoke_cl, &update_rs_oop_cl);
+
+  FilterOutOfRegionClosure filter_then_update_rs_oop_cl(r,
+                        (check_for_refs_into_cset ?
+                                (OopClosure*)&mux :
+                                (OopClosure*)&update_rs_oop_cl));
 
   // Undirty the card.
   *card_ptr = CardTableModRefBS::clean_card_val();
@@ -717,11 +822,18 @@
     out_of_histo.add_entry(filter_then_update_rs_oop_cl.out_of_region());
     _conc_refine_cards++;
   }
+
+  return trigger_cl.value();
 }
 
-void HRInto_G1RemSet::concurrentRefineOneCard(jbyte* card_ptr, int worker_i) {
+bool HRInto_G1RemSet::concurrentRefineOneCard(jbyte* card_ptr, int worker_i,
+                                              bool check_for_refs_into_cset) {
   // If the card is no longer dirty, nothing to do.
-  if (*card_ptr != CardTableModRefBS::dirty_card_val()) return;
+  if (*card_ptr != CardTableModRefBS::dirty_card_val()) {
+    // No need to return that this card contains refs that point
+    // into the collection set.
+    return false;
+  }
 
   // Construct the region representing the card.
   HeapWord* start = _ct_bs->addr_for(card_ptr);
@@ -729,7 +841,9 @@
   HeapRegion* r = _g1->heap_region_containing(start);
   if (r == NULL) {
     guarantee(_g1->is_in_permanent(start), "Or else where?");
-    return;  // Not in the G1 heap (might be in perm, for example.)
+    // Again no need to return that this card contains refs that
+    // point into the collection set.
+    return false;  // Not in the G1 heap (might be in perm, for example.)
   }
   // Why do we have to check here whether a card is on a young region,
   // given that we dirty young regions and, as a result, the
@@ -743,7 +857,7 @@
   // and it doesn't happen often, but it can happen. So, the extra
   // check below filters out those cards.
   if (r->is_young()) {
-    return;
+    return false;
   }
   // While we are processing RSet buffers during the collection, we
   // actually don't want to scan any cards on the collection set,
@@ -756,7 +870,7 @@
   // however, that if evacuation fails, we have to scan any objects
   // that were not moved and create any missing entries.
   if (r->in_collection_set()) {
-    return;
+    return false;
   }
 
   // Should we defer processing the card?
@@ -797,8 +911,14 @@
   //                  cache.
   //                  Immediately process res; no need to process card_ptr.
 
+
   jbyte* res = card_ptr;
   bool defer = false;
+
+  // This gets set to true if the card being refined has references
+  // that point into the collection set.
+  bool oops_into_cset = false;
+
   if (_cg1r->use_cache()) {
     jbyte* res = _cg1r->cache_insert(card_ptr, &defer);
     if (res != NULL && (res != card_ptr || defer)) {
@@ -815,14 +935,31 @@
         // Process card pointer we get back from the hot card cache. This
         // will check whether the region containing the card is young
         // _after_ checking that the region has been allocated from.
-        concurrentRefineOneCard_impl(res, worker_i);
+        oops_into_cset = concurrentRefineOneCard_impl(res, worker_i,
+                                                      false /* check_for_refs_into_cset */);
+        // The above call to concurrentRefineOneCard_impl is only
+        // performed if the hot card cache is enabled. This cache is
+        // disabled during an evacuation pause - which is the only
+        // time when we need know if the card contains references
+        // that point into the collection set. Also when the hot card
+        // cache is enabled, this code is executed by the concurrent
+        // refine threads - rather than the GC worker threads - and
+        // concurrentRefineOneCard_impl will return false.
+        assert(!oops_into_cset, "should not see true here");
       }
     }
   }
 
   if (!defer) {
-    concurrentRefineOneCard_impl(card_ptr, worker_i);
+    oops_into_cset =
+      concurrentRefineOneCard_impl(card_ptr, worker_i, check_for_refs_into_cset);
+    // We should only be detecting that the card contains references
+    // that point into the collection set if the current thread is
+    // a GC worker thread.
+    assert(!oops_into_cset || SafepointSynchronize::is_at_safepoint(),
+           "invalid result at non safepoint");
   }
+  return oops_into_cset;
 }
 
 class HRRSStatsIter: public HeapRegionClosure {
@@ -920,6 +1057,7 @@
 
   }
 }
+
 void HRInto_G1RemSet::prepare_for_verify() {
   if (G1HRRSFlushLogBuffersOnVerify &&
       (VerifyBeforeGC || VerifyAfterGC)
@@ -932,7 +1070,9 @@
     }
     bool cg1r_use_cache = _cg1r->use_cache();
     _cg1r->set_use_cache(false);
-    updateRS(0);
+    DirtyCardQueue into_cset_dcq(&_g1->into_cset_dirty_card_queue_set());
+    updateRS(&into_cset_dcq, 0);
+    _g1->into_cset_dirty_card_queue_set().clear();
     _cg1r->set_use_cache(cg1r_use_cache);
 
     assert(JavaThread::dirty_card_queue_set().completed_buffers_num() == 0, "All should be consumed");