< prev index next >

src/hotspot/share/gc/cms/concurrentMarkSweepGeneration.cpp

Print this page




2450   ClassLoaderDataGraph::cld_do(&verify_cld_oops);
2451 
2452   // Marking completed -- now verify that each bit marked in
2453   // verification_mark_bm() is also marked in markBitMap(); flag all
2454   // errors by printing corresponding objects.
2455   VerifyMarkedClosure vcl(markBitMap());
2456   verification_mark_bm()->iterate(&vcl);
2457   assert(!vcl.failed(), "Else verification above should not have succeeded");
2458 }
2459 
2460 void ConcurrentMarkSweepGeneration::save_marks() {
2461   // delegate to CMS space
2462   cmsSpace()->save_marks();
2463 }
2464 
2465 bool ConcurrentMarkSweepGeneration::no_allocs_since_save_marks() {
2466   return cmsSpace()->no_allocs_since_save_marks();
2467 }
2468 
2469 void
2470 ConcurrentMarkSweepGeneration::oop_iterate(ExtendedOopClosure* cl) {
2471   if (freelistLock()->owned_by_self()) {
2472     Generation::oop_iterate(cl);
2473   } else {
2474     MutexLockerEx x(freelistLock(), Mutex::_no_safepoint_check_flag);
2475     Generation::oop_iterate(cl);
2476   }
2477 }
2478 
2479 void
2480 ConcurrentMarkSweepGeneration::object_iterate(ObjectClosure* cl) {
2481   if (freelistLock()->owned_by_self()) {
2482     Generation::object_iterate(cl);
2483   } else {
2484     MutexLockerEx x(freelistLock(), Mutex::_no_safepoint_check_flag);
2485     Generation::object_iterate(cl);
2486   }
2487 }
2488 
2489 void
2490 ConcurrentMarkSweepGeneration::safe_object_iterate(ObjectClosure* cl) {


3288         // iteration should be incremental with periodic yields.
3289         ParMarkFromRootsClosure cl(this, _collector, my_span,
3290                                    &_collector->_markBitMap,
3291                                    work_queue(i),
3292                                    &_collector->_markStack);
3293         _collector->_markBitMap.iterate(&cl, my_span.start(), my_span.end());
3294       } // else nothing to do for this task
3295     }   // else nothing to do for this task
3296   }
3297   // We'd be tempted to assert here that since there are no
3298   // more tasks left to claim in this space, the global_finger
3299   // must exceed space->top() and a fortiori space->end(). However,
3300   // that would not quite be correct because the bumping of
3301   // global_finger occurs strictly after the claiming of a task,
3302   // so by the time we reach here the global finger may not yet
3303   // have been bumped up by the thread that claimed the last
3304   // task.
3305   pst->all_tasks_completed();
3306 }
3307 
3308 class ParConcMarkingClosure: public MetadataAwareOopClosure {
3309  private:
3310   CMSCollector* _collector;
3311   CMSConcMarkingTask* _task;
3312   MemRegion     _span;
3313   CMSBitMap*    _bit_map;
3314   CMSMarkStack* _overflow_stack;
3315   OopTaskQueue* _work_queue;
3316  protected:
3317   DO_OOP_WORK_DEFN
3318  public:
3319   ParConcMarkingClosure(CMSCollector* collector, CMSConcMarkingTask* task, OopTaskQueue* work_queue,
3320                         CMSBitMap* bit_map, CMSMarkStack* overflow_stack):
3321     MetadataAwareOopClosure(collector->ref_processor()),
3322     _collector(collector),
3323     _task(task),
3324     _span(collector->_span),
3325     _work_queue(work_queue),
3326     _bit_map(bit_map),
3327     _overflow_stack(overflow_stack)
3328   { }
3329   virtual void do_oop(oop* p);
3330   virtual void do_oop(narrowOop* p);
3331 
3332   void trim_queue(size_t max);
3333   void handle_stack_overflow(HeapWord* lost);
3334   void do_yield_check() {
3335     if (_task->should_yield()) {
3336       _task->yield();
3337     }
3338   }
3339 };
3340 
3341 DO_OOP_WORK_IMPL(ParConcMarkingClosure)
3342 
3343 // Grey object scanning during work stealing phase --
3344 // the salient assumption here is that any references
3345 // that are in these stolen objects being scanned must
3346 // already have been initialized (else they would not have
3347 // been published), so we do not need to check for
3348 // uninitialized objects before pushing here.
3349 void ParConcMarkingClosure::do_oop(oop obj) {
3350   assert(oopDesc::is_oop_or_null(obj, true), "Expected an oop or NULL at " PTR_FORMAT, p2i(obj));
3351   HeapWord* addr = (HeapWord*)obj;
3352   // Check if oop points into the CMS generation
3353   // and is not marked
3354   if (_span.contains(addr) && !_bit_map->isMarked(addr)) {
3355     // a white object ...
3356     // If we manage to "claim" the object, by being the
3357     // first thread to mark it, then we push it on our
3358     // marking stack
3359     if (_bit_map->par_mark(addr)) {     // ... now grey
3360       // push on work queue (grey set)
3361       bool simulate_overflow = false;


3365           // simulate a stack overflow
3366           simulate_overflow = true;
3367         }
3368       )
3369       if (simulate_overflow ||
3370           !(_work_queue->push(obj) || _overflow_stack->par_push(obj))) {
3371         // stack overflow
3372         log_trace(gc)("CMS marking stack overflow (benign) at " SIZE_FORMAT, _overflow_stack->capacity());
3373         // We cannot assert that the overflow stack is full because
3374         // it may have been emptied since.
3375         assert(simulate_overflow ||
3376                _work_queue->size() == _work_queue->max_elems(),
3377               "Else push should have succeeded");
3378         handle_stack_overflow(addr);
3379       }
3380     } // Else, some other thread got there first
3381     do_yield_check();
3382   }
3383 }
3384 
3385 void ParConcMarkingClosure::do_oop(oop* p)       { ParConcMarkingClosure::do_oop_work(p); }
3386 void ParConcMarkingClosure::do_oop(narrowOop* p) { ParConcMarkingClosure::do_oop_work(p); }
3387 
3388 void ParConcMarkingClosure::trim_queue(size_t max) {
3389   while (_work_queue->size() > max) {
3390     oop new_oop;
3391     if (_work_queue->pop_local(new_oop)) {
3392       assert(oopDesc::is_oop(new_oop), "Should be an oop");
3393       assert(_bit_map->isMarked((HeapWord*)new_oop), "Grey object");
3394       assert(_span.contains((HeapWord*)new_oop), "Not in span");
3395       new_oop->oop_iterate(this);  // do_oop() above
3396       do_yield_check();
3397     }
3398   }
3399 }
3400 
3401 // Upon stack overflow, we discard (part of) the stack,
3402 // remembering the least address amongst those discarded
3403 // in CMSCollector's _restart_address.
3404 void ParConcMarkingClosure::handle_stack_overflow(HeapWord* lost) {
3405   // We need to do this under a mutex to prevent other
3406   // workers from interfering with the work done below.
3407   MutexLockerEx ml(_overflow_stack->par_lock(),


4048         assert((_collectorState == AbortablePreclean && should_abort_preclean()),
4049                "Should only be AbortablePreclean.");
4050         _ct->invalidate(MemRegion(stop_point, dirtyRegion.end()));
4051         if (should_abort_preclean()) {
4052           break; // out of preclean loop
4053         } else {
4054           // Compute the next address at which preclean should pick up.
4055           lastAddr = next_card_start_after_block(stop_point);
4056         }
4057       }
4058     } else {
4059       break;
4060     }
4061   }
4062   verify_work_stacks_empty();
4063   verify_overflow_empty();
4064   return cumNumDirtyCards;
4065 }
4066 
4067 class PrecleanCLDClosure : public CLDClosure {
4068   MetadataAwareOopsInGenClosure* _cm_closure;
4069  public:
4070   PrecleanCLDClosure(MetadataAwareOopsInGenClosure* oop_closure) : _cm_closure(oop_closure) {}
4071   void do_cld(ClassLoaderData* cld) {
4072     if (cld->has_accumulated_modified_oops()) {
4073       cld->clear_accumulated_modified_oops();
4074 
4075       _cm_closure->do_cld(cld);
4076     }
4077   }
4078 };
4079 
4080 // The freelist lock is needed to prevent asserts, is it really needed?
4081 void CMSCollector::preclean_cld(MarkRefsIntoAndScanClosure* cl, Mutex* freelistLock) {
4082 
4083   cl->set_freelistLock(freelistLock);
4084 
4085   CMSTokenSyncWithLocks ts(true, freelistLock, bitMapLock());
4086 
4087   // SSS: Add equivalent to ScanMarkedObjectsAgainCarefullyClosure::do_yield_check and should_abort_preclean?
4088   // SSS: We should probably check if precleaning should be aborted, at suitable intervals?
4089   PrecleanCLDClosure preclean_closure(cl);
4090   ClassLoaderDataGraph::cld_do(&preclean_closure);


4412                           &par_mrias_cl,
4413                           NULL,     // The dirty klasses will be handled below
4414                           &_par_state_string);
4415 
4416   assert(_collector->should_unload_classes()
4417          || (_collector->CMSCollector::roots_scanning_options() & GenCollectedHeap::SO_AllCodeCache),
4418          "if we didn't scan the code cache, we have to be ready to drop nmethods with expired weak oops");
4419   _timer.stop();
4420   log_trace(gc, task)("Finished remaining root rescan work in %dth thread: %3.3f sec",  worker_id, _timer.seconds());
4421 
4422   // ---------- unhandled CLD scanning ----------
4423   if (worker_id == 0) { // Single threaded at the moment.
4424     _timer.reset();
4425     _timer.start();
4426 
4427     // Scan all new class loader data objects and new dependencies that were
4428     // introduced during concurrent marking.
4429     ResourceMark rm;
4430     GrowableArray<ClassLoaderData*>* array = ClassLoaderDataGraph::new_clds();
4431     for (int i = 0; i < array->length(); i++) {
4432       par_mrias_cl.do_cld_nv(array->at(i));
4433     }
4434 
4435     // We don't need to keep track of new CLDs anymore.
4436     ClassLoaderDataGraph::remember_new_clds(false);
4437 
4438     _timer.stop();
4439     log_trace(gc, task)("Finished unhandled CLD scanning work in %dth thread: %3.3f sec", worker_id, _timer.seconds());
4440   }
4441 
4442   // We might have added oops to ClassLoaderData::_handles during the
4443   // concurrent marking phase. These oops do not always point to newly allocated objects
4444   // that are guaranteed to be kept alive.  Hence,
4445   // we do have to revisit the _handles block during the remark phase.
4446 
4447   // ---------- dirty CLD scanning ----------
4448   if (worker_id == 0) { // Single threaded at the moment.
4449     _timer.reset();
4450     _timer.start();
4451 
4452     // Scan all classes that was dirtied during the concurrent marking phase.


4953                             GenCollectedHeap::ScanningOption(roots_scanning_options()),
4954                             should_unload_classes(),
4955                             &mrias_cl,
4956                             NULL); // The dirty klasses will be handled below
4957 
4958     assert(should_unload_classes()
4959            || (roots_scanning_options() & GenCollectedHeap::SO_AllCodeCache),
4960            "if we didn't scan the code cache, we have to be ready to drop nmethods with expired weak oops");
4961   }
4962 
4963   {
4964     GCTraceTime(Trace, gc, phases) t("Visit Unhandled CLDs", _gc_timer_cm);
4965 
4966     verify_work_stacks_empty();
4967 
4968     // Scan all class loader data objects that might have been introduced
4969     // during concurrent marking.
4970     ResourceMark rm;
4971     GrowableArray<ClassLoaderData*>* array = ClassLoaderDataGraph::new_clds();
4972     for (int i = 0; i < array->length(); i++) {
4973       mrias_cl.do_cld_nv(array->at(i));
4974     }
4975 
4976     // We don't need to keep track of new CLDs anymore.
4977     ClassLoaderDataGraph::remember_new_clds(false);
4978 
4979     verify_work_stacks_empty();
4980   }
4981 
4982   // We might have added oops to ClassLoaderData::_handles during the
4983   // concurrent marking phase. These oops do not point to newly allocated objects
4984   // that are guaranteed to be kept alive.  Hence,
4985   // we do have to revisit the _handles block during the remark phase.
4986   {
4987     GCTraceTime(Trace, gc, phases) t("Dirty CLD Scan", _gc_timer_cm);
4988 
4989     verify_work_stacks_empty();
4990 
4991     RemarkCLDClosure remark_closure(&mrias_cl);
4992     ClassLoaderDataGraph::cld_do(&remark_closure);
4993 


5786 
5787 MarkRefsIntoClosure::MarkRefsIntoClosure(
5788   MemRegion span, CMSBitMap* bitMap):
5789     _span(span),
5790     _bitMap(bitMap)
5791 {
5792   assert(ref_discoverer() == NULL, "deliberately left NULL");
5793   assert(_bitMap->covers(_span), "_bitMap/_span mismatch");
5794 }
5795 
5796 void MarkRefsIntoClosure::do_oop(oop obj) {
5797   // if p points into _span, then mark corresponding bit in _markBitMap
5798   assert(oopDesc::is_oop(obj), "expected an oop");
5799   HeapWord* addr = (HeapWord*)obj;
5800   if (_span.contains(addr)) {
5801     // this should be made more efficient
5802     _bitMap->mark(addr);
5803   }
5804 }
5805 
5806 void MarkRefsIntoClosure::do_oop(oop* p)       { MarkRefsIntoClosure::do_oop_work(p); }
5807 void MarkRefsIntoClosure::do_oop(narrowOop* p) { MarkRefsIntoClosure::do_oop_work(p); }
5808 
5809 ParMarkRefsIntoClosure::ParMarkRefsIntoClosure(
5810   MemRegion span, CMSBitMap* bitMap):
5811     _span(span),
5812     _bitMap(bitMap)
5813 {
5814   assert(ref_discoverer() == NULL, "deliberately left NULL");
5815   assert(_bitMap->covers(_span), "_bitMap/_span mismatch");
5816 }
5817 
5818 void ParMarkRefsIntoClosure::do_oop(oop obj) {
5819   // if p points into _span, then mark corresponding bit in _markBitMap
5820   assert(oopDesc::is_oop(obj), "expected an oop");
5821   HeapWord* addr = (HeapWord*)obj;
5822   if (_span.contains(addr)) {
5823     // this should be made more efficient
5824     _bitMap->par_mark(addr);
5825   }
5826 }
5827 
5828 void ParMarkRefsIntoClosure::do_oop(oop* p)       { ParMarkRefsIntoClosure::do_oop_work(p); }
5829 void ParMarkRefsIntoClosure::do_oop(narrowOop* p) { ParMarkRefsIntoClosure::do_oop_work(p); }
5830 
5831 // A variant of the above, used for CMS marking verification.
5832 MarkRefsIntoVerifyClosure::MarkRefsIntoVerifyClosure(
5833   MemRegion span, CMSBitMap* verification_bm, CMSBitMap* cms_bm):
5834     _span(span),
5835     _verification_bm(verification_bm),
5836     _cms_bm(cms_bm)
5837 {
5838   assert(ref_discoverer() == NULL, "deliberately left NULL");
5839   assert(_verification_bm->covers(_span), "_verification_bm/_span mismatch");
5840 }
5841 
5842 void MarkRefsIntoVerifyClosure::do_oop(oop obj) {
5843   // if p points into _span, then mark corresponding bit in _markBitMap
5844   assert(oopDesc::is_oop(obj), "expected an oop");
5845   HeapWord* addr = (HeapWord*)obj;
5846   if (_span.contains(addr)) {
5847     _verification_bm->mark(addr);
5848     if (!_cms_bm->isMarked(addr)) {
5849       Log(gc, verify) log;
5850       ResourceMark rm;
5851       LogStream ls(log.error());
5852       oop(addr)->print_on(&ls);
5853       log.error(" (" INTPTR_FORMAT " should have been marked)", p2i(addr));
5854       fatal("... aborting");
5855     }
5856   }
5857 }
5858 
5859 void MarkRefsIntoVerifyClosure::do_oop(oop* p)       { MarkRefsIntoVerifyClosure::do_oop_work(p); }
5860 void MarkRefsIntoVerifyClosure::do_oop(narrowOop* p) { MarkRefsIntoVerifyClosure::do_oop_work(p); }
5861 
5862 //////////////////////////////////////////////////
5863 // MarkRefsIntoAndScanClosure
5864 //////////////////////////////////////////////////
5865 
5866 MarkRefsIntoAndScanClosure::MarkRefsIntoAndScanClosure(MemRegion span,
5867                                                        ReferenceDiscoverer* rd,
5868                                                        CMSBitMap* bit_map,
5869                                                        CMSBitMap* mod_union_table,
5870                                                        CMSMarkStack*  mark_stack,
5871                                                        CMSCollector* collector,
5872                                                        bool should_yield,
5873                                                        bool concurrent_precleaning):
5874   _collector(collector),
5875   _span(span),
5876   _bit_map(bit_map),
5877   _mark_stack(mark_stack),
5878   _pushAndMarkClosure(collector, span, rd, bit_map, mod_union_table,
5879                       mark_stack, concurrent_precleaning),
5880   _yield(should_yield),
5881   _concurrent_precleaning(concurrent_precleaning),


5916                "only grey objects on this stack");
5917         // iterate over the oops in this oop, marking and pushing
5918         // the ones in CMS heap (i.e. in _span).
5919         new_oop->oop_iterate(&_pushAndMarkClosure);
5920         // check if it's time to yield
5921         do_yield_check();
5922       } while (!_mark_stack->isEmpty() ||
5923                (!_concurrent_precleaning && take_from_overflow_list()));
5924         // if marking stack is empty, and we are not doing this
5925         // during precleaning, then check the overflow list
5926     }
5927     assert(_mark_stack->isEmpty(), "post-condition (eager drainage)");
5928     assert(_collector->overflow_list_is_empty(),
5929            "overflow list was drained above");
5930 
5931     assert(_collector->no_preserved_marks(),
5932            "All preserved marks should have been restored above");
5933   }
5934 }
5935 
5936 void MarkRefsIntoAndScanClosure::do_oop(oop* p)       { MarkRefsIntoAndScanClosure::do_oop_work(p); }
5937 void MarkRefsIntoAndScanClosure::do_oop(narrowOop* p) { MarkRefsIntoAndScanClosure::do_oop_work(p); }
5938 
5939 void MarkRefsIntoAndScanClosure::do_yield_work() {
5940   assert(ConcurrentMarkSweepThread::cms_thread_has_cms_token(),
5941          "CMS thread should hold CMS token");
5942   assert_lock_strong(_freelistLock);
5943   assert_lock_strong(_bit_map->lock());
5944   // relinquish the free_list_lock and bitMaplock()
5945   _bit_map->lock()->unlock();
5946   _freelistLock->unlock();
5947   ConcurrentMarkSweepThread::desynchronize(true);
5948   _collector->stopTimer();
5949   _collector->incrementYields();
5950 
5951   // See the comment in coordinator_yield()
5952   for (unsigned i = 0;
5953        i < CMSYieldSleepCount &&
5954        ConcurrentMarkSweepThread::should_yield() &&
5955        !CMSCollector::foregroundGCIsActive();
5956        ++i) {
5957     os::sleep(Thread::current(), 1, false);
5958   }


5999         !_bit_map->isMarked(addr)) {
6000       // mark bit map (object will become grey):
6001       // It is possible for several threads to be
6002       // trying to "claim" this object concurrently;
6003       // the unique thread that succeeds in marking the
6004       // object first will do the subsequent push on
6005       // to the work queue (or overflow list).
6006       if (_bit_map->par_mark(addr)) {
6007         // push on work_queue (which may not be empty), and trim the
6008         // queue to an appropriate length by applying this closure to
6009         // the oops in the oops popped from the stack (i.e. blacken the
6010         // grey objects)
6011         bool res = _work_queue->push(obj);
6012         assert(res, "Low water mark should be less than capacity?");
6013         trim_queue(_low_water_mark);
6014       } // Else, another thread claimed the object
6015     }
6016   }
6017 }
6018 
6019 void ParMarkRefsIntoAndScanClosure::do_oop(oop* p)       { ParMarkRefsIntoAndScanClosure::do_oop_work(p); }
6020 void ParMarkRefsIntoAndScanClosure::do_oop(narrowOop* p) { ParMarkRefsIntoAndScanClosure::do_oop_work(p); }
6021 
6022 // This closure is used to rescan the marked objects on the dirty cards
6023 // in the mod union table and the card table proper.
6024 size_t ScanMarkedObjectsAgainCarefullyClosure::do_object_careful_m(
6025   oop p, MemRegion mr) {
6026 
6027   size_t size = 0;
6028   HeapWord* addr = (HeapWord*)p;
6029   DEBUG_ONLY(_collector->verify_work_stacks_empty();)
6030   assert(_span.contains(addr), "we are scanning the CMS generation");
6031   // check if it's time to yield
6032   if (do_yield_check()) {
6033     // We yielded for some foreground stop-world work,
6034     // and we have been asked to abort this ongoing preclean cycle.
6035     return 0;
6036   }
6037   if (_bitMap->isMarked(addr)) {
6038     // it's marked; is it potentially uninitialized?
6039     if (p->klass_or_null_acquire() != NULL) {
6040         // an initialized object; ignore mark word in verification below
6041         // since we are running concurrent with mutators


6580   _finger = addr + obj->size();
6581   assert(_finger > addr, "we just incremented it above");
6582   // Note: the finger doesn't advance while we drain
6583   // the stack below.
6584   bool res = _mark_stack->push(obj);
6585   assert(res, "Empty non-zero size stack should have space for single push");
6586   while (!_mark_stack->isEmpty()) {
6587     oop new_oop = _mark_stack->pop();
6588     assert(oopDesc::is_oop(new_oop), "Oops! expected to pop an oop");
6589     // now scan this oop's oops
6590     new_oop->oop_iterate(&_pam_verify_closure);
6591   }
6592   assert(_mark_stack->isEmpty(), "tautology, emphasizing post-condition");
6593   return true;
6594 }
6595 
6596 PushAndMarkVerifyClosure::PushAndMarkVerifyClosure(
6597   CMSCollector* collector, MemRegion span,
6598   CMSBitMap* verification_bm, CMSBitMap* cms_bm,
6599   CMSMarkStack*  mark_stack):
6600   MetadataAwareOopClosure(collector->ref_processor()),
6601   _collector(collector),
6602   _span(span),
6603   _verification_bm(verification_bm),
6604   _cms_bm(cms_bm),
6605   _mark_stack(mark_stack)
6606 { }
6607 
6608 template <class T> void PushAndMarkVerifyClosure::do_oop_work(T *p) {
6609   oop obj = RawAccess<>::oop_load(p);
6610   do_oop(obj);
6611 }
6612 
6613 void PushAndMarkVerifyClosure::do_oop(oop* p)       { PushAndMarkVerifyClosure::do_oop_work(p); }
6614 void PushAndMarkVerifyClosure::do_oop(narrowOop* p) { PushAndMarkVerifyClosure::do_oop_work(p); }
6615 
6616 // Upon stack overflow, we discard (part of) the stack,
6617 // remembering the least address amongst those discarded
6618 // in CMSCollector's _restart_address.
6619 void PushAndMarkVerifyClosure::handle_stack_overflow(HeapWord* lost) {
6620   // Remember the least grey address discarded


6637       oop(addr)->print_on(&ls);
6638       log.error(" (" INTPTR_FORMAT " should have been marked)", p2i(addr));
6639       fatal("... aborting");
6640     }
6641 
6642     if (!_mark_stack->push(obj)) { // stack overflow
6643       log_trace(gc)("CMS marking stack overflow (benign) at " SIZE_FORMAT, _mark_stack->capacity());
6644       assert(_mark_stack->isFull(), "Else push should have succeeded");
6645       handle_stack_overflow(addr);
6646     }
6647     // anything including and to the right of _finger
6648     // will be scanned as we iterate over the remainder of the
6649     // bit map
6650   }
6651 }
6652 
6653 PushOrMarkClosure::PushOrMarkClosure(CMSCollector* collector,
6654                      MemRegion span,
6655                      CMSBitMap* bitMap, CMSMarkStack*  markStack,
6656                      HeapWord* finger, MarkFromRootsClosure* parent) :
6657   MetadataAwareOopClosure(collector->ref_processor()),
6658   _collector(collector),
6659   _span(span),
6660   _bitMap(bitMap),
6661   _markStack(markStack),
6662   _finger(finger),
6663   _parent(parent)
6664 { }
6665 
6666 ParPushOrMarkClosure::ParPushOrMarkClosure(CMSCollector* collector,
6667                                            MemRegion span,
6668                                            CMSBitMap* bit_map,
6669                                            OopTaskQueue* work_queue,
6670                                            CMSMarkStack*  overflow_stack,
6671                                            HeapWord* finger,
6672                                            HeapWord* volatile* global_finger_addr,
6673                                            ParMarkFromRootsClosure* parent) :
6674   MetadataAwareOopClosure(collector->ref_processor()),
6675   _collector(collector),
6676   _whole_span(collector->_span),
6677   _span(span),
6678   _bit_map(bit_map),
6679   _work_queue(work_queue),
6680   _overflow_stack(overflow_stack),
6681   _finger(finger),
6682   _global_finger_addr(global_finger_addr),
6683   _parent(parent)
6684 { }
6685 
6686 // Assumes thread-safe access by callers, who are
6687 // responsible for mutual exclusion.
6688 void CMSCollector::lower_restart_addr(HeapWord* low) {
6689   assert(_span.contains(low), "Out of bounds addr");
6690   if (_restart_addr == NULL) {
6691     _restart_addr = low;
6692   } else {
6693     _restart_addr = MIN2(_restart_addr, low);
6694   }


6735       NOT_PRODUCT(
6736         if (CMSMarkStackOverflowALot &&
6737             _collector->simulate_overflow()) {
6738           // simulate a stack overflow
6739           simulate_overflow = true;
6740         }
6741       )
6742       if (simulate_overflow || !_markStack->push(obj)) { // stack overflow
6743         log_trace(gc)("CMS marking stack overflow (benign) at " SIZE_FORMAT, _markStack->capacity());
6744         assert(simulate_overflow || _markStack->isFull(), "Else push should have succeeded");
6745         handle_stack_overflow(addr);
6746       }
6747     }
6748     // anything including and to the right of _finger
6749     // will be scanned as we iterate over the remainder of the
6750     // bit map
6751     do_yield_check();
6752   }
6753 }
6754 
6755 void PushOrMarkClosure::do_oop(oop* p)       { PushOrMarkClosure::do_oop_work(p); }
6756 void PushOrMarkClosure::do_oop(narrowOop* p) { PushOrMarkClosure::do_oop_work(p); }
6757 
6758 void ParPushOrMarkClosure::do_oop(oop obj) {
6759   // Ignore mark word because we are running concurrent with mutators.
6760   assert(oopDesc::is_oop_or_null(obj, true), "Expected an oop or NULL at " PTR_FORMAT, p2i(obj));
6761   HeapWord* addr = (HeapWord*)obj;
6762   if (_whole_span.contains(addr) && !_bit_map->isMarked(addr)) {
6763     // Oop lies in _span and isn't yet grey or black
6764     // We read the global_finger (volatile read) strictly after marking oop
6765     bool res = _bit_map->par_mark(addr);    // now grey
6766     volatile HeapWord** gfa = (volatile HeapWord**)_global_finger_addr;
6767     // Should we push this marked oop on our stack?
6768     // -- if someone else marked it, nothing to do
6769     // -- if target oop is above global finger nothing to do
6770     // -- if target oop is in chunk and above local finger
6771     //      then nothing to do
6772     // -- else push on work queue
6773     if (   !res       // someone else marked it, they will deal with it
6774         || (addr >= *gfa)  // will be scanned in a later task
6775         || (_span.contains(addr) && addr >= _finger)) { // later in this chunk
6776       return;
6777     }


6784           _collector->simulate_overflow()) {
6785         // simulate a stack overflow
6786         simulate_overflow = true;
6787       }
6788     )
6789     if (simulate_overflow ||
6790         !(_work_queue->push(obj) || _overflow_stack->par_push(obj))) {
6791       // stack overflow
6792       log_trace(gc)("CMS marking stack overflow (benign) at " SIZE_FORMAT, _overflow_stack->capacity());
6793       // We cannot assert that the overflow stack is full because
6794       // it may have been emptied since.
6795       assert(simulate_overflow ||
6796              _work_queue->size() == _work_queue->max_elems(),
6797             "Else push should have succeeded");
6798       handle_stack_overflow(addr);
6799     }
6800     do_yield_check();
6801   }
6802 }
6803 
6804 void ParPushOrMarkClosure::do_oop(oop* p)       { ParPushOrMarkClosure::do_oop_work(p); }
6805 void ParPushOrMarkClosure::do_oop(narrowOop* p) { ParPushOrMarkClosure::do_oop_work(p); }
6806 
6807 PushAndMarkClosure::PushAndMarkClosure(CMSCollector* collector,
6808                                        MemRegion span,
6809                                        ReferenceDiscoverer* rd,
6810                                        CMSBitMap* bit_map,
6811                                        CMSBitMap* mod_union_table,
6812                                        CMSMarkStack*  mark_stack,
6813                                        bool           concurrent_precleaning):
6814   MetadataAwareOopClosure(rd),
6815   _collector(collector),
6816   _span(span),
6817   _bit_map(bit_map),
6818   _mod_union_table(mod_union_table),
6819   _mark_stack(mark_stack),
6820   _concurrent_precleaning(concurrent_precleaning)
6821 {
6822   assert(ref_discoverer() != NULL, "ref_discoverer shouldn't be NULL");
6823 }
6824 
6825 // Grey object rescan during pre-cleaning and second checkpoint phases --
6826 // the non-parallel version (the parallel version appears further below.)
6827 void PushAndMarkClosure::do_oop(oop obj) {
6828   // Ignore mark word verification. If during concurrent precleaning,
6829   // the object monitor may be locked. If during the checkpoint
6830   // phases, the object may already have been reached by a  different
6831   // path and may be at the end of the global overflow list (so
6832   // the mark word may be NULL).
6833   assert(oopDesc::is_oop_or_null(obj, true /* ignore mark word */),
6834          "Expected an oop or NULL at " PTR_FORMAT, p2i(obj));


6866            _mod_union_table->mark_range(redirty_range);
6867          } else {
6868            _mod_union_table->mark(addr);
6869          }
6870          _collector->_ser_pmc_preclean_ovflw++;
6871       } else {
6872          // During the remark phase, we need to remember this oop
6873          // in the overflow list.
6874          _collector->push_on_overflow_list(obj);
6875          _collector->_ser_pmc_remark_ovflw++;
6876       }
6877     }
6878   }
6879 }
6880 
6881 ParPushAndMarkClosure::ParPushAndMarkClosure(CMSCollector* collector,
6882                                              MemRegion span,
6883                                              ReferenceDiscoverer* rd,
6884                                              CMSBitMap* bit_map,
6885                                              OopTaskQueue* work_queue):
6886   MetadataAwareOopClosure(rd),
6887   _collector(collector),
6888   _span(span),
6889   _bit_map(bit_map),
6890   _work_queue(work_queue)
6891 {
6892   assert(ref_discoverer() != NULL, "ref_discoverer shouldn't be NULL");
6893 }
6894 
6895 void PushAndMarkClosure::do_oop(oop* p)       { PushAndMarkClosure::do_oop_work(p); }
6896 void PushAndMarkClosure::do_oop(narrowOop* p) { PushAndMarkClosure::do_oop_work(p); }
6897 
6898 // Grey object rescan during second checkpoint phase --
6899 // the parallel version.
6900 void ParPushAndMarkClosure::do_oop(oop obj) {
6901   // In the assert below, we ignore the mark word because
6902   // this oop may point to an already visited object that is
6903   // on the overflow stack (in which case the mark word has
6904   // been hijacked for chaining into the overflow stack --
6905   // if this is the last object in the overflow stack then
6906   // its mark word will be NULL). Because this object may
6907   // have been subsequently popped off the global overflow
6908   // stack, and the mark word possibly restored to the prototypical
6909   // value, by the time we get to examined this failing assert in
6910   // the debugger, is_oop_or_null(false) may subsequently start
6911   // to hold.
6912   assert(oopDesc::is_oop_or_null(obj, true),
6913          "Expected an oop or NULL at " PTR_FORMAT, p2i(obj));
6914   HeapWord* addr = (HeapWord*)obj;
6915   // Check if oop points into the CMS generation
6916   // and is not marked
6917   if (_span.contains(addr) && !_bit_map->isMarked(addr)) {


6920     // first thread to mark it, then we push it on our
6921     // marking stack
6922     if (_bit_map->par_mark(addr)) {     // ... now grey
6923       // push on work queue (grey set)
6924       bool simulate_overflow = false;
6925       NOT_PRODUCT(
6926         if (CMSMarkStackOverflowALot &&
6927             _collector->par_simulate_overflow()) {
6928           // simulate a stack overflow
6929           simulate_overflow = true;
6930         }
6931       )
6932       if (simulate_overflow || !_work_queue->push(obj)) {
6933         _collector->par_push_on_overflow_list(obj);
6934         _collector->_par_pmc_remark_ovflw++; //  imprecise OK: no need to CAS
6935       }
6936     } // Else, some other thread got there first
6937   }
6938 }
6939 
6940 void ParPushAndMarkClosure::do_oop(oop* p)       { ParPushAndMarkClosure::do_oop_work(p); }
6941 void ParPushAndMarkClosure::do_oop(narrowOop* p) { ParPushAndMarkClosure::do_oop_work(p); }
6942 
6943 void CMSPrecleanRefsYieldClosure::do_yield_work() {
6944   Mutex* bml = _collector->bitMapLock();
6945   assert_lock_strong(bml);
6946   assert(ConcurrentMarkSweepThread::cms_thread_has_cms_token(),
6947          "CMS thread should hold CMS token");
6948 
6949   bml->unlock();
6950   ConcurrentMarkSweepThread::desynchronize(true);
6951 
6952   _collector->stopTimer();
6953   _collector->incrementYields();
6954 
6955   // See the comment in coordinator_yield()
6956   for (unsigned i = 0; i < CMSYieldSleepCount &&
6957                        ConcurrentMarkSweepThread::should_yield() &&
6958                        !CMSCollector::foregroundGCIsActive(); ++i) {
6959     os::sleep(Thread::current(), 1, false);
6960   }
6961 
6962   ConcurrentMarkSweepThread::synchronize(true);


7589         // are needed since no one else can be mutating the mod union
7590         // table.
7591         if (obj->is_objArray()) {
7592           size_t sz = obj->size();
7593           HeapWord* end_card_addr = align_up(addr + sz, CardTable::card_size);
7594           MemRegion redirty_range = MemRegion(addr, end_card_addr);
7595           assert(!redirty_range.is_empty(), "Arithmetical tautology");
7596           _collector->_modUnionTable.mark_range(redirty_range);
7597         } else {
7598           _collector->_modUnionTable.mark(addr);
7599         }
7600         _collector->_ser_kac_preclean_ovflw++;
7601       } else {
7602         _collector->push_on_overflow_list(obj);
7603         _collector->_ser_kac_ovflw++;
7604       }
7605     }
7606   }
7607 }
7608 
7609 void CMSKeepAliveClosure::do_oop(oop* p)       { CMSKeepAliveClosure::do_oop_work(p); }
7610 void CMSKeepAliveClosure::do_oop(narrowOop* p) { CMSKeepAliveClosure::do_oop_work(p); }
7611 
7612 // CMSParKeepAliveClosure: a parallel version of the above.
7613 // The work queues are private to each closure (thread),
7614 // but (may be) available for stealing by other threads.
7615 void CMSParKeepAliveClosure::do_oop(oop obj) {
7616   HeapWord* addr = (HeapWord*)obj;
7617   if (_span.contains(addr) &&
7618       !_bit_map->isMarked(addr)) {
7619     // In general, during recursive tracing, several threads
7620     // may be concurrently getting here; the first one to
7621     // "tag" it, claims it.
7622     if (_bit_map->par_mark(addr)) {
7623       bool res = _work_queue->push(obj);
7624       assert(res, "Low water mark should be much less than capacity");
7625       // Do a recursive trim in the hope that this will keep
7626       // stack usage lower, but leave some oops for potential stealers
7627       trim_queue(_low_water_mark);
7628     } // Else, another thread got there first
7629   }
7630 }
7631 
7632 void CMSParKeepAliveClosure::do_oop(oop* p)       { CMSParKeepAliveClosure::do_oop_work(p); }
7633 void CMSParKeepAliveClosure::do_oop(narrowOop* p) { CMSParKeepAliveClosure::do_oop_work(p); }
7634 
7635 void CMSParKeepAliveClosure::trim_queue(uint max) {
7636   while (_work_queue->size() > max) {
7637     oop new_oop;
7638     if (_work_queue->pop_local(new_oop)) {
7639       assert(new_oop != NULL && oopDesc::is_oop(new_oop), "Expected an oop");
7640       assert(_bit_map->isMarked((HeapWord*)new_oop),
7641              "no white objects on this stack!");
7642       assert(_span.contains((HeapWord*)new_oop), "Out of bounds oop");
7643       // iterate over the oops in this oop, marking and pushing
7644       // the ones in CMS heap (i.e. in _span).
7645       new_oop->oop_iterate(&_mark_and_push);
7646     }
7647   }
7648 }
7649 
7650 CMSInnerParMarkAndPushClosure::CMSInnerParMarkAndPushClosure(
7651                                 CMSCollector* collector,
7652                                 MemRegion span, CMSBitMap* bit_map,
7653                                 OopTaskQueue* work_queue):
7654   _collector(collector),


