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

Print this page




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


3552   _collector->stopTimer();
3553   _wallclock.stop();
3554   if (PrintGCDetails) {
3555     gclog_or_tty->date_stamp(PrintGCDateStamps);
3556     gclog_or_tty->stamp(PrintGCTimeStamps);
3557     gclog_or_tty->print("[%s-concurrent-%s: %3.3f/%3.3f secs]",
3558                  _collector->cmsGen()->short_name(),
3559                  _phase, _collector->timerValue(), _wallclock.seconds());
3560     if (_print_cr) {
3561       gclog_or_tty->print_cr("");
3562     }
3563     if (PrintCMSStatistics != 0) {
3564       gclog_or_tty->print_cr(" (CMS-concurrent-%s yielded %d times)", _phase,
3565                     _collector->yields());
3566     }
3567   }
3568 }
3569 
3570 // CMS work
3571 

























3572 // Checkpoint the roots into this generation from outside
3573 // this generation. [Note this initial checkpoint need only
3574 // be approximate -- we'll do a catch up phase subsequently.]
3575 void CMSCollector::checkpointRootsInitial(bool asynch) {
3576   assert(_collectorState == InitialMarking, "Wrong collector state");
3577   check_correct_thread_executing();
3578   TraceCMSMemoryManagerStats tms(_collectorState,GenCollectedHeap::heap()->gc_cause());
3579 
3580   save_heap_summary();
3581   report_heap_summary(GCWhen::BeforeGC);
3582 
3583   ReferenceProcessor* rp = ref_processor();
3584   SpecializationStats::clear();
3585   assert(_restart_addr == NULL, "Control point invariant");
3586   if (asynch) {
3587     // acquire locks for subsequent manipulations
3588     MutexLockerEx x(bitMapLock(),
3589                     Mutex::_no_safepoint_check_flag);
3590     checkpointRootsInitialWork(asynch);
3591     // enable ("weak") refs discovery


3649   GenCollectedHeap* gch = GenCollectedHeap::heap();
3650 
3651   verify_work_stacks_empty();
3652   verify_overflow_empty();
3653 
3654   gch->ensure_parsability(false);  // fill TLABs, but no need to retire them
3655   // Update the saved marks which may affect the root scans.
3656   gch->save_marks();
3657 
3658   // weak reference processing has not started yet.
3659   ref_processor()->set_enqueuing_is_done(false);
3660 
3661   // Need to remember all newly created CLDs,
3662   // so that we can guarantee that the remark finds them.
3663   ClassLoaderDataGraph::remember_new_clds(true);
3664 
3665   // Whenever a CLD is found, it will be claimed before proceeding to mark
3666   // the klasses. The claimed marks need to be cleared before marking starts.
3667   ClassLoaderDataGraph::clear_claimed_marks();
3668 
3669   CMKlassClosure klass_closure(&notOlder);
3670   {
3671     COMPILER2_PRESENT(DerivedPointerTableDeactivate dpt_deact;)



















3672     gch->rem_set()->prepare_for_younger_refs_iterate(false); // Not parallel.
3673     gch->gen_process_strong_roots(_cmsGen->level(),
3674                                   true,   // younger gens are roots
3675                                   true,   // activate StrongRootsScope
3676                                   false,  // not scavenging
3677                                   SharedHeap::ScanningOption(roots_scanning_options()),
3678                                   &notOlder,
3679                                   true,   // walk all of code cache if (so & SO_CodeCache)
3680                                   NULL,
3681                                   &klass_closure);
3682   }

3683 
3684   // Clear mod-union table; it will be dirtied in the prologue of
3685   // CMS generation per each younger generation collection.
3686 
3687   assert(_modUnionTable.isAllClear(),
3688        "Was cleared in most recent final checkpoint phase"
3689        " or no bits are set in the gc_prologue before the start of the next "
3690        "subsequent marking phase.");
3691 
3692   assert(_ct->klass_rem_set()->mod_union_is_clear(), "Must be");
3693 
3694   // Save the end of the used_region of the constituent generations
3695   // to be used to limit the extent of sweep in each generation.
3696   save_sweep_limits();
3697   if (UseAdaptiveSizePolicy) {
3698     size_policy()->checkpoint_roots_initial_end(gch->gc_cause());
3699   }
3700   verify_overflow_empty();
3701 }
3702 


5119 
5120   if ((VerifyAfterGC || VerifyDuringGC) &&
5121       GenCollectedHeap::heap()->total_collections() >= VerifyGCStartAt) {
5122     verify_after_remark();
5123   }
5124 
5125   _gc_tracer_cm->report_object_count_after_gc(&_is_alive_closure);
5126 
5127   // Change under the freelistLocks.
5128   _collectorState = Sweeping;
5129   // Call isAllClear() under bitMapLock
5130   assert(_modUnionTable.isAllClear(),
5131       "Should be clear by end of the final marking");
5132   assert(_ct->klass_rem_set()->mod_union_is_clear(),
5133       "Should be clear by end of the final marking");
5134   if (UseAdaptiveSizePolicy) {
5135     size_policy()->checkpoint_roots_final_end(gch->gc_cause());
5136   }
5137 }
5138 













































