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

Print this page
rev 3708 : 8000244: G1: Ergonomically set MarkStackSize and use virtual space for global marking stack
Summary: Set the value of MarkStackSize to a value based on the number of parallel marking threads with a reasonable minimum. Expand the marking stack if we have to restart marking due to an overflow up to a reasonable maximum. Allocate the underlying space for the marking stack from virtual memory.
Reviewed-by: jmasa
rev 3709 : imported patch reuse-old-marking-stack


  29 #include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
  30 #include "gc_implementation/g1/g1CollectorPolicy.hpp"
  31 #include "gc_implementation/g1/g1ErgoVerbose.hpp"
  32 #include "gc_implementation/g1/g1Log.hpp"
  33 #include "gc_implementation/g1/g1OopClosures.inline.hpp"
  34 #include "gc_implementation/g1/g1RemSet.hpp"
  35 #include "gc_implementation/g1/heapRegion.inline.hpp"
  36 #include "gc_implementation/g1/heapRegionRemSet.hpp"
  37 #include "gc_implementation/g1/heapRegionSeq.inline.hpp"
  38 #include "gc_implementation/shared/vmGCOperations.hpp"
  39 #include "memory/genOopClosures.inline.hpp"
  40 #include "memory/referencePolicy.hpp"
  41 #include "memory/resourceArea.hpp"
  42 #include "oops/oop.inline.hpp"
  43 #include "runtime/handles.inline.hpp"
  44 #include "runtime/java.hpp"
  45 #include "services/memTracker.hpp"
  46 
  47 // Concurrent marking bit map wrapper
  48 
  49 CMBitMapRO::CMBitMapRO(ReservedSpace rs, int shifter) :
  50   _bm((uintptr_t*)NULL,0),
  51   _shifter(shifter) {
  52   _bmStartWord = (HeapWord*)(rs.base());
  53   _bmWordSize  = rs.size()/HeapWordSize;    // rs.size() is in bytes
  54   ReservedSpace brs(ReservedSpace::allocation_align_size_up(
  55                      (_bmWordSize >> (_shifter + LogBitsPerByte)) + 1));
  56 
  57   MemTracker::record_virtual_memory_type((address)brs.base(), mtGC);
  58 
  59   guarantee(brs.is_reserved(), "couldn't allocate concurrent marking bit map");
  60   // For now we'll just commit all of the bit map up fromt.
  61   // Later on we'll try to be more parsimonious with swap.
  62   guarantee(_virtual_space.initialize(brs, brs.size()),
  63             "couldn't reseve backing store for concurrent marking bit map");
  64   assert(_virtual_space.committed_size() == brs.size(),
  65          "didn't reserve backing store for all of concurrent marking bit map?");
  66   _bm.set_map((uintptr_t*)_virtual_space.low());
  67   assert(_virtual_space.committed_size() << (_shifter + LogBitsPerByte) >=
  68          _bmWordSize, "inconsistency in bit map sizing");
  69   _bm.set_size(_bmWordSize >> _shifter);
  70 }
  71 
  72 HeapWord* CMBitMapRO::getNextMarkedWordAddress(HeapWord* addr,
  73                                                HeapWord* limit) const {
  74   // First we must round addr *up* to a possible object boundary.
  75   addr = (HeapWord*)align_size_up((intptr_t)addr,
  76                                   HeapWordSize << _shifter);
  77   size_t addrOffset = heapWordToOffset(addr);
  78   if (limit == NULL) {
  79     limit = _bmStartWord + _bmWordSize;
  80   }
  81   size_t limitOffset = heapWordToOffset(limit);
  82   size_t nextOffset = _bm.get_next_one_offset(addrOffset, limitOffset);
  83   HeapWord* nextAddr = offsetToHeapWord(nextOffset);
  84   assert(nextAddr >= addr, "get_next_one postcondition");
  85   assert(nextAddr == limit || isMarked(nextAddr),
  86          "get_next_one postcondition");
  87   return nextAddr;
  88 }
  89 


  91                                                  HeapWord* limit) const {
  92   size_t addrOffset = heapWordToOffset(addr);
  93   if (limit == NULL) {
  94     limit = _bmStartWord + _bmWordSize;
  95   }
  96   size_t limitOffset = heapWordToOffset(limit);
  97   size_t nextOffset = _bm.get_next_zero_offset(addrOffset, limitOffset);
  98   HeapWord* nextAddr = offsetToHeapWord(nextOffset);
  99   assert(nextAddr >= addr, "get_next_one postcondition");
 100   assert(nextAddr == limit || !isMarked(nextAddr),
 101          "get_next_one postcondition");
 102   return nextAddr;
 103 }
 104 
 105 int CMBitMapRO::heapWordDiffToOffsetDiff(size_t diff) const {
 106   assert((diff & ((1 << _shifter) - 1)) == 0, "argument check");
 107   return (int) (diff >> _shifter);
 108 }
 109 
 110 #ifndef PRODUCT
 111 bool CMBitMapRO::covers(ReservedSpace rs) const {
 112   // assert(_bm.map() == _virtual_space.low(), "map inconsistency");
 113   assert(((size_t)_bm.size() * ((size_t)1 << _shifter)) == _bmWordSize,
 114          "size inconsistency");
 115   return _bmStartWord == (HeapWord*)(rs.base()) &&
 116          _bmWordSize  == rs.size()>>LogHeapWordSize;
 117 }
 118 #endif
 119 

























 120 void CMBitMap::clearAll() {
 121   _bm.clear();
 122   return;
 123 }
 124 
 125 void CMBitMap::markRange(MemRegion mr) {
 126   mr.intersection(MemRegion(_bmStartWord, _bmWordSize));
 127   assert(!mr.is_empty(), "unexpected empty region");
 128   assert((offsetToHeapWord(heapWordToOffset(mr.end())) ==
 129           ((HeapWord *) mr.end())),
 130          "markRange memory region end is not card aligned");
 131   // convert address range into offset range
 132   _bm.at_put_range(heapWordToOffset(mr.start()),
 133                    heapWordToOffset(mr.end()), true);
 134 }
 135 
 136 void CMBitMap::clearRange(MemRegion mr) {
 137   mr.intersection(MemRegion(_bmStartWord, _bmWordSize));
 138   assert(!mr.is_empty(), "unexpected empty region");
 139   // convert address range into offset range


 146   HeapWord* start = getNextMarkedWordAddress(addr);
 147   start = MIN2(start, end_addr);
 148   HeapWord* end   = getNextUnmarkedWordAddress(start);
 149   end = MIN2(end, end_addr);
 150   assert(start <= end, "Consistency check");
 151   MemRegion mr(start, end);
 152   if (!mr.is_empty()) {
 153     clearRange(mr);
 154   }
 155   return mr;
 156 }
 157 
 158 CMMarkStack::CMMarkStack(ConcurrentMark* cm) :
 159   _base(NULL), _cm(cm)
 160 #ifdef ASSERT
 161   , _drain_in_progress(false)
 162   , _drain_in_progress_yields(false)
 163 #endif
 164 {}
 165 
 166 void CMMarkStack::allocate(size_t size) {
 167   _base = NEW_C_HEAP_ARRAY(oop, size, mtGC);
 168   if (_base == NULL) {
 169     vm_exit_during_initialization("Failed to allocate CM region mark stack");


 170   }
 171   _index = 0;
 172   _capacity = (jint) size;











 173   _saved_index = -1;
 174   NOT_PRODUCT(_max_depth = 0);














































































 175 }
 176 
 177 CMMarkStack::~CMMarkStack() {
 178   if (_base != NULL) {
 179     FREE_C_HEAP_ARRAY(oop, _base, mtGC);

 180   }
 181 }
 182 
 183 void CMMarkStack::par_push(oop ptr) {
 184   while (true) {
 185     if (isFull()) {
 186       _overflow = true;
 187       return;
 188     }
 189     // Otherwise...
 190     jint index = _index;
 191     jint next_index = index+1;
 192     jint res = Atomic::cmpxchg(next_index, &_index, index);
 193     if (res == index) {
 194       _base[index] = ptr;
 195       // Note that we don't maintain this atomically.  We could, but it
 196       // doesn't seem necessary.
 197       NOT_PRODUCT(_max_depth = MAX2(_max_depth, next_index));
 198       return;
 199     }


 211     jint index = _index;
 212     jint next_index = index + n;
 213     if (next_index > _capacity) {
 214       _overflow = true;
 215       return;
 216     }
 217     jint res = Atomic::cmpxchg(next_index, &_index, index);
 218     if (res == index) {
 219       for (int i = 0; i < n; i++) {
 220         int ind = index + i;
 221         assert(ind < _capacity, "By overflow test above.");
 222         _base[ind] = ptr_arr[i];
 223       }
 224       NOT_PRODUCT(_max_depth = MAX2(_max_depth, next_index));
 225       return;
 226     }
 227     // Otherwise, we need to try again.
 228   }
 229 }
 230 
 231 
 232 void CMMarkStack::par_push_arr(oop* ptr_arr, int n) {
 233   MutexLockerEx x(ParGCRareEvent_lock, Mutex::_no_safepoint_check_flag);
 234   jint start = _index;
 235   jint next_index = start + n;
 236   if (next_index > _capacity) {
 237     _overflow = true;
 238     return;
 239   }
 240   // Otherwise.
 241   _index = next_index;
 242   for (int i = 0; i < n; i++) {
 243     int ind = start + i;
 244     assert(ind < _capacity, "By overflow test above.");
 245     _base[ind] = ptr_arr[i];
 246   }

 247 }
 248 
 249 
 250 bool CMMarkStack::par_pop_arr(oop* ptr_arr, int max, int* n) {
 251   MutexLockerEx x(ParGCRareEvent_lock, Mutex::_no_safepoint_check_flag);
 252   jint index = _index;
 253   if (index == 0) {
 254     *n = 0;
 255     return false;
 256   } else {
 257     int k = MIN2(max, index);
 258     jint new_ind = index - k;
 259     for (int j = 0; j < k; j++) {
 260       ptr_arr[j] = _base[new_ind + j];
 261     }
 262     _index = new_ind;
 263     *n = k;
 264     return true;
 265   }
 266 }
 267 
 268 template<class OopClosureClass>
 269 bool CMMarkStack::drain(OopClosureClass* cl, CMBitMap* bm, bool yield_after) {


 387 bool CMRootRegions::wait_until_scan_finished() {
 388   if (!scan_in_progress()) return false;
 389 
 390   {
 391     MutexLockerEx x(RootRegionScan_lock, Mutex::_no_safepoint_check_flag);
 392     while (scan_in_progress()) {
 393       RootRegionScan_lock->wait(Mutex::_no_safepoint_check_flag);
 394     }
 395   }
 396   return true;
 397 }
 398 
 399 #ifdef _MSC_VER // the use of 'this' below gets a warning, make it go away
 400 #pragma warning( disable:4355 ) // 'this' : used in base member initializer list
 401 #endif // _MSC_VER
 402 
 403 uint ConcurrentMark::scale_parallel_threads(uint n_par_threads) {
 404   return MAX2((n_par_threads + 2) / 4, 1U);
 405 }
 406 
 407 ConcurrentMark::ConcurrentMark(ReservedSpace rs, uint max_regions) :
 408   _markBitMap1(rs, MinObjAlignment - 1),
 409   _markBitMap2(rs, MinObjAlignment - 1),

 410 
 411   _parallel_marking_threads(0),
 412   _max_parallel_marking_threads(0),
 413   _sleep_factor(0.0),
 414   _marking_task_overhead(1.0),
 415   _cleanup_sleep_factor(0.0),
 416   _cleanup_task_overhead(1.0),
 417   _cleanup_list("Cleanup List"),
 418   _region_bm((BitMap::idx_t) max_regions, false /* in_resource_area*/),
 419   _card_bm((rs.size() + CardTableModRefBS::card_size - 1) >>
 420            CardTableModRefBS::card_shift,
 421            false /* in_resource_area*/),
 422 
 423   _prevMarkBitMap(&_markBitMap1),
 424   _nextMarkBitMap(&_markBitMap2),
 425 
 426   _markStack(this),
 427   // _finger set in set_non_marking_state
 428 
 429   _max_worker_id(MAX2((uint)ParallelGCThreads, 1U)),
 430   // _active_tasks set in set_non_marking_state
 431   // _tasks set inside the constructor
 432   _task_queues(new CMTaskQueueSet((int) _max_worker_id)),
 433   _terminator(ParallelTaskTerminator((int) _max_worker_id, _task_queues)),
 434 
 435   _has_overflown(false),
 436   _concurrent(false),
 437   _has_aborted(false),
 438   _restart_for_overflow(false),
 439   _concurrent_marking_in_progress(false),
 440 
 441   // _verbose_level set below
 442 
 443   _init_times(),
 444   _remark_times(), _remark_mark_times(), _remark_weak_ref_times(),
 445   _cleanup_times(),
 446   _total_counting_time(0.0),
 447   _total_rs_scrub_time(0.0),
 448 
 449   _parallel_workers(NULL),
 450 
 451   _count_card_bitmaps(NULL),
 452   _count_marked_bytes(NULL) {

 453   CMVerboseLevel verbose_level = (CMVerboseLevel) G1MarkingVerboseLevel;
 454   if (verbose_level < no_verbose) {
 455     verbose_level = no_verbose;
 456   }
 457   if (verbose_level > high_verbose) {
 458     verbose_level = high_verbose;
 459   }
 460   _verbose_level = verbose_level;
 461 
 462   if (verbose_low()) {
 463     gclog_or_tty->print_cr("[global] init, heap start = "PTR_FORMAT", "
 464                            "heap end = "PTR_FORMAT, _heap_start, _heap_end);
 465   }
 466 
 467   _markStack.allocate(MarkStackSize);







 468 
 469   // Create & start a ConcurrentMark thread.
 470   _cmThread = new ConcurrentMarkThread(this);
 471   assert(cmThread() != NULL, "CM Thread should have been created");
 472   assert(cmThread()->cm() != NULL, "CM Thread should refer to this cm");
 473 
 474   _g1h = G1CollectedHeap::heap();
 475   assert(CGC_lock != NULL, "Where's the CGC_lock?");
 476   assert(_markBitMap1.covers(rs), "_markBitMap1 inconsistency");
 477   assert(_markBitMap2.covers(rs), "_markBitMap2 inconsistency");
 478 
 479   SATBMarkQueueSet& satb_qs = JavaThread::satb_mark_queue_set();
 480   satb_qs.set_buffer_size(G1SATBBufferSize);
 481 
 482   _root_regions.init(_g1h, this);
 483 
 484   _tasks = NEW_C_HEAP_ARRAY(CMTask*, _max_worker_id, mtGC);
 485   _accum_task_vtime = NEW_C_HEAP_ARRAY(double, _max_worker_id, mtGC);
 486 
 487   _count_card_bitmaps = NEW_C_HEAP_ARRAY(BitMap,  _max_worker_id, mtGC);
 488   _count_marked_bytes = NEW_C_HEAP_ARRAY(size_t*, _max_worker_id, mtGC);
 489 
 490   BitMap::idx_t card_bm_size = _card_bm.size();
 491 
 492   // so that the assertion in MarkingTaskQueue::task_queue doesn't fail
 493   _active_tasks = _max_worker_id;
 494   for (uint i = 0; i < _max_worker_id; ++i) {
 495     CMTaskQueue* task_queue = new CMTaskQueue();
 496     task_queue->initialize();
 497     _task_queues->register_queue(i, task_queue);
 498 
 499     _count_card_bitmaps[i] = BitMap(card_bm_size, false);
 500     _count_marked_bytes[i] = NEW_C_HEAP_ARRAY(size_t, (size_t) max_regions, mtGC);
 501 
 502     _tasks[i] = new CMTask(i, this,
 503                            _count_marked_bytes[i],
 504                            &_count_card_bitmaps[i],
 505                            task_queue, _task_queues);
 506 
 507     _accum_task_vtime[i] = 0.0;
 508   }
 509 
 510   // Calculate the card number for the bottom of the heap. Used
 511   // in biasing indexes into the accounting card bitmaps.
 512   _heap_bottom_card_num =
 513     intptr_t(uintptr_t(_g1h->reserved_region().start()) >>
 514                                 CardTableModRefBS::card_shift);
 515 
 516   // Clear all the liveness counting data
 517   clear_all_count_data();
 518 
 519   if (ConcGCThreads > ParallelGCThreads) {
 520     vm_exit_during_initialization("Can't have more ConcGCThreads "
 521                                   "than ParallelGCThreads.");


 522   }
 523   if (ParallelGCThreads == 0) {
 524     // if we are not running with any parallel GC threads we will not
 525     // spawn any marking threads either
 526     _parallel_marking_threads =       0;
 527     _max_parallel_marking_threads =   0;
 528     _sleep_factor             =     0.0;
 529     _marking_task_overhead    =     1.0;
 530   } else {
 531     if (ConcGCThreads > 0) {
 532       // notice that ConcGCThreads overwrites G1MarkingOverheadPercent
 533       // if both are set
 534 
 535       _parallel_marking_threads = (uint) ConcGCThreads;
 536       _max_parallel_marking_threads = _parallel_marking_threads;
 537       _sleep_factor             = 0.0;
 538       _marking_task_overhead    = 1.0;
 539     } else if (G1MarkingOverheadPercent > 0) {
 540       // we will calculate the number of parallel marking threads
 541       // based on a target overhead with respect to the soft real-time


 573                      (1.0 - cleanup_task_overhead()) / cleanup_task_overhead();
 574 
 575 #if 0
 576     gclog_or_tty->print_cr("Marking Threads          %d", parallel_marking_threads());
 577     gclog_or_tty->print_cr("CM Marking Task Overhead %1.4lf", marking_task_overhead());
 578     gclog_or_tty->print_cr("CM Sleep Factor          %1.4lf", sleep_factor());
 579     gclog_or_tty->print_cr("CL Marking Task Overhead %1.4lf", cleanup_task_overhead());
 580     gclog_or_tty->print_cr("CL Sleep Factor          %1.4lf", cleanup_sleep_factor());
 581 #endif
 582 
 583     guarantee(parallel_marking_threads() > 0, "peace of mind");
 584     _parallel_workers = new FlexibleWorkGang("G1 Parallel Marking Threads",
 585          _max_parallel_marking_threads, false, true);
 586     if (_parallel_workers == NULL) {
 587       vm_exit_during_initialization("Failed necessary allocation.");
 588     } else {
 589       _parallel_workers->initialize_workers();
 590     }
 591   }
 592 












































































 593   // so that the call below can read a sensible value
 594   _heap_start = (HeapWord*) rs.base();
 595   set_non_marking_state();

 596 }
 597 
 598 void ConcurrentMark::update_g1_committed(bool force) {
 599   // If concurrent marking is not in progress, then we do not need to
 600   // update _heap_end.
 601   if (!concurrent_marking_in_progress() && !force) return;
 602 
 603   MemRegion committed = _g1h->g1_committed();
 604   assert(committed.start() == _heap_start, "start shouldn't change");
 605   HeapWord* new_end = committed.end();
 606   if (new_end > _heap_end) {
 607     // The heap has been expanded.
 608 
 609     _heap_end = new_end;
 610   }
 611   // Notice that the heap can also shrink. However, this only happens
 612   // during a Full GC (at least currently) and the entire marking
 613   // phase will bail out and the task will not be restarted. So, let's
 614   // do nothing.
 615 }


1148     // while marking.
1149     aggregate_count_data();
1150 
1151     SATBMarkQueueSet& satb_mq_set = JavaThread::satb_mark_queue_set();
1152     // We're done with marking.
1153     // This is the end of  the marking cycle, we're expected all
1154     // threads to have SATB queues with active set to true.
1155     satb_mq_set.set_active_all_threads(false, /* new active value */
1156                                        true /* expected_active */);
1157 
1158     if (VerifyDuringGC) {
1159       HandleMark hm;  // handle scope
1160       gclog_or_tty->print(" VerifyDuringGC:(after)");
1161       Universe::heap()->prepare_for_verify();
1162       Universe::verify(/* silent      */ false,
1163                        /* option      */ VerifyOption_G1UseNextMarking);
1164     }
1165     assert(!restart_for_overflow(), "sanity");
1166   }
1167 





1168   // Reset the marking state if marking completed
1169   if (!restart_for_overflow()) {
1170     set_non_marking_state();
1171   }
1172 
1173 #if VERIFY_OBJS_PROCESSED
1174   _scan_obj_cl.objs_processed = 0;
1175   ThreadLocalObjQueue::objs_enqueued = 0;
1176 #endif
1177 
1178   // Statistics
1179   double now = os::elapsedTime();
1180   _remark_mark_times.add((mark_work_end - start) * 1000.0);
1181   _remark_weak_ref_times.add((now - mark_work_end) * 1000.0);
1182   _remark_times.add((now - start) * 1000.0);
1183 
1184   g1p->record_concurrent_mark_remark_end();
1185 }
1186 
1187 // Base class of the closures that finalize and verify the