7659 void CMSInnerParMarkAndPushClosure::do_oop(oop obj) {
7660   HeapWord* addr = (HeapWord*)obj;
7661   if (_span.contains(addr) &&
7662       !_bit_map->isMarked(addr)) {
7663     if (_bit_map->par_mark(addr)) {
7664       bool simulate_overflow = false;
7665       NOT_PRODUCT(
7666         if (CMSMarkStackOverflowALot &&
7667             _collector->par_simulate_overflow()) {
7668           // simulate a stack overflow
7669           simulate_overflow = true;
7670         }
7671       )
7672       if (simulate_overflow || !_work_queue->push(obj)) {
7673         _collector->par_push_on_overflow_list(obj);
7674         _collector->_par_kac_ovflw++;
7675       }
7676     } // Else another thread got there already
7677   }
7678 }
7679 
7680 void CMSInnerParMarkAndPushClosure::do_oop(oop* p)       { CMSInnerParMarkAndPushClosure::do_oop_work(p); }
7681 void CMSInnerParMarkAndPushClosure::do_oop(narrowOop* p) { CMSInnerParMarkAndPushClosure::do_oop_work(p); }
7682 
7683 //////////////////////////////////////////////////////////////////
7684 //  CMSExpansionCause                /////////////////////////////
7685 //////////////////////////////////////////////////////////////////
7686 const char* CMSExpansionCause::to_string(CMSExpansionCause::Cause cause) {
7687   switch (cause) {
7688     case _no_expansion:
7689       return "No expansion";
7690     case _satisfy_free_ratio:
7691       return "Free ratio";
7692     case _satisfy_promotion:
7693       return "Satisfy promotion";
7694     case _satisfy_allocation:
7695       return "allocation";
7696     case _allocate_par_lab:
7697       return "Par LAB";
7698     case _allocate_par_spooling_space:
7699       return "Par Spooling Space";
7700     case _adaptive_size_policy:
7701       return "Ergonomics";




2450   ClassLoaderDataGraph::cld_do(&verify_cld_oops);
2451 
2452   // Marking completed -- now verify that each bit marked in
2453   // verification_mark_bm() is also marked in markBitMap(); flag all
2454   // errors by printing corresponding objects.
2455   VerifyMarkedClosure vcl(markBitMap());
2456   verification_mark_bm()->iterate(&vcl);
2457   assert(!vcl.failed(), "Else verification above should not have succeeded");
2458 }
2459 
2460 void ConcurrentMarkSweepGeneration::save_marks() {
2461   // delegate to CMS space
2462   cmsSpace()->save_marks();
2463 }
2464 
2465 bool ConcurrentMarkSweepGeneration::no_allocs_since_save_marks() {
2466   return cmsSpace()->no_allocs_since_save_marks();
2467 }
2468 
2469 void
2470 ConcurrentMarkSweepGeneration::oop_iterate(OopIterateClosure* cl) {
2471   if (freelistLock()->owned_by_self()) {
2472     Generation::oop_iterate(cl);
2473   } else {
2474     MutexLockerEx x(freelistLock(), Mutex::_no_safepoint_check_flag);
2475     Generation::oop_iterate(cl);
2476   }
2477 }
2478 
2479 void
2480 ConcurrentMarkSweepGeneration::object_iterate(ObjectClosure* cl) {
2481   if (freelistLock()->owned_by_self()) {
2482     Generation::object_iterate(cl);
2483   } else {
2484     MutexLockerEx x(freelistLock(), Mutex::_no_safepoint_check_flag);
2485     Generation::object_iterate(cl);
2486   }
2487 }
2488 
2489 void
2490 ConcurrentMarkSweepGeneration::safe_object_iterate(ObjectClosure* cl) {


3288         // iteration should be incremental with periodic yields.
3289         ParMarkFromRootsClosure cl(this, _collector, my_span,
3290                                    &_collector->_markBitMap,
3291                                    work_queue(i),
3292                                    &_collector->_markStack);
3293         _collector->_markBitMap.iterate(&cl, my_span.start(), my_span.end());
3294       } // else nothing to do for this task
3295     }   // else nothing to do for this task
3296   }
3297   // We'd be tempted to assert here that since there are no
3298   // more tasks left to claim in this space, the global_finger
3299   // must exceed space->top() and a fortiori space->end(). However,
3300   // that would not quite be correct because the bumping of
3301   // global_finger occurs strictly after the claiming of a task,
3302   // so by the time we reach here the global finger may not yet
3303   // have been bumped up by the thread that claimed the last
3304   // task.
3305   pst->all_tasks_completed();
3306 }
3307 
3308 class ParConcMarkingClosure: public MetadataVisitingOopIterateClosure {
3309  private:
3310   CMSCollector* _collector;
3311   CMSConcMarkingTask* _task;
3312   MemRegion     _span;
3313   CMSBitMap*    _bit_map;
3314   CMSMarkStack* _overflow_stack;
3315   OopTaskQueue* _work_queue;
3316  protected:
3317   DO_OOP_WORK_DEFN
3318  public:
3319   ParConcMarkingClosure(CMSCollector* collector, CMSConcMarkingTask* task, OopTaskQueue* work_queue,
3320                         CMSBitMap* bit_map, CMSMarkStack* overflow_stack):
3321                         MetadataVisitingOopIterateClosure(collector->ref_processor()),
3322     _collector(collector),
3323     _task(task),
3324     _span(collector->_span),
3325     _work_queue(work_queue),
3326     _bit_map(bit_map),
3327     _overflow_stack(overflow_stack)
3328   { }
3329   virtual void do_oop(oop* p);
3330   virtual void do_oop(narrowOop* p);
3331 
3332   void trim_queue(size_t max);
3333   void handle_stack_overflow(HeapWord* lost);
3334   void do_yield_check() {
3335     if (_task->should_yield()) {
3336       _task->yield();
3337     }
3338   }
3339 };
3340 
3341 DO_OOP_WORK_NV_IMPL(ParConcMarkingClosure)
3342 
3343 // Grey object scanning during work stealing phase --
3344 // the salient assumption here is that any references
3345 // that are in these stolen objects being scanned must
3346 // already have been initialized (else they would not have
3347 // been published), so we do not need to check for
3348 // uninitialized objects before pushing here.
3349 void ParConcMarkingClosure::do_oop(oop obj) {
3350   assert(oopDesc::is_oop_or_null(obj, true), "Expected an oop or NULL at " PTR_FORMAT, p2i(obj));
3351   HeapWord* addr = (HeapWord*)obj;
3352   // Check if oop points into the CMS generation
3353   // and is not marked
3354   if (_span.contains(addr) && !_bit_map->isMarked(addr)) {
3355     // a white object ...
3356     // If we manage to "claim" the object, by being the
3357     // first thread to mark it, then we push it on our
3358     // marking stack
3359     if (_bit_map->par_mark(addr)) {     // ... now grey
3360       // push on work queue (grey set)
3361       bool simulate_overflow = false;


3365           // simulate a stack overflow
3366           simulate_overflow = true;
3367         }
3368       )
3369       if (simulate_overflow ||
3370           !(_work_queue->push(obj) || _overflow_stack->par_push(obj))) {
3371         // stack overflow
3372         log_trace(gc)("CMS marking stack overflow (benign) at " SIZE_FORMAT, _overflow_stack->capacity());
3373         // We cannot assert that the overflow stack is full because
3374         // it may have been emptied since.
3375         assert(simulate_overflow ||
3376                _work_queue->size() == _work_queue->max_elems(),
3377               "Else push should have succeeded");
3378         handle_stack_overflow(addr);
3379       }
3380     } // Else, some other thread got there first
3381     do_yield_check();
3382   }
3383 }
3384 