5139 // Parallel remark task
5140 class CMSParRemarkTask: public AbstractGangTask {
5141   CMSCollector* _collector;
5142   int           _n_workers;
5143   CompactibleFreeListSpace* _cms_space;
5144 
5145   // The per-thread work queues, available here for stealing.
5146   OopTaskQueueSet*       _task_queues;
5147   ParallelTaskTerminator _term;
5148 
5149  public:
5150   // A value of 0 passed to n_workers will cause the number of
5151   // workers to be taken from the active workers in the work gang.
5152   CMSParRemarkTask(CMSCollector* collector,
5153                    CompactibleFreeListSpace* cms_space,
5154                    int n_workers, FlexibleWorkGang* workers,
5155                    OopTaskQueueSet* task_queues):
5156     AbstractGangTask("Rescan roots and grey objects in parallel"),
5157     _collector(collector),
5158     _cms_space(cms_space),
5159     _n_workers(n_workers),
5160     _task_queues(task_queues),
5161     _term(n_workers, task_queues) { }
5162 
5163   OopTaskQueueSet* task_queues() { return _task_queues; }
5164 
5165   OopTaskQueue* work_queue(int i) { return task_queues()->queue(i); }
5166 
5167   ParallelTaskTerminator* terminator() { return &_term; }
5168   int n_workers() { return _n_workers; }
5169 
5170   void work(uint worker_id);
5171 
5172  private:
5173   // Work method in support of parallel rescan ... of young gen spaces
5174   void do_young_space_rescan(int i, Par_MarkRefsIntoAndScanClosure* cl,
5175                              ContiguousSpace* space,
5176                              HeapWord** chunk_array, size_t chunk_top);
5177 
5178   // ... of  dirty cards in old space
5179   void do_dirty_card_rescan_tasks(CompactibleFreeListSpace* sp, int i,
5180                                   Par_MarkRefsIntoAndScanClosure* cl);
5181 
5182   // ... work stealing for the above
5183   void do_work_steal(int i, Par_MarkRefsIntoAndScanClosure* cl, int* seed);
5184 };
5185 
5186 class RemarkKlassClosure : public KlassClosure {
5187   CMKlassClosure _cm_klass_closure;
5188  public:
5189   RemarkKlassClosure(OopClosure* oop_closure) : _cm_klass_closure(oop_closure) {}
5190   void do_klass(Klass* k) {
5191     // Check if we have modified any oops in the Klass during the concurrent marking.
5192     if (k->has_accumulated_modified_oops()) {
5193       k->clear_accumulated_modified_oops();
5194 
5195       // We could have transfered the current modified marks to the accumulated marks,
5196       // like we do with the Card Table to Mod Union Table. But it's not really necessary.
5197     } else if (k->has_modified_oops()) {
5198       // Don't clear anything, this info is needed by the next young collection.
5199     } else {
5200       // No modified oops in the Klass.
5201       return;
5202     }
5203 
5204     // The klass has modified fields, need to scan the klass.
5205     _cm_klass_closure.do_klass(k);
5206   }
5207 };
5208 



















