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

Print this page
rev 2896 : 6484965: G1: piggy-back liveness accounting phase on marking
Summary: Remove the separate counting phase of concurrent marking by tracking the amount of marked bytes and the cards spanned by marked objects in marking task/worker thread local data structures, which are updated as individual objects are marked.
Reviewed-by: brutisso

@@ -476,10 +476,11 @@
   _cleanup_list("Cleanup List"),
   _region_bm(max_regions, false /* in_resource_area*/),
   _card_bm((rs.size() + CardTableModRefBS::card_size - 1) >>
            CardTableModRefBS::card_shift,
            false /* in_resource_area*/),
+
   _prevMarkBitMap(&_markBitMap1),
   _nextMarkBitMap(&_markBitMap2),
   _at_least_one_mark_complete(false),
 
   _markStack(this),

@@ -505,11 +506,15 @@
   _remark_times(), _remark_mark_times(), _remark_weak_ref_times(),
   _cleanup_times(),
   _total_counting_time(0.0),
   _total_rs_scrub_time(0.0),
 
-  _parallel_workers(NULL) {
+  _parallel_workers(NULL),
+
+  _count_card_bitmaps(NULL),
+  _count_marked_bytes(NULL)
+{
   CMVerboseLevel verbose_level = (CMVerboseLevel) G1MarkingVerboseLevel;
   if (verbose_level < no_verbose) {
     verbose_level = no_verbose;
   }
   if (verbose_level > high_verbose) {

@@ -539,21 +544,40 @@
   satb_qs.set_buffer_size(G1SATBBufferSize);
 
   _tasks = NEW_C_HEAP_ARRAY(CMTask*, _max_task_num);
   _accum_task_vtime = NEW_C_HEAP_ARRAY(double, _max_task_num);
 
+  _count_card_bitmaps = NEW_C_HEAP_ARRAY(BitMap,  _max_task_num);
+  _count_marked_bytes = NEW_C_HEAP_ARRAY(size_t*, _max_task_num);
+
+  BitMap::idx_t card_bm_size = _card_bm.size();
+
   // so that the assertion in MarkingTaskQueue::task_queue doesn't fail
   _active_tasks = _max_task_num;
   for (int i = 0; i < (int) _max_task_num; ++i) {
     CMTaskQueue* task_queue = new CMTaskQueue();
     task_queue->initialize();
     _task_queues->register_queue(i, task_queue);
 
-    _tasks[i] = new CMTask(i, this, task_queue, _task_queues);
+    _count_card_bitmaps[i] = BitMap(card_bm_size, false);
+    _count_marked_bytes[i] = NEW_C_HEAP_ARRAY(size_t, max_regions);
+
+    _tasks[i] = new CMTask(i, this, 
+                           _count_marked_bytes[i],
+                           &_count_card_bitmaps[i],
+                           task_queue, _task_queues);
+
     _accum_task_vtime[i] = 0.0;
   }
 
+  // Calculate the card number for the bottom of the heap. Used
+  // in biasing indexes into the accounting card bitmaps.
+  _heap_bottom_card_num =
+    intptr_t(uintptr_t(_g1h->reserved_region().start()) >> 
+                                CardTableModRefBS::card_shift);
+
+
   if (ConcGCThreads > ParallelGCThreads) {
     vm_exit_during_initialization("Can't have more ConcGCThreads "
                                   "than ParallelGCThreads.");
   }
   if (ParallelGCThreads == 0) {

@@ -673,10 +697,12 @@
   assert(_heap_start < _heap_end, "heap bounds should look ok");
 
   // reset all the marking data structures and any necessary flags
   clear_marking_state();
 
+  clear_all_count_data();
+
   if (verbose_low()) {
     gclog_or_tty->print_cr("[global] resetting");
   }
 
   // We do reset all of them, since different phases will use

@@ -724,19 +750,10 @@
   clear_marking_state();
   _active_tasks = 0;
   clear_concurrent_marking_in_progress();
 }
 
-ConcurrentMark::~ConcurrentMark() {
-  for (int i = 0; i < (int) _max_task_num; ++i) {
-    delete _task_queues->queue(i);
-    delete _tasks[i];
-  }
-  delete _task_queues;
-  FREE_C_HEAP_ARRAY(CMTask*, _max_task_num);
-}
-
 // This closure is used to mark refs into the g1 generation
 // from external roots in the CMS bit map.
 // Called at the first checkpoint.
 //
 

@@ -949,22 +966,22 @@
     return false;
   }
 }
 #endif // !PRODUCT
 
-void ConcurrentMark::grayRoot(oop p) {
+void ConcurrentMark::grayRoot(oop p, int worker_i) {
   HeapWord* addr = (HeapWord*) p;
   // We can't really check against _heap_start and _heap_end, since it
   // is possible during an evacuation pause with piggy-backed
   // initial-mark that the committed space is expanded during the
   // pause without CM observing this change. So the assertions below
   // is a bit conservative; but better than nothing.
   assert(_g1h->g1_committed().contains(addr),
          "address should be within the heap bounds");
 
   if (!_nextMarkBitMap->isMarked(addr)) {
-    _nextMarkBitMap->parMark(addr);
+    par_mark_and_count(p, worker_i);
   }
 }
 
 void ConcurrentMark::grayRegionIfNecessary(MemRegion mr) {
   // The objects on the region have already been marked "in bulk" by

@@ -1009,19 +1026,20 @@
       }
     }
   }
 }
 