3385 void ParConcMarkingClosure::trim_queue(size_t max) {
3386   while (_work_queue->size() > max) {
3387     oop new_oop;
3388     if (_work_queue->pop_local(new_oop)) {
3389       assert(oopDesc::is_oop(new_oop), "Should be an oop");
3390       assert(_bit_map->isMarked((HeapWord*)new_oop), "Grey object");
3391       assert(_span.contains((HeapWord*)new_oop), "Not in span");
3392       new_oop->oop_iterate(this);  // do_oop() above
3393       do_yield_check();
3394     }
3395   }
3396 }
3397 
3398 // Upon stack overflow, we discard (part of) the stack,
3399 // remembering the least address amongst those discarded
3400 // in CMSCollector's _restart_address.
3401 void ParConcMarkingClosure::handle_stack_overflow(HeapWord* lost) {
3402   // We need to do this under a mutex to prevent other
3403   // workers from interfering with the work done below.
3404   MutexLockerEx ml(_overflow_stack->par_lock(),


4045         assert((_collectorState == AbortablePreclean && should_abort_preclean()),
4046                "Should only be AbortablePreclean.");
4047         _ct->invalidate(MemRegion(stop_point, dirtyRegion.end()));
4048         if (should_abort_preclean()) {
4049           break; // out of preclean loop
4050         } else {
4051           // Compute the next address at which preclean should pick up.
4052           lastAddr = next_card_start_after_block(stop_point);
4053         }
4054       }
4055     } else {
4056       break;
4057     }
4058   }
4059   verify_work_stacks_empty();
4060   verify_overflow_empty();
4061   return cumNumDirtyCards;
4062 }
4063 
4064 class PrecleanCLDClosure : public CLDClosure {
4065   MetadataVisitingOopsInGenClosure* _cm_closure;
4066  public:
4067   PrecleanCLDClosure(MetadataVisitingOopsInGenClosure* oop_closure) : _cm_closure(oop_closure) {}
4068   void do_cld(ClassLoaderData* cld) {
4069     if (cld->has_accumulated_modified_oops()) {
4070       cld->clear_accumulated_modified_oops();
4071 
4072       _cm_closure->do_cld(cld);
4073     }
4074   }
4075 };
4076 
4077 // The freelist lock is needed to prevent asserts, is it really needed?
4078 void CMSCollector::preclean_cld(MarkRefsIntoAndScanClosure* cl, Mutex* freelistLock) {
4079 
4080   cl->set_freelistLock(freelistLock);
4081 
4082   CMSTokenSyncWithLocks ts(true, freelistLock, bitMapLock());
4083 
4084   // SSS: Add equivalent to ScanMarkedObjectsAgainCarefullyClosure::do_yield_check and should_abort_preclean?
4085   // SSS: We should probably check if precleaning should be aborted, at suitable intervals?
4086   PrecleanCLDClosure preclean_closure(cl);
4087   ClassLoaderDataGraph::cld_do(&preclean_closure);


4409                           &par_mrias_cl,
4410                           NULL,     // The dirty klasses will be handled below
4411                           &_par_state_string);
4412 
4413   assert(_collector->should_unload_classes()
4414          || (_collector->CMSCollector::roots_scanning_options() & GenCollectedHeap::SO_AllCodeCache),
4415          "if we didn't scan the code cache, we have to be ready to drop nmethods with expired weak oops");
4416   _timer.stop();
4417   log_trace(gc, task)("Finished remaining root rescan work in %dth thread: %3.3f sec",  worker_id, _timer.seconds());
4418 
4419   // ---------- unhandled CLD scanning ----------
4420   if (worker_id == 0) { // Single threaded at the moment.
4421     _timer.reset();
4422     _timer.start();
4423 
4424     // Scan all new class loader data objects and new dependencies that were
4425     // introduced during concurrent marking.
4426     ResourceMark rm;
4427     GrowableArray<ClassLoaderData*>* array = ClassLoaderDataGraph::new_clds();
4428     for (int i = 0; i < array->length(); i++) {
4429       Devirtualizer::do_cld(&par_mrias_cl, array->at(i));
4430     }
4431 
4432     // We don't need to keep track of new CLDs anymore.
4433     ClassLoaderDataGraph::remember_new_clds(false);
4434 
4435     _timer.stop();
4436     log_trace(gc, task)("Finished unhandled CLD scanning work in %dth thread: %3.3f sec", worker_id, _timer.seconds());
4437   }
4438 
4439   // We might have added oops to ClassLoaderData::_handles during the
4440   // concurrent marking phase. These oops do not always point to newly allocated objects
4441   // that are guaranteed to be kept alive.  Hence,
4442   // we do have to revisit the _handles block during the remark phase.
4443 
4444   // ---------- dirty CLD scanning ----------
4445   if (worker_id == 0) { // Single threaded at the moment.
4446     _timer.reset();
4447     _timer.start();
4448 
4449     // Scan all classes that was dirtied during the concurrent marking phase.


4950                             GenCollectedHeap::ScanningOption(roots_scanning_options()),
4951                             should_unload_classes(),
4952                             &mrias_cl,
4953                             NULL); // The dirty klasses will be handled below
4954 
4955     assert(should_unload_classes()
4956            || (roots_scanning_options() & GenCollectedHeap::SO_AllCodeCache),
4957            "if we didn't scan the code cache, we have to be ready to drop nmethods with expired weak oops");
4958   }
4959 
4960   {
4961     GCTraceTime(Trace, gc, phases) t("Visit Unhandled CLDs", _gc_timer_cm);
4962 
4963     verify_work_stacks_empty();
4964 
4965     // Scan all class loader data objects that might have been introduced
4966     // during concurrent marking.
4967     ResourceMark rm;
4968     GrowableArray<ClassLoaderData*>* array = ClassLoaderDataGraph::new_clds();
4969     for (int i = 0; i < array->length(); i++) {
4970       Devirtualizer::do_cld(&mrias_cl, array->at(i));
4971     }
4972 
4973     // We don't need to keep track of new CLDs anymore.
4974     ClassLoaderDataGraph::remember_new_clds(false);
4975 
4976     verify_work_stacks_empty();
4977   }
4978 
4979   // We might have added oops to ClassLoaderData::_handles during the
4980   // concurrent marking phase. These oops do not point to newly allocated objects
4981   // that are guaranteed to be kept alive.  Hence,
4982   // we do have to revisit the _handles block during the remark phase.
4983   {
4984     GCTraceTime(Trace, gc, phases) t("Dirty CLD Scan", _gc_timer_cm);
4985 
4986     verify_work_stacks_empty();
4987 
4988     RemarkCLDClosure remark_closure(&mrias_cl);
4989     ClassLoaderDataGraph::cld_do(&remark_closure);
4990 


5783 
5784 MarkRefsIntoClosure::MarkRefsIntoClosure(
5785   MemRegion span, CMSBitMap* bitMap):
5786     _span(span),
5787     _bitMap(bitMap)
5788 {
5789   assert(ref_discoverer() == NULL, "deliberately left NULL");
5790   assert(_bitMap->covers(_span), "_bitMap/_span mismatch");
5791 }
5792 
5793 void MarkRefsIntoClosure::do_oop(oop obj) {
5794   // if p points into _span, then mark corresponding bit in _markBitMap
5795   assert(oopDesc::is_oop(obj), "expected an oop");
5796   HeapWord* addr = (HeapWord*)obj;
5797   if (_span.contains(addr)) {
5798     // this should be made more efficient
5799     _bitMap->mark(addr);
5800   }
5801 }
5802 



5803 ParMarkRefsIntoClosure::ParMarkRefsIntoClosure(
5804   MemRegion span, CMSBitMap* bitMap):
5805     _span(span),
5806     _bitMap(bitMap)
5807 {
5808   assert(ref_discoverer() == NULL, "deliberately left NULL");
5809   assert(_bitMap->covers(_span), "_bitMap/_span mismatch");
5810 }
5811 
5812 void ParMarkRefsIntoClosure::do_oop(oop obj) {
5813   // if p points into _span, then mark corresponding bit in _markBitMap
5814   assert(oopDesc::is_oop(obj), "expected an oop");
5815   HeapWord* addr = (HeapWord*)obj;
5816   if (_span.contains(addr)) {
5817     // this should be made more efficient
5818     _bitMap->par_mark(addr);
5819   }
5820 }
5821 



5822 // A variant of the above, used for CMS marking verification.
5823 MarkRefsIntoVerifyClosure::MarkRefsIntoVerifyClosure(
5824   MemRegion span, CMSBitMap* verification_bm, CMSBitMap* cms_bm):
5825     _span(span),
5826     _verification_bm(verification_bm),
5827     _cms_bm(cms_bm)
5828 {
5829   assert(ref_discoverer() == NULL, "deliberately left NULL");
5830   assert(_verification_bm->covers(_span), "_verification_bm/_span mismatch");
5831 }
5832 
5833 void MarkRefsIntoVerifyClosure::do_oop(oop obj) {
5834   // if p points into _span, then mark corresponding bit in _markBitMap
5835   assert(oopDesc::is_oop(obj), "expected an oop");
5836   HeapWord* addr = (HeapWord*)obj;
5837   if (_span.contains(addr)) {
5838     _verification_bm->mark(addr);
5839     if (!_cms_bm->isMarked(addr)) {
5840       Log(gc, verify) log;
5841       ResourceMark rm;
5842       LogStream ls(log.error());
5843       oop(addr)->print_on(&ls);
5844       log.error(" (" INTPTR_FORMAT " should have been marked)", p2i(addr));
5845       fatal("... aborting");
5846     }
5847   }
5848 }
5849 



5850 //////////////////////////////////////////////////
5851 // MarkRefsIntoAndScanClosure
5852 //////////////////////////////////////////////////
5853 
5854 MarkRefsIntoAndScanClosure::MarkRefsIntoAndScanClosure(MemRegion span,
5855                                                        ReferenceDiscoverer* rd,
5856                                                        CMSBitMap* bit_map,
5857                                                        CMSBitMap* mod_union_table,
5858                                                        CMSMarkStack*  mark_stack,
5859                                                        CMSCollector* collector,
5860                                                        bool should_yield,
5861                                                        bool concurrent_precleaning):
5862   _collector(collector),
5863   _span(span),
5864   _bit_map(bit_map),
5865   _mark_stack(mark_stack),
5866   _pushAndMarkClosure(collector, span, rd, bit_map, mod_union_table,
5867                       mark_stack, concurrent_precleaning),
5868   _yield(should_yield),
5869   _concurrent_precleaning(concurrent_precleaning),


5904                "only grey objects on this stack");
5905         // iterate over the oops in this oop, marking and pushing
5906         // the ones in CMS heap (i.e. in _span).
5907         new_oop->oop_iterate(&_pushAndMarkClosure);
5908         // check if it's time to yield
5909         do_yield_check();
5910       } while (!_mark_stack->isEmpty() ||
5911                (!_concurrent_precleaning && take_from_overflow_list()));
5912         // if marking stack is empty, and we are not doing this
5913         // during precleaning, then check the overflow list
5914     }
5915     assert(_mark_stack->isEmpty(), "post-condition (eager drainage)");
5916     assert(_collector->overflow_list_is_empty(),
5917            "overflow list was drained above");
5918 
5919     assert(_collector->no_preserved_marks(),
5920            "All preserved marks should have been restored above");
5921   }
5922 }
5923 



