< prev index next >

src/share/vm/gc/g1/g1CollectedHeap.cpp

Print this page
rev 8719 : 8004687: G1: Parallelize object self-forwarding and scanning during an evacuation failure
Summary: Use the regular task queue during evacuation failure and allow per-thread preserved header queues to remove the global lock during evacuation failure.
Reviewed-by:
Contributed-by: Walter Florian Gugenberger <walter.gugenberger@gmail.com>
rev 8720 : [mq]: mikael-suggestions


1900   verify_region_sets_optional();
1901 }
1902 
1903 // Public methods.
1904 
1905 #ifdef _MSC_VER // the use of 'this' below gets a warning, make it go away
1906 #pragma warning( disable:4355 ) // 'this' : used in base member initializer list
1907 #endif // _MSC_VER
1908 
1909 
1910 G1CollectedHeap::G1CollectedHeap(G1CollectorPolicy* policy_) :
1911   CollectedHeap(),
1912   _g1_policy(policy_),
1913   _dirty_card_queue_set(false),
1914   _into_cset_dirty_card_queue_set(false),
1915   _is_alive_closure_cm(this),
1916   _is_alive_closure_stw(this),
1917   _ref_processor_cm(NULL),
1918   _ref_processor_stw(NULL),
1919   _bot_shared(NULL),
1920   _evac_failure_scan_stack(NULL),
1921   _cg1r(NULL),
1922   _g1mm(NULL),
1923   _refine_cte_cl(NULL),
1924   _secondary_free_list("Secondary Free List", new SecondaryFreeRegionListMtSafeChecker()),
1925   _old_set("Old Set", false /* humongous */, new OldRegionSetMtSafeChecker()),
1926   _humongous_set("Master Humongous Set", true /* humongous */, new HumongousRegionSetMtSafeChecker()),
1927   _humongous_reclaim_candidates(),
1928   _has_humongous_reclaim_candidates(false),
1929   _archive_allocator(NULL),
1930   _free_regions_coming(false),
1931   _young_list(new YoungList(this)),
1932   _gc_time_stamp(0),
1933   _survivor_plab_stats(YoungPLABSize, PLABWeight),
1934   _old_plab_stats(OldPLABSize, PLABWeight),
1935   _expand_heap_after_alloc_failure(true),
1936   _surviving_young_words(NULL),
1937   _old_marking_cycles_started(0),
1938   _old_marking_cycles_completed(0),
1939   _heap_summary_sent(false),
1940   _in_cset_fast_test(),


2187   // G1AllocRegion class.
2188   HeapRegion* dummy_region = _hrm.get_dummy_region();
2189 
2190   // We'll re-use the same region whether the alloc region will
2191   // require BOT updates or not and, if it doesn't, then a non-young
2192   // region will complain that it cannot support allocations without
2193   // BOT updates. So we'll tag the dummy region as eden to avoid that.
2194   dummy_region->set_eden();
2195   // Make sure it's full.
2196   dummy_region->set_top(dummy_region->end());
2197   G1AllocRegion::setup(this, dummy_region);
2198 
2199   _allocator->init_mutator_alloc_region();
2200 
2201   // Do create of the monitoring and management support so that
2202   // values in the heap have been properly initialized.
2203   _g1mm = new G1MonitoringSupport(this);
2204 
2205   G1StringDedup::initialize();
2206 





