diff src/share/vm/gc_implementation/g1/concurrentMark.cpp @ 20804:7848fc12602b

Merge with jdk8u40-b25
author Gilles Duboscq <gilles.m.duboscq@oracle.com>
date Tue, 07 Apr 2015 14:58:49 +0200
parents 52b4284cb496 8d27d6113625
children
line wrap: on
line diff
--- a/src/share/vm/gc_implementation/g1/concurrentMark.cpp	Tue Apr 07 11:20:51 2015 +0200
+++ b/src/share/vm/gc_implementation/g1/concurrentMark.cpp	Tue Apr 07 14:58:49 2015 +0200
@@ -23,7 +23,9 @@
  */
 
 #include "precompiled.hpp"
+#include "classfile/metadataOnStackMark.hpp"
 #include "classfile/symbolTable.hpp"
+#include "code/codeCache.hpp"
 #include "gc_implementation/g1/concurrentMark.inline.hpp"
 #include "gc_implementation/g1/concurrentMarkThread.inline.hpp"
 #include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
@@ -33,18 +35,21 @@
 #include "gc_implementation/g1/g1OopClosures.inline.hpp"
 #include "gc_implementation/g1/g1RemSet.hpp"
 #include "gc_implementation/g1/heapRegion.inline.hpp"
+#include "gc_implementation/g1/heapRegionManager.inline.hpp"
 #include "gc_implementation/g1/heapRegionRemSet.hpp"
-#include "gc_implementation/g1/heapRegionSeq.inline.hpp"
+#include "gc_implementation/g1/heapRegionSet.inline.hpp"
 #include "gc_implementation/shared/vmGCOperations.hpp"
 #include "gc_implementation/shared/gcTimer.hpp"
 #include "gc_implementation/shared/gcTrace.hpp"
 #include "gc_implementation/shared/gcTraceTime.hpp"
+#include "memory/allocation.hpp"
 #include "memory/genOopClosures.inline.hpp"
 #include "memory/referencePolicy.hpp"
 #include "memory/resourceArea.hpp"
 #include "oops/oop.inline.hpp"
 #include "runtime/handles.inline.hpp"
 #include "runtime/java.hpp"
+#include "runtime/prefetch.inline.hpp"
 #include "services/memTracker.hpp"
 
 // Concurrent marking bit map wrapper
@@ -56,8 +61,8 @@
   _bmWordSize = 0;
 }
 