-void ConcurrentMark::markAndGrayObjectIfNecessary(oop p) {
+void ConcurrentMark::markAndGrayObjectIfNecessary(oop p, int worker_i) {
   // The object is not marked by the caller. We need to at least mark
   // it and maybe push in on the stack.
 
   HeapWord* addr = (HeapWord*)p;
   if (!_nextMarkBitMap->isMarked(addr)) {
     // We definitely need to mark it, irrespective whether we bail out
     // because we're done with marking.
-    if (_nextMarkBitMap->parMark(addr)) {
+
+    if (par_mark_and_count(p, worker_i)) {
       if (!concurrent_marking_in_progress() || !_should_gray_objects) {
         // If we're done with concurrent marking and we're waiting for
         // remark, then we're not pushing anything on the stack.
         return;
       }

@@ -1219,10 +1237,14 @@
     clear_has_overflown();
     if (G1TraceMarkStackOverflow) {
       gclog_or_tty->print_cr("\nRemark led to restart for overflow.");
     }
   } else {
+    // Aggregate the per-task counting data that we have accumulated
+    // while marking.
+    aggregate_and_clear_count_data();
+
     SATBMarkQueueSet& satb_mq_set = JavaThread::satb_mark_queue_set();
     // We're done with marking.
     // This is the end of  the marking cycle, we're expected all
     // threads to have SATB queues with active set to true.
     satb_mq_set.set_active_all_threads(false, /* new active value */

@@ -1259,49 +1281,48 @@
   g1p->record_concurrent_mark_remark_end();
 }
 
 #define CARD_BM_TEST_MODE 0
 
+// Used to calculate the # live objects per region
+// for verification purposes
 class CalcLiveObjectsClosure: public HeapRegionClosure {
 
   CMBitMapRO* _bm;
   ConcurrentMark* _cm;
-  bool _changed;
-  bool _yield;
-  size_t _words_done;
+  BitMap* _region_bm;
+  BitMap* _card_bm;
+
+  size_t _tot_words_done;
   size_t _tot_live;
   size_t _tot_used;
-  size_t _regions_done;
-  double _start_vtime_sec;
 
-  BitMap* _region_bm;
-  BitMap* _card_bm;
+  size_t _region_marked_bytes;
+
   intptr_t _bottom_card_num;
-  bool _final;
 
   void mark_card_num_range(intptr_t start_card_num, intptr_t last_card_num) {
-    for (intptr_t i = start_card_num; i <= last_card_num; i++) {
+    BitMap::idx_t start_idx = start_card_num - _bottom_card_num;
+    BitMap::idx_t last_idx = last_card_num - _bottom_card_num;
+    
+    for (BitMap::idx_t i = start_idx; i <= last_idx; i += 1) {
 #if CARD_BM_TEST_MODE
-      guarantee(_card_bm->at(i - _bottom_card_num), "Should already be set.");
+      guarantee(_card_bm->at(i), "Should already be set.");
 #else
-      _card_bm->par_at_put(i - _bottom_card_num, 1);
+      _card_bm->par_at_put(i, 1);
 #endif
     }
   }
 
 public:
-  CalcLiveObjectsClosure(bool final,
-                         CMBitMapRO *bm, ConcurrentMark *cm,
+  CalcLiveObjectsClosure(CMBitMapRO *bm, ConcurrentMark *cm,
                          BitMap* region_bm, BitMap* card_bm) :
-    _bm(bm), _cm(cm), _changed(false), _yield(true),
-    _words_done(0), _tot_live(0), _tot_used(0),
-    _region_bm(region_bm), _card_bm(card_bm),_final(final),
-    _regions_done(0), _start_vtime_sec(0.0)
+    _bm(bm), _cm(cm), _region_bm(region_bm), _card_bm(card_bm),
+    _region_marked_bytes(0), _tot_words_done(0),
+    _tot_live(0), _tot_used(0)
   {
-    _bottom_card_num =
-      intptr_t(uintptr_t(G1CollectedHeap::heap()->reserved_region().start()) >>
-               CardTableModRefBS::card_shift);
+    _bottom_card_num = cm->heap_bottom_card_num();
   }
 
   // It takes a region that's not empty (i.e., it has at least one
   // live object in it and sets its corresponding bit on the region
   // bitmap to 1. If the region is "starts humongous" it will also set

@@ -1314,33 +1335,20 @@
     if (!hr->startsHumongous()) {
       // Normal (non-humongous) case: just set the bit.
       _region_bm->par_at_put((BitMap::idx_t) index, true);
     } else {
       // Starts humongous case: calculate how many regions are part of
-      // this humongous region and then set the bit range. It might
-      // have been a bit more efficient to look at the object that
-      // spans these humongous regions to calculate their number from
-      // the object's size. However, it's a good idea to calculate
-      // this based on the metadata itself, and not the region
-      // contents, so that this code is not aware of what goes into
-      // the humongous regions (in case this changes in the future).
+      // this humongous region and then set the bit range.
       G1CollectedHeap* g1h = G1CollectedHeap::heap();
-      size_t end_index = index + 1;
-      while (end_index < g1h->n_regions()) {
-        HeapRegion* chr = g1h->region_at(end_index);
-        if (!chr->continuesHumongous()) break;
-        end_index += 1;
-      }
+      HeapRegion *last_hr = g1h->heap_region_containing_raw(hr->end() - 1);
+      size_t end_index = last_hr->hrs_index() + 1;
       _region_bm->par_at_put_range((BitMap::idx_t) index,
                                    (BitMap::idx_t) end_index, true);
     }
   }
 
   bool doHeapRegion(HeapRegion* hr) {
-    if (!_final && _regions_done == 0) {
-      _start_vtime_sec = os::elapsedVTime();
-    }
 
     if (hr->continuesHumongous()) {
       // We will ignore these here and process them when their
       // associated "starts humongous" region is processed (see
       // set_bit_for_heap_region()). Note that we cannot rely on their

@@ -1350,52 +1358,44 @@
       // before its associated "starts humongous".
       return false;
     }
 
     HeapWord* nextTop = hr->next_top_at_mark_start();
-    HeapWord* start   = hr->top_at_conc_mark_count();
-    assert(hr->bottom() <= start && start <= hr->end() &&
-           hr->bottom() <= nextTop && nextTop <= hr->end() &&
-           start <= nextTop,
+    HeapWord* start   = hr->bottom();
+
+    assert(start <= hr->end() && start <= nextTop && nextTop <= hr->end(),
            "Preconditions.");
-    // Otherwise, record the number of word's we'll examine.
+
+    // Record the number of word's we'll examine.
     size_t words_done = (nextTop - start);
+
     // Find the first marked object at or after "start".
     start = _bm->getNextMarkedWordAddress(start, nextTop);
+
     size_t marked_bytes = 0;
+    _region_marked_bytes = 0;
 
     // Below, the term "card num" means the result of shifting an address
     // by the card shift -- address 0 corresponds to card number 0.  One
     // must subtract the card num of the bottom of the heap to obtain a
     // card table index.
+
     // The first card num of the sequence of live cards currently being
     // constructed.  -1 ==> no sequence.
     intptr_t start_card_num = -1;
+
     // The last card num of the sequence of live cards currently being
     // constructed.  -1 ==> no sequence.
     intptr_t last_card_num = -1;
 
     while (start < nextTop) {
-      if (_yield && _cm->do_yield_check()) {
-        // We yielded.  It might be for a full collection, in which case
-        // all bets are off; terminate the traversal.
-        if (_cm->has_aborted()) {
-          _changed = false;
-          return true;
-        } else {
-          // Otherwise, it might be a collection pause, and the region
-          // we're looking at might be in the collection set.  We'll
-          // abandon this region.
-          return false;
-        }
-      }
       oop obj = oop(start);
       int obj_sz = obj->size();
+
       // The card num of the start of the current object.
       intptr_t obj_card_num =
         intptr_t(uintptr_t(start) >> CardTableModRefBS::card_shift);
-
       HeapWord* obj_last = start + obj_sz - 1;
       intptr_t obj_last_card_num =
         intptr_t(uintptr_t(obj_last) >> CardTableModRefBS::card_shift);
 
       if (obj_card_num != last_card_num) {

@@ -1410,111 +1410,429 @@
             mark_card_num_range(start_card_num, last_card_num);
             start_card_num = obj_card_num;
           }
         }
 #if CARD_BM_TEST_MODE
-        /*
-        gclog_or_tty->print_cr("Setting bits from %d/%d.",
-                               obj_card_num - _bottom_card_num,
-                               obj_last_card_num - _bottom_card_num);
-        */
         for (intptr_t j = obj_card_num; j <= obj_last_card_num; j++) {
           _card_bm->par_at_put(j - _bottom_card_num, 1);
         }
-#endif
+#endif // CARD_BM_TEST_MODE
       }
       // In any case, we set the last card num.
       last_card_num = obj_last_card_num;
 
       marked_bytes += (size_t)obj_sz * HeapWordSize;
+
       // Find the next marked object after this one.
       start = _bm->getNextMarkedWordAddress(start + 1, nextTop);
-      _changed = true;
     }
+
     // Handle the last range, if any.
     if (start_card_num != -1) {
       mark_card_num_range(start_card_num, last_card_num);
     }
-    if (_final) {
+
       // Mark the allocated-since-marking portion...
-      HeapWord* tp = hr->top();
-      if (nextTop < tp) {
-        start_card_num =
-          intptr_t(uintptr_t(nextTop) >> CardTableModRefBS::card_shift);
-        last_card_num =
-          intptr_t(uintptr_t(tp) >> CardTableModRefBS::card_shift);
+    HeapWord* top = hr->top();
+    if (nextTop < top) {
+      start_card_num = intptr_t(uintptr_t(nextTop) >> CardTableModRefBS::card_shift);
+      last_card_num = intptr_t(uintptr_t(top) >> CardTableModRefBS::card_shift);
+
         mark_card_num_range(start_card_num, last_card_num);
+      
         // This definitely means the region has live objects.
         set_bit_for_region(hr);
       }
-    }
 
-    hr->add_to_marked_bytes(marked_bytes);
     // Update the live region bitmap.
     if (marked_bytes > 0) {
       set_bit_for_region(hr);
     }
-    hr->set_top_at_conc_mark_count(nextTop);
+
+    // Set the marked bytes for the current region so that
+    // it can be queried by a calling verificiation routine
+    _region_marked_bytes = marked_bytes;
+
     _tot_live += hr->next_live_bytes();
     _tot_used += hr->used();
-    _words_done = words_done;
+    _tot_words_done = words_done;
 
-    if (!_final) {
-      ++_regions_done;
-      if (_regions_done % 10 == 0) {
-        double end_vtime_sec = os::elapsedVTime();
-        double elapsed_vtime_sec = end_vtime_sec - _start_vtime_sec;
-        if (elapsed_vtime_sec > (10.0 / 1000.0)) {
-          jlong sleep_time_ms =
-            (jlong) (elapsed_vtime_sec * _cm->cleanup_sleep_factor() * 1000.0);
-          os::sleep(Thread::current(), sleep_time_ms, false);
-          _start_vtime_sec = end_vtime_sec;
+    return false;
+  }
+
+  size_t region_marked_bytes() const { return _region_marked_bytes; }
+  size_t tot_words_done() const      { return _tot_words_done; }
+  size_t tot_live() const            { return _tot_live; }
+  size_t tot_used() const            { return _tot_used; }
+};
+
+// Heap region closure used for verifying the counting data
+// that was accumulated concurrently and aggregated during
+// the remark pause. This closure is applied to the heap
+// regions during the STW cleanup pause.
+
+class VerifyLiveObjectDataHRClosure: public HeapRegionClosure {
+  ConcurrentMark* _cm;
+  CalcLiveObjectsClosure _calc_cl;
+  BitMap* _region_bm;   // Region BM to be verified
+  BitMap* _card_bm;     // Card BM to be verified
+  bool _verbose;        // verbose output?
+
+  BitMap* _exp_region_bm; // Expected Region BM values
+  BitMap* _exp_card_bm;   // Expected card BM values
+
+  intptr_t _bottom_card_num; // Used for calculatint bitmap indices
+
+  int _failures;
+
+public:
+  VerifyLiveObjectDataHRClosure(ConcurrentMark* cm,
+                                BitMap* region_bm,
+                                BitMap* card_bm,
+                                BitMap* exp_region_bm,
+                                BitMap* exp_card_bm,
+                                bool verbose) :
+    _cm(cm),
+    _calc_cl(_cm->nextMarkBitMap(), _cm, exp_region_bm, exp_card_bm),
+    _region_bm(region_bm), _card_bm(card_bm), _verbose(verbose),
+    _exp_region_bm(exp_region_bm), _exp_card_bm(exp_card_bm),
+    _failures(0)
+  { 
+    _bottom_card_num = cm->heap_bottom_card_num();
+  }
+
+  int failures() const { return _failures; }
+
+  bool doHeapRegion(HeapRegion* hr) {
+    if (hr->continuesHumongous()) {
+      // We will ignore these here and process them when their
+      // associated "starts humongous" region is processed (see
+      // set_bit_for_heap_region()). Note that we cannot rely on their
+      // associated "starts humongous" region to have their bit set to
+      // 1 since, due to the region chunking in the parallel region
+      // iteration, a "continues humongous" region might be visited
+      // before its associated "starts humongous".
+      return false;
         }
+
+    int failures = 0;
+
+    // Call the CalcLiveObjectsClosure to walk the marking bitmap for
+    // this region and set the corresponding bits in the expected region
+    // and card bitmaps.
+    bool res = _calc_cl.doHeapRegion(hr);
+    assert(res == false, "should be continuing");
+
+    // Note that the calculated count data could be a subset of the
+    // count data that was accumlated during marking. See the comment
+    // in G1ParCopyHelper::copy_to_survivor space for an explanation
+    // why.
+
+    // Verify that _top_at_conc_count == ntams
+    if (hr->top_at_conc_mark_count() != hr->next_top_at_mark_start()) {
+      if (_verbose) {
+        gclog_or_tty->print_cr("Region %d: top at conc count incorrect: expected "
+                               PTR_FORMAT", actual: "PTR_FORMAT,
+                               hr->hrs_index(), hr->next_top_at_mark_start(),
+                               hr->top_at_conc_mark_count());
       }
+      failures += 1;
     }
 
+    // Verify the marked bytes for this region. 
+    size_t exp_marked_bytes = _calc_cl.region_marked_bytes();
+    size_t act_marked_bytes = hr->next_marked_bytes();
+    
+    // We're not OK if expected marked bytes > actual marked bytes. It means
+    // we have missed accounting some objects during the actual marking.
+    if (exp_marked_bytes > act_marked_bytes) {
+      if (_verbose) {
+        gclog_or_tty->print_cr("Region %d: marked bytes mismatch: expected: "
+                               SIZE_FORMAT", actual: "SIZE_FORMAT,
+                               hr->hrs_index(), exp_marked_bytes, act_marked_bytes);
+      }
+      failures += 1;
+    }
+
+    // Verify the bit, for this region, in the actual and expected
+    // (which was just calculated) region bit maps.
+    // We're not OK if the expected bit is set and the actual is not set.
+    BitMap::idx_t index = (BitMap::idx_t)hr->hrs_index();
+    
+    bool expected = _exp_region_bm->at(index);
+    bool actual = _region_bm->at(index);
+    if (expected && !actual) {
+      if (_verbose) {
+        gclog_or_tty->print_cr("Region %d: region bitmap mismatch: expected: %d, actual: %d",
+                               hr->hrs_index(), expected, actual);
+      }
+      failures += 1;
+    }
+
+    // Verify that the card bit maps for the cards spanned by the current
+    // region match. The set of offsets that have set bits in the expected
+    // bitmap should be a subset of the offsets with set bits from the actual
+    // calculated card bitmap.
+    // Again it's more important that if the expected bit is set then the
+    // actual bit be set.
+    intptr_t start_card_num =
+        intptr_t(uintptr_t(hr->bottom()) >> CardTableModRefBS::card_shift);
+    intptr_t top_card_num =
+        intptr_t(uintptr_t(hr->top()) >> CardTableModRefBS::card_shift);
+
+    BitMap::idx_t start_idx = start_card_num - _bottom_card_num;
+    BitMap::idx_t end_idx = top_card_num - _bottom_card_num;
+
+    for (BitMap::idx_t i = start_idx; i < end_idx; i+=1) {
+      expected = _exp_card_bm->at(i);
+      actual = _card_bm->at(i);
+      
+      if (expected && !actual) {
+        if (_verbose) {
+          gclog_or_tty->print_cr("Region %d: card bitmap mismatch at idx %d: expected: %d, actual: %d",
+                                 hr->hrs_index(), i, expected, actual);
+        }
+        failures += 1;
+      }
+    }
+
+    if (failures > 0 && _verbose)  {
+      gclog_or_tty->print("Region %d: bottom: "PTR_FORMAT", ntams: "
+                          PTR_FORMAT", top: "PTR_FORMAT", end: "PTR_FORMAT,
+                          hr->hrs_index(), hr->bottom(), hr->next_top_at_mark_start(),
+                          hr->top(), hr->end());
+      gclog_or_tty->print_cr(", marked_bytes: calc/actual "SIZE_FORMAT"/"SIZE_FORMAT,
+                             _calc_cl.region_marked_bytes(),
+                             hr->next_marked_bytes());
+    }
+
+    _failures += failures;
+
+    // We could stop iteration over the heap when we
+    // find the first voilating region by returning true.
     return false;
   }
+};
+
+
+class G1ParVerifyFinalCountTask: public AbstractGangTask {
+protected:
+  G1CollectedHeap* _g1h;
+  ConcurrentMark* _cm;
+  BitMap* _actual_region_bm;
+  BitMap* _actual_card_bm;
+
+  size_t _n_workers;
+
+  BitMap* _expected_region_bm;
+  BitMap* _expected_card_bm;
+
+  int  _failures;
+  bool _verbose;
+
+public:
+  G1ParVerifyFinalCountTask(G1CollectedHeap* g1h,
+                            BitMap* region_bm, BitMap* card_bm,
+                            BitMap* expected_region_bm, BitMap* expected_card_bm)
+    : AbstractGangTask("G1 verify final counting"),
+      _g1h(g1h), _cm(_g1h->concurrent_mark()),
+      _actual_region_bm(region_bm), _actual_card_bm(card_bm),
+      _expected_region_bm(expected_region_bm), _expected_card_bm(expected_card_bm),
+      _failures(0), _verbose(false),
+      _n_workers(0)
+  {
+    assert(VerifyDuringGC, "don't call this otherwise");
+
+    // Use the value already set as the number of active threads
+    // in the call to run_task().
+    if (G1CollectedHeap::use_parallel_gc_threads()) {
+      assert( _g1h->workers()->active_workers() > 0,
+        "Should have been previously set");
+      _n_workers = _g1h->workers()->active_workers();
+    } else {
+      _n_workers = 1;
+    }
+
+    assert(_expected_card_bm->size() == _actual_card_bm->size(), "sanity");
+    assert(_expected_region_bm->size() == _actual_region_bm->size(), "sanity");
+
+    _verbose = _cm->verbose_medium();
+  }
+
+  void work(int worker_i) {
+    assert((size_t) worker_i < _n_workers, "invariant");
+
+    VerifyLiveObjectDataHRClosure verify_cl(_cm,
+                                            _actual_region_bm, _actual_card_bm,
+                                            _expected_region_bm,
+                                            _expected_card_bm,
+                                            _verbose);
+
+    if (G1CollectedHeap::use_parallel_gc_threads()) {
+      _g1h->heap_region_par_iterate_chunked(&verify_cl,
+                                            worker_i,
+                                            (int) _n_workers,
+                                            HeapRegion::VerifyCountClaimValue);
+    } else {
+      _g1h->heap_region_iterate(&verify_cl);
+    }
 
-  bool changed() { return _changed;  }
-  void reset()   { _changed = false; _words_done = 0; }
-  void no_yield() { _yield = false; }
-  size_t words_done() { return _words_done; }
-  size_t tot_live() { return _tot_live; }
-  size_t tot_used() { return _tot_used; }
+    Atomic::add(verify_cl.failures(), &_failures);
+  }
+
+  int failures() const { return _failures; }
 };
 
+// Final update of count data (during cleanup).
+// Adds [top_at_count, NTAMS) to the marked bytes for each
+// region. Sets the bits in the card bitmap corresponding
+// to the interval [top_at_count, top], and sets the
+// liveness bit for each region containing live data
+// in the region bitmap.
 
-void ConcurrentMark::calcDesiredRegions() {
-  _region_bm.clear();
-  _card_bm.clear();
-  CalcLiveObjectsClosure calccl(false /*final*/,
-                                nextMarkBitMap(), this,
-                                &_region_bm, &_card_bm);
-  G1CollectedHeap *g1h = G1CollectedHeap::heap();
-  g1h->heap_region_iterate(&calccl);
+class FinalCountDataUpdateClosure: public HeapRegionClosure {
+  ConcurrentMark* _cm;
+  BitMap* _region_bm;
+  BitMap* _card_bm;
+  intptr_t _bottom_card_num;
 
-  do {
-    calccl.reset();
-    g1h->heap_region_iterate(&calccl);
-  } while (calccl.changed());
-}
+  size_t _total_live_bytes;
+  size_t _total_used_bytes;
+  size_t _total_words_done;
+
+  void mark_card_num_range(intptr_t start_card_num, intptr_t last_card_num) {
+    BitMap::idx_t start_idx = start_card_num - _bottom_card_num;
+    BitMap::idx_t last_idx = last_card_num - _bottom_card_num;
+    
+    // Inclusive bit range [start_idx, last_idx]. par_at_put_range
+    // is exclusive so we have to also set the bit for last_idx.
+    // Passing last_idx+1 to the clear_range would work in
+    // most cases but could trip an OOB assertion.
+
+    if ((last_idx - start_idx) > 0) {
+      _card_bm->par_at_put_range(start_idx, last_idx, true);
+    }
+    _card_bm->par_set_bit(last_idx);
+  }
+
+  // It takes a region that's not empty (i.e., it has at least one
+  // live object in it and sets its corresponding bit on the region
+  // bitmap to 1. If the region is "starts humongous" it will also set
+  // to 1 the bits on the region bitmap that correspond to its
+  // associated "continues humongous" regions.
+  void set_bit_for_region(HeapRegion* hr) {
+    assert(!hr->continuesHumongous(), "should have filtered those out");
+
+    size_t index = hr->hrs_index();
+    if (!hr->startsHumongous()) {
+      // Normal (non-humongous) case: just set the bit.
+      _region_bm->par_set_bit((BitMap::idx_t) index);
+    } else {
+      // Starts humongous case: calculate how many regions are part of
+      // this humongous region and then set the bit range.
+      G1CollectedHeap* g1h = G1CollectedHeap::heap();
+      HeapRegion *last_hr = g1h->heap_region_containing_raw(hr->end() - 1);
+      size_t end_index = last_hr->hrs_index() + 1;
+      _region_bm->par_at_put_range((BitMap::idx_t) index,
+                                   (BitMap::idx_t) end_index, true);
+    }
+  }
+
+ public:
+  FinalCountDataUpdateClosure(ConcurrentMark* cm,
+                              BitMap* region_bm,
+                              BitMap* card_bm) :
+    _cm(cm), _region_bm(region_bm), _card_bm(card_bm),
+    _total_words_done(0), _total_live_bytes(0), _total_used_bytes(0)
+  {
+    _bottom_card_num = cm->heap_bottom_card_num();
+  }
+
+  bool doHeapRegion(HeapRegion* hr) {
+
+    if (hr->continuesHumongous()) {
+      // We will ignore these here and process them when their
+      // associated "starts humongous" region is processed (see
+      // set_bit_for_heap_region()). Note that we cannot rely on their
+      // associated "starts humongous" region to have their bit set to
+      // 1 since, due to the region chunking in the parallel region
+      // iteration, a "continues humongous" region might be visited
+      // before its associated "starts humongous".
+      return false;
+    }
+
+    HeapWord* start = hr->top_at_conc_mark_count();
+    HeapWord* ntams = hr->next_top_at_mark_start();
+    HeapWord* top   = hr->top();
+    
+    assert(hr->bottom() <= start && start <= hr->end() &&
+           hr->bottom() <= ntams && ntams <= hr->end(), "Preconditions.");
+    
+    size_t words_done = ntams - hr->bottom();
+
+    intptr_t start_card_num = intptr_t(uintptr_t(start) >> CardTableModRefBS::card_shift);
+    intptr_t last_card_num = intptr_t(uintptr_t(top) >> CardTableModRefBS::card_shift);
+
+
+    if (start < ntams) {
+      // Region was changed between remark and cleanup pauses
+      // We need to add (ntams - start) to the marked bytes
+      // for this region, and set bits for the range
+      // [ card_num(start), card_num(ntams) ) in the
+      // card bitmap.
+      size_t live_bytes = (ntams - start) * HeapWordSize;
+      hr->add_to_marked_bytes(live_bytes);
+      
+      // Record the new top at conc count
+      hr->set_top_at_conc_mark_count(ntams);
+
+      // The setting of the bits card bitmap takes place below
+    }
+
+    // Mark the allocated-since-marking portion...
+    if (ntams < top) {
+      // This definitely means the region has live objects.
+      set_bit_for_region(hr);
+    }
+
+    // Now set the bits for [start, top]
+    mark_card_num_range(start_card_num, last_card_num);
+
+    // Set the bit for the region if it contains live data
+    if (hr->next_marked_bytes() > 0) {
+      set_bit_for_region(hr);
+    }
+
+    _total_words_done += words_done;
+    _total_used_bytes += hr->used();
+    _total_live_bytes += hr->next_marked_bytes();
+
+    return false;
+  }
+
+  size_t total_words_done() const { return _total_words_done; }
+  size_t total_live_bytes() const { return _total_live_bytes; }
+  size_t total_used_bytes() const { return _total_used_bytes; }
+};
 
 class G1ParFinalCountTask: public AbstractGangTask {
 protected:
   G1CollectedHeap* _g1h;
-  CMBitMap* _bm;
+  ConcurrentMark* _cm;
+  BitMap* _actual_region_bm;
+  BitMap* _actual_card_bm;
+  
   size_t _n_workers;
+
   size_t *_live_bytes;
   size_t *_used_bytes;
-  BitMap* _region_bm;
-  BitMap* _card_bm;
+
 public:
-  G1ParFinalCountTask(G1CollectedHeap* g1h, CMBitMap* bm,
-                      BitMap* region_bm, BitMap* card_bm)
-    : AbstractGangTask("G1 final counting"), _g1h(g1h),
-    _bm(bm), _region_bm(region_bm), _card_bm(card_bm),
+  G1ParFinalCountTask(G1CollectedHeap* g1h, BitMap* region_bm, BitMap* card_bm)
+    : AbstractGangTask("G1 final counting"),
+      _g1h(g1h), _cm(_g1h->concurrent_mark()),
+      _actual_region_bm(region_bm), _actual_card_bm(card_bm),
     _n_workers(0)
   {
     // Use the value already set as the number of active threads
     // in the call to run_task().  Needed for the allocation of
     // _live_bytes and _used_bytes.

@@ -1533,34 +1851,37 @@
   ~G1ParFinalCountTask() {
     FREE_C_HEAP_ARRAY(size_t, _live_bytes);
     FREE_C_HEAP_ARRAY(size_t, _used_bytes);
   }
 
-  void work(int i) {
-    CalcLiveObjectsClosure calccl(true /*final*/,
-                                  _bm, _g1h->concurrent_mark(),
-                                  _region_bm, _card_bm);
-    calccl.no_yield();
+  void work(int worker_i) {
+    assert((size_t) worker_i < _n_workers, "invariant");
+
+    FinalCountDataUpdateClosure final_update_cl(_cm,
+                                                _actual_region_bm,
+                                                _actual_card_bm);
+
     if (G1CollectedHeap::use_parallel_gc_threads()) {
-      _g1h->heap_region_par_iterate_chunked(&calccl, i,
+      _g1h->heap_region_par_iterate_chunked(&final_update_cl,
+                                            worker_i,
                                             (int) _n_workers,
                                             HeapRegion::FinalCountClaimValue);
     } else {
-      _g1h->heap_region_iterate(&calccl);
+      _g1h->heap_region_iterate(&final_update_cl);
     }
-    assert(calccl.complete(), "Shouldn't have yielded!");
 
-    assert((size_t) i < _n_workers, "invariant");
-    _live_bytes[i] = calccl.tot_live();
-    _used_bytes[i] = calccl.tot_used();
+    _live_bytes[worker_i] = final_update_cl.total_live_bytes();
+    _used_bytes[worker_i] = final_update_cl.total_used_bytes();
   }
+
   size_t live_bytes()  {
     size_t live_bytes = 0;
     for (size_t i = 0; i < _n_workers; ++i)
       live_bytes += _live_bytes[i];
     return live_bytes;
   }
+
   size_t used_bytes()  {
     size_t used_bytes = 0;
     for (size_t i = 0; i < _n_workers; ++i)
       used_bytes += _used_bytes[i];
     return used_bytes;

@@ -1764,36 +2085,67 @@
 
   double start = os::elapsedTime();
 
   HeapRegionRemSet::reset_for_cleanup_tasks();
 
+  // Clear the global region bitmap - it will be filled as part
+  // of the final counting task.
+  _region_bm.clear();
+
   size_t n_workers;
 
   // Do counting once more with the world stopped for good measure.
-  G1ParFinalCountTask g1_par_count_task(g1h, nextMarkBitMap(),
-                                        &_region_bm, &_card_bm);
+  G1ParFinalCountTask g1_par_count_task(g1h, &_region_bm, &_card_bm);
+
   if (G1CollectedHeap::use_parallel_gc_threads()) {
-    assert(g1h->check_heap_region_claim_values(
-                                               HeapRegion::InitialClaimValue),
+   assert(g1h->check_heap_region_claim_values(HeapRegion::InitialClaimValue),
            "sanity check");
 
     g1h->set_par_threads();
     n_workers = g1h->n_par_threads();
     assert(g1h->n_par_threads() == (int) n_workers,
            "Should not have been reset");
     g1h->workers()->run_task(&g1_par_count_task);
     // Done with the parallel phase so reset to 0.
     g1h->set_par_threads(0);
 
-    assert(g1h->check_heap_region_claim_values(
-                                             HeapRegion::FinalCountClaimValue),
+    assert(g1h->check_heap_region_claim_values(HeapRegion::FinalCountClaimValue),
            "sanity check");
   } else {
     n_workers = 1;
     g1_par_count_task.work(0);
   }
 
+  if (VerifyDuringGC) {
+    // Verify that the counting data accumulated during marking matches
+    // that calculated by walking the marking bitmap.
+    
+    // Bitmaps to hold expected values
+    BitMap expected_region_bm(_region_bm.size(), false);
+    BitMap expected_card_bm(_card_bm.size(), false);
+
+    G1ParVerifyFinalCountTask g1_par_verify_task(g1h,
+                                                 &_region_bm,
+                                                 &_card_bm,
+                                                 &expected_region_bm,
+                                                 &expected_card_bm);
+    
+    if (G1CollectedHeap::use_parallel_gc_threads()) {
+      g1h->set_par_threads((int)n_workers);
+      g1h->workers()->run_task(&g1_par_verify_task);
+      // Done with the parallel phase so reset to 0.
+      g1h->set_par_threads(0);
+      
+      assert(g1h->check_heap_region_claim_values(HeapRegion::VerifyCountClaimValue),
+             "sanity check");
+    } else {
+      g1_par_verify_task.work(0);
+    }
+
+    guarantee(g1_par_verify_task.failures() == 0, "Unexpected accounting failures");
+  }
+
   size_t known_garbage_bytes =
     g1_par_count_task.used_bytes() - g1_par_count_task.live_bytes();
   g1p->set_known_garbage_bytes(known_garbage_bytes);
 
   size_t start_used_bytes = g1h->used();

@@ -1982,16 +2334,16 @@
 }
 
 class G1CMKeepAliveClosure: public OopClosure {
   G1CollectedHeap* _g1;
   ConcurrentMark*  _cm;
-  CMBitMap*        _bitMap;
  public:
-  G1CMKeepAliveClosure(G1CollectedHeap* g1, ConcurrentMark* cm,
-                       CMBitMap* bitMap) :
-    _g1(g1), _cm(cm),
-    _bitMap(bitMap) {}
+  G1CMKeepAliveClosure(G1CollectedHeap* g1, ConcurrentMark* cm) :
+    _g1(g1), _cm(cm)
+  {
+    assert(Thread::current()->is_VM_thread(), "otherwise fix worker id");
+  }
 
   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) {

@@ -2003,30 +2355,30 @@
                              "*"PTR_FORMAT" = "PTR_FORMAT,
                              p, (void*) obj);
     }
 
     if (_g1->is_in_g1_reserved(addr) && _g1->is_obj_ill(obj)) {
-      _bitMap->mark(addr);
+      _cm->mark_and_count(obj);
       _cm->mark_stack_push(obj);
     }
   }
 };
 
 class G1CMDrainMarkingStackClosure: public VoidClosure {
+  ConcurrentMark*               _cm;
   CMMarkStack*                  _markStack;
-  CMBitMap*                     _bitMap;
   G1CMKeepAliveClosure*         _oopClosure;
  public:
-  G1CMDrainMarkingStackClosure(CMBitMap* bitMap, CMMarkStack* markStack,
+  G1CMDrainMarkingStackClosure(ConcurrentMark* cm, CMMarkStack* markStack,
                                G1CMKeepAliveClosure* oopClosure) :
-    _bitMap(bitMap),
+    _cm(cm),
     _markStack(markStack),
     _oopClosure(oopClosure)
   {}
 
   void do_void() {
-    _markStack->drain((OopClosure*)_oopClosure, _bitMap, false);
+    _markStack->drain((OopClosure*)_oopClosure, _cm->nextMarkBitMap(), false);
   }
 };
 
 // 'Keep Alive' closure used by parallel reference processing.
 // An instance of this closure is used in the parallel reference processing

@@ -2241,13 +2593,13 @@
 
     // Process weak references.
     rp->setup_policy(clear_all_soft_refs);
     assert(_markStack.isEmpty(), "mark stack should be empty");
 
-    G1CMKeepAliveClosure g1_keep_alive(g1h, this, nextMarkBitMap());
+    G1CMKeepAliveClosure g1_keep_alive(g1h, this);
     G1CMDrainMarkingStackClosure
-      g1_drain_mark_stack(nextMarkBitMap(), &_markStack, &g1_keep_alive);
+      g1_drain_mark_stack(this, &_markStack, &g1_keep_alive);
 
     // We use the work gang from the G1CollectedHeap and we utilize all
     // the worker threads.
     int active_workers = g1h->workers() ? g1h->workers()->active_workers() : 1;
     active_workers = MAX2(MIN2(active_workers, (int)_max_task_num), 1);

@@ -2621,17 +2973,17 @@
 private:
   ConcurrentMark* _cm;
 
 public:
   void do_object(oop obj) {
-    _cm->deal_with_reference(obj);
+    _cm->deal_with_reference(obj, 0);
   }
 
   CMGlobalObjectClosure(ConcurrentMark* cm) : _cm(cm) { }
 };
 
-void ConcurrentMark::deal_with_reference(oop obj) {
+void ConcurrentMark::deal_with_reference(oop obj, int worker_i) {
   if (verbose_high()) {
     gclog_or_tty->print_cr("[global] we're dealing with reference "PTR_FORMAT,
                            (void*) obj);
   }
 

@@ -2649,13 +3001,14 @@
           gclog_or_tty->print_cr("[global] "PTR_FORMAT" is not considered "
                                  "marked", (void*) obj);
         }
 
         // we need to mark it first
-        if (_nextMarkBitMap->parMark(objAddr)) {
+        if (par_mark_and_count(obj, hr, worker_i)) {
           // No OrderAccess:store_load() is needed. It is implicit in the
-          // CAS done in parMark(objAddr) above
+          // CAS done in the call to CMBitMap::parMark() in the above
+          // routine.
           HeapWord* finger = _finger;
           if (objAddr < finger) {
             if (verbose_high()) {
               gclog_or_tty->print_cr("[global] below the global finger "
                                      "("PTR_FORMAT"), pushing it", finger);

@@ -2696,11 +3049,11 @@
   // Note we are overriding the read-only view of the prev map here, via
   // the cast.
   ((CMBitMap*)_prevMarkBitMap)->mark((HeapWord*)p);
 }
 
-void ConcurrentMark::clear(oop p) {
+void ConcurrentMark::clear_mark(oop p) {
   assert(p != NULL && p->is_oop(), "expected an oop");
   HeapWord* addr = (HeapWord*)p;
   assert(addr >= _nextMarkBitMap->startWord() ||
          addr < _nextMarkBitMap->endWord(), "in a region");
 

@@ -2896,10 +3249,249 @@
     // Clear any partial regions from the CMTasks
     _tasks[i]->clear_aborted_region();
   }
 }
 
+// Aggregate the counting data that was constructed concurrently
+// with marking.
+class AggregateCountDataHRClosure: public HeapRegionClosure {
+  ConcurrentMark* _cm;
+  BitMap* _cm_card_bm;
+  intptr_t _bottom_card_num;
+  size_t _max_task_num;
+
+ public:
+  AggregateCountDataHRClosure(ConcurrentMark *cm,
+                          BitMap* cm_card_bm,
+                          intptr_t bottom_card_num,
+                          size_t max_task_num) :
+    _cm(cm),
+    _cm_card_bm(cm_card_bm),
+    _bottom_card_num(bottom_card_num),
+    _max_task_num(max_task_num)
+  { }
+
+  bool is_card_aligned(HeapWord* p) {
+    return ((uintptr_t(p) & (CardTableModRefBS::card_size - 1)) == 0);
+  }
+
+  bool doHeapRegion(HeapRegion* hr) {
+    if (hr->continuesHumongous()) {
+      // We will ignore these here and process them when their
+      // associated "starts humongous" region is processed.
+      // Note that we cannot rely on their associated
+      // "starts humongous" region to have their bit set to 1
+      // since, due to the region chunking in the parallel region
+      // iteration, a "continues humongous" region might be visited
+      // before its associated "starts humongous".
+      return false;
+    }
+
+    HeapWord* start = hr->bottom();
+    HeapWord* limit = hr->next_top_at_mark_start();
+    HeapWord* end = hr->end();
+    
+    assert(start <= limit && limit <= hr->top() && 
+           hr->top() <= hr->end(), "Preconditions");
+
+    assert(hr->next_marked_bytes() == 0, "Precondition");
+
+    if (start == limit) {
+      // NTAMS of this region has not been set so nothing to do.
+      return false;
+    }
+
+    intptr_t start_card_num = intptr_t(uintptr_t(start) >> CardTableModRefBS::card_shift);
+    intptr_t limit_card_num = intptr_t(uintptr_t(limit) >> CardTableModRefBS::card_shift);
+    intptr_t end_card_num   = intptr_t(uintptr_t(end) >> CardTableModRefBS::card_shift);
+
+    assert(is_card_aligned(start), "sanity");
+    assert(is_card_aligned(end), "sanity");
+   
+    // If ntams is not card aligned then we bump the index for
+    // limit so that we get the card spanning ntams.
+    if (!is_card_aligned(limit)) {
+      limit_card_num += 1;
+    }
+
+    assert(limit_card_num <= end_card_num, "or else use atomics");
+
+    BitMap::idx_t start_idx = start_card_num - _bottom_card_num;
+    BitMap::idx_t limit_idx = limit_card_num - _bottom_card_num;
+
+    // Aggregate the "stripe" in the count data associated with hr.
+    size_t hrs_index = hr->hrs_index();
+    size_t marked_bytes = 0;
+
+    for (int i = 0; (size_t)i < _max_task_num; i += 1) {
+      size_t* marked_bytes_array = _cm->count_marked_bytes_array_for(i);
+      BitMap* task_card_bm = _cm->count_card_bitmap_for(i);
+  
+      // Fetch the marked_bytes in this region for task i and
+      // add it to the running total for this region.
+      marked_bytes += marked_bytes_array[hrs_index];
+
+      // Now clear the value in the task's marked bytes array
+      // for this region.
+      marked_bytes_array[hrs_index] = 0;
+  
+      // Now union the bitmaps[0,max_task_num)[start_idx..limit_idx)
+      // into the global card bitmap.
+      BitMap::idx_t scan_idx = task_card_bm->get_next_one_offset(start_idx, limit_idx);
+
+      while (scan_idx < limit_idx) {
+        assert(task_card_bm->at(scan_idx) == true, "should be");
+        _cm_card_bm->set_bit(scan_idx);
+        task_card_bm->clear_bit(scan_idx);
+        assert(_cm_card_bm->at(scan_idx) == true, "should be");
+        scan_idx = task_card_bm->get_next_one_offset(start_idx + 1, limit_idx);
+      }
+    }
+
+    // Update the marked bytes for this region.
+    hr->add_to_marked_bytes(marked_bytes);
+  
+    // Now set the top at count to NTAMS.
+    hr->set_top_at_conc_mark_count(limit);
+
+    // Next heap region
+    return false;
+  }
+};
+
+class G1AggregateCountDataTask: public AbstractGangTask {
+protected:
+  G1CollectedHeap* _g1h;
+  ConcurrentMark* _cm;
+  BitMap* _cm_card_bm;
+  intptr_t _heap_bottom_card_num;
+  size_t _max_task_num;
+  int _active_workers;
+
+public:
+  G1AggregateCountDataTask(G1CollectedHeap* g1h,
+                           ConcurrentMark* cm,
+                           BitMap* cm_card_bm,
+                           intptr_t bottom_card_num,
+                           size_t max_task_num,
+                           int n_workers) :
+    AbstractGangTask("Count Aggregation"),
+    _g1h(g1h), _cm(cm), _cm_card_bm(cm_card_bm),
+    _heap_bottom_card_num(bottom_card_num),
+    _max_task_num(max_task_num),
+    _active_workers(n_workers)
+  { }
+
+  void work(int worker_i) {
+    AggregateCountDataHRClosure cl(_cm, _cm_card_bm,
+                                 _heap_bottom_card_num, _max_task_num);
+
+    if (G1CollectedHeap::use_parallel_gc_threads()) {
+      _g1h->heap_region_par_iterate_chunked(&cl, worker_i,
+                                            _active_workers,
+                                            HeapRegion::AggregateCountClaimValue);
+    } else {
+      _g1h->heap_region_iterate(&cl);
+    }
+  }
+};
+
+
+void ConcurrentMark::aggregate_and_clear_count_data() {
+  // Clear the global card bitmap
+  _card_bm.clear();
+
+  int n_workers = (G1CollectedHeap::use_parallel_gc_threads() ?
+                        _g1h->workers()->active_workers() :
+                        1);
+
+  G1AggregateCountDataTask g1_par_agg_task(_g1h, this, &_card_bm,
+                                           _heap_bottom_card_num, _max_task_num,
+                                           n_workers);
+
+  if (G1CollectedHeap::use_parallel_gc_threads()) {
+    assert(_g1h->check_heap_region_claim_values(HeapRegion::InitialClaimValue),
+           "sanity check");
+    _g1h->set_par_threads(n_workers);
+    _g1h->workers()->run_task(&g1_par_agg_task);
+    _g1h->set_par_threads(0);
+
+    assert(_g1h->check_heap_region_claim_values(HeapRegion::AggregateCountClaimValue),
+           "sanity check");
+    _g1h->reset_heap_region_claim_values();
+  } else {
+    g1_par_agg_task.work(0);
+  }
+}
+
+// Clear the per-worker arrays used to store the per-region counting data
+void ConcurrentMark::clear_all_count_data() {
+  assert(SafepointSynchronize::is_at_safepoint() ||
+         !Universe::is_fully_initialized(), "must be");
+
+  size_t max_regions = _g1h->max_regions();
+  
+  assert(_max_task_num != 0, "unitialized");
+  assert(_count_card_bitmaps != NULL, "uninitialized");
+  assert(_count_marked_bytes != NULL, "uninitialized");
+
+  for (int i = 0; (size_t) i < _max_task_num; i += 1) {
+    BitMap* task_card_bm = count_card_bitmap_for(i);
+    size_t* marked_bytes_array = count_marked_bytes_array_for(i);
+
+    assert(task_card_bm->size() == _card_bm.size(), "size mismatch");
+    assert(marked_bytes_array != NULL, "uninitialized");
+
+    for (int j = 0; (size_t) j < max_regions; j++) {
+      marked_bytes_array[j] = 0;
+    }
+    task_card_bm->clear();
+  }
+}
+
+void ConcurrentMark::clear_count_data_for_heap_region(HeapRegion* hr) {
+  // Clears the count data for the given region from _all_ of
+  // the per-task counting data structures.
+
+  MemRegion used_region = hr->used_region();
+  HeapWord* start = used_region.start();
+  HeapWord* last = used_region.last();
+  size_t hr_index = hr->hrs_index();
+
+  intptr_t start_card_num =
+    intptr_t(uintptr_t(start) >> CardTableModRefBS::card_shift);
+  intptr_t last_card_num =
+    intptr_t(uintptr_t(last) >> CardTableModRefBS::card_shift);
+  
+  BitMap::idx_t start_idx = start_card_num - heap_bottom_card_num();
+  BitMap::idx_t last_idx = last_card_num - heap_bottom_card_num();
+
+  size_t used_region_bytes = used_region.byte_size();
+  size_t marked_bytes = 0;
+
+  for (int i=0; (size_t)i < _max_task_num; i += 1) {
+    BitMap* task_card_bm = count_card_bitmap_for(i);
+    size_t* marked_bytes_array = count_marked_bytes_array_for(i);
+
+    marked_bytes += marked_bytes_array[hr_index];
+    // clear the amount of marked bytes in the task array for this
+    // region
+    marked_bytes_array[hr_index] = 0;
+    
+    // Clear the inclusive range [start_idx, last_idx] from the
+    // card bitmap. The clear_range routine is exclusive so we
+    // need to also explicitly clear the bit at last_idx.
+    // Passing last_idx+1 to the clear_range would work in
+    // most cases but could trip an OOB assertion.
+
+    if ((last_idx - start_idx) > 0) {
+      task_card_bm->clear_range(start_idx, last_idx);
+    }
+    task_card_bm->clear_bit(last_idx);
+  }
+}
+
 void ConcurrentMark::print_stats() {
   if (verbose_stats()) {
     gclog_or_tty->print_cr("---------------------------------------------------------------------");
     for (size_t i = 0; i < _active_tasks; ++i) {
       _tasks[i]->print_stats();

@@ -2912,11 +3504,10 @@
 
 class CSetMarkOopClosure: public OopClosure {
   friend class CSetMarkBitMapClosure;
 
   G1CollectedHeap* _g1h;
-  CMBitMap*        _bm;
   ConcurrentMark*  _cm;
   oop*             _ms;
   jint*            _array_ind_stack;
   int              _ms_size;
   int              _ms_ind;

@@ -2972,11 +3563,10 @@
 
 public:
   CSetMarkOopClosure(ConcurrentMark* cm, int ms_size, int worker_i) :
     _g1h(G1CollectedHeap::heap()),
     _cm(cm),
-    _bm(cm->nextMarkBitMap()),
     _ms_size(ms_size), _ms_ind(0),
     _ms(NEW_C_HEAP_ARRAY(oop, ms_size)),
     _array_ind_stack(NEW_C_HEAP_ARRAY(jint, ms_size)),
     _array_increment(MAX2(ms_size/8, 16)),
     _worker_i(worker_i) { }

@@ -3002,20 +3592,20 @@
     }
     HeapRegion* hr = _g1h->heap_region_containing(obj);
     if (hr != NULL) {
       if (hr->in_collection_set()) {
         if (_g1h->is_obj_ill(obj)) {
-          if (_bm->parMark((HeapWord*)obj)) {
+          if (_cm->par_mark_and_count(obj, hr, _worker_i)) {
             if (!push(obj)) {
               gclog_or_tty->print_cr("Setting abort in CSetMarkOopClosure because push failed.");
               set_abort();
             }
           }
         }
       } else {
         // Outside the collection set; we need to gray it
-        _cm->deal_with_reference(obj);
+        _cm->deal_with_reference(obj, _worker_i);
       }
     }
   }
 };
 

@@ -3287,14 +3877,13 @@
   }
   gclog_or_tty->print_cr("  Total stop_world time = %8.2f s.",
                          (_init_times.sum() + _remark_times.sum() +
                           _cleanup_times.sum())/1000.0);
   gclog_or_tty->print_cr("  Total concurrent time = %8.2f s "
-                "(%8.2f s marking, %8.2f s counting).",
+                "(%8.2f s marking).",
                 cmThread()->vtime_accum(),
-                cmThread()->vtime_mark_accum(),
-                cmThread()->vtime_count_accum());
+                cmThread()->vtime_mark_accum());
 }
 
 void ConcurrentMark::print_worker_threads_on(outputStream* st) const {
   _parallel_workers->print_worker_threads_on(st);
 }

@@ -4585,20 +5174,24 @@
   _claimed = false;
 }
 
 CMTask::CMTask(int task_id,
                ConcurrentMark* cm,
+               size_t* marked_bytes,
+               BitMap* card_bm,
                CMTaskQueue* task_queue,
                CMTaskQueueSet* task_queues)
   : _g1h(G1CollectedHeap::heap()),
     _task_id(task_id), _cm(cm),
     _claimed(false),
     _nextMarkBitMap(NULL), _hash_seed(17),
     _task_queue(task_queue),
     _task_queues(task_queues),
     _cm_oop_closure(NULL),
-    _aborted_region(MemRegion()) {
+    _aborted_region(MemRegion()),
+    _marked_bytes_array(marked_bytes),
+    _card_bm(card_bm) {
   guarantee(task_queue != NULL, "invariant");
   guarantee(task_queues != NULL, "invariant");
 
   statsOnly( _clock_due_to_scanning = 0;
              _clock_due_to_marking  = 0 );