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

Print this page
rev 4973 : imported patch checkpointing.diff


  74 // apply to TLAB allocation, which is not part of this interface: it
  75 // is done by clients of this interface.)
  76 
  77 // Notes on implementation of parallelism in different tasks.
  78 //
  79 // G1ParVerifyTask uses heap_region_par_iterate_chunked() for parallelism.
  80 // The number of GC workers is passed to heap_region_par_iterate_chunked().
  81 // It does use run_task() which sets _n_workers in the task.
  82 // G1ParTask executes g1_process_strong_roots() ->
  83 // SharedHeap::process_strong_roots() which calls eventually to
  84 // CardTableModRefBS::par_non_clean_card_iterate_work() which uses
  85 // SequentialSubTasksDone.  SharedHeap::process_strong_roots() also
  86 // directly uses SubTasksDone (_process_strong_tasks field in SharedHeap).
  87 //
  88 
  89 // Local to this file.
  90 
  91 class RefineCardTableEntryClosure: public CardTableEntryClosure {
  92   SuspendibleThreadSet* _sts;
  93   G1RemSet* _g1rs;
  94   ConcurrentG1Refine* _cg1r;
  95   bool _concurrent;
  96 public:
  97   RefineCardTableEntryClosure(SuspendibleThreadSet* sts,
  98                               G1RemSet* g1rs,
  99                               ConcurrentG1Refine* cg1r) :
 100     _sts(sts), _g1rs(g1rs), _cg1r(cg1r), _concurrent(true)
 101   {}
 102   bool do_card_ptr(jbyte* card_ptr, int worker_i) {
 103     bool oops_into_cset = _g1rs->refine_card(card_ptr, worker_i, false);






 104     // This path is executed by the concurrent refine or mutator threads,
 105     // concurrently, and so we do not care if card_ptr contains references
 106     // that point into the collection set.
 107     assert(!oops_into_cset, "should be");
 108 
 109     if (_concurrent && _sts->should_yield()) {
 110       // Caller will actually yield.
 111       return false;
 112     }
 113     // Otherwise, we finished successfully; return true.
 114     return true;
 115   }
 116   void set_concurrent(bool b) { _concurrent = b; }
 117 };
 118 

































 119 
 120 class ClearLoggedCardTableEntryClosure: public CardTableEntryClosure {
 121   int _calls;
 122   G1CollectedHeap* _g1h;
 123   CardTableModRefBS* _ctbs;
 124   int _histo[256];
 125 public:
 126   ClearLoggedCardTableEntryClosure() :
 127     _calls(0)
 128   {
 129     _g1h = G1CollectedHeap::heap();
 130     _ctbs = (CardTableModRefBS*)_g1h->barrier_set();
 131     for (int i = 0; i < 256; i++) _histo[i] = 0;
 132   }
 133   bool do_card_ptr(jbyte* card_ptr, int worker_i) {
 134     if (_g1h->is_in_reserved(_ctbs->addr_for(card_ptr))) {
 135       _calls++;
 136       unsigned char* ujb = (unsigned char*)card_ptr;
 137       int ind = (int)(*ujb);
 138       _histo[ind]++;


1922 #pragma warning( disable:4355 ) // 'this' : used in base member initializer list
1923 #endif // _MSC_VER
1924 
1925 
1926 G1CollectedHeap::G1CollectedHeap(G1CollectorPolicy* policy_) :
1927   SharedHeap(policy_),
1928   _g1_policy(policy_),
1929   _dirty_card_queue_set(false),
1930   _into_cset_dirty_card_queue_set(false),
1931   _is_alive_closure_cm(this),
1932   _is_alive_closure_stw(this),
1933   _ref_processor_cm(NULL),
1934   _ref_processor_stw(NULL),
1935   _process_strong_tasks(new SubTasksDone(G1H_PS_NumElements)),
1936   _bot_shared(NULL),
1937   _evac_failure_scan_stack(NULL),
1938   _mark_in_progress(false),
1939   _cg1r(NULL), _summary_bytes_used(0),
1940   _g1mm(NULL),
1941   _refine_cte_cl(NULL),

1942   _full_collection(false),
1943   _free_list("Master Free List"),
1944   _secondary_free_list("Secondary Free List"),
1945   _old_set("Old Set"),
1946   _humongous_set("Master Humongous Set"),
1947   _free_regions_coming(false),
1948   _young_list(new YoungList(this)),
1949   _gc_time_stamp(0),
1950   _retained_old_gc_alloc_region(NULL),
1951   _survivor_plab_stats(YoungPLABSize, PLABWeight),
1952   _old_plab_stats(OldPLABSize, PLABWeight),
1953   _expand_heap_after_alloc_failure(true),
1954   _surviving_young_words(NULL),
1955   _old_marking_cycles_started(0),
1956   _old_marking_cycles_completed(0),
1957   _concurrent_cycle_started(false),
1958   _in_cset_fast_test(NULL),
1959   _in_cset_fast_test_base(NULL),
1960   _dirty_cards_region_list(NULL),
1961   _worker_cset_start_region(NULL),


2126   if (_cm == NULL || !_cm->completed_initialization()) {
2127     vm_shutdown_during_initialization("Could not create/initialize ConcurrentMark");
2128     return JNI_ENOMEM;
2129   }
2130   _cmThread = _cm->cmThread();
2131 
2132   // Initialize the from_card cache structure of HeapRegionRemSet.
2133   HeapRegionRemSet::init_heap(max_regions());
2134 
2135   // Now expand into the initial heap size.
2136   if (!expand(init_byte_size)) {
2137     vm_shutdown_during_initialization("Failed to allocate initial heap.");
2138     return JNI_ENOMEM;
2139   }
2140 
2141   // Perform any initialization actions delegated to the policy.
2142   g1_policy()->init();
2143 
2144   _refine_cte_cl =
2145     new RefineCardTableEntryClosure(ConcurrentG1RefineThread::sts(),
2146                                     g1_rem_set(),
2147                                     concurrent_g1_refine());
2148   JavaThread::dirty_card_queue_set().set_closure(_refine_cte_cl);





2149 
2150   JavaThread::satb_mark_queue_set().initialize(SATB_Q_CBL_mon,
2151                                                SATB_Q_FL_lock,
2152                                                G1SATBProcessCompletedThreshold,
2153                                                Shared_SATB_Q_lock);
2154 
2155   JavaThread::dirty_card_queue_set().initialize(DirtyCardQ_CBL_mon,
2156                                                 DirtyCardQ_FL_lock,
2157                                                 concurrent_g1_refine()->yellow_zone(),
2158                                                 concurrent_g1_refine()->red_zone(),
2159                                                 Shared_DirtyCardQ_lock);
2160 
2161   if (G1DeferredRSUpdate) {
2162     dirty_card_queue_set().initialize(DirtyCardQ_CBL_mon,
2163                                       DirtyCardQ_FL_lock,
2164                                       -1, // never trigger processing
2165                                       -1, // no limit on length
2166                                       Shared_DirtyCardQ_lock,
2167                                       &JavaThread::dirty_card_queue_set());
2168   }


2316     unsigned region_gc_time_stamp = hr->get_gc_time_stamp();
2317     if (_gc_time_stamp != region_gc_time_stamp) {
2318       gclog_or_tty->print_cr("Region "HR_FORMAT" has GC time stamp = %d, "
2319                              "expected %d", HR_FORMAT_PARAMS(hr),
2320                              region_gc_time_stamp, _gc_time_stamp);
2321       _failures = true;
2322     }
2323     return false;
2324   }
2325 
2326   bool failures() { return _failures; }
2327 };
2328 
2329 void G1CollectedHeap::check_gc_time_stamps() {
2330   CheckGCTimeStampsHRClosure cl(_gc_time_stamp);
2331   heap_region_iterate(&cl);
2332   guarantee(!cl.failures(), "all GC time stamps should have been reset");
2333 }
2334 #endif // PRODUCT
2335 
2336 void G1CollectedHeap::iterate_dirty_card_closure(CardTableEntryClosure* cl,

2337                                                  DirtyCardQueue* into_cset_dcq,
2338                                                  bool concurrent,
2339                                                  int worker_i) {
2340   // Clean cards in the hot card cache
2341   G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
2342   hot_card_cache->drain(worker_i, g1_rem_set(), into_cset_dcq);
2343 
2344   DirtyCardQueueSet& dcqs = JavaThread::dirty_card_queue_set();
2345   int n_completed_buffers = 0;









2346   while (dcqs.apply_closure_to_completed_buffer(cl, worker_i, 0, true)) {
2347     n_completed_buffers++;
2348   }
2349   g1_policy()->phase_times()->record_update_rs_processed_buffers(worker_i, n_completed_buffers);
2350   dcqs.clear_n_completed_buffers();
2351   assert(!dcqs.completed_buffers_exist_dirty(), "Completed buffers exist!");

2352 }
2353 
2354 
2355 // Computes the sum of the storage used by the various regions.
2356 
2357 size_t G1CollectedHeap::used() const {
2358   assert(Heap_lock->owner() != NULL,
2359          "Should be owned on this thread's behalf.");
2360   size_t result = _summary_bytes_used;
2361   // Read only once in case it is set to NULL concurrently
2362   HeapRegion* hr = _mutator_alloc_region.get();
2363   if (hr != NULL)
2364     result += hr->used();
2365   return result;
2366 }
2367 
2368 size_t G1CollectedHeap::used_unlocked() const {
2369   size_t result = _summary_bytes_used;
2370   return result;
2371 }


3031   return HeapRegion::GrainBytes;
3032 }
3033 
3034 size_t G1CollectedHeap::unsafe_max_tlab_alloc(Thread* ignored) const {
3035   // Return the remaining space in the cur alloc region, but not less than
3036   // the min TLAB size.
3037 
3038   // Also, this value can be at most the humongous object threshold,
3039   // since we can't allow tlabs to grow big enough to accommodate
3040   // humongous objects.
3041 
3042   HeapRegion* hr = _mutator_alloc_region.get();
3043   size_t max_tlab_size = _humongous_object_threshold_in_words * wordSize;
3044   if (hr == NULL) {
3045     return max_tlab_size;
3046   } else {
3047     return MIN2(MAX2(hr->free(), (size_t) MinTLABSize), max_tlab_size);
3048   }
3049 }
3050 





