< prev index next >

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

Print this page
rev 60060 : [mq]: 8210462-lkorinth-review


1789   SuspendibleThreadSet::desynchronize();
1790 }
1791 
1792 void G1CollectedHeap::post_initialize() {
1793   CollectedHeap::post_initialize();
1794   ref_processing_init();
1795 }
1796 
1797 void G1CollectedHeap::ref_processing_init() {
1798   // Reference processing in G1 currently works as follows:
1799   //
1800   // * There are two reference processor instances. One is
1801   //   used to record and process discovered references
1802   //   during concurrent marking; the other is used to
1803   //   record and process references during STW pauses
1804   //   (both full and incremental).
1805   // * Both ref processors need to 'span' the entire heap as
1806   //   the regions in the collection set may be dotted around.
1807   //
1808   // * For the concurrent marking ref processor:
1809   //   * Reference discovery is enabled at initial marking.
1810   //   * Reference discovery is disabled and the discovered
1811   //     references processed etc during remarking.
1812   //   * Reference discovery is MT (see below).
1813   //   * Reference discovery requires a barrier (see below).
1814   //   * Reference processing may or may not be MT
1815   //     (depending on the value of ParallelRefProcEnabled
1816   //     and ParallelGCThreads).
1817   //   * A full GC disables reference discovery by the CM
1818   //     ref processor and abandons any entries on it's
1819   //     discovered lists.
1820   //
1821   // * For the STW processor:
1822   //   * Non MT discovery is enabled at the start of a full GC.
1823   //   * Processing and enqueueing during a full GC is non-MT.
1824   //   * During a full GC, references are processed after marking.
1825   //
1826   //   * Discovery (may or may not be MT) is enabled at the start
1827   //     of an incremental evacuation pause.
1828   //   * References are processed near the end of a STW evacuation pause.
1829   //   * For both types of GC:


2030       ResourceMark rm; /* For thread name. */                           \
2031       LogStream LOG_COLLECT_CONCURRENTLY_s(&LOG_COLLECT_CONCURRENTLY_lt); \
2032       LOG_COLLECT_CONCURRENTLY_s.print("%s: Try Collect Concurrently (%s): ", \
2033                                        Thread::current()->name(),       \
2034                                        GCCause::to_string(cause));      \
2035       LOG_COLLECT_CONCURRENTLY_s.print(__VA_ARGS__);                    \
2036     }                                                                   \
2037   } while (0)
2038 
2039 #define LOG_COLLECT_CONCURRENTLY_COMPLETE(cause, result) \
2040   LOG_COLLECT_CONCURRENTLY(cause, "complete %s", BOOL_TO_STR(result))
2041 
2042 bool G1CollectedHeap::try_collect_concurrently(GCCause::Cause cause,
2043                                                uint gc_counter,
2044                                                uint old_marking_started_before) {
2045   assert_heap_not_locked();
2046   assert(should_do_concurrent_full_gc(cause),
2047          "Non-concurrent cause %s", GCCause::to_string(cause));
2048 
2049   for (uint i = 1; true; ++i) {
2050     // Try to schedule an initial-mark evacuation pause that will
2051     // start a concurrent cycle.
2052     LOG_COLLECT_CONCURRENTLY(cause, "attempt %u", i);
2053     VM_G1TryInitiateConcMark op(gc_counter,
2054                                 cause,
2055                                 policy()->max_pause_time_ms());
2056     VMThread::execute(&op);
2057 
2058     // Request is trivially finished.
2059     if (cause == GCCause::_g1_periodic_collection) {
2060       LOG_COLLECT_CONCURRENTLY_COMPLETE(cause, op.gc_succeeded());
2061       return op.gc_succeeded();
2062     }
2063 
2064     // If VMOp skipped initiating concurrent marking cycle because
2065     // we're terminating, then we're done.
2066     if (op.terminating()) {
2067       LOG_COLLECT_CONCURRENTLY(cause, "skipped: terminating");
2068       return false;
2069     }
2070 


2099       // to start a concurrent cycle.
2100       if (old_marking_started_before != old_marking_started_after) {
2101         LOG_COLLECT_CONCURRENTLY(cause, "ignoring STW full GC");
2102         old_marking_started_before = old_marking_started_after;
2103       }
2104     } else if (!GCCause::is_user_requested_gc(cause)) {
2105       // For an "automatic" (not user-requested) collection, we just need to
2106       // ensure that progress is made.
2107       //
2108       // Request is finished if any of
2109       // (1) the VMOp successfully performed a GC,
2110       // (2) a concurrent cycle was already in progress,
2111       // (3) whitebox is controlling concurrent cycles,
2112       // (4) a new cycle was started (by this thread or some other), or
2113       // (5) a Full GC was performed.
2114       // Cases (4) and (5) are detected together by a change to
2115       // _old_marking_cycles_started.
2116       //
2117       // Note that (1) does not imply (4).  If we're still in the mixed
2118       // phase of an earlier concurrent collection, the request to make the
2119       // collection an initial-mark won't be honored.  If we don't check for
2120       // both conditions we'll spin doing back-to-back collections.
2121       if (op.gc_succeeded() ||
2122           op.cycle_already_in_progress() ||
2123           op.whitebox_attached() ||
2124           (old_marking_started_before != old_marking_started_after)) {
2125         LOG_COLLECT_CONCURRENTLY_COMPLETE(cause, true);
2126         return true;
2127       }
2128     } else {                    // User-requested GC.
2129       // For a user-requested collection, we want to ensure that a complete
2130       // full collection has been performed before returning, but without
2131       // waiting for more than needed.
2132 
2133       // For user-requested GCs (unlike non-UR), a successful VMOp implies a
2134       // new cycle was started.  That's good, because it's not clear what we
2135       // should do otherwise.  Trying again just does back to back GCs.
2136       // Can't wait for someone else to start a cycle.  And returning fails
2137       // to meet the goal of ensuring a full collection was performed.
2138       assert(!op.gc_succeeded() ||
2139              (old_marking_started_before != old_marking_started_after),


2603                        stats->regions_filled(), stats->direct_allocated(),
2604                        stats->failure_used(), stats->failure_waste());
2605 }
2606 
2607 void G1CollectedHeap::trace_heap(GCWhen::Type when, const GCTracer* gc_tracer) {
2608   const G1HeapSummary& heap_summary = create_g1_heap_summary();
2609   gc_tracer->report_gc_heap_summary(when, heap_summary);
2610 
2611   const MetaspaceSummary& metaspace_summary = create_metaspace_summary();
2612   gc_tracer->report_metaspace_summary(when, metaspace_summary);
2613 }
2614 
2615 void G1CollectedHeap::gc_prologue(bool full) {
2616   assert(InlineCacheBuffer::is_empty(), "should have cleaned up ICBuffer");
2617 
2618   // This summary needs to be printed before incrementing total collections.
2619   rem_set()->print_periodic_summary_info("Before GC RS summary", total_collections());
2620 
2621   // Update common counters.
2622   increment_total_collections(full /* full gc */);
2623   if (full || collector_state()->in_initial_mark_gc()) {
2624     increment_old_marking_cycles_started();
2625   }
2626 
2627   // Fill TLAB's and such
2628   {
2629     Ticks start = Ticks::now();
2630     ensure_parsability(true);
2631     Tickspan dt = Ticks::now() - start;
2632     phase_times()->record_prepare_tlab_time_ms(dt.seconds() * MILLIUNITS);
2633   }
2634 
2635   if (!full) {
2636     // Flush dirty card queues to qset, so later phases don't need to account
2637     // for partially filled per-thread queues and such.  Not needed for full
2638     // collections, which ignore those logs.
2639     Ticks start = Ticks::now();
2640     G1BarrierSet::dirty_card_queue_set().concatenate_logs();
2641     Tickspan dt = Ticks::now() - start;
2642     phase_times()->record_concatenate_dirty_card_logs_time_ms(dt.seconds() * MILLIUNITS);
2643   }


2834 
2835   phase_times()->record_start_new_cset_time_ms((os::elapsedTime() - start) * 1000.0);
2836 }
2837 
2838 void G1CollectedHeap::calculate_collection_set(G1EvacuationInfo& evacuation_info, double target_pause_time_ms) {
2839 
2840   _collection_set.finalize_initial_collection_set(target_pause_time_ms, &_survivor);
2841   evacuation_info.set_collectionset_regions(collection_set()->region_length() +
2842                                             collection_set()->optional_region_length());
2843 
2844   _cm->verify_no_collection_set_oops();
2845 
2846   if (_hr_printer.is_active()) {
2847     G1PrintCollectionSetClosure cl(&_hr_printer);
2848     _collection_set.iterate(&cl);
2849     _collection_set.iterate_optional(&cl);
2850   }
2851 }
2852 
2853 G1HeapVerifier::G1VerifyType G1CollectedHeap::young_collection_verify_type() const {
2854   if (collector_state()->in_initial_mark_gc()) {
2855     return G1HeapVerifier::G1VerifyConcurrentStart;
2856   } else if (collector_state()->in_young_only_phase()) {
2857     return G1HeapVerifier::G1VerifyYoungNormal;
2858   } else {
2859     return G1HeapVerifier::G1VerifyMixed;
2860   }
2861 }
2862 
2863 void G1CollectedHeap::verify_before_young_collection(G1HeapVerifier::G1VerifyType type) {
2864   if (VerifyRememberedSets) {
2865     log_info(gc, verify)("[Verifying RemSets before GC]");
2866     VerifyRegionRemSetClosure v_cl;
2867     heap_region_iterate(&v_cl);
2868   }
2869   _verifier->verify_before_gc(type);
2870   _verifier->check_bitmaps("GC Start");
2871   verify_numa_regions("GC Start");
2872 }
2873 
2874 void G1CollectedHeap::verify_after_young_collection(G1HeapVerifier::G1VerifyType type) {


2879   }
2880   _verifier->verify_after_gc(type);
2881   _verifier->check_bitmaps("GC End");
2882   verify_numa_regions("GC End");
2883 }
2884 
2885 void G1CollectedHeap::expand_heap_after_young_collection(){
2886   size_t expand_bytes = _heap_sizing_policy->young_collection_expansion_amount();
2887   if (expand_bytes > 0) {
2888     // No need for an ergo logging here,
2889     // expansion_amount() does this when it returns a value > 0.
2890     double expand_ms;
2891     if (!expand(expand_bytes, _workers, &expand_ms)) {
2892       // We failed to expand the heap. Cannot do anything about it.
2893     }
2894     phase_times()->record_expand_heap_time(expand_ms);
2895   }
2896 }
2897 
2898 const char* G1CollectedHeap::young_gc_name() const {
2899   if (collector_state()->in_initial_mark_gc()) {
2900     return "Pause Young (Concurrent Start)";
2901   } else if (collector_state()->in_young_only_phase()) {
2902     if (collector_state()->in_young_gc_before_mixed()) {
2903       return "Pause Young (Prepare Mixed)";
2904     } else {
2905       return "Pause Young (Normal)";
2906     }
2907   } else {
2908     return "Pause Young (Mixed)";
2909   }
2910 }
2911 
2912 bool G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
2913   assert_at_safepoint_on_vm_thread();
2914   guarantee(!is_gc_active(), "collection is not reentrant");
2915 
2916   if (GCLocker::check_active_before_gc()) {
2917     return false;
2918   }
2919 