-HeapWord* CMBitMapRO::getNextMarkedWordAddress(HeapWord* addr,
-                                               HeapWord* limit) const {
+HeapWord* CMBitMapRO::getNextMarkedWordAddress(const HeapWord* addr,
+                                               const HeapWord* limit) const {
   // First we must round addr *up* to a possible object boundary.
   addr = (HeapWord*)align_size_up((intptr_t)addr,
                                   HeapWordSize << _shifter);
@@ -74,8 +79,8 @@
   return nextAddr;
 }
 
-HeapWord* CMBitMapRO::getNextUnmarkedWordAddress(HeapWord* addr,
-                                                 HeapWord* limit) const {
+HeapWord* CMBitMapRO::getNextUnmarkedWordAddress(const HeapWord* addr,
+                                                 const HeapWord* limit) const {
   size_t addrOffset = heapWordToOffset(addr);
   if (limit == NULL) {
     limit = _bmStartWord + _bmWordSize;
@@ -95,12 +100,12 @@
 }
 
 #ifndef PRODUCT
-bool CMBitMapRO::covers(ReservedSpace heap_rs) const {
+bool CMBitMapRO::covers(MemRegion heap_rs) const {
   // assert(_bm.map() == _virtual_space.low(), "map inconsistency");
   assert(((size_t)_bm.size() * ((size_t)1 << _shifter)) == _bmWordSize,
          "size inconsistency");
-  return _bmStartWord == (HeapWord*)(heap_rs.base()) &&
-         _bmWordSize  == heap_rs.size()>>LogHeapWordSize;
+  return _bmStartWord == (HeapWord*)(heap_rs.start()) &&
+         _bmWordSize  == heap_rs.word_size();
 }
 #endif
 
@@ -108,33 +113,76 @@
   _bm.print_on_error(st, prefix);
 }
 
-bool CMBitMap::allocate(ReservedSpace heap_rs) {
-  _bmStartWord = (HeapWord*)(heap_rs.base());
-  _bmWordSize  = heap_rs.size()/HeapWordSize;    // heap_rs.size() is in bytes
-  ReservedSpace brs(ReservedSpace::allocation_align_size_up(
-                     (_bmWordSize >> (_shifter + LogBitsPerByte)) + 1));
-  if (!brs.is_reserved()) {
-    warning("ConcurrentMark marking bit map allocation failure");
+size_t CMBitMap::compute_size(size_t heap_size) {
+  return heap_size / mark_distance();
+}
+
+size_t CMBitMap::mark_distance() {
+  return MinObjAlignmentInBytes * BitsPerByte;
+}
+
+void CMBitMap::initialize(MemRegion heap, G1RegionToSpaceMapper* storage) {
+  _bmStartWord = heap.start();
+  _bmWordSize = heap.word_size();
+
+  _bm.set_map((BitMap::bm_word_t*) storage->reserved().start());
+  _bm.set_size(_bmWordSize >> _shifter);
+
+  storage->set_mapping_changed_listener(&_listener);
+}
+
+void CMBitMapMappingChangedListener::on_commit(uint start_region, size_t num_regions, bool zero_filled) {
+  if (zero_filled) {
+    return;
+  }
+  // We need to clear the bitmap on commit, removing any existing information.
+  MemRegion mr(G1CollectedHeap::heap()->bottom_addr_for_region(start_region), num_regions * HeapRegion::GrainWords);
+  _bm->clearRange(mr);
+}
+
+// Closure used for clearing the given mark bitmap.
+class ClearBitmapHRClosure : public HeapRegionClosure {
+ private:
+  ConcurrentMark* _cm;
+  CMBitMap* _bitmap;
+  bool _may_yield;      // The closure may yield during iteration. If yielded, abort the iteration.
+ public:
+  ClearBitmapHRClosure(ConcurrentMark* cm, CMBitMap* bitmap, bool may_yield) : HeapRegionClosure(), _cm(cm), _bitmap(bitmap), _may_yield(may_yield) {
+    assert(!may_yield || cm != NULL, "CM must be non-NULL if this closure is expected to yield.");
+  }
+
+  virtual bool doHeapRegion(HeapRegion* r) {
+    size_t const chunk_size_in_words = M / HeapWordSize;
+
+    HeapWord* cur = r->bottom();
+    HeapWord* const end = r->end();
+
+    while (cur < end) {
+      MemRegion mr(cur, MIN2(cur + chunk_size_in_words, end));
+      _bitmap->clearRange(mr);
+
+      cur += chunk_size_in_words;
+
+      // Abort iteration if after yielding the marking has been aborted.
+      if (_may_yield && _cm->do_yield_check() && _cm->has_aborted()) {
+        return true;
+      }
+      // Repeat the asserts from before the start of the closure. We will do them
+      // as asserts here to minimize their overhead on the product. However, we
+      // will have them as guarantees at the beginning / end of the bitmap
+      // clearing to get some checking in the product.
+      assert(!_may_yield || _cm->cmThread()->during_cycle(), "invariant");
+      assert(!_may_yield || !G1CollectedHeap::heap()->mark_in_progress(), "invariant");
+    }
+
     return false;
   }
-  MemTracker::record_virtual_memory_type((address)brs.base(), mtGC);
-  // For now we'll just commit all of the bit map up front.
-  // Later on we'll try to be more parsimonious with swap.
-  if (!_virtual_space.initialize(brs, brs.size())) {
-    warning("ConcurrentMark marking bit map backing store failure");
-    return false;
-  }
-  assert(_virtual_space.committed_size() == brs.size(),
-         "didn't reserve backing store for all of concurrent marking bit map?");
-  _bm.set_map((uintptr_t*)_virtual_space.low());
-  assert(_virtual_space.committed_size() << (_shifter + LogBitsPerByte) >=
-         _bmWordSize, "inconsistency in bit map sizing");
-  _bm.set_size(_bmWordSize >> _shifter);
-  return true;
-}
+};
 
 void CMBitMap::clearAll() {
-  _bm.clear();
+  ClearBitmapHRClosure cl(NULL, this, false /* may_yield */);
+  G1CollectedHeap::heap()->heap_region_iterate(&cl);
+  guarantee(cl.complete(), "Must have completed iteration.");
   return;
 }
 
@@ -389,10 +437,6 @@
   }
 }
 
-bool ConcurrentMark::not_yet_marked(oop obj) const {
-  return _g1h->is_obj_ill(obj);
-}
-
 CMRootRegions::CMRootRegions() :
   _young_list(NULL), _cm(NULL), _scan_in_progress(false),
   _should_abort(false),  _next_survivor(NULL) { }
@@ -479,10 +523,10 @@
   return MAX2((n_par_threads + 2) / 4, 1U);
 }
 
-ConcurrentMark::ConcurrentMark(G1CollectedHeap* g1h, ReservedSpace heap_rs) :
+ConcurrentMark::ConcurrentMark(G1CollectedHeap* g1h, G1RegionToSpaceMapper* prev_bitmap_storage, G1RegionToSpaceMapper* next_bitmap_storage) :
   _g1h(g1h),
-  _markBitMap1(log2_intptr(MinObjAlignment)),
-  _markBitMap2(log2_intptr(MinObjAlignment)),
+  _markBitMap1(),
+  _markBitMap2(),
   _parallel_marking_threads(0),
   _max_parallel_marking_threads(0),
   _sleep_factor(0.0),
@@ -491,7 +535,7 @@
   _cleanup_task_overhead(1.0),
   _cleanup_list("Cleanup List"),
   _region_bm((BitMap::idx_t)(g1h->max_regions()), false /* in_resource_area*/),
-  _card_bm((heap_rs.size() + CardTableModRefBS::card_size - 1) >>
+  _card_bm((g1h->reserved_region().byte_size() + CardTableModRefBS::card_size - 1) >>
             CardTableModRefBS::card_shift,
             false /* in_resource_area*/),
 
@@ -510,6 +554,7 @@
   _has_overflown(false),
   _concurrent(false),
   _has_aborted(false),
+  _aborted_gc_id(GCId::undefined()),
   _restart_for_overflow(false),
   _concurrent_marking_in_progress(false),
 
@@ -540,14 +585,8 @@
                            "heap end = " INTPTR_FORMAT, p2i(_heap_start), p2i(_heap_end));
   }
 
-  if (!_markBitMap1.allocate(heap_rs)) {
-    warning("Failed to allocate first CM bit map");
-    return;
-  }
-  if (!_markBitMap2.allocate(heap_rs)) {
-    warning("Failed to allocate second CM bit map");
-    return;
-  }
+  _markBitMap1.initialize(g1h->reserved_region(), prev_bitmap_storage);
+  _markBitMap2.initialize(g1h->reserved_region(), next_bitmap_storage);
 
   // Create & start a ConcurrentMark thread.
   _cmThread = new ConcurrentMarkThread(this);
@@ -558,8 +597,8 @@
   }
 
   assert(CGC_lock != NULL, "Where's the CGC_lock?");
-  assert(_markBitMap1.covers(heap_rs), "_markBitMap1 inconsistency");
-  assert(_markBitMap2.covers(heap_rs), "_markBitMap2 inconsistency");
+  assert(_markBitMap1.covers(g1h->reserved_region()), "_markBitMap1 inconsistency");
+  assert(_markBitMap2.covers(g1h->reserved_region()), "_markBitMap2 inconsistency");
 
   SATBMarkQueueSet& satb_qs = JavaThread::satb_mark_queue_set();
   satb_qs.set_buffer_size(G1SATBBufferSize);
@@ -719,38 +758,17 @@
   clear_all_count_data();
 
   // so that the call below can read a sensible value
-  _heap_start = (HeapWord*) heap_rs.base();
+  _heap_start = g1h->reserved_region().start();
   set_non_marking_state();
   _completed_initialization = true;
 }
 