2207   return JNI_OK;
2208 }
2209 
2210 void G1CollectedHeap::stop() {
2211   // Stop all concurrent threads. We do this to make sure these threads
2212   // do not continue to execute and access resources (e.g. gclog_or_tty)
2213   // that are destroyed during shutdown.
2214   _cg1r->stop();
2215   _cmThread->stop();
2216   if (G1StringDedup::is_enabled()) {
2217     G1StringDedup::stop();
2218   }
2219 }
2220 
2221 size_t G1CollectedHeap::conservative_max_heap_alignment() {
2222   return HeapRegion::max_region_size();
2223 }
2224 
2225 void G1CollectedHeap::post_initialize() {
2226   CollectedHeap::post_initialize();


4238     _gc_tracer_stw->report_gc_end(_gc_timer_stw->gc_end(), _gc_timer_stw->time_partitions());
4239   }
4240   // It should now be safe to tell the concurrent mark thread to start
4241   // without its logging output interfering with the logging output
4242   // that came from the pause.
4243 
4244   if (should_start_conc_mark) {
4245     // CAUTION: after the doConcurrentMark() call below,
4246     // the concurrent marking thread(s) could be running
4247     // concurrently with us. Make sure that anything after
4248     // this point does not assume that we are the only GC thread
4249     // running. Note: of course, the actual marking work will
4250     // not start until the safepoint itself is released in
4251     // SuspendibleThreadSet::desynchronize().
4252     doConcurrentMark();
4253   }
4254 
4255   return true;
4256 }
4257 
4258 void G1CollectedHeap::init_for_evac_failure(OopsInHeapRegionClosure* cl) {
4259   _drain_in_progress = false;
4260   set_evac_failure_closure(cl);
4261   _evac_failure_scan_stack = new (ResourceObj::C_HEAP, mtGC) GrowableArray<oop>(40, true);
4262 }
4263 
4264 void G1CollectedHeap::finalize_for_evac_failure() {
4265   assert(_evac_failure_scan_stack != NULL &&
4266          _evac_failure_scan_stack->length() == 0,
4267          "Postcondition");
4268   assert(!_drain_in_progress, "Postcondition");
4269   delete _evac_failure_scan_stack;
4270   _evac_failure_scan_stack = NULL;
4271 }
4272 
4273 void G1CollectedHeap::remove_self_forwarding_pointers() {
4274   double remove_self_forwards_start = os::elapsedTime();
4275 
4276   G1ParRemoveSelfForwardPtrsTask rsfp_task;
4277   workers()->run_task(&rsfp_task);
4278 
4279   // Now restore saved marks, if any.
4280   assert(_objs_with_preserved_marks.size() ==
4281             _preserved_marks_of_objs.size(), "Both or none.");
4282   while (!_objs_with_preserved_marks.is_empty()) {
4283     oop obj = _objs_with_preserved_marks.pop();
4284     markOop m = _preserved_marks_of_objs.pop();
4285     obj->set_mark(m);

4286   }
4287   _objs_with_preserved_marks.clear(true);
4288   _preserved_marks_of_objs.clear(true);
4289 
4290   g1_policy()->phase_times()->record_evac_fail_remove_self_forwards((os::elapsedTime() - remove_self_forwards_start) * 1000.0);
4291 }
4292 
4293 void G1CollectedHeap::push_on_evac_failure_scan_stack(oop obj) {
4294   _evac_failure_scan_stack->push(obj);
4295 }
4296 
4297 void G1CollectedHeap::drain_evac_failure_scan_stack() {
4298   assert(_evac_failure_scan_stack != NULL, "precondition");
4299 
4300   while (_evac_failure_scan_stack->length() > 0) {
4301      oop obj = _evac_failure_scan_stack->pop();
4302      _evac_failure_closure->set_region(heap_region_containing(obj));
4303      obj->oop_iterate_backwards(_evac_failure_closure);
4304   }
4305 }
4306 
4307 oop
4308 G1CollectedHeap::handle_evacuation_failure_par(G1ParScanThreadState* _par_scan_state,
4309                                                oop old) {
4310   assert(obj_in_cs(old),
4311          err_msg("obj: " PTR_FORMAT " should still be in the CSet",
4312                  p2i(old)));
4313   markOop m = old->mark();
4314   oop forward_ptr = old->forward_to_atomic(old);
4315   if (forward_ptr == NULL) {
4316     // Forward-to-self succeeded.
4317     assert(_par_scan_state != NULL, "par scan state");
4318     OopsInHeapRegionClosure* cl = _par_scan_state->evac_failure_closure();
4319     uint queue_num = _par_scan_state->queue_num();
4320 
4321     _evacuation_failed = true;
4322     _evacuation_failed_info_array[queue_num].register_copy_failure(old->size());
4323     if (_evac_failure_closure != cl) {
4324       MutexLockerEx x(EvacFailureStack_lock, Mutex::_no_safepoint_check_flag);
4325       assert(!_drain_in_progress,
4326              "Should only be true while someone holds the lock.");
4327       // Set the global evac-failure closure to the current thread's.
4328       assert(_evac_failure_closure == NULL, "Or locking has failed.");
4329       set_evac_failure_closure(cl);
4330       // Now do the common part.
4331       handle_evacuation_failure_common(old, m);
4332       // Reset to NULL.
4333       set_evac_failure_closure(NULL);
4334     } else {
4335       // The lock is already held, and this is recursive.
4336       assert(_drain_in_progress, "This should only be the recursive case.");
4337       handle_evacuation_failure_common(old, m);
4338     }
4339     return old;
4340   } else {
4341     // Forward-to-self failed. Either someone else managed to allocate
4342     // space for this object (old != forward_ptr) or they beat us in
4343     // self-forwarding it (old == forward_ptr).
4344     assert(old == forward_ptr || !obj_in_cs(forward_ptr),
4345            err_msg("obj: " PTR_FORMAT " forwarded to: " PTR_FORMAT " "
4346                    "should not be in the CSet",
4347                    p2i(old), p2i(forward_ptr)));
4348     return forward_ptr;
4349   }
4350 }
4351 
4352 void G1CollectedHeap::handle_evacuation_failure_common(oop old, markOop m) {
4353   preserve_mark_if_necessary(old, m);
4354 
4355   HeapRegion* r = heap_region_containing(old);
4356   if (!r->evacuation_failed()) {
4357     r->set_evacuation_failed(true);
4358     _hr_printer.evac_failure(r);
4359   }
4360 
4361   push_on_evac_failure_scan_stack(old);
4362 
4363   if (!_drain_in_progress) {
4364     // prevent recursion in copy_to_survivor_space()
4365     _drain_in_progress = true;
4366     drain_evac_failure_scan_stack();
4367     _drain_in_progress = false;
4368   }
4369 }
4370 
4371 void G1CollectedHeap::preserve_mark_if_necessary(oop obj, markOop m) {
4372   assert(evacuation_failed(), "Oversaving!");
4373   // We want to call the "for_promotion_failure" version only in the
4374   // case of a promotion failure.
4375   if (m->must_be_preserved_for_promotion_failure(obj)) {
4376     _objs_with_preserved_marks.push(obj);
4377     _preserved_marks_of_objs.push(m);
4378   }
4379 }
4380 
4381 void G1ParCopyHelper::mark_object(oop obj) {
4382   assert(!_g1->heap_region_containing(obj)->in_collection_set(), "should not mark objects in the CSet");
4383 
4384   // We know that the object is not moving so it's safe to read its size.
4385   _cm->grayRoot(obj, (size_t) obj->size(), _worker_id);
4386 }
4387 
4388 void G1ParCopyHelper::mark_forwarded_object(oop from_obj, oop to_obj) {
4389   assert(from_obj->is_forwarded(), "from obj should be forwarded");
4390   assert(from_obj->forwardee() == to_obj, "to obj should be the forwardee");
4391   assert(from_obj != to_obj, "should not be self-forwarded");
4392 
4393   assert(_g1->heap_region_containing(from_obj)->in_collection_set(), "from obj should be in the CSet");
4394   assert(!_g1->heap_region_containing(to_obj)->in_collection_set(), "should not mark objects in the CSet");
4395 
4396   // The object might be in the process of being copied by another
4397   // worker so we cannot trust that its to-space image is


4433     oopDesc::encode_store_heap_oop(p, forwardee);
4434     if (do_mark_object != G1MarkNone && forwardee != obj) {
4435       // If the object is self-forwarded we don't need to explicitly
4436       // mark it, the evacuation failure protocol will do so.
4437       mark_forwarded_object(obj, forwardee);
4438     }
4439 
4440     if (barrier == G1BarrierKlass) {
4441       do_klass_barrier(p, forwardee);
4442     }
4443   } else {
4444     if (state.is_humongous()) {
4445       _g1->set_humongous_is_live(obj);
4446     }
4447     // The object is not in collection set. If we're a root scanning
4448     // closure during an initial mark pause then attempt to mark the object.
4449     if (do_mark_object == G1MarkFromRoot) {
4450       mark_object(obj);
4451     }
4452   }
4453 
4454   if (barrier == G1BarrierEvac) {
4455     _par_scan_state->update_rs(_from, p, _worker_id);
4456   }
4457 }
4458 
4459 template void G1ParCopyClosure<G1BarrierEvac, G1MarkNone>::do_oop_work(oop* p);
4460 template void G1ParCopyClosure<G1BarrierEvac, G1MarkNone>::do_oop_work(narrowOop* p);
4461 
4462 class G1ParEvacuateFollowersClosure : public VoidClosure {
4463 protected:
4464   G1CollectedHeap*              _g1h;
4465   G1ParScanThreadState*         _par_scan_state;
4466   RefToScanQueueSet*            _queues;
4467   ParallelTaskTerminator*       _terminator;
4468 
4469   G1ParScanThreadState*   par_scan_state() { return _par_scan_state; }
4470   RefToScanQueueSet*      queues()         { return _queues; }
4471   ParallelTaskTerminator* terminator()     { return _terminator; }
4472 
4473 public:
4474   G1ParEvacuateFollowersClosure(G1CollectedHeap* g1h,
4475                                 G1ParScanThreadState* par_scan_state,
4476                                 RefToScanQueueSet* queues,
4477                                 ParallelTaskTerminator* terminator)
4478     : _g1h(g1h), _par_scan_state(par_scan_state),
4479       _queues(queues), _terminator(terminator) {}
4480 
4481   void do_void();


4580 
4581     }
4582 
4583     void do_cld(ClassLoaderData* cld) {
4584       cld->oops_do(_oop_closure, &_klass_in_cld_closure, _claim);
4585     }
4586   };
4587 
4588   void work(uint worker_id) {
4589     if (worker_id >= _n_workers) return;  // no work needed this round
4590 
4591     _g1h->g1_policy()->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerStart, worker_id, os::elapsedTime());
4592 
4593     {
4594       ResourceMark rm;
4595       HandleMark   hm;
4596 
4597       ReferenceProcessor*             rp = _g1h->ref_processor_stw();
4598 
4599       G1ParScanThreadState            pss(_g1h, worker_id, rp);
4600       G1ParScanHeapEvacFailureClosure evac_failure_cl(_g1h, &pss, rp);
4601 
4602       pss.set_evac_failure_closure(&evac_failure_cl);
4603 
4604       bool only_young = _g1h->collector_state()->gcs_are_young();
4605 
4606       // Non-IM young GC.
4607       G1ParCopyClosure<G1BarrierNone, G1MarkNone>             scan_only_root_cl(_g1h, &pss, rp);
4608       G1CLDClosure<G1MarkNone>                                scan_only_cld_cl(&scan_only_root_cl,
4609                                                                                only_young, // Only process dirty klasses.
4610                                                                                false);     // No need to claim CLDs.
4611       // IM young GC.
4612       //    Strong roots closures.
4613       G1ParCopyClosure<G1BarrierNone, G1MarkFromRoot>         scan_mark_root_cl(_g1h, &pss, rp);
4614       G1CLDClosure<G1MarkFromRoot>                            scan_mark_cld_cl(&scan_mark_root_cl,
4615                                                                                false, // Process all klasses.
4616                                                                                true); // Need to claim CLDs.
4617       //    Weak roots closures.
4618       G1ParCopyClosure<G1BarrierNone, G1MarkPromotedFromRoot> scan_mark_weak_root_cl(_g1h, &pss, rp);
4619       G1CLDClosure<G1MarkPromotedFromRoot>                    scan_mark_weak_cld_cl(&scan_mark_weak_root_cl,
4620                                                                                     false, // Process all klasses.
4621                                                                                     true); // Need to claim CLDs.
4622 


5252 public:
5253   G1STWRefProcTaskProxy(ProcessTask& proc_task,
5254                      G1CollectedHeap* g1h,
5255                      RefToScanQueueSet *task_queues,
5256                      ParallelTaskTerminator* terminator) :
5257     AbstractGangTask("Process reference objects in parallel"),
5258     _proc_task(proc_task),
5259     _g1h(g1h),
5260     _task_queues(task_queues),
5261     _terminator(terminator)
5262   {}
5263 
5264   virtual void work(uint worker_id) {
5265     // The reference processing task executed by a single worker.
5266     ResourceMark rm;
5267     HandleMark   hm;
5268 
5269     G1STWIsAliveClosure is_alive(_g1h);
5270 
5271     G1ParScanThreadState            pss(_g1h, worker_id, NULL);
5272     G1ParScanHeapEvacFailureClosure evac_failure_cl(_g1h, &pss, NULL);
5273 
5274     pss.set_evac_failure_closure(&evac_failure_cl);
5275 
5276     G1ParScanExtRootClosure        only_copy_non_heap_cl(_g1h, &pss, NULL);
5277 
5278     G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(_g1h, &pss, NULL);
5279 
5280     OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
5281 
5282     if (_g1h->collector_state()->during_initial_mark_pause()) {
5283       // We also need to mark copied objects.
5284       copy_non_heap_cl = &copy_mark_non_heap_cl;
5285     }
5286 
5287     // Keep alive closure.
5288     G1CopyingKeepAliveClosure keep_alive(_g1h, copy_non_heap_cl, &pss);
5289 
5290     // Complete GC closure
5291     G1ParEvacuateFollowersClosure drain_queue(_g1h, &pss, _task_queues, _terminator);
5292 
5293     // Call the reference processing task's work routine.
5294     _proc_task.work(worker_id, is_alive, keep_alive, drain_queue);


5351 protected:
5352   G1CollectedHeap* _g1h;
5353   RefToScanQueueSet      *_queues;
5354   ParallelTaskTerminator _terminator;
5355   uint _n_workers;
5356 
5357 public:
5358   G1ParPreserveCMReferentsTask(G1CollectedHeap* g1h, uint workers, RefToScanQueueSet *task_queues) :
5359     AbstractGangTask("ParPreserveCMReferents"),
5360     _g1h(g1h),
5361     _queues(task_queues),
5362     _terminator(workers, _queues),
5363     _n_workers(workers)
5364   { }
5365 
5366   void work(uint worker_id) {
5367     ResourceMark rm;
5368     HandleMark   hm;
5369 
5370     G1ParScanThreadState            pss(_g1h, worker_id, NULL);
5371     G1ParScanHeapEvacFailureClosure evac_failure_cl(_g1h, &pss, NULL);
5372 
5373     pss.set_evac_failure_closure(&evac_failure_cl);
5374 
5375     assert(pss.queue_is_empty(), "both queue and overflow should be empty");
5376 
5377     G1ParScanExtRootClosure        only_copy_non_heap_cl(_g1h, &pss, NULL);
5378 
5379     G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(_g1h, &pss, NULL);
5380 
5381     OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
5382 
5383     if (_g1h->collector_state()->during_initial_mark_pause()) {
5384       // We also need to mark copied objects.
5385       copy_non_heap_cl = &copy_mark_non_heap_cl;
5386     }
5387 
5388     // Is alive closure
5389     G1AlwaysAliveClosure always_alive(_g1h);
5390 
5391     // Copying keep alive closure. Applied to referent objects that need
5392     // to be copied.
5393     G1CopyingKeepAliveClosure keep_alive(_g1h, copy_non_heap_cl, &pss);
5394 


5459   // object discovered by the STW ref processor.
5460 
5461   uint no_of_gc_workers = workers()->active_workers();
5462 
5463   G1ParPreserveCMReferentsTask keep_cm_referents(this,
5464                                                  no_of_gc_workers,
5465                                                  _task_queues);
5466 
5467   workers()->run_task(&keep_cm_referents);
5468 
5469   // Closure to test whether a referent is alive.
5470   G1STWIsAliveClosure is_alive(this);
5471 
5472   // Even when parallel reference processing is enabled, the processing
5473   // of JNI refs is serial and performed serially by the current thread
5474   // rather than by a worker. The following PSS will be used for processing
5475   // JNI refs.
5476 
5477   // Use only a single queue for this PSS.
5478   G1ParScanThreadState            pss(this, 0, NULL);

5479 
5480   // We do not embed a reference processor in the copying/scanning
5481   // closures while we're actually processing the discovered
5482   // reference objects.
5483   G1ParScanHeapEvacFailureClosure evac_failure_cl(this, &pss, NULL);
5484 
5485   pss.set_evac_failure_closure(&evac_failure_cl);
5486 
5487   assert(pss.queue_is_empty(), "pre-condition");
5488 
5489   G1ParScanExtRootClosure        only_copy_non_heap_cl(this, &pss, NULL);
5490 
5491   G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(this, &pss, NULL);
5492 
5493   OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
5494 
5495   if (collector_state()->during_initial_mark_pause()) {
5496     // We also need to mark copied objects.
5497     copy_non_heap_cl = &copy_mark_non_heap_cl;
5498   }
5499 
5500   // Keep alive closure.
5501   G1CopyingKeepAliveClosure keep_alive(this, copy_non_heap_cl, &pss);
5502 
5503   // Serial Complete GC closure
5504   G1STWDrainQueueClosure drain_queue(this, &pss);
5505 
5506   // Setup the soft refs policy...
5507   rp->setup_policy(false);


5573   double ref_enq_time = os::elapsedTime() - ref_enq_start;
5574   g1_policy()->phase_times()->record_ref_enq_time(ref_enq_time * 1000.0);
5575 }
5576 
5577 void G1CollectedHeap::evacuate_collection_set(EvacuationInfo& evacuation_info) {
5578   _expand_heap_after_alloc_failure = true;
5579   _evacuation_failed = false;
5580 
5581   // Should G1EvacuationFailureALot be in effect for this GC?
5582   NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
5583 
5584   g1_rem_set()->prepare_for_oops_into_collection_set_do();
5585 
5586   // Disable the hot card cache.
5587   G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
5588   hot_card_cache->reset_hot_cache_claimed_index();
5589   hot_card_cache->set_use_cache(false);
5590 
5591   const uint n_workers = workers()->active_workers();
5592 
5593   init_for_evac_failure(NULL);
5594 
5595   assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty");
5596   double start_par_time_sec = os::elapsedTime();
5597   double end_par_time_sec;
5598 
5599   {
5600     G1RootProcessor root_processor(this, n_workers);
5601     G1ParTask g1_par_task(this, _task_queues, &root_processor, n_workers);
5602     // InitialMark needs claim bits to keep track of the marked-through CLDs.
5603     if (collector_state()->during_initial_mark_pause()) {
5604       ClassLoaderDataGraph::clear_claimed_marks();
5605     }
5606 
5607     // The individual threads will set their evac-failure closures.
5608     if (PrintTerminationStats) G1ParScanThreadState::print_termination_stats_hdr();
5609 
5610     workers()->run_task(&g1_par_task);
5611     end_par_time_sec = os::elapsedTime();
5612 
5613     // Closing the inner scope will execute the destructor
5614     // for the G1RootProcessor object. We record the current


5637     double fixup_start = os::elapsedTime();
5638 
5639     G1STWIsAliveClosure is_alive(this);
5640     G1KeepAliveClosure keep_alive(this);
5641     G1StringDedup::unlink_or_oops_do(&is_alive, &keep_alive, true, phase_times);
5642 
5643     double fixup_time_ms = (os::elapsedTime() - fixup_start) * 1000.0;
5644     phase_times->record_string_dedup_fixup_time(fixup_time_ms);
5645   }
5646 
5647   _allocator->release_gc_alloc_regions(evacuation_info);
5648   g1_rem_set()->cleanup_after_oops_into_collection_set_do();
5649 
5650   // Reset and re-enable the hot card cache.
5651   // Note the counts for the cards in the regions in the
5652   // collection set are reset when the collection set is freed.
5653   hot_card_cache->reset_hot_cache();
5654   hot_card_cache->set_use_cache(true);
5655 
5656   purge_code_root_memory();
5657 
5658   finalize_for_evac_failure();
5659 
5660   if (evacuation_failed()) {
5661     remove_self_forwarding_pointers();
5662 
5663     // Reset the G1EvacuationFailureALot counters and flags
5664     // Note: the values are reset only when an actual
5665     // evacuation failure occurs.
5666     NOT_PRODUCT(reset_evacuation_should_fail();)
5667   }
5668 
5669   // Enqueue any remaining references remaining on the STW
5670   // reference processor's discovered lists. We need to do
5671   // this after the card table is cleaned (and verified) as
5672   // the act of enqueueing entries on to the pending list
5673   // will log these updates (and dirty their associated
5674   // cards). We need these updates logged to update any
5675   // RSets.
5676   enqueue_discovered_references();
5677 
5678   redirty_logged_cards();




1900   verify_region_sets_optional();
1901 }
1902 
1903 // Public methods.
1904 
1905 #ifdef _MSC_VER // the use of 'this' below gets a warning, make it go away
1906 #pragma warning( disable:4355 ) // 'this' : used in base member initializer list
1907 #endif // _MSC_VER
1908 
1909 
1910 G1CollectedHeap::G1CollectedHeap(G1CollectorPolicy* policy_) :
1911   CollectedHeap(),
1912   _g1_policy(policy_),
1913   _dirty_card_queue_set(false),
1914   _into_cset_dirty_card_queue_set(false),
1915   _is_alive_closure_cm(this),
1916   _is_alive_closure_stw(this),
1917   _ref_processor_cm(NULL),
1918   _ref_processor_stw(NULL),
1919   _bot_shared(NULL),

1920   _cg1r(NULL),
1921   _g1mm(NULL),
1922   _refine_cte_cl(NULL),
1923   _secondary_free_list("Secondary Free List", new SecondaryFreeRegionListMtSafeChecker()),
1924   _old_set("Old Set", false /* humongous */, new OldRegionSetMtSafeChecker()),
1925   _humongous_set("Master Humongous Set", true /* humongous */, new HumongousRegionSetMtSafeChecker()),
1926   _humongous_reclaim_candidates(),
1927   _has_humongous_reclaim_candidates(false),
1928   _archive_allocator(NULL),
1929   _free_regions_coming(false),
1930   _young_list(new YoungList(this)),
1931   _gc_time_stamp(0),
1932   _survivor_plab_stats(YoungPLABSize, PLABWeight),
1933   _old_plab_stats(OldPLABSize, PLABWeight),
1934   _expand_heap_after_alloc_failure(true),
1935   _surviving_young_words(NULL),
1936   _old_marking_cycles_started(0),
1937   _old_marking_cycles_completed(0),
1938   _heap_summary_sent(false),
1939   _in_cset_fast_test(),


2186   // G1AllocRegion class.
2187   HeapRegion* dummy_region = _hrm.get_dummy_region();
2188 
2189   // We'll re-use the same region whether the alloc region will
2190   // require BOT updates or not and, if it doesn't, then a non-young
2191   // region will complain that it cannot support allocations without
2192   // BOT updates. So we'll tag the dummy region as eden to avoid that.
2193   dummy_region->set_eden();
2194   // Make sure it's full.
2195   dummy_region->set_top(dummy_region->end());
2196   G1AllocRegion::setup(this, dummy_region);
2197 
2198   _allocator->init_mutator_alloc_region();
2199 
2200   // Do create of the monitoring and management support so that
2201   // values in the heap have been properly initialized.
2202   _g1mm = new G1MonitoringSupport(this);
2203 
2204   G1StringDedup::initialize();
2205 
2206   _preserved_objs = NEW_C_HEAP_ARRAY(OopAndMarkOopStack, ParallelGCThreads, mtGC);
2207   for (uint i = 0; i < ParallelGCThreads; i++) {
2208     new (&_preserved_objs[i]) OopAndMarkOopStack();
2209   }
2210 
2211   return JNI_OK;
2212 }
2213 
2214 void G1CollectedHeap::stop() {
2215   // Stop all concurrent threads. We do this to make sure these threads
2216   // do not continue to execute and access resources (e.g. gclog_or_tty)
2217   // that are destroyed during shutdown.
2218   _cg1r->stop();
2219   _cmThread->stop();
2220   if (G1StringDedup::is_enabled()) {
2221     G1StringDedup::stop();
2222   }
2223 }
2224 
2225 size_t G1CollectedHeap::conservative_max_heap_alignment() {
2226   return HeapRegion::max_region_size();
2227 }
2228 
2229 void G1CollectedHeap::post_initialize() {
2230   CollectedHeap::post_initialize();


4242     _gc_tracer_stw->report_gc_end(_gc_timer_stw->gc_end(), _gc_timer_stw->time_partitions());
4243   }
4244   // It should now be safe to tell the concurrent mark thread to start
4245   // without its logging output interfering with the logging output
4246   // that came from the pause.
4247 
4248   if (should_start_conc_mark) {
4249     // CAUTION: after the doConcurrentMark() call below,
4250     // the concurrent marking thread(s) could be running
4251     // concurrently with us. Make sure that anything after
4252     // this point does not assume that we are the only GC thread
4253     // running. Note: of course, the actual marking work will
4254     // not start until the safepoint itself is released in
4255     // SuspendibleThreadSet::desynchronize().
4256     doConcurrentMark();
4257   }
4258 
4259   return true;
4260 }
4261 