5924 void MarkRefsIntoAndScanClosure::do_yield_work() {
5925   assert(ConcurrentMarkSweepThread::cms_thread_has_cms_token(),
5926          "CMS thread should hold CMS token");
5927   assert_lock_strong(_freelistLock);
5928   assert_lock_strong(_bit_map->lock());
5929   // relinquish the free_list_lock and bitMaplock()
5930   _bit_map->lock()->unlock();
5931   _freelistLock->unlock();
5932   ConcurrentMarkSweepThread::desynchronize(true);
5933   _collector->stopTimer();
5934   _collector->incrementYields();
5935 
5936   // See the comment in coordinator_yield()
5937   for (unsigned i = 0;
5938        i < CMSYieldSleepCount &&
5939        ConcurrentMarkSweepThread::should_yield() &&
5940        !CMSCollector::foregroundGCIsActive();
5941        ++i) {
5942     os::sleep(Thread::current(), 1, false);
5943   }


5984         !_bit_map->isMarked(addr)) {
5985       // mark bit map (object will become grey):
5986       // It is possible for several threads to be
5987       // trying to "claim" this object concurrently;
5988       // the unique thread that succeeds in marking the
5989       // object first will do the subsequent push on
5990       // to the work queue (or overflow list).
5991       if (_bit_map->par_mark(addr)) {
5992         // push on work_queue (which may not be empty), and trim the
5993         // queue to an appropriate length by applying this closure to
5994         // the oops in the oops popped from the stack (i.e. blacken the
5995         // grey objects)
5996         bool res = _work_queue->push(obj);
5997         assert(res, "Low water mark should be less than capacity?");
5998         trim_queue(_low_water_mark);
5999       } // Else, another thread claimed the object
6000     }
6001   }
6002 }
6003 