-void ConcurrentMark::update_g1_committed(bool force) {
-  // If concurrent marking is not in progress, then we do not need to
-  // update _heap_end.
-  if (!concurrent_marking_in_progress() && !force) return;
-
-  MemRegion committed = _g1h->g1_committed();
-  assert(committed.start() == _heap_start, "start shouldn't change");
-  HeapWord* new_end = committed.end();
-  if (new_end > _heap_end) {
-    // The heap has been expanded.
-
-    _heap_end = new_end;
-  }
-  // Notice that the heap can also shrink. However, this only happens
-  // during a Full GC (at least currently) and the entire marking
-  // phase will bail out and the task will not be restarted. So, let's
-  // do nothing.
-}
-
 void ConcurrentMark::reset() {
   // Starting values for these two. This should be called in a STW
-  // phase. CM will be notified of any future g1_committed expansions
-  // will be at the end of evacuation pauses, when tasks are
-  // inactive.
-  MemRegion committed = _g1h->g1_committed();
-  _heap_start = committed.start();
-  _heap_end   = committed.end();
+  // phase.
+  MemRegion reserved = _g1h->g1_reserved();
+  _heap_start = reserved.start();
+  _heap_end   = reserved.end();
 
   // Separated the asserts so that we know which one fires.
   assert(_heap_start != NULL, "heap bounds should look ok");
@@ -822,7 +840,6 @@
     assert(out_of_regions(),
            err_msg("only way to get here: _finger: "PTR_FORMAT", _heap_end: "PTR_FORMAT,
                    p2i(_finger), p2i(_heap_end)));
-    update_g1_committed(true);
   }
 }
 
@@ -841,7 +858,6 @@
 
 void ConcurrentMark::clearNextBitmap() {
   G1CollectedHeap* g1h = G1CollectedHeap::heap();
-  G1CollectorPolicy* g1p = g1h->g1_policy();
 
   // Make sure that the concurrent mark thread looks to still be in
   // the current cycle.
@@ -853,39 +869,47 @@
   // is the case.
   guarantee(!g1h->mark_in_progress(), "invariant");
 
-  // clear the mark bitmap (no grey objects to start with).
-  // We need to do this in chunks and offer to yield in between
-  // each chunk.
-  HeapWord* start  = _nextMarkBitMap->startWord();
-  HeapWord* end    = _nextMarkBitMap->endWord();
-  HeapWord* cur    = start;
-  size_t chunkSize = M;
-  while (cur < end) {
-    HeapWord* next = cur + chunkSize;
-    if (next > end) {
-      next = end;
-    }
-    MemRegion mr(cur,next);
-    _nextMarkBitMap->clearRange(mr);
-    cur = next;
-    do_yield_check();
-
-    // Repeat the asserts from above. We'll do them as asserts here to
-    // minimize their overhead on the product. However, we'll have
-    // them as guarantees at the beginning / end of the bitmap
-    // clearing to get some checking in the product.
-    assert(cmThread()->during_cycle(), "invariant");
-    assert(!g1h->mark_in_progress(), "invariant");
+  ClearBitmapHRClosure cl(this, _nextMarkBitMap, true /* may_yield */);
+  g1h->heap_region_iterate(&cl);
+
+  // Clear the liveness counting data. If the marking has been aborted, the abort()
+  // call already did that.
+  if (cl.complete()) {
+    clear_all_count_data();
   }
 
-  // Clear the liveness counting data
-  clear_all_count_data();
-
   // Repeat the asserts from above.
   guarantee(cmThread()->during_cycle(), "invariant");
   guarantee(!g1h->mark_in_progress(), "invariant");
 }
 
+class CheckBitmapClearHRClosure : public HeapRegionClosure {
+  CMBitMap* _bitmap;
+  bool _error;
+ public:
+  CheckBitmapClearHRClosure(CMBitMap* bitmap) : _bitmap(bitmap) {
+  }
+
+  virtual bool doHeapRegion(HeapRegion* r) {
+    // This closure can be called concurrently to the mutator, so we must make sure
+    // that the result of the getNextMarkedWordAddress() call is compared to the
+    // value passed to it as limit to detect any found bits.
+    // We can use the region's orig_end() for the limit and the comparison value
+    // as it always contains the "real" end of the region that never changes and
+    // has no side effects.
+    // Due to the latter, there can also be no problem with the compiler generating
+    // reloads of the orig_end() call.
+    HeapWord* end = r->orig_end();
+    return _bitmap->getNextMarkedWordAddress(r->bottom(), end) != end;
+  }
+};
+
+bool ConcurrentMark::nextMarkBitmapIsClear() {
+  CheckBitmapClearHRClosure cl(_nextMarkBitMap);
+  _g1h->heap_region_iterate(&cl);
+  return cl.complete();
+}
+
 class NoteStartOfMarkHRClosure: public HeapRegionClosure {
 public:
   bool doHeapRegion(HeapRegion* r) {
@@ -976,13 +1000,13 @@
   }
 
   if (concurrent()) {
-    ConcurrentGCThread::stsLeave();
+    SuspendibleThreadSet::leave();
   }
 
   bool barrier_aborted = !_first_overflow_barrier_sync.enter();
 
   if (concurrent()) {
-    ConcurrentGCThread::stsJoin();
+    SuspendibleThreadSet::join();
   }
   // at this point everyone should have synced up and not be doing any
   // more work
@@ -1019,8 +1043,7 @@
       force_overflow()->update();
 
       if (G1Log::fine()) {
-        gclog_or_tty->date_stamp(PrintGCDateStamps);
-        gclog_or_tty->stamp(PrintGCTimeStamps);
+        gclog_or_tty->gclog_stamp(concurrent_gc_id());
         gclog_or_tty->print_cr("[GC concurrent-mark-reset-for-overflow]");
       }
     }
@@ -1036,13 +1059,13 @@
   }
 
   if (concurrent()) {
-    ConcurrentGCThread::stsLeave();
+    SuspendibleThreadSet::leave();
   }
 
   bool barrier_aborted = !_second_overflow_barrier_sync.enter();
 
   if (concurrent()) {
-    ConcurrentGCThread::stsJoin();
+    SuspendibleThreadSet::join();
   }
   // at this point everything should be re-initialized and ready to go
 