4262 void G1CollectedHeap::remove_self_forwarding_pointers() {
4263   double remove_self_forwards_start = os::elapsedTime();
4264 
4265   G1ParRemoveSelfForwardPtrsTask rsfp_task;
4266   workers()->run_task(&rsfp_task);
4267 
4268   // Now restore saved marks, if any.
4269   for (uint i = 0; i < ParallelGCThreads; i++) {
4270     OopAndMarkOopStack& cur = _preserved_objs[i];
4271     while (!cur.is_empty()) {
4272       OopAndMarkOop elem = cur.pop();
4273       elem.set_mark();
4274     }
4275     cur.clear(true);
4276   }


4277 
4278   g1_policy()->phase_times()->record_evac_fail_remove_self_forwards((os::elapsedTime() - remove_self_forwards_start) * 1000.0);
4279 }
4280 
4281 void G1CollectedHeap::preserve_mark_during_evac_failure(uint queue_num, oop obj, markOop m) {
4282   if (!_evacuation_failed) {


























4283     _evacuation_failed = true;
















4284   }












4285 
4286   _evacuation_failed_info_array[queue_num].register_copy_failure(obj->size());

4287 


















4288   // We want to call the "for_promotion_failure" version only in the
4289   // case of a promotion failure.
4290   if (m->must_be_preserved_for_promotion_failure(obj)) {
4291     OopAndMarkOop elem(obj, m);
4292     _preserved_objs[queue_num].push(elem);
4293   }
4294 }
4295 
4296 void G1ParCopyHelper::mark_object(oop obj) {
4297   assert(!_g1->heap_region_containing(obj)->in_collection_set(), "should not mark objects in the CSet");
4298 
4299   // We know that the object is not moving so it's safe to read its size.
4300   _cm->grayRoot(obj, (size_t) obj->size(), _worker_id);
4301 }
4302 
4303 void G1ParCopyHelper::mark_forwarded_object(oop from_obj, oop to_obj) {
4304   assert(from_obj->is_forwarded(), "from obj should be forwarded");
4305   assert(from_obj->forwardee() == to_obj, "to obj should be the forwardee");
4306   assert(from_obj != to_obj, "should not be self-forwarded");
4307 
4308   assert(_g1->heap_region_containing(from_obj)->in_collection_set(), "from obj should be in the CSet");
4309   assert(!_g1->heap_region_containing(to_obj)->in_collection_set(), "should not mark objects in the CSet");
4310 
4311   // The object might be in the process of being copied by another
4312   // worker so we cannot trust that its to-space image is


4348     oopDesc::encode_store_heap_oop(p, forwardee);
4349     if (do_mark_object != G1MarkNone && forwardee != obj) {
4350       // If the object is self-forwarded we don't need to explicitly
4351       // mark it, the evacuation failure protocol will do so.
4352       mark_forwarded_object(obj, forwardee);
4353     }
4354 
4355     if (barrier == G1BarrierKlass) {
4356       do_klass_barrier(p, forwardee);
4357     }
4358   } else {
4359     if (state.is_humongous()) {
4360       _g1->set_humongous_is_live(obj);
4361     }
4362     // The object is not in collection set. If we're a root scanning
4363     // closure during an initial mark pause then attempt to mark the object.
4364     if (do_mark_object == G1MarkFromRoot) {
4365       mark_object(obj);
4366     }
4367   }




