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

Print this page

        

@@ -573,10 +573,11 @@
   // Construct the is_alive_closure with _span & markBitMap
   _is_alive_closure(_span, &_markBitMap),
   _restart_addr(NULL),
   _overflow_list(NULL),
   _stats(cmsGen),
+  _eden_chunk_lock(new Mutex(Mutex::leaf + 1, "CMS_eden_chunk_lock", true)),
   _eden_chunk_array(NULL),     // may be set in ctor body
   _eden_chunk_capacity(0),     // -- ditto --
   _eden_chunk_index(0),        // -- ditto --
   _survivor_plab_array(NULL),  // -- ditto --
   _survivor_chunk_array(NULL), // -- ditto --

@@ -752,11 +753,11 @@
     }
   }
   assert(_eden_chunk_array != NULL || _eden_chunk_capacity == 0, "Error");
 
   // Support for parallelizing survivor space rescan
-  if (CMSParallelRemarkEnabled && CMSParallelSurvivorRemarkEnabled) {
+  if ((CMSParallelRemarkEnabled && CMSParallelSurvivorRemarkEnabled) || CMSParallelInitialMarkEnabled) {
     const size_t max_plab_samples =
       ((DefNewGeneration*)_young_gen)->max_survivor_size()/MinTLABSize;
 
     _survivor_plab_array  = NEW_C_HEAP_ARRAY(ChunkArray, ParallelGCThreads, mtGC);
     _survivor_chunk_array = NEW_C_HEAP_ARRAY(HeapWord*, 2*max_plab_samples, mtGC);

@@ -2134,10 +2135,43 @@
   // For a mark-sweep, compute_new_size() will be called
   // in the heap's do_collection() method.
 }
 
 
+void CMSCollector::print_eden_and_survivor_chunk_arrays() {
+  DefNewGeneration* dng = _young_gen->as_DefNewGeneration();
+  EdenSpace* 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 ")",
+                           eden_space->bottom(), eden_space->top(),
+                           eden_space->end(), eden_space->capacity());
+    gclog_or_tty->print_cr("_eden_chunk_index=" SIZE_FORMAT ", "
+                           "_eden_chunk_capacity=" SIZE_FORMAT,
+                           _eden_chunk_index, _eden_chunk_capacity);
+    for (size_t i = 0; i < _eden_chunk_index; i++) {
+      gclog_or_tty->print_cr("_eden_chunk_array[" SIZE_FORMAT "]=" PTR_FORMAT,
+                             i, _eden_chunk_array[i]);
+    }
+  }
+  // Survivor
+  if (_survivor_chunk_array != NULL) {
+    gclog_or_tty->print_cr("survivor " PTR_FORMAT "-" PTR_FORMAT "-" PTR_FORMAT "(" SIZE_FORMAT ")",
+                           from_space->bottom(), from_space->top(),
+                           from_space->end(), from_space->capacity());
+    gclog_or_tty->print_cr("_survivor_chunk_index=" SIZE_FORMAT ", "
+                           "_survivor_chunk_capacity=" SIZE_FORMAT,
+                           _survivor_chunk_index, _survivor_chunk_capacity);
+    for (size_t i = 0; i < _survivor_chunk_index; i++) {
+      gclog_or_tty->print_cr("_survivor_chunk_array[" SIZE_FORMAT "]=" PTR_FORMAT,
+                             i, _survivor_chunk_array[i]);
+    }
+  }
+}
+
 void CMSCollector::getFreelistLocks() const {
   // Get locks for all free lists in all generations that this
   // collector is responsible for
   _cmsGen->freelistLock()->lock_without_safepoint_check();
   _permGen->freelistLock()->lock_without_safepoint_check();

@@ -3524,10 +3558,35 @@
   }
 }
 
 // CMS work
 