@@ -1094,7 +1117,7 @@
 
     double start_vtime = os::elapsedVTime();
 
-    ConcurrentGCThread::stsJoin();
+    SuspendibleThreadSet::join();
 
     assert(worker_id < _cm->active_tasks(), "invariant");
     CMTask* the_task = _cm->task(worker_id);
@@ -1102,46 +1125,32 @@
     if (!_cm->has_aborted()) {
       do {
         double start_vtime_sec = os::elapsedVTime();
-        double start_time_sec = os::elapsedTime();
         double mark_step_duration_ms = G1ConcMarkStepDurationMillis;
 
         the_task->do_marking_step(mark_step_duration_ms,
                                   true  /* do_termination */,
                                   false /* is_serial*/);
 
-        double end_time_sec = os::elapsedTime();
         double end_vtime_sec = os::elapsedVTime();
         double elapsed_vtime_sec = end_vtime_sec - start_vtime_sec;
-        double elapsed_time_sec = end_time_sec - start_time_sec;
         _cm->clear_has_overflown();
 
-        bool ret = _cm->do_yield_check(worker_id);
+        _cm->do_yield_check(worker_id);
 
         jlong sleep_time_ms;
         if (!_cm->has_aborted() && the_task->has_aborted()) {
           sleep_time_ms =
             (jlong) (elapsed_vtime_sec * _cm->sleep_factor() * 1000.0);
-          ConcurrentGCThread::stsLeave();
+          SuspendibleThreadSet::leave();
           os::sleep(Thread::current(), sleep_time_ms, false);
-          ConcurrentGCThread::stsJoin();
+          SuspendibleThreadSet::join();
         }
-        double end_time2_sec = os::elapsedTime();
-        double elapsed_time2_sec = end_time2_sec - start_time_sec;
-
-#if 0
-          gclog_or_tty->print_cr("CM: elapsed %1.4lf ms, sleep %1.4lf ms, "
-                                 "overhead %1.4lf",
-                                 elapsed_vtime_sec * 1000.0, (double) sleep_time_ms,
-                                 the_task->conc_overhead(os::elapsedTime()) * 8.0);
-          gclog_or_tty->print_cr("elapsed time %1.4lf ms, time 2: %1.4lf ms",
-                                 elapsed_time_sec * 1000.0, elapsed_time2_sec * 1000.0);
-#endif
       } while (!_cm->has_aborted() && the_task->has_aborted());
     }
     the_task->record_end_time();
     guarantee(!the_task->has_aborted() || _cm->has_aborted(), "invariant");
 
-    ConcurrentGCThread::stsLeave();
+    SuspendibleThreadSet::leave();
 
     double end_vtime = os::elapsedVTime();
     _cm->update_accum_task_vtime(worker_id, end_vtime - start_vtime);
@@ -1221,6 +1230,9 @@
 };
 
 void ConcurrentMark::scanRootRegions() {
+  // Start of concurrent marking.
+  ClassLoaderDataGraph::clear_claimed_marks();
+
   // scan_in_progress() will have been set to true only if there was
   // at least one root region to scan. So, if it's false, we
   // should not attempt to do any further work.
@@ -1269,7 +1281,7 @@
   CMConcurrentMarkingTask markingTask(this, cmThread());
   if (use_parallel_marking_threads()) {
     _parallel_workers->set_active_workers((int)active_workers);
-    // Don't set _n_par_threads because it affects MT in proceess_strong_roots()
+    // Don't set _n_par_threads because it affects MT in process_roots()
     // and the decisions on that MT processing is made elsewhere.
     assert(_parallel_workers->active_workers() > 0, "Should have been set");
     _parallel_workers->run_task(&markingTask);
@@ -1300,6 +1312,7 @@
     Universe::verify(VerifyOption_G1UsePrevMarking,
                      " VerifyDuringGC:(before)");
   }
+  g1h->check_bitmaps("Remark Start");
 
   G1CollectorPolicy* g1p = g1h->g1_policy();
   g1p->record_concurrent_mark_remark_start();
@@ -1348,6 +1361,7 @@
       Universe::verify(VerifyOption_G1UseNextMarking,
                        " VerifyDuringGC:(after)");
     }
+    g1h->check_bitmaps("Remark End");
     assert(!restart_for_overflow(), "sanity");
     // Completely reset the marking state since marking completed
     set_non_marking_state();
@@ -1389,7 +1403,7 @@
   void set_bit_for_region(HeapRegion* hr) {
     assert(!hr->continuesHumongous(), "should have filtered those out");
 
-    BitMap::idx_t index = (BitMap::idx_t) hr->hrs_index();
+    BitMap::idx_t index = (BitMap::idx_t) hr->hrm_index();
     if (!hr->startsHumongous()) {
       // Normal (non-humongous) case: just set the bit.
       _region_bm->par_at_put(index, true);
@@ -1577,7 +1591,7 @@
       if (_verbose) {
         gclog_or_tty->print_cr("Region %u: marked bytes mismatch: "
                                "expected: " SIZE_FORMAT ", actual: " SIZE_FORMAT,
-                               hr->hrs_index(), exp_marked_bytes, act_marked_bytes);
+                               hr->hrm_index(), exp_marked_bytes, act_marked_bytes);
       }
       failures += 1;
     }
@@ -1586,7 +1600,7 @@
     // (which was just calculated) region bit maps.
     // We're not OK if the bit in the calculated expected region
     // bitmap is set and the bit in the actual region bitmap is not.
-    BitMap::idx_t index = (BitMap::idx_t) hr->hrs_index();
+    BitMap::idx_t index = (BitMap::idx_t) hr->hrm_index();
 
     bool expected = _exp_region_bm->at(index);
     bool actual = _region_bm->at(index);
@@ -1594,7 +1608,7 @@
       if (_verbose) {
         gclog_or_tty->print_cr("Region %u: region bitmap mismatch: "
                                "expected: %s, actual: %s",
-                               hr->hrs_index(),
+                               hr->hrm_index(),
                                BOOL_TO_STR(expected), BOOL_TO_STR(actual));
       }
       failures += 1;