2932 void G1CollectedHeap::do_collection_pause_at_safepoint_helper(double target_pause_time_ms) {
2933   GCIdMark gc_id_mark;
2934 
2935   SvcGCMarker sgcm(SvcGCMarker::MINOR);
2936   ResourceMark rm;
2937 
2938   policy()->note_gc_start();
2939 
2940   _gc_timer_stw->register_gc_start();
2941   _gc_tracer_stw->report_gc_start(gc_cause(), _gc_timer_stw->gc_start());
2942 
2943   wait_for_root_region_scanning();
2944 
2945   print_heap_before_gc();
2946   print_heap_regions();
2947   trace_heap_before_gc(_gc_tracer_stw);
2948 
2949   _verifier->verify_region_sets_optional();
2950   _verifier->verify_dirty_young_regions();
2951 
2952   // We should not be doing initial mark unless the conc mark thread is running
2953   if (!_cm_thread->should_terminate()) {
2954     // This call will decide whether this pause is an initial-mark
2955     // pause. If it is, in_initial_mark_gc() will return true
2956     // for the duration of this pause.
2957     policy()->decide_on_conc_mark_initiation();
2958   }
2959 
2960   // We do not allow initial-mark to be piggy-backed on a mixed GC.
2961   assert(!collector_state()->in_initial_mark_gc() ||
2962          collector_state()->in_young_only_phase(), "sanity");
2963   // We also do not allow mixed GCs during marking.
2964   assert(!collector_state()->mark_or_rebuild_in_progress() || collector_state()->in_young_only_phase(), "sanity");
2965 
2966   // Record whether this pause is an initial mark. When the current
2967   // thread has completed its logging output and it's safe to signal
2968   // the CM thread, the flag's value in the policy has been reset.
2969   bool should_start_conc_mark = collector_state()->in_initial_mark_gc();
2970   if (should_start_conc_mark) {
2971     _cm->gc_tracer_cm()->set_gc_cause(gc_cause());
2972   }
2973 
2974   // Inner scope for scope based logging, timers, and stats collection
2975   {
2976     G1EvacuationInfo evacuation_info;
2977 
2978     _gc_tracer_stw->report_yc_type(collector_state()->yc_type());
2979 
2980     GCTraceCPUTime tcpu;
2981 
2982     GCTraceTime(Info, gc) tm(young_gc_name(), NULL, gc_cause(), true);
2983 
2984     uint active_workers = WorkerPolicy::calc_active_workers(workers()->total_workers(),
2985                                                             workers()->active_workers(),
2986                                                             Threads::number_of_non_daemon_threads());
2987     active_workers = workers()->update_active_workers(active_workers);
2988     log_info(gc,task)("Using %u workers of %u for evacuation", active_workers, workers()->total_workers());
2989 


3033                                                   collection_set()->optional_region_length());
3034         pre_evacuate_collection_set(evacuation_info, &per_thread_states);
3035 
3036         // Actually do the work...
3037         evacuate_initial_collection_set(&per_thread_states);
3038 
3039         if (_collection_set.optional_region_length() != 0) {
3040           evacuate_optional_collection_set(&per_thread_states);
3041         }
3042         post_evacuate_collection_set(evacuation_info, &rdcqs, &per_thread_states);
3043 
3044         start_new_collection_set();
3045 
3046         _survivor_evac_stats.adjust_desired_plab_sz();
3047         _old_evac_stats.adjust_desired_plab_sz();
3048 
3049         if (should_start_conc_mark) {
3050           // We have to do this before we notify the CM threads that
3051           // they can start working to make sure that all the
3052           // appropriate initialization is done on the CM object.
3053           concurrent_mark()->post_initial_mark();
3054           // Note that we don't actually trigger the CM thread at
3055           // this point. We do that later when we're sure that
3056           // the current thread has completed its logging output.
3057         }
3058 
3059         allocate_dummy_regions();
3060 
3061         _allocator->init_mutator_alloc_regions();
3062 
3063         expand_heap_after_young_collection();
3064 
3065         double sample_end_time_sec = os::elapsedTime();
3066         double pause_time_ms = (sample_end_time_sec - sample_start_time_sec) * MILLIUNITS;
3067         policy()->record_collection_pause_end(pause_time_ms);
3068       }
3069 
3070       verify_after_young_collection(verify_type);
3071 
3072       gc_epilogue(false);
3073     }


