< prev index next >

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

Print this page
rev 51649 : version 1
rev 51652 : Added support for eager mixed collection of evacuation failure regions
rev 51878 : Minor changes
rev 52017 : All changes for G1 GC moved from 'combined' repo folder
rev 52487 : Worked on comments from Sangheon, Stefan
rev 52488 : Merge
rev 52493 : Changing class name
rev 52494 : adding jtreg tests, whitebox api.
rev 52631 : Merge

@@ -164,11 +164,11 @@
 HeapRegion* G1CollectedHeap::new_region(size_t word_size, bool is_old, bool do_expand) {
   assert(!is_humongous(word_size) || word_size <= HeapRegion::GrainWords,
          "the only time we use this to allocate a humongous region is "
          "when we are allocating a single humongous region");
 
-  HeapRegion* res = _hrm.allocate_free_region(is_old);
+  HeapRegion* res = _hrm->allocate_free_region(is_old);
 
   if (res == NULL && do_expand && _expand_heap_after_alloc_failure) {
     // Currently, only attempts to allocate GC alloc regions set
     // do_expand to true. So, we should only reach here during a
     // safepoint. If this assumption changes we might have to

@@ -181,11 +181,11 @@
     if (expand(word_size * HeapWordSize)) {
       // Given that expand() succeeded in expanding the heap, and we
       // always expand the heap by an amount aligned to the heap
       // region size, the free list should in theory not be empty.
       // In either case allocate_free_region() will check for NULL.
-      res = _hrm.allocate_free_region(is_old);
+      res = _hrm->allocate_free_region(is_old);
     } else {
       _expand_heap_after_alloc_failure = false;
     }
   }
   return res;

@@ -335,39 +335,39 @@
       first = hr->hrm_index();
     }
   } else {
     // Policy: Try only empty regions (i.e. already committed first). Maybe we
     // are lucky enough to find some.
-    first = _hrm.find_contiguous_only_empty(obj_regions);
+    first = _hrm->find_contiguous_only_empty(obj_regions);
     if (first != G1_NO_HRM_INDEX) {
-      _hrm.allocate_free_regions_starting_at(first, obj_regions);
+      _hrm->allocate_free_regions_starting_at(first, obj_regions);
     }
   }
 
   if (first == G1_NO_HRM_INDEX) {
     // Policy: We could not find enough regions for the humongous object in the
     // free list. Look through the heap to find a mix of free and uncommitted regions.
     // If so, try expansion.
-    first = _hrm.find_contiguous_empty_or_unavailable(obj_regions);
+    first = _hrm->find_contiguous_empty_or_unavailable(obj_regions);
     if (first != G1_NO_HRM_INDEX) {
       // We found something. Make sure these regions are committed, i.e. expand
       // the heap. Alternatively we could do a defragmentation GC.
       log_debug(gc, ergo, heap)("Attempt heap expansion (humongous allocation request failed). Allocation request: " SIZE_FORMAT "B",
                                     word_size * HeapWordSize);
 
-      _hrm.expand_at(first, obj_regions, workers());
+      _hrm->expand_at(first, obj_regions, workers());
       g1_policy()->record_new_heap_size(num_regions());
 
 #ifdef ASSERT
       for (uint i = first; i < first + obj_regions; ++i) {
         HeapRegion* hr = region_at(i);
         assert(hr->is_free(), "sanity");
         assert(hr->is_empty(), "sanity");
         assert(is_on_master_free_list(hr), "sanity");
       }
 #endif
-      _hrm.allocate_free_regions_starting_at(first, obj_regions);
+      _hrm->allocate_free_regions_starting_at(first, obj_regions);
     } else {
       // Policy: Potentially trigger a defragmentation GC.
     }
   }
 

