< prev index next >

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

Print this page
rev 7209 : [mq]: inccms
rev 7210 : [mq]: icms-reviews

@@ -165,20 +165,10 @@
     _locker3(mutex3, Mutex::_no_safepoint_check_flag)
   { }
 };
 
 
-// Wrapper class to temporarily disable icms during a foreground cms collection.
-class ICMSDisabler: public StackObj {
- public:
-  // The ctor disables icms and wakes up the thread so it notices the change;
-  // the dtor re-enables icms.  Note that the CMSCollector methods will check
-  // CMSIncrementalMode.
-  ICMSDisabler()  { CMSCollector::disable_icms(); CMSCollector::start_icms(); }
-  ~ICMSDisabler() { CMSCollector::enable_icms(); }
-};
-
 //////////////////////////////////////////////////////////////////
 //  Concurrent Mark-Sweep Generation /////////////////////////////
 //////////////////////////////////////////////////////////////////
 
 NOT_PRODUCT(CompactibleFreeListSpace* debug_cms_space;)

@@ -361,11 +351,10 @@
 
   _cms_used_at_gc0_begin = 0;
   _cms_used_at_gc0_end = 0;
   _allow_duty_cycle_reduction = false;
   _valid_bits = 0;
-  _icms_duty_cycle = CMSIncrementalDutyCycle;
 }
 
 double CMSStats::cms_free_adjustment_factor(size_t free) const {
   // TBD: CR 6909490
   return 1.0;

@@ -440,90 +429,21 @@
     return 0.0;
   }
   return work - deadline;
 }
 
