src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.cpp

Print this page




 707   assert(CGC_lock != NULL, "Where's the CGC_lock?");
 708 
 709   // Support for parallelizing young gen rescan
 710   GenCollectedHeap* gch = GenCollectedHeap::heap();
 711   _young_gen = gch->prev_gen(_cmsGen);
 712   if (gch->supports_inline_contig_alloc()) {
 713     _top_addr = gch->top_addr();
 714     _end_addr = gch->end_addr();
 715     assert(_young_gen != NULL, "no _young_gen");
 716     _eden_chunk_index = 0;
 717     _eden_chunk_capacity = (_young_gen->max_capacity()+CMSSamplingGrain)/CMSSamplingGrain;
 718     _eden_chunk_array = NEW_C_HEAP_ARRAY(HeapWord*, _eden_chunk_capacity, mtGC);
 719     if (_eden_chunk_array == NULL) {
 720       _eden_chunk_capacity = 0;
 721       warning("GC/CMS: _eden_chunk_array allocation failure");
 722     }
 723   }
 724   assert(_eden_chunk_array != NULL || _eden_chunk_capacity == 0, "Error");
 725 
 726   // Support for parallelizing survivor space rescan
 727   if (CMSParallelRemarkEnabled && CMSParallelSurvivorRemarkEnabled) {
 728     const size_t max_plab_samples =
 729       ((DefNewGeneration*)_young_gen)->max_survivor_size()/MinTLABSize;
 730 
 731     _survivor_plab_array  = NEW_C_HEAP_ARRAY(ChunkArray, ParallelGCThreads, mtGC);
 732     _survivor_chunk_array = NEW_C_HEAP_ARRAY(HeapWord*, 2*max_plab_samples, mtGC);
 733     _cursor               = NEW_C_HEAP_ARRAY(size_t, ParallelGCThreads, mtGC);
 734     if (_survivor_plab_array == NULL || _survivor_chunk_array == NULL
 735         || _cursor == NULL) {
 736       warning("Failed to allocate survivor plab/chunk array");
 737       if (_survivor_plab_array  != NULL) {
 738         FREE_C_HEAP_ARRAY(ChunkArray, _survivor_plab_array, mtGC);
 739         _survivor_plab_array = NULL;
 740       }
 741       if (_survivor_chunk_array != NULL) {
 742         FREE_C_HEAP_ARRAY(HeapWord*, _survivor_chunk_array, mtGC);
 743         _survivor_chunk_array = NULL;
 744       }
 745       if (_cursor != NULL) {
 746         FREE_C_HEAP_ARRAY(size_t, _cursor, mtGC);
 747         _cursor = NULL;


3480   _collector->stopTimer();
3481   _wallclock.stop();
3482   if (PrintGCDetails) {
3483     gclog_or_tty->date_stamp(PrintGCDateStamps);
3484     gclog_or_tty->stamp(PrintGCTimeStamps);
3485     gclog_or_tty->print("[%s-concurrent-%s: %3.3f/%3.3f secs]",
3486                  _collector->cmsGen()->short_name(),
3487                  _phase, _collector->timerValue(), _wallclock.seconds());
3488     if (_print_cr) {
3489       gclog_or_tty->print_cr("");
3490     }
3491     if (PrintCMSStatistics != 0) {
3492       gclog_or_tty->print_cr(" (CMS-concurrent-%s yielded %d times)", _phase,
3493                     _collector->yields());
3494     }
3495   }
3496 }
3497 
3498 // CMS work
3499 


























3500 // Checkpoint the roots into this generation from outside
3501 // this generation. [Note this initial checkpoint need only
3502 // be approximate -- we'll do a catch up phase subsequently.]
3503 void CMSCollector::checkpointRootsInitial(bool asynch) {
3504   assert(_collectorState == InitialMarking, "Wrong collector state");
3505   check_correct_thread_executing();
3506   TraceCMSMemoryManagerStats tms(_collectorState,GenCollectedHeap::heap()->gc_cause());
3507 
3508   ReferenceProcessor* rp = ref_processor();
3509   SpecializationStats::clear();
3510   assert(_restart_addr == NULL, "Control point invariant");
3511   if (asynch) {
3512     // acquire locks for subsequent manipulations
3513     MutexLockerEx x(bitMapLock(),
3514                     Mutex::_no_safepoint_check_flag);
3515     checkpointRootsInitialWork(asynch);
3516     // enable ("weak") refs discovery
3517     rp->enable_discovery(true /*verify_disabled*/, true /*check_no_refs*/);
3518     _collectorState = Marking;
3519   } else {


3574   GenCollectedHeap* gch = GenCollectedHeap::heap();
3575 
3576   verify_work_stacks_empty();
3577   verify_overflow_empty();
3578 
3579   gch->ensure_parsability(false);  // fill TLABs, but no need to retire them
3580   // Update the saved marks which may affect the root scans.
3581   gch->save_marks();
3582 
3583   // weak reference processing has not started yet.
3584   ref_processor()->set_enqueuing_is_done(false);
3585 
3586   // Need to remember all newly created CLDs,
3587   // so that we can guarantee that the remark finds them.
3588   ClassLoaderDataGraph::remember_new_clds(true);
3589 
3590   // Whenever a CLD is found, it will be claimed before proceeding to mark
3591   // the klasses. The claimed marks need to be cleared before marking starts.
3592   ClassLoaderDataGraph::clear_claimed_marks();
3593 
3594   CMKlassClosure klass_closure(&notOlder);
3595   {
3596     COMPILER2_PRESENT(DerivedPointerTableDeactivate dpt_deact;)



















3597     gch->rem_set()->prepare_for_younger_refs_iterate(false); // Not parallel.
3598     gch->gen_process_strong_roots(_cmsGen->level(),
3599                                   true,   // younger gens are roots
3600                                   true,   // activate StrongRootsScope
3601                                   false,  // not scavenging
3602                                   SharedHeap::ScanningOption(roots_scanning_options()),
3603                                   &notOlder,
3604                                   true,   // walk all of code cache if (so & SO_CodeCache)
3605                                   NULL,
3606                                   &klass_closure);
3607   }

3608 
3609   // Clear mod-union table; it will be dirtied in the prologue of
3610   // CMS generation per each younger generation collection.
3611 
3612   assert(_modUnionTable.isAllClear(),
3613        "Was cleared in most recent final checkpoint phase"
3614        " or no bits are set in the gc_prologue before the start of the next "
3615        "subsequent marking phase.");
3616 
3617   assert(_ct->klass_rem_set()->mod_union_is_clear(), "Must be");
3618 
3619   // Save the end of the used_region of the constituent generations
3620   // to be used to limit the extent of sweep in each generation.
3621   save_sweep_limits();
3622   if (UseAdaptiveSizePolicy) {
3623     size_policy()->checkpoint_roots_initial_end(gch->gc_cause());
3624   }
3625   verify_overflow_empty();
3626 }
3627 


5040   _markStack._hit_limit = 0;
5041   _markStack._failed_double = 0;
5042 
5043   if ((VerifyAfterGC || VerifyDuringGC) &&
5044       GenCollectedHeap::heap()->total_collections() >= VerifyGCStartAt) {
5045     verify_after_remark();
5046   }
5047 
5048   // Change under the freelistLocks.
5049   _collectorState = Sweeping;
5050   // Call isAllClear() under bitMapLock
5051   assert(_modUnionTable.isAllClear(),
5052       "Should be clear by end of the final marking");
5053   assert(_ct->klass_rem_set()->mod_union_is_clear(),
5054       "Should be clear by end of the final marking");
5055   if (UseAdaptiveSizePolicy) {
5056     size_policy()->checkpoint_roots_final_end(gch->gc_cause());
5057   }
5058 }
5059 













































5060 // Parallel remark task
5061 class CMSParRemarkTask: public AbstractGangTask {
5062   CMSCollector* _collector;
5063   int           _n_workers;
5064   CompactibleFreeListSpace* _cms_space;
5065 
5066   // The per-thread work queues, available here for stealing.
5067   OopTaskQueueSet*       _task_queues;
5068   ParallelTaskTerminator _term;
5069 
5070  public:
5071   // A value of 0 passed to n_workers will cause the number of
5072   // workers to be taken from the active workers in the work gang.
5073   CMSParRemarkTask(CMSCollector* collector,
5074                    CompactibleFreeListSpace* cms_space,
5075                    int n_workers, FlexibleWorkGang* workers,
5076                    OopTaskQueueSet* task_queues):
5077     AbstractGangTask("Rescan roots and grey objects in parallel"),
5078     _collector(collector),
5079     _cms_space(cms_space),
5080     _n_workers(n_workers),
5081     _task_queues(task_queues),
5082     _term(n_workers, task_queues) { }
5083 
5084   OopTaskQueueSet* task_queues() { return _task_queues; }
5085 
5086   OopTaskQueue* work_queue(int i) { return task_queues()->queue(i); }
5087 
5088   ParallelTaskTerminator* terminator() { return &_term; }
5089   int n_workers() { return _n_workers; }
5090 
5091   void work(uint worker_id);
5092 
5093  private:
5094   // Work method in support of parallel rescan ... of young gen spaces
5095   void do_young_space_rescan(int i, Par_MarkRefsIntoAndScanClosure* cl,
5096                              ContiguousSpace* space,
5097                              HeapWord** chunk_array, size_t chunk_top);
5098 
5099   // ... of  dirty cards in old space
5100   void do_dirty_card_rescan_tasks(CompactibleFreeListSpace* sp, int i,
5101                                   Par_MarkRefsIntoAndScanClosure* cl);
5102 
5103   // ... work stealing for the above
5104   void do_work_steal(int i, Par_MarkRefsIntoAndScanClosure* cl, int* seed);
5105 };
5106 
5107 class RemarkKlassClosure : public KlassClosure {
5108   CMKlassClosure _cm_klass_closure;
5109  public:
5110   RemarkKlassClosure(OopClosure* oop_closure) : _cm_klass_closure(oop_closure) {}
5111   void do_klass(Klass* k) {
5112     // Check if we have modified any oops in the Klass during the concurrent marking.
5113     if (k->has_accumulated_modified_oops()) {
5114       k->clear_accumulated_modified_oops();
5115 
5116       // We could have transfered the current modified marks to the accumulated marks,
5117       // like we do with the Card Table to Mod Union Table. But it's not really necessary.
5118     } else if (k->has_modified_oops()) {
5119       // Don't clear anything, this info is needed by the next young collection.
5120     } else {
5121       // No modified oops in the Klass.
5122       return;
5123     }
5124 
5125     // The klass has modified fields, need to scan the klass.
5126     _cm_klass_closure.do_klass(k);
5127   }
5128 };
5129 



















5130 // work_queue(i) is passed to the closure
5131 // Par_MarkRefsIntoAndScanClosure.  The "i" parameter
5132 // also is passed to do_dirty_card_rescan_tasks() and to
5133 // do_work_steal() to select the i-th task_queue.
5134 
5135 void CMSParRemarkTask::work(uint worker_id) {
5136   elapsedTimer _timer;
5137   ResourceMark rm;
5138   HandleMark   hm;
5139 
5140   // ---------- rescan from roots --------------
5141   _timer.start();
5142   GenCollectedHeap* gch = GenCollectedHeap::heap();
5143   Par_MarkRefsIntoAndScanClosure par_mrias_cl(_collector,
5144     _collector->_span, _collector->ref_processor(),
5145     &(_collector->_markBitMap),
5146     work_queue(worker_id));
5147 
5148   // Rescan young gen roots first since these are likely
5149   // coarsely partitioned and may, on that account, constitute
5150   // the critical path; thus, it's best to start off that
5151   // work first.
5152   // ---------- young gen roots --------------
5153   {
5154     DefNewGeneration* dng = _collector->_young_gen->as_DefNewGeneration();
5155     EdenSpace* eden_space = dng->eden();
5156     ContiguousSpace* from_space = dng->from();
5157     ContiguousSpace* to_space   = dng->to();
5158 
5159     HeapWord** eca = _collector->_eden_chunk_array;
5160     size_t     ect = _collector->_eden_chunk_index;
5161     HeapWord** sca = _collector->_survivor_chunk_array;
5162     size_t     sct = _collector->_survivor_chunk_index;
5163 
5164     assert(ect <= _collector->_eden_chunk_capacity, "out of bounds");
5165     assert(sct <= _collector->_survivor_chunk_capacity, "out of bounds");
5166 
5167     do_young_space_rescan(worker_id, &par_mrias_cl, to_space, NULL, 0);
5168     do_young_space_rescan(worker_id, &par_mrias_cl, from_space, sca, sct);
5169     do_young_space_rescan(worker_id, &par_mrias_cl, eden_space, eca, ect);
5170 
5171     _timer.stop();
5172     if (PrintCMSStatistics != 0) {
5173       gclog_or_tty->print_cr(
5174         "Finished young gen rescan work in %dth thread: %3.3f sec",
5175         worker_id, _timer.seconds());
5176     }
5177   }
5178 
5179   // ---------- remaining roots --------------
5180   _timer.reset();
5181   _timer.start();
5182   gch->gen_process_strong_roots(_collector->_cmsGen->level(),
5183                                 false,     // yg was scanned above
5184                                 false,     // this is parallel code
5185                                 false,     // not scavenging
5186                                 SharedHeap::ScanningOption(_collector->CMSCollector::roots_scanning_options()),
5187                                 &par_mrias_cl,
5188                                 true,   // walk all of code cache if (so & SO_CodeCache)
5189                                 NULL,
5190                                 NULL);     // The dirty klasses will be handled below


5258     gclog_or_tty->print_cr(
5259       "Finished dirty card rescan work in %dth thread: %3.3f sec",
5260       worker_id, _timer.seconds());
5261   }
5262 
5263   // ---------- steal work from other threads ...
5264   // ---------- ... and drain overflow list.
5265   _timer.reset();
5266   _timer.start();
5267   do_work_steal(worker_id, &par_mrias_cl, _collector->hash_seed(worker_id));
5268   _timer.stop();
5269   if (PrintCMSStatistics != 0) {
5270     gclog_or_tty->print_cr(
5271       "Finished work stealing in %dth thread: %3.3f sec",
5272       worker_id, _timer.seconds());
5273   }
5274 }
5275 
5276 // Note that parameter "i" is not used.
5277 void
5278 CMSParRemarkTask::do_young_space_rescan(int i,
5279   Par_MarkRefsIntoAndScanClosure* cl, ContiguousSpace* space,
5280   HeapWord** chunk_array, size_t chunk_top) {
5281   // Until all tasks completed:
5282   // . claim an unclaimed task
5283   // . compute region boundaries corresponding to task claimed
5284   //   using chunk_array
5285   // . par_oop_iterate(cl) over that region
5286 
5287   ResourceMark rm;
5288   HandleMark   hm;
5289 
5290   SequentialSubTasksDone* pst = space->par_seq_tasks();
5291   assert(pst->valid(), "Uninitialized use?");
5292 
5293   uint nth_task = 0;
5294   uint n_tasks  = pst->n_tasks();
5295 
5296   HeapWord *start, *end;
5297   while (!pst->is_task_claimed(/* reference */ nth_task)) {
5298     // We claimed task # nth_task; compute its boundaries.
5299     if (chunk_top == 0) {  // no samples were taken


5482     return (void*) ca;
5483   } else {
5484     return NULL;
5485   }
5486 }
5487 
5488 // Reset all the thread-local PLAB recording arrays
5489 void CMSCollector::reset_survivor_plab_arrays() {
5490   for (uint i = 0; i < ParallelGCThreads; i++) {
5491     _survivor_plab_array[i].reset();
5492   }
5493 }
5494 
5495 // Merge the per-thread plab arrays into the global survivor chunk
5496 // array which will provide the partitioning of the survivor space
5497 // for CMS rescan.
5498 void CMSCollector::merge_survivor_plab_arrays(ContiguousSpace* surv,
5499                                               int no_of_gc_threads) {
5500   assert(_survivor_plab_array  != NULL, "Error");
5501   assert(_survivor_chunk_array != NULL, "Error");
5502   assert(_collectorState == FinalMarking, "Error");

5503   for (int j = 0; j < no_of_gc_threads; j++) {
5504     _cursor[j] = 0;
5505   }
5506   HeapWord* top = surv->top();
5507   size_t i;
5508   for (i = 0; i < _survivor_chunk_capacity; i++) {  // all sca entries
5509     HeapWord* min_val = top;          // Higher than any PLAB address
5510     uint      min_tid = 0;            // position of min_val this round
5511     for (int j = 0; j < no_of_gc_threads; j++) {
5512       ChunkArray* cur_sca = &_survivor_plab_array[j];
5513       if (_cursor[j] == cur_sca->end()) {
5514         continue;
5515       }
5516       assert(_cursor[j] < cur_sca->end(), "ctl pt invariant");
5517       HeapWord* cur_val = cur_sca->nth(_cursor[j]);
5518       assert(surv->used_region().contains(cur_val), "Out of bounds value");
5519       if (cur_val < min_val) {
5520         min_tid = j;
5521         min_val = cur_val;
5522       } else {


6664     _span(span),
6665     _bitMap(bitMap)
6666 {
6667     assert(_ref_processor == NULL, "deliberately left NULL");
6668     assert(_bitMap->covers(_span), "_bitMap/_span mismatch");
6669 }
6670 
6671 void MarkRefsIntoClosure::do_oop(oop obj) {
6672   // if p points into _span, then mark corresponding bit in _markBitMap
6673   assert(obj->is_oop(), "expected an oop");
6674   HeapWord* addr = (HeapWord*)obj;
6675   if (_span.contains(addr)) {
6676     // this should be made more efficient
6677     _bitMap->mark(addr);
6678   }
6679 }
6680 
6681 void MarkRefsIntoClosure::do_oop(oop* p)       { MarkRefsIntoClosure::do_oop_work(p); }
6682 void MarkRefsIntoClosure::do_oop(narrowOop* p) { MarkRefsIntoClosure::do_oop_work(p); }
6683 






















6684 // A variant of the above, used for CMS marking verification.
6685 MarkRefsIntoVerifyClosure::MarkRefsIntoVerifyClosure(
6686   MemRegion span, CMSBitMap* verification_bm, CMSBitMap* cms_bm):
6687     _span(span),
6688     _verification_bm(verification_bm),
6689     _cms_bm(cms_bm)
6690 {
6691     assert(_ref_processor == NULL, "deliberately left NULL");
6692     assert(_verification_bm->covers(_span), "_verification_bm/_span mismatch");
6693 }
6694 
6695 void MarkRefsIntoVerifyClosure::do_oop(oop obj) {
6696   // if p points into _span, then mark corresponding bit in _markBitMap
6697   assert(obj->is_oop(), "expected an oop");
6698   HeapWord* addr = (HeapWord*)obj;
6699   if (_span.contains(addr)) {
6700     _verification_bm->mark(addr);
6701     if (!_cms_bm->isMarked(addr)) {
6702       oop(addr)->print();
6703       gclog_or_tty->print_cr(" (" INTPTR_FORMAT " should have been marked)", addr);


9293                  false /* recordGCEndTime */,
9294                  false /* countCollection */  );
9295       break;
9296 
9297     case CMSCollector::Sweeping:
9298       initialize(true  /* fullGC */ ,
9299                  cause /* cause of the GC */,
9300                  false /* recordGCBeginTime */,
9301                  false /* recordPreGCUsage */,
9302                  true  /* recordPeakUsage */,
9303                  true  /* recordPostGCusage */,
9304                  false /* recordAccumulatedGCTime */,
9305                  true  /* recordGCEndTime */,
9306                  true  /* countCollection */  );
9307       break;
9308 
9309     default:
9310       ShouldNotReachHere();
9311   }
9312 }
9313 


 707   assert(CGC_lock != NULL, "Where's the CGC_lock?");
 708 
 709   // Support for parallelizing young gen rescan
 710   GenCollectedHeap* gch = GenCollectedHeap::heap();
 711   _young_gen = gch->prev_gen(_cmsGen);
 712   if (gch->supports_inline_contig_alloc()) {
 713     _top_addr = gch->top_addr();
 714     _end_addr = gch->end_addr();
 715     assert(_young_gen != NULL, "no _young_gen");
 716     _eden_chunk_index = 0;
 717     _eden_chunk_capacity = (_young_gen->max_capacity()+CMSSamplingGrain)/CMSSamplingGrain;
 718     _eden_chunk_array = NEW_C_HEAP_ARRAY(HeapWord*, _eden_chunk_capacity, mtGC);
 719     if (_eden_chunk_array == NULL) {
 720       _eden_chunk_capacity = 0;
 721       warning("GC/CMS: _eden_chunk_array allocation failure");
 722     }
 723   }
 724   assert(_eden_chunk_array != NULL || _eden_chunk_capacity == 0, "Error");
 725 
 726   // Support for parallelizing survivor space rescan
 727   if ((CMSParallelRemarkEnabled && CMSParallelSurvivorRemarkEnabled) || CMSParallelInitialMarkEnabled) {
 728     const size_t max_plab_samples =
 729       ((DefNewGeneration*)_young_gen)->max_survivor_size()/MinTLABSize;
 730 
 731     _survivor_plab_array  = NEW_C_HEAP_ARRAY(ChunkArray, ParallelGCThreads, mtGC);
 732     _survivor_chunk_array = NEW_C_HEAP_ARRAY(HeapWord*, 2*max_plab_samples, mtGC);
 733     _cursor               = NEW_C_HEAP_ARRAY(size_t, ParallelGCThreads, mtGC);
 734     if (_survivor_plab_array == NULL || _survivor_chunk_array == NULL
 735         || _cursor == NULL) {
 736       warning("Failed to allocate survivor plab/chunk array");
 737       if (_survivor_plab_array  != NULL) {
 738         FREE_C_HEAP_ARRAY(ChunkArray, _survivor_plab_array, mtGC);
 739         _survivor_plab_array = NULL;
 740       }
 741       if (_survivor_chunk_array != NULL) {
 742         FREE_C_HEAP_ARRAY(HeapWord*, _survivor_chunk_array, mtGC);
 743         _survivor_chunk_array = NULL;
 744       }
 745       if (_cursor != NULL) {
 746         FREE_C_HEAP_ARRAY(size_t, _cursor, mtGC);
 747         _cursor = NULL;


3480   _collector->stopTimer();
3481   _wallclock.stop();
3482   if (PrintGCDetails) {
3483     gclog_or_tty->date_stamp(PrintGCDateStamps);
3484     gclog_or_tty->stamp(PrintGCTimeStamps);
3485     gclog_or_tty->print("[%s-concurrent-%s: %3.3f/%3.3f secs]",
3486                  _collector->cmsGen()->short_name(),
3487                  _phase, _collector->timerValue(), _wallclock.seconds());
3488     if (_print_cr) {
3489       gclog_or_tty->print_cr("");
3490     }
3491     if (PrintCMSStatistics != 0) {
3492       gclog_or_tty->print_cr(" (CMS-concurrent-%s yielded %d times)", _phase,
3493                     _collector->yields());
3494     }
3495   }
3496 }
3497 
3498 // CMS work
3499 
3500 // The common parts of CMSParInitialMarkTask and CMSParRemarkTask.
3501 class CMSParMarkTask : public AbstractGangTask {
3502  protected:
3503   CMSCollector*     _collector;
3504   int               _n_workers;
3505   CMSParMarkTask(const char* name, CMSCollector* collector, int n_workers) :
3506       AbstractGangTask(name),
3507       _collector(collector),
3508       _n_workers(n_workers) {}
3509   // Work method in support of parallel rescan ... of young gen spaces
3510   void do_young_space_rescan(uint worker_id, OopsInGenClosure* cl,
3511                              ContiguousSpace* space,
3512                              HeapWord** chunk_array, size_t chunk_top);
3513   void work_on_young_gen_roots(uint worker_id, OopsInGenClosure* cl);
3514 };
3515 
3516 // Parallel initial mark task
3517 class CMSParInitialMarkTask: public CMSParMarkTask {
3518   FlexibleWorkGang* _workers;
3519  public:
3520   CMSParInitialMarkTask(CMSCollector* collector, int n_workers) :
3521       CMSParMarkTask("Scan roots and young gen for initial mark in parallel",
3522                      collector, n_workers) {}
3523   void work(uint worker_id);
3524 };
3525 
3526 // Checkpoint the roots into this generation from outside
3527 // this generation. [Note this initial checkpoint need only
3528 // be approximate -- we'll do a catch up phase subsequently.]
3529 void CMSCollector::checkpointRootsInitial(bool asynch) {
3530   assert(_collectorState == InitialMarking, "Wrong collector state");
3531   check_correct_thread_executing();
3532   TraceCMSMemoryManagerStats tms(_collectorState,GenCollectedHeap::heap()->gc_cause());
3533 
3534   ReferenceProcessor* rp = ref_processor();
3535   SpecializationStats::clear();
3536   assert(_restart_addr == NULL, "Control point invariant");
3537   if (asynch) {
3538     // acquire locks for subsequent manipulations
3539     MutexLockerEx x(bitMapLock(),
3540                     Mutex::_no_safepoint_check_flag);
3541     checkpointRootsInitialWork(asynch);
3542     // enable ("weak") refs discovery
3543     rp->enable_discovery(true /*verify_disabled*/, true /*check_no_refs*/);
3544     _collectorState = Marking;
3545   } else {


3600   GenCollectedHeap* gch = GenCollectedHeap::heap();
3601 
3602   verify_work_stacks_empty();
3603   verify_overflow_empty();
3604 
3605   gch->ensure_parsability(false);  // fill TLABs, but no need to retire them
3606   // Update the saved marks which may affect the root scans.
3607   gch->save_marks();
3608 
3609   // weak reference processing has not started yet.
3610   ref_processor()->set_enqueuing_is_done(false);
3611 
3612   // Need to remember all newly created CLDs,
3613   // so that we can guarantee that the remark finds them.
3614   ClassLoaderDataGraph::remember_new_clds(true);
3615 
3616   // Whenever a CLD is found, it will be claimed before proceeding to mark
3617   // the klasses. The claimed marks need to be cleared before marking starts.
3618   ClassLoaderDataGraph::clear_claimed_marks();
3619 

3620   {
3621     COMPILER2_PRESENT(DerivedPointerTableDeactivate dpt_deact;)
3622     if (CMSParallelInitialMarkEnabled && CollectedHeap::use_parallel_gc_threads()) {
3623       // The parallel version.
3624       FlexibleWorkGang* workers = gch->workers();
3625       assert(workers != NULL, "Need parallel worker threads.");
3626       int n_workers = workers->active_workers();
3627       CMSParInitialMarkTask tsk(this, n_workers);
3628       gch->set_par_threads(n_workers);
3629       initialize_sequential_subtasks_for_young_gen_rescan(n_workers);
3630       if (n_workers > 1) {
3631         GenCollectedHeap::StrongRootsScope srs(gch);
3632         workers->run_task(&tsk);
3633       } else {
3634         GenCollectedHeap::StrongRootsScope srs(gch);
3635         tsk.work(0);
3636       }
3637       gch->set_par_threads(0);
3638     } else {
3639       // The serial version.
3640       CMKlassClosure klass_closure(&notOlder);
3641       gch->rem_set()->prepare_for_younger_refs_iterate(false); // Not parallel.
3642       gch->gen_process_strong_roots(_cmsGen->level(),
3643                                     true,   // younger gens are roots
3644                                     true,   // activate StrongRootsScope
3645                                     false,  // not scavenging
3646                                     SharedHeap::ScanningOption(roots_scanning_options()),
3647                                     &notOlder,
3648                                     true,   // walk all of code cache if (so & SO_CodeCache)
3649                                     NULL,
3650                                     &klass_closure);
3651     }
3652   }
3653 
3654   // Clear mod-union table; it will be dirtied in the prologue of
3655   // CMS generation per each younger generation collection.
3656 
3657   assert(_modUnionTable.isAllClear(),
3658        "Was cleared in most recent final checkpoint phase"
3659        " or no bits are set in the gc_prologue before the start of the next "
3660        "subsequent marking phase.");
3661 
3662   assert(_ct->klass_rem_set()->mod_union_is_clear(), "Must be");
3663 
3664   // Save the end of the used_region of the constituent generations
3665   // to be used to limit the extent of sweep in each generation.
3666   save_sweep_limits();
3667   if (UseAdaptiveSizePolicy) {
3668     size_policy()->checkpoint_roots_initial_end(gch->gc_cause());
3669   }
3670   verify_overflow_empty();
3671 }
3672 


5085   _markStack._hit_limit = 0;
5086   _markStack._failed_double = 0;
5087 
5088   if ((VerifyAfterGC || VerifyDuringGC) &&
5089       GenCollectedHeap::heap()->total_collections() >= VerifyGCStartAt) {
5090     verify_after_remark();
5091   }
5092 
5093   // Change under the freelistLocks.
5094   _collectorState = Sweeping;
5095   // Call isAllClear() under bitMapLock
5096   assert(_modUnionTable.isAllClear(),
5097       "Should be clear by end of the final marking");
5098   assert(_ct->klass_rem_set()->mod_union_is_clear(),
5099       "Should be clear by end of the final marking");
5100   if (UseAdaptiveSizePolicy) {
5101     size_policy()->checkpoint_roots_final_end(gch->gc_cause());
5102   }
5103 }
5104 
5105 void CMSParInitialMarkTask::work(uint worker_id) {
5106   elapsedTimer _timer;
5107   ResourceMark rm;
5108   HandleMark   hm;
5109 
5110   // ---------- scan from roots --------------
5111   _timer.start();
5112   GenCollectedHeap* gch = GenCollectedHeap::heap();
5113   Par_MarkRefsIntoClosure par_mri_cl(_collector->_span, &(_collector->_markBitMap));
5114   CMKlassClosure klass_closure(&par_mri_cl);
5115 
5116   // ---------- young gen roots --------------
5117   {
5118     work_on_young_gen_roots(worker_id, &par_mri_cl);
5119     _timer.stop();
5120     if (PrintCMSStatistics != 0) {
5121       gclog_or_tty->print_cr(
5122         "Finished young gen initial mark scan work in %dth thread: %3.3f sec",
5123         worker_id, _timer.seconds());
5124     }
5125   }
5126 
5127   // ---------- remaining roots --------------
5128   _timer.reset();
5129   _timer.start();
5130   gch->gen_process_strong_roots(_collector->_cmsGen->level(),
5131                                 false,     // yg was scanned above
5132                                 false,     // this is parallel code
5133                                 false,     // not scavenging
5134                                 SharedHeap::ScanningOption(_collector->CMSCollector::roots_scanning_options()),
5135                                 &par_mri_cl,
5136                                 true,   // walk all of code cache if (so & SO_CodeCache)
5137                                 NULL,
5138                                 &klass_closure);
5139   assert(_collector->should_unload_classes()
5140          || (_collector->CMSCollector::roots_scanning_options() & SharedHeap::SO_CodeCache),
5141          "if we didn't scan the code cache, we have to be ready to drop nmethods with expired weak oops");
5142   _timer.stop();
5143   if (PrintCMSStatistics != 0) {
5144     gclog_or_tty->print_cr(
5145       "Finished remaining root initial mark scan work in %dth thread: %3.3f sec",
5146       worker_id, _timer.seconds());
5147   }
5148 }
5149 
5150 // Parallel remark task
5151 class CMSParRemarkTask: public CMSParMarkTask {


5152   CompactibleFreeListSpace* _cms_space;
5153 
5154   // The per-thread work queues, available here for stealing.
5155   OopTaskQueueSet*       _task_queues;
5156   ParallelTaskTerminator _term;
5157 
5158  public:
5159   // A value of 0 passed to n_workers will cause the number of
5160   // workers to be taken from the active workers in the work gang.
5161   CMSParRemarkTask(CMSCollector* collector,
5162                    CompactibleFreeListSpace* cms_space,
5163                    int n_workers, FlexibleWorkGang* workers,
5164                    OopTaskQueueSet* task_queues):
5165     CMSParMarkTask("Rescan roots and grey objects in parallel",
5166                    collector, n_workers),
5167     _cms_space(cms_space),

5168     _task_queues(task_queues),
5169     _term(n_workers, task_queues) { }
5170 
5171   OopTaskQueueSet* task_queues() { return _task_queues; }
5172 
5173   OopTaskQueue* work_queue(int i) { return task_queues()->queue(i); }
5174 
5175   ParallelTaskTerminator* terminator() { return &_term; }
5176   int n_workers() { return _n_workers; }
5177 
5178   void work(uint worker_id);
5179 
5180  private:





5181   // ... of  dirty cards in old space
5182   void do_dirty_card_rescan_tasks(CompactibleFreeListSpace* sp, int i,
5183                                   Par_MarkRefsIntoAndScanClosure* cl);
5184 
5185   // ... work stealing for the above
5186   void do_work_steal(int i, Par_MarkRefsIntoAndScanClosure* cl, int* seed);
5187 };
5188 
5189 class RemarkKlassClosure : public KlassClosure {
5190   CMKlassClosure _cm_klass_closure;
5191  public:
5192   RemarkKlassClosure(OopClosure* oop_closure) : _cm_klass_closure(oop_closure) {}
5193   void do_klass(Klass* k) {
5194     // Check if we have modified any oops in the Klass during the concurrent marking.
5195     if (k->has_accumulated_modified_oops()) {
5196       k->clear_accumulated_modified_oops();
5197 
5198       // We could have transfered the current modified marks to the accumulated marks,
5199       // like we do with the Card Table to Mod Union Table. But it's not really necessary.
5200     } else if (k->has_modified_oops()) {
5201       // Don't clear anything, this info is needed by the next young collection.
5202     } else {
5203       // No modified oops in the Klass.
5204       return;
5205     }
5206 
5207     // The klass has modified fields, need to scan the klass.
5208     _cm_klass_closure.do_klass(k);
5209   }
5210 };
5211 
5212 void CMSParMarkTask::work_on_young_gen_roots(uint worker_id, OopsInGenClosure* cl) {
5213   DefNewGeneration* dng = _collector->_young_gen->as_DefNewGeneration();
5214   EdenSpace* eden_space = dng->eden();
5215   ContiguousSpace* from_space = dng->from();
5216   ContiguousSpace* to_space   = dng->to();
5217 
5218   HeapWord** eca = _collector->_eden_chunk_array;
5219   size_t     ect = _collector->_eden_chunk_index;
5220   HeapWord** sca = _collector->_survivor_chunk_array;
5221   size_t     sct = _collector->_survivor_chunk_index;
5222 
5223   assert(ect <= _collector->_eden_chunk_capacity, "out of bounds");
5224   assert(sct <= _collector->_survivor_chunk_capacity, "out of bounds");
5225 
5226   do_young_space_rescan(worker_id, cl, to_space, NULL, 0);
5227   do_young_space_rescan(worker_id, cl, from_space, sca, sct);
5228   do_young_space_rescan(worker_id, cl, eden_space, eca, ect);
5229 }
5230 
5231 // work_queue(i) is passed to the closure
5232 // Par_MarkRefsIntoAndScanClosure.  The "i" parameter
5233 // also is passed to do_dirty_card_rescan_tasks() and to
5234 // do_work_steal() to select the i-th task_queue.
5235 
5236 void CMSParRemarkTask::work(uint worker_id) {
5237   elapsedTimer _timer;
5238   ResourceMark rm;
5239   HandleMark   hm;
5240 
5241   // ---------- rescan from roots --------------
5242   _timer.start();
5243   GenCollectedHeap* gch = GenCollectedHeap::heap();
5244   Par_MarkRefsIntoAndScanClosure par_mrias_cl(_collector,
5245     _collector->_span, _collector->ref_processor(),
5246     &(_collector->_markBitMap),
5247     work_queue(worker_id));
5248 
5249   // Rescan young gen roots first since these are likely
5250   // coarsely partitioned and may, on that account, constitute
5251   // the critical path; thus, it's best to start off that
5252   // work first.
5253   // ---------- young gen roots --------------
5254   {
5255     work_on_young_gen_roots(worker_id, &par_mrias_cl);
















5256     _timer.stop();
5257     if (PrintCMSStatistics != 0) {
5258       gclog_or_tty->print_cr(
5259         "Finished young gen rescan work in %dth thread: %3.3f sec",
5260         worker_id, _timer.seconds());
5261     }
5262   }
5263 
5264   // ---------- remaining roots --------------
5265   _timer.reset();
5266   _timer.start();
5267   gch->gen_process_strong_roots(_collector->_cmsGen->level(),
5268                                 false,     // yg was scanned above
5269                                 false,     // this is parallel code
5270                                 false,     // not scavenging
5271                                 SharedHeap::ScanningOption(_collector->CMSCollector::roots_scanning_options()),
5272                                 &par_mrias_cl,
5273                                 true,   // walk all of code cache if (so & SO_CodeCache)
5274                                 NULL,
5275                                 NULL);     // The dirty klasses will be handled below


5343     gclog_or_tty->print_cr(
5344       "Finished dirty card rescan work in %dth thread: %3.3f sec",
5345       worker_id, _timer.seconds());
5346   }
5347 
5348   // ---------- steal work from other threads ...
5349   // ---------- ... and drain overflow list.
5350   _timer.reset();
5351   _timer.start();
5352   do_work_steal(worker_id, &par_mrias_cl, _collector->hash_seed(worker_id));
5353   _timer.stop();
5354   if (PrintCMSStatistics != 0) {
5355     gclog_or_tty->print_cr(
5356       "Finished work stealing in %dth thread: %3.3f sec",
5357       worker_id, _timer.seconds());
5358   }
5359 }
5360 
5361 // Note that parameter "i" is not used.
5362 void
5363 CMSParMarkTask::do_young_space_rescan(uint worker_id,
5364   OopsInGenClosure* cl, ContiguousSpace* space,
5365   HeapWord** chunk_array, size_t chunk_top) {
5366   // Until all tasks completed:
5367   // . claim an unclaimed task
5368   // . compute region boundaries corresponding to task claimed
5369   //   using chunk_array
5370   // . par_oop_iterate(cl) over that region
5371 
5372   ResourceMark rm;
5373   HandleMark   hm;
5374 
5375   SequentialSubTasksDone* pst = space->par_seq_tasks();
5376   assert(pst->valid(), "Uninitialized use?");
5377 
5378   uint nth_task = 0;
5379   uint n_tasks  = pst->n_tasks();
5380 
5381   HeapWord *start, *end;
5382   while (!pst->is_task_claimed(/* reference */ nth_task)) {
5383     // We claimed task # nth_task; compute its boundaries.
5384     if (chunk_top == 0) {  // no samples were taken


5567     return (void*) ca;
5568   } else {
5569     return NULL;
5570   }
5571 }
5572 
5573 // Reset all the thread-local PLAB recording arrays
5574 void CMSCollector::reset_survivor_plab_arrays() {
5575   for (uint i = 0; i < ParallelGCThreads; i++) {
5576     _survivor_plab_array[i].reset();
5577   }
5578 }
5579 
5580 // Merge the per-thread plab arrays into the global survivor chunk
5581 // array which will provide the partitioning of the survivor space
5582 // for CMS rescan.
5583 void CMSCollector::merge_survivor_plab_arrays(ContiguousSpace* surv,
5584                                               int no_of_gc_threads) {
5585   assert(_survivor_plab_array  != NULL, "Error");
5586   assert(_survivor_chunk_array != NULL, "Error");
5587   assert(_collectorState == FinalMarking ||
5588          (CMSParallelInitialMarkEnabled && _collectorState == InitialMarking), "Error");
5589   for (int j = 0; j < no_of_gc_threads; j++) {
5590     _cursor[j] = 0;
5591   }
5592   HeapWord* top = surv->top();
5593   size_t i;
5594   for (i = 0; i < _survivor_chunk_capacity; i++) {  // all sca entries
5595     HeapWord* min_val = top;          // Higher than any PLAB address
5596     uint      min_tid = 0;            // position of min_val this round
5597     for (int j = 0; j < no_of_gc_threads; j++) {
5598       ChunkArray* cur_sca = &_survivor_plab_array[j];
5599       if (_cursor[j] == cur_sca->end()) {
5600         continue;
5601       }
5602       assert(_cursor[j] < cur_sca->end(), "ctl pt invariant");
5603       HeapWord* cur_val = cur_sca->nth(_cursor[j]);
5604       assert(surv->used_region().contains(cur_val), "Out of bounds value");
5605       if (cur_val < min_val) {
5606         min_tid = j;
5607         min_val = cur_val;
5608       } else {


6750     _span(span),
6751     _bitMap(bitMap)
6752 {
6753     assert(_ref_processor == NULL, "deliberately left NULL");
6754     assert(_bitMap->covers(_span), "_bitMap/_span mismatch");
6755 }
6756 
6757 void MarkRefsIntoClosure::do_oop(oop obj) {
6758   // if p points into _span, then mark corresponding bit in _markBitMap
6759   assert(obj->is_oop(), "expected an oop");
6760   HeapWord* addr = (HeapWord*)obj;
6761   if (_span.contains(addr)) {
6762     // this should be made more efficient
6763     _bitMap->mark(addr);
6764   }
6765 }
6766 
6767 void MarkRefsIntoClosure::do_oop(oop* p)       { MarkRefsIntoClosure::do_oop_work(p); }
6768 void MarkRefsIntoClosure::do_oop(narrowOop* p) { MarkRefsIntoClosure::do_oop_work(p); }
6769 
6770 Par_MarkRefsIntoClosure::Par_MarkRefsIntoClosure(
6771   MemRegion span, CMSBitMap* bitMap):
6772     _span(span),
6773     _bitMap(bitMap)
6774 {
6775     assert(_ref_processor == NULL, "deliberately left NULL");
6776     assert(_bitMap->covers(_span), "_bitMap/_span mismatch");
6777 }
6778 
6779 void Par_MarkRefsIntoClosure::do_oop(oop obj) {
6780   // if p points into _span, then mark corresponding bit in _markBitMap
6781   assert(obj->is_oop(), "expected an oop");
6782   HeapWord* addr = (HeapWord*)obj;
6783   if (_span.contains(addr)) {
6784     // this should be made more efficient
6785     _bitMap->par_mark(addr);
6786   }
6787 }
6788 
6789 void Par_MarkRefsIntoClosure::do_oop(oop* p)       { Par_MarkRefsIntoClosure::do_oop_work(p); }
6790 void Par_MarkRefsIntoClosure::do_oop(narrowOop* p) { Par_MarkRefsIntoClosure::do_oop_work(p); }
6791 
6792 // A variant of the above, used for CMS marking verification.
6793 MarkRefsIntoVerifyClosure::MarkRefsIntoVerifyClosure(
6794   MemRegion span, CMSBitMap* verification_bm, CMSBitMap* cms_bm):
6795     _span(span),
6796     _verification_bm(verification_bm),
6797     _cms_bm(cms_bm)
6798 {
6799     assert(_ref_processor == NULL, "deliberately left NULL");
6800     assert(_verification_bm->covers(_span), "_verification_bm/_span mismatch");
6801 }
6802 
6803 void MarkRefsIntoVerifyClosure::do_oop(oop obj) {
6804   // if p points into _span, then mark corresponding bit in _markBitMap
6805   assert(obj->is_oop(), "expected an oop");
6806   HeapWord* addr = (HeapWord*)obj;
6807   if (_span.contains(addr)) {
6808     _verification_bm->mark(addr);
6809     if (!_cms_bm->isMarked(addr)) {
6810       oop(addr)->print();
6811       gclog_or_tty->print_cr(" (" INTPTR_FORMAT " should have been marked)", addr);


9401                  false /* recordGCEndTime */,
9402                  false /* countCollection */  );
9403       break;
9404 
9405     case CMSCollector::Sweeping:
9406       initialize(true  /* fullGC */ ,
9407                  cause /* cause of the GC */,
9408                  false /* recordGCBeginTime */,
9409                  false /* recordPreGCUsage */,
9410                  true  /* recordPeakUsage */,
9411                  true  /* recordPostGCusage */,
9412                  false /* recordAccumulatedGCTime */,
9413                  true  /* recordGCEndTime */,
9414                  true  /* countCollection */  );
9415       break;
9416 
9417     default:
9418       ShouldNotReachHere();
9419   }
9420 }