@@ -552,11 +552,11 @@
 }
 
 bool G1CollectedHeap::check_archive_addresses(MemRegion* ranges, size_t count) {
   assert(ranges != NULL, "MemRegion array NULL");
   assert(count != 0, "No MemRegions provided");
-  MemRegion reserved = _hrm.reserved();
+  MemRegion reserved = _hrm->reserved();
   for (size_t i = 0; i < count; i++) {
     if (!reserved.contains(ranges[i].start()) || !reserved.contains(ranges[i].last())) {
       return false;
     }
   }

@@ -569,11 +569,11 @@
   assert(!is_init_completed(), "Expect to be called at JVM init time");
   assert(ranges != NULL, "MemRegion array NULL");
   assert(count != 0, "No MemRegions provided");
   MutexLockerEx x(Heap_lock);
 
-  MemRegion reserved = _hrm.reserved();
+  MemRegion reserved = _hrm->reserved();
   HeapWord* prev_last_addr = NULL;
   HeapRegion* prev_last_region = NULL;
 
   // Temporarily disable pretouching of heap pages. This interface is used
   // when mmap'ing archived heap data in, so pre-touching is wasted.

@@ -603,26 +603,26 @@
 
     // Check for ranges that start in the same G1 region in which the previous
     // range ended, and adjust the start address so we don't try to allocate
     // the same region again. If the current range is entirely within that
     // region, skip it, just adjusting the recorded top.
-    HeapRegion* start_region = _hrm.addr_to_region(start_address);
+    HeapRegion* start_region = _hrm->addr_to_region(start_address);
     if ((prev_last_region != NULL) && (start_region == prev_last_region)) {
       start_address = start_region->end();
       if (start_address > last_address) {
         increase_used(word_size * HeapWordSize);
         start_region->set_top(last_address + 1);
         continue;
       }
       start_region->set_top(start_address);
       curr_range = MemRegion(start_address, last_address + 1);
-      start_region = _hrm.addr_to_region(start_address);
+      start_region = _hrm->addr_to_region(start_address);
     }
 
     // Perform the actual region allocation, exiting if it fails.
     // Then note how much new space we have allocated.
-    if (!_hrm.allocate_containing_regions(curr_range, &commits, workers())) {
+    if (!_hrm->allocate_containing_regions(curr_range, &commits, workers())) {
       return false;
     }
     increase_used(word_size * HeapWordSize);
     if (commits != 0) {
       log_debug(gc, ergo, heap)("Attempt heap expansion (allocate archive regions). Total size: " SIZE_FORMAT "B",

@@ -630,12 +630,12 @@
 
     }
 
     // Mark each G1 region touched by the range as archive, add it to
     // the old set, and set top.
-    HeapRegion* curr_region = _hrm.addr_to_region(start_address);
-    HeapRegion* last_region = _hrm.addr_to_region(last_address);
+    HeapRegion* curr_region = _hrm->addr_to_region(start_address);
+    HeapRegion* last_region = _hrm->addr_to_region(last_address);
     prev_last_region = last_region;
 
     while (curr_region != NULL) {
       assert(curr_region->is_empty() && !curr_region->is_pinned(),
              "Region already in use (index %u)", curr_region->hrm_index());

@@ -648,11 +648,11 @@
       _archive_set.add(curr_region);
       HeapWord* top;
       HeapRegion* next_region;
       if (curr_region != last_region) {
         top = curr_region->end();
-        next_region = _hrm.next_region_in_heap(curr_region);
+        next_region = _hrm->next_region_in_heap(curr_region);
       } else {
         top = last_address + 1;
         next_region = NULL;
       }
       curr_region->set_top(top);

@@ -669,11 +669,11 @@
 
 void G1CollectedHeap::fill_archive_regions(MemRegion* ranges, size_t count) {
   assert(!is_init_completed(), "Expect to be called at JVM init time");
   assert(ranges != NULL, "MemRegion array NULL");
   assert(count != 0, "No MemRegions provided");
-  MemRegion reserved = _hrm.reserved();
+  MemRegion reserved = _hrm->reserved();
   HeapWord *prev_last_addr = NULL;
   HeapRegion* prev_last_region = NULL;
 
   // For each MemRegion, create filler objects, if needed, in the G1 regions
   // that contain the address range. The address range actually within the

@@ -689,12 +689,12 @@
            p2i(start_address), p2i(last_address));
     assert(start_address > prev_last_addr,
            "Ranges not in ascending order: " PTR_FORMAT " <= " PTR_FORMAT ,
            p2i(start_address), p2i(prev_last_addr));
 
-    HeapRegion* start_region = _hrm.addr_to_region(start_address);
-    HeapRegion* last_region = _hrm.addr_to_region(last_address);
+    HeapRegion* start_region = _hrm->addr_to_region(start_address);
+    HeapRegion* last_region = _hrm->addr_to_region(last_address);
     HeapWord* bottom_address = start_region->bottom();
 
     // Check for a range beginning in the same region in which the
     // previous one ended.
     if (start_region == prev_last_region) {

@@ -706,11 +706,11 @@
     HeapRegion* curr_region = start_region;
     while (curr_region != NULL) {
       guarantee(curr_region->is_archive(),
                 "Expected archive region at index %u", curr_region->hrm_index());
       if (curr_region != last_region) {
-        curr_region = _hrm.next_region_in_heap(curr_region);
+        curr_region = _hrm->next_region_in_heap(curr_region);
       } else {
         curr_region = NULL;
       }
     }
 

@@ -755,11 +755,11 @@
 
 void G1CollectedHeap::dealloc_archive_regions(MemRegion* ranges, size_t count) {
   assert(!is_init_completed(), "Expect to be called at JVM init time");
   assert(ranges != NULL, "MemRegion array NULL");
   assert(count != 0, "No MemRegions provided");
-  MemRegion reserved = _hrm.reserved();
+  MemRegion reserved = _hrm->reserved();
   HeapWord* prev_last_addr = NULL;
   HeapRegion* prev_last_region = NULL;
   size_t size_used = 0;
   size_t uncommitted_regions = 0;
 

@@ -777,23 +777,23 @@
            "Ranges not in ascending order: " PTR_FORMAT " <= " PTR_FORMAT ,
            p2i(start_address), p2i(prev_last_addr));
     size_used += ranges[i].byte_size();
     prev_last_addr = last_address;
 
-    HeapRegion* start_region = _hrm.addr_to_region(start_address);
-    HeapRegion* last_region = _hrm.addr_to_region(last_address);
+    HeapRegion* start_region = _hrm->addr_to_region(start_address);
+    HeapRegion* last_region = _hrm->addr_to_region(last_address);
 
     // Check for ranges that start in the same G1 region in which the previous
     // range ended, and adjust the start address so we don't try to free
     // the same region again. If the current range is entirely within that
     // region, skip it.
     if (start_region == prev_last_region) {
       start_address = start_region->end();
       if (start_address > last_address) {
         continue;
       }
-      start_region = _hrm.addr_to_region(start_address);
+      start_region = _hrm->addr_to_region(start_address);
     }
     prev_last_region = last_region;
 
     // After verifying that each region was marked as an archive region by
     // alloc_archive_regions, set it free and empty and uncommit it.

@@ -804,15 +804,15 @@
       uint curr_index = curr_region->hrm_index();
       _archive_set.remove(curr_region);
       curr_region->set_free();
       curr_region->set_top(curr_region->bottom());
       if (curr_region != last_region) {
-        curr_region = _hrm.next_region_in_heap(curr_region);
+        curr_region = _hrm->next_region_in_heap(curr_region);
       } else {
         curr_region = NULL;
       }
-      _hrm.shrink_at(curr_index, 1);
+      _hrm->shrink_at(curr_index, 1);
       uncommitted_regions++;
     }
 
     // Notify mark-sweep that this is no longer an archive range.
     G1ArchiveAllocator::set_range_archive(ranges[i], false);

@@ -1023,10 +1023,12 @@
   // incremental collection set and then start rebuilding it afresh
   // after this full GC.
   abandon_collection_set(collection_set());
 
   tear_down_region_sets(false /* free_list_only */);
+
+  hrm()->prepare_for_full_collection_start();
 }
 
 void G1CollectedHeap::verify_before_full_collection(bool explicit_gc) {
   assert(!GCCause::is_user_requested_gc(gc_cause()) || explicit_gc, "invariant");
   assert(used() == recalculate_used(), "Should be equal");

@@ -1034,10 +1036,12 @@
   _verifier->verify_before_gc(G1HeapVerifier::G1VerifyFull);
   _verifier->check_bitmaps("Full GC Start");
 }
 
 void G1CollectedHeap::prepare_heap_for_mutators() {
+  hrm()->prepare_for_full_collection_end();
+
   // Delete metaspaces for unloaded class loaders and clean up loader_data graph
   ClassLoaderDataGraph::purge();
   MetaspaceUtils::verify_metrics();
 
   // Prepare heap for normal collections.

@@ -1070,11 +1074,11 @@
   G1BarrierSet::dirty_card_queue_set().abandon_logs();
   assert(dirty_card_queue_set().completed_buffers_num() == 0, "DCQS should be empty");
 }
 
 void G1CollectedHeap::verify_after_full_collection() {
-  _hrm.verify_optional();
+  _hrm->verify_optional();
   _verifier->verify_region_sets_optional();
   _verifier->verify_after_gc(G1HeapVerifier::G1VerifyFull);
   // Clear the previous marking bitmap, if needed for bitmap verification.
   // Note we cannot do this when we clear the next marking bitmap in
   // G1ConcurrentMark::abort() above since VerifyDuringGC verifies the

@@ -1322,11 +1326,11 @@
   log_debug(gc, ergo, heap)("Attempt heap expansion (allocation request failed). Allocation request: " SIZE_FORMAT "B",
                             word_size * HeapWordSize);
 
 
   if (expand(expand_bytes, _workers)) {
-    _hrm.verify_optional();
+    _hrm->verify_optional();
     _verifier->verify_region_sets_optional();
     return attempt_allocation_at_safepoint(word_size,
                                            false /* expect_null_mutator_alloc_region */);
   }
   return NULL;

@@ -1347,11 +1351,11 @@
 
   double expand_heap_start_time_sec = os::elapsedTime();
   uint regions_to_expand = (uint)(aligned_expand_bytes / HeapRegion::GrainBytes);
   assert(regions_to_expand > 0, "Must expand by at least one region");
 
-  uint expanded_by = _hrm.expand_by(regions_to_expand, pretouch_workers);
+  uint expanded_by = _hrm->expand_by(regions_to_expand, pretouch_workers);
   if (expand_time_ms != NULL) {
     *expand_time_ms = (os::elapsedTime() - expand_heap_start_time_sec) * MILLIUNITS;
   }
 
   if (expanded_by > 0) {

@@ -1362,11 +1366,11 @@
     log_debug(gc, ergo, heap)("Did not expand the heap (heap expansion operation failed)");
 
     // The expansion of the virtual storage space was unsuccessful.
     // Let's see if it was because we ran out of swap.
     if (G1ExitOnExpansionFailure &&
-        _hrm.available() >= regions_to_expand) {
+        _hrm->available() >= regions_to_expand) {
       // We had head room...
       vm_exit_out_of_memory(aligned_expand_bytes, OOM_MMAP_ERROR, "G1 heap expansion");
     }
   }
   return regions_to_expand > 0;

@@ -1377,11 +1381,11 @@
     ReservedSpace::page_align_size_down(shrink_bytes);
   aligned_shrink_bytes = align_down(aligned_shrink_bytes,
                                          HeapRegion::GrainBytes);
   uint num_regions_to_remove = (uint)(shrink_bytes / HeapRegion::GrainBytes);
 
-  uint num_regions_removed = _hrm.shrink_by(num_regions_to_remove);
+  uint num_regions_removed = _hrm->shrink_by(num_regions_to_remove);
   size_t shrunk_bytes = num_regions_removed * HeapRegion::GrainBytes;
 
 
   log_debug(gc, ergo, heap)("Shrink the heap. requested shrinking amount: " SIZE_FORMAT "B aligned shrinking amount: " SIZE_FORMAT "B attempted shrinking amount: " SIZE_FORMAT "B",
                             shrink_bytes, aligned_shrink_bytes, shrunk_bytes);

@@ -1405,11 +1409,11 @@
   // remove only the ones that we need to remove.
   tear_down_region_sets(true /* free_list_only */);
   shrink_helper(shrink_bytes);
   rebuild_region_sets(true /* free_list_only */);
 
-  _hrm.verify_optional();
+  _hrm->verify_optional();
   _verifier->verify_region_sets_optional();
 }
 
 class OldRegionSetChecker : public HeapRegionSetChecker {
 public:

@@ -1483,11 +1487,12 @@
   _old_set("Old Region Set", new OldRegionSetChecker()),
   _archive_set("Archive Region Set", new ArchiveRegionSetChecker()),
   _humongous_set("Humongous Region Set", new HumongousRegionSetChecker()),
   _bot(NULL),
   _listener(),
-  _hrm(),
+  _hrm(NULL),
+  _is_hetero_heap(AllocateOldGenAt != NULL),
   _allocator(NULL),
   _verifier(NULL),
   _summary_bytes_used(0),
   _archive_allocator(NULL),
   _survivor_evac_stats("Young", YoungPLABSize, PLABWeight),

@@ -1616,11 +1621,11 @@
   // cases incorrectly returns the size in wordSize units rather than
   // HeapWordSize).
   guarantee(HeapWordSize == wordSize, "HeapWordSize must equal wordSize");
 
   size_t init_byte_size = collector_policy()->initial_heap_byte_size();
-  size_t max_byte_size = collector_policy()->max_heap_byte_size();
+  size_t max_byte_size = g1_collector_policy()->heap_reservation_size_bytes();
   size_t heap_alignment = collector_policy()->heap_alignment();
 
   // Ensure that the sizes are properly aligned.
   Universe::check_alignment(init_byte_size, HeapRegion::GrainBytes, "g1 heap");
   Universe::check_alignment(max_byte_size, HeapRegion::GrainBytes, "g1 heap");

@@ -1680,16 +1685,21 @@
 
   // Carve out the G1 part of the heap.
   ReservedSpace g1_rs = heap_rs.first_part(max_byte_size);
   size_t page_size = UseLargePages ? os::large_page_size() : os::vm_page_size();
   G1RegionToSpaceMapper* heap_storage =
-    G1RegionToSpaceMapper::create_mapper(g1_rs,
+    G1RegionToSpaceMapper::create_heap_mapper(g1_rs,
                                          g1_rs.size(),
                                          page_size,
                                          HeapRegion::GrainBytes,
                                          1,
                                          mtJavaHeap);
+  if(heap_storage == NULL) {
+    vm_shutdown_during_initialization("Could not initialize G1 heap");
+    return JNI_ERR;
+  }
+
   os::trace_page_sizes("Heap",
                        collector_policy()->min_heap_byte_size(),
                        max_byte_size,
                        page_size,
                        heap_rs.base(),

@@ -1716,11 +1726,13 @@
   G1RegionToSpaceMapper* prev_bitmap_storage =
     create_aux_memory_mapper("Prev Bitmap", bitmap_size, G1CMBitMap::heap_map_factor());
   G1RegionToSpaceMapper* next_bitmap_storage =
     create_aux_memory_mapper("Next Bitmap", bitmap_size, G1CMBitMap::heap_map_factor());
 
-  _hrm.initialize(heap_storage, prev_bitmap_storage, next_bitmap_storage, bot_storage, cardtable_storage, card_counts_storage);
+  _hrm = HeapRegionManager::create_manager(this, collector_policy());
+
+  _hrm->initialize(heap_storage, prev_bitmap_storage, next_bitmap_storage, bot_storage, cardtable_storage, card_counts_storage);
   _card_table->initialize(cardtable_storage);
   // Do later initialization work for concurrent refinement.
   _hot_card_cache->initialize(card_counts_storage);
 
   // 6843694 - ensure that the maximum region index can fit

@@ -1731,24 +1743,24 @@
   // The G1FromCardCache reserves card with value 0 as "invalid", so the heap must not
   // start within the first card.
   guarantee(g1_rs.base() >= (char*)G1CardTable::card_size, "Java heap must not start within the first card.");
   // Also create a G1 rem set.
   _g1_rem_set = new G1RemSet(this, _card_table, _hot_card_cache);
-  _g1_rem_set->initialize(max_capacity(), max_regions());
+  _g1_rem_set->initialize(max_reserved_capacity(), max_regions());
 
   size_t max_cards_per_region = ((size_t)1 << (sizeof(CardIdx_t)*BitsPerByte-1)) - 1;
   guarantee(HeapRegion::CardsPerRegion > 0, "make sure it's initialized");
   guarantee(HeapRegion::CardsPerRegion < max_cards_per_region,
             "too many cards per region");
 
-  FreeRegionList::set_unrealistically_long_length(max_regions() + 1);
+  FreeRegionList::set_unrealistically_long_length(max_expandable_regions() + 1);
 
   _bot = new G1BlockOffsetTable(reserved_region(), bot_storage);
 
   {
-    HeapWord* start = _hrm.reserved().start();
-    HeapWord* end = _hrm.reserved().end();
+    HeapWord* start = _hrm->reserved().start();
+    HeapWord* end = _hrm->reserved().end();
     size_t granularity = HeapRegion::GrainBytes;
 
     _in_cset_fast_test.initialize(start, end, granularity);
     _humongous_reclaim_candidates.initialize(start, end, granularity);
   }

@@ -1768,10 +1780,14 @@
     return JNI_ENOMEM;
   }
 
   // Perform any initialization actions delegated to the policy.
   g1_policy()->init(this, &_collection_set);
+  // Now we know the target length of young list. So adjust the heap to provision that many regions on dram.
+  if (is_hetero_heap()) {
+    static_cast<HeterogeneousHeapRegionManager*>(hrm())->adjust_dram_regions((uint)g1_policy()->young_list_target_length(), workers());
+  }
 
   jint ecode = initialize_concurrent_refinement();
   if (ecode != JNI_OK) {
     return ecode;
   }

@@ -1787,11 +1803,11 @@
     dcqs.set_max_completed_queue((int)concurrent_refine()->red_zone());
   }
 
   // Here we allocate the dummy HeapRegion that is required by the
   // G1AllocRegion class.
-  HeapRegion* dummy_region = _hrm.get_dummy_region();
+  HeapRegion* dummy_region = _hrm->get_dummy_region();
 
   // We'll re-use the same region whether the alloc region will
   // require BOT updates or not and, if it doesn't, then a non-young
   // region will complain that it cannot support allocations without
   // BOT updates. So we'll tag the dummy region as eden to avoid that.

@@ -1907,20 +1923,24 @@
 
 CollectorPolicy* G1CollectedHeap::collector_policy() const {
   return _collector_policy;
 }
 
+G1CollectorPolicy* G1CollectedHeap::g1_collector_policy() const {
+  return _collector_policy;
+}
+
 SoftRefPolicy* G1CollectedHeap::soft_ref_policy() {
   return &_soft_ref_policy;
 }
 
 size_t G1CollectedHeap::capacity() const {
-  return _hrm.length() * HeapRegion::GrainBytes;
+  return _hrm->length() * HeapRegion::GrainBytes;
 }
 
 size_t G1CollectedHeap::unused_committed_regions_in_bytes() const {
-  return _hrm.total_free_bytes();
+  return _hrm->total_free_bytes();
 }
 
 void G1CollectedHeap::iterate_hcc_closure(CardTableEntryClosure* cl, uint worker_i) {
   _hot_card_cache->drain(cl, worker_i);
 }

@@ -2131,11 +2151,11 @@
     }
   } while (retry_gc);
 }
 
 bool G1CollectedHeap::is_in(const void* p) const {
-  if (_hrm.reserved().contains(p)) {
+  if (_hrm->reserved().contains(p)) {
     // Given that we know that p is in the reserved space,
     // heap_region_containing() should successfully
     // return the containing region.
     HeapRegion* hr = heap_region_containing(p);
     return hr->is_in(p);

@@ -2145,11 +2165,11 @@
 }
 
 #ifdef ASSERT
 bool G1CollectedHeap::is_in_exact(const void* p) const {
   bool contains = reserved_region().contains(p);
-  bool available = _hrm.is_available(addr_to_region((HeapWord*)p));
+  bool available = _hrm->is_available(addr_to_region((HeapWord*)p));
   if (contains && available) {
     return true;
   } else {
     return false;
   }

@@ -2176,22 +2196,22 @@
   IterateObjectClosureRegionClosure blk(cl);
   heap_region_iterate(&blk);
 }
 
 void G1CollectedHeap::heap_region_iterate(HeapRegionClosure* cl) const {
-  _hrm.iterate(cl);
+  _hrm->iterate(cl);
 }
 
 void G1CollectedHeap::heap_region_par_iterate_from_worker_offset(HeapRegionClosure* cl,
                                                                  HeapRegionClaimer *hrclaimer,
                                                                  uint worker_id) const {
-  _hrm.par_iterate(cl, hrclaimer, hrclaimer->offset_for_worker(worker_id));
+  _hrm->par_iterate(cl, hrclaimer, hrclaimer->offset_for_worker(worker_id));
 }
 
 void G1CollectedHeap::heap_region_par_iterate_from_start(HeapRegionClosure* cl,
                                                          HeapRegionClaimer *hrclaimer) const {
-  _hrm.par_iterate(cl, hrclaimer, 0);
+  _hrm->par_iterate(cl, hrclaimer, 0);
 }
 
 void G1CollectedHeap::collection_set_iterate(HeapRegionClosure* cl) {
   _collection_set.iterate(cl);
 }

@@ -2236,11 +2256,15 @@
 size_t G1CollectedHeap::unsafe_max_tlab_alloc(Thread* ignored) const {
   return _allocator->unsafe_max_tlab_alloc();
 }
 
 size_t G1CollectedHeap::max_capacity() const {
-  return _hrm.reserved().byte_size();
+  return _hrm->max_expandable_length() * HeapRegion::GrainBytes;
+}
+
+size_t G1CollectedHeap::max_reserved_capacity() const {
+  return _hrm->max_length() * HeapRegion::GrainBytes;
 }
 
 jlong G1CollectedHeap::millis_since_last_gc() {
   // See the notes in GenCollectedHeap::millis_since_last_gc()
   // for more information about the implementation.

@@ -2326,12 +2350,12 @@
 void G1CollectedHeap::print_on(outputStream* st) const {
   st->print(" %-20s", "garbage-first heap");
   st->print(" total " SIZE_FORMAT "K, used " SIZE_FORMAT "K",
             capacity()/K, used_unlocked()/K);
   st->print(" [" PTR_FORMAT ", " PTR_FORMAT ")",
-            p2i(_hrm.reserved().start()),
-            p2i(_hrm.reserved().end()));
+            p2i(_hrm->reserved().start()),
+            p2i(_hrm->reserved().end()));
   st->cr();
   st->print("  region size " SIZE_FORMAT "K, ", HeapRegion::GrainBytes / K);
   uint young_regions = young_regions_count();
   st->print("%u young (" SIZE_FORMAT "K), ", young_regions,
             (size_t) young_regions * HeapRegion::GrainBytes / K);

@@ -2502,10 +2526,14 @@
 void G1CollectedHeap::gc_epilogue(bool full) {
   // Update common counters.
   if (full) {
     // Update the number of full collections that have been completed.
     increment_old_marking_cycles_completed(false /* concurrent */);
+    // Now we know the target length of young list. So adjust the heap to provision that many regions on dram.
+    if (is_hetero_heap()) {
+      static_cast<HeterogeneousHeapRegionManager*>(hrm())->adjust_dram_regions((uint)g1_policy()->young_list_target_length(), workers());
+    }
   }
 
   // We are at the end of the GC. Total collections has already been increased.
   g1_rem_set()->print_periodic_summary_info("After GC RS summary", total_collections() - 1);
 

@@ -3113,11 +3141,11 @@
     // It is not yet to safe to tell the concurrent mark to
     // start as we have some optional output below. We don't want the
     // output from the concurrent mark thread interfering with this
     // logging output either.
 
-    _hrm.verify_optional();
+    _hrm->verify_optional();
     _verifier->verify_region_sets_optional();
 
     TASKQUEUE_STATS_ONLY(print_taskqueue_stats());
     TASKQUEUE_STATS_ONLY(reset_taskqueue_stats());
 

@@ -3790,11 +3818,11 @@
                                   bool skip_remset,
                                   bool skip_hot_card_cache,
                                   bool locked) {
   assert(!hr->is_free(), "the region should not be free");
   assert(!hr->is_empty(), "the region should not be empty");
-  assert(_hrm.is_available(hr->hrm_index()), "region should be committed");
+  assert(_hrm->is_available(hr->hrm_index()), "region should be committed");
   assert(free_list != NULL, "pre-condition");
 
   if (G1VerifyBitmaps) {
     MemRegion mr(hr->bottom(), hr->end());
     concurrent_mark()->clear_range_in_prev_bitmap(mr);

@@ -3831,11 +3859,11 @@
 
 void G1CollectedHeap::prepend_to_freelist(FreeRegionList* list) {
   assert(list != NULL, "list can't be null");
   if (!list->is_empty()) {
     MutexLockerEx x(FreeList_lock, Mutex::_no_safepoint_check_flag);
-    _hrm.insert_list_into_free_list(list);
+    _hrm->insert_list_into_free_list(list);
   }
 }
 
 void G1CollectedHeap::decrement_summary_bytes(size_t bytes) {
   decrease_used(bytes);

@@ -3911,11 +3939,19 @@
         // still in the region and the ones already moved are accounted for elsewhere.
         if (r->is_young()) {
           _bytes_allocated_in_old_since_last_gc += HeapRegion::GrainBytes;
         }
         // The region is now considered to be old.
+        if(g1h->is_hetero_heap()) {
+          if(!r->is_old()) {
+            // The region was young before, set it as pre-matured old so that next mixed gc can move
+            // its contents to old region which is on nv-dimm
+            r->set_premature_old();
+          }
+        } else {
         r->set_old();
+        }
         // Do some allocation statistics accounting. Regions that failed evacuation
         // are always made old, so there is no need to update anything in the young
         // gen statistics, but we need to update old gen statistics.
         size_t used_words = r->marked_bytes() / HeapWordSize;
 

@@ -4364,11 +4400,11 @@
     // Note that emptying the _young_list is postponed and instead done as
     // the first step when rebuilding the regions sets again. The reason for
     // this is that during a full GC string deduplication needs to know if
     // a collected region was young or old when the full GC was initiated.
   }
-  _hrm.remove_all_free_regions();
+  _hrm->remove_all_free_regions();
 }
 
 void G1CollectedHeap::increase_used(size_t bytes) {
   _summary_bytes_used += bytes;
 }

@@ -4439,11 +4475,11 @@
   if (!free_list_only) {
     _eden.clear();
     _survivor.clear();
   }
 
-  RebuildRegionSetsClosure cl(free_list_only, &_old_set, &_hrm);
+  RebuildRegionSetsClosure cl(free_list_only, &_old_set, _hrm);
   heap_region_iterate(&cl);
 
   if (!free_list_only) {
     set_used(cl.total_used());
     if (_archive_allocator != NULL) {

@@ -4547,18 +4583,18 @@
   _hr_printer.retire(alloc_region);
 }
 
 HeapRegion* G1CollectedHeap::alloc_highest_free_region() {
   bool expanded = false;
-  uint index = _hrm.find_highest_free(&expanded);
+  uint index = _hrm->find_highest_free(&expanded);
 
   if (index != G1_NO_HRM_INDEX) {
     if (expanded) {
       log_debug(gc, ergo, heap)("Attempt heap expansion (requested address range outside heap bounds). region size: " SIZE_FORMAT "B",
                                 HeapRegion::GrainWords * HeapWordSize);
     }
-    _hrm.allocate_free_regions_starting_at(index, 1);
+    _hrm->allocate_free_regions_starting_at(index, 1);
     return region_at(index);
   }
   return NULL;
 }
 
< prev index next >