-// Return a duty cycle based on old_duty_cycle and new_duty_cycle, limiting the
-// amount of change to prevent wild oscillation.
-unsigned int CMSStats::icms_damped_duty_cycle(unsigned int old_duty_cycle,
-                                              unsigned int new_duty_cycle) {
-  assert(old_duty_cycle <= 100, "bad input value");
-  assert(new_duty_cycle <= 100, "bad input value");
-
-  // Note:  use subtraction with caution since it may underflow (values are
-  // unsigned).  Addition is safe since we're in the range 0-100.
-  unsigned int damped_duty_cycle = new_duty_cycle;
-  if (new_duty_cycle < old_duty_cycle) {
-    const unsigned int largest_delta = MAX2(old_duty_cycle / 4, 5U);
-    if (new_duty_cycle + largest_delta < old_duty_cycle) {
-      damped_duty_cycle = old_duty_cycle - largest_delta;
-    }
-  } else if (new_duty_cycle > old_duty_cycle) {
-    const unsigned int largest_delta = MAX2(old_duty_cycle / 4, 15U);
-    if (new_duty_cycle > old_duty_cycle + largest_delta) {
-      damped_duty_cycle = MIN2(old_duty_cycle + largest_delta, 100U);
-    }
-  }
-  assert(damped_duty_cycle <= 100, "invalid duty cycle computed");
-
-  if (CMSTraceIncrementalPacing) {
-    gclog_or_tty->print(" [icms_damped_duty_cycle(%d,%d) = %d] ",
-                           old_duty_cycle, new_duty_cycle, damped_duty_cycle);
-  }
-  return damped_duty_cycle;
-}
-
-unsigned int CMSStats::icms_update_duty_cycle_impl() {
-  assert(CMSIncrementalPacing && valid(),
-         "should be handled in icms_update_duty_cycle()");
-
-  double cms_time_so_far = cms_timer().seconds();
-  double scaled_duration = cms_duration_per_mb() * _cms_used_at_gc0_end / M;
-  double scaled_duration_remaining = fabsd(scaled_duration - cms_time_so_far);
-
-  // Avoid division by 0.
-  double time_until_full = MAX2(time_until_cms_gen_full(), 0.01);
-  double duty_cycle_dbl = 100.0 * scaled_duration_remaining / time_until_full;
-
-  unsigned int new_duty_cycle = MIN2((unsigned int)duty_cycle_dbl, 100U);
-  if (new_duty_cycle > _icms_duty_cycle) {
-    // Avoid very small duty cycles (1 or 2); 0 is allowed.
-    if (new_duty_cycle > 2) {
-      _icms_duty_cycle = icms_damped_duty_cycle(_icms_duty_cycle,
-                                                new_duty_cycle);
-    }
-  } else if (_allow_duty_cycle_reduction) {
-    // The duty cycle is reduced only once per cms cycle (see record_cms_end()).
-    new_duty_cycle = icms_damped_duty_cycle(_icms_duty_cycle, new_duty_cycle);
-    // Respect the minimum duty cycle.
-    unsigned int min_duty_cycle = (unsigned int)CMSIncrementalDutyCycleMin;
-    _icms_duty_cycle = MAX2(new_duty_cycle, min_duty_cycle);
-  }
-
-  if (PrintGCDetails || CMSTraceIncrementalPacing) {
-    gclog_or_tty->print(" icms_dc=%d ", _icms_duty_cycle);
-  }
-
-  _allow_duty_cycle_reduction = false;
-  return _icms_duty_cycle;
-}
-
 #ifndef PRODUCT
 void CMSStats::print_on(outputStream *st) const {
   st->print(" gc0_alpha=%d,cms_alpha=%d", _gc0_alpha, _cms_alpha);
   st->print(",gc0_dur=%g,gc0_per=%g,gc0_promo=" SIZE_FORMAT,
                gc0_duration(), gc0_period(), gc0_promoted());
-  st->print(",cms_dur=%g,cms_dur_per_mb=%g,cms_per=%g,cms_alloc=" SIZE_FORMAT,
-            cms_duration(), cms_duration_per_mb(),
-            cms_period(), cms_allocated());
+  st->print(",cms_dur=%g,cms_per=%g,cms_alloc=" SIZE_FORMAT,
+            cms_duration(), cms_period(), cms_allocated());
   st->print(",cms_since_beg=%g,cms_since_end=%g",
             cms_time_since_begin(), cms_time_since_end());
   st->print(",cms_used_beg=" SIZE_FORMAT ",cms_used_end=" SIZE_FORMAT,
             _cms_used_at_gc0_begin, _cms_used_at_gc0_end);
-  if (CMSIncrementalMode) {
-    st->print(",dc=%d", icms_duty_cycle());
-  }
 
   if (valid()) {
     st->print(",promo_rate=%g,cms_alloc_rate=%g",
               promotion_rate(), cms_allocation_rate());
     st->print(",cms_consumption_rate=%g,time_until_full=%g",

@@ -577,12 +497,10 @@
 #ifndef PRODUCT
   _num_par_pushes(0),
 #endif
   _collection_count_start(0),
   _verifying(false),
-  _icms_start_limit(NULL),
-  _icms_stop_limit(NULL),
   _verification_mark_bm(0, Mutex::leaf + 1, "CMS_verification_mark_bm_lock"),
   _completed_initialization(false),
   _collector_policy(cp),
   _should_unload_classes(CMSClassUnloadingEnabled),
   _concurrent_cycles_since_last_unload(0),

@@ -1116,141 +1034,10 @@
       }
     }
   }
 }
 
