< 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 */),


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() {


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


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


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


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


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


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


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() {


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 >