8219613: Use NonJavaThread PtrQueues

Init and use NJT queues, remove shared SATB queue.

Co-authored-by: Aleksey Shipilev <shade@redhat.com>
Reviewed-by: shade, zgu, pliden, tschatzl
This commit is contained in:
Kim Barrett 2019-03-05 19:54:33 -05:00
parent 8b57cdf5f8
commit 725a467ad8
25 changed files with 217 additions and 229 deletions

View File

@ -64,18 +64,7 @@ G1BarrierSet::G1BarrierSet(G1CardTable* card_table) :
void G1BarrierSet::enqueue(oop pre_val) {
// Nulls should have been already filtered.
assert(oopDesc::is_oop(pre_val, true), "Error");
G1SATBMarkQueueSet& queue_set = satb_mark_queue_set();
if (!queue_set.is_active()) {
return;
}
Thread* thr = Thread::current();
if (thr->is_Java_thread()) {
G1ThreadLocalData::satb_mark_queue(thr).enqueue(pre_val);
} else {
MutexLockerEx x(Shared_SATB_Q_lock, Mutex::_no_safepoint_check_flag);
queue_set.shared_satb_queue()->enqueue(pre_val);
}
G1ThreadLocalData::satb_mark_queue(Thread::current()).enqueue(pre_val);
}
template <class T> void
@ -109,13 +98,7 @@ void G1BarrierSet::write_ref_field_post_slow(volatile jbyte* byte) {
if (*byte != G1CardTable::dirty_card_val()) {
*byte = G1CardTable::dirty_card_val();
Thread* thr = Thread::current();
if (thr->is_Java_thread()) {
G1ThreadLocalData::dirty_card_queue(thr).enqueue(byte);
} else {
MutexLockerEx x(Shared_DirtyCardQ_lock,
Mutex::_no_safepoint_check_flag);
_dirty_card_queue_set.shared_dirty_card_queue()->enqueue(byte);
}
G1ThreadLocalData::dirty_card_queue(thr).enqueue(byte);
}
}
@ -125,34 +108,20 @@ void G1BarrierSet::invalidate(MemRegion mr) {
}
volatile jbyte* byte = _card_table->byte_for(mr.start());
jbyte* last_byte = _card_table->byte_for(mr.last());
Thread* thr = Thread::current();
// skip all consecutive young cards
// skip initial young cards
for (; byte <= last_byte && *byte == G1CardTable::g1_young_card_val(); byte++);
if (byte <= last_byte) {
OrderAccess::storeload();
// Enqueue if necessary.
if (thr->is_Java_thread()) {
for (; byte <= last_byte; byte++) {
if (*byte == G1CardTable::g1_young_card_val()) {
continue;
}
if (*byte != G1CardTable::dirty_card_val()) {
*byte = G1CardTable::dirty_card_val();
G1ThreadLocalData::dirty_card_queue(thr).enqueue(byte);
}
}
} else {
MutexLockerEx x(Shared_DirtyCardQ_lock,
Mutex::_no_safepoint_check_flag);
for (; byte <= last_byte; byte++) {
if (*byte == G1CardTable::g1_young_card_val()) {
continue;
}
if (*byte != G1CardTable::dirty_card_val()) {
*byte = G1CardTable::dirty_card_val();
_dirty_card_queue_set.shared_dirty_card_queue()->enqueue(byte);
}
Thread* thr = Thread::current();
G1DirtyCardQueue& queue = G1ThreadLocalData::dirty_card_queue(thr);
for (; byte <= last_byte; byte++) {
jbyte bv = *byte;
if ((bv != G1CardTable::g1_young_card_val()) &&
(bv != G1CardTable::dirty_card_val())) {
*byte = G1CardTable::dirty_card_val();
queue.enqueue(byte);
}
}
}
@ -168,38 +137,40 @@ void G1BarrierSet::on_thread_destroy(Thread* thread) {
G1ThreadLocalData::destroy(thread);
}
void G1BarrierSet::on_thread_attach(JavaThread* thread) {
// This method initializes the SATB and dirty card queues before a
// JavaThread is added to the Java thread list. Right now, we don't
// have to do anything to the dirty card queue (it should have been
// activated when the thread was created), but we have to activate
// the SATB queue if the thread is created while a marking cycle is
// in progress. The activation / de-activation of the SATB queues at
// the beginning / end of a marking cycle is done during safepoints
// so we have to make sure this method is called outside one to be
// able to safely read the active field of the SATB queue set. Right
// now, it is called just before the thread is added to the Java
// thread list in the Threads::add() method. That method is holding
// the Threads_lock which ensures we are outside a safepoint. We
// cannot do the obvious and set the active field of the SATB queue
// when the thread is created given that, in some cases, safepoints
// might happen between the JavaThread constructor being called and the
// thread being added to the Java thread list (an example of this is
// when the structure for the DestroyJavaVM thread is created).
assert(!SafepointSynchronize::is_at_safepoint(), "We should not be at a safepoint");
void G1BarrierSet::on_thread_attach(Thread* thread) {
assert(!G1ThreadLocalData::satb_mark_queue(thread).is_active(), "SATB queue should not be active");
assert(G1ThreadLocalData::satb_mark_queue(thread).is_empty(), "SATB queue should be empty");
assert(G1ThreadLocalData::dirty_card_queue(thread).is_active(), "Dirty card queue should be active");
// Can't assert that the DCQ is empty. There is early execution on
// the main thread, before it gets added to the threads list, which
// is where this is called. That execution may enqueue dirty cards.
// If we are creating the thread during a marking cycle, we should
// set the active field of the SATB queue to true.
if (_satb_mark_queue_set.is_active()) {
G1ThreadLocalData::satb_mark_queue(thread).set_active(true);
}
// set the active field of the SATB queue to true. That involves
// copying the global is_active value to this thread's queue, which
// is done without any direct synchronization here.
//
// The activation and deactivation of the SATB queues occurs at the
// beginning / end of a marking cycle, and is done during
// safepoints. This function is called just before a thread is
// added to its corresponding threads list (for Java or non-Java
// threads, respectively).
//
// For Java threads, that's done while holding the Threads_lock,
// which ensures we're not at a safepoint, so reading the global
// is_active state is synchronized against update.
assert(!thread->is_Java_thread() || !SafepointSynchronize::is_at_safepoint(),
"Should not be at a safepoint");
// For non-Java threads, thread creation (and list addition) may,
// and indeed usually does, occur during a safepoint. But such
// creation isn't concurrent with updating the global SATB active
// state.
bool is_satb_active = _satb_mark_queue_set.is_active();
G1ThreadLocalData::satb_mark_queue(thread).set_active(is_satb_active);
}
void G1BarrierSet::on_thread_detach(JavaThread* thread) {
// Flush any deferred card marks, SATB buffers and dirty card queue buffers
void G1BarrierSet::on_thread_detach(Thread* thread) {
// Flush any deferred card marks.
CardTableBarrierSet::on_thread_detach(thread);
G1ThreadLocalData::satb_mark_queue(thread).flush();
G1ThreadLocalData::dirty_card_queue(thread).flush();

View File

@ -77,8 +77,8 @@ class G1BarrierSet: public CardTableBarrierSet {
virtual void on_thread_create(Thread* thread);
virtual void on_thread_destroy(Thread* thread);
virtual void on_thread_attach(JavaThread* thread);
virtual void on_thread_detach(JavaThread* thread);
virtual void on_thread_attach(Thread* thread);
virtual void on_thread_detach(Thread* thread);
BufferNode::Allocator& satb_mark_queue_buffer_allocator();
BufferNode::Allocator& dirty_card_queue_buffer_allocator();

View File

@ -1676,8 +1676,7 @@ jint G1CollectedHeap::initialize() {
SATB_Q_CBL_mon,
&bs->satb_mark_queue_buffer_allocator(),
G1SATBProcessCompletedThreshold,
G1SATBBufferEnqueueingThresholdPercent,
Shared_SATB_Q_lock);
G1SATBBufferEnqueueingThresholdPercent);
// process_completed_buffers_threshold and max_completed_buffers are updated
// later, based on the concurrent refinement object.
@ -2612,16 +2611,20 @@ void G1CollectedHeap::do_concurrent_mark() {
}
size_t G1CollectedHeap::pending_card_num() {
size_t extra_cards = 0;
for (JavaThreadIteratorWithHandle jtiwh; JavaThread *curr = jtiwh.next(); ) {
G1DirtyCardQueue& dcq = G1ThreadLocalData::dirty_card_queue(curr);
extra_cards += dcq.size();
}
struct CountCardsClosure : public ThreadClosure {
size_t _cards;
CountCardsClosure() : _cards(0) {}
virtual void do_thread(Thread* t) {
_cards += G1ThreadLocalData::dirty_card_queue(t).size();
}
} count_from_threads;
Threads::threads_do(&count_from_threads);
G1DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
size_t buffer_size = dcqs.buffer_size();
size_t buffer_num = dcqs.completed_buffers_num();
return buffer_size * buffer_num + extra_cards;
return buffer_size * buffer_num + count_from_threads._cards;
}
bool G1CollectedHeap::is_potential_eager_reclaim_candidate(HeapRegion* r) const {

View File

@ -1774,23 +1774,18 @@ class G1RemarkThreadsClosure : public ThreadClosure {
_thread_parity(Threads::thread_claim_parity()) {}
void do_thread(Thread* thread) {
if (thread->is_Java_thread()) {
if (thread->claim_oops_do(true, _thread_parity)) {
JavaThread* jt = (JavaThread*)thread;
if (thread->claim_oops_do(true, _thread_parity)) {
SATBMarkQueue& queue = G1ThreadLocalData::satb_mark_queue(thread);
queue.apply_closure_and_empty(&_cm_satb_cl);
if (thread->is_Java_thread()) {
// In theory it should not be neccessary to explicitly walk the nmethods to find roots for concurrent marking
// however the liveness of oops reachable from nmethods have very complex lifecycles:
// * Alive if on the stack of an executing method
// * Weakly reachable otherwise
// Some objects reachable from nmethods, such as the class loader (or klass_holder) of the receiver should be
// live by the SATB invariant but other oops recorded in nmethods may behave differently.
JavaThread* jt = (JavaThread*)thread;
jt->nmethods_do(&_code_cl);
G1ThreadLocalData::satb_mark_queue(jt).apply_closure_and_empty(&_cm_satb_cl);
}
} else if (thread->is_VM_thread()) {
if (thread->claim_oops_do(true, _thread_parity)) {
G1BarrierSet::satb_mark_queue_set().shared_satb_queue()->apply_closure_and_empty(&_cm_satb_cl);
}
}
}

View File

@ -99,7 +99,7 @@ void G1DirtyCardQueueSet::initialize(Monitor* cbl_mon,
}
}
void G1DirtyCardQueueSet::handle_zero_index_for_thread(JavaThread* t) {
void G1DirtyCardQueueSet::handle_zero_index_for_thread(Thread* t) {
G1ThreadLocalData::dirty_card_queue(t).handle_zero_index();
}
@ -207,11 +207,16 @@ void G1DirtyCardQueueSet::par_apply_closure_to_all_completed_buffers(G1CardTable
void G1DirtyCardQueueSet::abandon_logs() {
assert(SafepointSynchronize::is_at_safepoint(), "Must be at safepoint.");
abandon_completed_buffers();
// Since abandon is done only at safepoints, we can safely manipulate
// these queues.
for (JavaThreadIteratorWithHandle jtiwh; JavaThread *t = jtiwh.next(); ) {
G1ThreadLocalData::dirty_card_queue(t).reset();
}
struct AbandonThreadLogClosure : public ThreadClosure {
virtual void do_thread(Thread* t) {
G1ThreadLocalData::dirty_card_queue(t).reset();
}
} closure;
Threads::threads_do(&closure);
shared_dirty_card_queue()->reset();
}
@ -228,9 +233,17 @@ void G1DirtyCardQueueSet::concatenate_logs() {
assert(SafepointSynchronize::is_at_safepoint(), "Must be at safepoint.");
size_t old_limit = max_completed_buffers();
set_max_completed_buffers(MaxCompletedBuffersUnlimited);
for (JavaThreadIteratorWithHandle jtiwh; JavaThread *t = jtiwh.next(); ) {
concatenate_log(G1ThreadLocalData::dirty_card_queue(t));
}
class ConcatenateThreadLogClosure : public ThreadClosure {
G1DirtyCardQueueSet* _qset;
public:
ConcatenateThreadLogClosure(G1DirtyCardQueueSet* qset) : _qset(qset) {}
virtual void do_thread(Thread* t) {
_qset->concatenate_log(G1ThreadLocalData::dirty_card_queue(t));
}
} closure(this);
Threads::threads_do(&closure);
concatenate_log(_shared_dirty_card_queue);
set_max_completed_buffers(old_limit);
}

View File

@ -30,7 +30,7 @@
class G1DirtyCardQueueSet;
class G1FreeIdSet;
class JavaThread;
class Thread;
class Monitor;
// A closure class for processing card table entries. Note that we don't
@ -128,7 +128,7 @@ public:
// mutator threads to do card-processing work.
static uint num_par_ids();
static void handle_zero_index_for_thread(JavaThread* t);
static void handle_zero_index_for_thread(Thread* t);
// Apply G1RefineCardConcurrentlyClosure to completed buffers until there are stop_at
// completed buffers remaining.

View File

@ -1,5 +1,5 @@
/*
* Copyright (c) 2018, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2018, 2019, 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
@ -38,21 +38,19 @@ void G1SATBMarkQueueSet::initialize(G1CollectedHeap* g1h,
Monitor* cbl_mon,
BufferNode::Allocator* allocator,
size_t process_completed_buffers_threshold,
uint buffer_enqueue_threshold_percentage,
Mutex* lock) {
uint buffer_enqueue_threshold_percentage) {
SATBMarkQueueSet::initialize(cbl_mon,
allocator,
process_completed_buffers_threshold,
buffer_enqueue_threshold_percentage,
lock);
buffer_enqueue_threshold_percentage);
_g1h = g1h;
}
void G1SATBMarkQueueSet::handle_zero_index_for_thread(JavaThread* t) {
void G1SATBMarkQueueSet::handle_zero_index_for_thread(Thread* t) {
G1ThreadLocalData::satb_mark_queue(t).handle_zero_index();
}
SATBMarkQueue& G1SATBMarkQueueSet::satb_queue_for_thread(JavaThread* const t) const{
SATBMarkQueue& G1SATBMarkQueueSet::satb_queue_for_thread(Thread* const t) const{
return G1ThreadLocalData::satb_mark_queue(t);
}

View File

@ -29,7 +29,7 @@
class G1CollectedHeap;
class Monitor;
class JavaThread;
class Thread;
class G1SATBMarkQueueSet : public SATBMarkQueueSet {
G1CollectedHeap* _g1h;
@ -41,11 +41,10 @@ public:
Monitor* cbl_mon,
BufferNode::Allocator* allocator,
size_t process_completed_buffers_threshold,
uint buffer_enqueue_threshold_percentage,
Mutex* lock);
uint buffer_enqueue_threshold_percentage);
static void handle_zero_index_for_thread(JavaThread* t);
virtual SATBMarkQueue& satb_queue_for_thread(JavaThread* const t) const;
static void handle_zero_index_for_thread(Thread* t);
virtual SATBMarkQueue& satb_queue_for_thread(Thread* const t) const;
virtual void filter(SATBMarkQueue* queue);
};

View File

@ -130,8 +130,8 @@ public:
virtual void on_slowpath_allocation_exit(JavaThread* thread, oop new_obj) {}
virtual void on_thread_create(Thread* thread) {}
virtual void on_thread_destroy(Thread* thread) {}
virtual void on_thread_attach(JavaThread* thread) {}
virtual void on_thread_detach(JavaThread* thread) {}
virtual void on_thread_attach(Thread* thread) {}
virtual void on_thread_detach(Thread* thread) {}
virtual void make_parsable(JavaThread* thread) {}
#ifdef CHECK_UNHANDLED_OOPS

View File

@ -1,5 +1,5 @@
/*
* Copyright (c) 2000, 2018, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2000, 2019, 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,11 +194,13 @@ void CardTableBarrierSet::flush_deferred_card_mark_barrier(JavaThread* thread) {
#endif
}
void CardTableBarrierSet::on_thread_detach(JavaThread* thread) {
void CardTableBarrierSet::on_thread_detach(Thread* thread) {
// The deferred store barriers must all have been flushed to the
// card-table (or other remembered set structure) before GC starts
// processing the card-table (or other remembered set).
flush_deferred_card_mark_barrier(thread);
if (thread->is_Java_thread()) { // Only relevant for Java threads.
flush_deferred_card_mark_barrier((JavaThread*)thread);
}
}
bool CardTableBarrierSet::card_mark_must_follow_store() const {

View File

@ -102,7 +102,7 @@ class CardTableBarrierSet: public ModRefBarrierSet {
virtual bool card_mark_must_follow_store() const;
virtual void on_slowpath_allocation_exit(JavaThread* thread, oop new_obj);
virtual void on_thread_detach(JavaThread* thread);
virtual void on_thread_detach(Thread* thread);
virtual void make_parsable(JavaThread* thread) { flush_deferred_card_mark_barrier(thread); }

View File

@ -1,5 +1,5 @@
/*
* Copyright (c) 2001, 2018, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2001, 2019, 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
@ -41,7 +41,7 @@ SATBMarkQueue::SATBMarkQueue(SATBMarkQueueSet* qset, bool permanent) :
// created during a cycle and its SATB queue needs to be activated
// before the thread starts running, we'll need to set its active
// field to true. This must be done in the collector-specific
// BarrierSet::on_thread_attach() implementation.
// BarrierSet thread attachment protocol.
PtrQueue(qset, permanent, false /* active */)
{ }
@ -57,9 +57,6 @@ void SATBMarkQueue::flush() {
// use the buffer as-is, instead of enqueueing and replacing it.
bool SATBMarkQueue::should_enqueue_buffer() {
assert(_lock == NULL || _lock->owned_by_self(),
"we should have taken the lock before calling this");
// This method should only be called if there is a non-NULL buffer
// that is full.
assert(index() == 0, "pre-condition");
@ -107,18 +104,15 @@ void SATBMarkQueue::print(const char* name) {
SATBMarkQueueSet::SATBMarkQueueSet() :
PtrQueueSet(),
_shared_satb_queue(this, true /* permanent */),
_buffer_enqueue_threshold(0)
{}
void SATBMarkQueueSet::initialize(Monitor* cbl_mon,
BufferNode::Allocator* allocator,
size_t process_completed_buffers_threshold,
uint buffer_enqueue_threshold_percentage,
Mutex* lock) {
uint buffer_enqueue_threshold_percentage) {
PtrQueueSet::initialize(cbl_mon, allocator);
set_process_completed_buffers_threshold(process_completed_buffers_threshold);
_shared_satb_queue.set_lock(lock);
assert(buffer_size() != 0, "buffer size not initialized");
// Minimum threshold of 1 ensures enqueuing of completely full buffers.
size_t size = buffer_size();
@ -131,32 +125,43 @@ void SATBMarkQueueSet::dump_active_states(bool expected_active) {
log_error(gc, verify)("Expected SATB active state: %s", expected_active ? "ACTIVE" : "INACTIVE");
log_error(gc, verify)("Actual SATB active states:");
log_error(gc, verify)(" Queue set: %s", is_active() ? "ACTIVE" : "INACTIVE");
for (JavaThreadIteratorWithHandle jtiwh; JavaThread *t = jtiwh.next(); ) {
log_error(gc, verify)(" Thread \"%s\" queue: %s", t->name(), satb_queue_for_thread(t).is_active() ? "ACTIVE" : "INACTIVE");
}
log_error(gc, verify)(" Shared queue: %s", shared_satb_queue()->is_active() ? "ACTIVE" : "INACTIVE");
class DumpThreadStateClosure : public ThreadClosure {
SATBMarkQueueSet* _qset;
public:
DumpThreadStateClosure(SATBMarkQueueSet* qset) : _qset(qset) {}
virtual void do_thread(Thread* t) {
SATBMarkQueue& queue = _qset->satb_queue_for_thread(t);
log_error(gc, verify)(" Thread \"%s\" queue: %s",
t->name(),
queue.is_active() ? "ACTIVE" : "INACTIVE");
}
} closure(this);
Threads::threads_do(&closure);
}
void SATBMarkQueueSet::verify_active_states(bool expected_active) {
// Verify queue set state
if (is_active() != expected_active) {
dump_active_states(expected_active);
guarantee(false, "SATB queue set has an unexpected active state");
fatal("SATB queue set has an unexpected active state");
}
// Verify thread queue states
for (JavaThreadIteratorWithHandle jtiwh; JavaThread *t = jtiwh.next(); ) {
if (satb_queue_for_thread(t).is_active() != expected_active) {
dump_active_states(expected_active);
guarantee(false, "Thread SATB queue has an unexpected active state");
class VerifyThreadStatesClosure : public ThreadClosure {
SATBMarkQueueSet* _qset;
bool _expected_active;
public:
VerifyThreadStatesClosure(SATBMarkQueueSet* qset, bool expected_active) :
_qset(qset), _expected_active(expected_active) {}
virtual void do_thread(Thread* t) {
if (_qset->satb_queue_for_thread(t).is_active() != _expected_active) {
_qset->dump_active_states(_expected_active);
fatal("Thread SATB queue has an unexpected active state");
}
}
}
// Verify shared queue state
if (shared_satb_queue()->is_active() != expected_active) {
dump_active_states(expected_active);
guarantee(false, "Shared SATB queue has an unexpected active state");
}
} closure(this, expected_active);
Threads::threads_do(&closure);
}
#endif // ASSERT
@ -166,17 +171,30 @@ void SATBMarkQueueSet::set_active_all_threads(bool active, bool expected_active)
verify_active_states(expected_active);
#endif // ASSERT
_all_active = active;
for (JavaThreadIteratorWithHandle jtiwh; JavaThread *t = jtiwh.next(); ) {
satb_queue_for_thread(t).set_active(active);
}
shared_satb_queue()->set_active(active);
class SetThreadActiveClosure : public ThreadClosure {
SATBMarkQueueSet* _qset;
bool _active;
public:
SetThreadActiveClosure(SATBMarkQueueSet* qset, bool active) :
_qset(qset), _active(active) {}
virtual void do_thread(Thread* t) {
_qset->satb_queue_for_thread(t).set_active(_active);
}
} closure(this, active);
Threads::threads_do(&closure);
}
void SATBMarkQueueSet::filter_thread_buffers() {
for (JavaThreadIteratorWithHandle jtiwh; JavaThread *t = jtiwh.next(); ) {
satb_queue_for_thread(t).filter();
}
shared_satb_queue()->filter();
class FilterThreadBufferClosure : public ThreadClosure {
SATBMarkQueueSet* _qset;
public:
FilterThreadBufferClosure(SATBMarkQueueSet* qset) : _qset(qset) {}
virtual void do_thread(Thread* t) {
_qset->satb_queue_for_thread(t).filter();
}
} closure(this);
Threads::threads_do(&closure);
}
bool SATBMarkQueueSet::apply_closure_to_completed_buffer(SATBBufferClosure* cl) {
@ -216,23 +234,36 @@ void SATBMarkQueueSet::print_all(const char* msg) {
i += 1;
}
for (JavaThreadIteratorWithHandle jtiwh; JavaThread *t = jtiwh.next(); ) {
os::snprintf(buffer, SATB_PRINTER_BUFFER_SIZE, "Thread: %s", t->name());
satb_queue_for_thread(t).print(buffer);
}
class PrintThreadClosure : public ThreadClosure {
SATBMarkQueueSet* _qset;
char* _buffer;
shared_satb_queue()->print("Shared");
public:
PrintThreadClosure(SATBMarkQueueSet* qset, char* buffer) :
_qset(qset), _buffer(buffer) {}
virtual void do_thread(Thread* t) {
os::snprintf(_buffer, SATB_PRINTER_BUFFER_SIZE, "Thread: %s", t->name());
_qset->satb_queue_for_thread(t).print(_buffer);
}
} closure(this, buffer);
Threads::threads_do(&closure);
tty->cr();
}
#endif // PRODUCT
void SATBMarkQueueSet::abandon_partial_marking() {
abandon_completed_buffers();
assert(SafepointSynchronize::is_at_safepoint(), "Must be at safepoint.");
// So we can safely manipulate these queues.
for (JavaThreadIteratorWithHandle jtiwh; JavaThread *t = jtiwh.next(); ) {
satb_queue_for_thread(t).reset();
}
shared_satb_queue()->reset();
abandon_completed_buffers();
class AbandonThreadQueueClosure : public ThreadClosure {
SATBMarkQueueSet* _qset;
public:
AbandonThreadQueueClosure(SATBMarkQueueSet* qset) : _qset(qset) {}
virtual void do_thread(Thread* t) {
_qset->satb_queue_for_thread(t).reset();
}
} closure(this);
Threads::threads_do(&closure);
}

View File

@ -28,7 +28,7 @@
#include "gc/shared/ptrQueue.hpp"
#include "memory/allocation.hpp"
class JavaThread;
class Thread;
class Monitor;
class SATBMarkQueueSet;
@ -92,7 +92,6 @@ public:
};
class SATBMarkQueueSet: public PtrQueueSet {
SATBMarkQueue _shared_satb_queue;
size_t _buffer_enqueue_threshold;
#ifdef ASSERT
@ -112,11 +111,10 @@ protected:
void initialize(Monitor* cbl_mon,
BufferNode::Allocator* allocator,
size_t process_completed_buffers_threshold,
uint buffer_enqueue_threshold_percentage,
Mutex* lock);
uint buffer_enqueue_threshold_percentage);
public:
virtual SATBMarkQueue& satb_queue_for_thread(JavaThread* const t) const = 0;
virtual SATBMarkQueue& satb_queue_for_thread(Thread* const t) const = 0;
// Apply "set_active(active)" to all SATB queues in the set. It should be
// called only with the world stopped. The method will assert that the
@ -141,8 +139,6 @@ public:
void print_all(const char* msg);
#endif // PRODUCT
SATBMarkQueue* shared_satb_queue() { return &_shared_satb_queue; }
// If a marking is being abandoned, reset any unprocessed log buffers.
void abandon_partial_marking();
};

View File

@ -1,5 +1,5 @@
/*
* Copyright (c) 2013, 2018, Red Hat, Inc. All rights reserved.
* Copyright (c) 2013, 2019, Red Hat, Inc. All rights reserved.
*
* 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
@ -337,13 +337,7 @@ void ShenandoahBarrierSet::enqueue(oop obj) {
// filtering here helps to avoid wasteful SATB queueing work to begin with.
if (!_heap->requires_marking<false>(obj)) return;
Thread* thr = Thread::current();
if (thr->is_Java_thread()) {
ShenandoahThreadLocalData::satb_mark_queue(thr).enqueue(obj);
} else {
MutexLockerEx x(Shared_SATB_Q_lock, Mutex::_no_safepoint_check_flag);
_satb_mark_queue_set.shared_satb_queue()->enqueue(obj);
}
ShenandoahThreadLocalData::satb_mark_queue(Thread::current()).enqueue(obj);
}
void ShenandoahBarrierSet::on_thread_create(Thread* thread) {
@ -356,21 +350,26 @@ void ShenandoahBarrierSet::on_thread_destroy(Thread* thread) {
ShenandoahThreadLocalData::destroy(thread);
}
void ShenandoahBarrierSet::on_thread_attach(JavaThread* thread) {
assert(!SafepointSynchronize::is_at_safepoint(), "We should not be at a safepoint");
assert(!ShenandoahThreadLocalData::satb_mark_queue(thread).is_active(), "SATB queue should not be active");
assert(ShenandoahThreadLocalData::satb_mark_queue(thread).is_empty(), "SATB queue should be empty");
if (ShenandoahBarrierSet::satb_mark_queue_set().is_active()) {
ShenandoahThreadLocalData::satb_mark_queue(thread).set_active(true);
void ShenandoahBarrierSet::on_thread_attach(Thread *thread) {
assert(!thread->is_Java_thread() || !SafepointSynchronize::is_at_safepoint(),
"We should not be at a safepoint");
SATBMarkQueue& queue = ShenandoahThreadLocalData::satb_mark_queue(thread);
assert(!queue.is_active(), "SATB queue should not be active");
assert( queue.is_empty(), "SATB queue should be empty");
queue.set_active(_satb_mark_queue_set.is_active());
if (thread->is_Java_thread()) {
ShenandoahThreadLocalData::set_gc_state(thread, _heap->gc_state());
ShenandoahThreadLocalData::initialize_gclab(thread);
}
ShenandoahThreadLocalData::set_gc_state(thread, _heap->gc_state());
ShenandoahThreadLocalData::initialize_gclab(thread);
}
void ShenandoahBarrierSet::on_thread_detach(JavaThread* thread) {
ShenandoahThreadLocalData::satb_mark_queue(thread).flush();
PLAB* gclab = ShenandoahThreadLocalData::gclab(thread);
if (gclab != NULL) {
gclab->retire();
void ShenandoahBarrierSet::on_thread_detach(Thread *thread) {
SATBMarkQueue& queue = ShenandoahThreadLocalData::satb_mark_queue(thread);
queue.flush();
if (thread->is_Java_thread()) {
PLAB* gclab = ShenandoahThreadLocalData::gclab(thread);
if (gclab != NULL) {
gclab->retire();
}
}
}

View File

@ -1,5 +1,5 @@
/*
* Copyright (c) 2013, 2018, Red Hat, Inc. All rights reserved.
* Copyright (c) 2013, 2019, Red Hat, Inc. All rights reserved.
*
* 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
@ -84,8 +84,8 @@ public:
virtual void on_thread_create(Thread* thread);
virtual void on_thread_destroy(Thread* thread);
virtual void on_thread_attach(JavaThread* thread);
virtual void on_thread_detach(JavaThread* thread);
virtual void on_thread_attach(Thread* thread);
virtual void on_thread_detach(Thread* thread);
virtual oop read_barrier(oop src);

View File

@ -227,15 +227,8 @@ public:
_thread_parity(Threads::thread_claim_parity()) {}
void do_thread(Thread* thread) {
if (thread->is_Java_thread()) {
if (thread->claim_oops_do(true, _thread_parity)) {
JavaThread* jt = (JavaThread*)thread;
ShenandoahThreadLocalData::satb_mark_queue(jt).apply_closure_and_empty(_satb_cl);
}
} else if (thread->is_VM_thread()) {
if (thread->claim_oops_do(true, _thread_parity)) {
ShenandoahBarrierSet::satb_mark_queue_set().shared_satb_queue()->apply_closure_and_empty(_satb_cl);
}
if (thread->claim_oops_do(true, _thread_parity)) {
ShenandoahThreadLocalData::satb_mark_queue(thread).apply_closure_and_empty(_satb_cl);
}
}
};

View File

@ -180,9 +180,8 @@ jint ShenandoahHeap::initialize() {
// belong into a shared location.
ShenandoahBarrierSet::satb_mark_queue_set().initialize(this,
SATB_Q_CBL_mon,
20 /*G1SATBProcessCompletedThreshold */,
60 /* G1SATBBufferEnqueueingThresholdPercent */,
Shared_SATB_Q_lock);
20 /* G1SATBProcessCompletedThreshold */,
60 /* G1SATBBufferEnqueueingThresholdPercent */);
// Reserve space for prev and next bitmap.
size_t bitmap_page_size = UseLargePages ? (size_t)os::large_page_size() : (size_t)os::vm_page_size();

View File

@ -35,17 +35,15 @@ ShenandoahSATBMarkQueueSet::ShenandoahSATBMarkQueueSet() :
void ShenandoahSATBMarkQueueSet::initialize(ShenandoahHeap* const heap,
Monitor* cbl_mon,
int process_completed_threshold,
uint buffer_enqueue_threshold_percentage,
Mutex* lock) {
uint buffer_enqueue_threshold_percentage) {
SATBMarkQueueSet::initialize(cbl_mon,
&_satb_mark_queue_buffer_allocator,
process_completed_threshold,
buffer_enqueue_threshold_percentage,
lock);
buffer_enqueue_threshold_percentage);
_heap = heap;
}
SATBMarkQueue& ShenandoahSATBMarkQueueSet::satb_queue_for_thread(JavaThread* const t) const {
SATBMarkQueue& ShenandoahSATBMarkQueueSet::satb_queue_for_thread(Thread* const t) const {
return ShenandoahThreadLocalData::satb_mark_queue(t);
}

View File

@ -1,5 +1,5 @@
/*
* Copyright (c) 2018, Red Hat, Inc. All rights reserved.
* Copyright (c) 2018, 2019, Red Hat, Inc. All rights reserved.
*
* 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
@ -45,10 +45,9 @@ public:
void initialize(ShenandoahHeap* const heap,
Monitor* cbl_mon,
int process_completed_threshold,
uint buffer_enqueue_threshold_percentage,
Mutex* lock);
uint buffer_enqueue_threshold_percentage);
virtual SATBMarkQueue& satb_queue_for_thread(JavaThread* const t) const;
virtual SATBMarkQueue& satb_queue_for_thread(Thread* const t) const;
virtual void filter(SATBMarkQueue* queue);
};

View File

@ -127,12 +127,7 @@ public:
_satb_cl(satb_cl) {}
void do_thread(Thread* thread) {
if (thread->is_Java_thread()) {
JavaThread* jt = (JavaThread*)thread;
ShenandoahThreadLocalData::satb_mark_queue(jt).apply_closure_and_empty(_satb_cl);
} else if (thread->is_VM_thread()) {
ShenandoahBarrierSet::satb_mark_queue_set().shared_satb_queue()->apply_closure_and_empty(_satb_cl);
}
ShenandoahThreadLocalData::satb_mark_queue(thread).apply_closure_and_empty(_satb_cl);
}
};

View File

@ -1,5 +1,5 @@
/*
* Copyright (c) 2018, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2018, 2019, 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
@ -86,12 +86,12 @@ void ZBarrierSet::on_thread_destroy(Thread* thread) {
ZThreadLocalData::destroy(thread);
}
void ZBarrierSet::on_thread_attach(JavaThread* thread) {
void ZBarrierSet::on_thread_attach(Thread* thread) {
// Set thread local address bad mask
ZThreadLocalData::set_address_bad_mask(thread, ZAddressBadMask);
}
void ZBarrierSet::on_thread_detach(JavaThread* thread) {
void ZBarrierSet::on_thread_detach(Thread* thread) {
// Flush and free any remaining mark stacks
ZHeap::heap()->mark_flush_and_free(thread);
}

View File

@ -1,5 +1,5 @@
/*
* Copyright (c) 2015, 2018, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2015, 2019, 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
@ -37,8 +37,8 @@ public:
virtual void on_thread_create(Thread* thread);
virtual void on_thread_destroy(Thread* thread);
virtual void on_thread_attach(JavaThread* thread);
virtual void on_thread_detach(JavaThread* thread);
virtual void on_thread_attach(Thread* thread);
virtual void on_thread_detach(Thread* thread);
virtual void print_on(outputStream* st) const {}

View File

@ -80,7 +80,6 @@ Monitor* CGC_lock = NULL;
Monitor* STS_lock = NULL;
Monitor* FullGCCount_lock = NULL;
Monitor* SATB_Q_CBL_mon = NULL;
Mutex* Shared_SATB_Q_lock = NULL;
Monitor* DirtyCardQ_CBL_mon = NULL;
Mutex* Shared_DirtyCardQ_lock = NULL;
Mutex* MarkStackFreeList_lock = NULL;
@ -216,7 +215,6 @@ void mutex_init() {
}
if (UseG1GC) {
def(SATB_Q_CBL_mon , PaddedMonitor, access, true, Monitor::_safepoint_check_never);
def(Shared_SATB_Q_lock , PaddedMutex , access + 1, true, Monitor::_safepoint_check_never);
def(DirtyCardQ_CBL_mon , PaddedMonitor, access, true, Monitor::_safepoint_check_never);
def(Shared_DirtyCardQ_lock , PaddedMutex , access + 1, true, Monitor::_safepoint_check_never);
@ -235,7 +233,6 @@ void mutex_init() {
}
if (UseShenandoahGC) {
def(SATB_Q_CBL_mon , PaddedMonitor, access, true, Monitor::_safepoint_check_never);
def(Shared_SATB_Q_lock , PaddedMutex , access + 1, true, Monitor::_safepoint_check_never);
def(StringDedupQueue_lock , PaddedMonitor, leaf, true, Monitor::_safepoint_check_never);
def(StringDedupTable_lock , PaddedMutex , leaf, true, Monitor::_safepoint_check_never);

View File

@ -78,10 +78,6 @@ extern Monitor* STS_lock; // used for joining/leaving Sus
extern Monitor* FullGCCount_lock; // in support of "concurrent" full gc
extern Monitor* SATB_Q_CBL_mon; // Protects SATB Q
// completed buffer queue.
extern Mutex* Shared_SATB_Q_lock; // Lock protecting SATB
// queue shared by
// non-Java threads.
extern Monitor* DirtyCardQ_CBL_mon; // Protects dirty card Q
// completed buffer queue.
extern Mutex* Shared_DirtyCardQ_lock; // Lock protecting dirty card

View File

@ -1311,7 +1311,9 @@ void NonJavaThread::remove_from_the_list() {
}
void NonJavaThread::pre_run() {
// Initialize BarrierSet-related data before adding to list.
assert(BarrierSet::barrier_set() != NULL, "invariant");
BarrierSet::barrier_set()->on_thread_attach(this);
add_to_the_list();
// This is slightly odd in that NamedThread is a subclass, but
@ -1322,6 +1324,8 @@ void NonJavaThread::pre_run() {
void NonJavaThread::post_run() {
JFR_ONLY(Jfr::on_thread_exit(this);)
// Clean up BarrierSet data before removing from list.
BarrierSet::barrier_set()->on_thread_detach(this);
remove_from_the_list();
// Ensure thread-local-storage is cleared before termination.
Thread::clear_thread_current();