@@ -1615,7 +1629,7 @@
         if (_verbose) {
           gclog_or_tty->print_cr("Region %u: card bitmap mismatch at " SIZE_FORMAT ": "
                                  "expected: %s, actual: %s",
-                                 hr->hrs_index(), i,
+                                 hr->hrm_index(), i,
                                  BOOL_TO_STR(expected), BOOL_TO_STR(actual));
         }
         failures += 1;
@@ -1997,6 +2011,7 @@
     Universe::verify(VerifyOption_G1UsePrevMarking,
                      " VerifyDuringGC:(before)");
   }
+  g1h->check_bitmaps("Cleanup Start");
 
   G1CollectorPolicy* g1p = G1CollectedHeap::heap()->g1_policy();
   g1p->record_concurrent_mark_cleanup_start();
@@ -2034,8 +2049,8 @@
     // 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);
+    BitMap expected_region_bm(_region_bm.size(), true);
+    BitMap expected_card_bm(_card_bm.size(), true);
 
     G1ParVerifyFinalCountTask g1_par_verify_task(g1h,
                                                  &_region_bm,
@@ -2137,22 +2152,31 @@
   // Update the soft reference policy with the new heap occupancy.
   Universe::update_heap_info_at_gc();
 
-  // We need to make this be a "collection" so any collection pause that
-  // races with it goes around and waits for completeCleanup to finish.
-  g1h->increment_total_collections();
-
-  // We reclaimed old regions so we should calculate the sizes to make
-  // sure we update the old gen/space data.
-  g1h->g1mm()->update_sizes();
-
   if (VerifyDuringGC) {
     HandleMark hm;  // handle scope
     Universe::heap()->prepare_for_verify();
     Universe::verify(VerifyOption_G1UsePrevMarking,
                      " VerifyDuringGC:(after)");
   }
+  g1h->check_bitmaps("Cleanup End");
 
   g1h->verify_region_sets_optional();
+
+  // We need to make this be a "collection" so any collection pause that
+  // races with it goes around and waits for completeCleanup to finish.
+  g1h->increment_total_collections();
+
+  // Clean out dead classes and update Metaspace sizes.
+  if (ClassUnloadingWithConcurrentMark) {
+    ClassLoaderDataGraph::purge();
+  }
+  MetaspaceGC::compute_new_size();
+
+  // We reclaimed old regions so we should calculate the sizes to make
+  // sure we update the old gen/space data.
+  g1h->g1mm()->update_sizes();
+  g1h->allocation_context_stats().update_after_mark();
+
   g1h->trace_heap_after_concurrent_cycle();
 }
 
@@ -2170,10 +2194,10 @@
                            _cleanup_list.length());
   }
 
-  // Noone else should be accessing the _cleanup_list at this point,
-  // so it's not necessary to take any locks
+  // No one else should be accessing the _cleanup_list at this point,
+  // so it is not necessary to take any locks
   while (!_cleanup_list.is_empty()) {
-    HeapRegion* hr = _cleanup_list.remove_head();
+    HeapRegion* hr = _cleanup_list.remove_region(true /* from_head */);
     assert(hr != NULL, "Got NULL from a non-empty list");
     hr->par_clear();
     tmp_free_list.add_ordered(hr);
@@ -2382,6 +2406,8 @@
   }
 
   virtual void work(uint worker_id) {
+    ResourceMark rm;
+    HandleMark hm;
     CMTask* task = _cm->task(worker_id);
     G1CMIsAliveClosure g1_is_alive(_g1h);
     G1CMKeepAliveAndDrainClosure g1_par_keep_alive(_cm, task, false /* is_serial */);
@@ -2439,6 +2465,26 @@
   _g1h->set_par_threads(0);
 }
 
+void ConcurrentMark::weakRefsWorkParallelPart(BoolObjectClosure* is_alive, bool purged_classes) {
+  G1CollectedHeap::heap()->parallel_cleaning(is_alive, true, true, purged_classes);
+}
+
+// Helper class to get rid of some boilerplate code.
+class G1RemarkGCTraceTime : public GCTraceTime {
+  static bool doit_and_prepend(bool doit) {
+    if (doit) {
+      gclog_or_tty->put(' ');
+    }
+    return doit;
+  }
+
+ public:
+  G1RemarkGCTraceTime(const char* title, bool doit)
+    : GCTraceTime(title, doit_and_prepend(doit), false, G1CollectedHeap::heap()->gc_timer_cm(),
+        G1CollectedHeap::heap()->concurrent_mark()->concurrent_gc_id()) {
+  }
+};
+
 void ConcurrentMark::weakRefsWork(bool clear_all_soft_refs) {
   if (has_overflown()) {
     // Skip processing the discovered references if we have
@@ -2464,7 +2510,7 @@
     if (G1Log::finer()) {
       gclog_or_tty->put(' ');
     }
-    GCTraceTime t("GC ref-proc", G1Log::finer(), false, g1h->gc_timer_cm());
+    GCTraceTime t("GC ref-proc", G1Log::finer(), false, g1h->gc_timer_cm(), concurrent_gc_id());
 
     ReferenceProcessor* rp = g1h->ref_processor_cm();
 
@@ -2521,7 +2567,8 @@
                                           &g1_keep_alive,
                                           &g1_drain_mark_stack,
                                           executor,
-                                          g1h->gc_timer_cm());
+                                          g1h->gc_timer_cm(),
+                                          concurrent_gc_id());
     g1h->gc_tracer_cm()->report_gc_reference_stats(stats);
 
     // The do_oop work routines of the keep_alive and drain_marking_stack
@@ -2550,9 +2597,41 @@
     return;
   }
 