4368 }
4369 



4370 class G1ParEvacuateFollowersClosure : public VoidClosure {
4371 protected:
4372   G1CollectedHeap*              _g1h;
4373   G1ParScanThreadState*         _par_scan_state;
4374   RefToScanQueueSet*            _queues;
4375   ParallelTaskTerminator*       _terminator;
4376 
4377   G1ParScanThreadState*   par_scan_state() { return _par_scan_state; }
4378   RefToScanQueueSet*      queues()         { return _queues; }
4379   ParallelTaskTerminator* terminator()     { return _terminator; }
4380 
4381 public:
4382   G1ParEvacuateFollowersClosure(G1CollectedHeap* g1h,
4383                                 G1ParScanThreadState* par_scan_state,
4384                                 RefToScanQueueSet* queues,
4385                                 ParallelTaskTerminator* terminator)
4386     : _g1h(g1h), _par_scan_state(par_scan_state),
4387       _queues(queues), _terminator(terminator) {}
4388 
4389   void do_void();


4488 
4489     }
4490 
4491     void do_cld(ClassLoaderData* cld) {
4492       cld->oops_do(_oop_closure, &_klass_in_cld_closure, _claim);
4493     }
4494   };
4495 
4496   void work(uint worker_id) {
4497     if (worker_id >= _n_workers) return;  // no work needed this round
4498 
4499     _g1h->g1_policy()->phase_times()->record_time_secs(G1GCPhaseTimes::GCWorkerStart, worker_id, os::elapsedTime());
4500 
4501     {
4502       ResourceMark rm;
4503       HandleMark   hm;
4504 
4505       ReferenceProcessor*             rp = _g1h->ref_processor_stw();
4506 
4507       G1ParScanThreadState            pss(_g1h, worker_id, rp);



4508 
4509       bool only_young = _g1h->collector_state()->gcs_are_young();
4510 
4511       // Non-IM young GC.
4512       G1ParCopyClosure<G1BarrierNone, G1MarkNone>             scan_only_root_cl(_g1h, &pss, rp);
4513       G1CLDClosure<G1MarkNone>                                scan_only_cld_cl(&scan_only_root_cl,
4514                                                                                only_young, // Only process dirty klasses.
4515                                                                                false);     // No need to claim CLDs.
4516       // IM young GC.
4517       //    Strong roots closures.
4518       G1ParCopyClosure<G1BarrierNone, G1MarkFromRoot>         scan_mark_root_cl(_g1h, &pss, rp);
4519       G1CLDClosure<G1MarkFromRoot>                            scan_mark_cld_cl(&scan_mark_root_cl,
4520                                                                                false, // Process all klasses.
4521                                                                                true); // Need to claim CLDs.
4522       //    Weak roots closures.
4523       G1ParCopyClosure<G1BarrierNone, G1MarkPromotedFromRoot> scan_mark_weak_root_cl(_g1h, &pss, rp);
4524       G1CLDClosure<G1MarkPromotedFromRoot>                    scan_mark_weak_cld_cl(&scan_mark_weak_root_cl,
4525                                                                                     false, // Process all klasses.
4526                                                                                     true); // Need to claim CLDs.
4527 


5157 public:
5158   G1STWRefProcTaskProxy(ProcessTask& proc_task,
5159                      G1CollectedHeap* g1h,
5160                      RefToScanQueueSet *task_queues,
5161                      ParallelTaskTerminator* terminator) :
5162     AbstractGangTask("Process reference objects in parallel"),
5163     _proc_task(proc_task),
5164     _g1h(g1h),
5165     _task_queues(task_queues),
5166     _terminator(terminator)
5167   {}
5168 
5169   virtual void work(uint worker_id) {
5170     // The reference processing task executed by a single worker.
5171     ResourceMark rm;
5172     HandleMark   hm;
5173 
5174     G1STWIsAliveClosure is_alive(_g1h);
5175 
5176     G1ParScanThreadState            pss(_g1h, worker_id, NULL);



5177 
5178     G1ParScanExtRootClosure        only_copy_non_heap_cl(_g1h, &pss, NULL);
5179 
5180     G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(_g1h, &pss, NULL);
5181 
5182     OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
5183 
5184     if (_g1h->collector_state()->during_initial_mark_pause()) {
5185       // We also need to mark copied objects.
5186       copy_non_heap_cl = &copy_mark_non_heap_cl;
5187     }
5188 
5189     // Keep alive closure.
5190     G1CopyingKeepAliveClosure keep_alive(_g1h, copy_non_heap_cl, &pss);
5191 
5192     // Complete GC closure
5193     G1ParEvacuateFollowersClosure drain_queue(_g1h, &pss, _task_queues, _terminator);
5194 
5195     // Call the reference processing task's work routine.
5196     _proc_task.work(worker_id, is_alive, keep_alive, drain_queue);


5253 protected:
5254   G1CollectedHeap* _g1h;
5255   RefToScanQueueSet      *_queues;
5256   ParallelTaskTerminator _terminator;
5257   uint _n_workers;
5258 
5259 public:
5260   G1ParPreserveCMReferentsTask(G1CollectedHeap* g1h, uint workers, RefToScanQueueSet *task_queues) :
5261     AbstractGangTask("ParPreserveCMReferents"),
5262     _g1h(g1h),
5263     _queues(task_queues),
5264     _terminator(workers, _queues),
5265     _n_workers(workers)
5266   { }
5267 
5268   void work(uint worker_id) {
5269     ResourceMark rm;
5270     HandleMark   hm;
5271 
5272     G1ParScanThreadState            pss(_g1h, worker_id, NULL);




5273     assert(pss.queue_is_empty(), "both queue and overflow should be empty");
5274 
5275     G1ParScanExtRootClosure        only_copy_non_heap_cl(_g1h, &pss, NULL);
5276 
5277     G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(_g1h, &pss, NULL);
5278 
5279     OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
5280 
5281     if (_g1h->collector_state()->during_initial_mark_pause()) {
5282       // We also need to mark copied objects.
5283       copy_non_heap_cl = &copy_mark_non_heap_cl;
5284     }
5285 
5286     // Is alive closure
5287     G1AlwaysAliveClosure always_alive(_g1h);
5288 
5289     // Copying keep alive closure. Applied to referent objects that need
5290     // to be copied.
5291     G1CopyingKeepAliveClosure keep_alive(_g1h, copy_non_heap_cl, &pss);
5292 


5357   // object discovered by the STW ref processor.
5358 
5359   uint no_of_gc_workers = workers()->active_workers();
5360 
5361   G1ParPreserveCMReferentsTask keep_cm_referents(this,
5362                                                  no_of_gc_workers,
5363                                                  _task_queues);
5364 
5365   workers()->run_task(&keep_cm_referents);
5366 
5367   // Closure to test whether a referent is alive.
5368   G1STWIsAliveClosure is_alive(this);
5369 
5370   // Even when parallel reference processing is enabled, the processing
5371   // of JNI refs is serial and performed serially by the current thread
5372   // rather than by a worker. The following PSS will be used for processing
5373   // JNI refs.
5374 
5375   // Use only a single queue for this PSS.
5376   G1ParScanThreadState            pss(this, 0, NULL);
5377   assert(pss.queue_is_empty(), "pre-condition");
5378 
5379   // We do not embed a reference processor in the copying/scanning
5380   // closures while we're actually processing the discovered
5381   // reference objects.





5382 
5383   G1ParScanExtRootClosure        only_copy_non_heap_cl(this, &pss, NULL);
5384 
5385   G1ParScanAndMarkExtRootClosure copy_mark_non_heap_cl(this, &pss, NULL);
5386 
5387   OopClosure*                    copy_non_heap_cl = &only_copy_non_heap_cl;
5388 
5389   if (collector_state()->during_initial_mark_pause()) {
5390     // We also need to mark copied objects.
5391     copy_non_heap_cl = &copy_mark_non_heap_cl;
5392   }
5393 
5394   // Keep alive closure.
5395   G1CopyingKeepAliveClosure keep_alive(this, copy_non_heap_cl, &pss);
5396 
5397   // Serial Complete GC closure
5398   G1STWDrainQueueClosure drain_queue(this, &pss);
5399 
5400   // Setup the soft refs policy...
5401   rp->setup_policy(false);


5467   double ref_enq_time = os::elapsedTime() - ref_enq_start;
5468   g1_policy()->phase_times()->record_ref_enq_time(ref_enq_time * 1000.0);
5469 }
5470 
5471 void G1CollectedHeap::evacuate_collection_set(EvacuationInfo& evacuation_info) {
5472   _expand_heap_after_alloc_failure = true;
5473   _evacuation_failed = false;
5474 
5475   // Should G1EvacuationFailureALot be in effect for this GC?
5476   NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
5477 
5478   g1_rem_set()->prepare_for_oops_into_collection_set_do();
5479 
5480   // Disable the hot card cache.
5481   G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
5482   hot_card_cache->reset_hot_cache_claimed_index();
5483   hot_card_cache->set_use_cache(false);
5484 
5485   const uint n_workers = workers()->active_workers();
5486 


5487   assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty");
5488   double start_par_time_sec = os::elapsedTime();
5489   double end_par_time_sec;
5490 
5491   {
5492     G1RootProcessor root_processor(this, n_workers);
5493     G1ParTask g1_par_task(this, _task_queues, &root_processor, n_workers);
5494     // InitialMark needs claim bits to keep track of the marked-through CLDs.
5495     if (collector_state()->during_initial_mark_pause()) {
5496       ClassLoaderDataGraph::clear_claimed_marks();
5497     }
5498 
5499     // The individual threads will set their evac-failure closures.
5500     if (PrintTerminationStats) G1ParScanThreadState::print_termination_stats_hdr();
5501 
5502     workers()->run_task(&g1_par_task);
5503     end_par_time_sec = os::elapsedTime();
5504 
5505     // Closing the inner scope will execute the destructor
5506     // for the G1RootProcessor object. We record the current


5529     double fixup_start = os::elapsedTime();
5530 
5531     G1STWIsAliveClosure is_alive(this);
5532     G1KeepAliveClosure keep_alive(this);
5533     G1StringDedup::unlink_or_oops_do(&is_alive, &keep_alive, true, phase_times);
5534 
5535     double fixup_time_ms = (os::elapsedTime() - fixup_start) * 1000.0;
5536     phase_times->record_string_dedup_fixup_time(fixup_time_ms);
5537   }
5538 
5539   _allocator->release_gc_alloc_regions(evacuation_info);
5540   g1_rem_set()->cleanup_after_oops_into_collection_set_do();
5541 
5542   // Reset and re-enable the hot card cache.
5543   // Note the counts for the cards in the regions in the
5544   // collection set are reset when the collection set is freed.
5545   hot_card_cache->reset_hot_cache();
5546   hot_card_cache->set_use_cache(true);
5547 
5548   purge_code_root_memory();


5549 
5550   if (evacuation_failed()) {
5551     remove_self_forwarding_pointers();
5552 
5553     // Reset the G1EvacuationFailureALot counters and flags
5554     // Note: the values are reset only when an actual
5555     // evacuation failure occurs.
5556     NOT_PRODUCT(reset_evacuation_should_fail();)
5557   }
5558 
5559   // Enqueue any remaining references remaining on the STW
5560   // reference processor's discovered lists. We need to do
5561   // this after the card table is cleaned (and verified) as
5562   // the act of enqueueing entries on to the pending list
5563   // will log these updates (and dirty their associated
5564   // cards). We need these updates logged to update any
5565   // RSets.
5566   enqueue_discovered_references();
5567 
5568   redirty_logged_cards();


< prev index next >