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

Print this page
rev 6796 : [mq]: templateOopIterate
rev 6801 : imported patch defaultToTrue


  16  * 2 along with this work; if not, write to the Free Software Foundation,
  17  * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
  18  *
  19  * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
  20  * or visit www.oracle.com if you need additional information or have any
  21  * questions.
  22  *
  23  */
  24 
  25 #include "precompiled.hpp"
  26 #include "classfile/classLoaderData.hpp"
  27 #include "classfile/stringTable.hpp"
  28 #include "classfile/systemDictionary.hpp"
  29 #include "code/codeCache.hpp"
  30 #include "gc_implementation/shared/adaptiveSizePolicy.hpp"
  31 #include "gc_implementation/concurrentMarkSweep/cmsCollectorPolicy.hpp"
  32 #include "gc_implementation/concurrentMarkSweep/cmsOopClosures.inline.hpp"
  33 #include "gc_implementation/concurrentMarkSweep/compactibleFreeListSpace.hpp"
  34 #include "gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.inline.hpp"
  35 #include "gc_implementation/concurrentMarkSweep/concurrentMarkSweepThread.hpp"

  36 #include "gc_implementation/concurrentMarkSweep/vmCMSOperations.hpp"
  37 #include "gc_implementation/parNew/parNewGeneration.hpp"
  38 #include "gc_implementation/shared/collectorCounters.hpp"
  39 #include "gc_implementation/shared/gcTimer.hpp"
  40 #include "gc_implementation/shared/gcTrace.hpp"
  41 #include "gc_implementation/shared/gcTraceTime.hpp"
  42 #include "gc_implementation/shared/isGCActiveMark.hpp"
  43 #include "gc_interface/collectedHeap.inline.hpp"
  44 #include "memory/allocation.hpp"
  45 #include "memory/cardTableRS.hpp"
  46 #include "memory/collectorPolicy.hpp"
  47 #include "memory/gcLocker.inline.hpp"
  48 #include "memory/genCollectedHeap.hpp"
  49 #include "memory/genMarkSweep.hpp"
  50 #include "memory/genOopClosures.inline.hpp"
  51 #include "memory/iterator.inline.hpp"
  52 #include "memory/padded.hpp"
  53 #include "memory/referencePolicy.hpp"
  54 #include "memory/resourceArea.hpp"

  55 #include "memory/tenuredGeneration.hpp"
  56 #include "oops/oop.inline.hpp"
  57 #include "prims/jvmtiExport.hpp"
  58 #include "runtime/atomic.inline.hpp"
  59 #include "runtime/globals_extension.hpp"
  60 #include "runtime/handles.inline.hpp"
  61 #include "runtime/java.hpp"
  62 #include "runtime/orderAccess.inline.hpp"
  63 #include "runtime/vmThread.hpp"
  64 #include "services/memoryService.hpp"
  65 #include "services/runtimeService.hpp"
  66 
  67 PRAGMA_FORMAT_MUTE_WARNINGS_FOR_GCC
  68 
  69 // statics
  70 CMSCollector* ConcurrentMarkSweepGeneration::_collector = NULL;
  71 bool CMSCollector::_full_gc_requested = false;
  72 GCCause::Cause CMSCollector::_full_gc_cause = GCCause::_no_gc;
  73 
  74 //////////////////////////////////////////////////////////////////


 183 
 184 NOT_PRODUCT(CompactibleFreeListSpace* debug_cms_space;)
 185 
 186 // This struct contains per-thread things necessary to support parallel
 187 // young-gen collection.
 188 class CMSParGCThreadState: public CHeapObj<mtGC> {
 189  public:
 190   CFLS_LAB lab;
 191   PromotionInfo promo;
 192 
 193   // Constructor.
 194   CMSParGCThreadState(CompactibleFreeListSpace* cfls) : lab(cfls) {
 195     promo.setSpace(cfls);
 196   }
 197 };
 198 
 199 ConcurrentMarkSweepGeneration::ConcurrentMarkSweepGeneration(
 200      ReservedSpace rs, size_t initial_byte_size, int level,
 201      CardTableRS* ct, bool use_adaptive_freelists,
 202      FreeBlockDictionary<FreeChunk>::DictionaryChoice dictionaryChoice) :
 203   CardGeneration(rs, initial_byte_size, level, ct),
 204   _dilatation_factor(((double)MinChunkSize)/((double)(CollectedHeap::min_fill_size()))),
 205   _debug_collection_type(Concurrent_collection_type),
 206   _did_compact(false)
 207 {
 208   HeapWord* bottom = (HeapWord*) _virtual_space.low();
 209   HeapWord* end    = (HeapWord*) _virtual_space.high();
 210 
 211   _direct_allocated_words = 0;
 212   NOT_PRODUCT(
 213     _numObjectsPromoted = 0;
 214     _numWordsPromoted = 0;
 215     _numObjectsAllocated = 0;
 216     _numWordsAllocated = 0;
 217   )
 218 
 219   _cmsSpace = new CompactibleFreeListSpace(_bts, MemRegion(bottom, end),
 220                                            use_adaptive_freelists,
 221                                            dictionaryChoice);
 222   NOT_PRODUCT(debug_cms_space = _cmsSpace;)
 223   if (_cmsSpace == NULL) {


1423   )
1424 
1425   return obj;
1426 }
1427 
1428 void
1429 ConcurrentMarkSweepGeneration::
1430 par_promote_alloc_undo(int thread_num,
1431                        HeapWord* obj, size_t word_sz) {
1432   // CMS does not support promotion undo.
1433   ShouldNotReachHere();
1434 }
1435 
1436 void
1437 ConcurrentMarkSweepGeneration::
1438 par_promote_alloc_done(int thread_num) {
1439   CMSParGCThreadState* ps = _par_gc_thread_states[thread_num];
1440   ps->lab.retire(thread_num);
1441 }
1442 



