提交 a921f9ad 编写于 作者: A amurillo

Merge

......@@ -77,30 +77,40 @@ ifeq ($(INCLUDE_ALL_GCS), false)
CXXFLAGS += -DINCLUDE_ALL_GCS=0
CFLAGS += -DINCLUDE_ALL_GCS=0
Src_Files_EXCLUDE += \
cmsAdaptiveSizePolicy.cpp cmsCollectorPolicy.cpp \
cmsGCAdaptivePolicyCounters.cpp cmsLockVerifier.cpp compactibleFreeListSpace.cpp \
concurrentMarkSweepGeneration.cpp concurrentMarkSweepThread.cpp \
freeChunk.cpp adaptiveFreeList.cpp promotionInfo.cpp vmCMSOperations.cpp \
collectionSetChooser.cpp concurrentG1Refine.cpp concurrentG1RefineThread.cpp \
concurrentMark.cpp concurrentMarkThread.cpp dirtyCardQueue.cpp g1AllocRegion.cpp \
g1BlockOffsetTable.cpp g1CardCounts.cpp g1CollectedHeap.cpp g1CollectorPolicy.cpp \
g1ErgoVerbose.cpp g1GCPhaseTimes.cpp g1HRPrinter.cpp g1HotCardCache.cpp g1Log.cpp \
g1MMUTracker.cpp g1MarkSweep.cpp g1MemoryPool.cpp g1MonitoringSupport.cpp g1OopClosures.cpp \
g1RemSet.cpp g1RemSetSummary.cpp g1SATBCardTableModRefBS.cpp g1StringDedup.cpp g1StringDedupStat.cpp \
g1StringDedupTable.cpp g1StringDedupThread.cpp g1StringDedupQueue.cpp g1_globals.cpp heapRegion.cpp \
g1BiasedArray.cpp heapRegionRemSet.cpp heapRegionSeq.cpp heapRegionSet.cpp heapRegionSets.cpp \
ptrQueue.cpp satbQueue.cpp sparsePRT.cpp survRateGroup.cpp vm_operations_g1.cpp g1CodeCacheRemSet.cpp \
adjoiningGenerations.cpp adjoiningVirtualSpaces.cpp asPSOldGen.cpp asPSYoungGen.cpp \
cardTableExtension.cpp gcTaskManager.cpp gcTaskThread.cpp objectStartArray.cpp \
parallelScavengeHeap.cpp parMarkBitMap.cpp pcTasks.cpp psAdaptiveSizePolicy.cpp \
psCompactionManager.cpp psGCAdaptivePolicyCounters.cpp psGenerationCounters.cpp \
psMarkSweep.cpp psMarkSweepDecorator.cpp psMemoryPool.cpp psOldGen.cpp \
psParallelCompact.cpp psPromotionLAB.cpp psPromotionManager.cpp psScavenge.cpp \
psTasks.cpp psVirtualspace.cpp psYoungGen.cpp vmPSOperations.cpp asParNewGeneration.cpp \
parCardTableModRefBS.cpp parGCAllocBuffer.cpp parNewGeneration.cpp mutableSpace.cpp \
gSpaceCounters.cpp allocationStats.cpp spaceCounters.cpp gcAdaptivePolicyCounters.cpp \
mutableNUMASpace.cpp immutableSpace.cpp yieldingWorkGroup.cpp hSpaceCounters.cpp
gc_impl := $(GAMMADIR)/src/share/vm/gc_implementation
gc_exclude := \
$(notdir $(wildcard $(gc_impl)/concurrentMarkSweep/*.cpp)) \
$(notdir $(wildcard $(gc_impl)/g1/*.cpp)) \
$(notdir $(wildcard $(gc_impl)/parallelScavenge/*.cpp)) \
$(notdir $(wildcard $(gc_impl)/parNew/*.cpp))
Src_Files_EXCLUDE += $(gc_exclude)
# Exclude everything in $(gc_impl)/shared except the files listed
# in $(gc_shared_keep).
gc_shared_all := $(notdir $(wildcard $(gc_impl)/shared/*.cpp))
gc_shared_keep := \
adaptiveSizePolicy.cpp \
ageTable.cpp \
collectorCounters.cpp \
cSpaceCounters.cpp \
gcPolicyCounters.cpp \
gcStats.cpp \
gcTimer.cpp \
gcTrace.cpp \
gcTraceSend.cpp \
gcTraceTime.cpp \
gcUtil.cpp \
generationCounters.cpp \
markSweep.cpp \
objectCountEventSender.cpp \
spaceDecorator.cpp \
vmGCOperations.cpp
Src_Files_EXCLUDE += $(filter-out $(gc_shared_keep),$(gc_shared_all))
# src/share/vm/services
Src_Files_EXCLUDE += \
g1MemoryPool.cpp \
psMemoryPool.cpp
endif
ifeq ($(INCLUDE_NMT), false)
......
......@@ -35,7 +35,7 @@ HOTSPOT_VM_COPYRIGHT=Copyright 2014
HS_MAJOR_VER=25
HS_MINOR_VER=40
HS_BUILD_NUMBER=01
HS_BUILD_NUMBER=02
JDK_MAJOR_VER=1
JDK_MINOR_VER=8
......
......@@ -138,6 +138,17 @@ ciField::ciField(ciInstanceKlass* klass, int index): _known_to_link_with_put(NUL
return;
}
// Access check based on declared_holder. canonical_holder should not be used
// to check access because it can erroneously succeed. If this check fails,
// propagate the declared holder to will_link() which in turn will bail out
// compilation for this field access.
if (!Reflection::verify_field_access(klass->get_Klass(), declared_holder->get_Klass(), canonical_holder, field_desc.access_flags(), true)) {
_holder = declared_holder;
_offset = -1;
_is_constant = false;
return;
}
assert(canonical_holder == field_desc.field_holder(), "just checking");
initialize_from(&field_desc);
}
......
......@@ -69,7 +69,12 @@ class PcDescCache VALUE_OBJ_CLASS_SPEC {
friend class VMStructs;
private:
enum { cache_size = 4 };
PcDesc* _pc_descs[cache_size]; // last cache_size pc_descs found
// The array elements MUST be volatile! Several threads may modify
// and read from the cache concurrently. find_pc_desc_internal has
// returned wrong results. C++ compiler (namely xlC12) may duplicate
// C++ field accesses if the elements are not volatile.
typedef PcDesc* PcDescPtr;
volatile PcDescPtr _pc_descs[cache_size]; // last cache_size pc_descs found
public:
PcDescCache() { debug_only(_pc_descs[0] = NULL); }
void reset_to(PcDesc* initial_pc_desc);
......
......@@ -475,7 +475,7 @@ MethodLivenessResult MethodLiveness::get_liveness_at(int entry_bci) {
bci = 0;
}
MethodLivenessResult answer((uintptr_t*)NULL,0);
MethodLivenessResult answer((BitMap::bm_word_t*)NULL,0);
if (_block_count > 0) {
if (TimeLivenessAnalysis) _time_total.start();
......@@ -1000,7 +1000,7 @@ bool MethodLiveness::BasicBlock::merge_exception(BitMap other) {
}
MethodLivenessResult MethodLiveness::BasicBlock::get_liveness_at(ciMethod* method, int bci) {
MethodLivenessResult answer(NEW_RESOURCE_ARRAY(uintptr_t, _analyzer->bit_map_size_words()),
MethodLivenessResult answer(NEW_RESOURCE_ARRAY(BitMap::bm_word_t, _analyzer->bit_map_size_words()),
_analyzer->bit_map_size_bits());
answer.set_is_valid();
......
......@@ -467,7 +467,6 @@ void OopMapSet::update_register_map(const frame *fr, RegisterMap *reg_map) {
assert(cb != NULL, "no codeblob");
// Any reg might be saved by a safepoint handler (see generate_handler_blob).
const int max_saved_on_entry_reg_count = ConcreteRegisterImpl::number_of_registers;
assert( reg_map->_update_for_id == NULL || fr->is_older(reg_map->_update_for_id),
"already updated this map; do not 'update' it twice!" );
debug_only(reg_map->_update_for_id = fr->id());
......@@ -477,27 +476,20 @@ void OopMapSet::update_register_map(const frame *fr, RegisterMap *reg_map) {
!cb->caller_must_gc_arguments(reg_map->thread())),
"include_argument_oops should already be set");
int nof_callee = 0;
oop* locs[2*max_saved_on_entry_reg_count+1];
VMReg regs[2*max_saved_on_entry_reg_count+1];
// ("+1" because max_saved_on_entry_reg_count might be zero)
// Scan through oopmap and find location of all callee-saved registers
// (we do not do update in place, since info could be overwritten)
address pc = fr->pc();
OopMap* map = cb->oop_map_for_return_address(pc);
assert(map != NULL, "no ptr map found");
DEBUG_ONLY(int nof_callee = 0;)
assert(map != NULL, " no ptr map found");
OopMapValue omv;
for(OopMapStream oms(map,OopMapValue::callee_saved_value); !oms.is_done(); oms.next()) {
omv = oms.current();
assert(nof_callee < 2*max_saved_on_entry_reg_count, "overflow");
regs[nof_callee] = omv.content_reg();
locs[nof_callee] = fr->oopmapreg_to_location(omv.reg(),reg_map);
nof_callee++;
for (OopMapStream oms(map, OopMapValue::callee_saved_value); !oms.is_done(); oms.next()) {
OopMapValue omv = oms.current();
VMReg reg = omv.content_reg();
oop* loc = fr->oopmapreg_to_location(omv.reg(), reg_map);
reg_map->set_location(reg, (address) loc);
DEBUG_ONLY(nof_callee++;)
}
// Check that runtime stubs save all callee-saved registers
......@@ -506,11 +498,6 @@ void OopMapSet::update_register_map(const frame *fr, RegisterMap *reg_map) {
(nof_callee >= SAVED_ON_ENTRY_REG_COUNT || nof_callee >= C_SAVED_ON_ENTRY_REG_COUNT),
"must save all");
#endif // COMPILER2
// Copy found callee-saved register to reg_map
for(int i = 0; i < nof_callee; i++) {
reg_map->set_location(regs[i], (address)locs[i]);
}
}
//=============================================================================
......
......@@ -158,7 +158,7 @@ void AdaptiveFreeList<Chunk>::verify_stats() const {
" coal_deaths(" SIZE_FORMAT ")"
" + count(" SSIZE_FORMAT ")",
p2i(this), size(), _allocation_stats.prev_sweep(), _allocation_stats.split_births(),
_allocation_stats.split_births(), _allocation_stats.split_deaths(),
_allocation_stats.coal_births(), _allocation_stats.split_deaths(),
_allocation_stats.coal_deaths(), count()));
}
#endif
......
......@@ -29,7 +29,7 @@
#include "gc_implementation/g1/g1HotCardCache.hpp"
#include "runtime/java.hpp"
ConcurrentG1Refine::ConcurrentG1Refine(G1CollectedHeap* g1h) :
ConcurrentG1Refine::ConcurrentG1Refine(G1CollectedHeap* g1h, CardTableEntryClosure* refine_closure) :
_threads(NULL), _n_threads(0),
_hot_card_cache(g1h)
{
......@@ -61,7 +61,7 @@ ConcurrentG1Refine::ConcurrentG1Refine(G1CollectedHeap* g1h) :
ConcurrentG1RefineThread *next = NULL;
for (uint i = _n_threads - 1; i != UINT_MAX; i--) {
ConcurrentG1RefineThread* t = new ConcurrentG1RefineThread(this, next, worker_id_offset, i);
ConcurrentG1RefineThread* t = new ConcurrentG1RefineThread(this, next, refine_closure, worker_id_offset, i);
assert(t != NULL, "Conc refine should have been created");
if (t->osthread() == NULL) {
vm_shutdown_during_initialization("Could not create ConcurrentG1RefineThread");
......
......@@ -71,7 +71,7 @@ class ConcurrentG1Refine: public CHeapObj<mtGC> {
void reset_threshold_step();
public:
ConcurrentG1Refine(G1CollectedHeap* g1h);
ConcurrentG1Refine(G1CollectedHeap* g1h, CardTableEntryClosure* refine_closure);
~ConcurrentG1Refine();
void init(); // Accomplish some initialization that has to wait.
......
......@@ -33,8 +33,10 @@
ConcurrentG1RefineThread::
ConcurrentG1RefineThread(ConcurrentG1Refine* cg1r, ConcurrentG1RefineThread *next,
CardTableEntryClosure* refine_closure,
uint worker_id_offset, uint worker_id) :
ConcurrentGCThread(),
_refine_closure(refine_closure),
_worker_id_offset(worker_id_offset),
_worker_id(worker_id),
_active(false),
......@@ -204,7 +206,7 @@ void ConcurrentG1RefineThread::run() {
if (_next != NULL && !_next->is_active() && curr_buffer_num > _next->_threshold) {
_next->activate();
}
} while (dcqs.apply_closure_to_completed_buffer(_worker_id + _worker_id_offset, cg1r()->green_zone()));
} while (dcqs.apply_closure_to_completed_buffer(_refine_closure, _worker_id + _worker_id_offset, cg1r()->green_zone()));
// We can exit the loop above while being active if there was a yield request.
if (is_active()) {
......
......@@ -28,6 +28,7 @@
#include "gc_implementation/shared/concurrentGCThread.hpp"
// Forward Decl.
class CardTableEntryClosure;
class ConcurrentG1Refine;
// The G1 Concurrent Refinement Thread (could be several in the future).
......@@ -49,6 +50,9 @@ class ConcurrentG1RefineThread: public ConcurrentGCThread {
Monitor* _monitor;
ConcurrentG1Refine* _cg1r;
// The closure applied to completed log buffers.
CardTableEntryClosure* _refine_closure;
int _thread_threshold_step;
// This thread activation threshold
int _threshold;
......@@ -68,6 +72,7 @@ public:
virtual void run();
// Constructor
ConcurrentG1RefineThread(ConcurrentG1Refine* cg1r, ConcurrentG1RefineThread* next,
CardTableEntryClosure* refine_closure,
uint worker_id_offset, uint worker_id);
void initialize();
......
......@@ -127,7 +127,7 @@ bool CMBitMap::allocate(ReservedSpace heap_rs) {
}
assert(_virtual_space.committed_size() == brs.size(),
"didn't reserve backing store for all of concurrent marking bit map?");
_bm.set_map((uintptr_t*)_virtual_space.low());
_bm.set_map((BitMap::bm_word_t*)_virtual_space.low());
assert(_virtual_space.committed_size() << (_shifter + LogBitsPerByte) >=
_bmWordSize, "inconsistency in bit map sizing");
_bm.set_size(_bmWordSize >> _shifter);
......
......@@ -70,7 +70,7 @@ bool DirtyCardQueue::apply_closure_to_buffer(CardTableEntryClosure* cl,
DirtyCardQueueSet::DirtyCardQueueSet(bool notify_when_complete) :
PtrQueueSet(notify_when_complete),
_closure(NULL),
_mut_process_closure(NULL),
_shared_dirty_card_queue(this, true /*perm*/),
_free_ids(NULL),
_processed_buffers_mut(0), _processed_buffers_rs_thread(0)
......@@ -83,10 +83,11 @@ uint DirtyCardQueueSet::num_par_ids() {
return (uint)os::processor_count();
}
void DirtyCardQueueSet::initialize(Monitor* cbl_mon, Mutex* fl_lock,
void DirtyCardQueueSet::initialize(CardTableEntryClosure* cl, Monitor* cbl_mon, Mutex* fl_lock,
int process_completed_threshold,
int max_completed_queue,
Mutex* lock, PtrQueueSet* fl_owner) {
_mut_process_closure = cl;
PtrQueueSet::initialize(cbl_mon, fl_lock, process_completed_threshold,
max_completed_queue, fl_owner);
set_buffer_size(G1UpdateBufferSize);
......@@ -98,18 +99,15 @@ void DirtyCardQueueSet::handle_zero_index_for_thread(JavaThread* t) {
t->dirty_card_queue().handle_zero_index();
}
void DirtyCardQueueSet::set_closure(CardTableEntryClosure* closure) {
_closure = closure;
}
void DirtyCardQueueSet::iterate_closure_all_threads(bool consume,
void DirtyCardQueueSet::iterate_closure_all_threads(CardTableEntryClosure* cl,
bool consume,
uint worker_i) {
assert(SafepointSynchronize::is_at_safepoint(), "Must be at safepoint.");
for(JavaThread* t = Threads::first(); t; t = t->next()) {
bool b = t->dirty_card_queue().apply_closure(_closure, consume);
bool b = t->dirty_card_queue().apply_closure(cl, consume);
guarantee(b, "Should not be interrupted.");
}
bool b = shared_dirty_card_queue()->apply_closure(_closure,
bool b = shared_dirty_card_queue()->apply_closure(cl,
consume,
worker_i);
guarantee(b, "Should not be interrupted.");
......@@ -143,7 +141,7 @@ bool DirtyCardQueueSet::mut_process_buffer(void** buf) {
bool b = false;
if (worker_i != UINT_MAX) {
b = DirtyCardQueue::apply_closure_to_buffer(_closure, buf, 0,
b = DirtyCardQueue::apply_closure_to_buffer(_mut_process_closure, buf, 0,
_sz, true, worker_i);
if (b) Atomic::inc(&_processed_buffers_mut);
......@@ -218,18 +216,11 @@ bool DirtyCardQueueSet::apply_closure_to_completed_buffer(CardTableEntryClosure*
return res;
}
bool DirtyCardQueueSet::apply_closure_to_completed_buffer(uint worker_i,
int stop_at,
bool during_pause) {
return apply_closure_to_completed_buffer(_closure, worker_i,
stop_at, during_pause);
}
void DirtyCardQueueSet::apply_closure_to_all_completed_buffers() {
void DirtyCardQueueSet::apply_closure_to_all_completed_buffers(CardTableEntryClosure* cl) {
BufferNode* nd = _completed_buffers_head;
while (nd != NULL) {
bool b =
DirtyCardQueue::apply_closure_to_buffer(_closure,
DirtyCardQueue::apply_closure_to_buffer(cl,
BufferNode::make_buffer_from_node(nd),
0, _sz, false);
guarantee(b, "Should not stop early.");
......@@ -237,6 +228,24 @@ void DirtyCardQueueSet::apply_closure_to_all_completed_buffers() {
}
}
void DirtyCardQueueSet::par_apply_closure_to_all_completed_buffers(CardTableEntryClosure* cl) {
BufferNode* nd = _cur_par_buffer_node;
while (nd != NULL) {
BufferNode* next = (BufferNode*)nd->next();
BufferNode* actual = (BufferNode*)Atomic::cmpxchg_ptr((void*)next, (volatile void*)&_cur_par_buffer_node, (void*)nd);
if (actual == nd) {
bool b =
DirtyCardQueue::apply_closure_to_buffer(cl,
BufferNode::make_buffer_from_node(actual),
0, _sz, false);
guarantee(b, "Should not stop early.");
nd = next;
} else {
nd = actual;
}
}
}
// Deallocates any completed log buffers
void DirtyCardQueueSet::clear() {
BufferNode* buffers_to_delete = NULL;
......
......@@ -73,7 +73,8 @@ public:
class DirtyCardQueueSet: public PtrQueueSet {
CardTableEntryClosure* _closure;
// The closure used in mut_process_buffer().
CardTableEntryClosure* _mut_process_closure;
DirtyCardQueue _shared_dirty_card_queue;
......@@ -88,10 +89,12 @@ class DirtyCardQueueSet: public PtrQueueSet {
jint _processed_buffers_mut;
jint _processed_buffers_rs_thread;
// Current buffer node used for parallel iteration.
BufferNode* volatile _cur_par_buffer_node;
public:
DirtyCardQueueSet(bool notify_when_complete = true);
void initialize(Monitor* cbl_mon, Mutex* fl_lock,
void initialize(CardTableEntryClosure* cl, Monitor* cbl_mon, Mutex* fl_lock,
int process_completed_threshold,
int max_completed_queue,
Mutex* lock, PtrQueueSet* fl_owner = NULL);
......@@ -102,32 +105,14 @@ public:
static void handle_zero_index_for_thread(JavaThread* t);
// Register "blk" as "the closure" for all queues. Only one such closure
// is allowed. The "apply_closure_to_completed_buffer" method will apply
// this closure to a completed buffer, and "iterate_closure_all_threads"
// applies it to partially-filled buffers (the latter should only be done
// with the world stopped).
void set_closure(CardTableEntryClosure* closure);
// If there is a registered closure for buffers, apply it to all entries
// in all currently-active buffers. This should only be applied at a
// safepoint. (Currently must not be called in parallel; this should
// change in the future.) If "consume" is true, processed entries are
// discarded.
void iterate_closure_all_threads(bool consume = true,
// Apply the given closure to all entries in all currently-active buffers.
// This should only be applied at a safepoint. (Currently must not be called
// in parallel; this should change in the future.) If "consume" is true,
// processed entries are discarded.
void iterate_closure_all_threads(CardTableEntryClosure* cl,
bool consume = true,
uint worker_i = 0);
// If there exists some completed buffer, pop it, then apply the
// registered closure to all its elements, nulling out those elements
// processed. If all elements are processed, returns "true". If no
// completed buffers exist, returns false. If a completed buffer exists,
// but is only partially completed before a "yield" happens, the
// partially completed buffer (with its processed elements set to NULL)
// is returned to the completed buffer set, and this call returns false.
bool apply_closure_to_completed_buffer(uint worker_i = 0,
int stop_at = 0,
bool during_pause = false);
// If there exists some completed buffer, pop it, then apply the
// specified closure to all its elements, nulling out those elements
// processed. If all elements are processed, returns "true". If no
......@@ -149,7 +134,12 @@ public:
// Applies the current closure to all completed buffers,
// non-consumptively.
void apply_closure_to_all_completed_buffers();
void apply_closure_to_all_completed_buffers(CardTableEntryClosure* cl);
void reset_for_par_iteration() { _cur_par_buffer_node = _completed_buffers_head; }
// Applies the current closure to all completed buffers, non-consumptively.
// Parallel version.
void par_apply_closure_to_all_completed_buffers(CardTableEntryClosure* cl);
DirtyCardQueue* shared_dirty_card_queue() {
return &_shared_dirty_card_queue;
......
......@@ -47,32 +47,27 @@ void G1CodeRootChunk::nmethods_do(CodeBlobClosure* cl) {
}
}
FreeList<G1CodeRootChunk> G1CodeRootSet::_free_list;
size_t G1CodeRootSet::_num_chunks_handed_out = 0;
G1CodeRootChunk* G1CodeRootSet::new_chunk() {
G1CodeRootChunk* result = _free_list.get_chunk_at_head();
if (result == NULL) {
result = new G1CodeRootChunk();
}
G1CodeRootSet::_num_chunks_handed_out++;
result->reset();
return result;
G1CodeRootChunkManager::G1CodeRootChunkManager() : _free_list(), _num_chunks_handed_out(0) {
_free_list.initialize();
_free_list.set_size(G1CodeRootChunk::word_size());
}
void G1CodeRootSet::free_chunk(G1CodeRootChunk* chunk) {
_free_list.return_chunk_at_head(chunk);
G1CodeRootSet::_num_chunks_handed_out--;
size_t G1CodeRootChunkManager::fl_mem_size() {
return _free_list.count() * _free_list.size();
}
void G1CodeRootSet::free_all_chunks(FreeList<G1CodeRootChunk>* list) {
G1CodeRootSet::_num_chunks_handed_out -= list->count();
void G1CodeRootChunkManager::free_all_chunks(FreeList<G1CodeRootChunk>* list) {
_num_chunks_handed_out -= list->count();
_free_list.prepend(list);
}
void G1CodeRootSet::purge_chunks(size_t keep_ratio) {
size_t keep = G1CodeRootSet::_num_chunks_handed_out * keep_ratio / 100;
void G1CodeRootChunkManager::free_chunk(G1CodeRootChunk* chunk) {
_free_list.return_chunk_at_head(chunk);
_num_chunks_handed_out--;
}
void G1CodeRootChunkManager::purge_chunks(size_t keep_ratio) {
size_t keep = _num_chunks_handed_out * keep_ratio / 100;
if (keep >= (size_t)_free_list.count()) {
return;
}
......@@ -90,20 +85,51 @@ void G1CodeRootSet::purge_chunks(size_t keep_ratio) {
}
}
size_t G1CodeRootSet::static_mem_size() {
return sizeof(_free_list) + sizeof(_num_chunks_handed_out);
size_t G1CodeRootChunkManager::static_mem_size() {
return sizeof(G1CodeRootChunkManager);
}
size_t G1CodeRootSet::fl_mem_size() {
return _free_list.count() * _free_list.size();
G1CodeRootChunk* G1CodeRootChunkManager::new_chunk() {
G1CodeRootChunk* result = _free_list.get_chunk_at_head();
if (result == NULL) {
result = new G1CodeRootChunk();
}
_num_chunks_handed_out++;
result->reset();
return result;
}
void G1CodeRootSet::initialize() {
_free_list.initialize();
_free_list.set_size(G1CodeRootChunk::word_size());
#ifndef PRODUCT
size_t G1CodeRootChunkManager::num_chunks_handed_out() const {
return _num_chunks_handed_out;
}
size_t G1CodeRootChunkManager::num_free_chunks() const {
return (size_t)_free_list.count();
}
#endif
G1CodeRootChunkManager G1CodeRootSet::_default_chunk_manager;
void G1CodeRootSet::purge_chunks(size_t keep_ratio) {
_default_chunk_manager.purge_chunks(keep_ratio);
}
G1CodeRootSet::G1CodeRootSet() : _list(), _length(0) {
size_t G1CodeRootSet::free_chunks_static_mem_size() {
return _default_chunk_manager.static_mem_size();
}
size_t G1CodeRootSet::free_chunks_mem_size() {
return _default_chunk_manager.fl_mem_size();
}
G1CodeRootSet::G1CodeRootSet(G1CodeRootChunkManager* manager) : _manager(manager), _list(), _length(0) {
if (_manager == NULL) {
_manager = &_default_chunk_manager;
}
_list.initialize();
_list.set_size(G1CodeRootChunk::word_size());
}
......@@ -189,28 +215,38 @@ void G1CodeRootSet::nmethods_do(CodeBlobClosure* blk) const {
}
}
size_t G1CodeRootSet::static_mem_size() {
return sizeof(G1CodeRootSet);
}
size_t G1CodeRootSet::mem_size() {
return sizeof(this) + _list.count() * _list.size();
return G1CodeRootSet::static_mem_size() + _list.count() * _list.size();
}
#ifndef PRODUCT
void G1CodeRootSet::test() {
initialize();
G1CodeRootChunkManager mgr;
assert(mgr.num_chunks_handed_out() == 0, "Must not have handed out chunks yet");
assert(_free_list.count() == 0, "Free List must be empty");
assert(_num_chunks_handed_out == 0, "No elements must have been handed out yet");
assert(G1CodeRootChunkManager::static_mem_size() > sizeof(void*),
err_msg("The chunk manager's static memory usage seems too small, is only "SIZE_FORMAT" bytes.", G1CodeRootChunkManager::static_mem_size()));
// The number of chunks that we allocate for purge testing.
size_t const num_chunks = 10;
{
G1CodeRootSet set1;
G1CodeRootSet set1(&mgr);
assert(set1.is_empty(), "Code root set must be initially empty but is not.");
assert(G1CodeRootSet::static_mem_size() > sizeof(void*),
err_msg("The code root set's static memory usage seems too small, is only "SIZE_FORMAT" bytes", G1CodeRootSet::static_mem_size()));
set1.add((nmethod*)1);
assert(_num_chunks_handed_out == 1,
assert(mgr.num_chunks_handed_out() == 1,
err_msg("Must have allocated and handed out one chunk, but handed out "
SIZE_FORMAT" chunks", _num_chunks_handed_out));
SIZE_FORMAT" chunks", mgr.num_chunks_handed_out()));
assert(set1.length() == 1, err_msg("Added exactly one element, but set contains "
SIZE_FORMAT" elements", set1.length()));
......@@ -219,19 +255,19 @@ void G1CodeRootSet::test() {
for (uint i = 0; i < G1CodeRootChunk::word_size() + 1; i++) {
set1.add((nmethod*)1);
}
assert(_num_chunks_handed_out == 1,
assert(mgr.num_chunks_handed_out() == 1,
err_msg("Duplicate detection must have prevented allocation of further "
"chunks but contains "SIZE_FORMAT, _num_chunks_handed_out));
"chunks but allocated "SIZE_FORMAT, mgr.num_chunks_handed_out()));
assert(set1.length() == 1,
err_msg("Duplicate detection should not have increased the set size but "
"is "SIZE_FORMAT, set1.length()));
size_t num_total_after_add = G1CodeRootChunk::word_size() + 1;
for (size_t i = 0; i < num_total_after_add - 1; i++) {
set1.add((nmethod*)(2 + i));
set1.add((nmethod*)(uintptr_t)(2 + i));
}
assert(_num_chunks_handed_out > 1,
"After adding more code roots, more than one chunks should have been handed out");
assert(mgr.num_chunks_handed_out() > 1,
"After adding more code roots, more than one additional chunk should have been handed out");
assert(set1.length() == num_total_after_add,
err_msg("After adding in total "SIZE_FORMAT" distinct code roots, they "
"need to be in the set, but there are only "SIZE_FORMAT,
......@@ -244,27 +280,27 @@ void G1CodeRootSet::test() {
assert(num_popped == num_total_after_add,
err_msg("Managed to pop "SIZE_FORMAT" code roots, but only "SIZE_FORMAT" "
"were added", num_popped, num_total_after_add));
assert(_num_chunks_handed_out == 0,
assert(mgr.num_chunks_handed_out() == 0,
err_msg("After popping all elements, all chunks must have been returned "
"but are still "SIZE_FORMAT, _num_chunks_handed_out));
"but there are still "SIZE_FORMAT" additional", mgr.num_chunks_handed_out()));
purge_chunks(0);
assert(_free_list.count() == 0,
mgr.purge_chunks(0);
assert(mgr.num_free_chunks() == 0,
err_msg("After purging everything, the free list must be empty but still "
"contains "SIZE_FORMAT" chunks", _free_list.count()));
"contains "SIZE_FORMAT" chunks", mgr.num_free_chunks()));
// Add some more handed out chunks.
size_t i = 0;
while (_num_chunks_handed_out < num_chunks) {
while (mgr.num_chunks_handed_out() < num_chunks) {
set1.add((nmethod*)i);
i++;
}
{
// Generate chunks on the free list.
G1CodeRootSet set2;
G1CodeRootSet set2(&mgr);
size_t i = 0;
while (_num_chunks_handed_out < num_chunks * 2) {
while (mgr.num_chunks_handed_out() < (num_chunks * 2)) {
set2.add((nmethod*)i);
i++;
}
......@@ -272,45 +308,45 @@ void G1CodeRootSet::test() {
// num_chunks elements on the free list.
}
assert(_num_chunks_handed_out == num_chunks,
assert(mgr.num_chunks_handed_out() == num_chunks,
err_msg("Deletion of the second set must have resulted in giving back "
"those, but there is still "SIZE_FORMAT" handed out, expecting "
SIZE_FORMAT, _num_chunks_handed_out, num_chunks));
assert((size_t)_free_list.count() == num_chunks,
"those, but there are still "SIZE_FORMAT" additional handed out, expecting "
SIZE_FORMAT, mgr.num_chunks_handed_out(), num_chunks));
assert(mgr.num_free_chunks() == num_chunks,
err_msg("After freeing "SIZE_FORMAT" chunks, they must be on the free list "
"but there are only "SIZE_FORMAT, num_chunks, _free_list.count()));
"but there are only "SIZE_FORMAT, num_chunks, mgr.num_free_chunks()));
size_t const test_percentage = 50;
purge_chunks(test_percentage);
assert(_num_chunks_handed_out == num_chunks,
mgr.purge_chunks(test_percentage);
assert(mgr.num_chunks_handed_out() == num_chunks,
err_msg("Purging must not hand out chunks but there are "SIZE_FORMAT,
_num_chunks_handed_out));
assert((size_t)_free_list.count() == (ssize_t)(num_chunks * test_percentage / 100),
mgr.num_chunks_handed_out()));
assert(mgr.num_free_chunks() == (size_t)(mgr.num_chunks_handed_out() * test_percentage / 100),
err_msg("Must have purged "SIZE_FORMAT" percent of "SIZE_FORMAT" chunks"
"but there are "SSIZE_FORMAT, test_percentage, num_chunks,
_free_list.count()));
"but there are "SIZE_FORMAT, test_percentage, num_chunks,
mgr.num_free_chunks()));
// Purge the remainder of the chunks on the free list.
purge_chunks(0);
assert(_free_list.count() == 0, "Free List must be empty");
assert(_num_chunks_handed_out == num_chunks,
mgr.purge_chunks(0);
assert(mgr.num_free_chunks() == 0, "Free List must be empty");
assert(mgr.num_chunks_handed_out() == num_chunks,
err_msg("Expected to be "SIZE_FORMAT" chunks handed out from the first set "
"but there are "SIZE_FORMAT, num_chunks, _num_chunks_handed_out));
"but there are "SIZE_FORMAT, num_chunks, mgr.num_chunks_handed_out()));
// Exit of the scope of the set1 object will call the destructor that generates
// num_chunks additional elements on the free list.
}
}
assert(_num_chunks_handed_out == 0,
assert(mgr.num_chunks_handed_out() == 0,
err_msg("Deletion of the only set must have resulted in no chunks handed "
"out, but there is still "SIZE_FORMAT" handed out", _num_chunks_handed_out));
assert((size_t)_free_list.count() == num_chunks,
"out, but there is still "SIZE_FORMAT" handed out", mgr.num_chunks_handed_out()));
assert(mgr.num_free_chunks() == num_chunks,
err_msg("After freeing "SIZE_FORMAT" chunks, they must be on the free list "
"but there are only "SSIZE_FORMAT, num_chunks, _free_list.count()));
"but there are only "SIZE_FORMAT, num_chunks, mgr.num_free_chunks()));
// Restore initial state.
purge_chunks(0);
assert(_free_list.count() == 0, "Free List must be empty");
assert(_num_chunks_handed_out == 0, "No elements must have been handed out yet");
mgr.purge_chunks(0);
assert(mgr.num_free_chunks() == 0, "Free List must be empty");
assert(mgr.num_chunks_handed_out() == 0, "No additional elements must have been handed out yet");
}
void TestCodeCacheRemSet_test() {
......
......@@ -128,19 +128,45 @@ class G1CodeRootChunk : public CHeapObj<mtGC> {
}
};
// Manages free chunks.
class G1CodeRootChunkManager VALUE_OBJ_CLASS_SPEC {
private:
// Global free chunk list management
FreeList<G1CodeRootChunk> _free_list;
// Total number of chunks handed out
size_t _num_chunks_handed_out;
public:
G1CodeRootChunkManager();
G1CodeRootChunk* new_chunk();
void free_chunk(G1CodeRootChunk* chunk);
// Free all elements of the given list.
void free_all_chunks(FreeList<G1CodeRootChunk>* list);
void initialize();
void purge_chunks(size_t keep_ratio);
static size_t static_mem_size();
size_t fl_mem_size();
#ifndef PRODUCT
size_t num_chunks_handed_out() const;
size_t num_free_chunks() const;
#endif
};
// Implements storage for a set of code roots.
// All methods that modify the set are not thread-safe except if otherwise noted.
class G1CodeRootSet VALUE_OBJ_CLASS_SPEC {
private:
// Global free chunk list management
static FreeList<G1CodeRootChunk> _free_list;
// Total number of chunks handed out
static size_t _num_chunks_handed_out;
// Global default free chunk manager instance.
static G1CodeRootChunkManager _default_chunk_manager;
static G1CodeRootChunk* new_chunk();
static void free_chunk(G1CodeRootChunk* chunk);
G1CodeRootChunk* new_chunk() { return _manager->new_chunk(); }
void free_chunk(G1CodeRootChunk* chunk) { _manager->free_chunk(chunk); }
// Free all elements of the given list.
static void free_all_chunks(FreeList<G1CodeRootChunk>* list);
void free_all_chunks(FreeList<G1CodeRootChunk>* list) { _manager->free_all_chunks(list); }
// Return the chunk that contains the given nmethod, NULL otherwise.
// Scans the list of chunks backwards, as this method is used to add new
......@@ -150,16 +176,18 @@ class G1CodeRootSet VALUE_OBJ_CLASS_SPEC {
size_t _length;
FreeList<G1CodeRootChunk> _list;
G1CodeRootChunkManager* _manager;
public:
G1CodeRootSet();
// If an instance is initialized with a chunk manager of NULL, use the global
// default one.
G1CodeRootSet(G1CodeRootChunkManager* manager = NULL);
~G1CodeRootSet();
static void initialize();
static void purge_chunks(size_t keep_ratio);
static size_t static_mem_size();
static size_t fl_mem_size();
static size_t free_chunks_static_mem_size();
static size_t free_chunks_mem_size();
// Search for the code blob from the recently allocated ones to find duplicates more quickly, as this
// method is likely to be repeatedly called with the same nmethod.
......@@ -179,6 +207,8 @@ class G1CodeRootSet VALUE_OBJ_CLASS_SPEC {
// Length in elements
size_t length() const { return _length; }
// Static data memory size in bytes of this set.
static size_t static_mem_size();
// Memory size in bytes taken by this set.
size_t mem_size();
......
......@@ -28,9 +28,9 @@
#include "gc_implementation/g1/concurrentMark.hpp"
#include "gc_implementation/g1/evacuationInfo.hpp"
#include "gc_implementation/g1/g1AllocRegion.hpp"
#include "gc_implementation/g1/g1BiasedArray.hpp"
#include "gc_implementation/g1/g1HRPrinter.hpp"
#include "gc_implementation/g1/g1MonitoringSupport.hpp"
#include "gc_implementation/g1/g1RemSet.hpp"
#include "gc_implementation/g1/g1SATBCardTableModRefBS.hpp"
#include "gc_implementation/g1/g1YCTypes.hpp"
#include "gc_implementation/g1/heapRegionSeq.hpp"
......@@ -197,6 +197,16 @@ public:
bool do_object_b(oop p);
};
// Instances of this class are used for quick tests on whether a reference points
// into the collection set. Each of the array's elements denotes whether the
// corresponding region is in the collection set.
class G1FastCSetBiasedMappedArray : public G1BiasedMappedArray<bool> {
protected:
bool default_value() const { return false; }
public:
void clear() { G1BiasedMappedArray<bool>::clear(); }
};
class RefineCardTableEntryClosure;
class G1CollectedHeap : public SharedHeap {
......@@ -353,26 +363,10 @@ private:
// than the current allocation region.
size_t _summary_bytes_used;
// This is used for a quick test on whether a reference points into
// the collection set or not. Basically, we have an array, with one
// byte per region, and that byte denotes whether the corresponding
// region is in the collection set or not. The entry corresponding
// the bottom of the heap, i.e., region 0, is pointed to by
// _in_cset_fast_test_base. The _in_cset_fast_test field has been
// biased so that it actually points to address 0 of the address
// space, to make the test as fast as possible (we can simply shift
// the address to address into it, instead of having to subtract the
// bottom of the heap from the address before shifting it; basically
// it works in the same way the card table works).
bool* _in_cset_fast_test;
// The allocated array used for the fast test on whether a reference
// points into the collection set or not. This field is also used to
// free the array.
bool* _in_cset_fast_test_base;
// The length of the _in_cset_fast_test_base array.
uint _in_cset_fast_test_length;
// This array is used for a quick test on whether a reference points into
// the collection set or not. Each of the array's elements denotes whether the
// corresponding region is in the collection set or not.
G1FastCSetBiasedMappedArray _in_cset_fast_test;
volatile unsigned _gc_time_stamp;
......@@ -695,12 +689,7 @@ public:
// We register a region with the fast "in collection set" test. We
// simply set to true the array slot corresponding to this region.
void register_region_with_in_cset_fast_test(HeapRegion* r) {
assert(_in_cset_fast_test_base != NULL, "sanity");
assert(r->in_collection_set(), "invariant");
uint index = r->hrs_index();
assert(index < _in_cset_fast_test_length, "invariant");
assert(!_in_cset_fast_test_base[index], "invariant");
_in_cset_fast_test_base[index] = true;
_in_cset_fast_test.set_by_index(r->hrs_index(), true);
}
// This is a fast test on whether a reference points into the
......@@ -709,9 +698,7 @@ public:
inline bool in_cset_fast_test(oop obj);
void clear_cset_fast_test() {
assert(_in_cset_fast_test_base != NULL, "sanity");
memset(_in_cset_fast_test_base, false,
(size_t) _in_cset_fast_test_length * sizeof(bool));
_in_cset_fast_test.clear();
}
// This is called at the start of either a concurrent cycle or a Full
......@@ -1703,268 +1690,22 @@ private:
public:
G1ParGCAllocBuffer(size_t gclab_word_size);
virtual ~G1ParGCAllocBuffer() {
guarantee(_retired, "Allocation buffer has not been retired");
}
void set_buf(HeapWord* buf) {
virtual void set_buf(HeapWord* buf) {
ParGCAllocBuffer::set_buf(buf);
_retired = false;
}
void retire(bool end_of_gc, bool retain) {
if (_retired)
virtual void retire(bool end_of_gc, bool retain) {
if (_retired) {
return;
}
ParGCAllocBuffer::retire(end_of_gc, retain);
_retired = true;
}
};
class G1ParScanThreadState : public StackObj {
protected:
G1CollectedHeap* _g1h;
RefToScanQueue* _refs;
DirtyCardQueue _dcq;
G1SATBCardTableModRefBS* _ct_bs;
G1RemSet* _g1_rem;
G1ParGCAllocBuffer _surviving_alloc_buffer;
G1ParGCAllocBuffer _tenured_alloc_buffer;
G1ParGCAllocBuffer* _alloc_buffers[GCAllocPurposeCount];
ageTable _age_table;
G1ParScanClosure _scanner;
size_t _alloc_buffer_waste;
size_t _undo_waste;
OopsInHeapRegionClosure* _evac_failure_cl;
int _hash_seed;
uint _queue_num;
size_t _term_attempts;
double _start;
double _start_strong_roots;
double _strong_roots_time;
double _start_term;
double _term_time;
// Map from young-age-index (0 == not young, 1 is youngest) to
// surviving words. base is what we get back from the malloc call
size_t* _surviving_young_words_base;
// this points into the array, as we use the first few entries for padding
size_t* _surviving_young_words;
#define PADDING_ELEM_NUM (DEFAULT_CACHE_LINE_SIZE / sizeof(size_t))
void add_to_alloc_buffer_waste(size_t waste) { _alloc_buffer_waste += waste; }
void add_to_undo_waste(size_t waste) { _undo_waste += waste; }
DirtyCardQueue& dirty_card_queue() { return _dcq; }
G1SATBCardTableModRefBS* ctbs() { return _ct_bs; }
template <class T> inline void immediate_rs_update(HeapRegion* from, T* p, int tid);
template <class T> void deferred_rs_update(HeapRegion* from, T* p, int tid) {
// If the new value of the field points to the same region or
// is the to-space, we don't need to include it in the Rset updates.
if (!from->is_in_reserved(oopDesc::load_decode_heap_oop(p)) && !from->is_survivor()) {
size_t card_index = ctbs()->index_for(p);
// If the card hasn't been added to the buffer, do it.
if (ctbs()->mark_card_deferred(card_index)) {
dirty_card_queue().enqueue((jbyte*)ctbs()->byte_for_index(card_index));
}
}
}
public:
G1ParScanThreadState(G1CollectedHeap* g1h, uint queue_num, ReferenceProcessor* rp);
~G1ParScanThreadState() {
FREE_C_HEAP_ARRAY(size_t, _surviving_young_words_base, mtGC);
}
RefToScanQueue* refs() { return _refs; }
ageTable* age_table() { return &_age_table; }
G1ParGCAllocBuffer* alloc_buffer(GCAllocPurpose purpose) {
return _alloc_buffers[purpose];
}
size_t alloc_buffer_waste() const { return _alloc_buffer_waste; }
size_t undo_waste() const { return _undo_waste; }
#ifdef ASSERT
bool verify_ref(narrowOop* ref) const;
bool verify_ref(oop* ref) const;
bool verify_task(StarTask ref) const;
#endif // ASSERT
template <class T> void push_on_queue(T* ref) {
assert(verify_ref(ref), "sanity");
refs()->push(ref);
}
template <class T> inline void update_rs(HeapRegion* from, T* p, int tid);
HeapWord* allocate_slow(GCAllocPurpose purpose, size_t word_sz) {
HeapWord* obj = NULL;
size_t gclab_word_size = _g1h->desired_plab_sz(purpose);
if (word_sz * 100 < gclab_word_size * ParallelGCBufferWastePct) {
G1ParGCAllocBuffer* alloc_buf = alloc_buffer(purpose);
add_to_alloc_buffer_waste(alloc_buf->words_remaining());
alloc_buf->retire(false /* end_of_gc */, false /* retain */);
HeapWord* buf = _g1h->par_allocate_during_gc(purpose, gclab_word_size);
if (buf == NULL) return NULL; // Let caller handle allocation failure.
// Otherwise.
alloc_buf->set_word_size(gclab_word_size);
alloc_buf->set_buf(buf);
obj = alloc_buf->allocate(word_sz);
assert(obj != NULL, "buffer was definitely big enough...");
} else {
obj = _g1h->par_allocate_during_gc(purpose, word_sz);
}
return obj;
}
HeapWord* allocate(GCAllocPurpose purpose, size_t word_sz) {
HeapWord* obj = alloc_buffer(purpose)->allocate(word_sz);
if (obj != NULL) return obj;
return allocate_slow(purpose, word_sz);
}
void undo_allocation(GCAllocPurpose purpose, HeapWord* obj, size_t word_sz) {
if (alloc_buffer(purpose)->contains(obj)) {
assert(alloc_buffer(purpose)->contains(obj + word_sz - 1),
"should contain whole object");
alloc_buffer(purpose)->undo_allocation(obj, word_sz);
} else {
CollectedHeap::fill_with_object(obj, word_sz);
add_to_undo_waste(word_sz);
}
}
void set_evac_failure_closure(OopsInHeapRegionClosure* evac_failure_cl) {
_evac_failure_cl = evac_failure_cl;
}
OopsInHeapRegionClosure* evac_failure_closure() {
return _evac_failure_cl;
}
int* hash_seed() { return &_hash_seed; }
uint queue_num() { return _queue_num; }
size_t term_attempts() const { return _term_attempts; }
void note_term_attempt() { _term_attempts++; }
void start_strong_roots() {
_start_strong_roots = os::elapsedTime();
}
void end_strong_roots() {
_strong_roots_time += (os::elapsedTime() - _start_strong_roots);
}
double strong_roots_time() const { return _strong_roots_time; }
void start_term_time() {
note_term_attempt();
_start_term = os::elapsedTime();
}
void end_term_time() {
_term_time += (os::elapsedTime() - _start_term);
}
double term_time() const { return _term_time; }
double elapsed_time() const {
return os::elapsedTime() - _start;
}
static void
print_termination_stats_hdr(outputStream* const st = gclog_or_tty);
void
print_termination_stats(int i, outputStream* const st = gclog_or_tty) const;
size_t* surviving_young_words() {
// We add on to hide entry 0 which accumulates surviving words for
// age -1 regions (i.e. non-young ones)
return _surviving_young_words;
}
void retire_alloc_buffers() {
for (int ap = 0; ap < GCAllocPurposeCount; ++ap) {
size_t waste = _alloc_buffers[ap]->words_remaining();
add_to_alloc_buffer_waste(waste);
_alloc_buffers[ap]->flush_stats_and_retire(_g1h->stats_for_purpose((GCAllocPurpose)ap),
true /* end_of_gc */,
false /* retain */);
}
}
private:
#define G1_PARTIAL_ARRAY_MASK 0x2
inline bool has_partial_array_mask(oop* ref) const {
return ((uintptr_t)ref & G1_PARTIAL_ARRAY_MASK) == G1_PARTIAL_ARRAY_MASK;
}
// We never encode partial array oops as narrowOop*, so return false immediately.
// This allows the compiler to create optimized code when popping references from
// the work queue.
inline bool has_partial_array_mask(narrowOop* ref) const {
assert(((uintptr_t)ref & G1_PARTIAL_ARRAY_MASK) != G1_PARTIAL_ARRAY_MASK, "Partial array oop reference encoded as narrowOop*");
return false;
}
// Only implement set_partial_array_mask() for regular oops, not for narrowOops.
// We always encode partial arrays as regular oop, to allow the
// specialization for has_partial_array_mask() for narrowOops above.
// This means that unintentional use of this method with narrowOops are caught
// by the compiler.
inline oop* set_partial_array_mask(oop obj) const {
assert(((uintptr_t)(void *)obj & G1_PARTIAL_ARRAY_MASK) == 0, "Information loss!");
return (oop*) ((uintptr_t)(void *)obj | G1_PARTIAL_ARRAY_MASK);
}
inline oop clear_partial_array_mask(oop* ref) const {
return cast_to_oop((intptr_t)ref & ~G1_PARTIAL_ARRAY_MASK);
}
inline void do_oop_partial_array(oop* p);
// This method is applied to the fields of the objects that have just been copied.
template <class T> void do_oop_evac(T* p, HeapRegion* from) {
assert(!oopDesc::is_null(oopDesc::load_decode_heap_oop(p)),
"Reference should not be NULL here as such are never pushed to the task queue.");
oop obj = oopDesc::load_decode_heap_oop_not_null(p);
// Although we never intentionally push references outside of the collection
// set, due to (benign) races in the claim mechanism during RSet scanning more
// than one thread might claim the same card. So the same card may be
// processed multiple times. So redo this check.
if (_g1h->in_cset_fast_test(obj)) {
oop forwardee;
if (obj->is_forwarded()) {
forwardee = obj->forwardee();
} else {
forwardee = copy_to_survivor_space(obj);
}
assert(forwardee != NULL, "forwardee should not be NULL");
oopDesc::encode_store_heap_oop(p, forwardee);
}
assert(obj != NULL, "Must be");
update_rs(from, p, queue_num());
}
public:
oop copy_to_survivor_space(oop const obj);
template <class T> inline void deal_with_reference(T* ref_to_scan);
inline void deal_with_reference(StarTask ref);
public:
void trim_queue();
};
#endif // SHARE_VM_GC_IMPLEMENTATION_G1_G1COLLECTEDHEAP_HPP
......@@ -29,7 +29,6 @@
#include "gc_implementation/g1/g1CollectedHeap.hpp"
#include "gc_implementation/g1/g1AllocRegion.inline.hpp"
#include "gc_implementation/g1/g1CollectorPolicy.hpp"
#include "gc_implementation/g1/g1RemSet.inline.hpp"
#include "gc_implementation/g1/g1SATBCardTableModRefBS.hpp"
#include "gc_implementation/g1/heapRegionSet.inline.hpp"
#include "gc_implementation/g1/heapRegionSeq.inline.hpp"
......@@ -178,12 +177,7 @@ inline bool G1CollectedHeap::isMarkedNext(oop obj) const {
// collection set or not. Assume that the reference
// points into the heap.
inline bool G1CollectedHeap::in_cset_fast_test(oop obj) {
assert(_in_cset_fast_test != NULL, "sanity");
assert(_g1_committed.contains((HeapWord*) obj), err_msg("Given reference outside of heap, is "PTR_FORMAT, p2i((HeapWord*)obj)));
// no need to subtract the bottom of the heap from obj,
// _in_cset_fast_test is biased
uintx index = cast_from_oop<uintx>(obj) >> HeapRegion::LogOfHRGrainBytes;
bool ret = _in_cset_fast_test[index];
bool ret = _in_cset_fast_test.get_by_address((HeapWord*)obj);
// let's make sure the result is consistent with what the slower
// test returns
assert( ret || !obj_in_cs(obj), "sanity");
......@@ -296,89 +290,4 @@ inline bool G1CollectedHeap::is_obj_ill(const oop obj) const {
else return is_obj_ill(obj, hr);
}
template <class T> inline void G1ParScanThreadState::immediate_rs_update(HeapRegion* from, T* p, int tid) {
if (!from->is_survivor()) {
_g1_rem->par_write_ref(from, p, tid);
}
}
template <class T> void G1ParScanThreadState::update_rs(HeapRegion* from, T* p, int tid) {
if (G1DeferredRSUpdate) {
deferred_rs_update(from, p, tid);
} else {
immediate_rs_update(from, p, tid);
}
}
inline void G1ParScanThreadState::do_oop_partial_array(oop* p) {
assert(has_partial_array_mask(p), "invariant");
oop from_obj = clear_partial_array_mask(p);
assert(Universe::heap()->is_in_reserved(from_obj), "must be in heap.");
assert(from_obj->is_objArray(), "must be obj array");
objArrayOop from_obj_array = objArrayOop(from_obj);
// The from-space object contains the real length.
int length = from_obj_array->length();
assert(from_obj->is_forwarded(), "must be forwarded");
oop to_obj = from_obj->forwardee();
assert(from_obj != to_obj, "should not be chunking self-forwarded objects");
objArrayOop to_obj_array = objArrayOop(to_obj);
// We keep track of the next start index in the length field of the
// to-space object.
int next_index = to_obj_array->length();
assert(0 <= next_index && next_index < length,
err_msg("invariant, next index: %d, length: %d", next_index, length));
int start = next_index;
int end = length;
int remainder = end - start;
// We'll try not to push a range that's smaller than ParGCArrayScanChunk.
if (remainder > 2 * ParGCArrayScanChunk) {
end = start + ParGCArrayScanChunk;
to_obj_array->set_length(end);
// Push the remainder before we process the range in case another
// worker has run out of things to do and can steal it.
oop* from_obj_p = set_partial_array_mask(from_obj);
push_on_queue(from_obj_p);
} else {
assert(length == end, "sanity");
// We'll process the final range for this object. Restore the length
// so that the heap remains parsable in case of evacuation failure.
to_obj_array->set_length(end);
}
_scanner.set_region(_g1h->heap_region_containing_raw(to_obj));
// Process indexes [start,end). It will also process the header
// along with the first chunk (i.e., the chunk with start == 0).
// Note that at this point the length field of to_obj_array is not
// correct given that we are using it to keep track of the next
// start index. oop_iterate_range() (thankfully!) ignores the length
// field and only relies on the start / end parameters. It does
// however return the size of the object which will be incorrect. So
// we have to ignore it even if we wanted to use it.
to_obj_array->oop_iterate_range(&_scanner, start, end);
}
template <class T> inline void G1ParScanThreadState::deal_with_reference(T* ref_to_scan) {
if (!has_partial_array_mask(ref_to_scan)) {
// Note: we can use "raw" versions of "region_containing" because
// "obj_to_scan" is definitely in the heap, and is not in a
// humongous region.
HeapRegion* r = _g1h->heap_region_containing_raw(ref_to_scan);
do_oop_evac(ref_to_scan, r);
} else {
do_oop_partial_array((oop*)ref_to_scan);
}
}
inline void G1ParScanThreadState::deal_with_reference(StarTask ref) {
assert(verify_task(ref), "sanity");
if (ref.is_narrow()) {
deal_with_reference((narrowOop*)ref);
} else {
deal_with_reference((oop*)ref);
}
}
#endif // SHARE_VM_GC_IMPLEMENTATION_G1_G1COLLECTEDHEAP_INLINE_HPP
......@@ -173,6 +173,8 @@ G1GCPhaseTimes::G1GCPhaseTimes(uint max_gc_threads) :
_last_gc_worker_end_times_ms(_max_gc_threads, "%.1lf", false),
_last_gc_worker_times_ms(_max_gc_threads, "%.1lf"),
_last_gc_worker_other_times_ms(_max_gc_threads, "%.1lf"),
_last_redirty_logged_cards_time_ms(_max_gc_threads, "%.1lf"),
_last_redirty_logged_cards_processed_cards(_max_gc_threads, SIZE_FORMAT),
_cur_string_dedup_queue_fixup_worker_times_ms(_max_gc_threads, "%.1lf"),
_cur_string_dedup_table_fixup_worker_times_ms(_max_gc_threads, "%.1lf")
{
......@@ -198,6 +200,10 @@ void G1GCPhaseTimes::note_gc_start(uint active_gc_threads) {
_last_gc_worker_end_times_ms.reset();
_last_gc_worker_times_ms.reset();
_last_gc_worker_other_times_ms.reset();
_last_redirty_logged_cards_time_ms.reset();
_last_redirty_logged_cards_processed_cards.reset();
}
void G1GCPhaseTimes::note_gc_end() {
......@@ -233,6 +239,9 @@ void G1GCPhaseTimes::note_gc_end() {
_last_gc_worker_times_ms.verify();
_last_gc_worker_other_times_ms.verify();
_last_redirty_logged_cards_time_ms.verify();
_last_redirty_logged_cards_processed_cards.verify();
}
void G1GCPhaseTimes::note_string_dedup_fixup_start() {
......@@ -352,6 +361,10 @@ void G1GCPhaseTimes::print(double pause_time_sec) {
print_stats(2, "Ref Enq", _cur_ref_enq_time_ms);
if (G1DeferredRSUpdate) {
print_stats(2, "Redirty Cards", _recorded_redirty_logged_cards_time_ms);
if (G1Log::finest()) {
_last_redirty_logged_cards_time_ms.print(3, "Parallel Redirty");
_last_redirty_logged_cards_processed_cards.print(3, "Redirtied Cards");
}
}
print_stats(2, "Free CSet",
(_recorded_young_free_cset_time_ms +
......
......@@ -151,6 +151,8 @@ class G1GCPhaseTimes : public CHeapObj<mtGC> {
double _recorded_young_cset_choice_time_ms;
double _recorded_non_young_cset_choice_time_ms;
WorkerDataArray<double> _last_redirty_logged_cards_time_ms;
WorkerDataArray<size_t> _last_redirty_logged_cards_processed_cards;
double _recorded_redirty_logged_cards_time_ms;
double _recorded_young_free_cset_time_ms;
......@@ -293,6 +295,14 @@ class G1GCPhaseTimes : public CHeapObj<mtGC> {
_recorded_non_young_cset_choice_time_ms = time_ms;
}
void record_redirty_logged_cards_time_ms(uint worker_i, double time_ms) {
_last_redirty_logged_cards_time_ms.set(worker_i, time_ms);
}
void record_redirty_logged_cards_processed_cards(uint worker_i, size_t processed_buffers) {
_last_redirty_logged_cards_processed_cards.set(worker_i, processed_buffers);
}
void record_redirty_logged_cards_time_ms(double time_ms) {
_recorded_redirty_logged_cards_time_ms = time_ms;
}
......
......@@ -25,7 +25,28 @@
#include "precompiled.hpp"
#include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
#include "gc_implementation/g1/g1OopClosures.inline.hpp"
#include "gc_implementation/g1/g1ParScanThreadState.hpp"
G1ParCopyHelper::G1ParCopyHelper(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state) :
G1ParClosureSuper(g1, par_scan_state), _scanned_klass(NULL),
_cm(_g1->concurrent_mark()) {}
G1ParClosureSuper::G1ParClosureSuper(G1CollectedHeap* g1) :
_g1(g1), _par_scan_state(NULL), _worker_id(UINT_MAX) { }
G1ParClosureSuper::G1ParClosureSuper(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state) :
_g1(g1), _par_scan_state(NULL),
_worker_id(UINT_MAX) {
set_par_scan_thread_state(par_scan_state);
}
void G1ParClosureSuper::set_par_scan_thread_state(G1ParScanThreadState* par_scan_state) {
assert(_par_scan_state == NULL, "_par_scan_state must only be set once");
assert(par_scan_state != NULL, "Must set par_scan_state to non-NULL.");
_par_scan_state = par_scan_state;
_worker_id = par_scan_state->queue_num();
assert(_worker_id < MAX2((uint)ParallelGCThreads, 1u),
err_msg("The given worker id %u must be less than the number of threads %u", _worker_id, MAX2((uint)ParallelGCThreads, 1u)));
}
......@@ -51,8 +51,13 @@ protected:
G1ParScanThreadState* _par_scan_state;
uint _worker_id;
public:
// Initializes the instance, leaving _par_scan_state uninitialized. Must be done
// later using the set_par_scan_thread_state() method.
G1ParClosureSuper(G1CollectedHeap* g1);
G1ParClosureSuper(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state);
bool apply_to_weak_ref_discovered_field() { return true; }
void set_par_scan_thread_state(G1ParScanThreadState* par_scan_state);
};
class G1ParPushHeapRSClosure : public G1ParClosureSuper {
......@@ -68,9 +73,8 @@ public:
class G1ParScanClosure : public G1ParClosureSuper {
public:
G1ParScanClosure(G1CollectedHeap* g1, G1ParScanThreadState* par_scan_state, ReferenceProcessor* rp) :
G1ParClosureSuper(g1, par_scan_state)
{
G1ParScanClosure(G1CollectedHeap* g1, ReferenceProcessor* rp) :
G1ParClosureSuper(g1) {
assert(_ref_processor == NULL, "sanity");
_ref_processor = rp;
}
......
......@@ -28,6 +28,7 @@
#include "gc_implementation/g1/concurrentMark.inline.hpp"
#include "gc_implementation/g1/g1CollectedHeap.hpp"
#include "gc_implementation/g1/g1OopClosures.hpp"
#include "gc_implementation/g1/g1ParScanThreadState.inline.hpp"
#include "gc_implementation/g1/g1RemSet.hpp"
#include "gc_implementation/g1/g1RemSet.inline.hpp"
#include "gc_implementation/g1/heapRegionRemSet.hpp"
......
/*
* Copyright (c) 2014, 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/g1CollectedHeap.inline.hpp"
#include "gc_implementation/g1/g1OopClosures.inline.hpp"
#include "gc_implementation/g1/g1ParScanThreadState.inline.hpp"
#include "oops/oop.inline.hpp"
#include "oops/oop.pcgc.inline.hpp"
#include "runtime/prefetch.inline.hpp"
G1ParScanThreadState::G1ParScanThreadState(G1CollectedHeap* g1h, uint queue_num, ReferenceProcessor* rp)
: _g1h(g1h),
_refs(g1h->task_queue(queue_num)),
_dcq(&g1h->dirty_card_queue_set()),
_ct_bs(g1h->g1_barrier_set()),
_g1_rem(g1h->g1_rem_set()),
_hash_seed(17), _queue_num(queue_num),
_term_attempts(0),
_surviving_alloc_buffer(g1h->desired_plab_sz(GCAllocForSurvived)),
_tenured_alloc_buffer(g1h->desired_plab_sz(GCAllocForTenured)),
_age_table(false), _scanner(g1h, rp),
_strong_roots_time(0), _term_time(0),
_alloc_buffer_waste(0), _undo_waste(0) {
_scanner.set_par_scan_thread_state(this);
// we allocate G1YoungSurvRateNumRegions plus one entries, since
// we "sacrifice" entry 0 to keep track of surviving bytes for
// non-young regions (where the age is -1)
// We also add a few elements at the beginning and at the end in
// an attempt to eliminate cache contention
uint real_length = 1 + _g1h->g1_policy()->young_cset_region_length();
uint array_length = PADDING_ELEM_NUM +
real_length +
PADDING_ELEM_NUM;
_surviving_young_words_base = NEW_C_HEAP_ARRAY(size_t, array_length, mtGC);
if (_surviving_young_words_base == NULL)
vm_exit_out_of_memory(array_length * sizeof(size_t), OOM_MALLOC_ERROR,
"Not enough space for young surv histo.");
_surviving_young_words = _surviving_young_words_base + PADDING_ELEM_NUM;
memset(_surviving_young_words, 0, (size_t) real_length * sizeof(size_t));
_alloc_buffers[GCAllocForSurvived] = &_surviving_alloc_buffer;
_alloc_buffers[GCAllocForTenured] = &_tenured_alloc_buffer;
_start = os::elapsedTime();
}
G1ParScanThreadState::~G1ParScanThreadState() {
retire_alloc_buffers();
FREE_C_HEAP_ARRAY(size_t, _surviving_young_words_base, mtGC);
}
void
G1ParScanThreadState::print_termination_stats_hdr(outputStream* const st)
{
st->print_raw_cr("GC Termination Stats");
st->print_raw_cr(" elapsed --strong roots-- -------termination-------"
" ------waste (KiB)------");
st->print_raw_cr("thr ms ms % ms % attempts"
" total alloc undo");
st->print_raw_cr("--- --------- --------- ------ --------- ------ --------"
" ------- ------- -------");
}
void
G1ParScanThreadState::print_termination_stats(int i,
outputStream* const st) const
{
const double elapsed_ms = elapsed_time() * 1000.0;
const double s_roots_ms = strong_roots_time() * 1000.0;
const double term_ms = term_time() * 1000.0;
st->print_cr("%3d %9.2f %9.2f %6.2f "
"%9.2f %6.2f " SIZE_FORMAT_W(8) " "
SIZE_FORMAT_W(7) " " SIZE_FORMAT_W(7) " " SIZE_FORMAT_W(7),
i, elapsed_ms, s_roots_ms, s_roots_ms * 100 / elapsed_ms,
term_ms, term_ms * 100 / elapsed_ms, term_attempts(),
(alloc_buffer_waste() + undo_waste()) * HeapWordSize / K,
alloc_buffer_waste() * HeapWordSize / K,
undo_waste() * HeapWordSize / K);
}
#ifdef ASSERT
bool G1ParScanThreadState::verify_ref(narrowOop* ref) const {
assert(ref != NULL, "invariant");
assert(UseCompressedOops, "sanity");
assert(!has_partial_array_mask(ref), err_msg("ref=" PTR_FORMAT, p2i(ref)));
oop p = oopDesc::load_decode_heap_oop(ref);
assert(_g1h->is_in_g1_reserved(p),
err_msg("ref=" PTR_FORMAT " p=" PTR_FORMAT, p2i(ref), p2i(p)));
return true;
}
bool G1ParScanThreadState::verify_ref(oop* ref) const {
assert(ref != NULL, "invariant");
if (has_partial_array_mask(ref)) {
// Must be in the collection set--it's already been copied.
oop p = clear_partial_array_mask(ref);
assert(_g1h->obj_in_cs(p),
err_msg("ref=" PTR_FORMAT " p=" PTR_FORMAT, p2i(ref), p2i(p)));
} else {
oop p = oopDesc::load_decode_heap_oop(ref);
assert(_g1h->is_in_g1_reserved(p),
err_msg("ref=" PTR_FORMAT " p=" PTR_FORMAT, p2i(ref), p2i(p)));
}
return true;
}
bool G1ParScanThreadState::verify_task(StarTask ref) const {
if (ref.is_narrow()) {
return verify_ref((narrowOop*) ref);
} else {
return verify_ref((oop*) ref);
}
}
#endif // ASSERT
void G1ParScanThreadState::trim_queue() {
assert(_evac_failure_cl != NULL, "not set");
StarTask ref;
do {
// Drain the overflow stack first, so other threads can steal.
while (_refs->pop_overflow(ref)) {
dispatch_reference(ref);
}
while (_refs->pop_local(ref)) {
dispatch_reference(ref);
}
} while (!_refs->is_empty());
}
oop G1ParScanThreadState::copy_to_survivor_space(oop const old) {
size_t word_sz = old->size();
HeapRegion* from_region = _g1h->heap_region_containing_raw(old);
// +1 to make the -1 indexes valid...
int young_index = from_region->young_index_in_cset()+1;
assert( (from_region->is_young() && young_index > 0) ||
(!from_region->is_young() && young_index == 0), "invariant" );
G1CollectorPolicy* g1p = _g1h->g1_policy();
markOop m = old->mark();
int age = m->has_displaced_mark_helper() ? m->displaced_mark_helper()->age()
: m->age();
GCAllocPurpose alloc_purpose = g1p->evacuation_destination(from_region, age,
word_sz);
HeapWord* obj_ptr = allocate(alloc_purpose, word_sz);
#ifndef PRODUCT
// Should this evacuation fail?
if (_g1h->evacuation_should_fail()) {
if (obj_ptr != NULL) {
undo_allocation(alloc_purpose, obj_ptr, word_sz);
obj_ptr = NULL;
}
}
#endif // !PRODUCT
if (obj_ptr == NULL) {
// This will either forward-to-self, or detect that someone else has
// installed a forwarding pointer.
return _g1h->handle_evacuation_failure_par(this, old);
}
oop obj = oop(obj_ptr);
// We're going to allocate linearly, so might as well prefetch ahead.
Prefetch::write(obj_ptr, PrefetchCopyIntervalInBytes);
oop forward_ptr = old->forward_to_atomic(obj);
if (forward_ptr == NULL) {
Copy::aligned_disjoint_words((HeapWord*) old, obj_ptr, word_sz);
// alloc_purpose is just a hint to allocate() above, recheck the type of region
// we actually allocated from and update alloc_purpose accordingly
HeapRegion* to_region = _g1h->heap_region_containing_raw(obj_ptr);
alloc_purpose = to_region->is_young() ? GCAllocForSurvived : GCAllocForTenured;
if (g1p->track_object_age(alloc_purpose)) {
// We could simply do obj->incr_age(). However, this causes a
// performance issue. obj->incr_age() will first check whether
// the object has a displaced mark by checking its mark word;
// getting the mark word from the new location of the object
// stalls. So, given that we already have the mark word and we
// are about to install it anyway, it's better to increase the
// age on the mark word, when the object does not have a
// displaced mark word. We're not expecting many objects to have
// a displaced marked word, so that case is not optimized
// further (it could be...) and we simply call obj->incr_age().
if (m->has_displaced_mark_helper()) {
// in this case, we have to install the mark word first,
// otherwise obj looks to be forwarded (the old mark word,
// which contains the forward pointer, was copied)
obj->set_mark(m);
obj->incr_age();
} else {
m = m->incr_age();
obj->set_mark(m);
}
age_table()->add(obj, word_sz);
} else {
obj->set_mark(m);
}
if (G1StringDedup::is_enabled()) {
G1StringDedup::enqueue_from_evacuation(from_region->is_young(),
to_region->is_young(),
queue_num(),
obj);
}
size_t* surv_young_words = surviving_young_words();
surv_young_words[young_index] += word_sz;
if (obj->is_objArray() && arrayOop(obj)->length() >= ParGCArrayScanChunk) {
// We keep track of the next start index in the length field of
// the to-space object. The actual length can be found in the
// length field of the from-space object.
arrayOop(obj)->set_length(0);
oop* old_p = set_partial_array_mask(old);
push_on_queue(old_p);
} else {
// No point in using the slower heap_region_containing() method,
// given that we know obj is in the heap.
_scanner.set_region(_g1h->heap_region_containing_raw(obj));
obj->oop_iterate_backwards(&_scanner);
}
} else {
undo_allocation(alloc_purpose, obj_ptr, word_sz);
obj = forward_ptr;
}
return obj;
}
HeapWord* G1ParScanThreadState::allocate_slow(GCAllocPurpose purpose, size_t word_sz) {
HeapWord* obj = NULL;
size_t gclab_word_size = _g1h->desired_plab_sz(purpose);
if (word_sz * 100 < gclab_word_size * ParallelGCBufferWastePct) {
G1ParGCAllocBuffer* alloc_buf = alloc_buffer(purpose);
add_to_alloc_buffer_waste(alloc_buf->words_remaining());
alloc_buf->retire(false /* end_of_gc */, false /* retain */);
HeapWord* buf = _g1h->par_allocate_during_gc(purpose, gclab_word_size);
if (buf == NULL) {
return NULL; // Let caller handle allocation failure.
}
// Otherwise.
alloc_buf->set_word_size(gclab_word_size);
alloc_buf->set_buf(buf);
obj = alloc_buf->allocate(word_sz);
assert(obj != NULL, "buffer was definitely big enough...");
} else {
obj = _g1h->par_allocate_during_gc(purpose, word_sz);
}
return obj;
}
void G1ParScanThreadState::undo_allocation(GCAllocPurpose purpose, HeapWord* obj, size_t word_sz) {
if (alloc_buffer(purpose)->contains(obj)) {
assert(alloc_buffer(purpose)->contains(obj + word_sz - 1),
"should contain whole object");
alloc_buffer(purpose)->undo_allocation(obj, word_sz);
} else {
CollectedHeap::fill_with_object(obj, word_sz);
add_to_undo_waste(word_sz);
}
}
HeapWord* G1ParScanThreadState::allocate(GCAllocPurpose purpose, size_t word_sz) {
HeapWord* obj = alloc_buffer(purpose)->allocate(word_sz);
if (obj != NULL) {
return obj;
}
return allocate_slow(purpose, word_sz);
}
void G1ParScanThreadState::retire_alloc_buffers() {
for (int ap = 0; ap < GCAllocPurposeCount; ++ap) {
size_t waste = _alloc_buffers[ap]->words_remaining();
add_to_alloc_buffer_waste(waste);
_alloc_buffers[ap]->flush_stats_and_retire(_g1h->stats_for_purpose((GCAllocPurpose)ap),
true /* end_of_gc */,
false /* retain */);
}
}
/*
* Copyright (c) 2014, 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_G1PARSCANTHREADSTATE_HPP
#define SHARE_VM_GC_IMPLEMENTATION_G1_G1PARSCANTHREADSTATE_HPP
#include "gc_implementation/g1/dirtyCardQueue.hpp"
#include "gc_implementation/g1/g1SATBCardTableModRefBS.hpp"
#include "gc_implementation/g1/g1CollectedHeap.hpp"
#include "gc_implementation/g1/g1CollectorPolicy.hpp"
#include "gc_implementation/g1/g1OopClosures.hpp"
#include "gc_implementation/g1/g1RemSet.hpp"
#include "gc_implementation/shared/ageTable.hpp"
#include "memory/allocation.hpp"
#include "oops/oop.hpp"
class HeapRegion;
class outputStream;
class G1ParScanThreadState : public StackObj {
private:
G1CollectedHeap* _g1h;
RefToScanQueue* _refs;
DirtyCardQueue _dcq;
G1SATBCardTableModRefBS* _ct_bs;
G1RemSet* _g1_rem;
G1ParGCAllocBuffer _surviving_alloc_buffer;
G1ParGCAllocBuffer _tenured_alloc_buffer;
G1ParGCAllocBuffer* _alloc_buffers[GCAllocPurposeCount];
ageTable _age_table;
G1ParScanClosure _scanner;
size_t _alloc_buffer_waste;
size_t _undo_waste;
OopsInHeapRegionClosure* _evac_failure_cl;
int _hash_seed;
uint _queue_num;
size_t _term_attempts;
double _start;
double _start_strong_roots;
double _strong_roots_time;
double _start_term;
double _term_time;
// Map from young-age-index (0 == not young, 1 is youngest) to
// surviving words. base is what we get back from the malloc call
size_t* _surviving_young_words_base;
// this points into the array, as we use the first few entries for padding
size_t* _surviving_young_words;
#define PADDING_ELEM_NUM (DEFAULT_CACHE_LINE_SIZE / sizeof(size_t))
void add_to_alloc_buffer_waste(size_t waste) { _alloc_buffer_waste += waste; }
void add_to_undo_waste(size_t waste) { _undo_waste += waste; }
DirtyCardQueue& dirty_card_queue() { return _dcq; }
G1SATBCardTableModRefBS* ctbs() { return _ct_bs; }
template <class T> inline void immediate_rs_update(HeapRegion* from, T* p, int tid);
template <class T> void deferred_rs_update(HeapRegion* from, T* p, int tid) {
// If the new value of the field points to the same region or
// is the to-space, we don't need to include it in the Rset updates.
if (!from->is_in_reserved(oopDesc::load_decode_heap_oop(p)) && !from->is_survivor()) {
size_t card_index = ctbs()->index_for(p);
// If the card hasn't been added to the buffer, do it.
if (ctbs()->mark_card_deferred(card_index)) {
dirty_card_queue().enqueue((jbyte*)ctbs()->byte_for_index(card_index));
}
}
}
public:
G1ParScanThreadState(G1CollectedHeap* g1h, uint queue_num, ReferenceProcessor* rp);
~G1ParScanThreadState();
ageTable* age_table() { return &_age_table; }
G1ParGCAllocBuffer* alloc_buffer(GCAllocPurpose purpose) {
return _alloc_buffers[purpose];
}
size_t alloc_buffer_waste() const { return _alloc_buffer_waste; }
size_t undo_waste() const { return _undo_waste; }
#ifdef ASSERT
bool queue_is_empty() const { return _refs->is_empty(); }
bool verify_ref(narrowOop* ref) const;
bool verify_ref(oop* ref) const;
bool verify_task(StarTask ref) const;
#endif // ASSERT
template <class T> void push_on_queue(T* ref) {
assert(verify_ref(ref), "sanity");
_refs->push(ref);
}
template <class T> inline void update_rs(HeapRegion* from, T* p, int tid);
private:
inline HeapWord* allocate(GCAllocPurpose purpose, size_t word_sz);
inline HeapWord* allocate_slow(GCAllocPurpose purpose, size_t word_sz);
inline void undo_allocation(GCAllocPurpose purpose, HeapWord* obj, size_t word_sz);
public:
void set_evac_failure_closure(OopsInHeapRegionClosure* evac_failure_cl) {
_evac_failure_cl = evac_failure_cl;
}
OopsInHeapRegionClosure* evac_failure_closure() { return _evac_failure_cl; }
int* hash_seed() { return &_hash_seed; }
uint queue_num() { return _queue_num; }
size_t term_attempts() const { return _term_attempts; }
void note_term_attempt() { _term_attempts++; }
void start_strong_roots() {
_start_strong_roots = os::elapsedTime();
}
void end_strong_roots() {
_strong_roots_time += (os::elapsedTime() - _start_strong_roots);
}
double strong_roots_time() const { return _strong_roots_time; }
void start_term_time() {
note_term_attempt();
_start_term = os::elapsedTime();
}
void end_term_time() {
_term_time += (os::elapsedTime() - _start_term);
}
double term_time() const { return _term_time; }
double elapsed_time() const {
return os::elapsedTime() - _start;
}
static void print_termination_stats_hdr(outputStream* const st = gclog_or_tty);
void print_termination_stats(int i, outputStream* const st = gclog_or_tty) const;
size_t* surviving_young_words() {
// We add on to hide entry 0 which accumulates surviving words for
// age -1 regions (i.e. non-young ones)
return _surviving_young_words;
}
private:
void retire_alloc_buffers();
#define G1_PARTIAL_ARRAY_MASK 0x2
inline bool has_partial_array_mask(oop* ref) const {
return ((uintptr_t)ref & G1_PARTIAL_ARRAY_MASK) == G1_PARTIAL_ARRAY_MASK;
}
// We never encode partial array oops as narrowOop*, so return false immediately.
// This allows the compiler to create optimized code when popping references from
// the work queue.
inline bool has_partial_array_mask(narrowOop* ref) const {
assert(((uintptr_t)ref & G1_PARTIAL_ARRAY_MASK) != G1_PARTIAL_ARRAY_MASK, "Partial array oop reference encoded as narrowOop*");
return false;
}
// Only implement set_partial_array_mask() for regular oops, not for narrowOops.
// We always encode partial arrays as regular oop, to allow the
// specialization for has_partial_array_mask() for narrowOops above.
// This means that unintentional use of this method with narrowOops are caught
// by the compiler.
inline oop* set_partial_array_mask(oop obj) const {
assert(((uintptr_t)(void *)obj & G1_PARTIAL_ARRAY_MASK) == 0, "Information loss!");
return (oop*) ((uintptr_t)(void *)obj | G1_PARTIAL_ARRAY_MASK);
}
inline oop clear_partial_array_mask(oop* ref) const {
return cast_to_oop((intptr_t)ref & ~G1_PARTIAL_ARRAY_MASK);
}
inline void do_oop_partial_array(oop* p);
// This method is applied to the fields of the objects that have just been copied.
template <class T> inline void do_oop_evac(T* p, HeapRegion* from);
template <class T> inline void deal_with_reference(T* ref_to_scan);
inline void dispatch_reference(StarTask ref);
public:
oop copy_to_survivor_space(oop const obj);
void trim_queue();
inline void steal_and_trim_queue(RefToScanQueueSet *task_queues);
};
#endif // SHARE_VM_GC_IMPLEMENTATION_G1_G1PARSCANTHREADSTATE_HPP
/*
* Copyright (c) 2014, 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_G1PARSCANTHREADSTATE_INLINE_HPP
#define SHARE_VM_GC_IMPLEMENTATION_G1_G1PARSCANTHREADSTATE_INLINE_HPP
#include "gc_implementation/g1/g1ParScanThreadState.hpp"
#include "gc_implementation/g1/g1RemSet.inline.hpp"
#include "oops/oop.inline.hpp"
template <class T> inline void G1ParScanThreadState::immediate_rs_update(HeapRegion* from, T* p, int tid) {
if (!from->is_survivor()) {
_g1_rem->par_write_ref(from, p, tid);
}
}
template <class T> void G1ParScanThreadState::update_rs(HeapRegion* from, T* p, int tid) {
if (G1DeferredRSUpdate) {
deferred_rs_update(from, p, tid);
} else {
immediate_rs_update(from, p, tid);
}
}
template <class T> void G1ParScanThreadState::do_oop_evac(T* p, HeapRegion* from) {
assert(!oopDesc::is_null(oopDesc::load_decode_heap_oop(p)),
"Reference should not be NULL here as such are never pushed to the task queue.");
oop obj = oopDesc::load_decode_heap_oop_not_null(p);
// Although we never intentionally push references outside of the collection
// set, due to (benign) races in the claim mechanism during RSet scanning more
// than one thread might claim the same card. So the same card may be
// processed multiple times. So redo this check.
if (_g1h->in_cset_fast_test(obj)) {
oop forwardee;
if (obj->is_forwarded()) {
forwardee = obj->forwardee();
} else {
forwardee = copy_to_survivor_space(obj);
}
assert(forwardee != NULL, "forwardee should not be NULL");
oopDesc::encode_store_heap_oop(p, forwardee);
}
assert(obj != NULL, "Must be");
update_rs(from, p, queue_num());
}
inline void G1ParScanThreadState::do_oop_partial_array(oop* p) {
assert(has_partial_array_mask(p), "invariant");
oop from_obj = clear_partial_array_mask(p);
assert(Universe::heap()->is_in_reserved(from_obj), "must be in heap.");
assert(from_obj->is_objArray(), "must be obj array");
objArrayOop from_obj_array = objArrayOop(from_obj);
// The from-space object contains the real length.
int length = from_obj_array->length();
assert(from_obj->is_forwarded(), "must be forwarded");
oop to_obj = from_obj->forwardee();
assert(from_obj != to_obj, "should not be chunking self-forwarded objects");
objArrayOop to_obj_array = objArrayOop(to_obj);
// We keep track of the next start index in the length field of the
// to-space object.
int next_index = to_obj_array->length();
assert(0 <= next_index && next_index < length,
err_msg("invariant, next index: %d, length: %d", next_index, length));
int start = next_index;
int end = length;
int remainder = end - start;
// We'll try not to push a range that's smaller than ParGCArrayScanChunk.
if (remainder > 2 * ParGCArrayScanChunk) {
end = start + ParGCArrayScanChunk;
to_obj_array->set_length(end);
// Push the remainder before we process the range in case another
// worker has run out of things to do and can steal it.
oop* from_obj_p = set_partial_array_mask(from_obj);
push_on_queue(from_obj_p);
} else {
assert(length == end, "sanity");
// We'll process the final range for this object. Restore the length
// so that the heap remains parsable in case of evacuation failure.
to_obj_array->set_length(end);
}
_scanner.set_region(_g1h->heap_region_containing_raw(to_obj));
// Process indexes [start,end). It will also process the header
// along with the first chunk (i.e., the chunk with start == 0).
// Note that at this point the length field of to_obj_array is not
// correct given that we are using it to keep track of the next
// start index. oop_iterate_range() (thankfully!) ignores the length
// field and only relies on the start / end parameters. It does
// however return the size of the object which will be incorrect. So
// we have to ignore it even if we wanted to use it.
to_obj_array->oop_iterate_range(&_scanner, start, end);
}
template <class T> inline void G1ParScanThreadState::deal_with_reference(T* ref_to_scan) {
if (!has_partial_array_mask(ref_to_scan)) {
// Note: we can use "raw" versions of "region_containing" because
// "obj_to_scan" is definitely in the heap, and is not in a
// humongous region.
HeapRegion* r = _g1h->heap_region_containing_raw(ref_to_scan);
do_oop_evac(ref_to_scan, r);
} else {
do_oop_partial_array((oop*)ref_to_scan);
}
}
inline void G1ParScanThreadState::dispatch_reference(StarTask ref) {
assert(verify_task(ref), "sanity");
if (ref.is_narrow()) {
deal_with_reference((narrowOop*)ref);
} else {
deal_with_reference((oop*)ref);
}
}
void G1ParScanThreadState::steal_and_trim_queue(RefToScanQueueSet *task_queues) {
StarTask stolen_task;
while (task_queues->steal(queue_num(), hash_seed(), stolen_task)) {
assert(verify_task(stolen_task), "sanity");
dispatch_reference(stolen_task);
// We've just processed a reference and we might have made
// available new entries on the queues. So we have to make sure
// we drain the queues as necessary.
trim_queue();
}
}
#endif /* SHARE_VM_GC_IMPLEMENTATION_G1_G1PARSCANTHREADSTATE_INLINE_HPP */
......@@ -26,6 +26,7 @@
#define SHARE_VM_GC_IMPLEMENTATION_G1_G1REMSET_INLINE_HPP
#include "gc_implementation/g1/g1RemSet.hpp"
#include "gc_implementation/g1/heapRegion.hpp"
#include "gc_implementation/g1/heapRegionRemSet.hpp"
#include "oops/oop.inline.hpp"
......
......@@ -169,7 +169,7 @@ public:
// Mem size in bytes.
size_t mem_size() const {
return sizeof(this) + _bm.size_in_words() * HeapWordSize;
return sizeof(PerRegionTable) + _bm.size_in_words() * HeapWordSize;
}
// Requires "from" to be in "hr()".
......@@ -735,7 +735,7 @@ size_t OtherRegionsTable::mem_size() const {
sum += (sizeof(PerRegionTable*) * _max_fine_entries);
sum += (_coarse_map.size_in_words() * HeapWordSize);
sum += (_sparse_table.mem_size());
sum += sizeof(*this) - sizeof(_sparse_table); // Avoid double counting above.
sum += sizeof(OtherRegionsTable) - sizeof(_sparse_table); // Avoid double counting above.
return sum;
}
......@@ -770,30 +770,6 @@ void OtherRegionsTable::clear() {
clear_fcc();
}
void OtherRegionsTable::clear_incoming_entry(HeapRegion* from_hr) {
MutexLockerEx x(_m, Mutex::_no_safepoint_check_flag);
size_t hrs_ind = (size_t) from_hr->hrs_index();
size_t ind = hrs_ind & _mod_max_fine_entries_mask;
if (del_single_region_table(ind, from_hr)) {
assert(!_coarse_map.at(hrs_ind), "Inv");
} else {
_coarse_map.par_at_put(hrs_ind, 0);
}
// Check to see if any of the fcc entries come from here.
uint hr_ind = hr()->hrs_index();
for (uint tid = 0; tid < HeapRegionRemSet::num_par_rem_sets(); tid++) {
int fcc_ent = FromCardCache::at(tid, hr_ind);
if (fcc_ent != FromCardCache::InvalidCard) {
HeapWord* card_addr = (HeapWord*)
(uintptr_t(fcc_ent) << CardTableModRefBS::card_shift);
if (hr()->is_in_reserved(card_addr)) {
// Clear the from card cache.
FromCardCache::set(tid, hr_ind, FromCardCache::InvalidCard);
}
}
}
}
bool OtherRegionsTable::del_single_region_table(size_t ind,
HeapRegion* hr) {
assert(0 <= ind && ind < _max_fine_entries, "Preconditions.");
......@@ -1048,20 +1024,16 @@ size_t HeapRegionRemSet::strong_code_roots_mem_size() {
return _code_roots.mem_size();
}
//-------------------- Iteration --------------------
HeapRegionRemSetIterator:: HeapRegionRemSetIterator(HeapRegionRemSet* hrrs) :
_hrrs(hrrs),
_g1h(G1CollectedHeap::heap()),
_coarse_map(&hrrs->_other_regions._coarse_map),
_fine_grain_regions(hrrs->_other_regions._fine_grain_regions),
_bosa(hrrs->bosa()),
_is(Sparse),
// Set these values so that we increment to the first region.
_coarse_cur_region_index(-1),
_coarse_cur_region_cur_card(HeapRegion::CardsPerRegion-1),
_cur_region_cur_card(0),
_fine_array_index(-1),
_cur_card_in_prt(HeapRegion::CardsPerRegion),
_fine_cur_prt(NULL),
_n_yielded_coarse(0),
_n_yielded_fine(0),
......@@ -1093,58 +1065,59 @@ bool HeapRegionRemSetIterator::coarse_has_next(size_t& card_index) {
return true;
}
void HeapRegionRemSetIterator::fine_find_next_non_null_prt() {
// Otherwise, find the next bucket list in the array.
_fine_array_index++;
while (_fine_array_index < (int) OtherRegionsTable::_max_fine_entries) {
_fine_cur_prt = _fine_grain_regions[_fine_array_index];
if (_fine_cur_prt != NULL) return;
else _fine_array_index++;
}
assert(_fine_cur_prt == NULL, "Loop post");
}
bool HeapRegionRemSetIterator::fine_has_next(size_t& card_index) {
if (fine_has_next()) {
_cur_region_cur_card =
_fine_cur_prt->_bm.get_next_one_offset(_cur_region_cur_card + 1);
_cur_card_in_prt =
_fine_cur_prt->_bm.get_next_one_offset(_cur_card_in_prt + 1);
}
while (!fine_has_next()) {
if (_cur_region_cur_card == (size_t) HeapRegion::CardsPerRegion) {
_cur_region_cur_card = 0;
_fine_cur_prt = _fine_cur_prt->collision_list_next();
}
if (_fine_cur_prt == NULL) {
fine_find_next_non_null_prt();
if (_fine_cur_prt == NULL) return false;
if (_cur_card_in_prt == HeapRegion::CardsPerRegion) {
// _fine_cur_prt may still be NULL in case if there are not PRTs at all for
// the remembered set.
if (_fine_cur_prt == NULL || _fine_cur_prt->next() == NULL) {
return false;
}
assert(_fine_cur_prt != NULL && _cur_region_cur_card == 0,
"inv.");
HeapWord* r_bot =
_fine_cur_prt->hr()->bottom();
_cur_region_card_offset = _bosa->index_for(r_bot);
_cur_region_cur_card = _fine_cur_prt->_bm.get_next_one_offset(0);
}
assert(fine_has_next(), "Or else we exited the loop via the return.");
card_index = _cur_region_card_offset + _cur_region_cur_card;
PerRegionTable* next_prt = _fine_cur_prt->next();
switch_to_prt(next_prt);
_cur_card_in_prt = _fine_cur_prt->_bm.get_next_one_offset(_cur_card_in_prt + 1);
}
card_index = _cur_region_card_offset + _cur_card_in_prt;
guarantee(_cur_card_in_prt < HeapRegion::CardsPerRegion,
err_msg("Card index "SIZE_FORMAT" must be within the region", _cur_card_in_prt));
return true;
}
bool HeapRegionRemSetIterator::fine_has_next() {
return
_fine_cur_prt != NULL &&
_cur_region_cur_card < HeapRegion::CardsPerRegion;
return _cur_card_in_prt != HeapRegion::CardsPerRegion;
}
void HeapRegionRemSetIterator::switch_to_prt(PerRegionTable* prt) {
assert(prt != NULL, "Cannot switch to NULL prt");
_fine_cur_prt = prt;
HeapWord* r_bot = _fine_cur_prt->hr()->bottom();
_cur_region_card_offset = _bosa->index_for(r_bot);
// The bitmap scan for the PRT always scans from _cur_region_cur_card + 1.
// To avoid special-casing this start case, and not miss the first bitmap
// entry, initialize _cur_region_cur_card with -1 instead of 0.
_cur_card_in_prt = (size_t)-1;
}
bool HeapRegionRemSetIterator::has_next(size_t& card_index) {
switch (_is) {
case Sparse:
case Sparse: {
if (_sparse_iter.has_next(card_index)) {
_n_yielded_sparse++;
return true;
}
// Otherwise, deliberate fall-through
_is = Fine;
PerRegionTable* initial_fine_prt = _hrrs->_other_regions._first_all_fine_prts;
if (initial_fine_prt != NULL) {
switch_to_prt(_hrrs->_other_regions._first_all_fine_prts);
}
}
case Fine:
if (fine_has_next(card_index)) {
_n_yielded_fine++;
......@@ -1276,6 +1249,11 @@ HeapRegionRemSet::finish_cleanup_task(HRRSCleanupTask* hrrs_cleanup_task) {
#ifndef PRODUCT
void PerRegionTable::test_fl_mem_size() {
PerRegionTable* dummy = alloc(NULL);
size_t min_prt_size = sizeof(void*) + dummy->bm()->size_in_words() * HeapWordSize;
assert(dummy->mem_size() > min_prt_size,
err_msg("PerRegionTable memory usage is suspiciously small, only has "SIZE_FORMAT" bytes. "
"Should be at least "SIZE_FORMAT" bytes.", dummy->mem_size(), min_prt_size));
free(dummy);
guarantee(dummy->mem_size() == fl_mem_size(), "fl_mem_size() does not return the correct element size");
// try to reset the state
......
......@@ -206,9 +206,6 @@ public:
// Specifically clear the from_card_cache.
void clear_fcc();
// "from_hr" is being cleared; remove any entries from it.
void clear_incoming_entry(HeapRegion* from_hr);
void do_cleanup_work(HRRSCleanupTask* hrrs_cleanup_task);
// Declare the heap size (in # of regions) to the OtherRegionsTable.
......@@ -342,20 +339,20 @@ public:
return _other_regions.mem_size()
// This correction is necessary because the above includes the second
// part.
+ (sizeof(this) - sizeof(OtherRegionsTable))
+ (sizeof(HeapRegionRemSet) - sizeof(OtherRegionsTable))
+ strong_code_roots_mem_size();
}
// Returns the memory occupancy of all static data structures associated
// with remembered sets.
static size_t static_mem_size() {
return OtherRegionsTable::static_mem_size() + G1CodeRootSet::static_mem_size();
return OtherRegionsTable::static_mem_size() + G1CodeRootSet::free_chunks_static_mem_size();
}
// Returns the memory occupancy of all free_list data structures associated
// with remembered sets.
static size_t fl_mem_size() {
return OtherRegionsTable::fl_mem_size() + G1CodeRootSet::fl_mem_size();
return OtherRegionsTable::fl_mem_size() + G1CodeRootSet::free_chunks_mem_size();
}
bool contains_reference(OopOrNarrowOopStar from) const {
......@@ -400,7 +397,6 @@ public:
// Declare the heap size (in # of regions) to the HeapRegionRemSet(s).
// (Uses it to initialize from_card_cache).
static void init_heap(uint max_regions) {
G1CodeRootSet::initialize();
OtherRegionsTable::init_from_card_cache(max_regions);
}
......@@ -433,26 +429,24 @@ public:
};
class HeapRegionRemSetIterator : public StackObj {
// The region RSet over which we're iterating.
private:
// The region RSet over which we are iterating.
HeapRegionRemSet* _hrrs;
// Local caching of HRRS fields.
const BitMap* _coarse_map;
PerRegionTable** _fine_grain_regions;
G1BlockOffsetSharedArray* _bosa;
G1CollectedHeap* _g1h;
// The number yielded since initialization.
// The number of cards yielded since initialization.
size_t _n_yielded_fine;
size_t _n_yielded_coarse;
size_t _n_yielded_sparse;
// Indicates what granularity of table that we're currently iterating over.
// Indicates what granularity of table that we are currently iterating over.
// We start iterating over the sparse table, progress to the fine grain
// table, and then finish with the coarse table.
// See HeapRegionRemSetIterator::has_next().
enum IterState {
Sparse,
Fine,
......@@ -460,38 +454,30 @@ class HeapRegionRemSetIterator : public StackObj {
};
IterState _is;
// In both kinds of iteration, heap offset of first card of current
// region.
// For both Coarse and Fine remembered set iteration this contains the
// first card number of the heap region we currently iterate over.
size_t _cur_region_card_offset;
// Card offset within cur region.
size_t _cur_region_cur_card;
// Coarse table iteration fields:
// Current region index;
// Current region index for the Coarse remembered set iteration.
int _coarse_cur_region_index;
size_t _coarse_cur_region_cur_card;
bool coarse_has_next(size_t& card_index);
// Fine table iteration fields:
// Index of bucket-list we're working on.
int _fine_array_index;
// Per Region Table we're doing within current bucket list.
// The PRT we are currently iterating over.
PerRegionTable* _fine_cur_prt;
// Card offset within the current PRT.
size_t _cur_card_in_prt;
/* SparsePRT::*/ SparsePRTIter _sparse_iter;
void fine_find_next_non_null_prt();
// Update internal variables when switching to the given PRT.
void switch_to_prt(PerRegionTable* prt);
bool fine_has_next();
bool fine_has_next(size_t& card_index);
public:
// We require an iterator to be initialized before use, so the
// constructor does little.
// The Sparse remembered set iterator.
SparsePRTIter _sparse_iter;
public:
HeapRegionRemSetIterator(HeapRegionRemSet* hrrs);
// If there remains one or more cards to be yielded, returns true and
......
......@@ -370,7 +370,7 @@ bool RSHashTable::contains_card(RegionIdx_t region_index, CardIdx_t card_index)
}
size_t RSHashTable::mem_size() const {
return sizeof(this) +
return sizeof(RSHashTable) +
capacity() * (SparsePRTEntry::size() + sizeof(int));
}
......@@ -472,7 +472,7 @@ SparsePRT::~SparsePRT() {
size_t SparsePRT::mem_size() const {
// We ignore "_cur" here, because it either = _next, or else it is
// on the deleted list.
return sizeof(this) + _next->mem_size();
return sizeof(SparsePRT) + _next->mem_size();
}
bool SparsePRT::add_card(RegionIdx_t region_id, CardIdx_t card_index) {
......
......@@ -71,7 +71,7 @@ ParMarkBitMap::initialize(MemRegion covered_region)
if (_virtual_space != NULL && _virtual_space->expand_by(_reserved_byte_size)) {
_region_start = covered_region.start();
_region_size = covered_region.word_size();
idx_t* map = (idx_t*)_virtual_space->reserved_low_addr();
BitMap::bm_word_t* map = (BitMap::bm_word_t*)_virtual_space->reserved_low_addr();
_beg_bits.set_map(map);
_beg_bits.set_size(bits / 2);
_end_bits.set_map(map + words / 2);
......
......@@ -60,6 +60,7 @@ public:
// Initializes the buffer to be empty, but with the given "word_sz".
// Must get initialized with "set_buf" for an allocation to succeed.
ParGCAllocBuffer(size_t word_sz);
virtual ~ParGCAllocBuffer() {}
static const size_t min_size() {
return ThreadLocalAllocBuffer::min_size();
......@@ -113,7 +114,7 @@ public:
}
// Sets the space of the buffer to be [buf, space+word_sz()).
void set_buf(HeapWord* buf) {
virtual void set_buf(HeapWord* buf) {
_bottom = buf;
_top = _bottom;
_hard_end = _bottom + word_sz();
......@@ -158,7 +159,7 @@ public:
// Fills in the unallocated portion of the buffer with a garbage object.
// If "end_of_gc" is TRUE, is after the last use in the GC. IF "retain"
// is true, attempt to re-use the unused portion in the next GC.
void retire(bool end_of_gc, bool retain);
virtual void retire(bool end_of_gc, bool retain);
void print() PRODUCT_RETURN;
};
......@@ -238,14 +239,14 @@ public:
void undo_allocation(HeapWord* obj, size_t word_sz);
void set_buf(HeapWord* buf_start) {
virtual void set_buf(HeapWord* buf_start) {
ParGCAllocBuffer::set_buf(buf_start);
_true_end = _hard_end;
_bt.set_region(MemRegion(buf_start, word_sz()));
_bt.initialize_threshold();
}
void retire(bool end_of_gc, bool retain);
virtual void retire(bool end_of_gc, bool retain);
MemRegion range() {
return MemRegion(_top, _true_end);
......
/*
* Copyright (c) 1997, 2013, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 1997, 2014, 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
......@@ -194,7 +194,9 @@ bool ProjNode::is_uncommon_trap_if_pattern(Deoptimization::DeoptReason reason) {
}
}
ProjNode* other_proj = iff->proj_out(1-_con)->as_Proj();
ProjNode* other_proj = iff->proj_out(1-_con);
if (other_proj == NULL) // Should never happen, but make Parfait happy.
return false;
if (other_proj->is_uncommon_trap_proj(reason)) {
assert(reason == Deoptimization::Reason_none ||
Compile::current()->is_predicate_opaq(iff->in(1)->in(1)), "should be on the list");
......
......@@ -1401,7 +1401,7 @@ bool verify_object_alignment() {
return true;
}
uintx Arguments::max_heap_for_compressed_oops() {
size_t Arguments::max_heap_for_compressed_oops() {
// Avoid sign flip.
assert(OopEncodingHeapMax > (uint64_t)os::vm_page_size(), "Unusual page size");
// We need to fit both the NULL page and the heap into the memory budget, while
......
/*
* Copyright (c) 2014, 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.
*/
/*
* @test TestDeferredRSUpdate
* @bug 8040977
* @summary Ensure that running with -XX:-G1DeferredRSUpdate does not crash the VM
* @key gc
* @library /testlibrary
*/
import com.oracle.java.testlibrary.ProcessTools;
import com.oracle.java.testlibrary.OutputAnalyzer;
public class TestDeferredRSUpdate {
public static void main(String[] args) throws Exception {
GCTest.main(args);
ProcessBuilder pb = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-Xmx10M",
// G1DeferredRSUpdate is a develop option, but we cannot limit execution of this test to only debug VMs.
"-XX:+IgnoreUnrecognizedVMOptions",
"-XX:-G1DeferredRSUpdate",
GCTest.class.getName());
OutputAnalyzer output = new OutputAnalyzer(pb.start());
output.shouldHaveExitValue(0);
}
static class GCTest {
private static Object[] garbage = new Object[32];
public static void main(String [] args) {
System.out.println("Creating garbage");
// Create 128MB of garbage. This should result in at least one minor GC, with
// some objects copied to old gen. As references from old to young are installed,
// the crash due to the use before initialize occurs.
Object prev = null;
Object prevPrev = null;
for (int i = 0; i < 1024; i++) {
Object[] next = new Object[32 * 1024];
next[0] = prev;
next[1] = prevPrev;
Object[] cur = (Object[]) garbage[i % garbage.length];
if (cur != null) {
cur[0] = null;
cur[1] = null;
}
garbage[i % garbage.length] = next;
prevPrev = prev;
prev = next;
}
System.out.println("Done");
}
}
}
......@@ -23,7 +23,7 @@
/*
* @test TestPrintGCDetails
* @bug 8035406 8027295 8035398
* @bug 8035406 8027295 8035398 8019342
* @summary Ensure that the PrintGCDetails output for a minor GC with G1
* includes the expected necessary messages.
* @key gc
......@@ -48,6 +48,8 @@ public class TestGCLogMessages {
OutputAnalyzer output = new OutputAnalyzer(pb.start());
output.shouldNotContain("[Redirty Cards");
output.shouldNotContain("[Parallel Redirty");
output.shouldNotContain("[Redirtied Cards");
output.shouldNotContain("[Code Root Purge");
output.shouldNotContain("[String Dedup Fixup");
output.shouldNotContain("[Young Free CSet");
......@@ -63,6 +65,8 @@ public class TestGCLogMessages {
output = new OutputAnalyzer(pb.start());
output.shouldContain("[Redirty Cards");
output.shouldNotContain("[Parallel Redirty");
output.shouldNotContain("[Redirtied Cards");
output.shouldContain("[Code Root Purge");
output.shouldContain("[String Dedup Fixup");
output.shouldNotContain("[Young Free CSet");
......@@ -80,6 +84,8 @@ public class TestGCLogMessages {
output = new OutputAnalyzer(pb.start());
output.shouldContain("[Redirty Cards");
output.shouldContain("[Parallel Redirty");
output.shouldContain("[Redirtied Cards");
output.shouldContain("[Code Root Purge");
output.shouldContain("[String Dedup Fixup");
output.shouldContain("[Young Free CSet");
......
......@@ -88,7 +88,6 @@ public class TestSummarizeRSetStatsTools {
ArrayList<String> finalargs = new ArrayList<String>();
String[] defaultArgs = new String[] {
"-XX:+UseG1GC",
"-XX:+UseCompressedOops",
"-Xmn4m",
"-Xmx20m",
"-XX:InitiatingHeapOccupancyPercent=100", // we don't want the additional GCs due to initial marking
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册