+// The common parts of CMSParInitialMarkTask and CMSParRemarkTask.
+class CMSParMarkTask : public AbstractGangTask {
+ protected:
+  CMSCollector*     _collector;
+  int               _n_workers;
+  CMSParMarkTask(const char* name, CMSCollector* collector, int n_workers) :
+      AbstractGangTask(name),
+      _collector(collector),
+      _n_workers(n_workers) {}
+  // Work method in support of parallel rescan ... of young gen spaces
+  void do_young_space_rescan(uint worker_id, OopsInGenClosure* cl,
+                             ContiguousSpace* space,
+                             HeapWord** chunk_array, size_t chunk_top);
+  void work_on_young_gen_roots(uint worker_id, OopsInGenClosure* cl);
+};
+
+// Parallel initial mark task
+class CMSParInitialMarkTask: public CMSParMarkTask {
+ public:
+  CMSParInitialMarkTask(CMSCollector* collector, int n_workers) :
+      CMSParMarkTask("Scan roots and young gen for initial mark in parallel",
+                     collector, n_workers) {}
+  void work(uint worker_id);
+};
+
 // Checkpoint the roots into this generation from outside
 // this generation. [Note this initial checkpoint need only
 // be approximate -- we'll do a catch up phase subsequently.]
 void CMSCollector::checkpointRootsInitial(bool asynch) {
   assert(_collectorState == InitialMarking, "Wrong collector state");

@@ -3613,13 +3672,35 @@
   gch->save_marks();
 
   // weak reference processing has not started yet.
   ref_processor()->set_enqueuing_is_done(false);
 
+  if (CMSPrintEdenSurvivorChunks) {
+    print_eden_and_survivor_chunk_arrays();
+  }
+
   {
     // This is not needed. DEBUG_ONLY(RememberKlassesChecker imx(true);)
     COMPILER2_PRESENT(DerivedPointerTableDeactivate dpt_deact;)
+    if (CMSParallelInitialMarkEnabled && CollectedHeap::use_parallel_gc_threads()) {
+      // The parallel version.
+      FlexibleWorkGang* workers = gch->workers();
+      assert(workers != NULL, "Need parallel worker threads.");
+      int n_workers = workers->active_workers();
+      CMSParInitialMarkTask tsk(this, n_workers);
+      gch->set_par_threads(n_workers);
+      initialize_sequential_subtasks_for_young_gen_rescan(n_workers);
+      if (n_workers > 1) {
+        GenCollectedHeap::StrongRootsScope srs(gch);
+        workers->run_task(&tsk);
+      } else {
+        GenCollectedHeap::StrongRootsScope srs(gch);
+        tsk.work(0);
+      }
+      gch->set_par_threads(0);
+    } else {
+      // The serial version.
     gch->rem_set()->prepare_for_younger_refs_iterate(false); // Not parallel.
     gch->gen_process_strong_roots(_cmsGen->level(),
                                   true,   // younger gens are roots
                                   true,   // activate StrongRootsScope
                                   true,   // collecting perm gen

@@ -3626,11 +3707,11 @@
                                   SharedHeap::ScanningOption(roots_scanning_options()),
                                   &notOlder,
                                   true,   // walk all of code cache if (so & SO_CodeCache)
                                   NULL);
   }
-
+  }
   // Clear mod-union table; it will be dirtied in the prologue of
   // CMS generation per each younger generation collection.
 
   assert(_modUnionTable.isAllClear(),
        "Was cleared in most recent final checkpoint phase"

@@ -4408,11 +4489,13 @@
   assert(Thread::current()->is_ConcurrentGC_thread(), "Wrong thread");
   verify_work_stacks_empty();
   verify_overflow_empty();
   _abort_preclean = false;
   if (CMSPrecleaningEnabled) {
+    if (!CMSEdenChunksRecordAlways) {
     _eden_chunk_index = 0;
+    }
     size_t used = get_eden_used();
     size_t capacity = get_eden_capacity();
     // Don't start sampling unless we will get sufficiently
     // many samples.
     if (used < (capacity/(CMSScheduleRemarkSamplingRatio * 100)

@@ -4517,11 +4600,13 @@
   assert(ConcurrentMarkSweepThread::cms_thread_has_cms_token(),
          "Should collect samples while holding CMS token");
   if (!_start_sampling) {
     return;
   }
-  if (_eden_chunk_array) {
+  // When CMSEdenChunksRecordAlways is true, the eden chunk array
+  // is populated by the young generation.
+  if (_eden_chunk_array != NULL && !CMSEdenChunksRecordAlways) {
     if (_eden_chunk_index < _eden_chunk_capacity) {
       _eden_chunk_array[_eden_chunk_index] = *_top_addr;   // take sample
       assert(_eden_chunk_array[_eden_chunk_index] <= *_end_addr,
              "Unexpected state of Eden");
       // We'd like to check that what we just sampled is an oop-start address;

@@ -4994,10 +5079,14 @@
     // so here just in case a scavenge did not happen.
     gch->ensure_parsability(false);  // fill TLAB's, but no need to retire them
     // Update the saved marks which may affect the root scans.
     gch->save_marks();
 
+    if (CMSPrintEdenSurvivorChunks) {
+      print_eden_and_survivor_chunk_arrays();
+    }
+
     {
       COMPILER2_PRESENT(DerivedPointerTableDeactivate dpt_deact;)
 
       // Note on the role of the mod union table:
       // Since the marker in "markFromRoots" marks concurrently with

@@ -5101,14 +5190,55 @@
   if (UseAdaptiveSizePolicy) {
     size_policy()->checkpoint_roots_final_end(gch->gc_cause());
   }
 }
 
+void CMSParInitialMarkTask::work(uint worker_id) {
+  elapsedTimer _timer;
+  ResourceMark rm;
+  HandleMark   hm;
+
+  // ---------- scan from roots --------------
+  _timer.start();
+  GenCollectedHeap* gch = GenCollectedHeap::heap();
+  Par_MarkRefsIntoClosure par_mri_cl(_collector->_span, &(_collector->_markBitMap));
+
+  // ---------- young gen roots --------------
+  {
+    work_on_young_gen_roots(worker_id, &par_mri_cl);
+    _timer.stop();
+    if (PrintCMSStatistics != 0) {
+      gclog_or_tty->print_cr(
+        "Finished young gen initial mark scan work in %dth thread: %3.3f sec",
+        worker_id, _timer.seconds());
+    }
+  }
+
+  // ---------- remaining roots --------------
+  _timer.reset();
+  _timer.start();
+  gch->gen_process_strong_roots(_collector->_cmsGen->level(),
+                                false,     // yg was scanned above
+                                false,     // this is parallel code
+                                true,      // collecting perm gen
+                                SharedHeap::ScanningOption(_collector->CMSCollector::roots_scanning_options()),
+                                &par_mri_cl,
+                                true,   // walk all of code cache if (so & SO_CodeCache)
+                                NULL);
+  assert(_collector->should_unload_classes()
+         || (_collector->CMSCollector::roots_scanning_options() & SharedHeap::SO_CodeCache),
+         "if we didn't scan the code cache, we have to be ready to drop nmethods with expired weak oops");
+  _timer.stop();
+  if (PrintCMSStatistics != 0) {
+    gclog_or_tty->print_cr(
+      "Finished remaining root initial mark scan work in %dth thread: %3.3f sec",
+      worker_id, _timer.seconds());
+  }
+}
+
 // Parallel remark task
-class CMSParRemarkTask: public AbstractGangTask {
-  CMSCollector* _collector;
-  int           _n_workers;
+class CMSParRemarkTask: public CMSParMarkTask {
   CompactibleFreeListSpace* _cms_space;
   CompactibleFreeListSpace* _perm_space;
 
   // The per-thread work queues, available here for stealing.
   OopTaskQueueSet*       _task_queues;

@@ -5120,14 +5250,13 @@
   CMSParRemarkTask(CMSCollector* collector,
                    CompactibleFreeListSpace* cms_space,
                    CompactibleFreeListSpace* perm_space,
                    int n_workers, FlexibleWorkGang* workers,
                    OopTaskQueueSet* task_queues):
-    AbstractGangTask("Rescan roots and grey objects in parallel"),
-    _collector(collector),
+    CMSParMarkTask("Rescan roots and grey objects in parallel",
+                   collector, n_workers), 
     _cms_space(cms_space), _perm_space(perm_space),
-    _n_workers(n_workers),
     _task_queues(task_queues),
     _term(n_workers, task_queues) { }
 
   OopTaskQueueSet* task_queues() { return _task_queues; }
 

@@ -5137,23 +5266,37 @@
   int n_workers() { return _n_workers; }
 
   void work(uint worker_id);
 
  private:
-  // Work method in support of parallel rescan ... of young gen spaces
-  void do_young_space_rescan(int i, Par_MarkRefsIntoAndScanClosure* cl,
-                             ContiguousSpace* space,
-                             HeapWord** chunk_array, size_t chunk_top);
-
   // ... of  dirty cards in old space
   void do_dirty_card_rescan_tasks(CompactibleFreeListSpace* sp, int i,
                                   Par_MarkRefsIntoAndScanClosure* cl);
 
   // ... work stealing for the above
   void do_work_steal(int i, Par_MarkRefsIntoAndScanClosure* cl, int* seed);
 };
 
+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* from_space = dng->from();
+  ContiguousSpace* to_space   = dng->to();
+
+  HeapWord** eca = _collector->_eden_chunk_array;
+  size_t     ect = _collector->_eden_chunk_index;
+  HeapWord** sca = _collector->_survivor_chunk_array;
+  size_t     sct = _collector->_survivor_chunk_index;
+
+  assert(ect <= _collector->_eden_chunk_capacity, "out of bounds");
+  assert(sct <= _collector->_survivor_chunk_capacity, "out of bounds");
+
+  do_young_space_rescan(worker_id, cl, to_space, NULL, 0);
+  do_young_space_rescan(worker_id, cl, from_space, sca, sct);
+  do_young_space_rescan(worker_id, cl, eden_space, eca, ect);
+}
+
 // work_queue(i) is passed to the closure
 // Par_MarkRefsIntoAndScanClosure.  The "i" parameter
 // also is passed to do_dirty_card_rescan_tasks() and to
 // do_work_steal() to select the i-th task_queue.
 

@@ -5174,27 +5317,11 @@
   // coarsely partitioned and may, on that account, constitute
   // the critical path; thus, it's best to start off that
   // work first.
   // ---------- young gen roots --------------
   {
-    DefNewGeneration* dng = _collector->_young_gen->as_DefNewGeneration();
-    EdenSpace* 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;
-    HeapWord** sca = _collector->_survivor_chunk_array;
-    size_t     sct = _collector->_survivor_chunk_index;
-
-    assert(ect <= _collector->_eden_chunk_capacity, "out of bounds");
-    assert(sct <= _collector->_survivor_chunk_capacity, "out of bounds");
-
-    do_young_space_rescan(worker_id, &par_mrias_cl, to_space, NULL, 0);
-    do_young_space_rescan(worker_id, &par_mrias_cl, from_space, sca, sct);
-    do_young_space_rescan(worker_id, &par_mrias_cl, eden_space, eca, ect);
-
+    work_on_young_gen_roots(worker_id, &par_mrias_cl);
     _timer.stop();
     if (PrintCMSStatistics != 0) {
       gclog_or_tty->print_cr(
         "Finished young gen rescan work in %dth thread: %3.3f sec",
         worker_id, _timer.seconds());

@@ -5251,12 +5378,12 @@
   }
 }
 
 // Note that parameter "i" is not used.
 void
-CMSParRemarkTask::do_young_space_rescan(int i,
-  Par_MarkRefsIntoAndScanClosure* cl, ContiguousSpace* space,
+CMSParMarkTask::do_young_space_rescan(uint worker_id,
+  OopsInGenClosure* cl, ContiguousSpace* space,
   HeapWord** chunk_array, size_t chunk_top) {
   // Until all tasks completed:
   // . claim an unclaimed task
   // . compute region boundaries corresponding to task claimed
   //   using chunk_array

@@ -5448,10 +5575,36 @@
   )
   assert(work_q->size() == 0 && _collector->overflow_list_is_empty(),
          "Else our work is not yet done");
 }
 
+// Record object boundaries in _eden_chunk_array by sampling the eden
+// top in the slow-path eden object allocation code path and record
+// the boundaries, if CMSEdenChunksRecordAlways is true. If
+// CMSEdenChunksRecordAlways is false, we use the other asynchronous
+// sampling in sample_eden() that activates during the part of the
+// preclean phase.
+void CMSCollector::sample_eden_chunk() {
+  if (CMSEdenChunksRecordAlways && _eden_chunk_array != NULL) {
+    if (_eden_chunk_lock->try_lock()) {
+      // Record a sample. This is the critical section. The contents
+      // of the _eden_chunk_array have to be non-decreasing in the
+      // address order.
+      _eden_chunk_array[_eden_chunk_index] = *_top_addr;
+      assert(_eden_chunk_array[_eden_chunk_index] <= *_end_addr,
+             "Unexpected state of Eden");
+      if (_eden_chunk_index == 0 ||
+          ((_eden_chunk_array[_eden_chunk_index] > _eden_chunk_array[_eden_chunk_index-1]) &&
+           (pointer_delta(_eden_chunk_array[_eden_chunk_index],
+                          _eden_chunk_array[_eden_chunk_index-1]) >= CMSSamplingGrain))) {
+        _eden_chunk_index++;  // commit sample
+      }
+      _eden_chunk_lock->unlock();
+    }
+  }
+}
+
 // Return a thread-local PLAB recording array, as appropriate.
 void* CMSCollector::get_data_recorder(int thr_num) {
   if (_survivor_plab_array != NULL &&
       (CMSPLABRecordAlways ||
        (_collectorState > Marking && _collectorState < FinalMarking))) {

@@ -5471,16 +5624,17 @@
   }
 }
 
 // Merge the per-thread plab arrays into the global survivor chunk
 // array which will provide the partitioning of the survivor space
-// for CMS rescan.
+// for CMS initial scan and rescan.
 void CMSCollector::merge_survivor_plab_arrays(ContiguousSpace* surv,
                                               int no_of_gc_threads) {
   assert(_survivor_plab_array  != NULL, "Error");
   assert(_survivor_chunk_array != NULL, "Error");
-  assert(_collectorState == FinalMarking, "Error");
+  assert(_collectorState == FinalMarking ||
+         (CMSParallelInitialMarkEnabled && _collectorState == InitialMarking), "Error");
   for (int j = 0; j < no_of_gc_threads; j++) {
     _cursor[j] = 0;
   }
   HeapWord* top = surv->top();
   size_t i;

@@ -5539,11 +5693,11 @@
     }
   #endif // ASSERT
 }
 
 // Set up the space's par_seq_tasks structure for work claiming
-// for parallel rescan of young gen.
+// for parallel initial scan and rescan of young gen.
 // See ParRescanTask where this is currently used.
 void
 CMSCollector::
 initialize_sequential_subtasks_for_young_gen_rescan(int n_threads) {
   assert(n_threads > 0, "Unexpected n_threads argument");

@@ -6689,10 +6843,32 @@
 }
 
 void MarkRefsIntoClosure::do_oop(oop* p)       { MarkRefsIntoClosure::do_oop_work(p); }
 void MarkRefsIntoClosure::do_oop(narrowOop* p) { MarkRefsIntoClosure::do_oop_work(p); }
 
+Par_MarkRefsIntoClosure::Par_MarkRefsIntoClosure(
+  MemRegion span, CMSBitMap* bitMap):
+    _span(span),
+    _bitMap(bitMap)
+{
+    assert(_ref_processor == NULL, "deliberately left NULL");
+    assert(_bitMap->covers(_span), "_bitMap/_span mismatch");
+}
+
+void Par_MarkRefsIntoClosure::do_oop(oop obj) {
+  // if p points into _span, then mark corresponding bit in _markBitMap
+  assert(obj->is_oop(), "expected an oop");
+  HeapWord* addr = (HeapWord*)obj;
+  if (_span.contains(addr)) {
+    // this should be made more efficient
+    _bitMap->par_mark(addr);
+  }
+}
+
+void Par_MarkRefsIntoClosure::do_oop(oop* p)       { Par_MarkRefsIntoClosure::do_oop_work(p); }
+void Par_MarkRefsIntoClosure::do_oop(narrowOop* p) { Par_MarkRefsIntoClosure::do_oop_work(p); }
+
 // A variant of the above, used for CMS marking verification.
 MarkRefsIntoVerifyClosure::MarkRefsIntoVerifyClosure(
   MemRegion span, CMSBitMap* verification_bm, CMSBitMap* cms_bm):
     _span(span),
     _verification_bm(verification_bm),

@@ -9354,11 +9530,10 @@
     assert(chunk_at_end->end() == _cmsSpace->end(),
       "Shrinking is inconsistent");
     return;
   }
 }
-
 // Transfer some number of overflown objects to usual marking
 // stack. Return true if some objects were transferred.
 bool MarkRefsIntoAndScanClosure::take_from_overflow_list() {
   size_t num = MIN2((size_t)(_mark_stack->capacity() - _mark_stack->length())/4,
                     (size_t)ParGCDesiredObjsFromOverflowList);