3514                                               &drain_queue,
3515                                               &par_task_executor,
3516                                               pt);
3517   }
3518 
3519   _gc_tracer_stw->report_gc_reference_stats(stats);
3520 
3521   // We have completed copying any necessary live referent objects.
3522   assert(pss->queue_is_empty(), "both queue and overflow should be empty");
3523 
3524   make_pending_list_reachable();
3525 
3526   assert(!rp->discovery_enabled(), "Postcondition");
3527   rp->verify_no_references_recorded();
3528 
3529   double ref_proc_time = os::elapsedTime() - ref_proc_start;
3530   phase_times()->record_ref_proc_time(ref_proc_time * 1000.0);
3531 }
3532 
3533 void G1CollectedHeap::make_pending_list_reachable() {
3534   if (collector_state()->in_initial_mark_gc()) {
3535     oop pll_head = Universe::reference_pending_list();
3536     if (pll_head != NULL) {
3537       // Any valid worker id is fine here as we are in the VM thread and single-threaded.
3538       _cm->mark_in_next_bitmap(0 /* worker_id */, pll_head);
3539     }
3540   }
3541 }
3542 
3543 void G1CollectedHeap::merge_per_thread_state_info(G1ParScanThreadStateSet* per_thread_states) {
3544   Ticks start = Ticks::now();
3545   per_thread_states->flush();
3546   phase_times()->record_or_add_time_secs(G1GCPhaseTimes::MergePSS, 0 /* worker_id */, (Ticks::now() - start).seconds());
3547 }
3548 
3549 class G1PrepareEvacuationTask : public AbstractGangTask {
3550   class G1PrepareRegionsClosure : public HeapRegionClosure {
3551     G1CollectedHeap* _g1h;
3552     G1PrepareEvacuationTask* _parent_task;
3553     size_t _worker_humongous_total;
3554     size_t _worker_humongous_candidates;


3703     rem_set()->prepare_for_scan_heap_roots();
3704     phase_times()->record_prepare_heap_roots_time_ms((Ticks::now() - start).seconds() * 1000.0);
3705   }
3706 
3707   {
3708     G1PrepareEvacuationTask g1_prep_task(this);
3709     Tickspan task_time = run_task(&g1_prep_task);
3710 
3711     phase_times()->record_register_regions(task_time.seconds() * 1000.0,
3712                                            g1_prep_task.humongous_total(),
3713                                            g1_prep_task.humongous_candidates());
3714   }
3715 
3716   assert(_verifier->check_region_attr_table(), "Inconsistency in the region attributes table.");
3717   _preserved_marks_set.assert_empty();
3718 
3719 #if COMPILER2_OR_JVMCI
3720   DerivedPointerTable::clear();
3721 #endif
3722 
3723   // InitialMark needs claim bits to keep track of the marked-through CLDs.
3724   if (collector_state()->in_initial_mark_gc()) {
3725     concurrent_mark()->pre_initial_mark();
3726 
3727     double start_clear_claimed_marks = os::elapsedTime();
3728 
3729     ClassLoaderDataGraph::clear_claimed_marks();
3730 
3731     double recorded_clear_claimed_marks_time_ms = (os::elapsedTime() - start_clear_claimed_marks) * 1000.0;
3732     phase_times()->record_clear_claimed_marks_time_ms(recorded_clear_claimed_marks_time_ms);
3733   }
3734 
3735   // Should G1EvacuationFailureALot be in effect for this GC?
3736   NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
3737 }
3738 
3739 class G1EvacuateRegionsBaseTask : public AbstractGangTask {
3740 protected:
3741   G1CollectedHeap* _g1h;
3742   G1ParScanThreadStateSet* _per_thread_states;
3743   G1ScannerTasksQueueSet* _task_queues;
3744   TaskTerminator _terminator;
3745   uint _num_workers;


4773     }
4774     _policy->remset_tracker()->update_at_allocate(new_alloc_region);
4775     register_region_with_region_attr(new_alloc_region);
4776     _hr_printer.alloc(new_alloc_region);
4777     return new_alloc_region;
4778   }
4779   return NULL;
4780 }
4781 
4782 void G1CollectedHeap::retire_gc_alloc_region(HeapRegion* alloc_region,
4783                                              size_t allocated_bytes,
4784                                              G1HeapRegionAttr dest) {
4785   _bytes_used_during_gc += allocated_bytes;
4786   if (dest.is_old()) {
4787     old_set_add(alloc_region);
4788   } else {
4789     assert(dest.is_young(), "Retiring alloc region should be young (%d)", dest.type());
4790     _survivor.add_used_bytes(allocated_bytes);
4791   }
4792 
4793   bool const during_im = collector_state()->in_initial_mark_gc();
4794   if (during_im && allocated_bytes > 0) {
4795     _cm->root_regions()->add(alloc_region->next_top_at_mark_start(), alloc_region->top());
4796   }
4797   _hr_printer.retire(alloc_region);
4798 }
4799 
4800 HeapRegion* G1CollectedHeap::alloc_highest_free_region() {
4801   bool expanded = false;
4802   uint index = _hrm->find_highest_free(&expanded);
4803 
4804   if (index != G1_NO_HRM_INDEX) {
4805     if (expanded) {
4806       log_debug(gc, ergo, heap)("Attempt heap expansion (requested address range outside heap bounds). region size: " SIZE_FORMAT "B",
4807                                 HeapRegion::GrainWords * HeapWordSize);
4808     }
4809     return _hrm->allocate_free_regions_starting_at(index, 1);
4810   }
4811   return NULL;
4812 }
4813 




