aboutsummaryrefslogtreecommitdiff
path: root/src/share/vm/gc_implementation
diff options
context:
space:
mode:
Diffstat (limited to 'src/share/vm/gc_implementation')
-rw-r--r--src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.cpp21
-rw-r--r--src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.hpp8
-rw-r--r--src/share/vm/gc_implementation/g1/concurrentG1Refine.cpp436
-rw-r--r--src/share/vm/gc_implementation/g1/concurrentG1Refine.hpp171
-rw-r--r--src/share/vm/gc_implementation/g1/g1CardCounts.cpp212
-rw-r--r--src/share/vm/gc_implementation/g1/g1CardCounts.hpp126
-rw-r--r--src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp87
-rw-r--r--src/share/vm/gc_implementation/g1/g1CollectorPolicy.cpp3
-rw-r--r--src/share/vm/gc_implementation/g1/g1GCPhaseTimes.cpp42
-rw-r--r--src/share/vm/gc_implementation/g1/g1GCPhaseTimes.hpp11
-rw-r--r--src/share/vm/gc_implementation/g1/g1HotCardCache.cpp148
-rw-r--r--src/share/vm/gc_implementation/g1/g1HotCardCache.hpp128
-rw-r--r--src/share/vm/gc_implementation/g1/g1RemSet.cpp246
-rw-r--r--src/share/vm/gc_implementation/g1/g1RemSet.hpp22
-rw-r--r--src/share/vm/gc_implementation/g1/g1_globals.hpp12
-rw-r--r--src/share/vm/gc_implementation/g1/heapRegionSeq.cpp46
-rw-r--r--src/share/vm/gc_implementation/g1/heapRegionSeq.hpp25
17 files changed, 854 insertions, 890 deletions
diff --git a/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.cpp b/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.cpp
index ffb7531ca..0f92b3602 100644
--- a/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.cpp
+++ b/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.cpp
@@ -193,7 +193,8 @@ ConcurrentMarkSweepGeneration::ConcurrentMarkSweepGeneration(
FreeBlockDictionary<FreeChunk>::DictionaryChoice dictionaryChoice) :
CardGeneration(rs, initial_byte_size, level, ct),
_dilatation_factor(((double)MinChunkSize)/((double)(CollectedHeap::min_fill_size()))),
- _debug_collection_type(Concurrent_collection_type)
+ _debug_collection_type(Concurrent_collection_type),
+ _did_compact(false)
{
HeapWord* bottom = (HeapWord*) _virtual_space.low();
HeapWord* end = (HeapWord*) _virtual_space.high();
@@ -917,18 +918,15 @@ void ConcurrentMarkSweepGeneration::compute_new_size() {
return;
}
- // Compute some numbers about the state of the heap.
- const size_t used_after_gc = used();
- const size_t capacity_after_gc = capacity();
+ // The heap has been compacted but not reset yet.
+ // Any metric such as free() or used() will be incorrect.
CardGeneration::compute_new_size();
// Reset again after a possible resizing
- cmsSpace()->reset_after_compaction();
-
- assert(used() == used_after_gc && used_after_gc <= capacity(),
- err_msg("used: " SIZE_FORMAT " used_after_gc: " SIZE_FORMAT
- " capacity: " SIZE_FORMAT, used(), used_after_gc, capacity()));
+ if (did_compact()) {
+ cmsSpace()->reset_after_compaction();
+ }
}
void ConcurrentMarkSweepGeneration::compute_new_size_free_list() {
@@ -1578,6 +1576,8 @@ bool CMSCollector::shouldConcurrentCollect() {
return false;
}
+void CMSCollector::set_did_compact(bool v) { _cmsGen->set_did_compact(v); }
+
// Clear _expansion_cause fields of constituent generations
void CMSCollector::clear_expansion_cause() {
_cmsGen->clear_expansion_cause();
@@ -1675,7 +1675,6 @@ void CMSCollector::collect(bool full,
}
acquire_control_and_collect(full, clear_all_soft_refs);
_full_gcs_since_conc_gc++;
-
}
void CMSCollector::request_full_gc(unsigned int full_gc_count) {
@@ -1857,6 +1856,7 @@ NOT_PRODUCT(
}
}
+ set_did_compact(should_compact);
if (should_compact) {
// If the collection is being acquired from the background
// collector, there may be references on the discovered
@@ -2718,6 +2718,7 @@ void CMSCollector::gc_epilogue(bool full) {
Chunk::clean_chunk_pool();
}
+ set_did_compact(false);
_between_prologue_and_epilogue = false; // ready for next cycle
}
diff --git a/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.hpp b/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.hpp
index 7040f218e..e930b5863 100644
--- a/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.hpp
+++ b/src/share/vm/gc_implementation/concurrentMarkSweep/concurrentMarkSweepGeneration.hpp
@@ -604,6 +604,8 @@ class CMSCollector: public CHeapObj<mtGC> {
ConcurrentMarkSweepPolicy* _collector_policy;
ConcurrentMarkSweepPolicy* collector_policy() { return _collector_policy; }
+ void set_did_compact(bool v);
+
// XXX Move these to CMSStats ??? FIX ME !!!
elapsedTimer _inter_sweep_timer; // time between sweeps
elapsedTimer _intra_sweep_timer; // time _in_ sweeps
@@ -1081,6 +1083,10 @@ class ConcurrentMarkSweepGeneration: public CardGeneration {
CollectionTypes _debug_collection_type;
+ // True if a compactiing collection was done.
+ bool _did_compact;
+ bool did_compact() { return _did_compact; }
+
// Fraction of current occupancy at which to start a CMS collection which
// will collect this generation (at least).
double _initiating_occupancy;
@@ -1121,6 +1127,8 @@ class ConcurrentMarkSweepGeneration: public CardGeneration {
// Adaptive size policy
CMSAdaptiveSizePolicy* size_policy();
+ void set_did_compact(bool v) { _did_compact = v; }
+
bool refs_discovery_is_atomic() const { return false; }
bool refs_discovery_is_mt() const {
// Note: CMS does MT-discovery during the parallel-remark
diff --git a/src/share/vm/gc_implementation/g1/concurrentG1Refine.cpp b/src/share/vm/gc_implementation/g1/concurrentG1Refine.cpp
index 3eab85e13..5f049209e 100644
--- a/src/share/vm/gc_implementation/g1/concurrentG1Refine.cpp
+++ b/src/share/vm/gc_implementation/g1/concurrentG1Refine.cpp
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2001, 2012, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2001, 2013, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@@ -26,40 +26,12 @@
#include "gc_implementation/g1/concurrentG1Refine.hpp"
#include "gc_implementation/g1/concurrentG1RefineThread.hpp"
#include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
-#include "gc_implementation/g1/g1CollectorPolicy.hpp"
-#include "gc_implementation/g1/g1GCPhaseTimes.hpp"
-#include "gc_implementation/g1/g1RemSet.hpp"
-#include "gc_implementation/g1/heapRegionSeq.inline.hpp"
-#include "memory/space.inline.hpp"
-#include "runtime/atomic.hpp"
-#include "runtime/java.hpp"
-#include "utilities/copy.hpp"
+#include "gc_implementation/g1/g1HotCardCache.hpp"
-// Possible sizes for the card counts cache: odd primes that roughly double in size.
-// (See jvmtiTagMap.cpp).
-
-#define MAX_SIZE ((size_t) -1)
-
-size_t ConcurrentG1Refine::_cc_cache_sizes[] = {
- 16381, 32771, 76831, 150001, 307261,
- 614563, 1228891, 2457733, 4915219, 9830479,
- 19660831, 39321619, 78643219, 157286461, MAX_SIZE
- };
-
-ConcurrentG1Refine::ConcurrentG1Refine() :
- _card_counts(NULL), _card_epochs(NULL),
- _n_card_counts(0), _max_cards(0), _max_n_card_counts(0),
- _cache_size_index(0), _expand_card_counts(false),
- _hot_cache(NULL),
- _def_use_cache(false), _use_cache(false),
- // We initialize the epochs of the array to 0. By initializing
- // _n_periods to 1 and not 0 we automatically invalidate all the
- // entries on the array. Otherwise we might accidentally think that
- // we claimed a card that was in fact never set (see CR7033292).
- _n_periods(1),
- _threads(NULL), _n_threads(0)
+ConcurrentG1Refine::ConcurrentG1Refine(G1CollectedHeap* g1h) :
+ _threads(NULL), _n_threads(0),
+ _hot_card_cache(g1h)
{
-
// Ergomonically select initial concurrent refinement parameters
if (FLAG_IS_DEFAULT(G1ConcRefinementGreenZone)) {
FLAG_SET_DEFAULT(G1ConcRefinementGreenZone, MAX2<int>(ParallelGCThreads, 1));
@@ -75,13 +47,17 @@ ConcurrentG1Refine::ConcurrentG1Refine() :
FLAG_SET_DEFAULT(G1ConcRefinementRedZone, yellow_zone() * 2);
}
set_red_zone(MAX2<int>(G1ConcRefinementRedZone, yellow_zone()));
+
_n_worker_threads = thread_num();
// We need one extra thread to do the young gen rset size sampling.
_n_threads = _n_worker_threads + 1;
+
reset_threshold_step();
_threads = NEW_C_HEAP_ARRAY(ConcurrentG1RefineThread*, _n_threads, mtGC);
+
int worker_id_offset = (int)DirtyCardQueueSet::num_par_ids();
+
ConcurrentG1RefineThread *next = NULL;
for (int i = _n_threads - 1; i >= 0; i--) {
ConcurrentG1RefineThread* t = new ConcurrentG1RefineThread(this, next, worker_id_offset, i);
@@ -100,74 +76,8 @@ void ConcurrentG1Refine::reset_threshold_step() {
}
}
-int ConcurrentG1Refine::thread_num() {
- return MAX2<int>((G1ConcRefinementThreads > 0) ? G1ConcRefinementThreads : ParallelGCThreads, 1);
-}
-
void ConcurrentG1Refine::init() {
- if (G1ConcRSLogCacheSize > 0) {
- _g1h = G1CollectedHeap::heap();
-
- _max_cards = _g1h->max_capacity() >> CardTableModRefBS::card_shift;
- _max_n_card_counts = _max_cards * G1MaxHotCardCountSizePercent / 100;
-
- size_t max_card_num = ((size_t)1 << (sizeof(unsigned)*BitsPerByte-1)) - 1;
- guarantee(_max_cards < max_card_num, "card_num representation");
-
- // We need _n_card_counts to be less than _max_n_card_counts here
- // so that the expansion call (below) actually allocates the
- // _counts and _epochs arrays.
- assert(_n_card_counts == 0, "pre-condition");
- assert(_max_n_card_counts > 0, "pre-condition");
-
- // Find the index into cache size array that is of a size that's
- // large enough to hold desired_sz.
- size_t desired_sz = _max_cards / InitialCacheFraction;
- int desired_sz_index = 0;
- while (_cc_cache_sizes[desired_sz_index] < desired_sz) {
- desired_sz_index += 1;
- assert(desired_sz_index < MAX_CC_CACHE_INDEX, "invariant");
- }
- assert(desired_sz_index < MAX_CC_CACHE_INDEX, "invariant");
-
- // If the desired_sz value is between two sizes then
- // _cc_cache_sizes[desired_sz_index-1] < desired_sz <= _cc_cache_sizes[desired_sz_index]
- // we will start with the lower size in the optimistic expectation that
- // we will not need to expand up. Note desired_sz_index could also be 0.
- if (desired_sz_index > 0 &&
- _cc_cache_sizes[desired_sz_index] > desired_sz) {
- desired_sz_index -= 1;
- }
-
- if (!expand_card_count_cache(desired_sz_index)) {
- // Allocation was unsuccessful - exit
- vm_exit_during_initialization("Could not reserve enough space for card count cache");
- }
- assert(_n_card_counts > 0, "post-condition");
- assert(_cache_size_index == desired_sz_index, "post-condition");
-
- Copy::fill_to_bytes(&_card_counts[0],
- _n_card_counts * sizeof(CardCountCacheEntry));
- Copy::fill_to_bytes(&_card_epochs[0], _n_card_counts * sizeof(CardEpochCacheEntry));
-
- ModRefBarrierSet* bs = _g1h->mr_bs();
- guarantee(bs->is_a(BarrierSet::CardTableModRef), "Precondition");
- _ct_bs = (CardTableModRefBS*)bs;
- _ct_bot = _ct_bs->byte_for_const(_g1h->reserved_region().start());
-
- _def_use_cache = true;
- _use_cache = true;
- _hot_cache_size = (1 << G1ConcRSLogCacheSize);
- _hot_cache = NEW_C_HEAP_ARRAY(jbyte*, _hot_cache_size, mtGC);
- _n_hot = 0;
- _hot_cache_idx = 0;
-
- // For refining the cards in the hot cache in parallel
- int n_workers = (ParallelGCThreads > 0 ?
- _g1h->workers()->total_workers() : 1);
- _hot_cache_par_chunk_size = MAX2(1, _hot_cache_size / n_workers);
- _hot_cache_par_claimed_idx = 0;
- }
+ _hot_card_cache.initialize();
}
void ConcurrentG1Refine::stop() {
@@ -188,17 +98,6 @@ void ConcurrentG1Refine::reinitialize_threads() {
}
ConcurrentG1Refine::~ConcurrentG1Refine() {
- if (G1ConcRSLogCacheSize > 0) {
- // Please see the comment in allocate_card_count_cache
- // for why we call os::malloc() and os::free() directly.
- assert(_card_counts != NULL, "Logic");
- os::free(_card_counts, mtGC);
- assert(_card_epochs != NULL, "Logic");
- os::free(_card_epochs, mtGC);
-
- assert(_hot_cache != NULL, "Logic");
- FREE_C_HEAP_ARRAY(jbyte*, _hot_cache, mtGC);
- }
if (_threads != NULL) {
for (int i = 0; i < _n_threads; i++) {
delete _threads[i];
@@ -215,317 +114,10 @@ void ConcurrentG1Refine::threads_do(ThreadClosure *tc) {
}
}
-bool ConcurrentG1Refine::is_young_card(jbyte* card_ptr) {
- HeapWord* start = _ct_bs->addr_for(card_ptr);
- HeapRegion* r = _g1h->heap_region_containing(start);
- if (r != NULL && r->is_young()) {
- return true;
- }
- // This card is not associated with a heap region
- // so can't be young.
- return false;
-}
-
-jbyte* ConcurrentG1Refine::add_card_count(jbyte* card_ptr, int* count, bool* defer) {
- unsigned new_card_num = ptr_2_card_num(card_ptr);
- unsigned bucket = hash(new_card_num);
- assert(0 <= bucket && bucket < _n_card_counts, "Bounds");
-
- CardCountCacheEntry* count_ptr = &_card_counts[bucket];
- CardEpochCacheEntry* epoch_ptr = &_card_epochs[bucket];
-
- // We have to construct a new entry if we haven't updated the counts
- // during the current period, or if the count was updated for a
- // different card number.
- unsigned int new_epoch = (unsigned int) _n_periods;
- julong new_epoch_entry = make_epoch_entry(new_card_num, new_epoch);
-
- while (true) {
- // Fetch the previous epoch value
- julong prev_epoch_entry = epoch_ptr->_value;
- julong cas_res;
-
- if (extract_epoch(prev_epoch_entry) != new_epoch) {
- // This entry has not yet been updated during this period.
- // Note: we update the epoch value atomically to ensure
- // that there is only one winner that updates the cached
- // card_ptr value even though all the refine threads share
- // the same epoch value.
-
- cas_res = (julong) Atomic::cmpxchg((jlong) new_epoch_entry,
- (volatile jlong*)&epoch_ptr->_value,
- (jlong) prev_epoch_entry);
-
- if (cas_res == prev_epoch_entry) {
- // We have successfully won the race to update the
- // epoch and card_num value. Make it look like the
- // count and eviction count were previously cleared.
- count_ptr->_count = 1;
- count_ptr->_evict_count = 0;
- *count = 0;
- // We can defer the processing of card_ptr
- *defer = true;
- return card_ptr;
- }
- // We did not win the race to update the epoch field, so some other
- // thread must have done it. The value that gets returned by CAS
- // should be the new epoch value.
- assert(extract_epoch(cas_res) == new_epoch, "unexpected epoch");
- // We could 'continue' here or just re-read the previous epoch value
- prev_epoch_entry = epoch_ptr->_value;
- }
-
- // The epoch entry for card_ptr has been updated during this period.
- unsigned old_card_num = extract_card_num(prev_epoch_entry);
-
- // The card count that will be returned to caller
- *count = count_ptr->_count;
-
- // Are we updating the count for the same card?
- if (new_card_num == old_card_num) {
- // Same card - just update the count. We could have more than one
- // thread racing to update count for the current card. It should be
- // OK not to use a CAS as the only penalty should be some missed
- // increments of the count which delays identifying the card as "hot".
-
- if (*count < max_jubyte) count_ptr->_count++;
- // We can defer the processing of card_ptr
- *defer = true;
- return card_ptr;
- }
-
- // Different card - evict old card info
- if (count_ptr->_evict_count < max_jubyte) count_ptr->_evict_count++;
- if (count_ptr->_evict_count > G1CardCountCacheExpandThreshold) {
- // Trigger a resize the next time we clear
- _expand_card_counts = true;
- }
-
- cas_res = (julong) Atomic::cmpxchg((jlong) new_epoch_entry,
- (volatile jlong*)&epoch_ptr->_value,
- (jlong) prev_epoch_entry);
-
- if (cas_res == prev_epoch_entry) {
- // We successfully updated the card num value in the epoch entry
- count_ptr->_count = 0; // initialize counter for new card num
- jbyte* old_card_ptr = card_num_2_ptr(old_card_num);
-
- // Even though the region containg the card at old_card_num was not
- // in the young list when old_card_num was recorded in the epoch
- // cache it could have been added to the free list and subsequently
- // added to the young list in the intervening time. See CR 6817995.
- // We do not deal with this case here - it will be handled in
- // HeapRegion::oops_on_card_seq_iterate_careful after it has been
- // determined that the region containing the card has been allocated
- // to, and it's safe to check the young type of the region.
-
- // We do not want to defer processing of card_ptr in this case
- // (we need to refine old_card_ptr and card_ptr)
- *defer = false;
- return old_card_ptr;
- }
- // Someone else beat us - try again.
- }
-}
-
-jbyte* ConcurrentG1Refine::cache_insert(jbyte* card_ptr, bool* defer) {
- int count;
- jbyte* cached_ptr = add_card_count(card_ptr, &count, defer);
- assert(cached_ptr != NULL, "bad cached card ptr");
-
- // We've just inserted a card pointer into the card count cache
- // and got back the card that we just inserted or (evicted) the
- // previous contents of that count slot.
-
- // The card we got back could be in a young region. When the
- // returned card (if evicted) was originally inserted, we had
- // determined that its containing region was not young. However
- // it is possible for the region to be freed during a cleanup
- // pause, then reallocated and tagged as young which will result
- // in the returned card residing in a young region.
- //
- // We do not deal with this case here - the change from non-young
- // to young could be observed at any time - it will be handled in
- // HeapRegion::oops_on_card_seq_iterate_careful after it has been
- // determined that the region containing the card has been allocated
- // to.
-
- // The card pointer we obtained from card count cache is not hot
- // so do not store it in the cache; return it for immediate
- // refining.
- if (count < G1ConcRSHotCardLimit) {
- return cached_ptr;
- }
-
- // Otherwise, the pointer we got from the _card_counts cache is hot.
- jbyte* res = NULL;
- MutexLockerEx x(HotCardCache_lock, Mutex::_no_safepoint_check_flag);
- if (_n_hot == _hot_cache_size) {
- res = _hot_cache[_hot_cache_idx];
- _n_hot--;
- }
- // Now _n_hot < _hot_cache_size, and we can insert at _hot_cache_idx.
- _hot_cache[_hot_cache_idx] = cached_ptr;
- _hot_cache_idx++;
- if (_hot_cache_idx == _hot_cache_size) _hot_cache_idx = 0;
- _n_hot++;
-
- // The card obtained from the hot card cache could be in a young
- // region. See above on how this can happen.
-
- return res;
-}
-
-void ConcurrentG1Refine::clean_up_cache(int worker_i,
- G1RemSet* g1rs,
- DirtyCardQueue* into_cset_dcq) {
- assert(!use_cache(), "cache should be disabled");
- int start_idx;
-
- while ((start_idx = _hot_cache_par_claimed_idx) < _n_hot) { // read once
- int end_idx = start_idx + _hot_cache_par_chunk_size;
-
- if (start_idx ==
- Atomic::cmpxchg(end_idx, &_hot_cache_par_claimed_idx, start_idx)) {
- // The current worker has successfully claimed the chunk [start_idx..end_idx)
- end_idx = MIN2(end_idx, _n_hot);
- for (int i = start_idx; i < end_idx; i++) {
- jbyte* entry = _hot_cache[i];
- if (entry != NULL) {
- if (g1rs->concurrentRefineOneCard(entry, worker_i, true)) {
- // 'entry' contains references that point into the current
- // collection set. We need to record 'entry' in the DCQS
- // that's used for that purpose.
- //
- // The only time we care about recording cards that contain
- // references that point into the collection set is during
- // RSet updating while within an evacuation pause.
- // In this case worker_i should be the id of a GC worker thread
- assert(SafepointSynchronize::is_at_safepoint(), "not during an evacuation pause");
- assert(worker_i < (int) (ParallelGCThreads == 0 ? 1 : ParallelGCThreads), "incorrect worker id");
- into_cset_dcq->enqueue(entry);
- }
- }
- }
- }
- }
-}
-
-// The arrays used to hold the card counts and the epochs must have
-// a 1:1 correspondence. Hence they are allocated and freed together
-// Returns true if the allocations of both the counts and epochs
-// were successful; false otherwise.
-bool ConcurrentG1Refine::allocate_card_count_cache(size_t n,
- CardCountCacheEntry** counts,
- CardEpochCacheEntry** epochs) {
- // We call the allocation/free routines directly for the counts
- // and epochs arrays. The NEW_C_HEAP_ARRAY/FREE_C_HEAP_ARRAY
- // macros call AllocateHeap and FreeHeap respectively.
- // AllocateHeap will call vm_exit_out_of_memory in the event
- // of an allocation failure and abort the JVM. With the
- // _counts/epochs arrays we only need to abort the JVM if the
- // initial allocation of these arrays fails.
- //
- // Additionally AllocateHeap/FreeHeap do some tracing of
- // allocate/free calls so calling one without calling the
- // other can cause inconsistencies in the tracing. So we
- // call neither.
-
- assert(*counts == NULL, "out param");
- assert(*epochs == NULL, "out param");
-
- size_t counts_size = n * sizeof(CardCountCacheEntry);
- size_t epochs_size = n * sizeof(CardEpochCacheEntry);
-
- *counts = (CardCountCacheEntry*) os::malloc(counts_size, mtGC);
- if (*counts == NULL) {
- // allocation was unsuccessful
- return false;
- }
-
- *epochs = (CardEpochCacheEntry*) os::malloc(epochs_size, mtGC);
- if (*epochs == NULL) {
- // allocation was unsuccessful - free counts array
- assert(*counts != NULL, "must be");
- os::free(*counts, mtGC);
- *counts = NULL;
- return false;
- }
-
- // We successfully allocated both counts and epochs
- return true;
-}
-
-// Returns true if the card counts/epochs cache was
-// successfully expanded; false otherwise.
-bool ConcurrentG1Refine::expand_card_count_cache(int cache_size_idx) {
- // Can we expand the card count and epoch tables?
- if (_n_card_counts < _max_n_card_counts) {
- assert(cache_size_idx >= 0 && cache_size_idx < MAX_CC_CACHE_INDEX, "oob");
-
- size_t cache_size = _cc_cache_sizes[cache_size_idx];
- // Make sure we don't go bigger than we will ever need
- cache_size = MIN2(cache_size, _max_n_card_counts);
-
- // Should we expand the card count and card epoch tables?
- if (cache_size > _n_card_counts) {
- // We have been asked to allocate new, larger, arrays for
- // the card counts and the epochs. Attempt the allocation
- // of both before we free the existing arrays in case
- // the allocation is unsuccessful...
- CardCountCacheEntry* counts = NULL;
- CardEpochCacheEntry* epochs = NULL;
-
- if (allocate_card_count_cache(cache_size, &counts, &epochs)) {
- // Allocation was successful.
- // We can just free the old arrays; we're
- // not interested in preserving the contents
- if (_card_counts != NULL) os::free(_card_counts, mtGC);
- if (_card_epochs != NULL) os::free(_card_epochs, mtGC);
-
- // Cache the size of the arrays and the index that got us there.
- _n_card_counts = cache_size;
- _cache_size_index = cache_size_idx;
-
- _card_counts = counts;
- _card_epochs = epochs;
-
- // We successfully allocated/expanded the caches.
- return true;
- }
- }
- }
-
- // We did not successfully expand the caches.
- return false;
-}
-
-void ConcurrentG1Refine::clear_and_record_card_counts() {
- if (G1ConcRSLogCacheSize == 0) {
- return;
- }
-
- double start = os::elapsedTime();
-
- if (_expand_card_counts) {
- int new_idx = _cache_size_index + 1;
-
- if (expand_card_count_cache(new_idx)) {
- // Allocation was successful and _n_card_counts has
- // been updated to the new size. We only need to clear
- // the epochs so we don't read a bogus epoch value
- // when inserting a card into the hot card cache.
- Copy::fill_to_bytes(&_card_epochs[0], _n_card_counts * sizeof(CardEpochCacheEntry));
- }
- _expand_card_counts = false;
- }
-
- int this_epoch = (int) _n_periods;
- assert((this_epoch+1) <= max_jint, "to many periods");
- // Update epoch
- _n_periods++;
- double cc_clear_time_ms = (os::elapsedTime() - start) * 1000;
- _g1h->g1_policy()->phase_times()->record_cc_clear_time_ms(cc_clear_time_ms);
+int ConcurrentG1Refine::thread_num() {
+ int n_threads = (G1ConcRefinementThreads > 0) ? G1ConcRefinementThreads
+ : ParallelGCThreads;
+ return MAX2<int>(n_threads, 1);
}
void ConcurrentG1Refine::print_worker_threads_on(outputStream* st) const {
diff --git a/src/share/vm/gc_implementation/g1/concurrentG1Refine.hpp b/src/share/vm/gc_implementation/g1/concurrentG1Refine.hpp
index 75a31179a..46e6622ee 100644
--- a/src/share/vm/gc_implementation/g1/concurrentG1Refine.hpp
+++ b/src/share/vm/gc_implementation/g1/concurrentG1Refine.hpp
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2001, 2012, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2001, 2013, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@@ -25,13 +25,15 @@
#ifndef SHARE_VM_GC_IMPLEMENTATION_G1_CONCURRENTG1REFINE_HPP
#define SHARE_VM_GC_IMPLEMENTATION_G1_CONCURRENTG1REFINE_HPP
+#include "gc_implementation/g1/g1HotCardCache.hpp"
#include "memory/allocation.hpp"
-#include "memory/cardTableModRefBS.hpp"
#include "runtime/thread.hpp"
#include "utilities/globalDefinitions.hpp"
// Forward decl
class ConcurrentG1RefineThread;
+class G1CollectedHeap;
+class G1HotCardCache;
class G1RemSet;
class ConcurrentG1Refine: public CHeapObj<mtGC> {
@@ -61,141 +63,14 @@ class ConcurrentG1Refine: public CHeapObj<mtGC> {
int _thread_threshold_step;
+ // We delay the refinement of 'hot' cards using the hot card cache.
+ G1HotCardCache _hot_card_cache;
+
// Reset the threshold step value based of the current zone boundaries.
void reset_threshold_step();
- // The cache for card refinement.
- bool _use_cache;
- bool _def_use_cache;
-
- size_t _n_periods; // Used as clearing epoch
-
- // An evicting cache of the number of times each card
- // is accessed. Reduces, but does not eliminate, the amount
- // of duplicated processing of dirty cards.
-
- enum SomePrivateConstants {
- epoch_bits = 32,
- card_num_shift = epoch_bits,
- epoch_mask = AllBits,
- card_num_mask = AllBits,
-
- // The initial cache size is approximately this fraction
- // of a maximal cache (i.e. the size needed for all cards
- // in the heap)
- InitialCacheFraction = 512
- };
-
- const static julong card_num_mask_in_place =
- (julong) card_num_mask << card_num_shift;
-
- typedef struct {
- julong _value; // | card_num | epoch |
- } CardEpochCacheEntry;
-
- julong make_epoch_entry(unsigned int card_num, unsigned int epoch) {
- assert(0 <= card_num && card_num < _max_cards, "Bounds");
- assert(0 <= epoch && epoch <= _n_periods, "must be");
-
- return ((julong) card_num << card_num_shift) | epoch;
- }
-
- unsigned int extract_epoch(julong v) {
- return (v & epoch_mask);
- }
-
- unsigned int extract_card_num(julong v) {
- return (v & card_num_mask_in_place) >> card_num_shift;
- }
-
- typedef struct {
- unsigned char _count;
- unsigned char _evict_count;
- } CardCountCacheEntry;
-
- CardCountCacheEntry* _card_counts;
- CardEpochCacheEntry* _card_epochs;
-
- // The current number of buckets in the card count cache
- size_t _n_card_counts;
-
- // The number of cards for the entire reserved heap
- size_t _max_cards;
-
- // The max number of buckets for the card counts and epochs caches.
- // This is the maximum that the counts and epochs will grow to.
- // It is specified as a fraction or percentage of _max_cards using
- // G1MaxHotCardCountSizePercent.
- size_t _max_n_card_counts;
-
- // Possible sizes of the cache: odd primes that roughly double in size.
- // (See jvmtiTagMap.cpp).
- enum {
- MAX_CC_CACHE_INDEX = 15 // maximum index into the cache size array.
- };
-
- static size_t _cc_cache_sizes[MAX_CC_CACHE_INDEX];
-
- // The index in _cc_cache_sizes corresponding to the size of
- // _card_counts.
- int _cache_size_index;
-
- bool _expand_card_counts;
-
- const jbyte* _ct_bot;
-
- jbyte** _hot_cache;
- int _hot_cache_size;
- int _n_hot;
- int _hot_cache_idx;
-
- int _hot_cache_par_chunk_size;
- volatile int _hot_cache_par_claimed_idx;
-
- // Needed to workaround 6817995
- CardTableModRefBS* _ct_bs;
- G1CollectedHeap* _g1h;
-
- // Helper routine for expand_card_count_cache().
- // The arrays used to hold the card counts and the epochs must have
- // a 1:1 correspondence. Hence they are allocated and freed together.
- // Returns true if the allocations of both the counts and epochs
- // were successful; false otherwise.
- bool allocate_card_count_cache(size_t n,
- CardCountCacheEntry** counts,
- CardEpochCacheEntry** epochs);
-
- // Expands the arrays that hold the card counts and epochs
- // to the cache size at index. Returns true if the expansion/
- // allocation was successful; false otherwise.
- bool expand_card_count_cache(int index);
-
- // hash a given key (index of card_ptr) with the specified size
- static unsigned int hash(size_t key, size_t size) {
- return (unsigned int) (key % size);
- }
-
- // hash a given key (index of card_ptr)
- unsigned int hash(size_t key) {
- return hash(key, _n_card_counts);
- }
-
- unsigned int ptr_2_card_num(jbyte* card_ptr) {
- return (unsigned int) (card_ptr - _ct_bot);
- }
-
- jbyte* card_num_2_ptr(unsigned int card_num) {
- return (jbyte*) (_ct_bot + card_num);
- }
-
- // Returns the count of this card after incrementing it.
- jbyte* add_card_count(jbyte* card_ptr, int* count, bool* defer);
-
- // Returns true if this card is in a young region
- bool is_young_card(jbyte* card_ptr);
-
public:
- ConcurrentG1Refine();
+ ConcurrentG1Refine(G1CollectedHeap* g1h);
~ConcurrentG1Refine();
void init(); // Accomplish some initialization that has to wait.
@@ -206,34 +81,6 @@ class ConcurrentG1Refine: public CHeapObj<mtGC> {
// Iterate over the conc refine threads
void threads_do(ThreadClosure *tc);
- // If this is the first entry for the slot, writes into the cache and
- // returns NULL. If it causes an eviction, returns the evicted pointer.
- // Otherwise, its a cache hit, and returns NULL.
- jbyte* cache_insert(jbyte* card_ptr, bool* defer);
-
- // Process the cached entries.
- void clean_up_cache(int worker_i, G1RemSet* g1rs, DirtyCardQueue* into_cset_dcq);
-
- // Set up for parallel processing of the cards in the hot cache
- void clear_hot_cache_claimed_index() {
- _hot_cache_par_claimed_idx = 0;
- }
-
- // Discard entries in the hot cache.
- void clear_hot_cache() {
- _hot_cache_idx = 0; _n_hot = 0;
- }
-
- bool hot_cache_is_empty() { return _n_hot == 0; }
-
- bool use_cache() { return _use_cache; }
- void set_use_cache(bool b) {
- if (b) _use_cache = _def_use_cache;
- else _use_cache = false;
- }
-
- void clear_and_record_card_counts();
-
static int thread_num();
void print_worker_threads_on(outputStream* st) const;
@@ -250,6 +97,8 @@ class ConcurrentG1Refine: public CHeapObj<mtGC> {
int worker_thread_num() const { return _n_worker_threads; }
int thread_threshold_step() const { return _thread_threshold_step; }
+
+ G1HotCardCache* hot_card_cache() { return &_hot_card_cache; }
};
#endif // SHARE_VM_GC_IMPLEMENTATION_G1_CONCURRENTG1REFINE_HPP
diff --git a/src/share/vm/gc_implementation/g1/g1CardCounts.cpp b/src/share/vm/gc_implementation/g1/g1CardCounts.cpp
new file mode 100644
index 000000000..a055d4f28
--- /dev/null
+++ b/src/share/vm/gc_implementation/g1/g1CardCounts.cpp
@@ -0,0 +1,212 @@
+/*
+ * Copyright (c) 2013, Oracle and/or its affiliates. All rights reserved.
+ * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
+ *
+ * This code is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This code is distributed in the hope that it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
+ * version 2 for more details (a copy is included in the LICENSE file that
+ * accompanied this code).
+ *
+ * You should have received a copy of the GNU General Public License version
+ * 2 along with this work; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
+ *
+ * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
+ * or visit www.oracle.com if you need additional information or have any
+ * questions.
+ *
+ */
+
+#include "precompiled.hpp"
+#include "gc_implementation/g1/g1CardCounts.hpp"
+#include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
+#include "gc_implementation/g1/g1CollectorPolicy.hpp"
+#include "gc_implementation/g1/g1GCPhaseTimes.hpp"
+#include "memory/cardTableModRefBS.hpp"
+#include "services/memTracker.hpp"
+#include "utilities/copy.hpp"
+
+void G1CardCounts::clear_range(size_t from_card_num, size_t to_card_num) {
+ if (has_count_table()) {
+ check_card_num(from_card_num,
+ err_msg("from card num out of range: "SIZE_FORMAT, from_card_num));
+ assert(from_card_num < to_card_num,
+ err_msg("Wrong order? from: " SIZE_FORMAT ", to: "SIZE_FORMAT,
+ from_card_num, to_card_num));
+ assert(to_card_num <= _committed_max_card_num,
+ err_msg("to card num out of range: "
+ "to: "SIZE_FORMAT ", "
+ "max: "SIZE_FORMAT,
+ to_card_num, _committed_max_card_num));
+
+ to_card_num = MIN2(_committed_max_card_num, to_card_num);
+
+ Copy::fill_to_bytes(&_card_counts[from_card_num], (to_card_num - from_card_num));
+ }
+}
+
+G1CardCounts::G1CardCounts(G1CollectedHeap *g1h):
+ _g1h(g1h), _card_counts(NULL),
+ _reserved_max_card_num(0), _committed_max_card_num(0),
+ _committed_size(0) {}
+
+void G1CardCounts::initialize() {
+ assert(_g1h->max_capacity() > 0, "initialization order");
+ assert(_g1h->capacity() == 0, "initialization order");
+
+ if (G1ConcRSHotCardLimit > 0) {
+ // The max value we can store in the counts table is
+ // max_jubyte. Guarantee the value of the hot
+ // threshold limit is no more than this.
+ guarantee(G1ConcRSHotCardLimit <= max_jubyte, "sanity");
+
+ ModRefBarrierSet* bs = _g1h->mr_bs();
+ guarantee(bs->is_a(BarrierSet::CardTableModRef), "Precondition");
+ _ct_bs = (CardTableModRefBS*)bs;
+ _ct_bot = _ct_bs->byte_for_const(_g1h->reserved_region().start());
+
+ // Allocate/Reserve the counts table
+ size_t reserved_bytes = _g1h->max_capacity();
+ _reserved_max_card_num = reserved_bytes >> CardTableModRefBS::card_shift;
+
+ size_t reserved_size = _reserved_max_card_num * sizeof(jbyte);
+ ReservedSpace rs(ReservedSpace::allocation_align_size_up(reserved_size));
+ if (!rs.is_reserved()) {
+ warning("Could not reserve enough space for the card counts table");
+ guarantee(!has_reserved_count_table(), "should be NULL");
+ return;
+ }
+
+ MemTracker::record_virtual_memory_type((address)rs.base(), mtGC);
+
+ _card_counts_storage.initialize(rs, 0);
+ _card_counts = (jubyte*) _card_counts_storage.low();
+ }
+}
+
+void G1CardCounts::resize(size_t heap_capacity) {
+ // Expand the card counts table to handle a heap with the given capacity.
+
+ if (!has_reserved_count_table()) {
+ // Don't expand if we failed to reserve the card counts table.
+ return;
+ }
+
+ assert(_committed_size ==
+ ReservedSpace::allocation_align_size_up(_committed_size),
+ err_msg("Unaligned? committed_size: " SIZE_FORMAT, _committed_size));
+
+ // Verify that the committed space for the card counts
+ // matches our committed max card num.
+ size_t prev_committed_size = _committed_size;
+ size_t prev_committed_card_num = prev_committed_size / sizeof(jbyte);
+ assert(prev_committed_card_num == _committed_max_card_num,
+ err_msg("Card mismatch: "
+ "prev: " SIZE_FORMAT ", "
+ "committed: "SIZE_FORMAT,
+ prev_committed_card_num, _committed_max_card_num));
+
+ size_t new_size = (heap_capacity >> CardTableModRefBS::card_shift) * sizeof(jbyte);
+ size_t new_committed_size = ReservedSpace::allocation_align_size_up(new_size);
+ size_t new_committed_card_num =
+ MIN2(_reserved_max_card_num, new_committed_size / sizeof(jbyte));
+
+ if (_committed_max_card_num < new_committed_card_num) {
+ // we need to expand the backing store for the card counts
+ size_t expand_size = new_committed_size - prev_committed_size;
+
+ if (!_card_counts_storage.expand_by(expand_size)) {
+ warning("Card counts table backing store commit failure");
+ return;
+ }
+ assert(_card_counts_storage.committed_size() == new_committed_size,
+ "expansion commit failure");
+
+ _committed_size = new_committed_size;
+ _committed_max_card_num = new_committed_card_num;
+
+ clear_range(prev_committed_card_num, _committed_max_card_num);
+ }
+}
+
+uint G1CardCounts::add_card_count(jbyte* card_ptr) {
+ // Returns the number of times the card has been refined.
+ // If we failed to reserve/commit the counts table, return 0.
+ // If card_ptr is beyond the committed end of the counts table,
+ // return 0.
+ // Otherwise return the actual count.
+ // Unless G1ConcRSHotCardLimit has been set appropriately,
+ // returning 0 will result in the card being considered
+ // cold and will be refined immediately.
+ uint count = 0;
+ if (has_count_table()) {
+ size_t card_num = ptr_2_card_num(card_ptr);
+ if (card_num < _committed_max_card_num) {
+ count = (uint) _card_counts[card_num];
+ if (count < G1ConcRSHotCardLimit) {
+ _card_counts[card_num] += 1;
+ }
+ assert(_card_counts[card_num] <= G1ConcRSHotCardLimit,
+ err_msg("Refinement count overflow? "
+ "new count: "UINT32_FORMAT,
+ (uint) _card_counts[card_num]));
+ }
+ }
+ return count;
+}
+
+bool G1CardCounts::is_hot(uint count) {
+ return (count >= G1ConcRSHotCardLimit);
+}
+
+void G1CardCounts::clear_region(HeapRegion* hr) {
+ assert(!hr->isHumongous(), "Should have been cleared");
+ if (has_count_table()) {
+ HeapWord* bottom = hr->bottom();
+
+ // We use the last address in hr as hr could be the
+ // last region in the heap. In which case trying to find
+ // the card for hr->end() will be an OOB accesss to the
+ // card table.
+ HeapWord* last = hr->end() - 1;
+ assert(_g1h->g1_committed().contains(last),
+ err_msg("last not in committed: "
+ "last: " PTR_FORMAT ", "
+ "committed: [" PTR_FORMAT ", " PTR_FORMAT ")",
+ last,
+ _g1h->g1_committed().start(),
+ _g1h->g1_committed().end()));
+
+ const jbyte* from_card_ptr = _ct_bs->byte_for_const(bottom);
+ const jbyte* last_card_ptr = _ct_bs->byte_for_const(last);
+
+#ifdef ASSERT
+ HeapWord* start_addr = _ct_bs->addr_for(from_card_ptr);
+ assert(start_addr == hr->bottom(), "alignment");
+ HeapWord* last_addr = _ct_bs->addr_for(last_card_ptr);
+ assert((last_addr + CardTableModRefBS::card_size_in_words) == hr->end(), "alignment");
+#endif // ASSERT
+
+ // Clear the counts for the (exclusive) card range.
+ size_t from_card_num = ptr_2_card_num(from_card_ptr);
+ size_t to_card_num = ptr_2_card_num(last_card_ptr) + 1;
+ clear_range(from_card_num, to_card_num);
+ }
+}
+
+void G1CardCounts::clear_all() {
+ assert(SafepointSynchronize::is_at_safepoint(), "don't call this otherwise");
+ clear_range((size_t)0, _committed_max_card_num);
+}
+
+G1CardCounts::~G1CardCounts() {
+ if (has_reserved_count_table()) {
+ _card_counts_storage.release();
+ }
+}
+
diff --git a/src/share/vm/gc_implementation/g1/g1CardCounts.hpp b/src/share/vm/gc_implementation/g1/g1CardCounts.hpp
new file mode 100644
index 000000000..cef297bd2
--- /dev/null
+++ b/src/share/vm/gc_implementation/g1/g1CardCounts.hpp
@@ -0,0 +1,126 @@
+/*
+ * Copyright (c) 2013, Oracle and/or its affiliates. All rights reserved.
+ * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
+ *
+ * This code is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This code is distributed in the hope that it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
+ * version 2 for more details (a copy is included in the LICENSE file that
+ * accompanied this code).
+ *
+ * You should have received a copy of the GNU General Public License version
+ * 2 along with this work; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
+ *
+ * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
+ * or visit www.oracle.com if you need additional information or have any
+ * questions.
+ *
+ */
+
+#ifndef SHARE_VM_GC_IMPLEMENTATION_G1_G1CARDCOUNTS_HPP
+#define SHARE_VM_GC_IMPLEMENTATION_G1_G1CARDCOUNTS_HPP
+
+#include "memory/allocation.hpp"
+#include "runtime/virtualspace.hpp"
+#include "utilities/globalDefinitions.hpp"
+
+class CardTableModRefBS;
+class G1CollectedHeap;
+class HeapRegion;
+
+// Table to track the number of times a card has been refined. Once
+// a card has been refined a certain number of times, it is
+// considered 'hot' and its refinement is delayed by inserting the
+// card into the hot card cache. The card will then be refined when
+// it is evicted from the hot card cache, or when the hot card cache
+// is 'drained' during the next evacuation pause.
+
+class G1CardCounts: public CHeapObj<mtGC> {
+ G1CollectedHeap* _g1h;
+
+ // The table of counts
+ jubyte* _card_counts;
+
+ // Max capacity of the reserved space for the counts table
+ size_t _reserved_max_card_num;
+
+ // Max capacity of the committed space for the counts table
+ size_t _committed_max_card_num;
+
+ // Size of committed space for the counts table
+ size_t _committed_size;
+
+ // CardTable bottom.
+ const jbyte* _ct_bot;
+
+ // Barrier set
+ CardTableModRefBS* _ct_bs;
+
+ // The virtual memory backing the counts table
+ VirtualSpace _card_counts_storage;
+
+ // Returns true if the card counts table has been reserved.
+ bool has_reserved_count_table() { return _card_counts != NULL; }
+
+ // Returns true if the card counts table has been reserved and committed.
+ bool has_count_table() {
+ return has_reserved_count_table() && _committed_max_card_num > 0;
+ }
+
+ void check_card_num(size_t card_num, const char* msg) {
+ assert(card_num >= 0 && card_num < _committed_max_card_num, msg);
+ }
+
+ size_t ptr_2_card_num(const jbyte* card_ptr) {
+ assert(card_ptr >= _ct_bot,
+ err_msg("Inavalied card pointer: "
+ "card_ptr: " PTR_FORMAT ", "
+ "_ct_bot: " PTR_FORMAT,
+ card_ptr, _ct_bot));
+ size_t card_num = pointer_delta(card_ptr, _ct_bot, sizeof(jbyte));
+ check_card_num(card_num,
+ err_msg("card pointer out of range: " PTR_FORMAT, card_ptr));
+ return card_num;
+ }
+
+ jbyte* card_num_2_ptr(size_t card_num) {
+ check_card_num(card_num,
+ err_msg("card num out of range: "SIZE_FORMAT, card_num));
+ return (jbyte*) (_ct_bot + card_num);
+ }
+
+ // Clear the counts table for the given (exclusive) index range.
+ void clear_range(size_t from_card_num, size_t to_card_num);
+
+ public:
+ G1CardCounts(G1CollectedHeap* g1h);
+ ~G1CardCounts();
+
+ void initialize();
+
+ // Resize the committed space for the card counts table in
+ // response to a resize of the committed space for the heap.
+ void resize(size_t heap_capacity);
+
+ // Increments the refinement count for the given card.
+ // Returns the pre-increment count value.
+ uint add_card_count(jbyte* card_ptr);
+
+ // Returns true if the given count is high enough to be considered
+ // 'hot'; false otherwise.
+ bool is_hot(uint count);
+
+ // Clears the card counts for the cards spanned by the region
+ void clear_region(HeapRegion* hr);
+
+ // Clear the entire card counts table during GC.
+ // Updates the policy stats with the duration.
+ void clear_all();
+};
+
+#endif // SHARE_VM_GC_IMPLEMENTATION_G1_G1CARDCOUNTS_HPP
diff --git a/src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp b/src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp
index 76ce145eb..49cde4272 100644
--- a/src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp
+++ b/src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp
@@ -96,7 +96,7 @@ public:
_sts(sts), _g1rs(g1rs), _cg1r(cg1r), _concurrent(true)
{}
bool do_card_ptr(jbyte* card_ptr, int worker_i) {
- bool oops_into_cset = _g1rs->concurrentRefineOneCard(card_ptr, worker_i, false);
+ bool oops_into_cset = _g1rs->refine_card(card_ptr, worker_i, false);
// This path is executed by the concurrent refine or mutator threads,
// concurrently, and so we do not care if card_ptr contains references
// that point into the collection set.
@@ -1452,9 +1452,10 @@ bool G1CollectedHeap::do_collection(bool explicit_gc,
_hr_printer.end_gc(true /* full */, (size_t) total_collections());
}
- if (_cg1r->use_cache()) {
- _cg1r->clear_and_record_card_counts();
- _cg1r->clear_hot_cache();
+ G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
+ if (hot_card_cache->use_cache()) {
+ hot_card_cache->reset_card_counts();
+ hot_card_cache->reset_hot_cache();
}
// Rebuild remembered sets of all regions.
@@ -1767,6 +1768,8 @@ void G1CollectedHeap::update_committed_space(HeapWord* old_end,
Universe::heap()->barrier_set()->resize_covered_region(_g1_committed);
// Tell the BOT about the update.
_bot_shared->resize(_g1_committed.word_size());
+ // Tell the hot card cache about the update
+ _cg1r->hot_card_cache()->resize_card_counts(capacity());
}
bool G1CollectedHeap::expand(size_t expand_bytes) {
@@ -1843,33 +1846,32 @@ void G1CollectedHeap::shrink_helper(size_t shrink_bytes) {
ReservedSpace::page_align_size_down(shrink_bytes);
aligned_shrink_bytes = align_size_down(aligned_shrink_bytes,
HeapRegion::GrainBytes);
- uint num_regions_deleted = 0;
- MemRegion mr = _hrs.shrink_by(aligned_shrink_bytes, &num_regions_deleted);
+ uint num_regions_to_remove = (uint)(shrink_bytes / HeapRegion::GrainBytes);
+
+ uint num_regions_removed = _hrs.shrink_by(num_regions_to_remove);
HeapWord* old_end = (HeapWord*) _g1_storage.high();
- assert(mr.end() == old_end, "post-condition");
+ size_t shrunk_bytes = num_regions_removed * HeapRegion::GrainBytes;
ergo_verbose3(ErgoHeapSizing,
"shrink the heap",
ergo_format_byte("requested shrinking amount")
ergo_format_byte("aligned shrinking amount")
ergo_format_byte("attempted shrinking amount"),
- shrink_bytes, aligned_shrink_bytes, mr.byte_size());
- if (mr.byte_size() > 0) {
+ shrink_bytes, aligned_shrink_bytes, shrunk_bytes);
+ if (num_regions_removed > 0) {
+ _g1_storage.shrink_by(shrunk_bytes);
+ HeapWord* new_end = (HeapWord*) _g1_storage.high();
+
if (_hr_printer.is_active()) {
- HeapWord* curr = mr.end();
- while (curr > mr.start()) {
+ HeapWord* curr = old_end;
+ while (curr > new_end) {
HeapWord* curr_end = curr;
curr -= HeapRegion::GrainWords;
_hr_printer.uncommit(curr, curr_end);
}
- assert(curr == mr.start(), "post-condition");
}
- _g1_storage.shrink_by(mr.byte_size());
- HeapWord* new_end = (HeapWord*) _g1_storage.high();
- assert(mr.start() == new_end, "post-condition");
-
- _expansion_regions += num_regions_deleted;
+ _expansion_regions += num_regions_removed;
update_committed_space(old_end, new_end);
HeapRegionRemSet::shrink_heap(n_regions());
g1_policy()->record_new_heap_size(n_regions());
@@ -2000,7 +2002,7 @@ jint G1CollectedHeap::initialize() {
Universe::check_alignment(init_byte_size, HeapRegion::GrainBytes, "g1 heap");
Universe::check_alignment(max_byte_size, HeapRegion::GrainBytes, "g1 heap");
- _cg1r = new ConcurrentG1Refine();
+ _cg1r = new ConcurrentG1Refine(this);
// Reserve the maximum.
@@ -2061,6 +2063,9 @@ jint G1CollectedHeap::initialize() {
(HeapWord*) _g1_reserved.end(),
_expansion_regions);
+ // Do later initialization work for concurrent refinement.
+ _cg1r->init();
+
// 6843694 - ensure that the maximum region index can fit
// in the remembered set structures.
const uint max_region_idx = (1U << (sizeof(RegionIdx_t)*BitsPerByte-1)) - 1;
@@ -2078,20 +2083,20 @@ jint G1CollectedHeap::initialize() {
_g1h = this;
- _in_cset_fast_test_length = max_regions();
- _in_cset_fast_test_base =
+ _in_cset_fast_test_length = max_regions();
+ _in_cset_fast_test_base =
NEW_C_HEAP_ARRAY(bool, (size_t) _in_cset_fast_test_length, mtGC);
- // We're biasing _in_cset_fast_test to avoid subtracting the
- // beginning of the heap every time we want to index; basically
- // it's the same with what we do with the card table.
- _in_cset_fast_test = _in_cset_fast_test_base -
+ // We're biasing _in_cset_fast_test to avoid subtracting the
+ // beginning of the heap every time we want to index; basically
+ // it's the same with what we do with the card table.
+ _in_cset_fast_test = _in_cset_fast_test_base -
((uintx) _g1_reserved.start() >> HeapRegion::LogOfHRGrainBytes);
- // Clear the _cset_fast_test bitmap in anticipation of adding
- // regions to the incremental collection set for the first
- // evacuation pause.
- clear_cset_fast_test();
+ // Clear the _cset_fast_test bitmap in anticipation of adding
+ // regions to the incremental collection set for the first
+ // evacuation pause.
+ clear_cset_fast_test();
// Create the ConcurrentMark data structure and thread.
// (Must do this late, so that "max_regions" is defined.)
@@ -2153,9 +2158,6 @@ jint G1CollectedHeap::initialize() {
// counts and that mechanism.
SpecializationStats::clear();
- // Do later initialization work for concurrent refinement.
- _cg1r->init();
-
// Here we allocate the dummy full region that is required by the
// G1AllocRegion class. If we don't pass an address in the reserved
// space here, lots of asserts fire.
@@ -2314,7 +2316,8 @@ void G1CollectedHeap::iterate_dirty_card_closure(CardTableEntryClosure* cl,
bool concurrent,
int worker_i) {
// Clean cards in the hot card cache
- concurrent_g1_refine()->clean_up_cache(worker_i, g1_rem_set(), into_cset_dcq);
+ G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
+ hot_card_cache->drain(worker_i, g1_rem_set(), into_cset_dcq);
DirtyCardQueueSet& dcqs = JavaThread::dirty_card_queue_set();
int n_completed_buffers = 0;
@@ -5604,8 +5607,11 @@ void G1CollectedHeap::evacuate_collection_set() {
NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
g1_rem_set()->prepare_for_oops_into_collection_set_do();
- concurrent_g1_refine()->set_use_cache(false);
- concurrent_g1_refine()->clear_hot_cache_claimed_index();
+
+ // Disable the hot card cache.
+ G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
+ hot_card_cache->reset_hot_cache_claimed_index();
+ hot_card_cache->set_use_cache(false);
uint n_workers;
if (G1CollectedHeap::use_parallel_gc_threads()) {
@@ -5687,8 +5693,11 @@ void G1CollectedHeap::evacuate_collection_set() {
release_gc_alloc_regions(n_workers);
g1_rem_set()->cleanup_after_oops_into_collection_set_do();
- concurrent_g1_refine()->clear_hot_cache();
- concurrent_g1_refine()->set_use_cache(true);
+ // Reset and re-enable the hot card cache.
+ // Note the counts for the cards in the regions in the
+ // collection set are reset when the collection set is freed.
+ hot_card_cache->reset_hot_cache();
+ hot_card_cache->set_use_cache(true);
finalize_for_evac_failure();
@@ -5750,6 +5759,12 @@ void G1CollectedHeap::free_region(HeapRegion* hr,
assert(!hr->is_empty(), "the region should not be empty");
assert(free_list != NULL, "pre-condition");
+ // Clear the card counts for this region.
+ // Note: we only need to do this if the region is not young
+ // (since we don't refine cards in young regions).
+ if (!hr->is_young()) {
+ _cg1r->hot_card_cache()->reset_card_counts(hr);
+ }
*pre_used += hr->used();
hr->hr_clear(par, true /* clear_space */);
free_list->add_as_head(hr);
diff --git a/src/share/vm/gc_implementation/g1/g1CollectorPolicy.cpp b/src/share/vm/gc_implementation/g1/g1CollectorPolicy.cpp
index a645cc33d..a21290c83 100644
--- a/src/share/vm/gc_implementation/g1/g1CollectorPolicy.cpp
+++ b/src/share/vm/gc_implementation/g1/g1CollectorPolicy.cpp
@@ -309,7 +309,8 @@ G1CollectorPolicy::G1CollectorPolicy() :
void G1CollectorPolicy::initialize_flags() {
set_min_alignment(HeapRegion::GrainBytes);
- set_max_alignment(GenRemSet::max_alignment_constraint(rem_set_name()));
+ size_t card_table_alignment = GenRemSet::max_alignment_constraint(rem_set_name());
+ set_max_alignment(MAX2(card_table_alignment, min_alignment()));
if (SurvivorRatio < 1) {
vm_exit_during_initialization("Invalid survivor ratio specified");
}
diff --git a/src/share/vm/gc_implementation/g1/g1GCPhaseTimes.cpp b/src/share/vm/gc_implementation/g1/g1GCPhaseTimes.cpp
index 9782b67c0..f6d3b6f4b 100644
--- a/src/share/vm/gc_implementation/g1/g1GCPhaseTimes.cpp
+++ b/src/share/vm/gc_implementation/g1/g1GCPhaseTimes.cpp
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2012, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2013, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@@ -155,11 +155,6 @@ void WorkerDataArray<T>::verify() {
G1GCPhaseTimes::G1GCPhaseTimes(uint max_gc_threads) :
_max_gc_threads(max_gc_threads),
- _min_clear_cc_time_ms(-1.0),
- _max_clear_cc_time_ms(-1.0),
- _cur_clear_cc_time_ms(0.0),
- _cum_clear_cc_time_ms(0.0),
- _num_cc_clears(0L),
_last_gc_worker_start_times_ms(_max_gc_threads, "%.1lf", false),
_last_ext_root_scan_times_ms(_max_gc_threads, "%.1lf"),
_last_satb_filtering_times_ms(_max_gc_threads, "%.1lf"),
@@ -212,11 +207,11 @@ void G1GCPhaseTimes::note_gc_end() {
_last_gc_worker_times_ms.set(i, worker_time);
double worker_known_time = _last_ext_root_scan_times_ms.get(i) +
- _last_satb_filtering_times_ms.get(i) +
- _last_update_rs_times_ms.get(i) +
- _last_scan_rs_times_ms.get(i) +
- _last_obj_copy_times_ms.get(i) +
- _last_termination_times_ms.get(i);
+ _last_satb_filtering_times_ms.get(i) +
+ _last_update_rs_times_ms.get(i) +
+ _last_scan_rs_times_ms.get(i) +
+ _last_obj_copy_times_ms.get(i) +
+ _last_termination_times_ms.get(i);
double worker_other_time = worker_time - worker_known_time;
_last_gc_worker_other_times_ms.set(i, worker_other_time);
@@ -285,15 +280,6 @@ void G1GCPhaseTimes::print(double pause_time_sec) {
}
print_stats(1, "Code Root Fixup", _cur_collection_code_root_fixup_time_ms);
print_stats(1, "Clear CT", _cur_clear_ct_time_ms);
- if (Verbose && G1Log::finest()) {
- print_stats(1, "Cur Clear CC", _cur_clear_cc_time_ms);
- print_stats(1, "Cum Clear CC", _cum_clear_cc_time_ms);
- print_stats(1, "Min Clear CC", _min_clear_cc_time_ms);
- print_stats(1, "Max Clear CC", _max_clear_cc_time_ms);
- if (_num_cc_clears > 0) {
- print_stats(1, "Avg Clear CC", _cum_clear_cc_time_ms / ((double)_num_cc_clears));
- }
- }
double misc_time_ms = pause_time_sec * MILLIUNITS - accounted_time_ms();
print_stats(1, "Other", misc_time_ms);
if (_cur_verify_before_time_ms > 0.0) {
@@ -311,19 +297,3 @@ void G1GCPhaseTimes::print(double pause_time_sec) {
print_stats(2, "Verify After", _cur_verify_after_time_ms);
}
}
-
-void G1GCPhaseTimes::record_cc_clear_time_ms(double ms) {
- if (!(Verbose && G1Log::finest())) {
- return;
- }
-
- if (_min_clear_cc_time_ms < 0.0 || ms <= _min_clear_cc_time_ms) {
- _min_clear_cc_time_ms = ms;
- }
- if (_max_clear_cc_time_ms < 0.0 || ms >= _max_clear_cc_time_ms) {
- _max_clear_cc_time_ms = ms;
- }
- _cur_clear_cc_time_ms = ms;
- _cum_clear_cc_time_ms += ms;
- _num_cc_clears++;
-}
diff --git a/src/share/vm/gc_implementation/g1/g1GCPhaseTimes.hpp b/src/share/vm/gc_implementation/g1/g1GCPhaseTimes.hpp
index b6e289ed2..2fa5300b1 100644
--- a/src/share/vm/gc_implementation/g1/g1GCPhaseTimes.hpp
+++ b/src/share/vm/gc_implementation/g1/g1GCPhaseTimes.hpp
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2012, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2013, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@@ -133,13 +133,6 @@ class G1GCPhaseTimes : public CHeapObj<mtGC> {
double _cur_ref_proc_time_ms;
double _cur_ref_enq_time_ms;
- // Card Table Count Cache stats
- double _min_clear_cc_time_ms; // min
- double _max_clear_cc_time_ms; // max
- double _cur_clear_cc_time_ms; // clearing time during current pause
- double _cum_clear_cc_time_ms; // cummulative clearing time
- jlong _num_cc_clears; // number of times the card count cache has been cleared
-
double _cur_collection_start_sec;
double _root_region_scan_wait_time_ms;
@@ -227,8 +220,6 @@ class G1GCPhaseTimes : public CHeapObj<mtGC> {
_root_region_scan_wait_time_ms = time_ms;
}
- void record_cc_clear_time_ms(double ms);
-
void record_young_free_cset_time_ms(double time_ms) {
_recorded_young_free_cset_time_ms = time_ms;
}
diff --git a/src/share/vm/gc_implementation/g1/g1HotCardCache.cpp b/src/share/vm/gc_implementation/g1/g1HotCardCache.cpp
new file mode 100644
index 000000000..5d224ce8f
--- /dev/null
+++ b/src/share/vm/gc_implementation/g1/g1HotCardCache.cpp
@@ -0,0 +1,148 @@
+/*
+ * Copyright (c) 2013, Oracle and/or its affiliates. All rights reserved.
+ * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
+ *
+ * This code is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This code is distributed in the hope that it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
+ * version 2 for more details (a copy is included in the LICENSE file that
+ * accompanied this code).
+ *
+ * You should have received a copy of the GNU General Public License version
+ * 2 along with this work; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
+ *
+ * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
+ * or visit www.oracle.com if you need additional information or have any
+ * questions.
+ *
+ */
+
+#include "precompiled.hpp"
+#include "gc_implementation/g1/dirtyCardQueue.hpp"
+#include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
+#include "gc_implementation/g1/g1HotCardCache.hpp"
+#include "gc_implementation/g1/g1RemSet.hpp"
+#include "gc_implementation/g1/heapRegion.hpp"
+#include "runtime/atomic.hpp"
+
+G1HotCardCache::G1HotCardCache(G1CollectedHeap *g1h):
+ _g1h(g1h), _hot_cache(NULL), _use_cache(false), _card_counts(g1h) {}
+
+void G1HotCardCache::initialize() {
+ if (default_use_cache()) {
+ _use_cache = true;
+
+ _hot_cache_size = (1 << G1ConcRSLogCacheSize);
+ _hot_cache = NEW_C_HEAP_ARRAY(jbyte*, _hot_cache_size, mtGC);
+
+ _n_hot = 0;
+ _hot_cache_idx = 0;
+
+ // For refining the cards in the hot cache in parallel
+ int n_workers = (ParallelGCThreads > 0 ?
+ _g1h->workers()->total_workers() : 1);
+ _hot_cache_par_chunk_size = MAX2(1, _hot_cache_size / n_workers);
+ _hot_cache_par_claimed_idx = 0;
+
+ _card_counts.initialize();
+ }
+}
+
+G1HotCardCache::~G1HotCardCache() {
+ if (default_use_cache()) {
+ assert(_hot_cache != NULL, "Logic");
+ FREE_C_HEAP_ARRAY(jbyte*, _hot_cache, mtGC);
+ }
+}
+
+jbyte* G1HotCardCache::insert(jbyte* card_ptr) {
+ uint count = _card_counts.add_card_count(card_ptr);
+ if (!_card_counts.is_hot(count)) {
+ // The card is not hot so do not store it in the cache;
+ // return it for immediate refining.
+ return card_ptr;
+ }
+
+ // Otherwise, the card is hot.
+ jbyte* res = NULL;
+ MutexLockerEx x(HotCardCache_lock, Mutex::_no_safepoint_check_flag);
+ if (_n_hot == _hot_cache_size) {
+ res = _hot_cache[_hot_cache_idx];
+ _n_hot--;
+ }
+
+ // Now _n_hot < _hot_cache_size, and we can insert at _hot_cache_idx.
+ _hot_cache[_hot_cache_idx] = card_ptr;
+ _hot_cache_idx++;
+
+ if (_hot_cache_idx == _hot_cache_size) {
+ // Wrap around
+ _hot_cache_idx = 0;
+ }
+ _n_hot++;
+
+ return res;
+}
+
+void G1HotCardCache::drain(int worker_i,
+ G1RemSet* g1rs,
+ DirtyCardQueue* into_cset_dcq) {
+ if (!default_use_cache()) {
+ assert(_hot_cache == NULL, "Logic");
+ return;
+ }
+
+ assert(_hot_cache != NULL, "Logic");
+ assert(!use_cache(), "cache should be disabled");
+ int start_idx;
+
+ while ((start_idx = _hot_cache_par_claimed_idx) < _n_hot) { // read once
+ int end_idx = start_idx + _hot_cache_par_chunk_size;
+
+ if (start_idx ==
+ Atomic::cmpxchg(end_idx, &_hot_cache_par_claimed_idx, start_idx)) {
+ // The current worker has successfully claimed the chunk [start_idx..end_idx)
+ end_idx = MIN2(end_idx, _n_hot);
+ for (int i = start_idx; i < end_idx; i++) {
+ jbyte* card_ptr = _hot_cache[i];
+ if (card_ptr != NULL) {
+ if (g1rs->refine_card(card_ptr, worker_i, true)) {
+ // The part of the heap spanned by the card contains references
+ // that point into the current collection set.
+ // We need to record the card pointer in the DirtyCardQueueSet
+ // that we use for such cards.
+ //
+ // The only time we care about recording cards that contain
+ // references that point into the collection set is during
+ // RSet updating while within an evacuation pause.
+ // In this case worker_i should be the id of a GC worker thread
+ assert(SafepointSynchronize::is_at_safepoint(), "Should be at a safepoint");
+ assert(worker_i < (int) (ParallelGCThreads == 0 ? 1 : ParallelGCThreads),
+ err_msg("incorrect worker id: "INT32_FORMAT, worker_i));
+
+ into_cset_dcq->enqueue(card_ptr);
+ }
+ }
+ }
+ }
+ }
+ // The existing entries in the hot card cache, which were just refined
+ // above, are discarded prior to re-enabling the cache near the end of the GC.
+}
+
+void G1HotCardCache::resize_card_counts(size_t heap_capacity) {
+ _card_counts.resize(heap_capacity);
+}
+
+void G1HotCardCache::reset_card_counts(HeapRegion* hr) {
+ _card_counts.clear_region(hr);
+}
+
+void G1HotCardCache::reset_card_counts() {
+ _card_counts.clear_all();
+}
diff --git a/src/share/vm/gc_implementation/g1/g1HotCardCache.hpp b/src/share/vm/gc_implementation/g1/g1HotCardCache.hpp
new file mode 100644
index 000000000..99d700d63
--- /dev/null
+++ b/src/share/vm/gc_implementation/g1/g1HotCardCache.hpp
@@ -0,0 +1,128 @@
+/*
+ * Copyright (c) 2013, Oracle and/or its affiliates. All rights reserved.
+ * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
+ *
+ * This code is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This code is distributed in the hope that it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
+ * version 2 for more details (a copy is included in the LICENSE file that
+ * accompanied this code).
+ *
+ * You should have received a copy of the GNU General Public License version
+ * 2 along with this work; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
+ *
+ * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
+ * or visit www.oracle.com if you need additional information or have any
+ * questions.
+ *
+ */
+
+#ifndef SHARE_VM_GC_IMPLEMENTATION_G1_G1HOTCARDCACHE_HPP
+#define SHARE_VM_GC_IMPLEMENTATION_G1_G1HOTCARDCACHE_HPP
+
+#include "gc_implementation/g1/g1_globals.hpp"
+#include "gc_implementation/g1/g1CardCounts.hpp"
+#include "memory/allocation.hpp"
+#include "runtime/safepoint.hpp"
+#include "runtime/thread.inline.hpp"
+#include "utilities/globalDefinitions.hpp"
+
+class DirtyCardQueue;
+class G1CollectedHeap;
+class G1RemSet;
+class HeapRegion;
+
+// An evicting cache of cards that have been logged by the G1 post
+// write barrier. Placing a card in the cache delays the refinement
+// of the card until the card is evicted, or the cache is drained
+// during the next evacuation pause.
+//
+// The first thing the G1 post write barrier does is to check whether
+// the card containing the updated pointer is already dirty and, if
+// so, skips the remaining code in the barrier.
+//
+// Delaying the refinement of a card will make the card fail the
+// first is_dirty check in the write barrier, skipping the remainder
+// of the write barrier.
+//
+// This can significantly reduce the overhead of the write barrier
+// code, increasing throughput.
+
+class G1HotCardCache: public CHeapObj<mtGC> {
+ G1CollectedHeap* _g1h;
+
+ // The card cache table
+ jbyte** _hot_cache;
+
+ int _hot_cache_size;
+ int _n_hot;
+ int _hot_cache_idx;
+
+ int _hot_cache_par_chunk_size;
+ volatile int _hot_cache_par_claimed_idx;
+
+ bool _use_cache;
+
+ G1CardCounts _card_counts;
+
+ bool default_use_cache() const {
+ return (G1ConcRSLogCacheSize > 0);
+ }
+
+ public:
+ G1HotCardCache(G1CollectedHeap* g1h);
+ ~G1HotCardCache();
+
+ void initialize();
+
+ bool use_cache() { return _use_cache; }
+
+ void set_use_cache(bool b) {
+ _use_cache = (b ? default_use_cache() : false);
+ }
+
+ // Returns the card to be refined or NULL.
+ //
+ // Increments the count for given the card. if the card is not 'hot',
+ // it is returned for immediate refining. Otherwise the card is
+ // added to the hot card cache.
+ // If there is enough room in the hot card cache for the card we're
+ // adding, NULL is returned and no further action in needed.
+ // If we evict a card from the cache to make room for the new card,
+ // the evicted card is then returned for refinement.
+ jbyte* insert(jbyte* card_ptr);
+
+ // Refine the cards that have delayed as a result of
+ // being in the cache.
+ void drain(int worker_i, G1RemSet* g1rs, DirtyCardQueue* into_cset_dcq);
+
+ // Set up for parallel processing of the cards in the hot cache
+ void reset_hot_cache_claimed_index() {
+ _hot_cache_par_claimed_idx = 0;
+ }
+
+ // Resets the hot card cache and discards the entries.
+ void reset_hot_cache() {
+ assert(SafepointSynchronize::is_at_safepoint(), "Should be at a safepoint");
+ assert(Thread::current()->is_VM_thread(), "Current thread should be the VMthread");
+ _hot_cache_idx = 0; _n_hot = 0;
+ }
+
+ bool hot_cache_is_empty() { return _n_hot == 0; }
+
+ // Resizes the card counts table to match the given capacity
+ void resize_card_counts(size_t heap_capacity);
+
+ // Zeros the values in the card counts table for entire committed heap
+ void reset_card_counts();
+
+ // Zeros the values in the card counts table for the given region
+ void reset_card_counts(HeapRegion* hr);
+};
+
+#endif // SHARE_VM_GC_IMPLEMENTATION_G1_G1HOTCARDCACHE_HPP
diff --git a/src/share/vm/gc_implementation/g1/g1RemSet.cpp b/src/share/vm/gc_implementation/g1/g1RemSet.cpp
index e7151071e..d527a3fc6 100644
--- a/src/share/vm/gc_implementation/g1/g1RemSet.cpp
+++ b/src/share/vm/gc_implementation/g1/g1RemSet.cpp
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2001, 2012, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2001, 2013, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@@ -29,6 +29,7 @@
#include "gc_implementation/g1/g1BlockOffsetTable.inline.hpp"
#include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
#include "gc_implementation/g1/g1CollectorPolicy.hpp"
+#include "gc_implementation/g1/g1HotCardCache.hpp"
#include "gc_implementation/g1/g1GCPhaseTimes.hpp"
#include "gc_implementation/g1/g1OopClosures.inline.hpp"
#include "gc_implementation/g1/g1RemSet.inline.hpp"
@@ -247,7 +248,7 @@ public:
assert(SafepointSynchronize::is_at_safepoint(), "not during an evacuation pause");
assert(worker_i < (int) (ParallelGCThreads == 0 ? 1 : ParallelGCThreads), "should be a GC worker");
- if (_g1rs->concurrentRefineOneCard(card_ptr, worker_i, true)) {
+ if (_g1rs->refine_card(card_ptr, worker_i, true)) {
// 'card_ptr' contains references that point into the collection
// set. We need to record the card in the DCQS
// (G1CollectedHeap::into_cset_dirty_card_queue_set())
@@ -288,9 +289,6 @@ void G1RemSet::oops_into_collection_set_do(OopsInHeapRegionClosure* oc,
#if CARD_REPEAT_HISTO
ct_freq_update_histo_and_reset();
#endif
- if (worker_i == 0) {
- _cg1r->clear_and_record_card_counts();
- }
// We cache the value of 'oc' closure into the appropriate slot in the
// _cset_rs_update_cl for this worker
@@ -396,7 +394,7 @@ public:
// RSet updating,
// * the post-write barrier shouldn't be logging updates to young
// regions (but there is a situation where this can happen - see
- // the comment in G1RemSet::concurrentRefineOneCard below -
+ // the comment in G1RemSet::refine_card() below -
// that should not be applicable here), and
// * during actual RSet updating, the filtering of cards in young
// regions in HeapRegion::oops_on_card_seq_iterate_careful is
@@ -502,8 +500,6 @@ void G1RemSet::scrub_par(BitMap* region_bm, BitMap* card_bm,
claim_val);
}
-
-
G1TriggerClosure::G1TriggerClosure() :
_triggered(false) { }
@@ -524,13 +520,91 @@ G1UpdateRSOrPushRefOopClosure(G1CollectedHeap* g1h,
_record_refs_into_cset(record_refs_into_cset),
_push_ref_cl(push_ref_cl), _worker_i(worker_i) { }
-bool G1RemSet::concurrentRefineOneCard_impl(jbyte* card_ptr, int worker_i,
- bool check_for_refs_into_cset) {
+// Returns true if the given card contains references that point
+// into the collection set, if we're checking for such references;
+// false otherwise.
+
+bool G1RemSet::refine_card(jbyte* card_ptr, int worker_i,
+ bool check_for_refs_into_cset) {
+
+ // If the card is no longer dirty, nothing to do.
+ if (*card_ptr != CardTableModRefBS::dirty_card_val()) {
+ // No need to return that this card contains refs that point
+ // into the collection set.
+ return false;
+ }
+
// Construct the region representing the card.
HeapWord* start = _ct_bs->addr_for(card_ptr);
// And find the region containing it.
HeapRegion* r = _g1->heap_region_containing(start);
- assert(r != NULL, "unexpected null");
+ if (r == NULL) {
+ // Again no need to return that this card contains refs that
+ // point into the collection set.
+ return false; // Not in the G1 heap (might be in perm, for example.)
+ }
+
+ // Why do we have to check here whether a card is on a young region,
+ // given that we dirty young regions and, as a result, the
+ // post-barrier is supposed to filter them out and never to enqueue
+ // them? When we allocate a new region as the "allocation region" we
+ // actually dirty its cards after we release the lock, since card
+ // dirtying while holding the lock was a performance bottleneck. So,
+ // as a result, it is possible for other threads to actually
+ // allocate objects in the region (after the acquire the lock)
+ // before all the cards on the region are dirtied. This is unlikely,
+ // and it doesn't happen often, but it can happen. So, the extra
+ // check below filters out those cards.
+ if (r->is_young()) {
+ return false;
+ }
+
+ // While we are processing RSet buffers during the collection, we
+ // actually don't want to scan any cards on the collection set,
+ // since we don't want to update remebered sets with entries that
+ // point into the collection set, given that live objects from the
+ // collection set are about to move and such entries will be stale
+ // very soon. This change also deals with a reliability issue which
+ // involves scanning a card in the collection set and coming across
+ // an array that was being chunked and looking malformed. Note,
+ // however, that if evacuation fails, we have to scan any objects
+ // that were not moved and create any missing entries.
+ if (r->in_collection_set()) {
+ return false;
+ }
+
+ // The result from the hot card cache insert call is either:
+ // * pointer to the current card
+ // (implying that the current card is not 'hot'),
+ // * null
+ // (meaning we had inserted the card ptr into the "hot" card cache,
+ // which had some headroom),
+ // * a pointer to a "hot" card that was evicted from the "hot" cache.
+ //
+
+ G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
+ if (hot_card_cache->use_cache()) {
+ assert(!check_for_refs_into_cset, "sanity");
+ assert(!SafepointSynchronize::is_at_safepoint(), "sanity");
+
+ card_ptr = hot_card_cache->insert(card_ptr);
+ if (card_ptr == NULL) {
+ // There was no eviction. Nothing to do.
+ return false;
+ }
+
+ start = _ct_bs->addr_for(card_ptr);
+ r = _g1->heap_region_containing(start);
+ if (r == NULL) {
+ // Not in the G1 heap
+ return false;
+ }
+
+ // Checking whether the region we got back from the cache
+ // is young here is inappropriate. The region could have been
+ // freed, reallocated and tagged as young while in the cache.
+ // Hence we could see its young type change at any time.
+ }
// Don't use addr_for(card_ptr + 1) which can ask for
// a card beyond the heap. This is not safe without a perm
@@ -610,140 +684,17 @@ bool G1RemSet::concurrentRefineOneCard_impl(jbyte* card_ptr, int worker_i,
_conc_refine_cards++;
}
- return trigger_cl.triggered();
-}
-
-bool G1RemSet::concurrentRefineOneCard(jbyte* card_ptr, int worker_i,
- bool check_for_refs_into_cset) {
- // If the card is no longer dirty, nothing to do.
- if (*card_ptr != CardTableModRefBS::dirty_card_val()) {
- // No need to return that this card contains refs that point
- // into the collection set.
- return false;
- }
-
- // Construct the region representing the card.
- HeapWord* start = _ct_bs->addr_for(card_ptr);
- // And find the region containing it.
- HeapRegion* r = _g1->heap_region_containing(start);
- if (r == NULL) {
- // Again no need to return that this card contains refs that
- // point into the collection set.
- return false; // Not in the G1 heap (might be in perm, for example.)
- }
- // Why do we have to check here whether a card is on a young region,
- // given that we dirty young regions and, as a result, the
- // post-barrier is supposed to filter them out and never to enqueue
- // them? When we allocate a new region as the "allocation region" we
- // actually dirty its cards after we release the lock, since card
- // dirtying while holding the lock was a performance bottleneck. So,
- // as a result, it is possible for other threads to actually
- // allocate objects in the region (after the acquire the lock)
- // before all the cards on the region are dirtied. This is unlikely,
- // and it doesn't happen often, but it can happen. So, the extra
- // check below filters out those cards.
- if (r->is_young()) {
- return false;
- }
- // While we are processing RSet buffers during the collection, we
- // actually don't want to scan any cards on the collection set,
- // since we don't want to update remebered sets with entries that
- // point into the collection set, given that live objects from the
- // collection set are about to move and such entries will be stale
- // very soon. This change also deals with a reliability issue which
- // involves scanning a card in the collection set and coming across
- // an array that was being chunked and looking malformed. Note,
- // however, that if evacuation fails, we have to scan any objects
- // that were not moved and create any missing entries.
- if (r->in_collection_set()) {
- return false;
- }
+ // This gets set to true if the card being refined has
+ // references that point into the collection set.
+ bool has_refs_into_cset = trigger_cl.triggered();
- // Should we defer processing the card?
- //
- // Previously the result from the insert_cache call would be
- // either card_ptr (implying that card_ptr was currently "cold"),
- // null (meaning we had inserted the card ptr into the "hot"
- // cache, which had some headroom), or a "hot" card ptr
- // extracted from the "hot" cache.
- //
- // Now that the _card_counts cache in the ConcurrentG1Refine
- // instance is an evicting hash table, the result we get back
- // could be from evicting the card ptr in an already occupied
- // bucket (in which case we have replaced the card ptr in the
- // bucket with card_ptr and "defer" is set to false). To avoid
- // having a data structure (updates to which would need a lock)
- // to hold these unprocessed dirty cards, we need to immediately
- // process card_ptr. The actions needed to be taken on return
- // from cache_insert are summarized in the following table:
- //
- // res defer action
- // --------------------------------------------------------------
- // null false card evicted from _card_counts & replaced with
- // card_ptr; evicted ptr added to hot cache.
- // No need to process res; immediately process card_ptr
- //
- // null true card not evicted from _card_counts; card_ptr added
- // to hot cache.
- // Nothing to do.
- //
- // non-null false card evicted from _card_counts & replaced with
- // card_ptr; evicted ptr is currently "cold" or
- // caused an eviction from the hot cache.
- // Immediately process res; process card_ptr.
- //
- // non-null true card not evicted from _card_counts; card_ptr is
- // currently cold, or caused an eviction from hot
- // cache.
- // Immediately process res; no need to process card_ptr.
-
-
- jbyte* res = card_ptr;
- bool defer = false;
-
- // This gets set to true if the card being refined has references
- // that point into the collection set.
- bool oops_into_cset = false;
-
- if (_cg1r->use_cache()) {
- jbyte* res = _cg1r->cache_insert(card_ptr, &defer);
- if (res != NULL && (res != card_ptr || defer)) {
- start = _ct_bs->addr_for(res);
- r = _g1->heap_region_containing(start);
- if (r != NULL) {
- // Checking whether the region we got back from the cache
- // is young here is inappropriate. The region could have been
- // freed, reallocated and tagged as young while in the cache.
- // Hence we could see its young type change at any time.
- //
- // Process card pointer we get back from the hot card cache. This
- // will check whether the region containing the card is young
- // _after_ checking that the region has been allocated from.
- oops_into_cset = concurrentRefineOneCard_impl(res, worker_i,
- false /* check_for_refs_into_cset */);
- // The above call to concurrentRefineOneCard_impl is only
- // performed if the hot card cache is enabled. This cache is
- // disabled during an evacuation pause - which is the only
- // time when we need know if the card contains references
- // that point into the collection set. Also when the hot card
- // cache is enabled, this code is executed by the concurrent
- // refine threads - rather than the GC worker threads - and
- // concurrentRefineOneCard_impl will return false.
- assert(!oops_into_cset, "should not see true here");
- }
- }
- }
-
- if (!defer) {
- oops_into_cset =
- concurrentRefineOneCard_impl(card_ptr, worker_i, check_for_refs_into_cset);
- // We should only be detecting that the card contains references
- // that point into the collection set if the current thread is
- // a GC worker thread.
- assert(!oops_into_cset || SafepointSynchronize::is_at_safepoint(),
+ // We should only be detecting that the card contains references
+ // that point into the collection set if the current thread is
+ // a GC worker thread.
+ assert(!has_refs_into_cset || SafepointSynchronize::is_at_safepoint(),
"invalid result at non safepoint");
- }
- return oops_into_cset;
+
+ return has_refs_into_cset;
}
class HRRSStatsIter: public HeapRegionClosure {
@@ -846,13 +797,16 @@ void G1RemSet::prepare_for_verify() {
DirtyCardQueueSet& dcqs = JavaThread::dirty_card_queue_set();
dcqs.concatenate_logs();
}
- bool cg1r_use_cache = _cg1r->use_cache();
- _cg1r->set_use_cache(false);
+
+ G1HotCardCache* hot_card_cache = _cg1r->hot_card_cache();
+ bool use_hot_card_cache = hot_card_cache->use_cache();
+ hot_card_cache->set_use_cache(false);
+
DirtyCardQueue into_cset_dcq(&_g1->into_cset_dirty_card_queue_set());
updateRS(&into_cset_dcq, 0);
_g1->into_cset_dirty_card_queue_set().clear();
- _cg1r->set_use_cache(cg1r_use_cache);
+ hot_card_cache->set_use_cache(use_hot_card_cache);
assert(JavaThread::dirty_card_queue_set().completed_buffers_num() == 0, "All should be consumed");
}
}
diff --git a/src/share/vm/gc_implementation/g1/g1RemSet.hpp b/src/share/vm/gc_implementation/g1/g1RemSet.hpp
index eee6b4470..7444ae819 100644
--- a/src/share/vm/gc_implementation/g1/g1RemSet.hpp
+++ b/src/share/vm/gc_implementation/g1/g1RemSet.hpp
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2001, 2012, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2001, 2013, Oracle and/or its affiliates. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
@@ -66,14 +66,6 @@ protected:
// references into the collection set.
OopsInHeapRegionClosure** _cset_rs_update_cl;
- // The routine that performs the actual work of refining a dirty
- // card.
- // If check_for_refs_into_refs is true then a true result is returned
- // if the card contains oops that have references into the current
- // collection set.
- bool concurrentRefineOneCard_impl(jbyte* card_ptr, int worker_i,
- bool check_for_refs_into_cset);
-
public:
// This is called to reset dual hash tables after the gc pause
// is finished and the initial hash table is no longer being
@@ -90,8 +82,7 @@ public:
// function can be helpful in partitioning the work to be done. It
// should be the same as the "i" passed to the calling thread's
// work(i) function. In the sequential case this param will be ingored.
- void oops_into_collection_set_do(OopsInHeapRegionClosure* blk,
- int worker_i);
+ void oops_into_collection_set_do(OopsInHeapRegionClosure* blk, int worker_i);
// Prepare for and cleanup after an oops_into_collection_set_do
// call. Must call each of these once before and after (in sequential
@@ -124,14 +115,13 @@ public:
void scrub_par(BitMap* region_bm, BitMap* card_bm,
uint worker_num, int claim_val);
- // Refine the card corresponding to "card_ptr". If "sts" is non-NULL,
- // join and leave around parts that must be atomic wrt GC. (NULL means
- // being done at a safepoint.)
+ // Refine the card corresponding to "card_ptr".
// If check_for_refs_into_cset is true, a true result is returned
// if the given card contains oops that have references into the
// current collection set.
- virtual bool concurrentRefineOneCard(jbyte* card_ptr, int worker_i,
- bool check_for_refs_into_cset);
+ virtual bool refine_card(jbyte* card_ptr,
+ int worker_i,
+ bool check_for_refs_into_cset);
// Print any relevant summary info.
virtual void print_summary_info();
diff --git a/src/share/vm/gc_implementation/g1/g1_globals.hpp b/src/share/vm/gc_implementation/g1/g1_globals.hpp
index 8bd099bd4..7e62b70cd 100644
--- a/src/share/vm/gc_implementation/g1/g1_globals.hpp
+++ b/src/share/vm/gc_implementation/g1/g1_globals.hpp
@@ -163,16 +163,12 @@
"Select green, yellow and red zones adaptively to meet the " \
"the pause requirements.") \
\
- develop(intx, G1ConcRSLogCacheSize, 10, \
+ product(uintx, G1ConcRSLogCacheSize, 10, \
"Log base 2 of the length of conc RS hot-card cache.") \
\
- develop(intx, G1ConcRSHotCardLimit, 4, \
+ product(uintx, G1ConcRSHotCardLimit, 4, \
"The threshold that defines (>=) a hot card.") \
\
- develop(intx, G1MaxHotCardCountSizePercent, 25, \
- "The maximum size of the hot card count cache as a " \
- "percentage of the number of cards for the maximum heap.") \
- \
develop(bool, G1PrintOopAppls, false, \
"When true, print applications of closures to external locs.") \
\
@@ -247,10 +243,6 @@
"If non-0 is the number of parallel rem set update threads, " \
"otherwise the value is determined ergonomically.") \
\
- develop(intx, G1CardCountCacheExpandThreshold, 16, \
- "Expand the card count cache if the number of collisions for " \
- "a particular entry exceeds this value.") \
- \
develop(bool, G1VerifyCTCleanup, false, \
"Verify card table cleanup.") \
\
diff --git a/src/share/vm/gc_implementation/g1/heapRegionSeq.cpp b/src/share/vm/gc_implementation/g1/heapRegionSeq.cpp
index fcee7cb35..dade3dfdf 100644
--- a/src/share/vm/gc_implementation/g1/heapRegionSeq.cpp
+++ b/src/share/vm/gc_implementation/g1/heapRegionSeq.cpp
@@ -124,11 +124,11 @@ MemRegion HeapRegionSeq::expand_by(HeapWord* old_end,
}
assert(_regions[index] == NULL, "invariant");
_regions[index] = new_hr;
- increment_length(&_allocated_length);
+ increment_allocated_length();
}
// Have to increment the length first, otherwise we will get an
// assert failure at(index) below.
- increment_length(&_length);
+ increment_length();
HeapRegion* hr = at(index);
list->add_as_tail(hr);
@@ -201,45 +201,29 @@ void HeapRegionSeq::iterate_from(HeapRegion* hr, HeapRegionClosure* blk) const {
}
}
-MemRegion HeapRegionSeq::shrink_by(size_t shrink_bytes,
- uint* num_regions_deleted) {
+uint HeapRegionSeq::shrink_by(uint num_regions_to_remove) {
// Reset this in case it's currently pointing into the regions that
// we just removed.
_next_search_index = 0;
- assert(shrink_bytes % os::vm_page_size() == 0, "unaligned");
- assert(shrink_bytes % HeapRegion::GrainBytes == 0, "unaligned");
assert(length() > 0, "the region sequence should not be empty");
assert(length() <= _allocated_length, "invariant");
assert(_allocated_length > 0, "we should have at least one region committed");
+ assert(num_regions_to_remove < length(), "We should never remove all regions");
- // around the loop, i will be the next region to be removed
- uint i = length() - 1;
- assert(i > 0, "we should never remove all regions");
- // [last_start, end) is the MemRegion that covers the regions we will remove.
- HeapWord* end = at(i)->end();
- HeapWord* last_start = end;
- *num_regions_deleted = 0;
- while (shrink_bytes > 0) {
- HeapRegion* cur = at(i);
- // We should leave the humongous regions where they are.
- if (cur->isHumongous()) break;
- // We should stop shrinking if we come across a non-empty region.
- if (!cur->is_empty()) break;
+ uint i = 0;
+ for (; i < num_regions_to_remove; i++) {
+ HeapRegion* cur = at(length() - 1);
- i -= 1;
- *num_regions_deleted += 1;
- shrink_bytes -= cur->capacity();
- last_start = cur->bottom();
- decrement_length(&_length);
- // We will reclaim the HeapRegion. _allocated_length should be
- // covering this index. So, even though we removed the region from
- // the active set by decreasing _length, we still have it
- // available in the future if we need to re-use it.
- assert(i > 0, "we should never remove all regions");
- assert(length() > 0, "we should never remove all regions");
+ if (!cur->is_empty()) {
+ // We have to give up if the region can not be moved
+ break;
+ }
+ assert(!cur->isHumongous(), "Humongous regions should not be empty");
+
+ decrement_length();
}
- return MemRegion(last_start, end);
+ return i;
}
#ifndef PRODUCT
diff --git a/src/share/vm/gc_implementation/g1/heapRegionSeq.hpp b/src/share/vm/gc_implementation/g1/heapRegionSeq.hpp
index b1da14f6b..b7a58f76a 100644
--- a/src/share/vm/gc_implementation/g1/heapRegionSeq.hpp
+++ b/src/share/vm/gc_implementation/g1/heapRegionSeq.hpp
@@ -92,14 +92,19 @@ class HeapRegionSeq: public CHeapObj<mtGC> {
// address is valid.
inline uintx addr_to_index_biased(HeapWord* addr) const;
- void increment_length(uint* length) {
- assert(*length < _max_length, "pre-condition");
- *length += 1;
+ void increment_allocated_length() {
+ assert(_allocated_length < _max_length, "pre-condition");
+ _allocated_length++;
}
- void decrement_length(uint* length) {
- assert(*length > 0, "pre-condition");
- *length -= 1;
+ void increment_length() {
+ assert(_length < _max_length, "pre-condition");
+ _length++;
+ }
+
+ void decrement_length() {
+ assert(_length > 0, "pre-condition");
+ _length--;
}
public:
@@ -153,11 +158,9 @@ class HeapRegionSeq: public CHeapObj<mtGC> {
void iterate_from(HeapRegion* hr, HeapRegionClosure* blk) const;
// Tag as uncommitted as many regions that are completely free as
- // possible, up to shrink_bytes, from the suffix of the committed
- // sequence. Return a MemRegion that corresponds to the address
- // range of the uncommitted regions. Assume shrink_bytes is page and
- // heap region aligned.
- MemRegion shrink_by(size_t shrink_bytes, uint* num_regions_deleted);
+ // possible, up to num_regions_to_remove, from the suffix of the committed
+ // sequence. Return the actual number of removed regions.
+ uint shrink_by(uint num_regions_to_remove);
// Do some sanity checking.
void verify_optional() PRODUCT_RETURN;