提交 39040ebb 编写于 作者: T tonyp

6484956: G1: improve evacuation pause efficiency

Summary: A bunch of performance optimizations to decrease GC pause times in G1.
Reviewed-by: apetrusenko, jmasa, iveresov
上级 e7d93ed3
......@@ -1285,7 +1285,9 @@ G1CollectedHeap::G1CollectedHeap(G1CollectorPolicy* policy_) :
_unclean_regions_coming(false),
_young_list(new YoungList(this)),
_gc_time_stamp(0),
_surviving_young_words(NULL)
_surviving_young_words(NULL),
_in_cset_fast_test(NULL),
_in_cset_fast_test_base(NULL)
{
_g1h = this; // To catch bugs.
if (_process_strong_tasks == NULL || !_process_strong_tasks->valid()) {
......@@ -2485,6 +2487,19 @@ G1CollectedHeap::do_collection_pause_at_safepoint(HeapRegion* popular_region) {
g1_policy()->record_collection_pause_start(start_time_sec,
start_used_bytes);
guarantee(_in_cset_fast_test == NULL, "invariant");
guarantee(_in_cset_fast_test_base == NULL, "invariant");
_in_cset_fast_test_length = n_regions();
_in_cset_fast_test_base =
NEW_C_HEAP_ARRAY(bool, _in_cset_fast_test_length);
memset(_in_cset_fast_test_base, false,
_in_cset_fast_test_length * sizeof(bool));
// We're biasing _in_cset_fast_test to avoid subtracting the
// beginning of the heap every time we want to index; basically
// it's the same with what we do with the card table.
_in_cset_fast_test = _in_cset_fast_test_base -
((size_t) _g1_reserved.start() >> HeapRegion::LogOfHRGrainBytes);
#if SCAN_ONLY_VERBOSE
_young_list->print();
#endif // SCAN_ONLY_VERBOSE
......@@ -2553,6 +2568,12 @@ G1CollectedHeap::do_collection_pause_at_safepoint(HeapRegion* popular_region) {
free_collection_set(g1_policy()->collection_set());
g1_policy()->clear_collection_set();
FREE_C_HEAP_ARRAY(bool, _in_cset_fast_test_base);
// this is more for peace of mind; we're nulling them here and
// we're expecting them to be null at the beginning of the next GC
_in_cset_fast_test = NULL;
_in_cset_fast_test_base = NULL;
if (popular_region != NULL) {
// We have to wait until now, because we don't want the region to
// be rescheduled for pop-evac during RS update.
......@@ -3560,6 +3581,9 @@ public:
size_t undo_waste() { return _undo_waste; }
void push_on_queue(oop* ref) {
assert(ref != NULL, "invariant");
assert(has_partial_array_mask(ref) || _g1h->obj_in_cs(*ref), "invariant");
if (!refs()->push(ref)) {
overflowed_refs()->push(ref);
IF_G1_DETAILED_STATS(note_overflow_push());
......@@ -3572,6 +3596,10 @@ public:
if (!refs()->pop_local(ref)) {
ref = NULL;
} else {
assert(ref != NULL, "invariant");
assert(has_partial_array_mask(ref) || _g1h->obj_in_cs(*ref),
"invariant");
IF_G1_DETAILED_STATS(note_pop());
}
}
......@@ -3601,8 +3629,7 @@ public:
obj = alloc_buf->allocate(word_sz);
assert(obj != NULL, "buffer was definitely big enough...");
}
else {
} else {
obj = _g1h->par_allocate_during_gc(purpose, word_sz);
}
return obj;
......@@ -3695,24 +3722,57 @@ public:
}
}
private:
void deal_with_reference(oop* ref_to_scan) {
if (has_partial_array_mask(ref_to_scan)) {
_partial_scan_cl->do_oop_nv(ref_to_scan);
} else {
// 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);
_evac_cl->set_region(r);
_evac_cl->do_oop_nv(ref_to_scan);
}
}
public:
void trim_queue() {
// I've replicated the loop twice, first to drain the overflow
// queue, second to drain the task queue. This is better than
// having a single loop, which checks both conditions and, inside
// it, either pops the overflow queue or the task queue, as each
// loop is tighter. Also, the decision to drain the overflow queue
// first is not arbitrary, as the overflow queue is not visible
// to the other workers, whereas the task queue is. So, we want to
// drain the "invisible" entries first, while allowing the other
// workers to potentially steal the "visible" entries.
while (refs_to_scan() > 0 || overflowed_refs_to_scan() > 0) {
oop *ref_to_scan = NULL;
if (overflowed_refs_to_scan() == 0) {
pop_from_queue(ref_to_scan);
} else {
while (overflowed_refs_to_scan() > 0) {
oop *ref_to_scan = NULL;
pop_from_overflow_queue(ref_to_scan);
assert(ref_to_scan != NULL, "invariant");
// We shouldn't have pushed it on the queue if it was not
// pointing into the CSet.
assert(ref_to_scan != NULL, "sanity");
assert(has_partial_array_mask(ref_to_scan) ||
_g1h->obj_in_cs(*ref_to_scan), "sanity");
deal_with_reference(ref_to_scan);
}
if (ref_to_scan != NULL) {
if ((intptr_t)ref_to_scan & G1_PARTIAL_ARRAY_MASK) {
_partial_scan_cl->do_oop_nv(ref_to_scan);
} else {
// 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);
_evac_cl->set_region(r);
_evac_cl->do_oop_nv(ref_to_scan);
while (refs_to_scan() > 0) {
oop *ref_to_scan = NULL;
pop_from_queue(ref_to_scan);
if (ref_to_scan != NULL) {
// We shouldn't have pushed it on the queue if it was not
// pointing into the CSet.
assert(has_partial_array_mask(ref_to_scan) ||
_g1h->obj_in_cs(*ref_to_scan), "sanity");
deal_with_reference(ref_to_scan);
}
}
}
......@@ -3728,16 +3788,25 @@ G1ParClosureSuper::G1ParClosureSuper(G1CollectedHeap* g1, G1ParScanThreadState*
// Should probably be made inline and moved in g1OopClosures.inline.hpp.
void G1ParScanClosure::do_oop_nv(oop* p) {
oop obj = *p;
if (obj != NULL) {
if (_g1->obj_in_cs(obj)) {
if (obj->is_forwarded()) {
*p = obj->forwardee();
} else {
_par_scan_state->push_on_queue(p);
return;
}
if (_g1->in_cset_fast_test(obj)) {
// We're not going to even bother checking whether the object is
// already forwarded or not, as this usually causes an immediate
// stall. We'll try to prefetch the object (for write, given that
// we might need to install the forwarding reference) and we'll
// get back to it when pop it from the queue
Prefetch::write(obj->mark_addr(), 0);
Prefetch::read(obj->mark_addr(), (HeapWordSize*2));
// slightly paranoid test; I'm trying to catch potential
// problems before we go into push_on_queue to know where the
// problem is coming from
assert(obj == *p, "the value of *p should not have changed");
_par_scan_state->push_on_queue(p);
} else {
_g1_rem->par_write_ref(_from, p, _par_scan_state->queue_num());
}
_g1_rem->par_write_ref(_from, p, _par_scan_state->queue_num());
}
}
......@@ -3777,13 +3846,36 @@ oop G1ParCopyHelper::copy_to_survivor_space(oop old) {
return _g1->handle_evacuation_failure_par(cl, old);
}
// 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);
obj->set_mark(m);
if (g1p->track_object_age(alloc_purpose)) {
obj->incr_age();
// 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);
// preserve "next" mark bit
if (_g1->mark_in_progress() && !_g1->is_obj_ill(old)) {
if (!use_local_bitmaps ||
......@@ -3805,9 +3897,11 @@ oop G1ParCopyHelper::copy_to_survivor_space(oop old) {
if (obj->is_objArray() && arrayOop(obj)->length() >= ParGCArrayScanChunk) {
arrayOop(old)->set_length(0);
_par_scan_state->push_on_queue((oop*) ((intptr_t)old | G1_PARTIAL_ARRAY_MASK));
_par_scan_state->push_on_queue(set_partial_array_mask(old));
} else {
_scanner->set_region(_g1->heap_region_containing(obj));
// No point in using the slower heap_region_containing() method,
// given that we know obj is in the heap.
_scanner->set_region(_g1->heap_region_containing_raw(obj));
obj->oop_iterate_backwards(_scanner);
}
} else {
......@@ -3817,47 +3911,55 @@ oop G1ParCopyHelper::copy_to_survivor_space(oop old) {
return obj;
}
template<bool do_gen_barrier, G1Barrier barrier, bool do_mark_forwardee>
void G1ParCopyClosure<do_gen_barrier, barrier, do_mark_forwardee>::do_oop_work(oop* p) {
template<bool do_gen_barrier, G1Barrier barrier,
bool do_mark_forwardee, bool skip_cset_test>
void G1ParCopyClosure<do_gen_barrier, barrier,
do_mark_forwardee, skip_cset_test>::do_oop_work(oop* p) {
oop obj = *p;
assert(barrier != G1BarrierRS || obj != NULL,
"Precondition: G1BarrierRS implies obj is nonNull");
if (obj != NULL) {
if (_g1->obj_in_cs(obj)) {
// The only time we skip the cset test is when we're scanning
// references popped from the queue. And we only push on the queue
// references that we know point into the cset, so no point in
// checking again. But we'll leave an assert here for peace of mind.
assert(!skip_cset_test || _g1->obj_in_cs(obj), "invariant");
// here the null check is implicit in the cset_fast_test() test
if (skip_cset_test || _g1->in_cset_fast_test(obj)) {
#if G1_REM_SET_LOGGING
gclog_or_tty->print_cr("Loc "PTR_FORMAT" contains pointer "PTR_FORMAT" into CS.",
p, (void*) obj);
gclog_or_tty->print_cr("Loc "PTR_FORMAT" contains pointer "PTR_FORMAT" "
"into CS.", p, (void*) obj);
#endif
if (obj->is_forwarded()) {
*p = obj->forwardee();
} else {
*p = copy_to_survivor_space(obj);
}
// When scanning the RS, we only care about objs in CS.
if (barrier == G1BarrierRS) {
_g1_rem->par_write_ref(_from, p, _par_scan_state->queue_num());
}
if (obj->is_forwarded()) {
*p = obj->forwardee();
} else {
*p = copy_to_survivor_space(obj);
}
// When scanning moved objs, must look at all oops.
if (barrier == G1BarrierEvac) {
// When scanning the RS, we only care about objs in CS.
if (barrier == G1BarrierRS) {
_g1_rem->par_write_ref(_from, p, _par_scan_state->queue_num());
}
}
if (do_gen_barrier) {
par_do_barrier(p);
}
// When scanning moved objs, must look at all oops.
if (barrier == G1BarrierEvac && obj != NULL) {
_g1_rem->par_write_ref(_from, p, _par_scan_state->queue_num());
}
if (do_gen_barrier && obj != NULL) {
par_do_barrier(p);
}
}
template void G1ParCopyClosure<false, G1BarrierEvac, false>::do_oop_work(oop* p);
template void G1ParCopyClosure<false, G1BarrierEvac, false, true>::do_oop_work(oop* p);
template <class T> void G1ParScanPartialArrayClosure::process_array_chunk(
template<class T> void G1ParScanPartialArrayClosure::process_array_chunk(
oop obj, int start, int end) {
// process our set of indices (include header in first chunk)
assert(start < end, "invariant");
T* const base = (T*)objArrayOop(obj)->base();
T* const start_addr = base + start;
T* const start_addr = (start == 0) ? (T*) obj : base + start;
T* const end_addr = base + end;
MemRegion mr((HeapWord*)start_addr, (HeapWord*)end_addr);
_scanner.set_region(_g1->heap_region_containing(obj));
......@@ -3866,7 +3968,8 @@ template <class T> void G1ParScanPartialArrayClosure::process_array_chunk(
void G1ParScanPartialArrayClosure::do_oop_nv(oop* p) {
assert(!UseCompressedOops, "Needs to be fixed to work with compressed oops");
oop old = oop((intptr_t)p & ~G1_PARTIAL_ARRAY_MASK);
assert(has_partial_array_mask(p), "invariant");
oop old = clear_partial_array_mask(p);
assert(old->is_objArray(), "must be obj array");
assert(old->is_forwarded(), "must be forwarded");
assert(Universe::heap()->is_in_reserved(old), "must be in heap.");
......@@ -3884,7 +3987,7 @@ void G1ParScanPartialArrayClosure::do_oop_nv(oop* p) {
end = start + ParGCArrayScanChunk;
arrayOop(old)->set_length(end);
// Push remainder.
_par_scan_state->push_on_queue((oop*) ((intptr_t) old | G1_PARTIAL_ARRAY_MASK));
_par_scan_state->push_on_queue(set_partial_array_mask(old));
} else {
// Restore length so that the heap remains parsable in
// case of evacuation failure.
......@@ -3893,11 +3996,6 @@ void G1ParScanPartialArrayClosure::do_oop_nv(oop* p) {
// process our set of indices (include header in first chunk)
process_array_chunk<oop>(obj, start, end);
oop* start_addr = start == 0 ? (oop*)obj : obj->obj_at_addr<oop>(start);
oop* end_addr = (oop*)(obj->base()) + end; // obj_at_addr(end) asserts end < length
MemRegion mr((HeapWord*)start_addr, (HeapWord*)end_addr);
_scanner.set_region(_g1->heap_region_containing(obj));
obj->oop_iterate(&_scanner, mr);
}
int G1ScanAndBalanceClosure::_nq = 0;
......@@ -3931,6 +4029,13 @@ public:
pss->hash_seed(),
ref_to_scan)) {
IF_G1_DETAILED_STATS(pss->note_steal());
// slightly paranoid tests; I'm trying to catch potential
// problems before we go into push_on_queue to know where the
// problem is coming from
assert(ref_to_scan != NULL, "invariant");
assert(has_partial_array_mask(ref_to_scan) ||
_g1h->obj_in_cs(*ref_to_scan), "invariant");
pss->push_on_queue(ref_to_scan);
continue;
}
......@@ -3976,10 +4081,10 @@ public:
ResourceMark rm;
HandleMark hm;
G1ParScanThreadState pss(_g1h, i);
G1ParScanHeapEvacClosure scan_evac_cl(_g1h, &pss);
G1ParScanHeapEvacClosure evac_failure_cl(_g1h, &pss);
G1ParScanPartialArrayClosure partial_scan_cl(_g1h, &pss);
G1ParScanThreadState pss(_g1h, i);
G1ParScanHeapEvacClosure scan_evac_cl(_g1h, &pss);
G1ParScanHeapEvacFailureClosure evac_failure_cl(_g1h, &pss);
G1ParScanPartialArrayClosure partial_scan_cl(_g1h, &pss);
pss.set_evac_closure(&scan_evac_cl);
pss.set_evac_failure_closure(&evac_failure_cl);
......
......@@ -247,6 +247,27 @@ private:
NumberSeq _pop_obj_rc_at_copy;
void print_popularity_summary_info() const;
// 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.
size_t _in_cset_fast_test_length;
volatile unsigned _gc_time_stamp;
size_t* _surviving_young_words;
......@@ -368,6 +389,38 @@ public:
virtual void gc_prologue(bool full);
virtual void gc_epilogue(bool full);
// 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");
int index = r->hrs_index();
assert(0 <= (size_t) index && (size_t) index < _in_cset_fast_test_length,
"invariant");
assert(!_in_cset_fast_test_base[index], "invariant");
_in_cset_fast_test_base[index] = true;
}
// This is a fast test on whether a reference points into the
// collection set or not. It does not assume that the reference
// points into the heap; if it doesn't, it will return false.
bool in_cset_fast_test(oop obj) {
assert(_in_cset_fast_test != NULL, "sanity");
if (_g1_committed.contains((HeapWord*) obj)) {
// no need to subtract the bottom of the heap from obj,
// _in_cset_fast_test is biased
size_t index = ((size_t) obj) >> HeapRegion::LogOfHRGrainBytes;
bool ret = _in_cset_fast_test[index];
// let's make sure the result is consistent with what the slower
// test returns
assert( ret || !obj_in_cs(obj), "sanity");
assert(!ret || obj_in_cs(obj), "sanity");
return ret;
} else {
return false;
}
}
protected:
// Shrink the garbage-first heap by at most the given size (in bytes!).
......
......@@ -36,8 +36,11 @@ G1CollectedHeap::heap_region_containing(const void* addr) const {
inline HeapRegion*
G1CollectedHeap::heap_region_containing_raw(const void* addr) const {
HeapRegion* res = _hrs->addr_to_region(addr);
assert(res != NULL, "addr outside of heap?");
assert(_g1_reserved.contains(addr), "invariant");
size_t index = ((intptr_t) addr - (intptr_t) _g1_reserved.start())
>> HeapRegion::LogOfHRGrainBytes;
HeapRegion* res = _hrs->at(index);
assert(res == _hrs->addr_to_region(addr), "sanity");
return res;
}
......
......@@ -2985,6 +2985,7 @@ add_to_collection_set(HeapRegion* hr) {
_collection_set = hr;
_collection_set_size++;
_collection_set_bytes_used_before += hr->used();
_g1->register_region_with_in_cset_fast_test(hr);
}
void
......
......@@ -77,6 +77,18 @@ public:
#define G1_PARTIAL_ARRAY_MASK 1
inline bool has_partial_array_mask(oop* ref) {
return (intptr_t) ref & G1_PARTIAL_ARRAY_MASK;
}
inline oop* set_partial_array_mask(oop obj) {
return (oop*) ((intptr_t) obj | G1_PARTIAL_ARRAY_MASK);
}
inline oop clear_partial_array_mask(oop* ref) {
return oop((intptr_t) ref & ~G1_PARTIAL_ARRAY_MASK);
}
class G1ParScanPartialArrayClosure : public G1ParClosureSuper {
G1ParScanClosure _scanner;
template <class T> void process_array_chunk(oop obj, int start, int end);
......@@ -101,7 +113,8 @@ public:
G1ParClosureSuper(g1, par_scan_state), _scanner(scanner) { }
};
template<bool do_gen_barrier, G1Barrier barrier, bool do_mark_forwardee>
template<bool do_gen_barrier, G1Barrier barrier,
bool do_mark_forwardee, bool skip_cset_test>
class G1ParCopyClosure : public G1ParCopyHelper {
G1ParScanClosure _scanner;
void do_oop_work(oop* p);
......@@ -119,14 +132,22 @@ public:
virtual void do_oop(narrowOop* p) { do_oop_nv(p); }
};
typedef G1ParCopyClosure<false, G1BarrierNone, false> G1ParScanExtRootClosure;
typedef G1ParCopyClosure<true, G1BarrierNone, false> G1ParScanPermClosure;
typedef G1ParCopyClosure<false, G1BarrierNone, true> G1ParScanAndMarkExtRootClosure;
typedef G1ParCopyClosure<true, G1BarrierNone, true> G1ParScanAndMarkPermClosure;
typedef G1ParCopyClosure<false, G1BarrierRS, false> G1ParScanHeapRSClosure;
typedef G1ParCopyClosure<false, G1BarrierRS, true> G1ParScanAndMarkHeapRSClosure;
typedef G1ParCopyClosure<false, G1BarrierEvac, false> G1ParScanHeapEvacClosure;
typedef G1ParCopyClosure<false, G1BarrierNone, false, false> G1ParScanExtRootClosure;
typedef G1ParCopyClosure<true, G1BarrierNone, false, false> G1ParScanPermClosure;
typedef G1ParCopyClosure<false, G1BarrierNone, true, false> G1ParScanAndMarkExtRootClosure;
typedef G1ParCopyClosure<true, G1BarrierNone, true, false> G1ParScanAndMarkPermClosure;
typedef G1ParCopyClosure<false, G1BarrierRS, false, false> G1ParScanHeapRSClosure;
typedef G1ParCopyClosure<false, G1BarrierRS, true, false> G1ParScanAndMarkHeapRSClosure;
// This is the only case when we set skip_cset_test. Basically, this
// closure is (should?) only be called directly while we're draining
// the overflow and task queues. In that case we know that the
// reference in question points into the collection set, otherwise we
// would not have pushed it on the queue.
typedef G1ParCopyClosure<false, G1BarrierEvac, false, true> G1ParScanHeapEvacClosure;
// We need a separate closure to handle references during evacuation
// failure processing, as it cannot asume that the reference already
// points to the collection set (like G1ParScanHeapEvacClosure does).
typedef G1ParCopyClosure<false, G1BarrierEvac, false, false> G1ParScanHeapEvacFailureClosure;
class FilterIntoCSClosure: public OopClosure {
G1CollectedHeap* _g1;
......
......@@ -28,7 +28,7 @@
#define G1_FLAGS(develop, develop_pd, product, product_pd, diagnostic, experimental, notproduct, manageable, product_rw) \
\
product(intx, ParallelGCG1AllocBufferSize, 4*K, \
product(intx, ParallelGCG1AllocBufferSize, 8*K, \
"Size of parallel G1 allocation buffers in to-space.") \
\
product(intx, G1TimeSliceMS, 500, \
......
......@@ -32,11 +32,13 @@ enum G1Barrier {
G1BarrierNone, G1BarrierRS, G1BarrierEvac
};
template<bool do_gen_barrier, G1Barrier barrier, bool do_mark_forwardee>
template<bool do_gen_barrier, G1Barrier barrier,
bool do_mark_forwardee, bool skip_cset_test>
class G1ParCopyClosure;
class G1ParScanClosure;
typedef G1ParCopyClosure<false, G1BarrierEvac, false> G1ParScanHeapEvacClosure;
typedef G1ParCopyClosure<false, G1BarrierEvac, false, true>
G1ParScanHeapEvacClosure;
class FilterIntoCSClosure;
class FilterOutOfRegionClosure;
......
......@@ -31,7 +31,7 @@ bufferingOopClosure.hpp os.hpp
cardTableRS.cpp concurrentMark.hpp
cardTableRS.cpp g1SATBCardTableModRefBS.hpp
collectionSetChooser.cpp g1CollectedHeap.hpp
collectionSetChooser.cpp g1CollectedHeap.inline.hpp
collectionSetChooser.cpp g1CollectorPolicy.hpp
collectionSetChooser.cpp collectionSetChooser.hpp
......@@ -42,14 +42,14 @@ concurrentG1Refine.cpp atomic.hpp
concurrentG1Refine.cpp concurrentG1Refine.hpp
concurrentG1Refine.cpp concurrentG1RefineThread.hpp
concurrentG1Refine.cpp copy.hpp
concurrentG1Refine.cpp g1CollectedHeap.hpp
concurrentG1Refine.cpp g1CollectedHeap.inline.hpp
concurrentG1Refine.cpp g1RemSet.hpp
concurrentG1Refine.hpp globalDefinitions.hpp
concurrentG1RefineThread.cpp concurrentG1Refine.hpp
concurrentG1RefineThread.cpp concurrentG1RefineThread.hpp
concurrentG1RefineThread.cpp g1CollectedHeap.hpp
concurrentG1RefineThread.cpp g1CollectedHeap.inline.hpp
concurrentG1RefineThread.cpp g1CollectorPolicy.hpp
concurrentG1RefineThread.cpp handles.inline.hpp
concurrentG1RefineThread.cpp mutexLocker.hpp
......@@ -166,7 +166,7 @@ g1CollectorPolicy.cpp concurrentMark.hpp
g1CollectorPolicy.cpp concurrentMarkThread.inline.hpp
g1CollectorPolicy.cpp debug.hpp
g1CollectorPolicy.cpp java.hpp
g1CollectorPolicy.cpp g1CollectedHeap.hpp
g1CollectorPolicy.cpp g1CollectedHeap.inline.hpp
g1CollectorPolicy.cpp g1CollectorPolicy.hpp
g1CollectorPolicy.cpp heapRegionRemSet.hpp
g1CollectorPolicy.cpp mutexLocker.hpp
......@@ -187,7 +187,7 @@ g1MarkSweep.cpp biasedLocking.hpp
g1MarkSweep.cpp codeCache.hpp
g1MarkSweep.cpp events.hpp
g1MarkSweep.cpp fprofiler.hpp
g1MarkSweep.hpp g1CollectedHeap.hpp
g1MarkSweep.hpp g1CollectedHeap.inline.hpp
g1MarkSweep.cpp g1MarkSweep.hpp
g1MarkSweep.cpp gcLocker.hpp
g1MarkSweep.cpp genCollectedHeap.hpp
......@@ -283,7 +283,7 @@ heapRegionRemSet.cpp globalDefinitions.hpp
heapRegionRemSet.cpp space.inline.hpp
heapRegionSeq.cpp allocation.hpp
heapRegionSeq.cpp g1CollectedHeap.hpp
heapRegionSeq.cpp g1CollectedHeap.inline.hpp
heapRegionSeq.cpp heapRegionSeq.hpp
heapRegionSeq.hpp growableArray.hpp
......@@ -334,18 +334,18 @@ specialized_oop_closures.hpp g1_specialized_oop_closures.hpp
survRateGroup.hpp numberSeq.hpp
survRateGroup.cpp allocation.hpp
survRateGroup.cpp g1CollectedHeap.hpp
survRateGroup.cpp g1CollectedHeap.inline.hpp
survRateGroup.cpp g1CollectorPolicy.hpp
survRateGroup.cpp heapRegion.hpp
survRateGroup.cpp survRateGroup.hpp
thread.cpp concurrentMarkThread.inline.hpp
universe.cpp g1CollectedHeap.hpp
universe.cpp g1CollectedHeap.inline.hpp
universe.cpp g1CollectorPolicy.hpp
vm_operations_g1.hpp vmGCOperations.hpp
vm_operations_g1.cpp vm_operations_g1.hpp
vm_operations_g1.cpp g1CollectedHeap.hpp
vm_operations_g1.cpp g1CollectedHeap.inline.hpp
vm_operations_g1.cpp isGCActiveMark.hpp
......@@ -100,4 +100,4 @@ spaceCounters.hpp mutableSpace.hpp
spaceCounters.hpp perfData.hpp
spaceCounters.hpp generationCounters.hpp
vmGCOperations.cpp g1CollectedHeap.hpp
vmGCOperations.cpp g1CollectedHeap.inline.hpp
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册