1789   SuspendibleThreadSet::desynchronize();
1790 }
1791 
1792 void G1CollectedHeap::post_initialize() {
1793   CollectedHeap::post_initialize();
1794   ref_processing_init();
1795 }
1796 
1797 void G1CollectedHeap::ref_processing_init() {
1798   // Reference processing in G1 currently works as follows:
1799   //
1800   // * There are two reference processor instances. One is
1801   //   used to record and process discovered references
1802   //   during concurrent marking; the other is used to
1803   //   record and process references during STW pauses
1804   //   (both full and incremental).
1805   // * Both ref processors need to 'span' the entire heap as
1806   //   the regions in the collection set may be dotted around.
1807   //
1808   // * For the concurrent marking ref processor:
1809   //   * Reference discovery is enabled at concurrent start.
1810   //   * Reference discovery is disabled and the discovered
1811   //     references processed etc during remarking.
1812   //   * Reference discovery is MT (see below).
1813   //   * Reference discovery requires a barrier (see below).
1814   //   * Reference processing may or may not be MT
1815   //     (depending on the value of ParallelRefProcEnabled
1816   //     and ParallelGCThreads).
1817   //   * A full GC disables reference discovery by the CM
1818   //     ref processor and abandons any entries on it's
1819   //     discovered lists.
1820   //
1821   // * For the STW processor:
1822   //   * Non MT discovery is enabled at the start of a full GC.
1823   //   * Processing and enqueueing during a full GC is non-MT.
1824   //   * During a full GC, references are processed after marking.
1825   //
1826   //   * Discovery (may or may not be MT) is enabled at the start
1827   //     of an incremental evacuation pause.
1828   //   * References are processed near the end of a STW evacuation pause.
1829   //   * For both types of GC:


2030       ResourceMark rm; /* For thread name. */                           \
2031       LogStream LOG_COLLECT_CONCURRENTLY_s(&LOG_COLLECT_CONCURRENTLY_lt); \
2032       LOG_COLLECT_CONCURRENTLY_s.print("%s: Try Collect Concurrently (%s): ", \
2033                                        Thread::current()->name(),       \
2034                                        GCCause::to_string(cause));      \
2035       LOG_COLLECT_CONCURRENTLY_s.print(__VA_ARGS__);                    \
2036     }                                                                   \
2037   } while (0)
2038 
2039 #define LOG_COLLECT_CONCURRENTLY_COMPLETE(cause, result) \
2040   LOG_COLLECT_CONCURRENTLY(cause, "complete %s", BOOL_TO_STR(result))
2041 
2042 bool G1CollectedHeap::try_collect_concurrently(GCCause::Cause cause,
2043                                                uint gc_counter,
2044                                                uint old_marking_started_before) {
2045   assert_heap_not_locked();
2046   assert(should_do_concurrent_full_gc(cause),
2047          "Non-concurrent cause %s", GCCause::to_string(cause));
2048 
2049   for (uint i = 1; true; ++i) {
2050     // Try to schedule concurrent start evacuation pause that will
2051     // start a concurrent cycle.
2052     LOG_COLLECT_CONCURRENTLY(cause, "attempt %u", i);
2053     VM_G1TryInitiateConcMark op(gc_counter,
2054                                 cause,
2055                                 policy()->max_pause_time_ms());
2056     VMThread::execute(&op);
2057 
2058     // Request is trivially finished.
2059     if (cause == GCCause::_g1_periodic_collection) {
2060       LOG_COLLECT_CONCURRENTLY_COMPLETE(cause, op.gc_succeeded());
2061       return op.gc_succeeded();
2062     }
2063 
2064     // If VMOp skipped initiating concurrent marking cycle because
2065     // we're terminating, then we're done.
2066     if (op.terminating()) {
2067       LOG_COLLECT_CONCURRENTLY(cause, "skipped: terminating");
2068       return false;
2069     }
2070 


2099       // to start a concurrent cycle.
2100       if (old_marking_started_before != old_marking_started_after) {
2101         LOG_COLLECT_CONCURRENTLY(cause, "ignoring STW full GC");
2102         old_marking_started_before = old_marking_started_after;
2103       }
2104     } else if (!GCCause::is_user_requested_gc(cause)) {
2105       // For an "automatic" (not user-requested) collection, we just need to
2106       // ensure that progress is made.
2107       //
2108       // Request is finished if any of
2109       // (1) the VMOp successfully performed a GC,
2110       // (2) a concurrent cycle was already in progress,
2111       // (3) whitebox is controlling concurrent cycles,
2112       // (4) a new cycle was started (by this thread or some other), or
2113       // (5) a Full GC was performed.
2114       // Cases (4) and (5) are detected together by a change to
2115       // _old_marking_cycles_started.
2116       //
2117       // Note that (1) does not imply (4).  If we're still in the mixed
2118       // phase of an earlier concurrent collection, the request to make the
2119       // collection a concurrent start won't be honored.  If we don't check for
2120       // both conditions we'll spin doing back-to-back collections.
2121       if (op.gc_succeeded() ||
2122           op.cycle_already_in_progress() ||
2123           op.whitebox_attached() ||
2124           (old_marking_started_before != old_marking_started_after)) {
2125         LOG_COLLECT_CONCURRENTLY_COMPLETE(cause, true);
2126         return true;
2127       }
2128     } else {                    // User-requested GC.
2129       // For a user-requested collection, we want to ensure that a complete
2130       // full collection has been performed before returning, but without
2131       // waiting for more than needed.
2132 
2133       // For user-requested GCs (unlike non-UR), a successful VMOp implies a
2134       // new cycle was started.  That's good, because it's not clear what we
2135       // should do otherwise.  Trying again just does back to back GCs.
2136       // Can't wait for someone else to start a cycle.  And returning fails
2137       // to meet the goal of ensuring a full collection was performed.
2138       assert(!op.gc_succeeded() ||
2139              (old_marking_started_before != old_marking_started_after),


2603                        stats->regions_filled(), stats->direct_allocated(),
2604                        stats->failure_used(), stats->failure_waste());
2605 }
2606 
2607 void G1CollectedHeap::trace_heap(GCWhen::Type when, const GCTracer* gc_tracer) {
2608   const G1HeapSummary& heap_summary = create_g1_heap_summary();
2609   gc_tracer->report_gc_heap_summary(when, heap_summary);
2610 
2611   const MetaspaceSummary& metaspace_summary = create_metaspace_summary();
2612   gc_tracer->report_metaspace_summary(when, metaspace_summary);
2613 }
2614 
2615 void G1CollectedHeap::gc_prologue(bool full) {
2616   assert(InlineCacheBuffer::is_empty(), "should have cleaned up ICBuffer");
2617 
2618   // This summary needs to be printed before incrementing total collections.
2619   rem_set()->print_periodic_summary_info("Before GC RS summary", total_collections());
2620 
2621   // Update common counters.
2622   increment_total_collections(full /* full gc */);
2623   if (full || collector_state()->in_concurrent_start_gc()) {
2624     increment_old_marking_cycles_started();
2625   }
2626 
2627   // Fill TLAB's and such
2628   {
2629     Ticks start = Ticks::now();
2630     ensure_parsability(true);
2631     Tickspan dt = Ticks::now() - start;
2632     phase_times()->record_prepare_tlab_time_ms(dt.seconds() * MILLIUNITS);
2633   }
2634 
2635   if (!full) {
2636     // Flush dirty card queues to qset, so later phases don't need to account
2637     // for partially filled per-thread queues and such.  Not needed for full
2638     // collections, which ignore those logs.
2639     Ticks start = Ticks::now();
2640     G1BarrierSet::dirty_card_queue_set().concatenate_logs();
2641     Tickspan dt = Ticks::now() - start;
2642     phase_times()->record_concatenate_dirty_card_logs_time_ms(dt.seconds() * MILLIUNITS);
2643   }


2834 
2835   phase_times()->record_start_new_cset_time_ms((os::elapsedTime() - start) * 1000.0);
2836 }
2837 
2838 void G1CollectedHeap::calculate_collection_set(G1EvacuationInfo& evacuation_info, double target_pause_time_ms) {
2839 
2840   _collection_set.finalize_initial_collection_set(target_pause_time_ms, &_survivor);
2841   evacuation_info.set_collectionset_regions(collection_set()->region_length() +
2842                                             collection_set()->optional_region_length());
2843 
2844   _cm->verify_no_collection_set_oops();
2845 
2846   if (_hr_printer.is_active()) {
2847     G1PrintCollectionSetClosure cl(&_hr_printer);
2848     _collection_set.iterate(&cl);
2849     _collection_set.iterate_optional(&cl);
2850   }
2851 }
2852 
2853 G1HeapVerifier::G1VerifyType G1CollectedHeap::young_collection_verify_type() const {
2854   if (collector_state()->in_concurrent_start_gc()) {
2855     return G1HeapVerifier::G1VerifyConcurrentStart;
2856   } else if (collector_state()->in_young_only_phase()) {
2857     return G1HeapVerifier::G1VerifyYoungNormal;
2858   } else {
2859     return G1HeapVerifier::G1VerifyMixed;
2860   }
2861 }
2862 
2863 void G1CollectedHeap::verify_before_young_collection(G1HeapVerifier::G1VerifyType type) {
2864   if (VerifyRememberedSets) {
2865     log_info(gc, verify)("[Verifying RemSets before GC]");
2866     VerifyRegionRemSetClosure v_cl;
2867     heap_region_iterate(&v_cl);
2868   }
2869   _verifier->verify_before_gc(type);
2870   _verifier->check_bitmaps("GC Start");
2871   verify_numa_regions("GC Start");
2872 }
2873 
2874 void G1CollectedHeap::verify_after_young_collection(G1HeapVerifier::G1VerifyType type) {


2879   }
2880   _verifier->verify_after_gc(type);
2881   _verifier->check_bitmaps("GC End");
2882   verify_numa_regions("GC End");
2883 }
2884 
2885 void G1CollectedHeap::expand_heap_after_young_collection(){
2886   size_t expand_bytes = _heap_sizing_policy->young_collection_expansion_amount();
2887   if (expand_bytes > 0) {
2888     // No need for an ergo logging here,
2889     // expansion_amount() does this when it returns a value > 0.
2890     double expand_ms;
2891     if (!expand(expand_bytes, _workers, &expand_ms)) {
2892       // We failed to expand the heap. Cannot do anything about it.
2893     }
2894     phase_times()->record_expand_heap_time(expand_ms);
2895   }
2896 }
2897 
2898 const char* G1CollectedHeap::young_gc_name() const {
2899   if (collector_state()->in_concurrent_start_gc()) {
2900     return "Pause Young (Concurrent Start)";
2901   } else if (collector_state()->in_young_only_phase()) {
2902     if (collector_state()->in_young_gc_before_mixed()) {
2903       return "Pause Young (Prepare Mixed)";
2904     } else {
2905       return "Pause Young (Normal)";
2906     }
2907   } else {
2908     return "Pause Young (Mixed)";
2909   }
2910 }
2911 
2912 bool G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
2913   assert_at_safepoint_on_vm_thread();
2914   guarantee(!is_gc_active(), "collection is not reentrant");
2915 
2916   if (GCLocker::check_active_before_gc()) {
2917     return false;
2918   }
2919 