3051 size_t G1CollectedHeap::max_capacity() const {
3052   return _g1_reserved.byte_size();
3053 }
3054 
3055 jlong G1CollectedHeap::millis_since_last_gc() {
3056   // assert(false, "NYI");
3057   return 0;
3058 }
3059 
3060 void G1CollectedHeap::prepare_for_verify() {
3061   if (SafepointSynchronize::is_at_safepoint() || ! UseTLAB) {
3062     ensure_parsability(false);
3063   }
3064   g1_rem_set()->prepare_for_verify();
3065 }
3066 
3067 bool G1CollectedHeap::allocated_since_marking(oop obj, HeapRegion* hr,
3068                                               VerifyOption vo) {
3069   switch (vo) {
3070   case VerifyOption_G1UsePrevMarking:


5800     // Reset the G1EvacuationFailureALot counters and flags
5801     // Note: the values are reset only when an actual
5802     // evacuation failure occurs.
5803     NOT_PRODUCT(reset_evacuation_should_fail();)
5804   }
5805 
5806   // Enqueue any remaining references remaining on the STW
5807   // reference processor's discovered lists. We need to do
5808   // this after the card table is cleaned (and verified) as
5809   // the act of enqueueing entries on to the pending list
5810   // will log these updates (and dirty their associated
5811   // cards). We need these updates logged to update any
5812   // RSets.
5813   enqueue_discovered_references(n_workers);
5814 
5815   if (G1DeferredRSUpdate) {
5816     RedirtyLoggedCardTableEntryFastClosure redirty;
5817     dirty_card_queue_set().set_closure(&redirty);
5818     dirty_card_queue_set().apply_closure_to_all_completed_buffers();
5819 
5820     DirtyCardQueueSet& dcq = JavaThread::dirty_card_queue_set();
5821     dcq.merge_bufferlists(&dirty_card_queue_set());
5822     assert(dirty_card_queue_set().completed_buffers_num() == 0, "All should be consumed");
5823   }
5824   COMPILER2_PRESENT(DerivedPointerTable::update_pointers());
5825 }
5826 
5827 void G1CollectedHeap::free_region_if_empty(HeapRegion* hr,
5828                                      size_t* pre_used,
5829                                      FreeRegionList* free_list,
5830                                      OldRegionSet* old_proxy_set,
5831                                      HumongousRegionSet* humongous_proxy_set,
5832                                      HRRSCleanupTask* hrrs_cleanup_task,
5833                                      bool par) {
5834   if (hr->used() > 0 && hr->max_live_bytes() == 0 && !hr->is_young()) {
5835     if (hr->isHumongous()) {
5836       assert(hr->startsHumongous(), "we should only see starts humongous");
5837       free_humongous_region(hr, pre_used, free_list, humongous_proxy_set, par);
5838     } else {
5839       _old_set.remove_with_proxy(hr, old_proxy_set);
5840       free_region(hr, pre_used, free_list, par);


6339 };
6340 
6341 void G1CollectedHeap::rebuild_region_sets(bool free_list_only) {
6342   assert_at_safepoint(true /* should_be_vm_thread */);
6343 
6344   RebuildRegionSetsClosure cl(free_list_only, &_old_set, &_free_list);
6345   heap_region_iterate(&cl);
6346 
6347   if (!free_list_only) {
6348     _summary_bytes_used = cl.total_used();
6349   }
6350   assert(_summary_bytes_used == recalculate_used(),
6351          err_msg("inconsistent _summary_bytes_used, "
6352                  "value: "SIZE_FORMAT" recalculated: "SIZE_FORMAT,
6353                  _summary_bytes_used, recalculate_used()));
6354 }
6355 
6356 void G1CollectedHeap::set_refine_cte_cl_concurrency(bool concurrent) {
6357   _refine_cte_cl->set_concurrent(concurrent);
6358 }











6359 
6360 bool G1CollectedHeap::is_in_closed_subset(const void* p) const {
6361   HeapRegion* hr = heap_region_containing(p);
6362   if (hr == NULL) {
6363     return false;
6364   } else {
6365     return hr->is_in(p);
6366   }
6367 }
6368 
6369 // Methods for the mutator alloc region
6370 
6371 HeapRegion* G1CollectedHeap::new_mutator_alloc_region(size_t word_size,
6372                                                       bool force) {
6373   assert_heap_locked_or_at_safepoint(true /* should_be_vm_thread */);
6374   assert(!force || g1_policy()->can_expand_young_list(),
6375          "if force is true we should be able to expand the young list");
6376   bool young_list_full = g1_policy()->is_young_list_full();
6377   if (force || !young_list_full) {
6378     HeapRegion* new_alloc_region = new_region(word_size,




  74 // apply to TLAB allocation, which is not part of this interface: it
  75 // is done by clients of this interface.)
  76 
  77 // Notes on implementation of parallelism in different tasks.
  78 //
  79 // G1ParVerifyTask uses heap_region_par_iterate_chunked() for parallelism.
  80 // The number of GC workers is passed to heap_region_par_iterate_chunked().
  81 // It does use run_task() which sets _n_workers in the task.
  82 // G1ParTask executes g1_process_strong_roots() ->
  83 // SharedHeap::process_strong_roots() which calls eventually to
  84 // CardTableModRefBS::par_non_clean_card_iterate_work() which uses
  85 // SequentialSubTasksDone.  SharedHeap::process_strong_roots() also
  86 // directly uses SubTasksDone (_process_strong_tasks field in SharedHeap).
  87 //
  88 
  89 // Local to this file.
  90 
  91 class RefineCardTableEntryClosure: public CardTableEntryClosure {
  92   SuspendibleThreadSet* _sts;
  93   G1RemSet* _g1rs;

  94   bool _concurrent;
  95 public:
  96   RefineCardTableEntryClosure(SuspendibleThreadSet* sts,
  97                               G1RemSet* g1rs) :
  98     _sts(sts), _g1rs(g1rs), _concurrent(true)

  99   {}
 100   bool do_card_ptr(jbyte* card_ptr, int worker_i) {
 101     bool oops_into_cset;
 102     if (_concurrent) {
 103       oops_into_cset = _g1rs->refine_card_without_check(card_ptr, worker_i, false);
 104     } else {
 105       assert(SafepointSynchronize::is_at_safepoint(), "only safe if at safepoint");
 106       oops_into_cset = _g1rs->refine_card(card_ptr, worker_i, false);
 107     }
 108     // This path is executed by the concurrent refine or mutator threads,
 109     // concurrently, and so we do not care if card_ptr contains references
 110     // that point into the collection set.
 111     assert(!oops_into_cset, "should be");
 112 
 113     if (_concurrent && _sts->should_yield()) {
 114       // Caller will actually yield.
 115       return false;
 116     }
 117     // Otherwise, we finished successfully; return true.
 118     return true;
 119   }
 120   void set_concurrent(bool b) { _concurrent = b; }
 121 };
 122 
 123 class TransferDirtyCardsToRefinementClosure: public CardTableEntryClosure {
 124  public:
 125   class FlushTransferClosure: public VoidClosure {
 126     friend class TransferDirtyCardsToRefinementClosure;
 127     TransferDirtyCardsToRefinementClosure* _cl;
 128    public:
 129     void do_void() {
 130       assert(_cl != NULL, "should be inited");
 131       _cl->_dcq.flush();
 132     }
 133   };
 134 
 135  private:
 136   FlushTransferClosure _flush_cl;
 137   DirtyCardQueue _dcq;
 138  public:
 139   TransferDirtyCardsToRefinementClosure(DirtyCardQueueSet& target) : _dcq(&target) {
 140     _flush_cl._cl = this;
 141   }
 142 
 143   bool do_card_ptr(jbyte* card_ptr, int worker_i) {
 144     assert(SafepointSynchronize::is_at_safepoint(), "only safe to transfer/clean cards at safepoint");
 145     assert(!G1CollectedHeap::heap()->is_gc_active(), "should not get here if doing a gc");
 146     _dcq.enqueue(card_ptr);
 147     *card_ptr = CardTableModRefBS::clean_card_val();
 148     return true;
 149   }
 150 
 151  public:
 152   FlushTransferClosure* flush_cl() { return &_flush_cl; }
 153 };
 154 
 155 
 156 
 157 class ClearLoggedCardTableEntryClosure: public CardTableEntryClosure {
 158   int _calls;
 159   G1CollectedHeap* _g1h;
 160   CardTableModRefBS* _ctbs;
 161   int _histo[256];
 162 public:
 163   ClearLoggedCardTableEntryClosure() :
 164     _calls(0)
 165   {
 166     _g1h = G1CollectedHeap::heap();
 167     _ctbs = (CardTableModRefBS*)_g1h->barrier_set();
 168     for (int i = 0; i < 256; i++) _histo[i] = 0;
 169   }
 170   bool do_card_ptr(jbyte* card_ptr, int worker_i) {
 171     if (_g1h->is_in_reserved(_ctbs->addr_for(card_ptr))) {
 172       _calls++;
 173       unsigned char* ujb = (unsigned char*)card_ptr;
 174       int ind = (int)(*ujb);
 175       _histo[ind]++;


1959 #pragma warning( disable:4355 ) // 'this' : used in base member initializer list
1960 #endif // _MSC_VER
1961 
1962 
1963 G1CollectedHeap::G1CollectedHeap(G1CollectorPolicy* policy_) :
1964   SharedHeap(policy_),
1965   _g1_policy(policy_),
1966   _dirty_card_queue_set(false),
1967   _into_cset_dirty_card_queue_set(false),
1968   _is_alive_closure_cm(this),
1969   _is_alive_closure_stw(this),
1970   _ref_processor_cm(NULL),
1971   _ref_processor_stw(NULL),
1972   _process_strong_tasks(new SubTasksDone(G1H_PS_NumElements)),
1973   _bot_shared(NULL),
1974   _evac_failure_scan_stack(NULL),
1975   _mark_in_progress(false),
1976   _cg1r(NULL), _summary_bytes_used(0),
1977   _g1mm(NULL),
1978   _refine_cte_cl(NULL),
1979   _transfer_cte_cl(NULL),
1980   _full_collection(false),
1981   _free_list("Master Free List"),
1982   _secondary_free_list("Secondary Free List"),
1983   _old_set("Old Set"),
1984   _humongous_set("Master Humongous Set"),
1985   _free_regions_coming(false),
1986   _young_list(new YoungList(this)),
1987   _gc_time_stamp(0),
1988   _retained_old_gc_alloc_region(NULL),
1989   _survivor_plab_stats(YoungPLABSize, PLABWeight),
1990   _old_plab_stats(OldPLABSize, PLABWeight),
1991   _expand_heap_after_alloc_failure(true),
1992   _surviving_young_words(NULL),
1993   _old_marking_cycles_started(0),
1994   _old_marking_cycles_completed(0),
1995   _concurrent_cycle_started(false),
1996   _in_cset_fast_test(NULL),
1997   _in_cset_fast_test_base(NULL),
1998   _dirty_cards_region_list(NULL),
1999   _worker_cset_start_region(NULL),


2164   if (_cm == NULL || !_cm->completed_initialization()) {
2165     vm_shutdown_during_initialization("Could not create/initialize ConcurrentMark");
2166     return JNI_ENOMEM;
2167   }
2168   _cmThread = _cm->cmThread();
2169 
2170   // Initialize the from_card cache structure of HeapRegionRemSet.
2171   HeapRegionRemSet::init_heap(max_regions());
2172 
2173   // Now expand into the initial heap size.
2174   if (!expand(init_byte_size)) {
2175     vm_shutdown_during_initialization("Failed to allocate initial heap.");
2176     return JNI_ENOMEM;
2177   }
2178 
2179   // Perform any initialization actions delegated to the policy.
2180   g1_policy()->init();
2181 
2182   _refine_cte_cl =
2183     new RefineCardTableEntryClosure(ConcurrentG1RefineThread::sts(),
2184                                     g1_rem_set());
2185   _transfer_cte_cl =
2186     new TransferDirtyCardsToRefinementClosure(concurrent_g1_refine()->cards_ready_for_refinement());
2187 
2188   concurrent_g1_refine()->set_card_refinement_closure(_refine_cte_cl);
2189   concurrent_g1_refine()->set_flush_transfer_closure(_transfer_cte_cl->flush_cl());
2190 
2191   JavaThread::dirty_card_queue_set().set_closure(_transfer_cte_cl);
2192 
2193   JavaThread::satb_mark_queue_set().initialize(SATB_Q_CBL_mon,
2194                                                SATB_Q_FL_lock,
2195                                                G1SATBProcessCompletedThreshold,
2196                                                Shared_SATB_Q_lock);
2197 
2198   JavaThread::dirty_card_queue_set().initialize(DirtyCardQ_CBL_mon,
2199                                                 DirtyCardQ_FL_lock,
2200                                                 concurrent_g1_refine()->yellow_zone(),
2201                                                 concurrent_g1_refine()->red_zone(),
2202                                                 Shared_DirtyCardQ_lock);
2203 
2204   if (G1DeferredRSUpdate) {
2205     dirty_card_queue_set().initialize(DirtyCardQ_CBL_mon,
2206                                       DirtyCardQ_FL_lock,
2207                                       -1, // never trigger processing
2208                                       -1, // no limit on length
2209                                       Shared_DirtyCardQ_lock,
2210                                       &JavaThread::dirty_card_queue_set());
2211   }


2359     unsigned region_gc_time_stamp = hr->get_gc_time_stamp();
2360     if (_gc_time_stamp != region_gc_time_stamp) {
2361       gclog_or_tty->print_cr("Region "HR_FORMAT" has GC time stamp = %d, "
2362                              "expected %d", HR_FORMAT_PARAMS(hr),
2363                              region_gc_time_stamp, _gc_time_stamp);
2364       _failures = true;
2365     }
2366     return false;
2367   }
2368 
2369   bool failures() { return _failures; }
2370 };
2371 
2372 void G1CollectedHeap::check_gc_time_stamps() {
2373   CheckGCTimeStampsHRClosure cl(_gc_time_stamp);
2374   heap_region_iterate(&cl);
2375   guarantee(!cl.failures(), "all GC time stamps should have been reset");
2376 }
2377 #endif // PRODUCT
2378 
2379 int G1CollectedHeap::iterate_dirty_card_closure(CardTableEntryClosure* cl,
2380                                                  CardTableEntryClosure* cleaned_cl,
2381                                                  DirtyCardQueue* into_cset_dcq,
2382                                                  bool concurrent,
2383                                                  int worker_i) {
2384   // Clean cards in the hot card cache
2385   G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
2386   hot_card_cache->drain(worker_i, g1_rem_set(), into_cset_dcq);
2387 

2388   int n_completed_buffers = 0;
2389   if (worker_i == 0) {
2390     concurrent_g1_refine()->flush_cards_in_buffers();
2391   }
2392   DirtyCardQueueSet& refinement_dcqs = concurrent_g1_refine()->cards_ready_for_refinement();
2393   while (refinement_dcqs.apply_closure_to_completed_buffer(cleaned_cl, worker_i, 0, true)) {
2394     n_completed_buffers++;
2395   }
2396 
2397   DirtyCardQueueSet& dcqs = JavaThread::dirty_card_queue_set();
2398   while (dcqs.apply_closure_to_completed_buffer(cl, worker_i, 0, true)) {
2399     n_completed_buffers++;
2400   }
2401 
2402   dcqs.clear_n_completed_buffers();
2403   assert(!dcqs.completed_buffers_exist_dirty(), "Completed buffers exist!");
2404   return n_completed_buffers;
2405 }
2406 
2407 
2408 // Computes the sum of the storage used by the various regions.
2409 
2410 size_t G1CollectedHeap::used() const {
2411   assert(Heap_lock->owner() != NULL,
2412          "Should be owned on this thread's behalf.");
2413   size_t result = _summary_bytes_used;
2414   // Read only once in case it is set to NULL concurrently
2415   HeapRegion* hr = _mutator_alloc_region.get();
2416   if (hr != NULL)
2417     result += hr->used();
2418   return result;
2419 }
2420 
2421 size_t G1CollectedHeap::used_unlocked() const {
2422   size_t result = _summary_bytes_used;
2423   return result;
2424 }


3084   return HeapRegion::GrainBytes;
3085 }
3086 
3087 size_t G1CollectedHeap::unsafe_max_tlab_alloc(Thread* ignored) const {
3088   // Return the remaining space in the cur alloc region, but not less than
3089   // the min TLAB size.
3090 
3091   // Also, this value can be at most the humongous object threshold,
3092   // since we can't allow tlabs to grow big enough to accommodate
3093   // humongous objects.
3094 
3095   HeapRegion* hr = _mutator_alloc_region.get();
3096   size_t max_tlab_size = _humongous_object_threshold_in_words * wordSize;
3097   if (hr == NULL) {
3098     return max_tlab_size;
3099   } else {
3100     return MIN2(MAX2(hr->free(), (size_t) MinTLABSize), max_tlab_size);
3101   }
3102 }
3103 
3104 void G1CollectedHeap::inform_non_gc_safepoint() {
3105   DirtyCardQueueSet& dcqs = JavaThread::dirty_card_queue_set();
3106   while (dcqs.apply_closure_to_completed_buffer(0, 0, true));
3107 }
3108 
3109 size_t G1CollectedHeap::max_capacity() const {
3110   return _g1_reserved.byte_size();
3111 }
3112 
3113 jlong G1CollectedHeap::millis_since_last_gc() {
3114   // assert(false, "NYI");
3115   return 0;
3116 }
3117 
3118 void G1CollectedHeap::prepare_for_verify() {
3119   if (SafepointSynchronize::is_at_safepoint() || ! UseTLAB) {
3120     ensure_parsability(false);
3121   }
3122   g1_rem_set()->prepare_for_verify();
3123 }
3124 
3125 bool G1CollectedHeap::allocated_since_marking(oop obj, HeapRegion* hr,
3126                                               VerifyOption vo) {
3127   switch (vo) {
3128   case VerifyOption_G1UsePrevMarking:


5858     // Reset the G1EvacuationFailureALot counters and flags
5859     // Note: the values are reset only when an actual
5860     // evacuation failure occurs.
5861     NOT_PRODUCT(reset_evacuation_should_fail();)
5862   }
5863 
5864   // Enqueue any remaining references remaining on the STW
5865   // reference processor's discovered lists. We need to do
5866   // this after the card table is cleaned (and verified) as
5867   // the act of enqueueing entries on to the pending list
5868   // will log these updates (and dirty their associated
5869   // cards). We need these updates logged to update any
5870   // RSets.
5871   enqueue_discovered_references(n_workers);
5872 
5873   if (G1DeferredRSUpdate) {
5874     RedirtyLoggedCardTableEntryFastClosure redirty;
5875     dirty_card_queue_set().set_closure(&redirty);
5876     dirty_card_queue_set().apply_closure_to_all_completed_buffers();
5877 
5878     DirtyCardQueueSet& dcq = concurrent_g1_refine()->cards_ready_for_refinement();
5879     dcq.merge_bufferlists(&dirty_card_queue_set());
5880     assert(dirty_card_queue_set().completed_buffers_num() == 0, "All should be consumed");
5881   }
5882   COMPILER2_PRESENT(DerivedPointerTable::update_pointers());
5883 }
5884 
5885 void G1CollectedHeap::free_region_if_empty(HeapRegion* hr,
5886                                      size_t* pre_used,
5887                                      FreeRegionList* free_list,
5888                                      OldRegionSet* old_proxy_set,
5889                                      HumongousRegionSet* humongous_proxy_set,
5890                                      HRRSCleanupTask* hrrs_cleanup_task,
5891                                      bool par) {
5892   if (hr->used() > 0 && hr->max_live_bytes() == 0 && !hr->is_young()) {
5893     if (hr->isHumongous()) {
5894       assert(hr->startsHumongous(), "we should only see starts humongous");
5895       free_humongous_region(hr, pre_used, free_list, humongous_proxy_set, par);
5896     } else {
5897       _old_set.remove_with_proxy(hr, old_proxy_set);
5898       free_region(hr, pre_used, free_list, par);


6397 };
6398 
6399 void G1CollectedHeap::rebuild_region_sets(bool free_list_only) {
6400   assert_at_safepoint(true /* should_be_vm_thread */);
6401 
6402   RebuildRegionSetsClosure cl(free_list_only, &_old_set, &_free_list);
6403   heap_region_iterate(&cl);
6404 
6405   if (!free_list_only) {
6406     _summary_bytes_used = cl.total_used();
6407   }
6408   assert(_summary_bytes_used == recalculate_used(),
6409          err_msg("inconsistent _summary_bytes_used, "
6410                  "value: "SIZE_FORMAT" recalculated: "SIZE_FORMAT,
6411                  _summary_bytes_used, recalculate_used()));
6412 }
6413 
6414 void G1CollectedHeap::set_refine_cte_cl_concurrency(bool concurrent) {
6415   _refine_cte_cl->set_concurrent(concurrent);
6416 }
6417 
6418 void G1CollectedHeap::set_use_transferring_cte_cl(bool transferring) {
6419   CardTableEntryClosure* cte_cl = NULL;
6420   if (transferring) {
6421     cte_cl = _transfer_cte_cl;
6422   } else {
6423     cte_cl = _refine_cte_cl;
6424   }
6425   JavaThread::dirty_card_queue_set().set_closure(cte_cl);
6426 }
6427 
6428 
6429 bool G1CollectedHeap::is_in_closed_subset(const void* p) const {
6430   HeapRegion* hr = heap_region_containing(p);
6431   if (hr == NULL) {
6432     return false;
6433   } else {
6434     return hr->is_in(p);
6435   }
6436 }
6437 
6438 // Methods for the mutator alloc region
6439 
6440 HeapRegion* G1CollectedHeap::new_mutator_alloc_region(size_t word_size,
6441                                                       bool force) {
6442   assert_heap_locked_or_at_safepoint(true /* should_be_vm_thread */);
6443   assert(!force || g1_policy()->can_expand_young_list(),
6444          "if force is true we should be able to expand the young list");
6445   bool young_list_full = g1_policy()->is_young_list_full();
6446   if (force || !young_list_full) {
6447     HeapRegion* new_alloc_region = new_region(word_size,