-static inline size_t percent_of_space(Space* space, HeapWord* addr)
-{
-  size_t delta = pointer_delta(addr, space->bottom());
-  return (size_t)(delta * 100.0 / (space->capacity() / HeapWordSize));
-}
-
-void CMSCollector::icms_update_allocation_limits()
-{
-  Generation* young = GenCollectedHeap::heap()->get_gen(0);
-  EdenSpace* eden = young->as_DefNewGeneration()->eden();
-
-  const unsigned int duty_cycle = stats().icms_update_duty_cycle();
-  if (CMSTraceIncrementalPacing) {
-    stats().print();
-  }
-
-  assert(duty_cycle <= 100, "invalid duty cycle");
-  if (duty_cycle != 0) {
-    // The duty_cycle is a percentage between 0 and 100; convert to words and
-    // then compute the offset from the endpoints of the space.
-    size_t free_words = eden->free() / HeapWordSize;
-    double free_words_dbl = (double)free_words;
-    size_t duty_cycle_words = (size_t)(free_words_dbl * duty_cycle / 100.0);
-    size_t offset_words = (free_words - duty_cycle_words) / 2;
-
-    _icms_start_limit = eden->top() + offset_words;
-    _icms_stop_limit = eden->end() - offset_words;
-
-    // The limits may be adjusted (shifted to the right) by
-    // CMSIncrementalOffset, to allow the application more mutator time after a
-    // young gen gc (when all mutators were stopped) and before CMS starts and
-    // takes away one or more cpus.
-    if (CMSIncrementalOffset != 0) {
-      double adjustment_dbl = free_words_dbl * CMSIncrementalOffset / 100.0;
-      size_t adjustment = (size_t)adjustment_dbl;
-      HeapWord* tmp_stop = _icms_stop_limit + adjustment;
-      if (tmp_stop > _icms_stop_limit && tmp_stop < eden->end()) {
-        _icms_start_limit += adjustment;
-        _icms_stop_limit = tmp_stop;
-      }
-    }
-  }
-  if (duty_cycle == 0 || (_icms_start_limit == _icms_stop_limit)) {
-    _icms_start_limit = _icms_stop_limit = eden->end();
-  }
-
-  // Install the new start limit.
-  eden->set_soft_end(_icms_start_limit);
-
-  if (CMSTraceIncrementalMode) {
-    gclog_or_tty->print(" icms alloc limits:  "
-                           PTR_FORMAT "," PTR_FORMAT
-                           " (" SIZE_FORMAT "%%," SIZE_FORMAT "%%) ",
-                           p2i(_icms_start_limit), p2i(_icms_stop_limit),
-                           percent_of_space(eden, _icms_start_limit),
-                           percent_of_space(eden, _icms_stop_limit));
-    if (Verbose) {
-      gclog_or_tty->print("eden:  ");
-      eden->print_on(gclog_or_tty);
-    }
-  }
-}
-
-// Any changes here should try to maintain the invariant
-// that if this method is called with _icms_start_limit
-// and _icms_stop_limit both NULL, then it should return NULL
-// and not notify the icms thread.
-HeapWord*
-CMSCollector::allocation_limit_reached(Space* space, HeapWord* top,
-                                       size_t word_size)
-{
-  // A start_limit equal to end() means the duty cycle is 0, so treat that as a
-  // nop.
-  if (CMSIncrementalMode && _icms_start_limit != space->end()) {
-    if (top <= _icms_start_limit) {
-      if (CMSTraceIncrementalMode) {
-        space->print_on(gclog_or_tty);
-        gclog_or_tty->stamp();
-        gclog_or_tty->print_cr(" start limit top=" PTR_FORMAT
-                               ", new limit=" PTR_FORMAT
-                               " (" SIZE_FORMAT "%%)",
-                               p2i(top), p2i(_icms_stop_limit),
-                               percent_of_space(space, _icms_stop_limit));
-      }
-      ConcurrentMarkSweepThread::start_icms();
-      assert(top < _icms_stop_limit, "Tautology");
-      if (word_size < pointer_delta(_icms_stop_limit, top)) {
-        return _icms_stop_limit;
-      }
-
-      // The allocation will cross both the _start and _stop limits, so do the
-      // stop notification also and return end().
-      if (CMSTraceIncrementalMode) {
-        space->print_on(gclog_or_tty);
-        gclog_or_tty->stamp();
-        gclog_or_tty->print_cr(" +stop limit top=" PTR_FORMAT
-                               ", new limit=" PTR_FORMAT
-                               " (" SIZE_FORMAT "%%)",
-                               p2i(top), p2i(space->end()),
-                               percent_of_space(space, space->end()));
-      }
-      ConcurrentMarkSweepThread::stop_icms();
-      return space->end();
-    }
-
-    if (top <= _icms_stop_limit) {
-      if (CMSTraceIncrementalMode) {
-        space->print_on(gclog_or_tty);
-        gclog_or_tty->stamp();
-        gclog_or_tty->print_cr(" stop limit top=" PTR_FORMAT
-                               ", new limit=" PTR_FORMAT
-                               " (" SIZE_FORMAT "%%)",
-                               top, space->end(),
-                               percent_of_space(space, space->end()));
-      }
-      ConcurrentMarkSweepThread::stop_icms();
-      return space->end();
-    }
-
-    if (CMSTraceIncrementalMode) {
-      space->print_on(gclog_or_tty);
-      gclog_or_tty->stamp();
-      gclog_or_tty->print_cr(" end limit top=" PTR_FORMAT
-                             ", new limit=" PTR_FORMAT,
-                             top, NULL);
-    }
-  }
-
-  return NULL;
-}
-
 oop ConcurrentMarkSweepGeneration::promote(oop obj, size_t obj_size) {
   assert(obj_size == (size_t)obj->size(), "bad obj_size passed in");
   // allocate, copy and if necessary update promoinfo --
   // delegate to underlying space.
   assert_lock_strong(freelistLock());

@@ -1289,18 +1076,10 @@
   }
   return res;
 }
 
 