2768 
2769 void ConcurrentMark::verify_no_cset_oops(bool verify_stacks,
2770                                          bool verify_enqueued_buffers,
2771                                          bool verify_thread_buffers,
2772                                          bool verify_fingers) {
2773   assert(SafepointSynchronize::is_at_safepoint(), "should be at a safepoint");
2774   if (!G1CollectedHeap::heap()->mark_in_progress()) {
2775     return;
2776   }
2777 
2778   VerifyNoCSetOopsClosure cl;
2779 
2780   if (verify_stacks) {
2781     // Verify entries on the global mark stack
2782     cl.set_phase(VerifyNoCSetOopsStack);
2783     _markStack.oops_do(&cl);
2784 
2785     // Verify entries on the task queues
2786     for (uint i = 0; i < _max_worker_id; i += 1) {
2787       cl.set_phase(VerifyNoCSetOopsQueues, i);
2788       OopTaskQueue* queue = _task_queues->queue(i);
2789       queue->oops_do(&cl);
2790     }
2791   }
2792 
2793   SATBMarkQueueSet& satb_qs = JavaThread::satb_mark_queue_set();
2794 
2795   // Verify entries on the enqueued SATB buffers
2796   if (verify_enqueued_buffers) {
2797     cl.set_phase(VerifyNoCSetOopsSATBCompleted);
2798     satb_qs.iterate_completed_buffers_read_only(&cl);
2799   }
2800 
2801   // Verify entries on the per-thread SATB buffers
2802   if (verify_thread_buffers) {
2803     cl.set_phase(VerifyNoCSetOopsSATBThread);
2804     satb_qs.iterate_thread_buffers_read_only(&cl);
2805   }
2806 
2807   if (verify_fingers) {
2808     // Verify the global finger


2823 
2824     // Verify the task fingers
2825     assert(parallel_marking_threads() <= _max_worker_id, "sanity");
2826     for (int i = 0; i < (int) parallel_marking_threads(); i += 1) {
2827       CMTask* task = _tasks[i];
2828       HeapWord* task_finger = task->finger();
2829       if (task_finger != NULL && task_finger < _heap_end) {
2830         // See above note on the global finger verification.
2831         HeapRegion* task_hr = _g1h->heap_region_containing_raw(task_finger);
2832         guarantee(task_finger == task_hr->bottom() ||
2833                   !task_hr->in_collection_set(),
2834                   err_msg("task finger: "PTR_FORMAT" region: "HR_FORMAT,
2835                           task_finger, HR_FORMAT_PARAMS(task_hr)));
2836       }
2837     }
2838   }
2839 }
2840 #endif // PRODUCT
2841 
2842 void ConcurrentMark::clear_marking_state(bool clear_overflow) {
2843   _markStack.setEmpty();
2844   _markStack.clear_overflow();
2845   if (clear_overflow) {
2846     clear_has_overflown();
2847   } else {
2848     assert(has_overflown(), "pre-condition");
2849   }
2850   _finger = _heap_start;
2851 
2852   for (uint i = 0; i < _max_worker_id; ++i) {
2853     OopTaskQueue* queue = _task_queues->queue(i);
2854     queue->set_empty();
2855   }
2856 }
2857 
2858 // Aggregate the counting data that was constructed concurrently
2859 // with marking.
2860 class AggregateCountDataHRClosure: public HeapRegionClosure {
2861   G1CollectedHeap* _g1h;
2862   ConcurrentMark* _cm;
2863   CardTableModRefBS* _ct_bs;
2864   BitMap* _cm_card_bm;
2865   uint _max_worker_id;
2866 
2867  public:
2868   AggregateCountDataHRClosure(G1CollectedHeap* g1h,
2869                               BitMap* cm_card_bm,
2870                               uint max_worker_id) :
2871     _g1h(g1h), _cm(g1h->concurrent_mark()),
2872     _ct_bs((CardTableModRefBS*) (g1h->barrier_set())),
2873     _cm_card_bm(cm_card_bm), _max_worker_id(max_worker_id) { }




  29 #include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
  30 #include "gc_implementation/g1/g1CollectorPolicy.hpp"
  31 #include "gc_implementation/g1/g1ErgoVerbose.hpp"
  32 #include "gc_implementation/g1/g1Log.hpp"
  33 #include "gc_implementation/g1/g1OopClosures.inline.hpp"
  34 #include "gc_implementation/g1/g1RemSet.hpp"
  35 #include "gc_implementation/g1/heapRegion.inline.hpp"
  36 #include "gc_implementation/g1/heapRegionRemSet.hpp"
  37 #include "gc_implementation/g1/heapRegionSeq.inline.hpp"
  38 #include "gc_implementation/shared/vmGCOperations.hpp"
  39 #include "memory/genOopClosures.inline.hpp"
  40 #include "memory/referencePolicy.hpp"
  41 #include "memory/resourceArea.hpp"
  42 #include "oops/oop.inline.hpp"
  43 #include "runtime/handles.inline.hpp"
  44 #include "runtime/java.hpp"
  45 #include "services/memTracker.hpp"
  46 
  47 // Concurrent marking bit map wrapper
  48 
  49 CMBitMapRO::CMBitMapRO(int shifter) :
  50   _bm(),
  51   _shifter(shifter) {
  52   _bmStartWord = 0;
  53   _bmWordSize = 0;
















  54 }
  55 
  56 HeapWord* CMBitMapRO::getNextMarkedWordAddress(HeapWord* addr,
  57                                                HeapWord* limit) const {
  58   // First we must round addr *up* to a possible object boundary.
  59   addr = (HeapWord*)align_size_up((intptr_t)addr,
  60                                   HeapWordSize << _shifter);
  61   size_t addrOffset = heapWordToOffset(addr);
  62   if (limit == NULL) {
  63     limit = _bmStartWord + _bmWordSize;
  64   }
  65   size_t limitOffset = heapWordToOffset(limit);
  66   size_t nextOffset = _bm.get_next_one_offset(addrOffset, limitOffset);
  67   HeapWord* nextAddr = offsetToHeapWord(nextOffset);
  68   assert(nextAddr >= addr, "get_next_one postcondition");
  69   assert(nextAddr == limit || isMarked(nextAddr),
  70          "get_next_one postcondition");
  71   return nextAddr;
  72 }
  73 


  75                                                  HeapWord* limit) const {
  76   size_t addrOffset = heapWordToOffset(addr);
  77   if (limit == NULL) {
  78     limit = _bmStartWord + _bmWordSize;
  79   }
  80   size_t limitOffset = heapWordToOffset(limit);
  81   size_t nextOffset = _bm.get_next_zero_offset(addrOffset, limitOffset);
  82   HeapWord* nextAddr = offsetToHeapWord(nextOffset);
  83   assert(nextAddr >= addr, "get_next_one postcondition");
  84   assert(nextAddr == limit || !isMarked(nextAddr),
  85          "get_next_one postcondition");
  86   return nextAddr;
  87 }
  88 
  89 int CMBitMapRO::heapWordDiffToOffsetDiff(size_t diff) const {
  90   assert((diff & ((1 << _shifter) - 1)) == 0, "argument check");
  91   return (int) (diff >> _shifter);
  92 }
  93 
  94 #ifndef PRODUCT
  95 bool CMBitMapRO::covers(ReservedSpace heap_rs) const {
  96   // assert(_bm.map() == _virtual_space.low(), "map inconsistency");
  97   assert(((size_t)_bm.size() * ((size_t)1 << _shifter)) == _bmWordSize,
  98          "size inconsistency");
  99   return _bmStartWord == (HeapWord*)(heap_rs.base()) &&
 100          _bmWordSize  == heap_rs.size()>>LogHeapWordSize;
 101 }
 102 #endif
 103 
 104 bool CMBitMap::allocate(ReservedSpace heap_rs) {
 105   _bmStartWord = (HeapWord*)(heap_rs.base());
 106   _bmWordSize  = heap_rs.size()/HeapWordSize;    // heap_rs.size() is in bytes
 107   ReservedSpace brs(ReservedSpace::allocation_align_size_up(
 108                      (_bmWordSize >> (_shifter + LogBitsPerByte)) + 1));
 109   if (!brs.is_reserved()) {
 110     warning("ConcurrentMark marking bit map allocation failure");
 111     return false;
 112   }
 113   MemTracker::record_virtual_memory_type((address)brs.base(), mtGC);
 114   // For now we'll just commit all of the bit map up front.
 115   // Later on we'll try to be more parsimonious with swap.
 116   if (!_virtual_space.initialize(brs, brs.size())) {
 117     warning("ConcurrentMark marking bit map backing store failure");
 118     return false;
 119   }
 120   assert(_virtual_space.committed_size() == brs.size(),
 121          "didn't reserve backing store for all of concurrent marking bit map?");
 122   _bm.set_map((uintptr_t*)_virtual_space.low());
 123   assert(_virtual_space.committed_size() << (_shifter + LogBitsPerByte) >=
 124          _bmWordSize, "inconsistency in bit map sizing");
 125   _bm.set_size(_bmWordSize >> _shifter);
 126   return true;
 127 }
 128 
 129 void CMBitMap::clearAll() {
 130   _bm.clear();
 131   return;
 132 }
 133 
 134 void CMBitMap::markRange(MemRegion mr) {
 135   mr.intersection(MemRegion(_bmStartWord, _bmWordSize));
 136   assert(!mr.is_empty(), "unexpected empty region");
 137   assert((offsetToHeapWord(heapWordToOffset(mr.end())) ==
 138           ((HeapWord *) mr.end())),
 139          "markRange memory region end is not card aligned");
 140   // convert address range into offset range
 141   _bm.at_put_range(heapWordToOffset(mr.start()),
 142                    heapWordToOffset(mr.end()), true);
 143 }
 144 
 145 void CMBitMap::clearRange(MemRegion mr) {
 146   mr.intersection(MemRegion(_bmStartWord, _bmWordSize));
 147   assert(!mr.is_empty(), "unexpected empty region");
 148   // convert address range into offset range


 155   HeapWord* start = getNextMarkedWordAddress(addr);
 156   start = MIN2(start, end_addr);
 157   HeapWord* end   = getNextUnmarkedWordAddress(start);
 158   end = MIN2(end, end_addr);
 159   assert(start <= end, "Consistency check");
 160   MemRegion mr(start, end);
 161   if (!mr.is_empty()) {
 162     clearRange(mr);
 163   }
 164   return mr;
 165 }
 166 
 167 CMMarkStack::CMMarkStack(ConcurrentMark* cm) :
 168   _base(NULL), _cm(cm)
 169 #ifdef ASSERT
 170   , _drain_in_progress(false)
 171   , _drain_in_progress_yields(false)
 172 #endif
 173 {}
 174 
 175 bool CMMarkStack::allocate(size_t capacity) {
 176   // allocate a stack of the requisite depth
 177   ReservedSpace rs(ReservedSpace::allocation_align_size_up(capacity * sizeof(oop)));
 178   if (!rs.is_reserved()) {
 179     warning("ConcurrentMark MarkStack allocation failure");
 180     return false;
 181   }
 182   MemTracker::record_virtual_memory_type((address)rs.base(), mtGC);
 183   if (!_virtual_space.initialize(rs, rs.size())) {
 184     warning("ConcurrentMark MarkStack backing store failure");
 185     // Release the virtual memory reserved for the marking stack
 186     rs.release();
 187     return false;
 188   }
 189   assert(_virtual_space.committed_size() == rs.size(),
 190          "Didn't reserve backing store for all of ConcurrentMark stack?");
 191   _rs = rs;
 192   _base = (oop*) _virtual_space.low();
 193   setEmpty();
 194   _capacity = (jint) capacity;
 195   _saved_index = -1;
 196   NOT_PRODUCT(_max_depth = 0);
 197   return true;
 198 }
 199 
 200 void CMMarkStack::expand() {
 201   // Called, during remark, if we've overflown the marking stack during marking.
 202   assert(isEmpty(), "stack should been emptied while handling overflow");
 203   assert(_capacity <= (jint) MarkStackSizeMax, "stack bigger than permitted");
 204   // Clear expansion flag
 205   _should_expand = false;
 206   if (_capacity == (jint) MarkStackSizeMax) {
 207     if (PrintGCDetails && Verbose) {
 208       gclog_or_tty->print_cr(" (benign) Can't expand marking stack capacity, at max size limit");
 209     }
 210     return;
 211   }
 212   // Double capacity if possible
 213   jint new_capacity = MIN2(_capacity*2, (jint) MarkStackSizeMax);
 214   // Do not give up existing stack until we have managed to
 215   // get the double capacity that we desired.
 216   ReservedSpace rs(ReservedSpace::allocation_align_size_up(new_capacity *
 217                                                            sizeof(oop)));
 218   if (!rs.is_reserved()) {
 219     if (PrintGCDetails && Verbose) {
 220       // Failed to double capacity, continue;
 221       gclog_or_tty->print(" (benign) Failed to expand marking stack capacity from "
 222                           SIZE_FORMAT "K to " SIZE_FORMAT "K",
 223                           _capacity / K, new_capacity / K);
 224     }
 225     return;
 226   }
 227 
 228   // Clear the backing store fields associated with the space for the
 229   // old marking stack. Note this doesn't actuall release the space.
 230   _virtual_space.release();
 231 
 232   // Reinitialize virtual space for the expanded stack.
 233   if (!_virtual_space.initialize(rs, rs.size())) {
 234     // We failed to commit the the space for the expanded marking stack
 235     // Release the expanded reserved space...
 236     rs.release();
 237     // ... and reinitialize with the previous un-expanded space.
 238     if (_virtual_space.initialize(_rs, _rs.size())) {
 239       if (PrintGCDetails && Verbose) {
 240         gclog_or_tty->print(" (benign) Failed to expand marking stack capacity from "
 241                             SIZE_FORMAT "K to " SIZE_FORMAT "K",
 242                             _capacity / K, new_capacity / K);
 243       }
 244     } else {
 245       // The previous backing store space should have been already
 246       // committed but we failed to initialize the virtual space
 247       // for some reason.
 248       fatal("Error re-initializing marking stack with old capacity");
 249     }
 250   } else {
 251     // We successfully committed the space for the expanded marking stack.
 252     if (PrintGCDetails && Verbose) {
 253       gclog_or_tty->print(" Successfully expanded marking stack capacity from "
 254                           SIZE_FORMAT "K to " SIZE_FORMAT "K",
 255                           _capacity / K, new_capacity / K);
 256     }
 257     // Release the previous (unexpanded) space.
 258     _rs.release();
 259     // Record the new (expanded) space.
 260     _rs = rs;
 261     // Record the new capacity
 262     _capacity = new_capacity;
 263   }
 264   assert(_virtual_space.committed_size() == _rs.size(),
 265          "Didn't reserve backing store for all of ConcurrentMark stack?");
 266   _base = (oop*)(_virtual_space.low());
 267   _index = 0;
 268 }
 269 
 270 void CMMarkStack::set_should_expand() {
 271   // If we're resetting the marking state because of an
 272   // marking stack overflow, record that we should, if
 273   // possible, expand the stack.
 274   _should_expand = _cm->has_overflown();
 275 }
 276 
 277 CMMarkStack::~CMMarkStack() {
 278   if (_base != NULL) {
 279     _base = NULL;
 280     _virtual_space.release();
 281   }
 282 }
 283 
 284 void CMMarkStack::par_push(oop ptr) {
 285   while (true) {
 286     if (isFull()) {
 287       _overflow = true;
 288       return;
 289     }
 290     // Otherwise...
 291     jint index = _index;
 292     jint next_index = index+1;
 293     jint res = Atomic::cmpxchg(next_index, &_index, index);
 294     if (res == index) {
 295       _base[index] = ptr;
 296       // Note that we don't maintain this atomically.  We could, but it
 297       // doesn't seem necessary.
 298       NOT_PRODUCT(_max_depth = MAX2(_max_depth, next_index));
 299       return;
 300     }


 312     jint index = _index;
 313     jint next_index = index + n;
 314     if (next_index > _capacity) {
 315       _overflow = true;
 316       return;
 317     }
 318     jint res = Atomic::cmpxchg(next_index, &_index, index);
 319     if (res == index) {
 320       for (int i = 0; i < n; i++) {
 321         int  ind = index + i;
 322         assert(ind < _capacity, "By overflow test above.");
 323         _base[ind] = ptr_arr[i];
 324       }
 325       NOT_PRODUCT(_max_depth = MAX2(_max_depth, next_index));
 326       return;
 327     }
 328     // Otherwise, we need to try again.
 329   }
 330 }
 331 

 332 void CMMarkStack::par_push_arr(oop* ptr_arr, int n) {
 333   MutexLockerEx x(ParGCRareEvent_lock, Mutex::_no_safepoint_check_flag);
 334   jint start = _index;
 335   jint next_index = start + n;
 336   if (next_index > _capacity) {
 337     _overflow = true;
 338     return;
 339   }
 340   // Otherwise.
 341   _index = next_index;
 342   for (int i = 0; i < n; i++) {
 343     int ind = start + i;
 344     assert(ind < _capacity, "By overflow test above.");
 345     _base[ind] = ptr_arr[i];
 346   }
 347   NOT_PRODUCT(_max_depth = MAX2(_max_depth, next_index));
 348 }
 349 

 350 bool CMMarkStack::par_pop_arr(oop* ptr_arr, int max, int* n) {
 351   MutexLockerEx x(ParGCRareEvent_lock, Mutex::_no_safepoint_check_flag);
 352   jint index = _index;
 353   if (index == 0) {
 354     *n = 0;
 355     return false;
 356   } else {
 357     int k = MIN2(max, index);
 358     jint  new_ind = index - k;
 359     for (int j = 0; j < k; j++) {
 360       ptr_arr[j] = _base[new_ind + j];
 361     }
 362     _index = new_ind;
 363     *n = k;
 364     return true;
 365   }
 366 }
 367 
 368 template<class OopClosureClass>
 369 bool CMMarkStack::drain(OopClosureClass* cl, CMBitMap* bm, bool yield_after) {


 487 bool CMRootRegions::wait_until_scan_finished() {
 488   if (!scan_in_progress()) return false;
 489 
 490   {
 491     MutexLockerEx x(RootRegionScan_lock, Mutex::_no_safepoint_check_flag);
 492     while (scan_in_progress()) {
 493       RootRegionScan_lock->wait(Mutex::_no_safepoint_check_flag);
 494     }
 495   }
 496   return true;
 497 }
 498 
 499 #ifdef _MSC_VER // the use of 'this' below gets a warning, make it go away
 500 #pragma warning( disable:4355 ) // 'this' : used in base member initializer list
 501 #endif // _MSC_VER
 502 
 503 uint ConcurrentMark::scale_parallel_threads(uint n_par_threads) {
 504   return MAX2((n_par_threads + 2) / 4, 1U);
 505 }
 506 
 507 ConcurrentMark::ConcurrentMark(G1CollectedHeap* g1h, ReservedSpace heap_rs) :
 508   _g1h(g1h),
 509   _markBitMap1(MinObjAlignment - 1),
 510   _markBitMap2(MinObjAlignment - 1),
 511 
 512   _parallel_marking_threads(0),
 513   _max_parallel_marking_threads(0),
 514   _sleep_factor(0.0),
 515   _marking_task_overhead(1.0),
 516   _cleanup_sleep_factor(0.0),
 517   _cleanup_task_overhead(1.0),
 518   _cleanup_list("Cleanup List"),
 519   _region_bm((BitMap::idx_t)(g1h->max_regions()), false /* in_resource_area*/),
 520   _card_bm((heap_rs.size() + CardTableModRefBS::card_size - 1) >>
 521             CardTableModRefBS::card_shift,
 522             false /* in_resource_area*/),
 523 
 524   _prevMarkBitMap(&_markBitMap1),
 525   _nextMarkBitMap(&_markBitMap2),
 526 
 527   _markStack(this),
 528   // _finger set in set_non_marking_state
 529 
 530   _max_worker_id(MAX2((uint)ParallelGCThreads, 1U)),
 531   // _active_tasks set in set_non_marking_state
 532   // _tasks set inside the constructor
 533   _task_queues(new CMTaskQueueSet((int) _max_worker_id)),
 534   _terminator(ParallelTaskTerminator((int) _max_worker_id, _task_queues)),
 535 
 536   _has_overflown(false),
 537   _concurrent(false),
 538   _has_aborted(false),
 539   _restart_for_overflow(false),
 540   _concurrent_marking_in_progress(false),
 541 
 542   // _verbose_level set below
 543 
 544   _init_times(),
 545   _remark_times(), _remark_mark_times(), _remark_weak_ref_times(),
 546   _cleanup_times(),
 547   _total_counting_time(0.0),
 548   _total_rs_scrub_time(0.0),
 549 
 550   _parallel_workers(NULL),
 551 
 552   _count_card_bitmaps(NULL),
 553   _count_marked_bytes(NULL),
 554   _completed_initialization(false) {
 555   CMVerboseLevel verbose_level = (CMVerboseLevel) G1MarkingVerboseLevel;
 556   if (verbose_level < no_verbose) {
 557     verbose_level = no_verbose;
 558   }
 559   if (verbose_level > high_verbose) {
 560     verbose_level = high_verbose;
 561   }
 562   _verbose_level = verbose_level;
 563 
 564   if (verbose_low()) {
 565     gclog_or_tty->print_cr("[global] init, heap start = "PTR_FORMAT", "
 566                            "heap end = "PTR_FORMAT, _heap_start, _heap_end);
 567   }
 568 
 569   if (!_markBitMap1.allocate(heap_rs)) {
 570     warning("Failed to allocate first CM bit map");
 571     return;
 572   }
 573   if (!_markBitMap2.allocate(heap_rs)) {
 574     warning("Failed to allocate second CM bit map");
 575     return;
 576   }
 577 
 578   // Create & start a ConcurrentMark thread.
 579   _cmThread = new ConcurrentMarkThread(this);
 580   assert(cmThread() != NULL, "CM Thread should have been created");
 581   assert(cmThread()->cm() != NULL, "CM Thread should refer to this cm");
 582 

 583   assert(CGC_lock != NULL, "Where's the CGC_lock?");
 584   assert(_markBitMap1.covers(heap_rs), "_markBitMap1 inconsistency");
 585   assert(_markBitMap2.covers(heap_rs), "_markBitMap2 inconsistency");
 586 
 587   SATBMarkQueueSet& satb_qs = JavaThread::satb_mark_queue_set();
 588   satb_qs.set_buffer_size(G1SATBBufferSize);
 589 
 590   _root_regions.init(_g1h, this);
 591 



































 592   if (ConcGCThreads > ParallelGCThreads) {
 593     warning("Can't have more ConcGCThreads (" UINT32_FORMAT ") "
 594             "than ParallelGCThreads (" UINT32_FORMAT ").",
 595             ConcGCThreads, ParallelGCThreads);
 596     return;
 597   }
 598   if (ParallelGCThreads == 0) {
 599     // if we are not running with any parallel GC threads we will not
 600     // spawn any marking threads either
 601     _parallel_marking_threads =       0;
 602     _max_parallel_marking_threads =   0;
 603     _sleep_factor             =     0.0;
 604     _marking_task_overhead    =     1.0;
 605   } else {
 606     if (ConcGCThreads > 0) {
 607       // notice that ConcGCThreads overwrites G1MarkingOverheadPercent
 608       // if both are set
 609 
 610       _parallel_marking_threads = (uint) ConcGCThreads;
 611       _max_parallel_marking_threads = _parallel_marking_threads;
 612       _sleep_factor             = 0.0;
 613       _marking_task_overhead    = 1.0;
 614     } else if (G1MarkingOverheadPercent > 0) {
 615       // we will calculate the number of parallel marking threads
 616       // based on a target overhead with respect to the soft real-time


 648                      (1.0 - cleanup_task_overhead()) / cleanup_task_overhead();
 649 
 650 #if 0
 651     gclog_or_tty->print_cr("Marking Threads          %d", parallel_marking_threads());
 652     gclog_or_tty->print_cr("CM Marking Task Overhead %1.4lf", marking_task_overhead());
 653     gclog_or_tty->print_cr("CM Sleep Factor          %1.4lf", sleep_factor());
 654     gclog_or_tty->print_cr("CL Marking Task Overhead %1.4lf", cleanup_task_overhead());
 655     gclog_or_tty->print_cr("CL Sleep Factor          %1.4lf", cleanup_sleep_factor());
 656 #endif
 657 
 658     guarantee(parallel_marking_threads() > 0, "peace of mind");
 659     _parallel_workers = new FlexibleWorkGang("G1 Parallel Marking Threads",
 660          _max_parallel_marking_threads, false, true);
 661     if (_parallel_workers == NULL) {
 662       vm_exit_during_initialization("Failed necessary allocation.");
 663     } else {
 664       _parallel_workers->initialize_workers();
 665     }
 666   }
 667 
 668   if (FLAG_IS_DEFAULT(MarkStackSize)) {
 669     uintx mark_stack_size =
 670       MIN2(MarkStackSizeMax,
 671           MAX2(MarkStackSize, (uintx) (parallel_marking_threads() * TASKQUEUE_SIZE)));
 672     // Verify that the calculated value for MarkStackSize is in range.
 673     // It would be nice to use the private utility routine from Arguments.
 674     if (!(mark_stack_size >= 1 && mark_stack_size <= MarkStackSizeMax)) {
 675       warning("Invalid value calculated for MarkStackSize (" UINTX_FORMAT "): "
 676               "must be between " UINTX_FORMAT " and " UINTX_FORMAT,
 677               mark_stack_size, 1, MarkStackSizeMax);
 678       return;
 679     }
 680     FLAG_SET_ERGO(uintx, MarkStackSize, mark_stack_size);
 681   } else {
 682     // Verify MarkStackSize is in range.
 683     if (FLAG_IS_CMDLINE(MarkStackSize)) {
 684       if (FLAG_IS_DEFAULT(MarkStackSizeMax)) {
 685         if (!(MarkStackSize >= 1 && MarkStackSize <= MarkStackSizeMax)) {
 686           warning("Invalid value specified for MarkStackSize (" UINTX_FORMAT "): "
 687                   "must be between " UINTX_FORMAT " and " UINTX_FORMAT,
 688                   MarkStackSize, 1, MarkStackSizeMax);
 689           return;
 690         }
 691       } else if (FLAG_IS_CMDLINE(MarkStackSizeMax)) {
 692         if (!(MarkStackSize >= 1 && MarkStackSize <= MarkStackSizeMax)) {
 693           warning("Invalid value specified for MarkStackSize (" UINTX_FORMAT ")"
 694                   " or for MarkStackSizeMax (" UINTX_FORMAT ")",
 695                   MarkStackSize, MarkStackSizeMax);
 696           return;
 697         }
 698       }
 699     }
 700   }
 701 
 702   if (!_markStack.allocate(MarkStackSize)) {
 703     warning("Failed to allocate CM marking stack");
 704     return;
 705   }
 706 
 707   _tasks = NEW_C_HEAP_ARRAY(CMTask*, _max_worker_id, mtGC);
 708   _accum_task_vtime = NEW_C_HEAP_ARRAY(double, _max_worker_id, mtGC);
 709 
 710   _count_card_bitmaps = NEW_C_HEAP_ARRAY(BitMap,  _max_worker_id, mtGC);
 711   _count_marked_bytes = NEW_C_HEAP_ARRAY(size_t*, _max_worker_id, mtGC);
 712 
 713   BitMap::idx_t card_bm_size = _card_bm.size();
 714 
 715   // so that the assertion in MarkingTaskQueue::task_queue doesn't fail
 716   _active_tasks = _max_worker_id;
 717 
 718   size_t max_regions = (size_t) _g1h->max_regions();
 719   for (uint i = 0; i < _max_worker_id; ++i) {
 720     CMTaskQueue* task_queue = new CMTaskQueue();
 721     task_queue->initialize();
 722     _task_queues->register_queue(i, task_queue);
 723 
 724     _count_card_bitmaps[i] = BitMap(card_bm_size, false);
 725     _count_marked_bytes[i] = NEW_C_HEAP_ARRAY(size_t, max_regions, mtGC);
 726 
 727     _tasks[i] = new CMTask(i, this,
 728                            _count_marked_bytes[i],
 729                            &_count_card_bitmaps[i],
 730                            task_queue, _task_queues);
 731 
 732     _accum_task_vtime[i] = 0.0;
 733   }
 734 
 735   // Calculate the card number for the bottom of the heap. Used
 736   // in biasing indexes into the accounting card bitmaps.
 737   _heap_bottom_card_num =
 738     intptr_t(uintptr_t(_g1h->reserved_region().start()) >>
 739                                 CardTableModRefBS::card_shift);
 740 
 741   // Clear all the liveness counting data
 742   clear_all_count_data();
 743 
 744   // so that the call below can read a sensible value
 745   _heap_start = (HeapWord*) heap_rs.base();
 746   set_non_marking_state();
 747   _completed_initialization = true;
 748 }
 749 
 750 void ConcurrentMark::update_g1_committed(bool force) {
 751   // If concurrent marking is not in progress, then we do not need to
 752   // update _heap_end.
 753   if (!concurrent_marking_in_progress() && !force) return;
 754 
 755   MemRegion committed = _g1h->g1_committed();
 756   assert(committed.start() == _heap_start, "start shouldn't change");
 757   HeapWord* new_end = committed.end();
 758   if (new_end > _heap_end) {
 759     // The heap has been expanded.
 760 
 761     _heap_end = new_end;
 762   }
 763   // Notice that the heap can also shrink. However, this only happens
 764   // during a Full GC (at least currently) and the entire marking
 765   // phase will bail out and the task will not be restarted. So, let's
 766   // do nothing.
 767 }


1300     // while marking.
1301     aggregate_count_data();
1302 
1303     SATBMarkQueueSet& satb_mq_set = JavaThread::satb_mark_queue_set();
1304     // We're done with marking.
1305     // This is the end of  the marking cycle, we're expected all
1306     // threads to have SATB queues with active set to true.
1307     satb_mq_set.set_active_all_threads(false, /* new active value */
1308                                        true /* expected_active */);
1309 
1310     if (VerifyDuringGC) {
1311       HandleMark hm;  // handle scope
1312       gclog_or_tty->print(" VerifyDuringGC:(after)");
1313       Universe::heap()->prepare_for_verify();
1314       Universe::verify(/* silent      */ false,
1315                        /* option      */ VerifyOption_G1UseNextMarking);
1316     }
1317     assert(!restart_for_overflow(), "sanity");
1318   }
1319 
1320   // Expand the marking stack, if we have to and if we can.
1321   if (_markStack.should_expand()) {
1322     _markStack.expand();
1323   }
1324 
1325   // Reset the marking state if marking completed
1326   if (!restart_for_overflow()) {
1327     set_non_marking_state();
1328   }
1329 
1330 #if VERIFY_OBJS_PROCESSED
1331   _scan_obj_cl.objs_processed = 0;
1332   ThreadLocalObjQueue::objs_enqueued = 0;
1333 #endif
1334 
1335   // Statistics
1336   double now = os::elapsedTime();
1337   _remark_mark_times.add((mark_work_end - start) * 1000.0);
1338   _remark_weak_ref_times.add((now - mark_work_end) * 1000.0);
1339   _remark_times.add((now - start) * 1000.0);
1340 
1341   g1p->record_concurrent_mark_remark_end();
1342 }
1343 
1344 // Base class of the closures that finalize and verify the