6004 // This closure is used to rescan the marked objects on the dirty cards
6005 // in the mod union table and the card table proper.
6006 size_t ScanMarkedObjectsAgainCarefullyClosure::do_object_careful_m(
6007   oop p, MemRegion mr) {
6008 
6009   size_t size = 0;
6010   HeapWord* addr = (HeapWord*)p;
6011   DEBUG_ONLY(_collector->verify_work_stacks_empty();)
6012   assert(_span.contains(addr), "we are scanning the CMS generation");
6013   // check if it's time to yield
6014   if (do_yield_check()) {
6015     // We yielded for some foreground stop-world work,
6016     // and we have been asked to abort this ongoing preclean cycle.
6017     return 0;
6018   }
6019   if (_bitMap->isMarked(addr)) {
6020     // it's marked; is it potentially uninitialized?
6021     if (p->klass_or_null_acquire() != NULL) {
6022         // an initialized object; ignore mark word in verification below
6023         // since we are running concurrent with mutators


6562   _finger = addr + obj->size();
6563   assert(_finger > addr, "we just incremented it above");
6564   // Note: the finger doesn't advance while we drain
6565   // the stack below.
6566   bool res = _mark_stack->push(obj);
6567   assert(res, "Empty non-zero size stack should have space for single push");
6568   while (!_mark_stack->isEmpty()) {
6569     oop new_oop = _mark_stack->pop();
6570     assert(oopDesc::is_oop(new_oop), "Oops! expected to pop an oop");
6571     // now scan this oop's oops
6572     new_oop->oop_iterate(&_pam_verify_closure);
6573   }
6574   assert(_mark_stack->isEmpty(), "tautology, emphasizing post-condition");
6575   return true;
6576 }
6577 
6578 PushAndMarkVerifyClosure::PushAndMarkVerifyClosure(
6579   CMSCollector* collector, MemRegion span,
6580   CMSBitMap* verification_bm, CMSBitMap* cms_bm,
6581   CMSMarkStack*  mark_stack):
6582   MetadataVisitingOopIterateClosure(collector->ref_processor()),
6583   _collector(collector),
6584   _span(span),
6585   _verification_bm(verification_bm),
6586   _cms_bm(cms_bm),
6587   _mark_stack(mark_stack)
6588 { }
6589 
6590 template <class T> void PushAndMarkVerifyClosure::do_oop_work(T *p) {
6591   oop obj = RawAccess<>::oop_load(p);
6592   do_oop(obj);
6593 }
6594 
6595 void PushAndMarkVerifyClosure::do_oop(oop* p)       { PushAndMarkVerifyClosure::do_oop_work(p); }
6596 void PushAndMarkVerifyClosure::do_oop(narrowOop* p) { PushAndMarkVerifyClosure::do_oop_work(p); }
6597 
6598 // Upon stack overflow, we discard (part of) the stack,
6599 // remembering the least address amongst those discarded
6600 // in CMSCollector's _restart_address.
6601 void PushAndMarkVerifyClosure::handle_stack_overflow(HeapWord* lost) {
6602   // Remember the least grey address discarded


6619       oop(addr)->print_on(&ls);
6620       log.error(" (" INTPTR_FORMAT " should have been marked)", p2i(addr));
6621       fatal("... aborting");
6622     }
6623 
6624     if (!_mark_stack->push(obj)) { // stack overflow
6625       log_trace(gc)("CMS marking stack overflow (benign) at " SIZE_FORMAT, _mark_stack->capacity());
6626       assert(_mark_stack->isFull(), "Else push should have succeeded");
6627       handle_stack_overflow(addr);
6628     }
6629     // anything including and to the right of _finger
6630     // will be scanned as we iterate over the remainder of the
6631     // bit map
6632   }
6633 }
6634 
6635 PushOrMarkClosure::PushOrMarkClosure(CMSCollector* collector,
6636                      MemRegion span,
6637                      CMSBitMap* bitMap, CMSMarkStack*  markStack,
6638                      HeapWord* finger, MarkFromRootsClosure* parent) :
6639   MetadataVisitingOopIterateClosure(collector->ref_processor()),
6640   _collector(collector),
6641   _span(span),
6642   _bitMap(bitMap),
6643   _markStack(markStack),
6644   _finger(finger),
6645   _parent(parent)
6646 { }
6647 
6648 ParPushOrMarkClosure::ParPushOrMarkClosure(CMSCollector* collector,
6649                                            MemRegion span,
6650                                            CMSBitMap* bit_map,
6651                                            OopTaskQueue* work_queue,
6652                                            CMSMarkStack*  overflow_stack,
6653                                            HeapWord* finger,
6654                                            HeapWord* volatile* global_finger_addr,
6655                                            ParMarkFromRootsClosure* parent) :
6656   MetadataVisitingOopIterateClosure(collector->ref_processor()),
6657   _collector(collector),
6658   _whole_span(collector->_span),
6659   _span(span),
6660   _bit_map(bit_map),
6661   _work_queue(work_queue),
6662   _overflow_stack(overflow_stack),
6663   _finger(finger),
6664   _global_finger_addr(global_finger_addr),
6665   _parent(parent)
6666 { }
6667 
6668 // Assumes thread-safe access by callers, who are
6669 // responsible for mutual exclusion.
6670 void CMSCollector::lower_restart_addr(HeapWord* low) {
6671   assert(_span.contains(low), "Out of bounds addr");
6672   if (_restart_addr == NULL) {
6673     _restart_addr = low;
6674   } else {
6675     _restart_addr = MIN2(_restart_addr, low);
6676   }


6717       NOT_PRODUCT(
6718         if (CMSMarkStackOverflowALot &&
6719             _collector->simulate_overflow()) {
6720           // simulate a stack overflow
6721           simulate_overflow = true;
6722         }
6723       )
6724       if (simulate_overflow || !_markStack->push(obj)) { // stack overflow
6725         log_trace(gc)("CMS marking stack overflow (benign) at " SIZE_FORMAT, _markStack->capacity());
6726         assert(simulate_overflow || _markStack->isFull(), "Else push should have succeeded");
6727         handle_stack_overflow(addr);
6728       }
6729     }
6730     // anything including and to the right of _finger
6731     // will be scanned as we iterate over the remainder of the
6732     // bit map
6733     do_yield_check();
6734   }
6735 }
6736 