-  g1h->unlink_string_and_symbol_table(&g1_is_alive,
-                                      /* process_strings */ false, // currently strings are always roots
-                                      /* process_symbols */ true);
+  assert(_markStack.isEmpty(), "Marking should have completed");
+
+  // Unload Klasses, String, Symbols, Code Cache, etc.
+  {
+    G1RemarkGCTraceTime trace("Unloading", G1Log::finer());
+
+    if (ClassUnloadingWithConcurrentMark) {
+      // Cleaning of klasses depends on correct information from MetadataMarkOnStack. The CodeCache::mark_on_stack
+      // part is too slow to be done serially, so it is handled during the weakRefsWorkParallelPart phase.
+      // Defer the cleaning until we have complete on_stack data.
+      MetadataOnStackMark md_on_stack(false /* Don't visit the code cache at this point */);
+
+      bool purged_classes;
+
+      {
+        G1RemarkGCTraceTime trace("System Dictionary Unloading", G1Log::finest());
+        purged_classes = SystemDictionary::do_unloading(&g1_is_alive, false /* Defer klass cleaning */);
+      }
+
+      {
+        G1RemarkGCTraceTime trace("Parallel Unloading", G1Log::finest());
+        weakRefsWorkParallelPart(&g1_is_alive, purged_classes);
+      }
+
+      {
+        G1RemarkGCTraceTime trace("Deallocate Metadata", G1Log::finest());
+        ClassLoaderDataGraph::free_deallocate_lists();
+      }
+    }
+
+    if (G1StringDedup::is_enabled()) {
+      G1RemarkGCTraceTime trace("String Deduplication Unlink", G1Log::finest());
+      G1StringDedup::unlink(&g1_is_alive);
+    }
+  }
 }
 
 void ConcurrentMark::swapMarkBitMaps() {
@@ -2561,6 +2640,57 @@
   _nextMarkBitMap  = (CMBitMap*)  temp;
 }
 