2932 void G1CollectedHeap::do_collection_pause_at_safepoint_helper(double target_pause_time_ms) {
2933   GCIdMark gc_id_mark;
2934 
2935   SvcGCMarker sgcm(SvcGCMarker::MINOR);
2936   ResourceMark rm;
2937 
2938   policy()->note_gc_start();
2939 
2940   _gc_timer_stw->register_gc_start();
2941   _gc_tracer_stw->report_gc_start(gc_cause(), _gc_timer_stw->gc_start());
2942 
2943   wait_for_root_region_scanning();
2944 
2945   print_heap_before_gc();
2946   print_heap_regions();
2947   trace_heap_before_gc(_gc_tracer_stw);
2948 
2949   _verifier->verify_region_sets_optional();
2950   _verifier->verify_dirty_young_regions();
2951 
2952   // We should not be doing concurrent start unless the concurrent mark thread is running
2953   if (!_cm_thread->should_terminate()) {
2954     // This call will decide whether this pause is a concurrent start
2955     // pause. If it is, in_concurrent_start_gc() will return true
2956     // for the duration of this pause.
2957     policy()->decide_on_conc_mark_initiation();
2958   }
2959 
2960   // We do not allow concurrent start to be piggy-backed on a mixed GC.
2961   assert(!collector_state()->in_concurrent_start_gc() ||
2962          collector_state()->in_young_only_phase(), "sanity");
2963   // We also do not allow mixed GCs during marking.
2964   assert(!collector_state()->mark_or_rebuild_in_progress() || collector_state()->in_young_only_phase(), "sanity");
2965 
2966   // Record whether this pause is a concurrent start. When the current
2967   // thread has completed its logging output and it's safe to signal
2968   // the CM thread, the flag's value in the policy has been reset.
2969   bool should_start_conc_mark = collector_state()->in_concurrent_start_gc();
2970   if (should_start_conc_mark) {
2971     _cm->gc_tracer_cm()->set_gc_cause(gc_cause());
2972   }
2973 
2974   // Inner scope for scope based logging, timers, and stats collection
2975   {
2976     G1EvacuationInfo evacuation_info;
2977 
2978     _gc_tracer_stw->report_yc_type(collector_state()->yc_type());
2979 
2980     GCTraceCPUTime tcpu;
2981 
2982     GCTraceTime(Info, gc) tm(young_gc_name(), NULL, gc_cause(), true);
2983 
2984     uint active_workers = WorkerPolicy::calc_active_workers(workers()->total_workers(),
2985                                                             workers()->active_workers(),
2986                                                             Threads::number_of_non_daemon_threads());
2987     active_workers = workers()->update_active_workers(active_workers);
2988     log_info(gc,task)("Using %u workers of %u for evacuation", active_workers, workers()->total_workers());
2989 


3033                                                   collection_set()->optional_region_length());
3034         pre_evacuate_collection_set(evacuation_info, &per_thread_states);
3035 
3036         // Actually do the work...
3037         evacuate_initial_collection_set(&per_thread_states);
3038 
3039         if (_collection_set.optional_region_length() != 0) {
3040           evacuate_optional_collection_set(&per_thread_states);
3041         }
3042         post_evacuate_collection_set(evacuation_info, &rdcqs, &per_thread_states);
3043 
3044         start_new_collection_set();
3045 
3046         _survivor_evac_stats.adjust_desired_plab_sz();
3047         _old_evac_stats.adjust_desired_plab_sz();
3048 
3049         if (should_start_conc_mark) {
3050           // We have to do this before we notify the CM threads that
3051           // they can start working to make sure that all the
3052           // appropriate initialization is done on the CM object.
3053           concurrent_mark()->post_concurrent_start();
3054           // Note that we don't actually trigger the CM thread at
3055           // this point. We do that later when we're sure that
3056           // the current thread has completed its logging output.
3057         }
3058 
3059         allocate_dummy_regions();
3060 
3061         _allocator->init_mutator_alloc_regions();
3062 
3063         expand_heap_after_young_collection();
3064 
3065         double sample_end_time_sec = os::elapsedTime();
3066         double pause_time_ms = (sample_end_time_sec - sample_start_time_sec) * MILLIUNITS;
3067         policy()->record_collection_pause_end(pause_time_ms);
3068       }
3069 
3070       verify_after_young_collection(verify_type);
3071 
3072       gc_epilogue(false);
3073     }