6737 void ParPushOrMarkClosure::do_oop(oop obj) {
6738   // Ignore mark word because we are running concurrent with mutators.
6739   assert(oopDesc::is_oop_or_null(obj, true), "Expected an oop or NULL at " PTR_FORMAT, p2i(obj));
6740   HeapWord* addr = (HeapWord*)obj;
6741   if (_whole_span.contains(addr) && !_bit_map->isMarked(addr)) {
6742     // Oop lies in _span and isn't yet grey or black
6743     // We read the global_finger (volatile read) strictly after marking oop
6744     bool res = _bit_map->par_mark(addr);    // now grey
6745     volatile HeapWord** gfa = (volatile HeapWord**)_global_finger_addr;
6746     // Should we push this marked oop on our stack?
6747     // -- if someone else marked it, nothing to do
6748     // -- if target oop is above global finger nothing to do
6749     // -- if target oop is in chunk and above local finger
6750     //      then nothing to do
6751     // -- else push on work queue
6752     if (   !res       // someone else marked it, they will deal with it
6753         || (addr >= *gfa)  // will be scanned in a later task
6754         || (_span.contains(addr) && addr >= _finger)) { // later in this chunk
6755       return;
6756     }


6763           _collector->simulate_overflow()) {
6764         // simulate a stack overflow
6765         simulate_overflow = true;
6766       }
6767     )
6768     if (simulate_overflow ||
6769         !(_work_queue->push(obj) || _overflow_stack->par_push(obj))) {
6770       // stack overflow
6771       log_trace(gc)("CMS marking stack overflow (benign) at " SIZE_FORMAT, _overflow_stack->capacity());
6772       // We cannot assert that the overflow stack is full because
6773       // it may have been emptied since.
6774       assert(simulate_overflow ||
6775              _work_queue->size() == _work_queue->max_elems(),
6776             "Else push should have succeeded");
6777       handle_stack_overflow(addr);
6778     }
6779     do_yield_check();
6780   }
6781 }
6782 