+class CMObjectClosure;
+
+// Closure for iterating over objects, currently only used for
+// processing SATB buffers.
+class CMObjectClosure : public ObjectClosure {
+private:
+  CMTask* _task;
+
+public:
+  void do_object(oop obj) {
+    _task->deal_with_reference(obj);
+  }
+
+  CMObjectClosure(CMTask* task) : _task(task) { }
+};
+
+class G1RemarkThreadsClosure : public ThreadClosure {
+  CMObjectClosure _cm_obj;
+  G1CMOopClosure _cm_cl;
+  MarkingCodeBlobClosure _code_cl;
+  int _thread_parity;
+  bool _is_par;
+
+ public:
+  G1RemarkThreadsClosure(G1CollectedHeap* g1h, CMTask* task, bool is_par) :
+    _cm_obj(task), _cm_cl(g1h, g1h->concurrent_mark(), task), _code_cl(&_cm_cl, !CodeBlobToOopClosure::FixRelocations),
+    _thread_parity(SharedHeap::heap()->strong_roots_parity()), _is_par(is_par) {}
+
+  void do_thread(Thread* thread) {
+    if (thread->is_Java_thread()) {
+      if (thread->claim_oops_do(_is_par, _thread_parity)) {
+        JavaThread* jt = (JavaThread*)thread;
+
+        // In theory it should not be neccessary to explicitly walk the nmethods to find roots for concurrent marking
+        // however the liveness of oops reachable from nmethods have very complex lifecycles:
+        // * Alive if on the stack of an executing method
+        // * Weakly reachable otherwise
+        // Some objects reachable from nmethods, such as the class loader (or klass_holder) of the receiver should be
+        // live by the SATB invariant but other oops recorded in nmethods may behave differently.
+        jt->nmethods_do(&_code_cl);
+
+        jt->satb_mark_queue().apply_closure_and_empty(&_cm_obj);
+      }
+    } else if (thread->is_VM_thread()) {
+      if (thread->claim_oops_do(_is_par, _thread_parity)) {
+        JavaThread::satb_mark_queue_set().shared_satb_queue()->apply_closure_and_empty(&_cm_obj);
+      }
+    }
+  }
+};
+
 class CMRemarkTask: public AbstractGangTask {
 private:
   ConcurrentMark* _cm;
@@ -2572,6 +2702,14 @@
     if (worker_id < _cm->active_tasks()) {
       CMTask* task = _cm->task(worker_id);
       task->record_start_time();
+      {
+        ResourceMark rm;
+        HandleMark hm;
+
+        G1RemarkThreadsClosure threads_f(G1CollectedHeap::heap(), task, !_is_serial);
+        Threads::threads_do(&threads_f);
+      }
+
       do {
         task->do_marking_step(1000000000.0 /* something very large */,
                               true         /* do_termination       */,
@@ -2594,6 +2732,8 @@
   HandleMark   hm;
   G1CollectedHeap* g1h = G1CollectedHeap::heap();
 
+  G1RemarkGCTraceTime trace("Finalize Marking", G1Log::finer());
+
   g1h->ensure_parsability(false);
 
   if (G1CollectedHeap::use_parallel_gc_threads()) {
@@ -2673,7 +2813,6 @@
       str = " O";
     } else {
       HeapRegion* hr  = _g1h->heap_region_containing(obj);
-      guarantee(hr != NULL, "invariant");
       bool over_tams = _g1h->allocated_since_marking(obj, hr, _vo);
       bool marked = _g1h->is_marked(obj, _vo);
 
@@ -2814,11 +2953,6 @@
   _nextMarkBitMap->clearRange(mr);
 }
 
-void ConcurrentMark::clearRangeBothBitmaps(MemRegion mr) {
-  clearRangePrevBitmap(mr);
-  clearRangeNextBitmap(mr);
-}
-
 HeapRegion*
 ConcurrentMark::claim_region(uint worker_id) {
   // "checkpoint" the finger
@@ -2852,22 +2986,25 @@
     // claim_region() and a humongous object allocation might force us
     // to do a bit of unnecessary work (due to some unnecessary bitmap
     // iterations) but it should not introduce and correctness issues.
-    HeapRegion* curr_region   = _g1h->heap_region_containing_raw(finger);
-    HeapWord*   bottom        = curr_region->bottom();
-    HeapWord*   end           = curr_region->end();
-    HeapWord*   limit         = curr_region->next_top_at_mark_start();
-
-    if (verbose_low()) {
-      gclog_or_tty->print_cr("[%u] curr_region = "PTR_FORMAT" "
-                             "["PTR_FORMAT", "PTR_FORMAT"), "
-                             "limit = "PTR_FORMAT,
-                             worker_id, p2i(curr_region), p2i(bottom), p2i(end), p2i(limit));
-    }
+    HeapRegion* curr_region = _g1h->heap_region_containing_raw(finger);
+
+    // Above heap_region_containing_raw may return NULL as we always scan claim
+    // until the end of the heap. In this case, just jump to the next region.
+    HeapWord* end = curr_region != NULL ? curr_region->end() : finger + HeapRegion::GrainWords;
 
     // Is the gap between reading the finger and doing the CAS too long?
     HeapWord* res = (HeapWord*) Atomic::cmpxchg_ptr(end, &_finger, finger);
-    if (res == finger) {
+    if (res == finger && curr_region != NULL) {
       // we succeeded
+      HeapWord*   bottom        = curr_region->bottom();
+      HeapWord*   limit         = curr_region->next_top_at_mark_start();
+
+      if (verbose_low()) {
+        gclog_or_tty->print_cr("[%u] curr_region = "PTR_FORMAT" "
+                               "["PTR_FORMAT", "PTR_FORMAT"), "
+                               "limit = "PTR_FORMAT,
+                               worker_id, p2i(curr_region), p2i(bottom), p2i(end), p2i(limit));
+      }
 
       // notice that _finger == end cannot be guaranteed here since,
       // someone else might have moved the finger even further
@@ -2898,10 +3035,17 @@
     } else {
       assert(_finger > finger, "the finger should have moved forward");
       if (verbose_low()) {
-        gclog_or_tty->print_cr("[%u] somebody else moved the finger, "
-                               "global finger = "PTR_FORMAT", "
-                               "our finger = "PTR_FORMAT,
-                               worker_id, p2i(_finger), p2i(finger));
+        if (curr_region == NULL) {
+          gclog_or_tty->print_cr("[%u] found uncommitted region, moving finger, "
+                                 "global finger = "PTR_FORMAT", "
+                                 "our finger = "PTR_FORMAT,
+                                 worker_id, p2i(_finger), p2i(finger));
+        } else {
+          gclog_or_tty->print_cr("[%u] somebody else moved the finger, "
+                                 "global finger = "PTR_FORMAT", "
+                                 "our finger = "PTR_FORMAT,
+                                 worker_id, p2i(_finger), p2i(finger));
+        }
       }
 
       // read it again
@@ -3016,8 +3160,10 @@
       // happens, heap_region_containing() will return the bottom of the
       // corresponding starts humongous region and the check below will
       // not hold any more.
+      // Since we always iterate over all regions, we might get a NULL HeapRegion
+      // here.
       HeapRegion* global_hr = _g1h->heap_region_containing_raw(global_finger);
-      guarantee(global_finger == global_hr->bottom(),
+      guarantee(global_hr == NULL || global_finger == global_hr->bottom(),
                 err_msg("global finger: "PTR_FORMAT" region: "HR_FORMAT,
                         p2i(global_finger), HR_FORMAT_PARAMS(global_hr)));
     }
@@ -3030,7 +3176,7 @@
       if (task_finger != NULL && task_finger < _heap_end) {
         // See above note on the global finger verification.
         HeapRegion* task_hr = _g1h->heap_region_containing_raw(task_finger);
-        guarantee(task_finger == task_hr->bottom() ||
+        guarantee(task_hr == NULL || task_finger == task_hr->bottom() ||
                   !task_hr->in_collection_set(),
                   err_msg("task finger: "PTR_FORMAT" region: "HR_FORMAT,
                           p2i(task_finger), HR_FORMAT_PARAMS(task_hr)));
@@ -3109,7 +3255,7 @@
     assert(limit_idx <= end_idx, "or else use atomics");
 
     // Aggregate the "stripe" in the count data associated with hr.
-    uint hrs_index = hr->hrs_index();
+    uint hrm_index = hr->hrm_index();
     size_t marked_bytes = 0;
 
     for (uint i = 0; i < _max_worker_id; i += 1) {
@@ -3118,7 +3264,7 @@
 
       // 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];
+      marked_bytes += marked_bytes_array[hrm_index];
 
       // Now union the bitmaps[0,max_worker_id)[start_idx..limit_idx)
       // into the global card bitmap.
@@ -3241,8 +3387,14 @@
 
 // abandon current marking iteration due to a Full GC
 void ConcurrentMark::abort() {
-  // Clear all marks to force marking thread to do nothing
+  // Clear all marks in the next bitmap for the next marking cycle. This will allow us to skip the next
+  // concurrent bitmap clearing.
   _nextMarkBitMap->clearAll();
+
+  // Note we cannot clear the previous marking bitmap here
+  // since VerifyDuringGC verifies the objects marked during
+  // a full GC against the previous bitmap.
+
   // Clear the liveness counting data
   clear_all_count_data();
   // Empty mark stack
@@ -3252,6 +3404,12 @@
   }
   _first_overflow_barrier_sync.abort();
   _second_overflow_barrier_sync.abort();
+  const GCId& gc_id = _g1h->gc_tracer_cm()->gc_id();
+  if (!gc_id.is_undefined()) {
+    // We can do multiple full GCs before ConcurrentMarkThread::run() gets a chance
+    // to detect that it was aborted. Only keep track of the first GC id that we aborted.
+    _aborted_gc_id = gc_id;
+   }
   _has_aborted = true;
 
   SATBMarkQueueSet& satb_mq_set = JavaThread::satb_mark_queue_set();
@@ -3266,6 +3424,13 @@
   _g1h->register_concurrent_cycle_end();
 }
 
+const GCId& ConcurrentMark::concurrent_gc_id() {
+  if (has_aborted()) {
+    return _aborted_gc_id;
+  }
+  return _g1h->gc_tracer_cm()->gc_id();
+}
+
 static void print_ms_time_info(const char* prefix, const char* name,
                                NumberSeq& ns) {
   gclog_or_tty->print_cr("%s%5d %12s: total time = %8.2f s (avg = %8.2f ms).",
@@ -3322,32 +3487,17 @@
 
 // We take a break if someone is trying to stop the world.
 bool ConcurrentMark::do_yield_check(uint worker_id) {
-  if (should_yield()) {
+  if (SuspendibleThreadSet::should_yield()) {
     if (worker_id == 0) {
       _g1h->g1_policy()->record_concurrent_pause();
     }
-    cmThread()->yield();
+    SuspendibleThreadSet::yield();
     return true;
   } else {
     return false;
   }
 }
 
-bool ConcurrentMark::should_yield() {
-  return cmThread()->should_yield();
-}
-
-bool ConcurrentMark::containing_card_is_marked(void* p) {
-  size_t offset = pointer_delta(p, _g1h->reserved_region().start(), 1);
-  return _card_bm.at(offset >> CardTableModRefBS::card_shift);
-}
-
-bool ConcurrentMark::containing_cards_are_marked(void* start,
-                                                 void* last) {
-  return containing_card_is_marked(start) &&
-         containing_card_is_marked(last);
-}
-
 #ifndef PRODUCT
 // for debugging purposes
 void ConcurrentMark::print_finger() {
@@ -3410,20 +3560,6 @@
   }
 };
 
-// Closure for iterating over objects, currently only used for
-// processing SATB buffers.
-class CMObjectClosure : public ObjectClosure {
-private:
-  CMTask* _task;
-
-public:
-  void do_object(oop obj) {
-    _task->deal_with_reference(obj);
-  }
-
-  CMObjectClosure(CMTask* task) : _task(task) { }
-};
-
 G1CMOopClosure::G1CMOopClosure(G1CollectedHeap* g1h,
                                ConcurrentMark* cm,
                                CMTask* task)
@@ -3437,9 +3573,8 @@
 }
 
 void CMTask::setup_for_region(HeapRegion* hr) {
-  // Separated the asserts so that we know which one fires.
   assert(hr != NULL,
-        "claim_region() should have filtered out continues humongous regions");
+        "claim_region() should have filtered out NULL regions");
   assert(!hr->continuesHumongous(),
         "claim_region() should have filtered out continues humongous regions");
 
@@ -3615,7 +3750,7 @@
 
   if (_cm->verbose_medium()) {
       gclog_or_tty->print_cr("[%u] regular clock, interval = %1.2lfms, "
-                        "scanned = %d%s, refs reached = %d%s",
+                        "scanned = "SIZE_FORMAT"%s, refs reached = "SIZE_FORMAT"%s",
                         _worker_id, last_interval_ms,
                         _words_scanned,
                         (_words_scanned >= _words_scanned_limit) ? " (*)" : "",
@@ -3625,7 +3760,7 @@
 #endif // _MARKING_STATS_
 
   // (4) We check whether we should yield. If we have to, then we abort.
-  if (_cm->should_yield()) {
+  if (SuspendibleThreadSet::should_yield()) {
     // We should yield. To do this we abort the task. The caller is
     // responsible for yielding.
     set_has_aborted();
@@ -3889,15 +4024,6 @@
     }
   }
 
-  if (!concurrent() && !has_aborted()) {
-    // We should only do this during remark.
-    if (G1CollectedHeap::use_parallel_gc_threads()) {
-      satb_mq_set.par_iterate_closure_all_threads(_worker_id);
-    } else {
-      satb_mq_set.iterate_closure_all_threads();
-    }
-  }
-
   _draining_satb_buffers = false;
 
   assert(has_aborted() ||
@@ -4555,7 +4681,6 @@
     _hum_prev_live_bytes(0), _hum_next_live_bytes(0),
     _total_remset_bytes(0), _total_strong_code_roots_bytes(0) {
   G1CollectedHeap* g1h = G1CollectedHeap::heap();
-  MemRegion g1_committed = g1h->g1_committed();
   MemRegion g1_reserved = g1h->g1_reserved();
   double now = os::elapsedTime();
 
@@ -4563,10 +4688,8 @@
   _out->cr();
   _out->print_cr(G1PPRL_LINE_PREFIX" PHASE %s @ %1.3f", phase_name, now);
   _out->print_cr(G1PPRL_LINE_PREFIX" HEAP"
-                 G1PPRL_SUM_ADDR_FORMAT("committed")
                  G1PPRL_SUM_ADDR_FORMAT("reserved")
                  G1PPRL_SUM_BYTE_FORMAT("region-size"),
-                 p2i(g1_committed.start()), p2i(g1_committed.end()),
                  p2i(g1_reserved.start()), p2i(g1_reserved.end()),
                  HeapRegion::GrainBytes);
   _out->print_cr(G1PPRL_LINE_PREFIX);
@@ -4627,7 +4750,7 @@
 }
 
 bool G1PrintRegionLivenessInfoClosure::doHeapRegion(HeapRegion* r) {
-  const char* type = "";
+  const char* type       = r->get_type_str();
   HeapWord* bottom       = r->bottom();
   HeapWord* end          = r->end();
   size_t capacity_bytes  = r->capacity();
@@ -4638,15 +4761,7 @@
   size_t remset_bytes    = r->rem_set()->mem_size();
   size_t strong_code_roots_bytes = r->rem_set()->strong_code_roots_mem_size();
 
-  if (r->used() == 0) {
-    type = "FREE";
-  } else if (r->is_survivor()) {
-    type = "SURV";
-  } else if (r->is_young()) {
-    type = "EDEN";
-  } else if (r->startsHumongous()) {
-    type = "HUMS";
-
+  if (r->startsHumongous()) {
     assert(_hum_used_bytes == 0 && _hum_capacity_bytes == 0 &&
            _hum_prev_live_bytes == 0 && _hum_next_live_bytes == 0,
            "they should have been zeroed after the last time we used them");
@@ -4659,12 +4774,9 @@
                   &prev_live_bytes, &next_live_bytes);
     end = bottom + HeapRegion::GrainWords;
   } else if (r->continuesHumongous()) {
-    type = "HUMC";
     get_hum_bytes(&used_bytes, &capacity_bytes,
                   &prev_live_bytes, &next_live_bytes);
     assert(end == bottom + HeapRegion::GrainWords, "invariant");
-  } else {
-    type = "OLD";
   }
 
   _total_used_bytes      += used_bytes;