2925 
2926 void ConcurrentMark::verify_no_cset_oops(bool verify_stacks,
2927                                          bool verify_enqueued_buffers,
2928                                          bool verify_thread_buffers,
2929                                          bool verify_fingers) {
2930   assert(SafepointSynchronize::is_at_safepoint(), "should be at a safepoint");
2931   if (!G1CollectedHeap::heap()->mark_in_progress()) {
2932     return;
2933   }
2934 
2935   VerifyNoCSetOopsClosure cl;
2936 
2937   if (verify_stacks) {
2938     // Verify entries on the global mark stack
2939     cl.set_phase(VerifyNoCSetOopsStack);
2940     _markStack.oops_do(&cl);
2941 
2942     // Verify entries on the task queues
2943     for (uint i = 0; i < _max_worker_id; i += 1) {
2944       cl.set_phase(VerifyNoCSetOopsQueues, i);
2945       CMTaskQueue* queue = _task_queues->queue(i);
2946       queue->oops_do(&cl);
2947     }
2948   }
2949 
2950   SATBMarkQueueSet& satb_qs = JavaThread::satb_mark_queue_set();
2951 
2952   // Verify entries on the enqueued SATB buffers
2953   if (verify_enqueued_buffers) {
2954     cl.set_phase(VerifyNoCSetOopsSATBCompleted);
2955     satb_qs.iterate_completed_buffers_read_only(&cl);
2956   }
2957 
2958   // Verify entries on the per-thread SATB buffers
2959   if (verify_thread_buffers) {
2960     cl.set_phase(VerifyNoCSetOopsSATBThread);
2961     satb_qs.iterate_thread_buffers_read_only(&cl);
2962   }
2963 
2964   if (verify_fingers) {
2965     // Verify the global finger


2980 
2981     // Verify the task fingers
2982     assert(parallel_marking_threads() <= _max_worker_id, "sanity");
2983     for (int i = 0; i < (int) parallel_marking_threads(); i += 1) {
2984       CMTask* task = _tasks[i];
2985       HeapWord* task_finger = task->finger();
2986       if (task_finger != NULL && task_finger < _heap_end) {
2987         // See above note on the global finger verification.
2988         HeapRegion* task_hr = _g1h->heap_region_containing_raw(task_finger);
2989         guarantee(task_finger == task_hr->bottom() ||
2990                   !task_hr->in_collection_set(),
2991                   err_msg("task finger: "PTR_FORMAT" region: "HR_FORMAT,
2992                           task_finger, HR_FORMAT_PARAMS(task_hr)));
2993       }
2994     }
2995   }
2996 }
2997 #endif // PRODUCT
2998 
2999 void ConcurrentMark::clear_marking_state(bool clear_overflow) {
3000   _markStack.set_should_expand();
3001   _markStack.setEmpty();        // Also clears the _markStack overflow flag
3002   if (clear_overflow) {
3003     clear_has_overflown();
3004   } else {
3005     assert(has_overflown(), "pre-condition");
3006   }
3007   _finger = _heap_start;
3008 
3009   for (uint i = 0; i < _max_worker_id; ++i) {
3010     CMTaskQueue* queue = _task_queues->queue(i);
3011     queue->set_empty();
3012   }
3013 }
3014 
3015 // Aggregate the counting data that was constructed concurrently
3016 // with marking.
3017 class AggregateCountDataHRClosure: public HeapRegionClosure {
3018   G1CollectedHeap* _g1h;
3019   ConcurrentMark* _cm;
3020   CardTableModRefBS* _ct_bs;
3021   BitMap* _cm_card_bm;
3022   uint _max_worker_id;
3023 
3024  public:
3025   AggregateCountDataHRClosure(G1CollectedHeap* g1h,
3026                               BitMap* cm_card_bm,
3027                               uint max_worker_id) :
3028     _g1h(g1h), _cm(g1h->concurrent_mark()),
3029     _ct_bs((CardTableModRefBS*) (g1h->barrier_set())),
3030     _cm_card_bm(cm_card_bm), _max_worker_id(max_worker_id) { }