1443 void
1444 ConcurrentMarkSweepGeneration::
1445 par_oop_since_save_marks_iterate_done(int thread_num) {
1446   CMSParGCThreadState* ps = _par_gc_thread_states[thread_num];
1447   ParScanWithoutBarrierClosure* dummy_cl = NULL;
1448   ps->promo.promoted_oops_iterate_nv(dummy_cl);
1449 }
1450 
1451 bool ConcurrentMarkSweepGeneration::should_collect(bool   full,
1452                                                    size_t size,
1453                                                    bool   tlab)
1454 {
1455   // We allow a STW collection only if a full
1456   // collection was requested.
1457   return full || should_allocate(size, tlab); // FIX ME !!!
1458   // This and promotion failure handling are connected at the
1459   // hip and should be fixed by untying them.
1460 }
1461 
1462 bool CMSCollector::shouldConcurrentCollect() {
1463   if (_full_gc_requested) {
1464     if (Verbose && PrintGCDetails) {
1465       gclog_or_tty->print_cr("CMSCollector: collect because of explicit "
1466                              " gc request (or gc_locker)");
1467     }
1468     return true;


3103   // Marking completed -- now verify that each bit marked in
3104   // verification_mark_bm() is also marked in markBitMap(); flag all
3105   // errors by printing corresponding objects.
3106   VerifyMarkedClosure vcl(markBitMap());
3107   verification_mark_bm()->iterate(&vcl);
3108   assert(!vcl.failed(), "Else verification above should not have succeeded");
3109 }
3110 
3111 void ConcurrentMarkSweepGeneration::save_marks() {
3112   // delegate to CMS space
3113   cmsSpace()->save_marks();
3114   for (uint i = 0; i < ParallelGCThreads; i++) {
3115     _par_gc_thread_states[i]->promo.startTrackingPromotions();
3116   }
3117 }
3118 
3119 bool ConcurrentMarkSweepGeneration::no_allocs_since_save_marks() {
3120   return cmsSpace()->no_allocs_since_save_marks();
3121 }
3122 
3123 #define CMS_SINCE_SAVE_MARKS_DEFN(OopClosureType, nv_suffix)    \
3124                                                                 \
3125 void ConcurrentMarkSweepGeneration::                            \
3126 oop_since_save_marks_iterate##nv_suffix(OopClosureType* cl) {   \
3127   cl->set_generation(this);                                     \
3128   cmsSpace()->oop_since_save_marks_iterate##nv_suffix(cl);      \
3129   cl->reset_generation();                                       \
3130   save_marks();                                                 \
3131 }
3132 
3133 ALL_SINCE_SAVE_MARKS_CLOSURES(CMS_SINCE_SAVE_MARKS_DEFN)
3134 
3135 void
3136 ConcurrentMarkSweepGeneration::younger_refs_iterate(OopsInGenClosure* cl) {
3137   cl->set_generation(this);
3138   younger_refs_in_space_iterate(_cmsSpace, cl);
3139   cl->reset_generation();
3140 }
3141 
3142 void
3143 ConcurrentMarkSweepGeneration::oop_iterate(ExtendedOopClosure* cl) {
3144   if (freelistLock()->owned_by_self()) {
3145     Generation::oop_iterate(cl);
3146   } else {
3147     MutexLockerEx x(freelistLock(), Mutex::_no_safepoint_check_flag);
3148     Generation::oop_iterate(cl);
3149   }
3150 }
3151 
3152 void
3153 ConcurrentMarkSweepGeneration::object_iterate(ObjectClosure* cl) {
3154   if (freelistLock()->owned_by_self()) {


4196         assert(simulate_overflow ||
4197                _work_queue->size() == _work_queue->max_elems(),
4198               "Else push should have succeeded");
4199         handle_stack_overflow(addr);
4200       }
4201     } // Else, some other thread got there first
4202     do_yield_check();
4203   }
4204 }
4205 
4206 void Par_ConcMarkingClosure::do_oop(oop* p)       { Par_ConcMarkingClosure::do_oop_work(p); }
4207 void Par_ConcMarkingClosure::do_oop(narrowOop* p) { Par_ConcMarkingClosure::do_oop_work(p); }
4208 
4209 void Par_ConcMarkingClosure::trim_queue(size_t max) {
4210   while (_work_queue->size() > max) {
4211     oop new_oop;
4212     if (_work_queue->pop_local(new_oop)) {
4213       assert(new_oop->is_oop(), "Should be an oop");
4214       assert(_bit_map->isMarked((HeapWord*)new_oop), "Grey object");
4215       assert(_span.contains((HeapWord*)new_oop), "Not in span");
4216       new_oop->oop_iterate(this);  // do_oop() above

4217       do_yield_check();
4218     }
4219   }
4220 }
4221 
4222 // Upon stack overflow, we discard (part of) the stack,
4223 // remembering the least address amongst those discarded
4224 // in CMSCollector's _restart_address.
4225 void Par_ConcMarkingClosure::handle_stack_overflow(HeapWord* lost) {
4226   // We need to do this under a mutex to prevent other
4227   // workers from interfering with the work done below.
4228   MutexLockerEx ml(_overflow_stack->par_lock(),
4229                    Mutex::_no_safepoint_check_flag);
4230   // Remember the least grey address discarded
4231   HeapWord* ra = (HeapWord*)_overflow_stack->least_value(lost);
4232   _collector->lower_restart_addr(ra);
4233   _overflow_stack->reset();  // discard stack contents
4234   _overflow_stack->expand(); // expand the stack if possible
4235 }
4236 
4237 
4238 void CMSConcMarkingTask::do_work_steal(int i) {
4239   OopTaskQueue* work_q = work_queue(i);
4240   oop obj_to_scan;
4241   CMSBitMap* bm = &(_collector->_markBitMap);
4242   CMSMarkStack* ovflw = &(_collector->_markStack);
4243   int* seed = _collector->hash_seed(i);
4244   Par_ConcMarkingClosure cl(_collector, this, work_q, bm, ovflw);
4245   while (true) {
4246     cl.trim_queue(0);
4247     assert(work_q->size() == 0, "Should have been emptied above");
4248     if (get_work_from_overflow_stack(ovflw, work_q)) {
4249       // Can't assert below because the work obtained from the
4250       // overflow stack may already have been stolen from us.
4251       // assert(work_q->size() > 0, "Work from overflow stack");
4252       continue;
4253     } else if (task_queues()->steal(i, seed, /* reference */ obj_to_scan)) {
4254       assert(obj_to_scan->is_oop(), "Should be an oop");
4255       assert(bm->isMarked((HeapWord*)obj_to_scan), "Grey object");
4256       obj_to_scan->oop_iterate(&cl);

4257     } else if (terminator()->offer_termination(&_term_term)) {
4258       assert(work_q->size() == 0, "Impossible!");
4259       break;
4260     } else if (yielding() || should_yield()) {
4261       yield();
4262     }
4263   }
4264 }
4265 
4266 // This is run by the CMS (coordinator) thread.
4267 void CMSConcMarkingTask::coordinator_yield() {
4268   assert(ConcurrentMarkSweepThread::cms_thread_has_cms_token(),
4269          "CMS thread should hold CMS token");
4270   // First give up the locks, then yield, then re-lock
4271   // We should probably use a constructor/destructor idiom to
4272   // do this unlock/lock or modify the MutexUnlocker class to
4273   // serve our purpose. XXX
4274   assert_lock_strong(_bit_map_lock);
4275   _bit_map_lock->unlock();
4276   ConcurrentMarkSweepThread::desynchronize(true);


5417         end   = space->top();
5418       } else if (nth_task == 0) {
5419         start = space->bottom();
5420         end   = chunk_array[nth_task];
5421       } else if (nth_task < (uint)chunk_top) {
5422         assert(nth_task >= 1, "Control point invariant");
5423         start = chunk_array[nth_task - 1];
5424         end   = chunk_array[nth_task];
5425       } else {
5426         assert(nth_task == (uint)chunk_top, "Control point invariant");
5427         start = chunk_array[chunk_top - 1];
5428         end   = space->top();
5429       }
5430       MemRegion mr(start, end);
5431       // Verify that mr is in space
5432       assert(mr.is_empty() || space->used_region().contains(mr),
5433              "Should be in space");
5434       // Verify that "start" is an object boundary
5435       assert(mr.is_empty() || oop(mr.start())->is_oop(),
5436              "Should be an oop");
5437       space->par_oop_iterate(mr, cl);
5438     }
5439     pst->all_tasks_completed();
5440   }
5441 }
5442 
5443 void
5444 CMSParRemarkTask::do_dirty_card_rescan_tasks(
5445   CompactibleFreeListSpace* sp, int i,
5446   Par_MarkRefsIntoAndScanClosure* cl) {
5447   // Until all tasks completed:
5448   // . claim an unclaimed task
5449   // . compute region boundaries corresponding to task claimed
5450   // . transfer dirty bits ct->mut for that region
5451   // . apply rescanclosure to dirty mut bits for that region
5452 
5453   ResourceMark rm;
5454   HandleMark   hm;
5455 
5456   OopTaskQueue* work_q = work_queue(i);
5457   ModUnionClosure modUnionClosure(&(_collector->_modUnionTable));


6053     // pass ParallelGCThreads so this behavior is unchanged.
6054     if (_collector->par_take_from_overflow_list(num_from_overflow_list,
6055                                                 work_q,
6056                                                 ParallelGCThreads)) {
6057       // Found something in global overflow list;
6058       // not yet ready to go stealing work from others.
6059       // We'd like to assert(work_q->size() != 0, ...)
6060       // because we just took work from the overflow list,
6061       // but of course we can't, since all of that might have
6062       // been already stolen from us.
6063       continue;
6064     }
6065     // Verify that we have no work before we resort to stealing
6066     assert(work_q->size() == 0, "Have work, shouldn't steal");
6067     // Try to steal from other queues that have work
6068     if (task_queues()->steal(i, seed, /* reference */ obj_to_scan)) {
6069       NOT_PRODUCT(num_steals++;)
6070       assert(obj_to_scan->is_oop(), "Oops, not an oop!");
6071       assert(_mark_bit_map->isMarked((HeapWord*)obj_to_scan), "Stole an unmarked oop?");
6072       // Do scanning work
6073       obj_to_scan->oop_iterate(keep_alive);

6074       // Loop around, finish this work, and try to steal some more
6075     } else if (terminator()->offer_termination()) {
6076       break;  // nirvana from the infinite cycle
6077     }
6078   }
6079   NOT_PRODUCT(
6080     if (PrintCMSStatistics != 0) {
6081       gclog_or_tty->print("\n\t(%d: stole %d oops)", i, num_steals);
6082     }
6083   )
6084 }
6085 
6086 void CMSRefProcTaskExecutor::execute(ProcessTask& task)
6087 {
6088   GenCollectedHeap* gch = GenCollectedHeap::heap();
6089   FlexibleWorkGang* workers = gch->workers();
6090   assert(workers != NULL, "Need parallel worker threads.");
6091   CMSRefProcTaskProxy rp_task(task, &_collector,
6092                               _collector.ref_processor()->span(),
6093                               _collector.markBitMap(),


7614   assert(_verification_bm->isMarked(addr), "tautology");
7615   assert(_cms_bm->isMarked(addr), "tautology");
7616 
7617   assert(_mark_stack->isEmpty(),
7618          "should drain stack to limit stack usage");
7619   // convert addr to an oop preparatory to scanning
7620   oop obj = oop(addr);
7621   assert(obj->is_oop(), "should be an oop");
7622   assert(_finger <= addr, "_finger runneth ahead");
7623   // advance the finger to right end of this object
7624   _finger = addr + obj->size();
7625   assert(_finger > addr, "we just incremented it above");
7626   // Note: the finger doesn't advance while we drain
7627   // the stack below.
7628   bool res = _mark_stack->push(obj);
7629   assert(res, "Empty non-zero size stack should have space for single push");
7630   while (!_mark_stack->isEmpty()) {
7631     oop new_oop = _mark_stack->pop();
7632     assert(new_oop->is_oop(), "Oops! expected to pop an oop");
7633     // now scan this oop's oops
7634     new_oop->oop_iterate(&_pam_verify_closure);

7635   }
7636   assert(_mark_stack->isEmpty(), "tautology, emphasizing post-condition");
7637   return true;
7638 }
7639 
7640 PushAndMarkVerifyClosure::PushAndMarkVerifyClosure(
7641   CMSCollector* collector, MemRegion span,
7642   CMSBitMap* verification_bm, CMSBitMap* cms_bm,
7643   CMSMarkStack*  mark_stack):
7644   MetadataAwareOopClosure(collector->ref_processor()),
7645   _collector(collector),
7646   _span(span),
7647   _verification_bm(verification_bm),
7648   _cms_bm(cms_bm),
7649   _mark_stack(mark_stack)
7650 { }
7651 
7652 void PushAndMarkVerifyClosure::do_oop(oop* p)       { PushAndMarkVerifyClosure::do_oop_work(p); }
7653 void PushAndMarkVerifyClosure::do_oop(narrowOop* p) { PushAndMarkVerifyClosure::do_oop_work(p); }
7654 




  16  * 2 along with this work; if not, write to the Free Software Foundation,
  17  * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
  18  *
  19  * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
  20  * or visit www.oracle.com if you need additional information or have any
  21  * questions.
  22  *
  23  */
  24 
  25 #include "precompiled.hpp"
  26 #include "classfile/classLoaderData.hpp"
  27 #include "classfile/stringTable.hpp"
  28 #include "classfile/systemDictionary.hpp"
  29 #include "code/codeCache.hpp"
  30 #include "gc_implementation/shared/adaptiveSizePolicy.hpp"
  31 #include "gc_implementation/concurrentMarkSweep/cmsCollectorPolicy.hpp"
  32 #include "gc_implementation/concurrentMarkSweep/cmsOopClosures.inline.hpp"
  33 #include "gc_implementation/concurrentMarkSweep/compactibleFreeListSpace.hpp"
  34 #include "gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.inline.hpp"
  35 #include "gc_implementation/concurrentMarkSweep/concurrentMarkSweepThread.hpp"
  36 #include "gc_implementation/concurrentMarkSweep/promotionInfo.inline.hpp"
  37 #include "gc_implementation/concurrentMarkSweep/vmCMSOperations.hpp"
  38 #include "gc_implementation/parNew/parNewGeneration.hpp"
  39 #include "gc_implementation/shared/collectorCounters.hpp"
  40 #include "gc_implementation/shared/gcTimer.hpp"
  41 #include "gc_implementation/shared/gcTrace.hpp"
  42 #include "gc_implementation/shared/gcTraceTime.hpp"
  43 #include "gc_implementation/shared/isGCActiveMark.hpp"
  44 #include "gc_interface/collectedHeap.inline.hpp"
  45 #include "memory/allocation.hpp"
  46 #include "memory/cardTableRS.hpp"
  47 #include "memory/collectorPolicy.hpp"
  48 #include "memory/gcLocker.inline.hpp"
  49 #include "memory/genCollectedHeap.hpp"
  50 #include "memory/genMarkSweep.hpp"
  51 #include "memory/genOopClosures.inline.hpp"
  52 #include "memory/iterator.inline.hpp"
  53 #include "memory/padded.hpp"
  54 #include "memory/referencePolicy.hpp"
  55 #include "memory/resourceArea.hpp"
  56 #include "memory/space.inline.hpp"
  57 #include "memory/tenuredGeneration.hpp"
  58 #include "oops/oop.inline.hpp"
  59 #include "prims/jvmtiExport.hpp"
  60 #include "runtime/atomic.inline.hpp"
  61 #include "runtime/globals_extension.hpp"
  62 #include "runtime/handles.inline.hpp"
  63 #include "runtime/java.hpp"
  64 #include "runtime/orderAccess.inline.hpp"
  65 #include "runtime/vmThread.hpp"
  66 #include "services/memoryService.hpp"
  67 #include "services/runtimeService.hpp"
  68 
  69 PRAGMA_FORMAT_MUTE_WARNINGS_FOR_GCC
  70 
  71 // statics
  72 CMSCollector* ConcurrentMarkSweepGeneration::_collector = NULL;
  73 bool CMSCollector::_full_gc_requested = false;
  74 GCCause::Cause CMSCollector::_full_gc_cause = GCCause::_no_gc;
  75 
  76 //////////////////////////////////////////////////////////////////


 185 
 186 NOT_PRODUCT(CompactibleFreeListSpace* debug_cms_space;)
 187 
 188 // This struct contains per-thread things necessary to support parallel
 189 // young-gen collection.
 190 class CMSParGCThreadState: public CHeapObj<mtGC> {
 191  public:
 192   CFLS_LAB lab;
 193   PromotionInfo promo;
 194 
 195   // Constructor.
 196   CMSParGCThreadState(CompactibleFreeListSpace* cfls) : lab(cfls) {
 197     promo.setSpace(cfls);
 198   }
 199 };
 200 
 201 ConcurrentMarkSweepGeneration::ConcurrentMarkSweepGeneration(
 202      ReservedSpace rs, size_t initial_byte_size, int level,
 203      CardTableRS* ct, bool use_adaptive_freelists,
 204      FreeBlockDictionary<FreeChunk>::DictionaryChoice dictionaryChoice) :
 205   CardGeneration(rs, initial_byte_size, level, ct, _dispatch_index_generation_cms),
 206   _dilatation_factor(((double)MinChunkSize)/((double)(CollectedHeap::min_fill_size()))),
 207   _debug_collection_type(Concurrent_collection_type),
 208   _did_compact(false)
 209 {
 210   HeapWord* bottom = (HeapWord*) _virtual_space.low();
 211   HeapWord* end    = (HeapWord*) _virtual_space.high();
 212 
 213   _direct_allocated_words = 0;
 214   NOT_PRODUCT(
 215     _numObjectsPromoted = 0;
 216     _numWordsPromoted = 0;
 217     _numObjectsAllocated = 0;
 218     _numWordsAllocated = 0;
 219   )
 220 
 221   _cmsSpace = new CompactibleFreeListSpace(_bts, MemRegion(bottom, end),
 222                                            use_adaptive_freelists,
 223                                            dictionaryChoice);
 224   NOT_PRODUCT(debug_cms_space = _cmsSpace;)
 225   if (_cmsSpace == NULL) {


1425   )
1426 
1427   return obj;
1428 }
1429 
1430 void
1431 ConcurrentMarkSweepGeneration::
1432 par_promote_alloc_undo(int thread_num,
1433                        HeapWord* obj, size_t word_sz) {
1434   // CMS does not support promotion undo.
1435   ShouldNotReachHere();
1436 }
1437 
1438 void
1439 ConcurrentMarkSweepGeneration::
1440 par_promote_alloc_done(int thread_num) {
1441   CMSParGCThreadState* ps = _par_gc_thread_states[thread_num];
1442   ps->lab.retire(thread_num);
1443 }
1444 
1445 // SSS: Temporary include
1446 #include "gc_implementation/parNew/parOopClosures.inline.hpp"
1447 
1448 void
1449 ConcurrentMarkSweepGeneration::
1450 par_oop_since_save_marks_iterate_done(int thread_num) {
1451   CMSParGCThreadState* ps = _par_gc_thread_states[thread_num];
1452   ParScanWithoutBarrierClosure* dummy_cl = NULL;
1453   ps->promo.promoted_oops_iterate<false>(dummy_cl);
1454 }
1455 
1456 bool ConcurrentMarkSweepGeneration::should_collect(bool   full,
1457                                                    size_t size,
1458                                                    bool   tlab)
1459 {
1460   // We allow a STW collection only if a full
1461   // collection was requested.
1462   return full || should_allocate(size, tlab); // FIX ME !!!
1463   // This and promotion failure handling are connected at the
1464   // hip and should be fixed by untying them.
1465 }
1466 
1467 bool CMSCollector::shouldConcurrentCollect() {
1468   if (_full_gc_requested) {
1469     if (Verbose && PrintGCDetails) {
1470       gclog_or_tty->print_cr("CMSCollector: collect because of explicit "
1471                              " gc request (or gc_locker)");
1472     }
1473     return true;


3108   // Marking completed -- now verify that each bit marked in
3109   // verification_mark_bm() is also marked in markBitMap(); flag all
3110   // errors by printing corresponding objects.
3111   VerifyMarkedClosure vcl(markBitMap());
3112   verification_mark_bm()->iterate(&vcl);
3113   assert(!vcl.failed(), "Else verification above should not have succeeded");
3114 }
3115 
3116 void ConcurrentMarkSweepGeneration::save_marks() {
3117   // delegate to CMS space
3118   cmsSpace()->save_marks();
3119   for (uint i = 0; i < ParallelGCThreads; i++) {
3120     _par_gc_thread_states[i]->promo.startTrackingPromotions();
3121   }
3122 }
3123 
3124 bool ConcurrentMarkSweepGeneration::no_allocs_since_save_marks() {
3125   return cmsSpace()->no_allocs_since_save_marks();
3126 }
3127 












3128 void
3129 ConcurrentMarkSweepGeneration::younger_refs_iterate(OopsInGenClosure* cl) {
3130   cl->set_generation(this);
3131   younger_refs_in_space_iterate(_cmsSpace, cl);
3132   cl->reset_generation();
3133 }
3134 
3135 void
3136 ConcurrentMarkSweepGeneration::oop_iterate(ExtendedOopClosure* cl) {
3137   if (freelistLock()->owned_by_self()) {
3138     Generation::oop_iterate(cl);
3139   } else {
3140     MutexLockerEx x(freelistLock(), Mutex::_no_safepoint_check_flag);
3141     Generation::oop_iterate(cl);
3142   }
3143 }
3144 
3145 void
3146 ConcurrentMarkSweepGeneration::object_iterate(ObjectClosure* cl) {
3147   if (freelistLock()->owned_by_self()) {


4189         assert(simulate_overflow ||
4190                _work_queue->size() == _work_queue->max_elems(),
4191               "Else push should have succeeded");
4192         handle_stack_overflow(addr);
4193       }
4194     } // Else, some other thread got there first
4195     do_yield_check();
4196   }
4197 }
4198 
4199 void Par_ConcMarkingClosure::do_oop(oop* p)       { Par_ConcMarkingClosure::do_oop_work(p); }
4200 void Par_ConcMarkingClosure::do_oop(narrowOop* p) { Par_ConcMarkingClosure::do_oop_work(p); }
4201 
4202 void Par_ConcMarkingClosure::trim_queue(size_t max) {
4203   while (_work_queue->size() > max) {
4204     oop new_oop;
4205     if (_work_queue->pop_local(new_oop)) {
4206       assert(new_oop->is_oop(), "Should be an oop");
4207       assert(_bit_map->isMarked((HeapWord*)new_oop), "Grey object");
4208       assert(_span.contains((HeapWord*)new_oop), "Not in span");
4209       // SSS: Opportunity to devirtualize
4210       new_oop->oop_iterate<false>(this);  // do_oop() above
4211       do_yield_check();
4212     }
4213   }
4214 }
4215 
4216 // Upon stack overflow, we discard (part of) the stack,
4217 // remembering the least address amongst those discarded
4218 // in CMSCollector's _restart_address.
4219 void Par_ConcMarkingClosure::handle_stack_overflow(HeapWord* lost) {
4220   // We need to do this under a mutex to prevent other
4221   // workers from interfering with the work done below.
4222   MutexLockerEx ml(_overflow_stack->par_lock(),
4223                    Mutex::_no_safepoint_check_flag);
4224   // Remember the least grey address discarded
4225   HeapWord* ra = (HeapWord*)_overflow_stack->least_value(lost);
4226   _collector->lower_restart_addr(ra);
4227   _overflow_stack->reset();  // discard stack contents
4228   _overflow_stack->expand(); // expand the stack if possible
4229 }
4230 
4231 
4232 void CMSConcMarkingTask::do_work_steal(int i) {
4233   OopTaskQueue* work_q = work_queue(i);
4234   oop obj_to_scan;
4235   CMSBitMap* bm = &(_collector->_markBitMap);
4236   CMSMarkStack* ovflw = &(_collector->_markStack);
4237   int* seed = _collector->hash_seed(i);
4238   Par_ConcMarkingClosure cl(_collector, this, work_q, bm, ovflw);
4239   while (true) {
4240     cl.trim_queue(0);
4241     assert(work_q->size() == 0, "Should have been emptied above");
4242     if (get_work_from_overflow_stack(ovflw, work_q)) {
4243       // Can't assert below because the work obtained from the
4244       // overflow stack may already have been stolen from us.
4245       // assert(work_q->size() > 0, "Work from overflow stack");
4246       continue;
4247     } else if (task_queues()->steal(i, seed, /* reference */ obj_to_scan)) {
4248       assert(obj_to_scan->is_oop(), "Should be an oop");
4249       assert(bm->isMarked((HeapWord*)obj_to_scan), "Grey object");
4250       // SSS: Opportunity to devirtualize
4251       obj_to_scan->oop_iterate<false>(&cl);
4252     } else if (terminator()->offer_termination(&_term_term)) {
4253       assert(work_q->size() == 0, "Impossible!");
4254       break;
4255     } else if (yielding() || should_yield()) {
4256       yield();
4257     }
4258   }
4259 }
4260 
4261 // This is run by the CMS (coordinator) thread.
4262 void CMSConcMarkingTask::coordinator_yield() {
4263   assert(ConcurrentMarkSweepThread::cms_thread_has_cms_token(),
4264          "CMS thread should hold CMS token");
4265   // First give up the locks, then yield, then re-lock
4266   // We should probably use a constructor/destructor idiom to
4267   // do this unlock/lock or modify the MutexUnlocker class to
4268   // serve our purpose. XXX
4269   assert_lock_strong(_bit_map_lock);
4270   _bit_map_lock->unlock();
4271   ConcurrentMarkSweepThread::desynchronize(true);


5412         end   = space->top();
5413       } else if (nth_task == 0) {
5414         start = space->bottom();
5415         end   = chunk_array[nth_task];
5416       } else if (nth_task < (uint)chunk_top) {
5417         assert(nth_task >= 1, "Control point invariant");
5418         start = chunk_array[nth_task - 1];
5419         end   = chunk_array[nth_task];
5420       } else {
5421         assert(nth_task == (uint)chunk_top, "Control point invariant");
5422         start = chunk_array[chunk_top - 1];
5423         end   = space->top();
5424       }
5425       MemRegion mr(start, end);
5426       // Verify that mr is in space
5427       assert(mr.is_empty() || space->used_region().contains(mr),
5428              "Should be in space");
5429       // Verify that "start" is an object boundary
5430       assert(mr.is_empty() || oop(mr.start())->is_oop(),
5431              "Should be an oop");
5432       space->par_oop_iterate<false>(mr, cl);
5433     }
5434     pst->all_tasks_completed();
5435   }
5436 }
5437 
5438 void
5439 CMSParRemarkTask::do_dirty_card_rescan_tasks(
5440   CompactibleFreeListSpace* sp, int i,
5441   Par_MarkRefsIntoAndScanClosure* cl) {
5442   // Until all tasks completed:
5443   // . claim an unclaimed task
5444   // . compute region boundaries corresponding to task claimed
5445   // . transfer dirty bits ct->mut for that region
5446   // . apply rescanclosure to dirty mut bits for that region
5447 
5448   ResourceMark rm;
5449   HandleMark   hm;
5450 
5451   OopTaskQueue* work_q = work_queue(i);
5452   ModUnionClosure modUnionClosure(&(_collector->_modUnionTable));


6048     // pass ParallelGCThreads so this behavior is unchanged.
6049     if (_collector->par_take_from_overflow_list(num_from_overflow_list,
6050                                                 work_q,
6051                                                 ParallelGCThreads)) {
6052       // Found something in global overflow list;
6053       // not yet ready to go stealing work from others.
6054       // We'd like to assert(work_q->size() != 0, ...)
6055       // because we just took work from the overflow list,
6056       // but of course we can't, since all of that might have
6057       // been already stolen from us.
6058       continue;
6059     }
6060     // Verify that we have no work before we resort to stealing
6061     assert(work_q->size() == 0, "Have work, shouldn't steal");
6062     // Try to steal from other queues that have work
6063     if (task_queues()->steal(i, seed, /* reference */ obj_to_scan)) {
6064       NOT_PRODUCT(num_steals++;)
6065       assert(obj_to_scan->is_oop(), "Oops, not an oop!");
6066       assert(_mark_bit_map->isMarked((HeapWord*)obj_to_scan), "Stole an unmarked oop?");
6067       // Do scanning work
6068       // SSS: Opportunity to devirtualize
6069       obj_to_scan->oop_iterate<false>(keep_alive);
6070       // Loop around, finish this work, and try to steal some more
6071     } else if (terminator()->offer_termination()) {
6072       break;  // nirvana from the infinite cycle
6073     }
6074   }
6075   NOT_PRODUCT(
6076     if (PrintCMSStatistics != 0) {
6077       gclog_or_tty->print("\n\t(%d: stole %d oops)", i, num_steals);
6078     }
6079   )
6080 }
6081 
6082 void CMSRefProcTaskExecutor::execute(ProcessTask& task)
6083 {
6084   GenCollectedHeap* gch = GenCollectedHeap::heap();
6085   FlexibleWorkGang* workers = gch->workers();
6086   assert(workers != NULL, "Need parallel worker threads.");
6087   CMSRefProcTaskProxy rp_task(task, &_collector,
6088                               _collector.ref_processor()->span(),
6089                               _collector.markBitMap(),


7610   assert(_verification_bm->isMarked(addr), "tautology");
7611   assert(_cms_bm->isMarked(addr), "tautology");
7612 
7613   assert(_mark_stack->isEmpty(),
7614          "should drain stack to limit stack usage");
7615   // convert addr to an oop preparatory to scanning
7616   oop obj = oop(addr);
7617   assert(obj->is_oop(), "should be an oop");
7618   assert(_finger <= addr, "_finger runneth ahead");
7619   // advance the finger to right end of this object
7620   _finger = addr + obj->size();
7621   assert(_finger > addr, "we just incremented it above");
7622   // Note: the finger doesn't advance while we drain
7623   // the stack below.
7624   bool res = _mark_stack->push(obj);
7625   assert(res, "Empty non-zero size stack should have space for single push");
7626   while (!_mark_stack->isEmpty()) {
7627     oop new_oop = _mark_stack->pop();
7628     assert(new_oop->is_oop(), "Oops! expected to pop an oop");
7629     // now scan this oop's oops
7630     // SSS: This is not devirtualized, opportunity to fix?
7631     new_oop->oop_iterate<false>(&_pam_verify_closure);
7632   }
7633   assert(_mark_stack->isEmpty(), "tautology, emphasizing post-condition");
7634   return true;
7635 }
7636 
7637 PushAndMarkVerifyClosure::PushAndMarkVerifyClosure(
7638   CMSCollector* collector, MemRegion span,
7639   CMSBitMap* verification_bm, CMSBitMap* cms_bm,
7640   CMSMarkStack*  mark_stack):
7641   MetadataAwareOopClosure(collector->ref_processor()),
7642   _collector(collector),
7643   _span(span),
7644   _verification_bm(verification_bm),
7645   _cms_bm(cms_bm),
7646   _mark_stack(mark_stack)
7647 { }
7648 
7649 void PushAndMarkVerifyClosure::do_oop(oop* p)       { PushAndMarkVerifyClosure::do_oop_work(p); }
7650 void PushAndMarkVerifyClosure::do_oop(narrowOop* p) { PushAndMarkVerifyClosure::do_oop_work(p); }
7651