< prev index next >

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

Print this page




1312 
1313         print_hrm_post_compaction();
1314         _hr_printer.end_gc(true /* full */, (size_t) total_collections());
1315       }
1316 
1317       G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
1318       if (hot_card_cache->use_cache()) {
1319         hot_card_cache->reset_card_counts();
1320         hot_card_cache->reset_hot_cache();
1321       }
1322 
1323       // Rebuild remembered sets of all regions.
1324       uint n_workers =
1325         AdaptiveSizePolicy::calc_active_workers(workers()->total_workers(),
1326                                                 workers()->active_workers(),
1327                                                 Threads::number_of_non_daemon_threads());
1328       assert(UseDynamicNumberOfGCThreads ||
1329              n_workers == workers()->total_workers(),
1330              "If not dynamic should be using all the  workers");
1331       workers()->set_active_workers(n_workers);
1332       // Set parallel threads in the heap (_n_par_threads) only
1333       // before a parallel phase and always reset it to 0 after
1334       // the phase so that the number of parallel threads does
1335       // no get carried forward to a serial phase where there
1336       // may be code that is "possibly_parallel".
1337       set_par_threads(n_workers);
1338 
1339       ParRebuildRSTask rebuild_rs_task(this);
1340       assert(UseDynamicNumberOfGCThreads ||
1341              workers()->active_workers() == workers()->total_workers(),
1342              "Unless dynamic should use total workers");
1343       // Use the most recent number of  active workers
1344       assert(workers()->active_workers() > 0,
1345              "Active workers not properly set");
1346       set_par_threads(workers()->active_workers());
1347       workers()->run_task(&rebuild_rs_task);
1348       set_par_threads(0);
1349 
1350       // Rebuild the strong code root lists for each region
1351       rebuild_strong_code_roots();
1352 
1353       if (true) { // FIXME
1354         MetaspaceGC::compute_new_size();
1355       }
1356 
1357 #ifdef TRACESPINNING
1358       ParallelTaskTerminator::print_termination_counts();
1359 #endif
1360 
1361       // Discard all rset updates
1362       JavaThread::dirty_card_queue_set().abandon_logs();
1363       assert(dirty_card_queue_set().completed_buffers_num() == 0, "DCQS should be empty");
1364 
1365       _young_list->reset_sampled_info();
1366       // At this point there should be no regions in the
1367       // entire heap tagged as young.
1368       assert(check_young_list_empty(true /* check_heap */),


3003   }
3004 };
3005 
3006 void G1CollectedHeap::verify(bool silent, VerifyOption vo) {
3007   if (SafepointSynchronize::is_at_safepoint()) {
3008     assert(Thread::current()->is_VM_thread(),
3009            "Expected to be executed serially by the VM thread at this point");
3010 
3011     if (!silent) { gclog_or_tty->print("Roots "); }
3012     VerifyRootsClosure rootsCl(vo);
3013     VerifyKlassClosure klassCl(this, &rootsCl);
3014     CLDToKlassAndOopClosure cldCl(&klassCl, &rootsCl, false);
3015 
3016     // We apply the relevant closures to all the oops in the
3017     // system dictionary, class loader data graph, the string table
3018     // and the nmethods in the code cache.
3019     G1VerifyCodeRootOopClosure codeRootsCl(this, &rootsCl, vo);
3020     G1VerifyCodeRootBlobClosure blobsCl(&codeRootsCl);
3021 
3022     {
3023       G1RootProcessor root_processor(this);
3024       root_processor.process_all_roots(&rootsCl,
3025                                        &cldCl,
3026                                        &blobsCl);
3027     }
3028 
3029     bool failures = rootsCl.failures() || codeRootsCl.failures();
3030 
3031     if (vo != VerifyOption_G1UseMarkWord) {
3032       // If we're verifying during a full GC then the region sets
3033       // will have been torn down at the start of the GC. Therefore
3034       // verifying the region sets will fail. So we only verify
3035       // the region sets when not in a full GC.
3036       if (!silent) { gclog_or_tty->print("HeapRegionSets "); }
3037       verify_region_sets();
3038     }
3039 
3040     if (!silent) { gclog_or_tty->print("HeapRegions "); }
3041     if (GCParallelVerificationEnabled && ParallelGCThreads > 1) {
3042 
3043       G1ParVerifyTask task(this, vo);
3044       assert(UseDynamicNumberOfGCThreads ||
3045         workers()->active_workers() == workers()->total_workers(),
3046         "If not dynamic should be using all the workers");
3047       uint n_workers = workers()->active_workers();
3048       set_par_threads(n_workers);
3049       workers()->run_task(&task);
3050       set_par_threads(0);
3051       if (task.failures()) {
3052         failures = true;
3053       }
3054 
3055     } else {
3056       VerifyRegionClosure blk(false, vo);
3057       heap_region_iterate(&blk);
3058       if (blk.failures()) {
3059         failures = true;
3060       }
3061     }
3062 
3063     if (G1StringDedup::is_enabled()) {
3064       if (!silent) gclog_or_tty->print("StrDedup ");
3065       G1StringDedup::verify();
3066     }
3067 
3068     if (failures) {
3069       gclog_or_tty->print_cr("Heap:");
3070       // It helps to have the per-region information in the output to


4023 }
4024 
4025 void G1CollectedHeap::init_for_evac_failure(OopsInHeapRegionClosure* cl) {
4026   _drain_in_progress = false;
4027   set_evac_failure_closure(cl);
4028   _evac_failure_scan_stack = new (ResourceObj::C_HEAP, mtGC) GrowableArray<oop>(40, true);
4029 }
4030 
4031 void G1CollectedHeap::finalize_for_evac_failure() {
4032   assert(_evac_failure_scan_stack != NULL &&
4033          _evac_failure_scan_stack->length() == 0,
4034          "Postcondition");
4035   assert(!_drain_in_progress, "Postcondition");
4036   delete _evac_failure_scan_stack;
4037   _evac_failure_scan_stack = NULL;
4038 }
4039 
4040 void G1CollectedHeap::remove_self_forwarding_pointers() {
4041   double remove_self_forwards_start = os::elapsedTime();
4042 
4043   set_par_threads();
4044   G1ParRemoveSelfForwardPtrsTask rsfp_task(this);
4045   workers()->run_task(&rsfp_task);
4046   set_par_threads(0);
4047 
4048   // Now restore saved marks, if any.
4049   assert(_objs_with_preserved_marks.size() ==
4050             _preserved_marks_of_objs.size(), "Both or none.");
4051   while (!_objs_with_preserved_marks.is_empty()) {
4052     oop obj = _objs_with_preserved_marks.pop();
4053     markOop m = _preserved_marks_of_objs.pop();
4054     obj->set_mark(m);
4055   }
4056   _objs_with_preserved_marks.clear(true);
4057   _preserved_marks_of_objs.clear(true);
4058 
4059   g1_policy()->phase_times()->record_evac_fail_remove_self_forwards((os::elapsedTime() - remove_self_forwards_start) * 1000.0);
4060 }
4061 
4062 void G1CollectedHeap::push_on_evac_failure_scan_stack(oop obj) {
4063   _evac_failure_scan_stack->push(obj);
4064 }
4065 
4066 void G1CollectedHeap::drain_evac_failure_scan_stack() {


4308 
4309 public:
4310   G1ParTask(G1CollectedHeap* g1h, RefToScanQueueSet *task_queues, G1RootProcessor* root_processor)
4311     : AbstractGangTask("G1 collection"),
4312       _g1h(g1h),
4313       _queues(task_queues),
4314       _root_processor(root_processor),
4315       _terminator(0, _queues),
4316       _stats_lock(Mutex::leaf, "parallel G1 stats lock", true)
4317   {}
4318 
4319   RefToScanQueueSet* queues() { return _queues; }
4320 
4321   RefToScanQueue *work_queue(int i) {
4322     return queues()->queue(i);
4323   }
4324 
4325   ParallelTaskTerminator* terminator() { return &_terminator; }
4326 
4327   virtual void set_for_termination(uint active_workers) {
4328     _root_processor->set_num_workers(active_workers);
4329     terminator()->reset_for_reuse(active_workers);
4330     _n_workers = active_workers;
4331   }
4332 
4333   // Helps out with CLD processing.
4334   //
4335   // During InitialMark we need to:
4336   // 1) Scavenge all CLDs for the young GC.
4337   // 2) Mark all objects directly reachable from strong CLDs.
4338   template <G1Mark do_mark_object>
4339   class G1CLDClosure : public CLDClosure {
4340     G1ParCopyClosure<G1BarrierNone,  do_mark_object>* _oop_closure;
4341     G1ParCopyClosure<G1BarrierKlass, do_mark_object>  _oop_in_klass_closure;
4342     G1KlassScanClosure                                _klass_in_cld_closure;
4343     bool                                              _claim;
4344 
4345    public:
4346     G1CLDClosure(G1ParCopyClosure<G1BarrierNone, do_mark_object>* oop_closure,
4347                  bool only_young, bool claim)
4348         : _oop_closure(oop_closure),


4793     _code_cache_task.barrier_wait(worker_id);
4794 
4795     // Do the second code cache cleaning work, which realize on
4796     // the liveness information gathered during the first pass.
4797     _code_cache_task.work_second_pass(worker_id);
4798 
4799     // Clean all klasses that were not unloaded.
4800     _klass_cleaning_task.work();
4801   }
4802 };
4803 
4804 
4805 void G1CollectedHeap::parallel_cleaning(BoolObjectClosure* is_alive,
4806                                         bool process_strings,
4807                                         bool process_symbols,
4808                                         bool class_unloading_occurred) {
4809   uint n_workers = workers()->active_workers();
4810 
4811   G1ParallelCleaningTask g1_unlink_task(is_alive, process_strings, process_symbols,
4812                                         n_workers, class_unloading_occurred);
4813   set_par_threads(n_workers);
4814   workers()->run_task(&g1_unlink_task);
4815   set_par_threads(0);
4816 }
4817 
4818 void G1CollectedHeap::unlink_string_and_symbol_table(BoolObjectClosure* is_alive,
4819                                                      bool process_strings, bool process_symbols) {
4820   {
4821     uint n_workers = workers()->active_workers();
4822     G1StringSymbolTableUnlinkTask g1_unlink_task(is_alive, process_strings, process_symbols);
4823     set_par_threads(n_workers);
4824     workers()->run_task(&g1_unlink_task);
4825     set_par_threads(0);
4826   }
4827 
4828   if (G1StringDedup::is_enabled()) {
4829     G1StringDedup::unlink(is_alive);
4830   }
4831 }
4832 
4833 class G1RedirtyLoggedCardsTask : public AbstractGangTask {
4834  private:
4835   DirtyCardQueueSet* _queue;
4836  public:
4837   G1RedirtyLoggedCardsTask(DirtyCardQueueSet* queue) : AbstractGangTask("Redirty Cards"), _queue(queue) { }
4838 
4839   virtual void work(uint worker_id) {
4840     G1GCPhaseTimes* phase_times = G1CollectedHeap::heap()->g1_policy()->phase_times();
4841     G1GCParPhaseTimesTracker x(phase_times, G1GCPhaseTimes::RedirtyCards, worker_id);
4842 
4843     RedirtyLoggedCardTableEntryClosure cl;
4844     _queue->par_apply_closure_to_all_completed_buffers(&cl);
4845 
4846     phase_times->record_thread_work_item(G1GCPhaseTimes::RedirtyCards, worker_id, cl.num_processed());
4847   }
4848 };
4849 
4850 void G1CollectedHeap::redirty_logged_cards() {
4851   double redirty_logged_cards_start = os::elapsedTime();
4852 
4853   uint n_workers = workers()->active_workers();
4854 
4855   G1RedirtyLoggedCardsTask redirty_task(&dirty_card_queue_set());
4856   dirty_card_queue_set().reset_for_par_iteration();
4857   set_par_threads(n_workers);
4858   workers()->run_task(&redirty_task);
4859   set_par_threads(0);
4860 
4861   DirtyCardQueueSet& dcq = JavaThread::dirty_card_queue_set();
4862   dcq.merge_bufferlists(&dirty_card_queue_set());
4863   assert(dirty_card_queue_set().completed_buffers_num() == 0, "All should be consumed");
4864 
4865   g1_policy()->phase_times()->record_redirty_logged_cards_time_ms((os::elapsedTime() - redirty_logged_cards_start) * 1000.0);
4866 }
4867 
4868 // Weak Reference Processing support
4869 
4870 // An always "is_alive" closure that is used to preserve referents.
4871 // If the object is non-null then it's alive.  Used in the preservation
4872 // of referent objects that are pointed to by reference objects
4873 // discovered by the CM ref processor.
4874 class G1AlwaysAliveClosure: public BoolObjectClosure {
4875   G1CollectedHeap* _g1;
4876 public:
4877   G1AlwaysAliveClosure(G1CollectedHeap* g1) : _g1(g1) {}
4878   bool do_object_b(oop p) {
4879     if (p != NULL) {


5075 
5076     // Call the reference processing task's work routine.
5077     _proc_task.work(worker_id, is_alive, keep_alive, drain_queue);
5078 
5079     // Note we cannot assert that the refs array is empty here as not all
5080     // of the processing tasks (specifically phase2 - pp2_work) execute
5081     // the complete_gc closure (which ordinarily would drain the queue) so
5082     // the queue may not be empty.
5083   }
5084 };
5085 
5086 // Driver routine for parallel reference processing.
5087 // Creates an instance of the ref processing gang
5088 // task and has the worker threads execute it.
5089 void G1STWRefProcTaskExecutor::execute(ProcessTask& proc_task) {
5090   assert(_workers != NULL, "Need parallel worker threads.");
5091 
5092   ParallelTaskTerminator terminator(_active_workers, _queues);
5093   G1STWRefProcTaskProxy proc_task_proxy(proc_task, _g1h, _queues, &terminator);
5094 
5095   _g1h->set_par_threads(_active_workers);
5096   _workers->run_task(&proc_task_proxy);
5097   _g1h->set_par_threads(0);
5098 }
5099 
5100 // Gang task for parallel reference enqueueing.
5101 
5102 class G1STWRefEnqueueTaskProxy: public AbstractGangTask {
5103   typedef AbstractRefProcTaskExecutor::EnqueueTask EnqueueTask;
5104   EnqueueTask& _enq_task;
5105 
5106 public:
5107   G1STWRefEnqueueTaskProxy(EnqueueTask& enq_task) :
5108     AbstractGangTask("Enqueue reference objects in parallel"),
5109     _enq_task(enq_task)
5110   { }
5111 
5112   virtual void work(uint worker_id) {
5113     _enq_task.work(worker_id);
5114   }
5115 };
5116 
5117 // Driver routine for parallel reference enqueueing.
5118 // Creates an instance of the ref enqueueing gang
5119 // task and has the worker threads execute it.
5120 
5121 void G1STWRefProcTaskExecutor::execute(EnqueueTask& enq_task) {
5122   assert(_workers != NULL, "Need parallel worker threads.");
5123 
5124   G1STWRefEnqueueTaskProxy enq_task_proxy(enq_task);
5125 
5126   _g1h->set_par_threads(_active_workers);
5127   _workers->run_task(&enq_task_proxy);
5128   _g1h->set_par_threads(0);
5129 }
5130 
5131 // End of weak reference support closures
5132 
5133 // Abstract task used to preserve (i.e. copy) any referent objects
5134 // that are in the collection set and are pointed to by reference
5135 // objects discovered by the CM ref processor.
5136 
5137 class G1ParPreserveCMReferentsTask: public AbstractGangTask {
5138 protected:
5139   G1CollectedHeap* _g1h;
5140   RefToScanQueueSet      *_queues;
5141   ParallelTaskTerminator _terminator;
5142   uint _n_workers;
5143 
5144 public:
5145   G1ParPreserveCMReferentsTask(G1CollectedHeap* g1h, uint workers, RefToScanQueueSet *task_queues) :
5146     AbstractGangTask("ParPreserveCMReferents"),
5147     _g1h(g1h),
5148     _queues(task_queues),


5230   // by following an RSet entry).
5231   //
5232   // But some of the referents, that are in the collection set, that these
5233   // reference objects point to may not have been copied: the STW ref
5234   // processor would have seen that the reference object had already
5235   // been 'discovered' and would have skipped discovering the reference,
5236   // but would not have treated the reference object as a regular oop.
5237   // As a result the copy closure would not have been applied to the
5238   // referent object.
5239   //
5240   // We need to explicitly copy these referent objects - the references
5241   // will be processed at the end of remarking.
5242   //
5243   // We also need to do this copying before we process the reference
5244   // objects discovered by the STW ref processor in case one of these
5245   // referents points to another object which is also referenced by an
5246   // object discovered by the STW ref processor.
5247 
5248   assert(no_of_gc_workers == workers()->active_workers(), "Need to reset active GC workers");
5249 
5250   set_par_threads(no_of_gc_workers);
5251   G1ParPreserveCMReferentsTask keep_cm_referents(this,
5252                                                  no_of_gc_workers,
5253                                                  _task_queues);
5254 
5255   workers()->run_task(&keep_cm_referents);
5256 
5257   set_par_threads(0);
5258 
5259   // Closure to test whether a referent is alive.
5260   G1STWIsAliveClosure is_alive(this);
5261 
5262   // Even when parallel reference processing is enabled, the processing
5263   // of JNI refs is serial and performed serially by the current thread
5264   // rather than by a worker. The following PSS will be used for processing
5265   // JNI refs.
5266 
5267   // Use only a single queue for this PSS.
5268   G1ParScanThreadState            pss(this, 0, NULL);
5269 
5270   // We do not embed a reference processor in the copying/scanning
5271   // closures while we're actually processing the discovered
5272   // reference objects.
5273   G1ParScanHeapEvacFailureClosure evac_failure_cl(this, &pss, NULL);
5274 
5275   pss.set_evac_failure_closure(&evac_failure_cl);
5276 
5277   assert(pss.queue_is_empty(), "pre-condition");
5278 


5365 }
5366 
5367 void G1CollectedHeap::evacuate_collection_set(EvacuationInfo& evacuation_info) {
5368   _expand_heap_after_alloc_failure = true;
5369   _evacuation_failed = false;
5370 
5371   // Should G1EvacuationFailureALot be in effect for this GC?
5372   NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
5373 
5374   g1_rem_set()->prepare_for_oops_into_collection_set_do();
5375 
5376   // Disable the hot card cache.
5377   G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
5378   hot_card_cache->reset_hot_cache_claimed_index();
5379   hot_card_cache->set_use_cache(false);
5380 
5381   const uint n_workers = workers()->active_workers();
5382   assert(UseDynamicNumberOfGCThreads ||
5383          n_workers == workers()->total_workers(),
5384          "If not dynamic should be using all the  workers");
5385   set_par_threads(n_workers);
5386 
5387 
5388   init_for_evac_failure(NULL);
5389 
5390   assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty");
5391   double start_par_time_sec = os::elapsedTime();
5392   double end_par_time_sec;
5393 
5394   {
5395     G1RootProcessor root_processor(this);
5396     G1ParTask g1_par_task(this, _task_queues, &root_processor);
5397     // InitialMark needs claim bits to keep track of the marked-through CLDs.
5398     if (g1_policy()->during_initial_mark_pause()) {
5399       ClassLoaderDataGraph::clear_claimed_marks();
5400     }
5401 
5402      // The individual threads will set their evac-failure closures.
5403      if (PrintTerminationStats) G1ParScanThreadState::print_termination_stats_hdr();
5404      // These tasks use ShareHeap::_process_strong_tasks
5405      assert(UseDynamicNumberOfGCThreads ||
5406             workers()->active_workers() == workers()->total_workers(),
5407             "If not dynamic should be using all the  workers");
5408     workers()->run_task(&g1_par_task);
5409     end_par_time_sec = os::elapsedTime();
5410 
5411     // Closing the inner scope will execute the destructor
5412     // for the G1RootProcessor object. We record the current
5413     // elapsed time before closing the scope so that time
5414     // taken for the destructor is NOT included in the
5415     // reported parallel time.
5416   }
5417 
5418   G1GCPhaseTimes* phase_times = g1_policy()->phase_times();
5419 
5420   double par_time_ms = (end_par_time_sec - start_par_time_sec) * 1000.0;
5421   phase_times->record_par_time(par_time_ms);
5422 
5423   double code_root_fixup_time_ms =
5424         (os::elapsedTime() - end_par_time_sec) * 1000.0;
5425   phase_times->record_code_root_fixup_time(code_root_fixup_time_ms);
5426 
5427   set_par_threads(0);
5428 
5429   // Process any discovered reference objects - we have
5430   // to do this _before_ we retire the GC alloc regions
5431   // as we may have to copy some 'reachable' referent
5432   // objects (and their reachable sub-graphs) that were
5433   // not copied during the pause.
5434   process_discovered_references(n_workers);
5435 
5436   if (G1StringDedup::is_enabled()) {
5437     double fixup_start = os::elapsedTime();
5438 
5439     G1STWIsAliveClosure is_alive(this);
5440     G1KeepAliveClosure keep_alive(this);
5441     G1StringDedup::unlink_or_oops_do(&is_alive, &keep_alive, true, phase_times);
5442 
5443     double fixup_time_ms = (os::elapsedTime() - fixup_start) * 1000.0;
5444     phase_times->record_string_dedup_fixup_time(fixup_time_ms);
5445   }
5446 
5447   _allocator->release_gc_alloc_regions(n_workers, evacuation_info);
5448   g1_rem_set()->cleanup_after_oops_into_collection_set_do();


5761   }
5762 
5763   bool failures() const { return _failures; }
5764 };
5765 
5766 bool G1CollectedHeap::check_cset_fast_test() {
5767   G1CheckCSetFastTableClosure cl;
5768   _hrm.iterate(&cl);
5769   return !cl.failures();
5770 }
5771 #endif // PRODUCT
5772 
5773 void G1CollectedHeap::cleanUpCardTable() {
5774   G1SATBCardTableModRefBS* ct_bs = g1_barrier_set();
5775   double start = os::elapsedTime();
5776 
5777   {
5778     // Iterate over the dirty cards region list.
5779     G1ParCleanupCTTask cleanup_task(ct_bs, this);
5780 
5781     set_par_threads();
5782     workers()->run_task(&cleanup_task);
5783     set_par_threads(0);
5784 #ifndef PRODUCT
5785     if (G1VerifyCTCleanup || VerifyAfterGC) {
5786       G1VerifyCardTableCleanup cleanup_verifier(this, ct_bs);
5787       heap_region_iterate(&cleanup_verifier);
5788     }
5789 #endif
5790   }
5791 
5792   double elapsed = os::elapsedTime() - start;
5793   g1_policy()->phase_times()->record_clear_ct_time(elapsed * 1000.0);
5794 }
5795 
5796 void G1CollectedHeap::free_collection_set(HeapRegion* cs_head, EvacuationInfo& evacuation_info) {
5797   size_t pre_used = 0;
5798   FreeRegionList local_free_list("Local List for CSet Freeing");
5799 
5800   double young_time_ms     = 0.0;
5801   double non_young_time_ms = 0.0;
5802 
5803   // Since the collection set is a superset of the the young list,


6294       _hr_printer.alloc(new_alloc_region, G1HRPrinter::Eden, young_list_full);
6295       check_bitmaps("Mutator Region Allocation", new_alloc_region);
6296       return new_alloc_region;
6297     }
6298   }
6299   return NULL;
6300 }
6301 
6302 void G1CollectedHeap::retire_mutator_alloc_region(HeapRegion* alloc_region,
6303                                                   size_t allocated_bytes) {
6304   assert_heap_locked_or_at_safepoint(true /* should_be_vm_thread */);
6305   assert(alloc_region->is_eden(), "all mutator alloc regions should be eden");
6306 
6307   g1_policy()->add_region_to_incremental_cset_lhs(alloc_region);
6308   _allocator->increase_used(allocated_bytes);
6309   _hr_printer.retire(alloc_region);
6310   // We update the eden sizes here, when the region is retired,
6311   // instead of when it's allocated, since this is the point that its
6312   // used space has been recored in _summary_bytes_used.
6313   g1mm()->update_eden_size();
6314 }
6315 
6316 void G1CollectedHeap::set_par_threads() {
6317   // Don't change the number of workers.  Use the value previously set
6318   // in the workgroup.
6319   uint n_workers = workers()->active_workers();
6320   assert(UseDynamicNumberOfGCThreads ||
6321            n_workers == workers()->total_workers(),
6322       "Otherwise should be using the total number of workers");
6323   if (n_workers == 0) {
6324     assert(false, "Should have been set in prior evacuation pause.");
6325     n_workers = ParallelGCThreads;
6326     workers()->set_active_workers(n_workers);
6327   }
6328   set_par_threads(n_workers);
6329 }
6330 
6331 // Methods for the GC alloc regions
6332 
6333 HeapRegion* G1CollectedHeap::new_gc_alloc_region(size_t word_size,
6334                                                  uint count,
6335                                                  InCSetState dest) {
6336   assert(FreeList_lock->owned_by_self(), "pre-condition");
6337 
6338   if (count < g1_policy()->max_regions(dest)) {
6339     const bool is_survivor = (dest.is_young());
6340     HeapRegion* new_alloc_region = new_region(word_size,
6341                                               !is_survivor,
6342                                               true /* do_expand */);
6343     if (new_alloc_region != NULL) {
6344       // We really only need to do this for old regions given that we
6345       // should never scan survivors. But it doesn't hurt to do it
6346       // for survivors too.
6347       new_alloc_region->record_timestamp();
6348       if (is_survivor) {




1312 
1313         print_hrm_post_compaction();
1314         _hr_printer.end_gc(true /* full */, (size_t) total_collections());
1315       }
1316 
1317       G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
1318       if (hot_card_cache->use_cache()) {
1319         hot_card_cache->reset_card_counts();
1320         hot_card_cache->reset_hot_cache();
1321       }
1322 
1323       // Rebuild remembered sets of all regions.
1324       uint n_workers =
1325         AdaptiveSizePolicy::calc_active_workers(workers()->total_workers(),
1326                                                 workers()->active_workers(),
1327                                                 Threads::number_of_non_daemon_threads());
1328       assert(UseDynamicNumberOfGCThreads ||
1329              n_workers == workers()->total_workers(),
1330              "If not dynamic should be using all the  workers");
1331       workers()->set_active_workers(n_workers);






1332 
1333       ParRebuildRSTask rebuild_rs_task(this);
1334       assert(UseDynamicNumberOfGCThreads ||
1335              workers()->active_workers() == workers()->total_workers(),
1336              "Unless dynamic should use total workers");




1337       workers()->run_task(&rebuild_rs_task);

1338 
1339       // Rebuild the strong code root lists for each region
1340       rebuild_strong_code_roots();
1341 
1342       if (true) { // FIXME
1343         MetaspaceGC::compute_new_size();
1344       }
1345 
1346 #ifdef TRACESPINNING
1347       ParallelTaskTerminator::print_termination_counts();
1348 #endif
1349 
1350       // Discard all rset updates
1351       JavaThread::dirty_card_queue_set().abandon_logs();
1352       assert(dirty_card_queue_set().completed_buffers_num() == 0, "DCQS should be empty");
1353 
1354       _young_list->reset_sampled_info();
1355       // At this point there should be no regions in the
1356       // entire heap tagged as young.
1357       assert(check_young_list_empty(true /* check_heap */),


2992   }
2993 };
2994 
2995 void G1CollectedHeap::verify(bool silent, VerifyOption vo) {
2996   if (SafepointSynchronize::is_at_safepoint()) {
2997     assert(Thread::current()->is_VM_thread(),
2998            "Expected to be executed serially by the VM thread at this point");
2999 
3000     if (!silent) { gclog_or_tty->print("Roots "); }
3001     VerifyRootsClosure rootsCl(vo);
3002     VerifyKlassClosure klassCl(this, &rootsCl);
3003     CLDToKlassAndOopClosure cldCl(&klassCl, &rootsCl, false);
3004 
3005     // We apply the relevant closures to all the oops in the
3006     // system dictionary, class loader data graph, the string table
3007     // and the nmethods in the code cache.
3008     G1VerifyCodeRootOopClosure codeRootsCl(this, &rootsCl, vo);
3009     G1VerifyCodeRootBlobClosure blobsCl(&codeRootsCl);
3010 
3011     {
3012       G1RootProcessor root_processor(this, 1);
3013       root_processor.process_all_roots(&rootsCl,
3014                                        &cldCl,
3015                                        &blobsCl);
3016     }
3017 
3018     bool failures = rootsCl.failures() || codeRootsCl.failures();
3019 
3020     if (vo != VerifyOption_G1UseMarkWord) {
3021       // If we're verifying during a full GC then the region sets
3022       // will have been torn down at the start of the GC. Therefore
3023       // verifying the region sets will fail. So we only verify
3024       // the region sets when not in a full GC.
3025       if (!silent) { gclog_or_tty->print("HeapRegionSets "); }
3026       verify_region_sets();
3027     }
3028 
3029     if (!silent) { gclog_or_tty->print("HeapRegions "); }
3030     if (GCParallelVerificationEnabled && ParallelGCThreads > 1) {
3031 
3032       G1ParVerifyTask task(this, vo);
3033       assert(UseDynamicNumberOfGCThreads ||
3034         workers()->active_workers() == workers()->total_workers(),
3035         "If not dynamic should be using all the workers");


3036       workers()->run_task(&task);

3037       if (task.failures()) {
3038         failures = true;
3039       }
3040 
3041     } else {
3042       VerifyRegionClosure blk(false, vo);
3043       heap_region_iterate(&blk);
3044       if (blk.failures()) {
3045         failures = true;
3046       }
3047     }
3048 
3049     if (G1StringDedup::is_enabled()) {
3050       if (!silent) gclog_or_tty->print("StrDedup ");
3051       G1StringDedup::verify();
3052     }
3053 
3054     if (failures) {
3055       gclog_or_tty->print_cr("Heap:");
3056       // It helps to have the per-region information in the output to


4009 }
4010 
4011 void G1CollectedHeap::init_for_evac_failure(OopsInHeapRegionClosure* cl) {
4012   _drain_in_progress = false;
4013   set_evac_failure_closure(cl);
4014   _evac_failure_scan_stack = new (ResourceObj::C_HEAP, mtGC) GrowableArray<oop>(40, true);
4015 }
4016 
4017 void G1CollectedHeap::finalize_for_evac_failure() {
4018   assert(_evac_failure_scan_stack != NULL &&
4019          _evac_failure_scan_stack->length() == 0,
4020          "Postcondition");
4021   assert(!_drain_in_progress, "Postcondition");
4022   delete _evac_failure_scan_stack;
4023   _evac_failure_scan_stack = NULL;
4024 }
4025 
4026 void G1CollectedHeap::remove_self_forwarding_pointers() {
4027   double remove_self_forwards_start = os::elapsedTime();
4028 

4029   G1ParRemoveSelfForwardPtrsTask rsfp_task(this);
4030   workers()->run_task(&rsfp_task);

4031 
4032   // Now restore saved marks, if any.
4033   assert(_objs_with_preserved_marks.size() ==
4034             _preserved_marks_of_objs.size(), "Both or none.");
4035   while (!_objs_with_preserved_marks.is_empty()) {
4036     oop obj = _objs_with_preserved_marks.pop();
4037     markOop m = _preserved_marks_of_objs.pop();
4038     obj->set_mark(m);
4039   }
4040   _objs_with_preserved_marks.clear(true);
4041   _preserved_marks_of_objs.clear(true);
4042 
4043   g1_policy()->phase_times()->record_evac_fail_remove_self_forwards((os::elapsedTime() - remove_self_forwards_start) * 1000.0);
4044 }
4045 
4046 void G1CollectedHeap::push_on_evac_failure_scan_stack(oop obj) {
4047   _evac_failure_scan_stack->push(obj);
4048 }
4049 
4050 void G1CollectedHeap::drain_evac_failure_scan_stack() {


4292 
4293 public:
4294   G1ParTask(G1CollectedHeap* g1h, RefToScanQueueSet *task_queues, G1RootProcessor* root_processor)
4295     : AbstractGangTask("G1 collection"),
4296       _g1h(g1h),
4297       _queues(task_queues),
4298       _root_processor(root_processor),
4299       _terminator(0, _queues),
4300       _stats_lock(Mutex::leaf, "parallel G1 stats lock", true)
4301   {}
4302 
4303   RefToScanQueueSet* queues() { return _queues; }
4304 
4305   RefToScanQueue *work_queue(int i) {
4306     return queues()->queue(i);
4307   }
4308 
4309   ParallelTaskTerminator* terminator() { return &_terminator; }
4310 
4311   virtual void set_for_termination(uint active_workers) {

4312     terminator()->reset_for_reuse(active_workers);
4313     _n_workers = active_workers;
4314   }
4315 
4316   // Helps out with CLD processing.
4317   //
4318   // During InitialMark we need to:
4319   // 1) Scavenge all CLDs for the young GC.
4320   // 2) Mark all objects directly reachable from strong CLDs.
4321   template <G1Mark do_mark_object>
4322   class G1CLDClosure : public CLDClosure {
4323     G1ParCopyClosure<G1BarrierNone,  do_mark_object>* _oop_closure;
4324     G1ParCopyClosure<G1BarrierKlass, do_mark_object>  _oop_in_klass_closure;
4325     G1KlassScanClosure                                _klass_in_cld_closure;
4326     bool                                              _claim;
4327 
4328    public:
4329     G1CLDClosure(G1ParCopyClosure<G1BarrierNone, do_mark_object>* oop_closure,
4330                  bool only_young, bool claim)
4331         : _oop_closure(oop_closure),


4776     _code_cache_task.barrier_wait(worker_id);
4777 
4778     // Do the second code cache cleaning work, which realize on
4779     // the liveness information gathered during the first pass.
4780     _code_cache_task.work_second_pass(worker_id);
4781 
4782     // Clean all klasses that were not unloaded.
4783     _klass_cleaning_task.work();
4784   }
4785 };
4786 
4787 
4788 void G1CollectedHeap::parallel_cleaning(BoolObjectClosure* is_alive,
4789                                         bool process_strings,
4790                                         bool process_symbols,
4791                                         bool class_unloading_occurred) {
4792   uint n_workers = workers()->active_workers();
4793 
4794   G1ParallelCleaningTask g1_unlink_task(is_alive, process_strings, process_symbols,
4795                                         n_workers, class_unloading_occurred);

4796   workers()->run_task(&g1_unlink_task);

4797 }
4798 
4799 void G1CollectedHeap::unlink_string_and_symbol_table(BoolObjectClosure* is_alive,
4800                                                      bool process_strings, bool process_symbols) {
4801   {

4802     G1StringSymbolTableUnlinkTask g1_unlink_task(is_alive, process_strings, process_symbols);

4803     workers()->run_task(&g1_unlink_task);

4804   }
4805 
4806   if (G1StringDedup::is_enabled()) {
4807     G1StringDedup::unlink(is_alive);
4808   }
4809 }
4810 
4811 class G1RedirtyLoggedCardsTask : public AbstractGangTask {
4812  private:
4813   DirtyCardQueueSet* _queue;
4814  public:
4815   G1RedirtyLoggedCardsTask(DirtyCardQueueSet* queue) : AbstractGangTask("Redirty Cards"), _queue(queue) { }
4816 
4817   virtual void work(uint worker_id) {
4818     G1GCPhaseTimes* phase_times = G1CollectedHeap::heap()->g1_policy()->phase_times();
4819     G1GCParPhaseTimesTracker x(phase_times, G1GCPhaseTimes::RedirtyCards, worker_id);
4820 
4821     RedirtyLoggedCardTableEntryClosure cl;
4822     _queue->par_apply_closure_to_all_completed_buffers(&cl);
4823 
4824     phase_times->record_thread_work_item(G1GCPhaseTimes::RedirtyCards, worker_id, cl.num_processed());
4825   }
4826 };
4827 
4828 void G1CollectedHeap::redirty_logged_cards() {
4829   double redirty_logged_cards_start = os::elapsedTime();
4830 


4831   G1RedirtyLoggedCardsTask redirty_task(&dirty_card_queue_set());
4832   dirty_card_queue_set().reset_for_par_iteration();

4833   workers()->run_task(&redirty_task);

4834 
4835   DirtyCardQueueSet& dcq = JavaThread::dirty_card_queue_set();
4836   dcq.merge_bufferlists(&dirty_card_queue_set());
4837   assert(dirty_card_queue_set().completed_buffers_num() == 0, "All should be consumed");
4838 
4839   g1_policy()->phase_times()->record_redirty_logged_cards_time_ms((os::elapsedTime() - redirty_logged_cards_start) * 1000.0);
4840 }
4841 
4842 // Weak Reference Processing support
4843 
4844 // An always "is_alive" closure that is used to preserve referents.
4845 // If the object is non-null then it's alive.  Used in the preservation
4846 // of referent objects that are pointed to by reference objects
4847 // discovered by the CM ref processor.
4848 class G1AlwaysAliveClosure: public BoolObjectClosure {
4849   G1CollectedHeap* _g1;
4850 public:
4851   G1AlwaysAliveClosure(G1CollectedHeap* g1) : _g1(g1) {}
4852   bool do_object_b(oop p) {
4853     if (p != NULL) {


5049 
5050     // Call the reference processing task's work routine.
5051     _proc_task.work(worker_id, is_alive, keep_alive, drain_queue);
5052 
5053     // Note we cannot assert that the refs array is empty here as not all
5054     // of the processing tasks (specifically phase2 - pp2_work) execute
5055     // the complete_gc closure (which ordinarily would drain the queue) so
5056     // the queue may not be empty.
5057   }
5058 };
5059 
5060 // Driver routine for parallel reference processing.
5061 // Creates an instance of the ref processing gang
5062 // task and has the worker threads execute it.
5063 void G1STWRefProcTaskExecutor::execute(ProcessTask& proc_task) {
5064   assert(_workers != NULL, "Need parallel worker threads.");
5065 
5066   ParallelTaskTerminator terminator(_active_workers, _queues);
5067   G1STWRefProcTaskProxy proc_task_proxy(proc_task, _g1h, _queues, &terminator);
5068 

5069   _workers->run_task(&proc_task_proxy);

5070 }
5071 
5072 // Gang task for parallel reference enqueueing.
5073 
5074 class G1STWRefEnqueueTaskProxy: public AbstractGangTask {
5075   typedef AbstractRefProcTaskExecutor::EnqueueTask EnqueueTask;
5076   EnqueueTask& _enq_task;
5077 
5078 public:
5079   G1STWRefEnqueueTaskProxy(EnqueueTask& enq_task) :
5080     AbstractGangTask("Enqueue reference objects in parallel"),
5081     _enq_task(enq_task)
5082   { }
5083 
5084   virtual void work(uint worker_id) {
5085     _enq_task.work(worker_id);
5086   }
5087 };
5088 
5089 // Driver routine for parallel reference enqueueing.
5090 // Creates an instance of the ref enqueueing gang
5091 // task and has the worker threads execute it.
5092 
5093 void G1STWRefProcTaskExecutor::execute(EnqueueTask& enq_task) {
5094   assert(_workers != NULL, "Need parallel worker threads.");
5095 
5096   G1STWRefEnqueueTaskProxy enq_task_proxy(enq_task);
5097 

5098   _workers->run_task(&enq_task_proxy);

5099 }
5100 
5101 // End of weak reference support closures
5102 
5103 // Abstract task used to preserve (i.e. copy) any referent objects
5104 // that are in the collection set and are pointed to by reference
5105 // objects discovered by the CM ref processor.
5106 
5107 class G1ParPreserveCMReferentsTask: public AbstractGangTask {
5108 protected:
5109   G1CollectedHeap* _g1h;
5110   RefToScanQueueSet      *_queues;
5111   ParallelTaskTerminator _terminator;
5112   uint _n_workers;
5113 
5114 public:
5115   G1ParPreserveCMReferentsTask(G1CollectedHeap* g1h, uint workers, RefToScanQueueSet *task_queues) :
5116     AbstractGangTask("ParPreserveCMReferents"),
5117     _g1h(g1h),
5118     _queues(task_queues),


5200   // by following an RSet entry).
5201   //
5202   // But some of the referents, that are in the collection set, that these
5203   // reference objects point to may not have been copied: the STW ref
5204   // processor would have seen that the reference object had already
5205   // been 'discovered' and would have skipped discovering the reference,
5206   // but would not have treated the reference object as a regular oop.
5207   // As a result the copy closure would not have been applied to the
5208   // referent object.
5209   //
5210   // We need to explicitly copy these referent objects - the references
5211   // will be processed at the end of remarking.
5212   //
5213   // We also need to do this copying before we process the reference
5214   // objects discovered by the STW ref processor in case one of these
5215   // referents points to another object which is also referenced by an
5216   // object discovered by the STW ref processor.
5217 
5218   assert(no_of_gc_workers == workers()->active_workers(), "Need to reset active GC workers");
5219 

5220   G1ParPreserveCMReferentsTask keep_cm_referents(this,
5221                                                  no_of_gc_workers,
5222                                                  _task_queues);
5223 
5224   workers()->run_task(&keep_cm_referents);
5225 


5226   // Closure to test whether a referent is alive.
5227   G1STWIsAliveClosure is_alive(this);
5228 
5229   // Even when parallel reference processing is enabled, the processing
5230   // of JNI refs is serial and performed serially by the current thread
5231   // rather than by a worker. The following PSS will be used for processing
5232   // JNI refs.
5233 
5234   // Use only a single queue for this PSS.
5235   G1ParScanThreadState            pss(this, 0, NULL);
5236 
5237   // We do not embed a reference processor in the copying/scanning
5238   // closures while we're actually processing the discovered
5239   // reference objects.
5240   G1ParScanHeapEvacFailureClosure evac_failure_cl(this, &pss, NULL);
5241 
5242   pss.set_evac_failure_closure(&evac_failure_cl);
5243 
5244   assert(pss.queue_is_empty(), "pre-condition");
5245 


5332 }
5333 
5334 void G1CollectedHeap::evacuate_collection_set(EvacuationInfo& evacuation_info) {
5335   _expand_heap_after_alloc_failure = true;
5336   _evacuation_failed = false;
5337 
5338   // Should G1EvacuationFailureALot be in effect for this GC?
5339   NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
5340 
5341   g1_rem_set()->prepare_for_oops_into_collection_set_do();
5342 
5343   // Disable the hot card cache.
5344   G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
5345   hot_card_cache->reset_hot_cache_claimed_index();
5346   hot_card_cache->set_use_cache(false);
5347 
5348   const uint n_workers = workers()->active_workers();
5349   assert(UseDynamicNumberOfGCThreads ||
5350          n_workers == workers()->total_workers(),
5351          "If not dynamic should be using all the  workers");


5352 
5353   init_for_evac_failure(NULL);
5354 
5355   assert(dirty_card_queue_set().completed_buffers_num() == 0, "Should be empty");
5356   double start_par_time_sec = os::elapsedTime();
5357   double end_par_time_sec;
5358 
5359   {
5360     G1RootProcessor root_processor(this, n_workers);
5361     G1ParTask g1_par_task(this, _task_queues, &root_processor);
5362     // InitialMark needs claim bits to keep track of the marked-through CLDs.
5363     if (g1_policy()->during_initial_mark_pause()) {
5364       ClassLoaderDataGraph::clear_claimed_marks();
5365     }
5366 
5367      // The individual threads will set their evac-failure closures.
5368      if (PrintTerminationStats) G1ParScanThreadState::print_termination_stats_hdr();
5369      // These tasks use ShareHeap::_process_strong_tasks
5370      assert(UseDynamicNumberOfGCThreads ||
5371             workers()->active_workers() == workers()->total_workers(),
5372             "If not dynamic should be using all the  workers");
5373     workers()->run_task(&g1_par_task);
5374     end_par_time_sec = os::elapsedTime();
5375 
5376     // Closing the inner scope will execute the destructor
5377     // for the G1RootProcessor object. We record the current
5378     // elapsed time before closing the scope so that time
5379     // taken for the destructor is NOT included in the
5380     // reported parallel time.
5381   }
5382 
5383   G1GCPhaseTimes* phase_times = g1_policy()->phase_times();
5384 
5385   double par_time_ms = (end_par_time_sec - start_par_time_sec) * 1000.0;
5386   phase_times->record_par_time(par_time_ms);
5387 
5388   double code_root_fixup_time_ms =
5389         (os::elapsedTime() - end_par_time_sec) * 1000.0;
5390   phase_times->record_code_root_fixup_time(code_root_fixup_time_ms);
5391 


5392   // Process any discovered reference objects - we have
5393   // to do this _before_ we retire the GC alloc regions
5394   // as we may have to copy some 'reachable' referent
5395   // objects (and their reachable sub-graphs) that were
5396   // not copied during the pause.
5397   process_discovered_references(n_workers);
5398 
5399   if (G1StringDedup::is_enabled()) {
5400     double fixup_start = os::elapsedTime();
5401 
5402     G1STWIsAliveClosure is_alive(this);
5403     G1KeepAliveClosure keep_alive(this);
5404     G1StringDedup::unlink_or_oops_do(&is_alive, &keep_alive, true, phase_times);
5405 
5406     double fixup_time_ms = (os::elapsedTime() - fixup_start) * 1000.0;
5407     phase_times->record_string_dedup_fixup_time(fixup_time_ms);
5408   }
5409 
5410   _allocator->release_gc_alloc_regions(n_workers, evacuation_info);
5411   g1_rem_set()->cleanup_after_oops_into_collection_set_do();


5724   }
5725 
5726   bool failures() const { return _failures; }
5727 };
5728 
5729 bool G1CollectedHeap::check_cset_fast_test() {
5730   G1CheckCSetFastTableClosure cl;
5731   _hrm.iterate(&cl);
5732   return !cl.failures();
5733 }
5734 #endif // PRODUCT
5735 
5736 void G1CollectedHeap::cleanUpCardTable() {
5737   G1SATBCardTableModRefBS* ct_bs = g1_barrier_set();
5738   double start = os::elapsedTime();
5739 
5740   {
5741     // Iterate over the dirty cards region list.
5742     G1ParCleanupCTTask cleanup_task(ct_bs, this);
5743 

5744     workers()->run_task(&cleanup_task);

5745 #ifndef PRODUCT
5746     if (G1VerifyCTCleanup || VerifyAfterGC) {
5747       G1VerifyCardTableCleanup cleanup_verifier(this, ct_bs);
5748       heap_region_iterate(&cleanup_verifier);
5749     }
5750 #endif
5751   }
5752 
5753   double elapsed = os::elapsedTime() - start;
5754   g1_policy()->phase_times()->record_clear_ct_time(elapsed * 1000.0);
5755 }
5756 
5757 void G1CollectedHeap::free_collection_set(HeapRegion* cs_head, EvacuationInfo& evacuation_info) {
5758   size_t pre_used = 0;
5759   FreeRegionList local_free_list("Local List for CSet Freeing");
5760 
5761   double young_time_ms     = 0.0;
5762   double non_young_time_ms = 0.0;
5763 
5764   // Since the collection set is a superset of the the young list,


6255       _hr_printer.alloc(new_alloc_region, G1HRPrinter::Eden, young_list_full);
6256       check_bitmaps("Mutator Region Allocation", new_alloc_region);
6257       return new_alloc_region;
6258     }
6259   }
6260   return NULL;
6261 }
6262 
6263 void G1CollectedHeap::retire_mutator_alloc_region(HeapRegion* alloc_region,
6264                                                   size_t allocated_bytes) {
6265   assert_heap_locked_or_at_safepoint(true /* should_be_vm_thread */);
6266   assert(alloc_region->is_eden(), "all mutator alloc regions should be eden");
6267 
6268   g1_policy()->add_region_to_incremental_cset_lhs(alloc_region);
6269   _allocator->increase_used(allocated_bytes);
6270   _hr_printer.retire(alloc_region);
6271   // We update the eden sizes here, when the region is retired,
6272   // instead of when it's allocated, since this is the point that its
6273   // used space has been recored in _summary_bytes_used.
6274   g1mm()->update_eden_size();















6275 }
6276 
6277 // Methods for the GC alloc regions
6278 
6279 HeapRegion* G1CollectedHeap::new_gc_alloc_region(size_t word_size,
6280                                                  uint count,
6281                                                  InCSetState dest) {
6282   assert(FreeList_lock->owned_by_self(), "pre-condition");
6283 
6284   if (count < g1_policy()->max_regions(dest)) {
6285     const bool is_survivor = (dest.is_young());
6286     HeapRegion* new_alloc_region = new_region(word_size,
6287                                               !is_survivor,
6288                                               true /* do_expand */);
6289     if (new_alloc_region != NULL) {
6290       // We really only need to do this for old regions given that we
6291       // should never scan survivors. But it doesn't hurt to do it
6292       // for survivors too.
6293       new_alloc_region->record_timestamp();
6294       if (is_survivor) {


< prev index next >