6783 PushAndMarkClosure::PushAndMarkClosure(CMSCollector* collector,
6784                                        MemRegion span,
6785                                        ReferenceDiscoverer* rd,
6786                                        CMSBitMap* bit_map,
6787                                        CMSBitMap* mod_union_table,
6788                                        CMSMarkStack*  mark_stack,
6789                                        bool           concurrent_precleaning):
6790   MetadataVisitingOopIterateClosure(rd),
6791   _collector(collector),
6792   _span(span),
6793   _bit_map(bit_map),
6794   _mod_union_table(mod_union_table),
6795   _mark_stack(mark_stack),
6796   _concurrent_precleaning(concurrent_precleaning)
6797 {
6798   assert(ref_discoverer() != NULL, "ref_discoverer shouldn't be NULL");
6799 }
6800 
6801 // Grey object rescan during pre-cleaning and second checkpoint phases --
6802 // the non-parallel version (the parallel version appears further below.)
6803 void PushAndMarkClosure::do_oop(oop obj) {
6804   // Ignore mark word verification. If during concurrent precleaning,
6805   // the object monitor may be locked. If during the checkpoint
6806   // phases, the object may already have been reached by a  different
6807   // path and may be at the end of the global overflow list (so
6808   // the mark word may be NULL).
6809   assert(oopDesc::is_oop_or_null(obj, true /* ignore mark word */),
6810          "Expected an oop or NULL at " PTR_FORMAT, p2i(obj));


6842            _mod_union_table->mark_range(redirty_range);
6843          } else {
6844            _mod_union_table->mark(addr);
6845          }
6846          _collector->_ser_pmc_preclean_ovflw++;
6847       } else {
6848          // During the remark phase, we need to remember this oop
6849          // in the overflow list.
6850          _collector->push_on_overflow_list(obj);
6851          _collector->_ser_pmc_remark_ovflw++;
6852       }
6853     }
6854   }
6855 }
6856 
6857 ParPushAndMarkClosure::ParPushAndMarkClosure(CMSCollector* collector,
6858                                              MemRegion span,
6859                                              ReferenceDiscoverer* rd,
6860                                              CMSBitMap* bit_map,
6861                                              OopTaskQueue* work_queue):
6862   MetadataVisitingOopIterateClosure(rd),
6863   _collector(collector),
6864   _span(span),
6865   _bit_map(bit_map),
6866   _work_queue(work_queue)
6867 {
6868   assert(ref_discoverer() != NULL, "ref_discoverer shouldn't be NULL");
6869 }
6870 