3514                                               &drain_queue,
3515                                               &par_task_executor,
3516                                               pt);
3517   }
3518 
3519   _gc_tracer_stw->report_gc_reference_stats(stats);
3520 
3521   // We have completed copying any necessary live referent objects.
3522   assert(pss->queue_is_empty(), "both queue and overflow should be empty");
3523 
3524   make_pending_list_reachable();
3525 
3526   assert(!rp->discovery_enabled(), "Postcondition");
3527   rp->verify_no_references_recorded();
3528 
3529   double ref_proc_time = os::elapsedTime() - ref_proc_start;
3530   phase_times()->record_ref_proc_time(ref_proc_time * 1000.0);
3531 }
3532 
3533 void G1CollectedHeap::make_pending_list_reachable() {
3534   if (collector_state()->in_concurrent_start_gc()) {
3535     oop pll_head = Universe::reference_pending_list();
3536     if (pll_head != NULL) {
3537       // Any valid worker id is fine here as we are in the VM thread and single-threaded.
3538       _cm->mark_in_next_bitmap(0 /* worker_id */, pll_head);
3539     }
3540   }
3541 }
3542 
3543 void G1CollectedHeap::merge_per_thread_state_info(G1ParScanThreadStateSet* per_thread_states) {
3544   Ticks start = Ticks::now();
3545   per_thread_states->flush();
3546   phase_times()->record_or_add_time_secs(G1GCPhaseTimes::MergePSS, 0 /* worker_id */, (Ticks::now() - start).seconds());
3547 }
3548 
3549 class G1PrepareEvacuationTask : public AbstractGangTask {
3550   class G1PrepareRegionsClosure : public HeapRegionClosure {
3551     G1CollectedHeap* _g1h;
3552     G1PrepareEvacuationTask* _parent_task;
3553     size_t _worker_humongous_total;
3554     size_t _worker_humongous_candidates;


3703     rem_set()->prepare_for_scan_heap_roots();
3704     phase_times()->record_prepare_heap_roots_time_ms((Ticks::now() - start).seconds() * 1000.0);
3705   }
3706 
3707   {
3708     G1PrepareEvacuationTask g1_prep_task(this);
3709     Tickspan task_time = run_task(&g1_prep_task);
3710 
3711     phase_times()->record_register_regions(task_time.seconds() * 1000.0,
3712                                            g1_prep_task.humongous_total(),
3713                                            g1_prep_task.humongous_candidates());
3714   }
3715 
3716   assert(_verifier->check_region_attr_table(), "Inconsistency in the region attributes table.");
3717   _preserved_marks_set.assert_empty();
3718 
3719 #if COMPILER2_OR_JVMCI
3720   DerivedPointerTable::clear();
3721 #endif
3722 
3723   // Concurrent start needs claim bits to keep track of the marked-through CLDs.
3724   if (collector_state()->in_concurrent_start_gc()) {
3725     concurrent_mark()->pre_concurrent_start();
3726 
3727     double start_clear_claimed_marks = os::elapsedTime();
3728 
3729     ClassLoaderDataGraph::clear_claimed_marks();
3730 
3731     double recorded_clear_claimed_marks_time_ms = (os::elapsedTime() - start_clear_claimed_marks) * 1000.0;
3732     phase_times()->record_clear_claimed_marks_time_ms(recorded_clear_claimed_marks_time_ms);
3733   }
3734 
3735   // Should G1EvacuationFailureALot be in effect for this GC?
3736   NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
3737 }
3738 
3739 class G1EvacuateRegionsBaseTask : public AbstractGangTask {
3740 protected:
3741   G1CollectedHeap* _g1h;
3742   G1ParScanThreadStateSet* _per_thread_states;
3743   G1ScannerTasksQueueSet* _task_queues;
3744   TaskTerminator _terminator;
3745   uint _num_workers;


4773     }
4774     _policy->remset_tracker()->update_at_allocate(new_alloc_region);
4775     register_region_with_region_attr(new_alloc_region);
4776     _hr_printer.alloc(new_alloc_region);
4777     return new_alloc_region;
4778   }
4779   return NULL;
4780 }
4781 
4782 void G1CollectedHeap::retire_gc_alloc_region(HeapRegion* alloc_region,
4783                                              size_t allocated_bytes,
4784                                              G1HeapRegionAttr dest) {
4785   _bytes_used_during_gc += allocated_bytes;
4786   if (dest.is_old()) {
4787     old_set_add(alloc_region);
4788   } else {
4789     assert(dest.is_young(), "Retiring alloc region should be young (%d)", dest.type());
4790     _survivor.add_used_bytes(allocated_bytes);
4791   }
4792 
4793   bool const during_im = collector_state()->in_concurrent_start_gc();
4794   if (during_im && allocated_bytes > 0) {
4795     _cm->root_regions()->add(alloc_region->next_top_at_mark_start(), alloc_region->top());
4796   }
4797   _hr_printer.retire(alloc_region);
4798 }
4799 
4800 HeapRegion* G1CollectedHeap::alloc_highest_free_region() {
4801   bool expanded = false;
4802   uint index = _hrm->find_highest_free(&expanded);
4803 
4804   if (index != G1_NO_HRM_INDEX) {
4805     if (expanded) {
4806       log_debug(gc, ergo, heap)("Attempt heap expansion (requested address range outside heap bounds). region size: " SIZE_FORMAT "B",
4807                                 HeapRegion::GrainWords * HeapWordSize);
4808     }
4809     return _hrm->allocate_free_regions_starting_at(index, 1);
4810   }
4811   return NULL;
4812 }
4813 


< prev index next >