changeset 1755:c99c53f07c14

6692906: CMS: parallel concurrent marking may be prone to hanging or stalling mutators for periods of time Summary: Inserted missing yield(check)s in closures used during the work-stealing phase of parallel concurrent marking, a missing synchronous yield-request in the cms perm gen allocation path, and a terminator-terminator for the offer_termination invocation that monitors the yield status of the concurrent marking task. Elaborated some documentation comments and made some task queue termination loop flags configurable at start-up to aid debugging in the field. Reviewed-by: jmasa, johnc, poonam
author ysr
date Wed, 29 Sep 2010 16:17:02 -0700
parents 894b1d7c7e01
children 8f6f7587d292
files src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.cpp src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepThread.hpp src/share/vm/runtime/globals.hpp src/share/vm/utilities/yieldingWorkgroup.hpp
diffstat 4 files changed, 98 insertions(+), 35 deletions(-) [+]
line wrap: on
line diff
--- a/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.cpp	Tue Sep 28 15:56:15 2010 -0700
+++ b/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.cpp	Wed Sep 29 16:17:02 2010 -0700
@@ -3264,6 +3264,7 @@
 ConcurrentMarkSweepGeneration::expand_and_allocate(size_t word_size,
                                                    bool   tlab,
                                                    bool   parallel) {
+  CMSSynchronousYieldRequest yr;
   assert(!tlab, "Can't deal with TLAB allocation");
   MutexLockerEx x(freelistLock(), Mutex::_no_safepoint_check_flag);
   expand(word_size*HeapWordSize, MinHeapDeltaBytes,
@@ -3710,20 +3711,26 @@
 class CMSConcMarkingTerminator: public ParallelTaskTerminator {
   CMSCollector*       _collector;
   CMSConcMarkingTask* _task;
-  bool _yield;
- protected:
+ public:
   virtual void yield();
- public:
+
   // "n_threads" is the number of threads to be terminated.
   // "queue_set" is a set of work queues of other threads.
   // "collector" is the CMS collector associated with this task terminator.
   // "yield" indicates whether we need the gang as a whole to yield.
-  CMSConcMarkingTerminator(int n_threads, TaskQueueSetSuper* queue_set,
-                           CMSCollector* collector, bool yield) :
+  CMSConcMarkingTerminator(int n_threads, TaskQueueSetSuper* queue_set, CMSCollector* collector) :
     ParallelTaskTerminator(n_threads, queue_set),
-    _collector(collector),
-    _yield(yield) { }
-
+    _collector(collector) { }
+
+  void set_task(CMSConcMarkingTask* task) {
+    _task = task;
+  }
+};
+
+class CMSConcMarkingTerminatorTerminator: public TerminatorTerminator {
+  CMSConcMarkingTask* _task;
+ public:
+  bool should_exit_termination();
   void set_task(CMSConcMarkingTask* task) {
     _task = task;
   }
@@ -3737,7 +3744,9 @@
   bool          _result;
   CompactibleFreeListSpace*  _cms_space;
   CompactibleFreeListSpace* _perm_space;
-  HeapWord*     _global_finger;
+  char          _pad_front[64];   // padding to ...
+  HeapWord*     _global_finger;   // ... avoid sharing cache line
+  char          _pad_back[64];
   HeapWord*     _restart_addr;
 
   //  Exposed here for yielding support
@@ -3745,7 +3754,10 @@
 
   // The per thread work queues, available here for stealing
   OopTaskQueueSet*  _task_queues;
+
+  // Termination (and yielding) support
   CMSConcMarkingTerminator _term;
+  CMSConcMarkingTerminatorTerminator _term_term;
 
  public:
   CMSConcMarkingTask(CMSCollector* collector,
@@ -3760,11 +3772,12 @@
     _perm_space(perm_space),
     _asynch(asynch), _n_workers(0), _result(true),
     _task_queues(task_queues),
-    _term(_n_workers, task_queues, _collector, asynch),
+    _term(_n_workers, task_queues, _collector),
     _bit_map_lock(collector->bitMapLock())
   {
     _requested_size = _n_workers;
     _term.set_task(this);
+    _term_term.set_task(this);
     assert(_cms_space->bottom() < _perm_space->bottom(),
            "Finger incorrectly initialized below");
     _restart_addr = _global_finger = _cms_space->bottom();
@@ -3784,6 +3797,11 @@
   }
 
   void work(int i);
+  bool should_yield() {
+    return    ConcurrentMarkSweepThread::should_yield()
+           && !_collector->foregroundGCIsActive()
+           && _asynch;
+  }
 
   virtual void coordinator_yield();  // stuff done by coordinator
   bool result() { return _result; }
@@ -3805,10 +3823,17 @@
   void bump_global_finger(HeapWord* f);
 };
 
+bool CMSConcMarkingTerminatorTerminator::should_exit_termination() {
+  assert(_task != NULL, "Error");
+  return _task->yielding();
+  // Note that we do not need the disjunct || _task->should_yield() above
+  // because we want terminating threads to yield only if the task
+  // is already in the midst of yielding, which happens only after at least one
+  // thread has yielded.
+}
+
 void CMSConcMarkingTerminator::yield() {
-  if (ConcurrentMarkSweepThread::should_yield() &&
-      !_collector->foregroundGCIsActive() &&
-      _yield) {
+  if (_task->should_yield()) {
     _task->yield();
   } else {
     ParallelTaskTerminator::yield();
@@ -4033,6 +4058,7 @@
 
 class Par_ConcMarkingClosure: public Par_KlassRememberingOopClosure {
  private:
+  CMSConcMarkingTask* _task;
   MemRegion     _span;
   CMSBitMap*    _bit_map;
   CMSMarkStack* _overflow_stack;
@@ -4040,11 +4066,12 @@
  protected:
   DO_OOP_WORK_DEFN
  public:
-  Par_ConcMarkingClosure(CMSCollector* collector, OopTaskQueue* work_queue,
+  Par_ConcMarkingClosure(CMSCollector* collector, CMSConcMarkingTask* task, OopTaskQueue* work_queue,
                          CMSBitMap* bit_map, CMSMarkStack* overflow_stack,
                          CMSMarkStack* revisit_stack):
     Par_KlassRememberingOopClosure(collector, NULL, revisit_stack),
-    _span(_collector->_span),
+    _task(task),
+    _span(collector->_span),
     _work_queue(work_queue),
     _bit_map(bit_map),
     _overflow_stack(overflow_stack)
@@ -4053,6 +4080,11 @@
   virtual void do_oop(narrowOop* p);
   void trim_queue(size_t max);
   void handle_stack_overflow(HeapWord* lost);
+  void do_yield_check() {
+    if (_task->should_yield()) {
+      _task->yield();
+    }
+  }
 };
 
 // Grey object scanning during work stealing phase --
@@ -4096,6 +4128,7 @@
         handle_stack_overflow(addr);
       }
     } // Else, some other thread got there first
+    do_yield_check();
   }
 }
 
@@ -4111,6 +4144,7 @@
       assert(_span.contains((HeapWord*)new_oop), "Not in span");
       assert(new_oop->is_parsable(), "Should be parsable");
       new_oop->oop_iterate(this);  // do_oop() above
+      do_yield_check();
     }
   }
 }