5209 // work_queue(i) is passed to the closure
5210 // Par_MarkRefsIntoAndScanClosure.  The "i" parameter
5211 // also is passed to do_dirty_card_rescan_tasks() and to
5212 // do_work_steal() to select the i-th task_queue.
5213 
5214 void CMSParRemarkTask::work(uint worker_id) {
5215   elapsedTimer _timer;
5216   ResourceMark rm;
5217   HandleMark   hm;
5218 
5219   // ---------- rescan from roots --------------
5220   _timer.start();
5221   GenCollectedHeap* gch = GenCollectedHeap::heap();
5222   Par_MarkRefsIntoAndScanClosure par_mrias_cl(_collector,
5223     _collector->_span, _collector->ref_processor(),
5224     &(_collector->_markBitMap),
5225     work_queue(worker_id));
5226 
5227   // Rescan young gen roots first since these are likely
5228   // coarsely partitioned and may, on that account, constitute
5229   // the critical path; thus, it's best to start off that
5230   // work first.
5231   // ---------- young gen roots --------------
5232   {
5233     DefNewGeneration* dng = _collector->_young_gen->as_DefNewGeneration();
5234     EdenSpace* eden_space = dng->eden();
5235     ContiguousSpace* from_space = dng->from();
5236     ContiguousSpace* to_space   = dng->to();
5237 
5238     HeapWord** eca = _collector->_eden_chunk_array;
5239     size_t     ect = _collector->_eden_chunk_index;
5240     HeapWord** sca = _collector->_survivor_chunk_array;
5241     size_t     sct = _collector->_survivor_chunk_index;
5242 
5243     assert(ect <= _collector->_eden_chunk_capacity, "out of bounds");
5244     assert(sct <= _collector->_survivor_chunk_capacity, "out of bounds");
5245 
5246     do_young_space_rescan(worker_id, &par_mrias_cl, to_space, NULL, 0);
5247     do_young_space_rescan(worker_id, &par_mrias_cl, from_space, sca, sct);
5248     do_young_space_rescan(worker_id, &par_mrias_cl, eden_space, eca, ect);
5249 
5250     _timer.stop();
5251     if (PrintCMSStatistics != 0) {
5252       gclog_or_tty->print_cr(
5253         "Finished young gen rescan work in %dth thread: %3.3f sec",
5254         worker_id, _timer.seconds());
5255     }
5256   }
5257 
5258   // ---------- remaining roots --------------
5259   _timer.reset();
5260   _timer.start();
5261   gch->gen_process_strong_roots(_collector->_cmsGen->level(),
5262                                 false,     // yg was scanned above
5263                                 false,     // this is parallel code
5264                                 false,     // not scavenging
5265                                 SharedHeap::ScanningOption(_collector->CMSCollector::roots_scanning_options()),
5266                                 &par_mrias_cl,
5267                                 true,   // walk all of code cache if (so & SO_CodeCache)
5268                                 NULL,
5269                                 NULL);     // The dirty klasses will be handled below


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


5556       (CMSPLABRecordAlways ||
5557        (_collectorState > Marking && _collectorState < FinalMarking))) {
5558     assert(thr_num < (int)ParallelGCThreads, "thr_num is out of bounds");
5559     ChunkArray* ca = &_survivor_plab_array[thr_num];
5560     ca->reset();   // clear it so that fresh data is recorded
5561     return (void*) ca;
5562   } else {
5563     return NULL;
5564   }
5565 }
5566 
5567 // Reset all the thread-local PLAB recording arrays
5568 void CMSCollector::reset_survivor_plab_arrays() {
5569   for (uint i = 0; i < ParallelGCThreads; i++) {
5570     _survivor_plab_array[i].reset();
5571   }
5572 }
5573 
5574 // Merge the per-thread plab arrays into the global survivor chunk
5575 // array which will provide the partitioning of the survivor space
5576 // for CMS rescan.
5577 void CMSCollector::merge_survivor_plab_arrays(ContiguousSpace* surv,
5578                                               int no_of_gc_threads) {
5579   assert(_survivor_plab_array  != NULL, "Error");
5580   assert(_survivor_chunk_array != NULL, "Error");
5581   assert(_collectorState == FinalMarking, "Error");

5582   for (int j = 0; j < no_of_gc_threads; j++) {
5583     _cursor[j] = 0;
5584   }
5585   HeapWord* top = surv->top();
5586   size_t i;
5587   for (i = 0; i < _survivor_chunk_capacity; i++) {  // all sca entries
5588     HeapWord* min_val = top;          // Higher than any PLAB address
5589     uint      min_tid = 0;            // position of min_val this round
5590     for (int j = 0; j < no_of_gc_threads; j++) {
5591       ChunkArray* cur_sca = &_survivor_plab_array[j];
5592       if (_cursor[j] == cur_sca->end()) {
5593         continue;
5594       }
5595       assert(_cursor[j] < cur_sca->end(), "ctl pt invariant");
5596       HeapWord* cur_val = cur_sca->nth(_cursor[j]);
5597       assert(surv->used_region().contains(cur_val), "Out of bounds value");
5598       if (cur_val < min_val) {
5599         min_tid = j;
5600         min_val = cur_val;
5601       } else {


5624     for (int j = 0; j < no_of_gc_threads; j++) {
5625       assert(_cursor[j] == _survivor_plab_array[j].end(), "Ctl pt invariant");
5626       total += _cursor[j];
5627     }
5628     assert(total == _survivor_chunk_index, "Ctl Pt Invariant");
5629     // Check that the merged array is in sorted order
5630     if (total > 0) {
5631       for (size_t i = 0; i < total - 1; i++) {
5632         if (PrintCMSStatistics > 0) {
5633           gclog_or_tty->print(" (chunk" SIZE_FORMAT ":" INTPTR_FORMAT ") ",
5634                               i, _survivor_chunk_array[i]);
5635         }
5636         assert(_survivor_chunk_array[i] < _survivor_chunk_array[i+1],
5637                "Not sorted");
5638       }
5639     }
5640   #endif // ASSERT
5641 }
5642 
5643 // Set up the space's par_seq_tasks structure for work claiming
5644 // for parallel rescan of young gen.
5645 // See ParRescanTask where this is currently used.
5646 void
5647 CMSCollector::
5648 initialize_sequential_subtasks_for_young_gen_rescan(int n_threads) {
5649   assert(n_threads > 0, "Unexpected n_threads argument");
5650   DefNewGeneration* dng = (DefNewGeneration*)_young_gen;
5651 
5652   // Eden space
5653   {
5654     SequentialSubTasksDone* pst = dng->eden()->par_seq_tasks();
5655     assert(!pst->valid(), "Clobbering existing data?");
5656     // Each valid entry in [0, _eden_chunk_index) represents a task.
5657     size_t n_tasks = _eden_chunk_index + 1;
5658     assert(n_tasks == 1 || _eden_chunk_array != NULL, "Error");
5659     // Sets the condition for completion of the subtask (how many threads
5660     // need to finish in order to be done).
5661     pst->set_n_threads(n_threads);
5662     pst->set_n_tasks((int)n_tasks);
5663   }
5664 


6751     _span(span),
6752     _bitMap(bitMap)
6753 {
6754     assert(_ref_processor == NULL, "deliberately left NULL");
6755     assert(_bitMap->covers(_span), "_bitMap/_span mismatch");
6756 }
6757 
6758 void MarkRefsIntoClosure::do_oop(oop obj) {
6759   // if p points into _span, then mark corresponding bit in _markBitMap
6760   assert(obj->is_oop(), "expected an oop");
6761   HeapWord* addr = (HeapWord*)obj;
6762   if (_span.contains(addr)) {
6763     // this should be made more efficient
6764     _bitMap->mark(addr);
6765   }
6766 }
6767 
6768 void MarkRefsIntoClosure::do_oop(oop* p)       { MarkRefsIntoClosure::do_oop_work(p); }
6769 void MarkRefsIntoClosure::do_oop(narrowOop* p) { MarkRefsIntoClosure::do_oop_work(p); }
6770 






















6771 // A variant of the above, used for CMS marking verification.
6772 MarkRefsIntoVerifyClosure::MarkRefsIntoVerifyClosure(
6773   MemRegion span, CMSBitMap* verification_bm, CMSBitMap* cms_bm):
6774     _span(span),
6775     _verification_bm(verification_bm),
6776     _cms_bm(cms_bm)
6777 {
6778     assert(_ref_processor == NULL, "deliberately left NULL");
6779     assert(_verification_bm->covers(_span), "_verification_bm/_span mismatch");
6780 }
6781 
6782 void MarkRefsIntoVerifyClosure::do_oop(oop obj) {
6783   // if p points into _span, then mark corresponding bit in _markBitMap
6784   assert(obj->is_oop(), "expected an oop");
6785   HeapWord* addr = (HeapWord*)obj;
6786   if (_span.contains(addr)) {
6787     _verification_bm->mark(addr);
6788     if (!_cms_bm->isMarked(addr)) {
6789       oop(addr)->print();
6790       gclog_or_tty->print_cr(" (" INTPTR_FORMAT " should have been marked)", addr);


9380                  false /* recordGCEndTime */,
9381                  false /* countCollection */  );
9382       break;
9383 
9384     case CMSCollector::Sweeping:
9385       initialize(true  /* fullGC */ ,
9386                  cause /* cause of the GC */,
9387                  false /* recordGCBeginTime */,
9388                  false /* recordPreGCUsage */,
9389                  true  /* recordPeakUsage */,
9390                  true  /* recordPostGCusage */,
9391                  false /* recordAccumulatedGCTime */,
9392                  true  /* recordGCEndTime */,
9393                  true  /* countCollection */  );
9394       break;
9395 
9396     default:
9397       ShouldNotReachHere();
9398   }
9399 }
9400 


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


3552   _collector->stopTimer();
3553   _wallclock.stop();
3554   if (PrintGCDetails) {
3555     gclog_or_tty->date_stamp(PrintGCDateStamps);
3556     gclog_or_tty->stamp(PrintGCTimeStamps);
3557     gclog_or_tty->print("[%s-concurrent-%s: %3.3f/%3.3f secs]",
3558                  _collector->cmsGen()->short_name(),
3559                  _phase, _collector->timerValue(), _wallclock.seconds());
3560     if (_print_cr) {
3561       gclog_or_tty->print_cr("");
3562     }
3563     if (PrintCMSStatistics != 0) {
3564       gclog_or_tty->print_cr(" (CMS-concurrent-%s yielded %d times)", _phase,
3565                     _collector->yields());
3566     }
3567   }
3568 }
3569 
3570 // CMS work
3571 
3572 // The common parts of CMSParInitialMarkTask and CMSParRemarkTask.
3573 class CMSParMarkTask : public AbstractGangTask {
3574  protected:
3575   CMSCollector*     _collector;
3576   int               _n_workers;
3577   CMSParMarkTask(const char* name, CMSCollector* collector, int n_workers) :
3578       AbstractGangTask(name),
3579       _collector(collector),
3580       _n_workers(n_workers) {}
3581   // Work method in support of parallel rescan ... of young gen spaces
3582   void do_young_space_rescan(uint worker_id, OopsInGenClosure* cl,
3583                              ContiguousSpace* space,
3584                              HeapWord** chunk_array, size_t chunk_top);
3585   void work_on_young_gen_roots(uint worker_id, OopsInGenClosure* cl);
3586 };
3587 
3588 // Parallel initial mark task
3589 class CMSParInitialMarkTask: public CMSParMarkTask {
3590  public:
3591   CMSParInitialMarkTask(CMSCollector* collector, int n_workers) :
3592       CMSParMarkTask("Scan roots and young gen for initial mark in parallel",
3593                      collector, n_workers) {}
3594   void work(uint worker_id);
3595 };
3596 
3597 // Checkpoint the roots into this generation from outside
3598 // this generation. [Note this initial checkpoint need only
3599 // be approximate -- we'll do a catch up phase subsequently.]
3600 void CMSCollector::checkpointRootsInitial(bool asynch) {
3601   assert(_collectorState == InitialMarking, "Wrong collector state");
3602   check_correct_thread_executing();
3603   TraceCMSMemoryManagerStats tms(_collectorState,GenCollectedHeap::heap()->gc_cause());
3604 
3605   save_heap_summary();
3606   report_heap_summary(GCWhen::BeforeGC);
3607 
3608   ReferenceProcessor* rp = ref_processor();
3609   SpecializationStats::clear();
3610   assert(_restart_addr == NULL, "Control point invariant");
3611   if (asynch) {
3612     // acquire locks for subsequent manipulations
3613     MutexLockerEx x(bitMapLock(),
3614                     Mutex::_no_safepoint_check_flag);
3615     checkpointRootsInitialWork(asynch);
3616     // enable ("weak") refs discovery


3674   GenCollectedHeap* gch = GenCollectedHeap::heap();
3675 
3676   verify_work_stacks_empty();
3677   verify_overflow_empty();
3678 
3679   gch->ensure_parsability(false);  // fill TLABs, but no need to retire them
3680   // Update the saved marks which may affect the root scans.
3681   gch->save_marks();
3682 
3683   // weak reference processing has not started yet.
3684   ref_processor()->set_enqueuing_is_done(false);
3685 
3686   // Need to remember all newly created CLDs,
3687   // so that we can guarantee that the remark finds them.
3688   ClassLoaderDataGraph::remember_new_clds(true);
3689 
3690   // Whenever a CLD is found, it will be claimed before proceeding to mark
3691   // the klasses. The claimed marks need to be cleared before marking starts.
3692   ClassLoaderDataGraph::clear_claimed_marks();
3693 

3694   {
3695     COMPILER2_PRESENT(DerivedPointerTableDeactivate dpt_deact;)
3696     if (CMSParallelInitialMarkEnabled && CollectedHeap::use_parallel_gc_threads()) {
3697       // The parallel version.
3698       FlexibleWorkGang* workers = gch->workers();
3699       assert(workers != NULL, "Need parallel worker threads.");
3700       int n_workers = workers->active_workers();
3701       CMSParInitialMarkTask tsk(this, n_workers);
3702       gch->set_par_threads(n_workers);
3703       initialize_sequential_subtasks_for_young_gen_rescan(n_workers);
3704       if (n_workers > 1) {
3705         GenCollectedHeap::StrongRootsScope srs(gch);
3706         workers->run_task(&tsk);
3707       } else {
3708         GenCollectedHeap::StrongRootsScope srs(gch);
3709         tsk.work(0);
3710       }
3711       gch->set_par_threads(0);
3712     } else {
3713       // The serial version.
3714       CMKlassClosure klass_closure(&notOlder);
3715       gch->rem_set()->prepare_for_younger_refs_iterate(false); // Not parallel.
3716       gch->gen_process_strong_roots(_cmsGen->level(),
3717                                     true,   // younger gens are roots
3718                                     true,   // activate StrongRootsScope
3719                                     false,  // not scavenging
3720                                     SharedHeap::ScanningOption(roots_scanning_options()),
3721                                     &notOlder,
3722                                     true,   // walk all of code cache if (so & SO_CodeCache)
3723                                     NULL,
3724                                     &klass_closure);
3725     }
3726   }
3727 
3728   // Clear mod-union table; it will be dirtied in the prologue of
3729   // CMS generation per each younger generation collection.
3730 
3731   assert(_modUnionTable.isAllClear(),
3732        "Was cleared in most recent final checkpoint phase"
3733        " or no bits are set in the gc_prologue before the start of the next "
3734        "subsequent marking phase.");
3735 
3736   assert(_ct->klass_rem_set()->mod_union_is_clear(), "Must be");
3737 
3738   // Save the end of the used_region of the constituent generations
3739   // to be used to limit the extent of sweep in each generation.
3740   save_sweep_limits();
3741   if (UseAdaptiveSizePolicy) {
3742     size_policy()->checkpoint_roots_initial_end(gch->gc_cause());
3743   }
3744   verify_overflow_empty();
3745 }
3746 


5163 
5164   if ((VerifyAfterGC || VerifyDuringGC) &&
5165       GenCollectedHeap::heap()->total_collections() >= VerifyGCStartAt) {
5166     verify_after_remark();
5167   }
5168 
5169   _gc_tracer_cm->report_object_count_after_gc(&_is_alive_closure);
5170 
5171   // Change under the freelistLocks.
5172   _collectorState = Sweeping;
5173   // Call isAllClear() under bitMapLock
5174   assert(_modUnionTable.isAllClear(),
5175       "Should be clear by end of the final marking");
5176   assert(_ct->klass_rem_set()->mod_union_is_clear(),
5177       "Should be clear by end of the final marking");
5178   if (UseAdaptiveSizePolicy) {
5179     size_policy()->checkpoint_roots_final_end(gch->gc_cause());
5180   }
5181 }
5182 
5183 void CMSParInitialMarkTask::work(uint worker_id) {
5184   elapsedTimer _timer;
5185   ResourceMark rm;
5186   HandleMark   hm;
5187 
5188   // ---------- scan from roots --------------
5189   _timer.start();
5190   GenCollectedHeap* gch = GenCollectedHeap::heap();
5191   Par_MarkRefsIntoClosure par_mri_cl(_collector->_span, &(_collector->_markBitMap));
5192   CMKlassClosure klass_closure(&par_mri_cl);
5193 
5194   // ---------- young gen roots --------------
5195   {
5196     work_on_young_gen_roots(worker_id, &par_mri_cl);
5197     _timer.stop();
5198     if (PrintCMSStatistics != 0) {
5199       gclog_or_tty->print_cr(
5200         "Finished young gen initial mark scan work in %dth thread: %3.3f sec",
5201         worker_id, _timer.seconds());
5202     }
5203   }
5204 
5205   // ---------- remaining roots --------------
5206   _timer.reset();
5207   _timer.start();
5208   gch->gen_process_strong_roots(_collector->_cmsGen->level(),
5209                                 false,     // yg was scanned above
5210                                 false,     // this is parallel code
5211                                 false,     // not scavenging
5212                                 SharedHeap::ScanningOption(_collector->CMSCollector::roots_scanning_options()),
5213                                 &par_mri_cl,
5214                                 true,   // walk all of code cache if (so & SO_CodeCache)
5215                                 NULL,
5216                                 &klass_closure);
5217   assert(_collector->should_unload_classes()
5218          || (_collector->CMSCollector::roots_scanning_options() & SharedHeap::SO_CodeCache),
5219          "if we didn't scan the code cache, we have to be ready to drop nmethods with expired weak oops");
5220   _timer.stop();
5221   if (PrintCMSStatistics != 0) {
5222     gclog_or_tty->print_cr(
5223       "Finished remaining root initial mark scan work in %dth thread: %3.3f sec",
5224       worker_id, _timer.seconds());
5225   }
5226 }
5227 
5228 // Parallel remark task
5229 class CMSParRemarkTask: public CMSParMarkTask {


5230   CompactibleFreeListSpace* _cms_space;
5231 
5232   // The per-thread work queues, available here for stealing.
5233   OopTaskQueueSet*       _task_queues;
5234   ParallelTaskTerminator _term;
5235 
5236  public:
5237   // A value of 0 passed to n_workers will cause the number of
5238   // workers to be taken from the active workers in the work gang.
5239   CMSParRemarkTask(CMSCollector* collector,
5240                    CompactibleFreeListSpace* cms_space,
5241                    int n_workers, FlexibleWorkGang* workers,
5242                    OopTaskQueueSet* task_queues):
5243     CMSParMarkTask("Rescan roots and grey objects in parallel",
5244                    collector, n_workers),
5245     _cms_space(cms_space),

5246     _task_queues(task_queues),
5247     _term(n_workers, task_queues) { }
5248 
5249   OopTaskQueueSet* task_queues() { return _task_queues; }
5250 
5251   OopTaskQueue* work_queue(int i) { return task_queues()->queue(i); }
5252 
5253   ParallelTaskTerminator* terminator() { return &_term; }
5254   int n_workers() { return _n_workers; }
5255 
5256   void work(uint worker_id);
5257 
5258  private:





5259   // ... of  dirty cards in old space
5260   void do_dirty_card_rescan_tasks(CompactibleFreeListSpace* sp, int i,
5261                                   Par_MarkRefsIntoAndScanClosure* cl);
5262 
5263   // ... work stealing for the above
5264   void do_work_steal(int i, Par_MarkRefsIntoAndScanClosure* cl, int* seed);
5265 };
5266 
5267 class RemarkKlassClosure : public KlassClosure {
5268   CMKlassClosure _cm_klass_closure;
5269  public:
5270   RemarkKlassClosure(OopClosure* oop_closure) : _cm_klass_closure(oop_closure) {}
5271   void do_klass(Klass* k) {
5272     // Check if we have modified any oops in the Klass during the concurrent marking.
5273     if (k->has_accumulated_modified_oops()) {
5274       k->clear_accumulated_modified_oops();
5275 
5276       // We could have transfered the current modified marks to the accumulated marks,
5277       // like we do with the Card Table to Mod Union Table. But it's not really necessary.
5278     } else if (k->has_modified_oops()) {
5279       // Don't clear anything, this info is needed by the next young collection.
5280     } else {
5281       // No modified oops in the Klass.
5282       return;
5283     }
5284 
5285     // The klass has modified fields, need to scan the klass.
5286     _cm_klass_closure.do_klass(k);
5287   }
5288 };
5289 
5290 void CMSParMarkTask::work_on_young_gen_roots(uint worker_id, OopsInGenClosure* cl) {
5291   DefNewGeneration* dng = _collector->_young_gen->as_DefNewGeneration();
5292   EdenSpace* eden_space = dng->eden();
5293   ContiguousSpace* from_space = dng->from();
5294   ContiguousSpace* to_space   = dng->to();
5295 
5296   HeapWord** eca = _collector->_eden_chunk_array;
5297   size_t     ect = _collector->_eden_chunk_index;
5298   HeapWord** sca = _collector->_survivor_chunk_array;
5299   size_t     sct = _collector->_survivor_chunk_index;
5300 
5301   assert(ect <= _collector->_eden_chunk_capacity, "out of bounds");
5302   assert(sct <= _collector->_survivor_chunk_capacity, "out of bounds");
5303 
5304   do_young_space_rescan(worker_id, cl, to_space, NULL, 0);
5305   do_young_space_rescan(worker_id, cl, from_space, sca, sct);
5306   do_young_space_rescan(worker_id, cl, eden_space, eca, ect);
5307 }
5308 
5309 // work_queue(i) is passed to the closure
5310 // Par_MarkRefsIntoAndScanClosure.  The "i" parameter
5311 // also is passed to do_dirty_card_rescan_tasks() and to
5312 // do_work_steal() to select the i-th task_queue.
5313 
5314 void CMSParRemarkTask::work(uint worker_id) {
5315   elapsedTimer _timer;
5316   ResourceMark rm;
5317   HandleMark   hm;
5318 
5319   // ---------- rescan from roots --------------
5320   _timer.start();
5321   GenCollectedHeap* gch = GenCollectedHeap::heap();
5322   Par_MarkRefsIntoAndScanClosure par_mrias_cl(_collector,
5323     _collector->_span, _collector->ref_processor(),
5324     &(_collector->_markBitMap),
5325     work_queue(worker_id));
5326 
5327   // Rescan young gen roots first since these are likely
5328   // coarsely partitioned and may, on that account, constitute
5329   // the critical path; thus, it's best to start off that
5330   // work first.
5331   // ---------- young gen roots --------------
5332   {
5333     work_on_young_gen_roots(worker_id, &par_mrias_cl);
















5334     _timer.stop();
5335     if (PrintCMSStatistics != 0) {
5336       gclog_or_tty->print_cr(
5337         "Finished young gen rescan work in %dth thread: %3.3f sec",
5338         worker_id, _timer.seconds());
5339     }
5340   }
5341 
5342   // ---------- remaining roots --------------
5343   _timer.reset();
5344   _timer.start();
5345   gch->gen_process_strong_roots(_collector->_cmsGen->level(),
5346                                 false,     // yg was scanned above
5347                                 false,     // this is parallel code
5348                                 false,     // not scavenging
5349                                 SharedHeap::ScanningOption(_collector->CMSCollector::roots_scanning_options()),
5350                                 &par_mrias_cl,
5351                                 true,   // walk all of code cache if (so & SO_CodeCache)
5352                                 NULL,
5353                                 NULL);     // The dirty klasses will be handled below


5421     gclog_or_tty->print_cr(
5422       "Finished dirty card rescan work in %dth thread: %3.3f sec",
5423       worker_id, _timer.seconds());
5424   }
5425 
5426   // ---------- steal work from other threads ...
5427   // ---------- ... and drain overflow list.
5428   _timer.reset();
5429   _timer.start();
5430   do_work_steal(worker_id, &par_mrias_cl, _collector->hash_seed(worker_id));
5431   _timer.stop();
5432   if (PrintCMSStatistics != 0) {
5433     gclog_or_tty->print_cr(
5434       "Finished work stealing in %dth thread: %3.3f sec",
5435       worker_id, _timer.seconds());
5436   }
5437 }
5438 
5439 // Note that parameter "i" is not used.
5440 void
5441 CMSParMarkTask::do_young_space_rescan(uint worker_id,
5442   OopsInGenClosure* cl, ContiguousSpace* space,
5443   HeapWord** chunk_array, size_t chunk_top) {
5444   // Until all tasks completed:
5445   // . claim an unclaimed task
5446   // . compute region boundaries corresponding to task claimed
5447   //   using chunk_array
5448   // . par_oop_iterate(cl) over that region
5449 
5450   ResourceMark rm;
5451   HandleMark   hm;
5452 
5453   SequentialSubTasksDone* pst = space->par_seq_tasks();
5454   assert(pst->valid(), "Uninitialized use?");
5455 
5456   uint nth_task = 0;
5457   uint n_tasks  = pst->n_tasks();
5458 
5459   HeapWord *start, *end;
5460   while (!pst->is_task_claimed(/* reference */ nth_task)) {
5461     // We claimed task # nth_task; compute its boundaries.
5462     if (chunk_top == 0) {  // no samples were taken


5640       (CMSPLABRecordAlways ||
5641        (_collectorState > Marking && _collectorState < FinalMarking))) {
5642     assert(thr_num < (int)ParallelGCThreads, "thr_num is out of bounds");
5643     ChunkArray* ca = &_survivor_plab_array[thr_num];
5644     ca->reset();   // clear it so that fresh data is recorded
5645     return (void*) ca;
5646   } else {
5647     return NULL;
5648   }
5649 }
5650 
5651 // Reset all the thread-local PLAB recording arrays
5652 void CMSCollector::reset_survivor_plab_arrays() {
5653   for (uint i = 0; i < ParallelGCThreads; i++) {
5654     _survivor_plab_array[i].reset();
5655   }
5656 }
5657 
5658 // Merge the per-thread plab arrays into the global survivor chunk
5659 // array which will provide the partitioning of the survivor space
5660 // for CMS initial scan and rescan.
5661 void CMSCollector::merge_survivor_plab_arrays(ContiguousSpace* surv,
5662                                               int no_of_gc_threads) {
5663   assert(_survivor_plab_array  != NULL, "Error");
5664   assert(_survivor_chunk_array != NULL, "Error");
5665   assert(_collectorState == FinalMarking ||
5666          (CMSParallelInitialMarkEnabled && _collectorState == InitialMarking), "Error");
5667   for (int j = 0; j < no_of_gc_threads; j++) {
5668     _cursor[j] = 0;
5669   }
5670   HeapWord* top = surv->top();
5671   size_t i;
5672   for (i = 0; i < _survivor_chunk_capacity; i++) {  // all sca entries
5673     HeapWord* min_val = top;          // Higher than any PLAB address
5674     uint      min_tid = 0;            // position of min_val this round
5675     for (int j = 0; j < no_of_gc_threads; j++) {
5676       ChunkArray* cur_sca = &_survivor_plab_array[j];
5677       if (_cursor[j] == cur_sca->end()) {
5678         continue;
5679       }
5680       assert(_cursor[j] < cur_sca->end(), "ctl pt invariant");
5681       HeapWord* cur_val = cur_sca->nth(_cursor[j]);
5682       assert(surv->used_region().contains(cur_val), "Out of bounds value");
5683       if (cur_val < min_val) {
5684         min_tid = j;
5685         min_val = cur_val;
5686       } else {


5709     for (int j = 0; j < no_of_gc_threads; j++) {
5710       assert(_cursor[j] == _survivor_plab_array[j].end(), "Ctl pt invariant");
5711       total += _cursor[j];
5712     }
5713     assert(total == _survivor_chunk_index, "Ctl Pt Invariant");
5714     // Check that the merged array is in sorted order
5715     if (total > 0) {
5716       for (size_t i = 0; i < total - 1; i++) {
5717         if (PrintCMSStatistics > 0) {
5718           gclog_or_tty->print(" (chunk" SIZE_FORMAT ":" INTPTR_FORMAT ") ",
5719                               i, _survivor_chunk_array[i]);
5720         }
5721         assert(_survivor_chunk_array[i] < _survivor_chunk_array[i+1],
5722                "Not sorted");
5723       }
5724     }
5725   #endif // ASSERT
5726 }
5727 
5728 // Set up the space's par_seq_tasks structure for work claiming
5729 // for parallel initial scan and rescan of young gen.
5730 // See ParRescanTask where this is currently used.
5731 void
5732 CMSCollector::
5733 initialize_sequential_subtasks_for_young_gen_rescan(int n_threads) {
5734   assert(n_threads > 0, "Unexpected n_threads argument");
5735   DefNewGeneration* dng = (DefNewGeneration*)_young_gen;
5736 
5737   // Eden space
5738   {
5739     SequentialSubTasksDone* pst = dng->eden()->par_seq_tasks();
5740     assert(!pst->valid(), "Clobbering existing data?");
5741     // Each valid entry in [0, _eden_chunk_index) represents a task.
5742     size_t n_tasks = _eden_chunk_index + 1;
5743     assert(n_tasks == 1 || _eden_chunk_array != NULL, "Error");
5744     // Sets the condition for completion of the subtask (how many threads
5745     // need to finish in order to be done).
5746     pst->set_n_threads(n_threads);
5747     pst->set_n_tasks((int)n_tasks);
5748   }
5749 


6836     _span(span),
6837     _bitMap(bitMap)
6838 {
6839     assert(_ref_processor == NULL, "deliberately left NULL");
6840     assert(_bitMap->covers(_span), "_bitMap/_span mismatch");
6841 }
6842 
6843 void MarkRefsIntoClosure::do_oop(oop obj) {
6844   // if p points into _span, then mark corresponding bit in _markBitMap
6845   assert(obj->is_oop(), "expected an oop");
6846   HeapWord* addr = (HeapWord*)obj;
6847   if (_span.contains(addr)) {
6848     // this should be made more efficient
6849     _bitMap->mark(addr);
6850   }
6851 }
6852 
6853 void MarkRefsIntoClosure::do_oop(oop* p)       { MarkRefsIntoClosure::do_oop_work(p); }
6854 void MarkRefsIntoClosure::do_oop(narrowOop* p) { MarkRefsIntoClosure::do_oop_work(p); }
6855 
6856 Par_MarkRefsIntoClosure::Par_MarkRefsIntoClosure(
6857   MemRegion span, CMSBitMap* bitMap):
6858     _span(span),
6859     _bitMap(bitMap)
6860 {
6861     assert(_ref_processor == NULL, "deliberately left NULL");
6862     assert(_bitMap->covers(_span), "_bitMap/_span mismatch");
6863 }
6864 
6865 void Par_MarkRefsIntoClosure::do_oop(oop obj) {
6866   // if p points into _span, then mark corresponding bit in _markBitMap
6867   assert(obj->is_oop(), "expected an oop");
6868   HeapWord* addr = (HeapWord*)obj;
6869   if (_span.contains(addr)) {
6870     // this should be made more efficient
6871     _bitMap->par_mark(addr);
6872   }
6873 }
6874 
6875 void Par_MarkRefsIntoClosure::do_oop(oop* p)       { Par_MarkRefsIntoClosure::do_oop_work(p); }
6876 void Par_MarkRefsIntoClosure::do_oop(narrowOop* p) { Par_MarkRefsIntoClosure::do_oop_work(p); }
6877 
6878 // A variant of the above, used for CMS marking verification.
6879 MarkRefsIntoVerifyClosure::MarkRefsIntoVerifyClosure(
6880   MemRegion span, CMSBitMap* verification_bm, CMSBitMap* cms_bm):
6881     _span(span),
6882     _verification_bm(verification_bm),
6883     _cms_bm(cms_bm)
6884 {
6885     assert(_ref_processor == NULL, "deliberately left NULL");
6886     assert(_verification_bm->covers(_span), "_verification_bm/_span mismatch");
6887 }
6888 
6889 void MarkRefsIntoVerifyClosure::do_oop(oop obj) {
6890   // if p points into _span, then mark corresponding bit in _markBitMap
6891   assert(obj->is_oop(), "expected an oop");
6892   HeapWord* addr = (HeapWord*)obj;
6893   if (_span.contains(addr)) {
6894     _verification_bm->mark(addr);
6895     if (!_cms_bm->isMarked(addr)) {
6896       oop(addr)->print();
6897       gclog_or_tty->print_cr(" (" INTPTR_FORMAT " should have been marked)", addr);


9487                  false /* recordGCEndTime */,
9488                  false /* countCollection */  );
9489       break;
9490 
9491     case CMSCollector::Sweeping:
9492       initialize(true  /* fullGC */ ,
9493                  cause /* cause of the GC */,
9494                  false /* recordGCBeginTime */,
9495                  false /* recordPreGCUsage */,
9496                  true  /* recordPeakUsage */,
9497                  true  /* recordPostGCusage */,
9498                  false /* recordAccumulatedGCTime */,
9499                  true  /* recordGCEndTime */,
9500                  true  /* countCollection */  );
9501       break;
9502 
9503     default:
9504       ShouldNotReachHere();
9505   }
9506 }