提交 1cdf2972 编写于 作者: K kbarrett

8075215: SATB buffer processing found reclaimed humongous object

Summary: Don't assume SATB buffer entries are valid objects
Reviewed-by: brutisso, ecaspole
上级 761f8145
......@@ -2640,24 +2640,41 @@ void ConcurrentMark::swapMarkBitMaps() {
_nextMarkBitMap = (CMBitMap*) temp;
}
class CMObjectClosure;
// Closure for iterating over objects, currently only used for
// processing SATB buffers.
class CMObjectClosure : public ObjectClosure {
// Closure for marking entries in SATB buffers.
class CMSATBBufferClosure : public SATBBufferClosure {
private:
CMTask* _task;
G1CollectedHeap* _g1h;
public:
void do_object(oop obj) {
_task->deal_with_reference(obj);
// This is very similar to CMTask::deal_with_reference, but with
// more relaxed requirements for the argument, so this must be more
// circumspect about treating the argument as an object.
void do_entry(void* entry) const {
_task->increment_refs_reached();
HeapRegion* hr = _g1h->heap_region_containing_raw(entry);
if (entry < hr->next_top_at_mark_start()) {
// Until we get here, we don't know whether entry refers to a valid
// object; it could instead have been a stale reference.
oop obj = static_cast<oop>(entry);
assert(obj->is_oop(true /* ignore mark word */),
err_msg("Invalid oop in SATB buffer: " PTR_FORMAT, p2i(obj)));
_task->make_reference_grey(obj, hr);
}
}
CMObjectClosure(CMTask* task) : _task(task) { }
public:
CMSATBBufferClosure(CMTask* task, G1CollectedHeap* g1h)
: _task(task), _g1h(g1h) { }
virtual void do_buffer(void** buffer, size_t size) {
for (size_t i = 0; i < size; ++i) {
do_entry(buffer[i]);
}
}
};
class G1RemarkThreadsClosure : public ThreadClosure {
CMObjectClosure _cm_obj;
CMSATBBufferClosure _cm_satb_cl;
G1CMOopClosure _cm_cl;
MarkingCodeBlobClosure _code_cl;
int _thread_parity;
......@@ -2665,7 +2682,9 @@ class G1RemarkThreadsClosure : public ThreadClosure {
public:
G1RemarkThreadsClosure(G1CollectedHeap* g1h, CMTask* task, bool is_par) :
_cm_obj(task), _cm_cl(g1h, g1h->concurrent_mark(), task), _code_cl(&_cm_cl, !CodeBlobToOopClosure::FixRelocations),
_cm_satb_cl(task, g1h),
_cm_cl(g1h, g1h->concurrent_mark(), task),
_code_cl(&_cm_cl, !CodeBlobToOopClosure::FixRelocations),
_thread_parity(SharedHeap::heap()->strong_roots_parity()), _is_par(is_par) {}
void do_thread(Thread* thread) {
......@@ -2681,11 +2700,11 @@ class G1RemarkThreadsClosure : public ThreadClosure {
// live by the SATB invariant but other oops recorded in nmethods may behave differently.
jt->nmethods_do(&_code_cl);
jt->satb_mark_queue().apply_closure_and_empty(&_cm_obj);
jt->satb_mark_queue().apply_closure_and_empty(&_cm_satb_cl);
}
} else if (thread->is_VM_thread()) {
if (thread->claim_oops_do(_is_par, _thread_parity)) {
JavaThread::satb_mark_queue_set().shared_satb_queue()->apply_closure_and_empty(&_cm_obj);
JavaThread::satb_mark_queue_set().shared_satb_queue()->apply_closure_and_empty(&_cm_satb_cl);
}
}
}
......@@ -3976,13 +3995,13 @@ void CMTask::drain_satb_buffers() {
// very counter productive if it did that. :-)
_draining_satb_buffers = true;
CMObjectClosure oc(this);
CMSATBBufferClosure satb_cl(this, _g1h);
SATBMarkQueueSet& satb_mq_set = JavaThread::satb_mark_queue_set();
// This keeps claiming and applying the closure to completed buffers
// until we run out of buffers or we need to abort.
while (!has_aborted() &&
satb_mq_set.apply_closure_to_completed_buffer(&oc)) {
satb_mq_set.apply_closure_to_completed_buffer(&satb_cl)) {
if (_cm->verbose_medium()) {
gclog_or_tty->print_cr("[%u] processed an SATB buffer", _worker_id);
}
......
......@@ -1103,9 +1103,9 @@ private:
void regular_clock_call();
bool concurrent() { return _concurrent; }
// Test whether objAddr might have already been passed over by the
// Test whether obj might have already been passed over by the
// mark bitmap scan, and so needs to be pushed onto the mark stack.
bool is_below_finger(HeapWord* objAddr, HeapWord* global_finger) const;
bool is_below_finger(oop obj, HeapWord* global_finger) const;
template<bool scan> void process_grey_object(oop obj);
......@@ -1156,8 +1156,18 @@ public:
void set_cm_oop_closure(G1CMOopClosure* cm_oop_closure);
// It grays the object by marking it and, if necessary, pushing it
// on the local queue
// Increment the number of references this task has visited.
void increment_refs_reached() { ++_refs_reached; }
// Grey the object by marking it. If not already marked, push it on
// the local queue if below the finger.
// Precondition: obj is in region.
// Precondition: obj is below region's NTAMS.
inline void make_reference_grey(oop obj, HeapRegion* region);
// Grey the object (by calling make_grey_reference) if required,
// e.g. obj is below its containing region's NTAMS.
// Precondition: obj is a valid heap object.
inline void deal_with_reference(oop obj);
// It scans an object and visits its children.
......
......@@ -259,15 +259,15 @@ inline void CMTask::push(oop obj) {
++_local_pushes );
}
inline bool CMTask::is_below_finger(HeapWord* objAddr,
HeapWord* global_finger) const {
// If objAddr is above the global finger, then the mark bitmap scan
inline bool CMTask::is_below_finger(oop obj, HeapWord* global_finger) const {
// If obj is above the global finger, then the mark bitmap scan
// will find it later, and no push is needed. Similarly, if we have
// a current region and objAddr is between the local finger and the
// a current region and obj is between the local finger and the
// end of the current region, then no push is needed. The tradeoff
// of checking both vs only checking the global finger is that the
// local check will be more accurate and so result in fewer pushes,
// but may also be a little slower.
HeapWord* objAddr = (HeapWord*)obj;
if (_finger != NULL) {
// We have a current region.
......@@ -277,7 +277,7 @@ inline bool CMTask::is_below_finger(HeapWord* objAddr,
assert(_region_limit != NULL, "invariant");
assert(_region_limit <= global_finger, "invariant");
// True if objAddr is less than the local finger, or is between
// True if obj is less than the local finger, or is between
// the region limit and the global finger.
if (objAddr < _finger) {
return true;
......@@ -289,13 +289,65 @@ inline bool CMTask::is_below_finger(HeapWord* objAddr,
return objAddr < global_finger;
}
inline void CMTask::make_reference_grey(oop obj, HeapRegion* hr) {
if (_cm->par_mark_and_count(obj, hr, _marked_bytes_array, _card_bm)) {
if (_cm->verbose_high()) {
gclog_or_tty->print_cr("[%u] marked object " PTR_FORMAT,
_worker_id, p2i(obj));
}
// No OrderAccess:store_load() is needed. It is implicit in the
// CAS done in CMBitMap::parMark() call in the routine above.
HeapWord* global_finger = _cm->finger();
// We only need to push a newly grey object on the mark
// stack if it is in a section of memory the mark bitmap
// scan has already examined. Mark bitmap scanning
// maintains progress "fingers" for determining that.
//
// Notice that the global finger might be moving forward
// concurrently. This is not a problem. In the worst case, we
// mark the object while it is above the global finger and, by
// the time we read the global finger, it has moved forward
// past this object. In this case, the object will probably
// be visited when a task is scanning the region and will also
// be pushed on the stack. So, some duplicate work, but no
// correctness problems.
if (is_below_finger(obj, global_finger)) {
if (obj->is_typeArray()) {
// Immediately process arrays of primitive types, rather
// than pushing on the mark stack. This keeps us from
// adding humongous objects to the mark stack that might
// be reclaimed before the entry is processed - see
// selection of candidates for eager reclaim of humongous
// objects. The cost of the additional type test is
// mitigated by avoiding a trip through the mark stack,
// by only doing a bookkeeping update and avoiding the
// actual scan of the object - a typeArray contains no
// references, and the metadata is built-in.
process_grey_object<false>(obj);
} else {
if (_cm->verbose_high()) {
gclog_or_tty->print_cr("[%u] below a finger (local: " PTR_FORMAT
", global: " PTR_FORMAT ") pushing "
PTR_FORMAT " on mark stack",
_worker_id, p2i(_finger),
p2i(global_finger), p2i(obj));
}
push(obj);
}
}
}
}
inline void CMTask::deal_with_reference(oop obj) {
if (_cm->verbose_high()) {
gclog_or_tty->print_cr("[%u] we're dealing with reference = "PTR_FORMAT,
_worker_id, p2i((void*) obj));
}
++_refs_reached;
increment_refs_reached();
HeapWord* objAddr = (HeapWord*) obj;
assert(obj->is_oop_or_null(true /* ignore mark word */), "Error");
......@@ -307,55 +359,7 @@ inline void CMTask::deal_with_reference(oop obj) {
// anything with it).
HeapRegion* hr = _g1h->heap_region_containing_raw(obj);
if (!hr->obj_allocated_since_next_marking(obj)) {
if (_cm->verbose_high()) {
gclog_or_tty->print_cr("[%u] "PTR_FORMAT" is not considered marked",
_worker_id, p2i((void*) obj));
}
// we need to mark it first
if (_cm->par_mark_and_count(obj, hr, _marked_bytes_array, _card_bm)) {
// No OrderAccess:store_load() is needed. It is implicit in the
// CAS done in CMBitMap::parMark() call in the routine above.
HeapWord* global_finger = _cm->finger();
// We only need to push a newly grey object on the mark
// stack if it is in a section of memory the mark bitmap
// scan has already examined. Mark bitmap scanning
// maintains progress "fingers" for determining that.
//
// Notice that the global finger might be moving forward
// concurrently. This is not a problem. In the worst case, we
// mark the object while it is above the global finger and, by
// the time we read the global finger, it has moved forward
// past this object. In this case, the object will probably
// be visited when a task is scanning the region and will also
// be pushed on the stack. So, some duplicate work, but no
// correctness problems.
if (is_below_finger(objAddr, global_finger)) {
if (obj->is_typeArray()) {
// Immediately process arrays of primitive types, rather
// than pushing on the mark stack. This keeps us from
// adding humongous objects to the mark stack that might
// be reclaimed before the entry is processed - see
// selection of candidates for eager reclaim of humongous
// objects. The cost of the additional type test is
// mitigated by avoiding a trip through the mark stack,
// by only doing a bookkeeping update and avoiding the
// actual scan of the object - a typeArray contains no
// references, and the metadata is built-in.
process_grey_object<false>(obj);
} else {
if (_cm->verbose_high()) {
gclog_or_tty->print_cr("[%u] below a finger (local: " PTR_FORMAT
", global: " PTR_FORMAT ") pushing "
PTR_FORMAT " on mark stack",
_worker_id, p2i(_finger),
p2i(global_finger), p2i(objAddr));
}
push(obj);
}
}
}
make_reference_grey(obj, hr);
}
}
}
......
......@@ -29,6 +29,7 @@
#include "memory/sharedHeap.hpp"
#include "oops/oop.inline.hpp"
#include "runtime/mutexLocker.hpp"
#include "runtime/safepoint.hpp"
#include "runtime/thread.hpp"
#include "runtime/vmThread.hpp"
......@@ -162,10 +163,7 @@ bool ObjPtrQueue::should_enqueue_buffer() {
assert(_lock == NULL || _lock->owned_by_self(),
"we should have taken the lock before calling this");
// Even if G1SATBBufferEnqueueingThresholdPercent == 0 we have to
// filter the buffer given that this will remove any references into
// the CSet as we currently assume that no such refs will appear in
// enqueued buffers.
// If G1SATBBufferEnqueueingThresholdPercent == 0 we could skip filtering.
// This method should only be called if there is a non-NULL buffer
// that is full.
......@@ -182,25 +180,19 @@ bool ObjPtrQueue::should_enqueue_buffer() {
return should_enqueue;
}
void ObjPtrQueue::apply_closure_and_empty(ObjectClosure* cl) {
void ObjPtrQueue::apply_closure_and_empty(SATBBufferClosure* cl) {
assert(SafepointSynchronize::is_at_safepoint(),
"SATB queues must only be processed at safepoints");
if (_buf != NULL) {
apply_closure_to_buffer(cl, _buf, _index, _sz);
assert(_index % sizeof(void*) == 0, "invariant");
assert(_sz % sizeof(void*) == 0, "invariant");
assert(_index <= _sz, "invariant");
cl->do_buffer(_buf + byte_index_to_index((int)_index),
byte_index_to_index((int)(_sz - _index)));
_index = _sz;
}
}
void ObjPtrQueue::apply_closure_to_buffer(ObjectClosure* cl,
void** buf, size_t index, size_t sz) {
if (cl == NULL) return;
for (size_t i = index; i < sz; i += oopSize) {
oop obj = (oop)buf[byte_index_to_index((int)i)];
// There can be NULL entries because of destructors.
if (obj != NULL) {
cl->do_object(obj);
}
}
}
#ifndef PRODUCT
// Helpful for debugging
......@@ -291,7 +283,7 @@ void SATBMarkQueueSet::filter_thread_buffers() {
shared_satb_queue()->filter();
}
bool SATBMarkQueueSet::apply_closure_to_completed_buffer(ObjectClosure* cl) {
bool SATBMarkQueueSet::apply_closure_to_completed_buffer(SATBBufferClosure* cl) {
BufferNode* nd = NULL;
{
MutexLockerEx x(_cbl_mon, Mutex::_no_safepoint_check_flag);
......@@ -305,7 +297,18 @@ bool SATBMarkQueueSet::apply_closure_to_completed_buffer(ObjectClosure* cl) {
}
if (nd != NULL) {
void **buf = BufferNode::make_buffer_from_node(nd);
ObjPtrQueue::apply_closure_to_buffer(cl, buf, 0, _sz);
// Skip over NULL entries at beginning (e.g. push end) of buffer.
// Filtering can result in non-full completed buffers; see
// should_enqueue_buffer.
assert(_sz % sizeof(void*) == 0, "invariant");
size_t limit = ObjPtrQueue::byte_index_to_index((int)_sz);
for (size_t i = 0; i < limit; ++i) {
if (buf[i] != NULL) {
// Found the end of the block of NULLs; process the remainder.
cl->do_buffer(buf + i, limit - i);
break;
}
}
deallocate_buffer(buf);
return true;
} else {
......
......@@ -25,29 +25,30 @@
#ifndef SHARE_VM_GC_IMPLEMENTATION_G1_SATBQUEUE_HPP
#define SHARE_VM_GC_IMPLEMENTATION_G1_SATBQUEUE_HPP
#include "memory/allocation.hpp"
#include "gc_implementation/g1/ptrQueue.hpp"
class ObjectClosure;
class JavaThread;
class SATBMarkQueueSet;
// Base class for processing the contents of a SATB buffer.
class SATBBufferClosure : public StackObj {
protected:
~SATBBufferClosure() { }
public:
// Process the SATB entries in the designated buffer range.
virtual void do_buffer(void** buffer, size_t size) = 0;
};
// A ptrQueue whose elements are "oops", pointers to object heads.
class ObjPtrQueue: public PtrQueue {
friend class Threads;
friend class SATBMarkQueueSet;
friend class G1RemarkThreadsClosure;
private:
// Filter out unwanted entries from the buffer.
void filter();
// Apply the closure to all elements and empty the buffer;
void apply_closure_and_empty(ObjectClosure* cl);
// Apply the closure to all elements of "buf", down to "index" (inclusive.)
static void apply_closure_to_buffer(ObjectClosure* cl,
void** buf, size_t index, size_t sz);
public:
ObjPtrQueue(PtrQueueSet* qset, bool perm = false) :
// SATB queues are only active during marking cycles. We create
......@@ -60,6 +61,10 @@ public:
// Process queue entries and free resources.
void flush();
// Apply cl to the active part of the buffer.
// Prerequisite: Must be at a safepoint.
void apply_closure_and_empty(SATBBufferClosure* cl);
// Overrides PtrQueue::should_enqueue_buffer(). See the method's
// definition for more information.
virtual bool should_enqueue_buffer();
......@@ -97,10 +102,12 @@ public:
// Filter all the currently-active SATB buffers.
void filter_thread_buffers();
// If there exists some completed buffer, pop it, then apply the
// closure to all its elements, and return true. If no
// completed buffers exist, return false.
bool apply_closure_to_completed_buffer(ObjectClosure* closure);
// If there exists some completed buffer, pop and process it, and
// return true. Otherwise return false. Processing a buffer
// consists of applying the closure to the buffer range starting
// with the first non-NULL entry to the end of the buffer; the
// leading entries may be NULL due to filtering.
bool apply_closure_to_completed_buffer(SATBBufferClosure* cl);
#ifndef PRODUCT
// Helpful for debugging
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册