@@ -4138,7 +4172,7 @@
   CMSMarkStack* ovflw = &(_collector->_markStack);
   CMSMarkStack* revisit = &(_collector->_revisitStack);
   int* seed = _collector->hash_seed(i);
-  Par_ConcMarkingClosure cl(_collector, work_q, bm, ovflw, revisit);
+  Par_ConcMarkingClosure cl(_collector, this, work_q, bm, ovflw, revisit);
   while (true) {
     cl.trim_queue(0);
     assert(work_q->size() == 0, "Should have been emptied above");
@@ -4151,9 +4185,11 @@
       assert(obj_to_scan->is_oop(), "Should be an oop");
       assert(bm->isMarked((HeapWord*)obj_to_scan), "Grey object");
       obj_to_scan->oop_iterate(&cl);
-    } else if (terminator()->offer_termination()) {
+    } else if (terminator()->offer_termination(&_term_term)) {
       assert(work_q->size() == 0, "Impossible!");
       break;
+    } else if (yielding() || should_yield()) {
+      yield();
     }
   }
 }
--- a/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepThread.hpp	Tue Sep 28 15:56:15 2010 -0700
+++ b/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepThread.hpp	Wed Sep 29 16:17:02 2010 -0700
@@ -1,5 +1,5 @@
 /*
- * Copyright (c) 2001, 2006, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2001, 2010, Oracle and/or its affiliates. All rights reserved.
  * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
  *
  * This code is free software; you can redistribute it and/or modify it
@@ -123,24 +123,44 @@
   // or given timeout, whichever is earlier.
   void    wait_on_cms_lock(long t); // milliseconds
 
-  // The CMS thread will yield during the work portion of it's cycle
+  // The CMS thread will yield during the work portion of its cycle
   // only when requested to.  Both synchronous and asychronous requests
-  // are provided.  A synchronous request is used for young gen
-  // collections and direct allocations.  The requesting thread increments
-  // pending_yields at the beginning of an operation, and decrements it when
-  // the operation is completed.  The CMS thread yields when pending_yields
-  // is positive.  An asynchronous request is used by iCMS in the stop_icms()
-  // operation. A single yield satisfies the outstanding asynch yield requests.
-  // The requesting thread increments both pending_yields and pending_decrements.
-  // After yielding, the CMS thread decrements both by the amount in
-  // pending_decrements.
+  // are provided:
+  // (1) A synchronous request is used for young gen collections and
+  //     for direct allocations.  The requesting thread increments
+  //     _pending_yields at the beginning of an operation, and decrements
+  //     _pending_yields when that operation is completed.
+  //     In turn, the CMS thread yields when _pending_yields is positive,
+  //     and continues to yield until the value reverts to 0.
+  // (2) An asynchronous request, on the other hand, is used by iCMS
+  //     for the stop_icms() operation. A single yield satisfies all of
+  //     the outstanding asynch yield requests, of which there may
+  //     occasionally be several in close succession. To accomplish
+  //     this, an asynch-requesting thread atomically increments both
+  //     _pending_yields and _pending_decrements. An asynchr requesting
+  //     thread does not wait and "acknowledge" completion of an operation
+  //     and deregister the request, like the synchronous version described
+  //     above does. In turn, after yielding, the CMS thread decrements both
+  //     _pending_yields and _pending_decrements by the value seen in
+  //     _pending_decrements before the decrement.
+  //  NOTE: The above scheme is isomorphic to having two request counters,
+  //  one for async requests and one for sync requests, and for the CMS thread
+  //  to check the sum of the two counters to decide whether it should yield
+  //  and to clear only the async counter when it yields. However, it turns out
+  //  to be more efficient for CMS code to just check a single counter
+  //  _pending_yields that holds the sum (of both sync and async requests), and
+  //  a second counter _pending_decrements that only holds the async requests,
+  //  for greater efficiency, since in a typical CMS run, there are many more
+  //  pontential (i.e. static) yield points than there are actual
+  //  (i.e. dynamic) yields because of requests, which are few and far between.
+  //
   // Note that, while "_pending_yields >= _pending_decrements" is an invariant,
   // we cannot easily test that invariant, since the counters are manipulated via
   // atomic instructions without explicit locking and we cannot read
   // the two counters atomically together: one suggestion is to
   // use (for example) 16-bit counters so as to be able to read the
   // two counters atomically even on 32-bit platforms. Notice that
-  // the second assert in acknowledge_yield_request() does indeed
+  // the second assert in acknowledge_yield_request() below does indeed
   // check a form of the above invariant, albeit indirectly.
 
   static void increment_pending_yields()   {
@@ -152,6 +172,7 @@
     assert(_pending_yields >= 0, "can't be negative");
   }
   static void asynchronous_yield_request() {
+    assert(CMSIncrementalMode, "Currently only used w/iCMS");
     increment_pending_yields();
     Atomic::inc(&_pending_decrements);
     assert(_pending_decrements >= 0, "can't be negative");
@@ -159,6 +180,7 @@
   static void acknowledge_yield_request() {
     jint decrement = _pending_decrements;
     if (decrement > 0) {
+      assert(CMSIncrementalMode, "Currently only used w/iCMS");
       // Order important to preserve: _pending_yields >= _pending_decrements
       Atomic::add(-decrement, &_pending_decrements);
       Atomic::add(-decrement, &_pending_yields);
@@ -195,7 +217,7 @@
   }
 }
 
-// For scoped increment/decrement of yield requests
+// For scoped increment/decrement of (synchronous) yield requests
 class CMSSynchronousYieldRequest: public StackObj {
  public:
   CMSSynchronousYieldRequest() {
--- a/src/share/vm/runtime/globals.hpp	Tue Sep 28 15:56:15 2010 -0700
+++ b/src/share/vm/runtime/globals.hpp	Wed Sep 29 16:17:02 2010 -0700
@@ -280,6 +280,10 @@
 //    UnlockExperimentalVMOptions flag, which allows the control and
 //    modification of the experimental flags.
 //
+// Nota bene: neither diagnostic nor experimental options should be used casually,
+//    and they are not supported on production loads, except under explicit
+//    direction from support engineers.
+//
 // manageable flags are writeable external product flags.
 //    They are dynamically writeable through the JDK management interface
 //    (com.sun.management.HotSpotDiagnosticMXBean API) and also through JConsole.
@@ -1809,17 +1813,17 @@
   develop(uintx, PromotionFailureALotInterval, 5,                           \
           "Total collections between promotion failures alot")              \
                                                                             \
-  develop(intx, WorkStealingSleepMillis, 1,                                 \
+  experimental(intx, WorkStealingSleepMillis, 1,                            \
           "Sleep time when sleep is used for yields")                       \
                                                                             \
-  develop(uintx, WorkStealingYieldsBeforeSleep, 1000,                       \
+  experimental(uintx, WorkStealingYieldsBeforeSleep, 1000,                  \
           "Number of yields before a sleep is done during workstealing")    \
                                                                             \
-  develop(uintx, WorkStealingHardSpins, 4096,                               \
+  experimental(uintx, WorkStealingHardSpins, 4096,                          \
           "Number of iterations in a spin loop between checks on "          \
           "time out of hard spin")                                          \
                                                                             \
-  develop(uintx, WorkStealingSpinToYieldRatio, 10,                          \
+  experimental(uintx, WorkStealingSpinToYieldRatio, 10,                     \
           "Ratio of hard spins to calls to yield")                          \
                                                                             \
   product(uintx, PreserveMarkStackSize, 1024,                               \
--- a/src/share/vm/utilities/yieldingWorkgroup.hpp	Tue Sep 28 15:56:15 2010 -0700
+++ b/src/share/vm/utilities/yieldingWorkgroup.hpp	Wed Sep 29 16:17:02 2010 -0700
@@ -1,5 +1,5 @@
 /*
- * Copyright (c) 2005, 2010 Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2005, 2010, Oracle and/or its affiliates. All rights reserved.
  * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
  *
  * This code is free software; you can redistribute it and/or modify it
@@ -137,6 +137,7 @@
   virtual void abort();
 
   Status status()  const { return _status; }
+  bool yielding()  const { return _status == YIELDING; }
   bool yielded()   const { return _status == YIELDED; }
   bool completed() const { return _status == COMPLETED; }
   bool aborted()   const { return _status == ABORTED; }