6871 // Grey object rescan during second checkpoint phase --
6872 // the parallel version.
6873 void ParPushAndMarkClosure::do_oop(oop obj) {
6874   // In the assert below, we ignore the mark word because
6875   // this oop may point to an already visited object that is
6876   // on the overflow stack (in which case the mark word has
6877   // been hijacked for chaining into the overflow stack --
6878   // if this is the last object in the overflow stack then
6879   // its mark word will be NULL). Because this object may
6880   // have been subsequently popped off the global overflow
6881   // stack, and the mark word possibly restored to the prototypical
6882   // value, by the time we get to examined this failing assert in
6883   // the debugger, is_oop_or_null(false) may subsequently start
6884   // to hold.
6885   assert(oopDesc::is_oop_or_null(obj, true),
6886          "Expected an oop or NULL at " PTR_FORMAT, p2i(obj));
6887   HeapWord* addr = (HeapWord*)obj;
6888   // Check if oop points into the CMS generation
6889   // and is not marked
6890   if (_span.contains(addr) && !_bit_map->isMarked(addr)) {


6893     // first thread to mark it, then we push it on our
6894     // marking stack
6895     if (_bit_map->par_mark(addr)) {     // ... now grey
6896       // push on work queue (grey set)
6897       bool simulate_overflow = false;
6898       NOT_PRODUCT(
6899         if (CMSMarkStackOverflowALot &&
6900             _collector->par_simulate_overflow()) {
6901           // simulate a stack overflow
6902           simulate_overflow = true;
6903         }
6904       )
6905       if (simulate_overflow || !_work_queue->push(obj)) {
6906         _collector->par_push_on_overflow_list(obj);
6907         _collector->_par_pmc_remark_ovflw++; //  imprecise OK: no need to CAS
6908       }
6909     } // Else, some other thread got there first
6910   }
6911 }
6912 



6913 void CMSPrecleanRefsYieldClosure::do_yield_work() {
6914   Mutex* bml = _collector->bitMapLock();
6915   assert_lock_strong(bml);
6916   assert(ConcurrentMarkSweepThread::cms_thread_has_cms_token(),
6917          "CMS thread should hold CMS token");
6918 
6919   bml->unlock();
6920   ConcurrentMarkSweepThread::desynchronize(true);
6921 
6922   _collector->stopTimer();
6923   _collector->incrementYields();
6924 
6925   // See the comment in coordinator_yield()
6926   for (unsigned i = 0; i < CMSYieldSleepCount &&
6927                        ConcurrentMarkSweepThread::should_yield() &&
6928                        !CMSCollector::foregroundGCIsActive(); ++i) {
6929     os::sleep(Thread::current(), 1, false);
6930   }
6931 
6932   ConcurrentMarkSweepThread::synchronize(true);


7559         // are needed since no one else can be mutating the mod union
7560         // table.
7561         if (obj->is_objArray()) {
7562           size_t sz = obj->size();
7563           HeapWord* end_card_addr = align_up(addr + sz, CardTable::card_size);
7564           MemRegion redirty_range = MemRegion(addr, end_card_addr);
7565           assert(!redirty_range.is_empty(), "Arithmetical tautology");
7566           _collector->_modUnionTable.mark_range(redirty_range);
7567         } else {
7568           _collector->_modUnionTable.mark(addr);
7569         }
7570         _collector->_ser_kac_preclean_ovflw++;
7571       } else {
7572         _collector->push_on_overflow_list(obj);
7573         _collector->_ser_kac_ovflw++;
7574       }
7575     }
7576   }
7577 }
7578 



7579 // CMSParKeepAliveClosure: a parallel version of the above.
7580 // The work queues are private to each closure (thread),
7581 // but (may be) available for stealing by other threads.
7582 void CMSParKeepAliveClosure::do_oop(oop obj) {
7583   HeapWord* addr = (HeapWord*)obj;
7584   if (_span.contains(addr) &&
7585       !_bit_map->isMarked(addr)) {
7586     // In general, during recursive tracing, several threads
7587     // may be concurrently getting here; the first one to
7588     // "tag" it, claims it.
7589     if (_bit_map->par_mark(addr)) {
7590       bool res = _work_queue->push(obj);
7591       assert(res, "Low water mark should be much less than capacity");
7592       // Do a recursive trim in the hope that this will keep
7593       // stack usage lower, but leave some oops for potential stealers
7594       trim_queue(_low_water_mark);
7595     } // Else, another thread got there first
7596   }
7597 }
7598 



7599 void CMSParKeepAliveClosure::trim_queue(uint max) {
7600   while (_work_queue->size() > max) {
7601     oop new_oop;
7602     if (_work_queue->pop_local(new_oop)) {
7603       assert(new_oop != NULL && oopDesc::is_oop(new_oop), "Expected an oop");
7604       assert(_bit_map->isMarked((HeapWord*)new_oop),
7605              "no white objects on this stack!");
7606       assert(_span.contains((HeapWord*)new_oop), "Out of bounds oop");
7607       // iterate over the oops in this oop, marking and pushing
7608       // the ones in CMS heap (i.e. in _span).
7609       new_oop->oop_iterate(&_mark_and_push);
7610     }
7611   }
7612 }
7613 
7614 CMSInnerParMarkAndPushClosure::CMSInnerParMarkAndPushClosure(
7615                                 CMSCollector* collector,
7616                                 MemRegion span, CMSBitMap* bit_map,
7617                                 OopTaskQueue* work_queue):
7618   _collector(collector),


7623 void CMSInnerParMarkAndPushClosure::do_oop(oop obj) {
7624   HeapWord* addr = (HeapWord*)obj;
7625   if (_span.contains(addr) &&
7626       !_bit_map->isMarked(addr)) {
7627     if (_bit_map->par_mark(addr)) {
7628       bool simulate_overflow = false;
7629       NOT_PRODUCT(
7630         if (CMSMarkStackOverflowALot &&
7631             _collector->par_simulate_overflow()) {
7632           // simulate a stack overflow
7633           simulate_overflow = true;
7634         }
7635       )
7636       if (simulate_overflow || !_work_queue->push(obj)) {
7637         _collector->par_push_on_overflow_list(obj);
7638         _collector->_par_kac_ovflw++;
7639       }
7640     } // Else another thread got there already
7641   }
7642 }



7643 
7644 //////////////////////////////////////////////////////////////////
7645 //  CMSExpansionCause                /////////////////////////////
7646 //////////////////////////////////////////////////////////////////
7647 const char* CMSExpansionCause::to_string(CMSExpansionCause::Cause cause) {
7648   switch (cause) {
7649     case _no_expansion:
7650       return "No expansion";
7651     case _satisfy_free_ratio:
7652       return "Free ratio";
7653     case _satisfy_promotion:
7654       return "Satisfy promotion";
7655     case _satisfy_allocation:
7656       return "allocation";
7657     case _allocate_par_lab:
7658       return "Par LAB";
7659     case _allocate_par_spooling_space:
7660       return "Par Spooling Space";
7661     case _adaptive_size_policy:
7662       return "Ergonomics";


< prev index next >