-HeapWord*
-ConcurrentMarkSweepGeneration::allocation_limit_reached(Space* space,
-                                             HeapWord* top,
-                                             size_t word_sz)
-{
-  return collector()->allocation_limit_reached(space, top, word_sz);
-}
-
 // IMPORTANT: Notes on object size recognition in CMS.
 // ---------------------------------------------------
 // A block of storage in the CMS generation is always in
 // one of three states. A free block (FREE), an allocated
 // object (OBJECT) whose size() method reports the correct size,

@@ -1809,13 +1588,10 @@
 
   // Signal to a possibly ongoing concurrent collection that
   // we want to do a foreground collection.
   _foregroundGCIsActive = true;
 
-  // Disable incremental mode during a foreground collection.
-  ICMSDisabler icms_disabler;
-
   // release locks and wait for a notify from the background collector
   // releasing the locks in only necessary for phases which
   // do yields to improve the granularity of the collection.
   assert_lock_strong(bitMapLock());
   // We need to lock the Free list lock for the space that we are

@@ -2135,11 +1911,11 @@
 }
 
 
 void CMSCollector::print_eden_and_survivor_chunk_arrays() {
   DefNewGeneration* dng = _young_gen->as_DefNewGeneration();
-  EdenSpace* eden_space = dng->eden();
+  ContiguousSpace* eden_space = dng->eden();
   ContiguousSpace* from_space = dng->from();
   ContiguousSpace* to_space   = dng->to();
   // Eden
   if (_eden_chunk_array != NULL) {
     gclog_or_tty->print_cr("eden " PTR_FORMAT "-" PTR_FORMAT "-" PTR_FORMAT "(" SIZE_FORMAT ")",

@@ -2783,14 +2559,10 @@
   // update_counters() that allows the utilization to be passed as a
   // parameter, avoiding multiple calls to used().
   //
   _cmsGen->update_counters(cms_used);
 
-  if (CMSIncrementalMode) {
-    icms_update_allocation_limits();
-  }
-
   bitMapLock()->unlock();
   releaseFreelistLocks();
 
   if (!CleanChunkPoolAsync) {
     Chunk::clean_chunk_pool();

@@ -4274,16 +4046,14 @@
   // do this unlock/lock or modify the MutexUnlocker class to
   // serve our purpose. XXX
   assert_lock_strong(_bit_map_lock);
   _bit_map_lock->unlock();
   ConcurrentMarkSweepThread::desynchronize(true);
-  ConcurrentMarkSweepThread::acknowledge_yield_request();
   _collector->stopTimer();
   if (PrintCMSStatistics != 0) {
     _collector->incrementYields();
   }
-  _collector->icms_wait();
 
   // It is possible for whichever thread initiated the yield request
   // not to get a chance to wake up and take the bitmap lock between
   // this thread releasing it and reacquiring it. So, while the
   // should_yield() flag is on, let's sleep for a bit to give the

@@ -4309,11 +4079,10 @@
   // Tony 2006.06.29
   for (unsigned i = 0; i < CMSCoordinatorYieldSleepCount &&
                    ConcurrentMarkSweepThread::should_yield() &&
                    !CMSCollector::foregroundGCIsActive(); ++i) {
     os::sleep(Thread::current(), 1, false);
-    ConcurrentMarkSweepThread::acknowledge_yield_request();
   }
 
   ConcurrentMarkSweepThread::synchronize(true);
   _bit_map_lock->lock_without_safepoint_check();
   _collector->startTimer();

@@ -5240,11 +5009,11 @@
   }
 };
 
 void CMSParMarkTask::work_on_young_gen_roots(uint worker_id, OopsInGenClosure* cl) {
   DefNewGeneration* dng = _collector->_young_gen->as_DefNewGeneration();
-  EdenSpace* eden_space = dng->eden();
+  ContiguousSpace* eden_space = dng->eden();
   ContiguousSpace* from_space = dng->from();
   ContiguousSpace* to_space   = dng->to();
 
   HeapWord** eca = _collector->_eden_chunk_array;
   size_t     ect = _collector->_eden_chunk_index;

@@ -5412,11 +5181,11 @@
     assert(pst->valid(), "Uninitialized use?");
     HeapWord *start, *end;
     while (!pst->is_task_claimed(/* reference */ nth_task)) {
       // We claimed task # nth_task; compute its boundaries.
       if (chunk_top == 0) {  // no samples were taken
-        assert(nth_task == 0 && n_tasks == 1, "Can have only 1 EdenSpace task");
+        assert(nth_task == 0 && n_tasks == 1, "Can have only 1 eden task");
         start = space->bottom();
         end   = space->top();
       } else if (nth_task == 0) {
         start = space->bottom();
         end   = chunk_array[nth_task];

@@ -5790,11 +5559,11 @@
 
   // The young gen rescan work will not be done as part of
   // process_roots (which currently doesn't know how to
   // parallelize such a scan), but rather will be broken up into
   // a set of parallel tasks (via the sampling that the [abortable]
-  // preclean phase did of EdenSpace, plus the [two] tasks of
+  // preclean phase did of eden, plus the [two] tasks of
   // scanning the [two] survivor spaces. Further fine-grain
   // parallelization of the scanning of the survivor spaces
   // themselves, and of precleaning of the younger gen itself
   // is deferred to the future.
   initialize_sequential_subtasks_for_young_gen_rescan(n_workers);

@@ -6476,23 +6245,20 @@
         assert(ConcurrentMarkSweepThread::cms_thread_has_cms_token(),
                "CMS thread should hold CMS token");
         assert_lock_strong(bitMapLock());
         bitMapLock()->unlock();
         ConcurrentMarkSweepThread::desynchronize(true);
-        ConcurrentMarkSweepThread::acknowledge_yield_request();
         stopTimer();
         if (PrintCMSStatistics != 0) {
           incrementYields();
         }
-        icms_wait();
 
         // See the comment in coordinator_yield()
         for (unsigned i = 0; i < CMSYieldSleepCount &&
                          ConcurrentMarkSweepThread::should_yield() &&
                          !CMSCollector::foregroundGCIsActive(); ++i) {
           os::sleep(Thread::current(), 1, false);
-          ConcurrentMarkSweepThread::acknowledge_yield_request();
         }
 
         ConcurrentMarkSweepThread::synchronize(true);
         bitMapLock()->lock_without_safepoint_check();
         startTimer();

@@ -6511,14 +6277,10 @@
     assert_lock_strong(bitMapLock());
     _markBitMap.clear_all();
     _collectorState = Idling;
   }
 
-  // Stop incremental mode after a cycle completes, so that any future cycles
-  // are triggered by allocation.
-  stop_icms();
-
   NOT_PRODUCT(
     if (RotateCMSCollectionTypes) {
       _cmsGen->rotate_debug_collection_type();
     }
   )

@@ -6966,25 +6728,22 @@
   assert_lock_strong(_bit_map->lock());
   // relinquish the free_list_lock and bitMaplock()
   _bit_map->lock()->unlock();
   _freelistLock->unlock();
   ConcurrentMarkSweepThread::desynchronize(true);
-  ConcurrentMarkSweepThread::acknowledge_yield_request();
   _collector->stopTimer();
   if (PrintCMSStatistics != 0) {
     _collector->incrementYields();
   }
-  _collector->icms_wait();
 
   // See the comment in coordinator_yield()
   for (unsigned i = 0;
        i < CMSYieldSleepCount &&
        ConcurrentMarkSweepThread::should_yield() &&
        !CMSCollector::foregroundGCIsActive();
        ++i) {
     os::sleep(Thread::current(), 1, false);
-    ConcurrentMarkSweepThread::acknowledge_yield_request();
   }
 
   ConcurrentMarkSweepThread::synchronize(true);
   _freelistLock->lock_without_safepoint_check();
   _bit_map->lock()->lock_without_safepoint_check();

@@ -7126,23 +6885,20 @@
   assert_lock_strong(_bitMap->lock());
   // relinquish the free_list_lock and bitMaplock()
   _bitMap->lock()->unlock();
   _freelistLock->unlock();
   ConcurrentMarkSweepThread::desynchronize(true);
-  ConcurrentMarkSweepThread::acknowledge_yield_request();
   _collector->stopTimer();
   if (PrintCMSStatistics != 0) {
     _collector->incrementYields();
   }
-  _collector->icms_wait();
 
   // See the comment in coordinator_yield()
   for (unsigned i = 0; i < CMSYieldSleepCount &&
                    ConcurrentMarkSweepThread::should_yield() &&
                    !CMSCollector::foregroundGCIsActive(); ++i) {
     os::sleep(Thread::current(), 1, false);
-    ConcurrentMarkSweepThread::acknowledge_yield_request();
   }
 
   ConcurrentMarkSweepThread::synchronize(true);
   _freelistLock->lock_without_safepoint_check();
   _bitMap->lock()->lock_without_safepoint_check();

@@ -7198,23 +6954,20 @@
          "CMS thread should hold CMS token");
   assert_lock_strong(_bit_map->lock());
   // Relinquish the bit map lock
   _bit_map->lock()->unlock();
   ConcurrentMarkSweepThread::desynchronize(true);
-  ConcurrentMarkSweepThread::acknowledge_yield_request();
   _collector->stopTimer();
   if (PrintCMSStatistics != 0) {
     _collector->incrementYields();
   }
-  _collector->icms_wait();
 
   // See the comment in coordinator_yield()
   for (unsigned i = 0; i < CMSYieldSleepCount &&
                        ConcurrentMarkSweepThread::should_yield() &&
                        !CMSCollector::foregroundGCIsActive(); ++i) {
     os::sleep(Thread::current(), 1, false);
-    ConcurrentMarkSweepThread::acknowledge_yield_request();
   }
 
   ConcurrentMarkSweepThread::synchronize(true);
   _bit_map->lock()->lock_without_safepoint_check();
   _collector->startTimer();

@@ -7356,23 +7109,20 @@
   assert(ConcurrentMarkSweepThread::cms_thread_has_cms_token(),
          "CMS thread should hold CMS token");
   assert_lock_strong(_bitMap->lock());
   _bitMap->lock()->unlock();
   ConcurrentMarkSweepThread::desynchronize(true);
-  ConcurrentMarkSweepThread::acknowledge_yield_request();
   _collector->stopTimer();
   if (PrintCMSStatistics != 0) {
     _collector->incrementYields();
   }
-  _collector->icms_wait();
 
   // See the comment in coordinator_yield()
   for (unsigned i = 0; i < CMSYieldSleepCount &&
                        ConcurrentMarkSweepThread::should_yield() &&
                        !CMSCollector::foregroundGCIsActive(); ++i) {
     os::sleep(Thread::current(), 1, false);
-    ConcurrentMarkSweepThread::acknowledge_yield_request();
   }
 
   ConcurrentMarkSweepThread::synchronize(true);
   _bitMap->lock()->lock_without_safepoint_check();
   _collector->startTimer();

@@ -7390,11 +7140,11 @@
   assert(_finger <= ptr, "_finger runneth ahead");
   // advance the finger to right end of this object
   _finger = ptr + obj->size();
   assert(_finger > ptr, "we just incremented it above");
   // On large heaps, it may take us some time to get through
-  // the marking phase (especially if running iCMS). During
+  // the marking phase. During
   // this time it's possible that a lot of mutations have
   // accumulated in the card table and the mod union table --
   // these mutation records are redundant until we have
   // actually traced into the corresponding card.
   // Here, we check whether advancing the finger would make

@@ -7507,11 +7257,11 @@
   assert(_finger <= ptr, "_finger runneth ahead");
   // advance the finger to right end of this object
   _finger = ptr + obj->size();
   assert(_finger > ptr, "we just incremented it above");
   // On large heaps, it may take us some time to get through
-  // the marking phase (especially if running iCMS). During
+  // the marking phase. During
   // this time it's possible that a lot of mutations have
   // accumulated in the card table and the mod union table --
   // these mutation records are redundant until we have
   // actually traced into the corresponding card.
   // Here, we check whether advancing the finger would make

@@ -7996,24 +7746,20 @@
          "CMS thread should hold CMS token");
 
   bml->unlock();
   ConcurrentMarkSweepThread::desynchronize(true);
 
-  ConcurrentMarkSweepThread::acknowledge_yield_request();
-
   _collector->stopTimer();
   if (PrintCMSStatistics != 0) {
     _collector->incrementYields();
   }
-  _collector->icms_wait();
 
   // See the comment in coordinator_yield()
   for (unsigned i = 0; i < CMSYieldSleepCount &&
                        ConcurrentMarkSweepThread::should_yield() &&
                        !CMSCollector::foregroundGCIsActive(); ++i) {
     os::sleep(Thread::current(), 1, false);
-    ConcurrentMarkSweepThread::acknowledge_yield_request();
   }
 
   ConcurrentMarkSweepThread::synchronize(true);
   bml->lock();
 

@@ -8677,23 +8423,20 @@
   assert(ConcurrentMarkSweepThread::cms_thread_has_cms_token(),
          "CMS thread should hold CMS token");
   _bitMap->lock()->unlock();
   _freelistLock->unlock();
   ConcurrentMarkSweepThread::desynchronize(true);
-  ConcurrentMarkSweepThread::acknowledge_yield_request();
   _collector->stopTimer();
   if (PrintCMSStatistics != 0) {
     _collector->incrementYields();
   }
-  _collector->icms_wait();
 
   // See the comment in coordinator_yield()
   for (unsigned i = 0; i < CMSYieldSleepCount &&
                        ConcurrentMarkSweepThread::should_yield() &&
                        !CMSCollector::foregroundGCIsActive(); ++i) {
     os::sleep(Thread::current(), 1, false);
-    ConcurrentMarkSweepThread::acknowledge_yield_request();
   }
 
   ConcurrentMarkSweepThread::synchronize(true);
   _freelistLock->lock();
   _bitMap->lock()->lock_without_safepoint_check();
< prev index next >