8218089: Rename DirtyCardQueue et al to follow usual G1 naming conventions
Summary: Move files and rename classes.
Reviewed-by: tschatzl, lkorinth
--- a/src/hotspot/cpu/ppc/gc/g1/g1BarrierSetAssembler_ppc.cpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/cpu/ppc/gc/g1/g1BarrierSetAssembler_ppc.cpp Wed Feb 13 17:38:14 2019 -0500
@@ -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.
* Copyright (c) 2018, SAP SE. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
@@ -29,6 +29,7 @@
#include "gc/g1/g1BarrierSetAssembler.hpp"
#include "gc/g1/g1BarrierSetRuntime.hpp"
#include "gc/g1/g1CardTable.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1SATBMarkQueueSet.hpp"
#include "gc/g1/g1ThreadLocalData.hpp"
#include "gc/g1/heapRegion.hpp"
@@ -512,7 +513,7 @@
__ bind(restart);
- // Get the index into the update buffer. DirtyCardQueue::_index is
+ // Get the index into the update buffer. G1DirtyCardQueue::_index is
// a size_t so ld_ptr is appropriate here.
__ ld(tmp2, dirty_card_q_index_byte_offset, R16_thread);
@@ -539,7 +540,7 @@
__ mflr(R0);
__ std(R0, _abi(lr), R1_SP);
__ push_frame_reg_args(nbytes_save, R0); // dummy frame for C call
- __ call_VM_leaf(CAST_FROM_FN_PTR(address, DirtyCardQueueSet::handle_zero_index_for_thread), R16_thread);
+ __ call_VM_leaf(CAST_FROM_FN_PTR(address, G1DirtyCardQueueSet::handle_zero_index_for_thread), R16_thread);
__ pop_frame();
__ ld(R0, _abi(lr), R1_SP);
__ mtlr(R0);
--- a/src/hotspot/cpu/s390/gc/g1/g1BarrierSetAssembler_s390.cpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/cpu/s390/gc/g1/g1BarrierSetAssembler_s390.cpp Wed Feb 13 17:38:14 2019 -0500
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2018, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2019, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2018, SAP SE. All rights reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
@@ -30,6 +30,7 @@
#include "gc/g1/g1BarrierSet.hpp"
#include "gc/g1/g1BarrierSetAssembler.hpp"
#include "gc/g1/g1BarrierSetRuntime.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1SATBMarkQueueSet.hpp"
#include "gc/g1/g1ThreadLocalData.hpp"
#include "gc/g1/heapRegion.hpp"
@@ -587,7 +588,7 @@
__ bind(restart);
- // Get the index into the update buffer. DirtyCardQueue::_index is
+ // Get the index into the update buffer. G1DirtyCardQueue::_index is
// a size_t so z_ltg is appropriate here.
__ z_ltg(idx, Address(Z_thread, dirty_card_q_index_byte_offset));
@@ -607,7 +608,7 @@
__ bind(refill);
save_volatile_registers(sasm);
__ z_lgr(idx, addr_card); // Save addr_card, tmp3 must be non-volatile.
- __ call_VM_leaf(CAST_FROM_FN_PTR(address, DirtyCardQueueSet::handle_zero_index_for_thread),
+ __ call_VM_leaf(CAST_FROM_FN_PTR(address, G1DirtyCardQueueSet::handle_zero_index_for_thread),
Z_thread);
__ z_lgr(addr_card, idx);
restore_volatile_registers(sasm); // Restore addr_card.
--- a/src/hotspot/cpu/sparc/gc/g1/g1BarrierSetAssembler_sparc.cpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/cpu/sparc/gc/g1/g1BarrierSetAssembler_sparc.cpp Wed Feb 13 17:38:14 2019 -0500
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2018, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 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
@@ -28,6 +28,7 @@
#include "gc/g1/g1BarrierSetAssembler.hpp"
#include "gc/g1/g1BarrierSetRuntime.hpp"
#include "gc/g1/g1CardTable.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1SATBMarkQueueSet.hpp"
#include "gc/g1/g1ThreadLocalData.hpp"
#include "gc/g1/heapRegion.hpp"
@@ -315,7 +316,7 @@
int dirty_card_q_buf_byte_offset = in_bytes(G1ThreadLocalData::dirty_card_queue_buffer_offset());
__ bind(restart);
- // Load the index into the update buffer. DirtyCardQueue::_index is
+ // Load the index into the update buffer. G1DirtyCardQueue::_index is
// a size_t so ld_ptr is appropriate here.
__ ld_ptr(G2_thread, dirty_card_q_index_byte_offset, L0);
@@ -333,7 +334,7 @@
__ bind(refill);
address handle_zero =
CAST_FROM_FN_PTR(address,
- &DirtyCardQueueSet::handle_zero_index_for_thread);
+ &G1DirtyCardQueueSet::handle_zero_index_for_thread);
// This should be rare enough that we can afford to save all the
// scratch registers that the calling context might be using.
__ mov(G1_scratch, L3);
@@ -673,7 +674,7 @@
__ bind(restart);
- // Get the index into the update buffer. DirtyCardQueue::_index is
+ // Get the index into the update buffer. G1DirtyCardQueue::_index is
// a size_t so ld_ptr is appropriate here.
__ ld_ptr(G2_thread, dirty_card_q_index_byte_offset, tmp3);
@@ -694,7 +695,7 @@
__ call_VM_leaf(L7_thread_cache,
CAST_FROM_FN_PTR(address,
- DirtyCardQueueSet::handle_zero_index_for_thread),
+ G1DirtyCardQueueSet::handle_zero_index_for_thread),
G2_thread);
__ restore_live_registers(true);
--- a/src/hotspot/share/gc/g1/dirtyCardQueue.cpp Wed Feb 13 15:50:08 2019 -0500
+++ /dev/null Thu Jan 01 00:00:00 1970 +0000
@@ -1,236 +0,0 @@
-/*
- * 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
- * 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/g1/dirtyCardQueue.hpp"
-#include "gc/g1/g1CollectedHeap.inline.hpp"
-#include "gc/g1/g1FreeIdSet.hpp"
-#include "gc/g1/g1RemSet.hpp"
-#include "gc/g1/g1ThreadLocalData.hpp"
-#include "gc/g1/heapRegionRemSet.hpp"
-#include "gc/shared/suspendibleThreadSet.hpp"
-#include "gc/shared/workgroup.hpp"
-#include "runtime/atomic.hpp"
-#include "runtime/flags/flagSetting.hpp"
-#include "runtime/mutexLocker.hpp"
-#include "runtime/safepoint.hpp"
-#include "runtime/thread.inline.hpp"
-#include "runtime/threadSMR.hpp"
-
-// Closure used for updating remembered sets and recording references that
-// point into the collection set while the mutator is running.
-// Assumed to be only executed concurrently with the mutator. Yields via
-// SuspendibleThreadSet after every card.
-class G1RefineCardConcurrentlyClosure: public CardTableEntryClosure {
-public:
- bool do_card_ptr(jbyte* card_ptr, uint worker_i) {
- G1CollectedHeap::heap()->g1_rem_set()->refine_card_concurrently(card_ptr, worker_i);
-
- if (SuspendibleThreadSet::should_yield()) {
- // Caller will actually yield.
- return false;
- }
- // Otherwise, we finished successfully; return true.
- return true;
- }
-};
-
-DirtyCardQueue::DirtyCardQueue(DirtyCardQueueSet* qset, bool permanent) :
- // Dirty card queues are always active, so we create them with their
- // active field set to true.
- PtrQueue(qset, permanent, true /* active */)
-{ }
-
-DirtyCardQueue::~DirtyCardQueue() {
- if (!is_permanent()) {
- flush();
- }
-}
-
-DirtyCardQueueSet::DirtyCardQueueSet(bool notify_when_complete) :
- PtrQueueSet(notify_when_complete),
- _shared_dirty_card_queue(this, true /* permanent */),
- _free_ids(NULL),
- _processed_buffers_mut(0),
- _processed_buffers_rs_thread(0),
- _cur_par_buffer_node(NULL)
-{
- _all_active = true;
-}
-
-DirtyCardQueueSet::~DirtyCardQueueSet() {
- delete _free_ids;
-}
-
-// Determines how many mutator threads can process the buffers in parallel.
-uint DirtyCardQueueSet::num_par_ids() {
- return (uint)os::initial_active_processor_count();
-}
-
-void DirtyCardQueueSet::initialize(Monitor* cbl_mon,
- BufferNode::Allocator* allocator,
- Mutex* lock,
- bool init_free_ids) {
- PtrQueueSet::initialize(cbl_mon, allocator);
- _shared_dirty_card_queue.set_lock(lock);
- if (init_free_ids) {
- _free_ids = new G1FreeIdSet(0, num_par_ids());
- }
-}
-
-void DirtyCardQueueSet::handle_zero_index_for_thread(JavaThread* t) {
- G1ThreadLocalData::dirty_card_queue(t).handle_zero_index();
-}
-
-bool DirtyCardQueueSet::apply_closure_to_buffer(CardTableEntryClosure* cl,
- BufferNode* node,
- bool consume,
- uint worker_i) {
- if (cl == NULL) return true;
- bool result = true;
- void** buf = BufferNode::make_buffer_from_node(node);
- size_t i = node->index();
- size_t limit = buffer_size();
- for ( ; i < limit; ++i) {
- jbyte* card_ptr = static_cast<jbyte*>(buf[i]);
- assert(card_ptr != NULL, "invariant");
- if (!cl->do_card_ptr(card_ptr, worker_i)) {
- result = false; // Incomplete processing.
- break;
- }
- }
- if (consume) {
- assert(i <= buffer_size(), "invariant");
- node->set_index(i);
- }
- return result;
-}
-
-#ifndef ASSERT
-#define assert_fully_consumed(node, buffer_size)
-#else
-#define assert_fully_consumed(node, buffer_size) \
- do { \
- size_t _afc_index = (node)->index(); \
- size_t _afc_size = (buffer_size); \
- assert(_afc_index == _afc_size, \
- "Buffer was not fully consumed as claimed: index: " \
- SIZE_FORMAT ", size: " SIZE_FORMAT, \
- _afc_index, _afc_size); \
- } while (0)
-#endif // ASSERT
-
-bool DirtyCardQueueSet::mut_process_buffer(BufferNode* node) {
- guarantee(_free_ids != NULL, "must be");
-
- uint worker_i = _free_ids->claim_par_id(); // temporarily claim an id
- G1RefineCardConcurrentlyClosure cl;
- bool result = apply_closure_to_buffer(&cl, node, true, worker_i);
- _free_ids->release_par_id(worker_i); // release the id
-
- if (result) {
- assert_fully_consumed(node, buffer_size());
- Atomic::inc(&_processed_buffers_mut);
- }
- return result;
-}
-
-bool DirtyCardQueueSet::refine_completed_buffer_concurrently(uint worker_i, size_t stop_at) {
- G1RefineCardConcurrentlyClosure cl;
- return apply_closure_to_completed_buffer(&cl, worker_i, stop_at, false);
-}
-
-bool DirtyCardQueueSet::apply_closure_during_gc(CardTableEntryClosure* cl, uint worker_i) {
- assert_at_safepoint();
- return apply_closure_to_completed_buffer(cl, worker_i, 0, true);
-}
-
-bool DirtyCardQueueSet::apply_closure_to_completed_buffer(CardTableEntryClosure* cl,
- uint worker_i,
- size_t stop_at,
- bool during_pause) {
- assert(!during_pause || stop_at == 0, "Should not leave any completed buffers during a pause");
- BufferNode* nd = get_completed_buffer(stop_at);
- if (nd == NULL) {
- return false;
- } else {
- if (apply_closure_to_buffer(cl, nd, true, worker_i)) {
- assert_fully_consumed(nd, buffer_size());
- // Done with fully processed buffer.
- deallocate_buffer(nd);
- Atomic::inc(&_processed_buffers_rs_thread);
- } else {
- // Return partially processed buffer to the queue.
- guarantee(!during_pause, "Should never stop early");
- enqueue_completed_buffer(nd);
- }
- return true;
- }
-}
-
-void DirtyCardQueueSet::par_apply_closure_to_all_completed_buffers(CardTableEntryClosure* cl) {
- BufferNode* nd = _cur_par_buffer_node;
- while (nd != NULL) {
- BufferNode* next = nd->next();
- BufferNode* actual = Atomic::cmpxchg(next, &_cur_par_buffer_node, nd);
- if (actual == nd) {
- bool b = apply_closure_to_buffer(cl, nd, false);
- guarantee(b, "Should not stop early.");
- nd = next;
- } else {
- nd = actual;
- }
- }
-}
-
-void DirtyCardQueueSet::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();
- }
- shared_dirty_card_queue()->reset();
-}
-
-void DirtyCardQueueSet::concatenate_log(DirtyCardQueue& dcq) {
- if (!dcq.is_empty()) {
- dcq.flush();
- }
-}
-
-void DirtyCardQueueSet::concatenate_logs() {
- // Iterate over all the threads, if we find a partial log add it to
- // the global list of logs. Temporarily turn off the limit on the number
- // of outstanding buffers.
- 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));
- }
- concatenate_log(_shared_dirty_card_queue);
- set_max_completed_buffers(old_limit);
-}
--- a/src/hotspot/share/gc/g1/dirtyCardQueue.hpp Wed Feb 13 15:50:08 2019 -0500
+++ /dev/null Thu Jan 01 00:00:00 1970 +0000
@@ -1,167 +0,0 @@
-/*
- * 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
- * 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_GC_G1_DIRTYCARDQUEUE_HPP
-#define SHARE_GC_G1_DIRTYCARDQUEUE_HPP
-
-#include "gc/shared/ptrQueue.hpp"
-#include "memory/allocation.hpp"
-
-class DirtyCardQueueSet;
-class G1FreeIdSet;
-class JavaThread;
-class Monitor;
-
-// A closure class for processing card table entries. Note that we don't
-// require these closure objects to be stack-allocated.
-class CardTableEntryClosure: public CHeapObj<mtGC> {
-public:
- // Process the card whose card table entry is "card_ptr". If returns
- // "false", terminate the iteration early.
- virtual bool do_card_ptr(jbyte* card_ptr, uint worker_i) = 0;
-};
-
-// A ptrQueue whose elements are "oops", pointers to object heads.
-class DirtyCardQueue: public PtrQueue {
-public:
- DirtyCardQueue(DirtyCardQueueSet* qset, bool permanent = false);
-
- // Flush before destroying; queue may be used to capture pending work while
- // doing something else, with auto-flush on completion.
- ~DirtyCardQueue();
-
- // Process queue entries and release resources.
- void flush() { flush_impl(); }
-
- // Compiler support.
- static ByteSize byte_offset_of_index() {
- return PtrQueue::byte_offset_of_index<DirtyCardQueue>();
- }
- using PtrQueue::byte_width_of_index;
-
- static ByteSize byte_offset_of_buf() {
- return PtrQueue::byte_offset_of_buf<DirtyCardQueue>();
- }
- using PtrQueue::byte_width_of_buf;
-
-};
-
-
-
-class DirtyCardQueueSet: public PtrQueueSet {
- DirtyCardQueue _shared_dirty_card_queue;
-
- // Apply the closure to the elements of "node" from it's index to
- // buffer_size. If all closure applications return true, then
- // returns true. Stops processing after the first closure
- // application that returns false, and returns false from this
- // function. If "consume" is true, the node's index is updated to
- // exclude the processed elements, e.g. up to the element for which
- // the closure returned false.
- bool apply_closure_to_buffer(CardTableEntryClosure* cl,
- BufferNode* node,
- bool consume,
- uint worker_i = 0);
-
- // If there are more than stop_at completed buffers, pop one, apply
- // the specified closure to its active elements, and return true.
- // Otherwise return false.
- //
- // A completely processed buffer is freed. However, if a closure
- // invocation returns false, processing is stopped and the partially
- // processed buffer (with its index updated to exclude the processed
- // elements, e.g. up to the element for which the closure returned
- // false) is returned to the completed buffer set.
- //
- // If during_pause is true, stop_at must be zero, and the closure
- // must never return false.
- bool apply_closure_to_completed_buffer(CardTableEntryClosure* cl,
- uint worker_i,
- size_t stop_at,
- bool during_pause);
-
- bool mut_process_buffer(BufferNode* node);
-
- G1FreeIdSet* _free_ids;
-
- // The number of completed buffers processed by mutator and rs thread,
- // respectively.
- jint _processed_buffers_mut;
- jint _processed_buffers_rs_thread;
-
- // Current buffer node used for parallel iteration.
- BufferNode* volatile _cur_par_buffer_node;
-
- void concatenate_log(DirtyCardQueue& dcq);
-
-public:
- DirtyCardQueueSet(bool notify_when_complete = true);
- ~DirtyCardQueueSet();
-
- void initialize(Monitor* cbl_mon,
- BufferNode::Allocator* allocator,
- Mutex* lock,
- bool init_free_ids = false);
-
- // The number of parallel ids that can be claimed to allow collector or
- // mutator threads to do card-processing work.
- static uint num_par_ids();
-
- static void handle_zero_index_for_thread(JavaThread* t);
-
- // Apply G1RefineCardConcurrentlyClosure to completed buffers until there are stop_at
- // completed buffers remaining.
- bool refine_completed_buffer_concurrently(uint worker_i, size_t stop_at);
-
- // Apply the given closure to all completed buffers. The given closure's do_card_ptr
- // must never return false. Must only be called during GC.
- bool apply_closure_during_gc(CardTableEntryClosure* cl, uint worker_i);
-
- void reset_for_par_iteration() { _cur_par_buffer_node = completed_buffers_head(); }
- // Applies the current closure to all completed buffers, non-consumptively.
- // Can be used in parallel, all callers using the iteration state initialized
- // by reset_for_par_iteration.
- void par_apply_closure_to_all_completed_buffers(CardTableEntryClosure* cl);
-
- DirtyCardQueue* shared_dirty_card_queue() {
- return &_shared_dirty_card_queue;
- }
-
- // If a full collection is happening, reset partial logs, and ignore
- // completed ones: the full collection will make them all irrelevant.
- void abandon_logs();
-
- // If any threads have partial logs, add them to the global list of logs.
- void concatenate_logs();
-
- jint processed_buffers_mut() {
- return _processed_buffers_mut;
- }
- jint processed_buffers_rs_thread() {
- return _processed_buffers_rs_thread;
- }
-
-};
-
-#endif // SHARE_GC_G1_DIRTYCARDQUEUE_HPP
--- a/src/hotspot/share/gc/g1/g1BarrierSet.hpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1BarrierSet.hpp Wed Feb 13 17:38:14 2019 -0500
@@ -25,11 +25,10 @@
#ifndef SHARE_GC_G1_G1BARRIERSET_HPP
#define SHARE_GC_G1_G1BARRIERSET_HPP
-#include "gc/g1/dirtyCardQueue.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1SATBMarkQueueSet.hpp"
#include "gc/shared/cardTableBarrierSet.hpp"
-class DirtyCardQueueSet;
class CardTable;
class G1CardTable;
@@ -42,7 +41,7 @@
BufferNode::Allocator _satb_mark_queue_buffer_allocator;
BufferNode::Allocator _dirty_card_queue_buffer_allocator;
G1SATBMarkQueueSet _satb_mark_queue_set;
- DirtyCardQueueSet _dirty_card_queue_set;
+ G1DirtyCardQueueSet _dirty_card_queue_set;
static G1BarrierSet* g1_barrier_set() {
return barrier_set_cast<G1BarrierSet>(BarrierSet::barrier_set());
@@ -88,7 +87,7 @@
return g1_barrier_set()->_satb_mark_queue_set;
}
- static DirtyCardQueueSet& dirty_card_queue_set() {
+ static G1DirtyCardQueueSet& dirty_card_queue_set() {
return g1_barrier_set()->_dirty_card_queue_set;
}
--- a/src/hotspot/share/gc/g1/g1CollectedHeap.cpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1CollectedHeap.cpp Wed Feb 13 17:38:14 2019 -0500
@@ -37,6 +37,7 @@
#include "gc/g1/g1ConcurrentRefine.hpp"
#include "gc/g1/g1ConcurrentRefineThread.hpp"
#include "gc/g1/g1ConcurrentMarkThread.inline.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1EvacStats.inline.hpp"
#include "gc/g1/g1FullCollector.hpp"
#include "gc/g1/g1GCPhaseTimes.hpp"
@@ -107,7 +108,7 @@
// apply to TLAB allocation, which is not part of this interface: it
// is done by clients of this interface.)
-class RedirtyLoggedCardTableEntryClosure : public CardTableEntryClosure {
+class RedirtyLoggedCardTableEntryClosure : public G1CardTableEntryClosure {
private:
size_t _num_dirtied;
G1CollectedHeap* _g1h;
@@ -124,7 +125,7 @@
}
public:
- RedirtyLoggedCardTableEntryClosure(G1CollectedHeap* g1h) : CardTableEntryClosure(),
+ RedirtyLoggedCardTableEntryClosure(G1CollectedHeap* g1h) : G1CardTableEntryClosure(),
_num_dirtied(0), _g1h(g1h), _g1_ct(g1h->card_table()) { }
bool do_card_ptr(jbyte* card_ptr, uint worker_i) {
@@ -1811,7 +1812,7 @@
}
{
- DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
+ G1DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
dcqs.set_process_completed_buffers_threshold(concurrent_refine()->yellow_zone());
dcqs.set_max_completed_buffers(concurrent_refine()->red_zone());
}
@@ -1954,12 +1955,12 @@
return _hrm->total_free_bytes();
}
-void G1CollectedHeap::iterate_hcc_closure(CardTableEntryClosure* cl, uint worker_i) {
+void G1CollectedHeap::iterate_hcc_closure(G1CardTableEntryClosure* cl, uint worker_i) {
_hot_card_cache->drain(cl, worker_i);
}
-void G1CollectedHeap::iterate_dirty_card_closure(CardTableEntryClosure* cl, uint worker_i) {
- DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
+void G1CollectedHeap::iterate_dirty_card_closure(G1CardTableEntryClosure* cl, uint worker_i) {
+ G1DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
size_t n_completed_buffers = 0;
while (dcqs.apply_closure_during_gc(cl, worker_i)) {
n_completed_buffers++;
@@ -2605,10 +2606,10 @@
size_t G1CollectedHeap::pending_card_num() {
size_t extra_cards = 0;
for (JavaThreadIteratorWithHandle jtiwh; JavaThread *curr = jtiwh.next(); ) {
- DirtyCardQueue& dcq = G1ThreadLocalData::dirty_card_queue(curr);
+ G1DirtyCardQueue& dcq = G1ThreadLocalData::dirty_card_queue(curr);
extra_cards += dcq.size();
}
- DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
+ G1DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
size_t buffer_size = dcqs.buffer_size();
size_t buffer_num = dcqs.completed_buffers_num();
@@ -2630,7 +2631,7 @@
size_t _total_humongous;
size_t _candidate_humongous;
- DirtyCardQueue _dcq;
+ G1DirtyCardQueue _dcq;
bool humongous_region_is_candidate(G1CollectedHeap* g1h, HeapRegion* region) const {
assert(region->is_starts_humongous(), "Must start a humongous object");
@@ -3410,10 +3411,10 @@
class G1RedirtyLoggedCardsTask : public AbstractGangTask {
private:
- DirtyCardQueueSet* _queue;
+ G1DirtyCardQueueSet* _queue;
G1CollectedHeap* _g1h;
public:
- G1RedirtyLoggedCardsTask(DirtyCardQueueSet* queue, G1CollectedHeap* g1h) : AbstractGangTask("Redirty Cards"),
+ G1RedirtyLoggedCardsTask(G1DirtyCardQueueSet* queue, G1CollectedHeap* g1h) : AbstractGangTask("Redirty Cards"),
_queue(queue), _g1h(g1h) { }
virtual void work(uint worker_id) {
@@ -3434,7 +3435,7 @@
dirty_card_queue_set().reset_for_par_iteration();
workers()->run_task(&redirty_task);
- DirtyCardQueueSet& dcq = G1BarrierSet::dirty_card_queue_set();
+ G1DirtyCardQueueSet& dcq = G1BarrierSet::dirty_card_queue_set();
dcq.merge_bufferlists(&dirty_card_queue_set());
assert(dirty_card_queue_set().completed_buffers_num() == 0, "All should be consumed");
--- a/src/hotspot/share/gc/g1/g1CollectedHeap.hpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1CollectedHeap.hpp Wed Feb 13 17:38:14 2019 -0500
@@ -31,6 +31,7 @@
#include "gc/g1/g1CollectionSet.hpp"
#include "gc/g1/g1CollectorState.hpp"
#include "gc/g1/g1ConcurrentMark.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1EdenRegions.hpp"
#include "gc/g1/g1EvacFailure.hpp"
#include "gc/g1/g1EvacStats.hpp"
@@ -758,7 +759,7 @@
// A set of cards that cover the objects for which the Rsets should be updated
// concurrently after the collection.
- DirtyCardQueueSet _dirty_card_queue_set;
+ G1DirtyCardQueueSet _dirty_card_queue_set;
// After a collection pause, convert the regions in the collection set into free
// regions.
@@ -918,7 +919,7 @@
uint num_task_queues() const;
// A set of cards where updates happened during the GC
- DirtyCardQueueSet& dirty_card_queue_set() { return _dirty_card_queue_set; }
+ G1DirtyCardQueueSet& dirty_card_queue_set() { return _dirty_card_queue_set; }
// Create a G1CollectedHeap with the specified policy.
// Must call the initialize method afterwards.
@@ -983,10 +984,10 @@
void scrub_rem_set();
// Apply the given closure on all cards in the Hot Card Cache, emptying it.
- void iterate_hcc_closure(CardTableEntryClosure* cl, uint worker_i);
+ void iterate_hcc_closure(G1CardTableEntryClosure* cl, uint worker_i);
// Apply the given closure on all cards in the Dirty Card Queue Set, emptying it.
- void iterate_dirty_card_closure(CardTableEntryClosure* cl, uint worker_i);
+ void iterate_dirty_card_closure(G1CardTableEntryClosure* cl, uint worker_i);
// The shared block offset table array.
G1BlockOffsetTable* bot() const { return _bot; }
--- a/src/hotspot/share/gc/g1/g1ConcurrentMark.cpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1ConcurrentMark.cpp Wed Feb 13 17:38:14 2019 -0500
@@ -30,6 +30,7 @@
#include "gc/g1/g1CollectorState.hpp"
#include "gc/g1/g1ConcurrentMark.inline.hpp"
#include "gc/g1/g1ConcurrentMarkThread.inline.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1HeapVerifier.hpp"
#include "gc/g1/g1OopClosures.inline.hpp"
#include "gc/g1/g1Policy.hpp"
@@ -372,7 +373,7 @@
// _finger set in set_non_marking_state
- _worker_id_offset(DirtyCardQueueSet::num_par_ids() + G1ConcRefinementThreads),
+ _worker_id_offset(G1DirtyCardQueueSet::num_par_ids() + G1ConcRefinementThreads),
_max_num_tasks(ParallelGCThreads),
// _num_active_tasks set in set_non_marking_state()
// _tasks set inside the constructor
--- a/src/hotspot/share/gc/g1/g1ConcurrentRefine.cpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1ConcurrentRefine.cpp Wed Feb 13 17:38:14 2019 -0500
@@ -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
@@ -26,6 +26,7 @@
#include "gc/g1/g1BarrierSet.hpp"
#include "gc/g1/g1ConcurrentRefine.hpp"
#include "gc/g1/g1ConcurrentRefineThread.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "logging/log.hpp"
#include "memory/allocation.inline.hpp"
#include "runtime/java.hpp"
@@ -378,7 +379,7 @@
void G1ConcurrentRefine::adjust(double update_rs_time,
size_t update_rs_processed_buffers,
double goal_ms) {
- DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
+ G1DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
if (G1UseAdaptiveConcRefinement) {
update_zones(update_rs_time, update_rs_processed_buffers, goal_ms);
@@ -386,7 +387,7 @@
// Change the barrier params
if (max_num_threads() == 0) {
// Disable dcqs notification when there are no threads to notify.
- dcqs.set_process_completed_buffers_threshold(DirtyCardQueueSet::ProcessCompletedBuffersThresholdNever);
+ dcqs.set_process_completed_buffers_threshold(G1DirtyCardQueueSet::ProcessCompletedBuffersThresholdNever);
} else {
// Worker 0 is the primary; wakeup is via dcqs notification.
STATIC_ASSERT(max_yellow_zone <= INT_MAX);
@@ -417,7 +418,7 @@
}
uint G1ConcurrentRefine::worker_id_offset() {
- return DirtyCardQueueSet::num_par_ids();
+ return G1DirtyCardQueueSet::num_par_ids();
}
void G1ConcurrentRefine::maybe_activate_more_threads(uint worker_id, size_t num_cur_buffers) {
@@ -427,7 +428,7 @@
}
bool G1ConcurrentRefine::do_refinement_step(uint worker_id) {
- DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
+ G1DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
size_t curr_buffer_num = dcqs.completed_buffers_num();
// If the number of the buffers falls down into the yellow zone,
--- a/src/hotspot/share/gc/g1/g1ConcurrentRefine.hpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1ConcurrentRefine.hpp Wed Feb 13 17:38:14 2019 -0500
@@ -29,7 +29,6 @@
#include "utilities/globalDefinitions.hpp"
// Forward decl
-class CardTableEntryClosure;
class G1ConcurrentRefine;
class G1ConcurrentRefineThread;
class outputStream;
--- a/src/hotspot/share/gc/g1/g1ConcurrentRefineThread.cpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1ConcurrentRefineThread.cpp Wed Feb 13 17:38:14 2019 -0500
@@ -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
@@ -26,6 +26,7 @@
#include "gc/g1/g1BarrierSet.hpp"
#include "gc/g1/g1ConcurrentRefine.hpp"
#include "gc/g1/g1ConcurrentRefineThread.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/shared/suspendibleThreadSet.hpp"
#include "logging/log.hpp"
#include "memory/resourceArea.hpp"
@@ -65,7 +66,7 @@
}
bool G1ConcurrentRefineThread::is_active() {
- DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
+ G1DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
return is_primary() ? dcqs.process_completed_buffers() : _active;
}
@@ -74,7 +75,7 @@
if (!is_primary()) {
set_active(true);
} else {
- DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
+ G1DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
dcqs.set_process_completed_buffers(true);
}
_monitor->notify();
@@ -85,7 +86,7 @@
if (!is_primary()) {
set_active(false);
} else {
- DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
+ G1DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
dcqs.set_process_completed_buffers(false);
}
}
--- a/src/hotspot/share/gc/g1/g1ConcurrentRefineThread.hpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1ConcurrentRefineThread.hpp Wed Feb 13 17:38:14 2019 -0500
@@ -25,11 +25,9 @@
#ifndef SHARE_GC_G1_G1CONCURRENTREFINETHREAD_HPP
#define SHARE_GC_G1_G1CONCURRENTREFINETHREAD_HPP
-#include "gc/g1/dirtyCardQueue.hpp"
#include "gc/shared/concurrentGCThread.hpp"
// Forward Decl.
-class CardTableEntryClosure;
class G1ConcurrentRefine;
// One or more G1 Concurrent Refinement Threads may be active if concurrent
--- /dev/null Thu Jan 01 00:00:00 1970 +0000
+++ b/src/hotspot/share/gc/g1/g1DirtyCardQueue.cpp Wed Feb 13 17:38:14 2019 -0500
@@ -0,0 +1,236 @@
+/*
+ * 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
+ * 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/g1/g1CollectedHeap.inline.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
+#include "gc/g1/g1FreeIdSet.hpp"
+#include "gc/g1/g1RemSet.hpp"
+#include "gc/g1/g1ThreadLocalData.hpp"
+#include "gc/g1/heapRegionRemSet.hpp"
+#include "gc/shared/suspendibleThreadSet.hpp"
+#include "gc/shared/workgroup.hpp"
+#include "runtime/atomic.hpp"
+#include "runtime/flags/flagSetting.hpp"
+#include "runtime/mutexLocker.hpp"
+#include "runtime/safepoint.hpp"
+#include "runtime/thread.inline.hpp"
+#include "runtime/threadSMR.hpp"
+
+// Closure used for updating remembered sets and recording references that
+// point into the collection set while the mutator is running.
+// Assumed to be only executed concurrently with the mutator. Yields via
+// SuspendibleThreadSet after every card.
+class G1RefineCardConcurrentlyClosure: public G1CardTableEntryClosure {
+public:
+ bool do_card_ptr(jbyte* card_ptr, uint worker_i) {
+ G1CollectedHeap::heap()->g1_rem_set()->refine_card_concurrently(card_ptr, worker_i);
+
+ if (SuspendibleThreadSet::should_yield()) {
+ // Caller will actually yield.
+ return false;
+ }
+ // Otherwise, we finished successfully; return true.
+ return true;
+ }
+};
+
+G1DirtyCardQueue::G1DirtyCardQueue(G1DirtyCardQueueSet* qset, bool permanent) :
+ // Dirty card queues are always active, so we create them with their
+ // active field set to true.
+ PtrQueue(qset, permanent, true /* active */)
+{ }
+
+G1DirtyCardQueue::~G1DirtyCardQueue() {
+ if (!is_permanent()) {
+ flush();
+ }
+}
+
+G1DirtyCardQueueSet::G1DirtyCardQueueSet(bool notify_when_complete) :
+ PtrQueueSet(notify_when_complete),
+ _shared_dirty_card_queue(this, true /* permanent */),
+ _free_ids(NULL),
+ _processed_buffers_mut(0),
+ _processed_buffers_rs_thread(0),
+ _cur_par_buffer_node(NULL)
+{
+ _all_active = true;
+}
+
+G1DirtyCardQueueSet::~G1DirtyCardQueueSet() {
+ delete _free_ids;
+}
+
+// Determines how many mutator threads can process the buffers in parallel.
+uint G1DirtyCardQueueSet::num_par_ids() {
+ return (uint)os::initial_active_processor_count();
+}
+
+void G1DirtyCardQueueSet::initialize(Monitor* cbl_mon,
+ BufferNode::Allocator* allocator,
+ Mutex* lock,
+ bool init_free_ids) {
+ PtrQueueSet::initialize(cbl_mon, allocator);
+ _shared_dirty_card_queue.set_lock(lock);
+ if (init_free_ids) {
+ _free_ids = new G1FreeIdSet(0, num_par_ids());
+ }
+}
+
+void G1DirtyCardQueueSet::handle_zero_index_for_thread(JavaThread* t) {
+ G1ThreadLocalData::dirty_card_queue(t).handle_zero_index();
+}
+
+bool G1DirtyCardQueueSet::apply_closure_to_buffer(G1CardTableEntryClosure* cl,
+ BufferNode* node,
+ bool consume,
+ uint worker_i) {
+ if (cl == NULL) return true;
+ bool result = true;
+ void** buf = BufferNode::make_buffer_from_node(node);
+ size_t i = node->index();
+ size_t limit = buffer_size();
+ for ( ; i < limit; ++i) {
+ jbyte* card_ptr = static_cast<jbyte*>(buf[i]);
+ assert(card_ptr != NULL, "invariant");
+ if (!cl->do_card_ptr(card_ptr, worker_i)) {
+ result = false; // Incomplete processing.
+ break;
+ }
+ }
+ if (consume) {
+ assert(i <= buffer_size(), "invariant");
+ node->set_index(i);
+ }
+ return result;
+}
+
+#ifndef ASSERT
+#define assert_fully_consumed(node, buffer_size)
+#else
+#define assert_fully_consumed(node, buffer_size) \
+ do { \
+ size_t _afc_index = (node)->index(); \
+ size_t _afc_size = (buffer_size); \
+ assert(_afc_index == _afc_size, \
+ "Buffer was not fully consumed as claimed: index: " \
+ SIZE_FORMAT ", size: " SIZE_FORMAT, \
+ _afc_index, _afc_size); \
+ } while (0)
+#endif // ASSERT
+
+bool G1DirtyCardQueueSet::mut_process_buffer(BufferNode* node) {
+ guarantee(_free_ids != NULL, "must be");
+
+ uint worker_i = _free_ids->claim_par_id(); // temporarily claim an id
+ G1RefineCardConcurrentlyClosure cl;
+ bool result = apply_closure_to_buffer(&cl, node, true, worker_i);
+ _free_ids->release_par_id(worker_i); // release the id
+
+ if (result) {
+ assert_fully_consumed(node, buffer_size());
+ Atomic::inc(&_processed_buffers_mut);
+ }
+ return result;
+}
+
+bool G1DirtyCardQueueSet::refine_completed_buffer_concurrently(uint worker_i, size_t stop_at) {
+ G1RefineCardConcurrentlyClosure cl;
+ return apply_closure_to_completed_buffer(&cl, worker_i, stop_at, false);
+}
+
+bool G1DirtyCardQueueSet::apply_closure_during_gc(G1CardTableEntryClosure* cl, uint worker_i) {
+ assert_at_safepoint();
+ return apply_closure_to_completed_buffer(cl, worker_i, 0, true);
+}
+
+bool G1DirtyCardQueueSet::apply_closure_to_completed_buffer(G1CardTableEntryClosure* cl,
+ uint worker_i,
+ size_t stop_at,
+ bool during_pause) {
+ assert(!during_pause || stop_at == 0, "Should not leave any completed buffers during a pause");
+ BufferNode* nd = get_completed_buffer(stop_at);
+ if (nd == NULL) {
+ return false;
+ } else {
+ if (apply_closure_to_buffer(cl, nd, true, worker_i)) {
+ assert_fully_consumed(nd, buffer_size());
+ // Done with fully processed buffer.
+ deallocate_buffer(nd);
+ Atomic::inc(&_processed_buffers_rs_thread);
+ } else {
+ // Return partially processed buffer to the queue.
+ guarantee(!during_pause, "Should never stop early");
+ enqueue_completed_buffer(nd);
+ }
+ return true;
+ }
+}
+
+void G1DirtyCardQueueSet::par_apply_closure_to_all_completed_buffers(G1CardTableEntryClosure* cl) {
+ BufferNode* nd = _cur_par_buffer_node;
+ while (nd != NULL) {
+ BufferNode* next = nd->next();
+ BufferNode* actual = Atomic::cmpxchg(next, &_cur_par_buffer_node, nd);
+ if (actual == nd) {
+ bool b = apply_closure_to_buffer(cl, nd, false);
+ guarantee(b, "Should not stop early.");
+ nd = next;
+ } else {
+ nd = actual;
+ }
+ }
+}
+
+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();
+ }
+ shared_dirty_card_queue()->reset();
+}
+
+void G1DirtyCardQueueSet::concatenate_log(G1DirtyCardQueue& dcq) {
+ if (!dcq.is_empty()) {
+ dcq.flush();
+ }
+}
+
+void G1DirtyCardQueueSet::concatenate_logs() {
+ // Iterate over all the threads, if we find a partial log add it to
+ // the global list of logs. Temporarily turn off the limit on the number
+ // of outstanding buffers.
+ 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));
+ }
+ concatenate_log(_shared_dirty_card_queue);
+ set_max_completed_buffers(old_limit);
+}
--- /dev/null Thu Jan 01 00:00:00 1970 +0000
+++ b/src/hotspot/share/gc/g1/g1DirtyCardQueue.hpp Wed Feb 13 17:38:14 2019 -0500
@@ -0,0 +1,167 @@
+/*
+ * 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
+ * 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_GC_G1_G1DIRTYCARDQUEUE_HPP
+#define SHARE_GC_G1_G1DIRTYCARDQUEUE_HPP
+
+#include "gc/shared/ptrQueue.hpp"
+#include "memory/allocation.hpp"
+
+class G1DirtyCardQueueSet;
+class G1FreeIdSet;
+class JavaThread;
+class Monitor;
+
+// A closure class for processing card table entries. Note that we don't
+// require these closure objects to be stack-allocated.
+class G1CardTableEntryClosure: public CHeapObj<mtGC> {
+public:
+ // Process the card whose card table entry is "card_ptr". If returns
+ // "false", terminate the iteration early.
+ virtual bool do_card_ptr(jbyte* card_ptr, uint worker_i) = 0;
+};
+
+// A ptrQueue whose elements are "oops", pointers to object heads.
+class G1DirtyCardQueue: public PtrQueue {
+public:
+ G1DirtyCardQueue(G1DirtyCardQueueSet* qset, bool permanent = false);
+
+ // Flush before destroying; queue may be used to capture pending work while
+ // doing something else, with auto-flush on completion.
+ ~G1DirtyCardQueue();
+
+ // Process queue entries and release resources.
+ void flush() { flush_impl(); }
+
+ // Compiler support.
+ static ByteSize byte_offset_of_index() {
+ return PtrQueue::byte_offset_of_index<G1DirtyCardQueue>();
+ }
+ using PtrQueue::byte_width_of_index;
+
+ static ByteSize byte_offset_of_buf() {
+ return PtrQueue::byte_offset_of_buf<G1DirtyCardQueue>();
+ }
+ using PtrQueue::byte_width_of_buf;
+
+};
+
+
+
+class G1DirtyCardQueueSet: public PtrQueueSet {
+ G1DirtyCardQueue _shared_dirty_card_queue;
+
+ // Apply the closure to the elements of "node" from it's index to
+ // buffer_size. If all closure applications return true, then
+ // returns true. Stops processing after the first closure
+ // application that returns false, and returns false from this
+ // function. If "consume" is true, the node's index is updated to
+ // exclude the processed elements, e.g. up to the element for which
+ // the closure returned false.
+ bool apply_closure_to_buffer(G1CardTableEntryClosure* cl,
+ BufferNode* node,
+ bool consume,
+ uint worker_i = 0);
+
+ // If there are more than stop_at completed buffers, pop one, apply
+ // the specified closure to its active elements, and return true.
+ // Otherwise return false.
+ //
+ // A completely processed buffer is freed. However, if a closure
+ // invocation returns false, processing is stopped and the partially
+ // processed buffer (with its index updated to exclude the processed
+ // elements, e.g. up to the element for which the closure returned
+ // false) is returned to the completed buffer set.
+ //
+ // If during_pause is true, stop_at must be zero, and the closure
+ // must never return false.
+ bool apply_closure_to_completed_buffer(G1CardTableEntryClosure* cl,
+ uint worker_i,
+ size_t stop_at,
+ bool during_pause);
+
+ bool mut_process_buffer(BufferNode* node);
+
+ G1FreeIdSet* _free_ids;
+
+ // The number of completed buffers processed by mutator and rs thread,
+ // respectively.
+ jint _processed_buffers_mut;
+ jint _processed_buffers_rs_thread;
+
+ // Current buffer node used for parallel iteration.
+ BufferNode* volatile _cur_par_buffer_node;
+
+ void concatenate_log(G1DirtyCardQueue& dcq);
+
+public:
+ G1DirtyCardQueueSet(bool notify_when_complete = true);
+ ~G1DirtyCardQueueSet();
+
+ void initialize(Monitor* cbl_mon,
+ BufferNode::Allocator* allocator,
+ Mutex* lock,
+ bool init_free_ids = false);
+
+ // The number of parallel ids that can be claimed to allow collector or
+ // mutator threads to do card-processing work.
+ static uint num_par_ids();
+
+ static void handle_zero_index_for_thread(JavaThread* t);
+
+ // Apply G1RefineCardConcurrentlyClosure to completed buffers until there are stop_at
+ // completed buffers remaining.
+ bool refine_completed_buffer_concurrently(uint worker_i, size_t stop_at);
+
+ // Apply the given closure to all completed buffers. The given closure's do_card_ptr
+ // must never return false. Must only be called during GC.
+ bool apply_closure_during_gc(G1CardTableEntryClosure* cl, uint worker_i);
+
+ void reset_for_par_iteration() { _cur_par_buffer_node = completed_buffers_head(); }
+ // Applies the current closure to all completed buffers, non-consumptively.
+ // Can be used in parallel, all callers using the iteration state initialized
+ // by reset_for_par_iteration.
+ void par_apply_closure_to_all_completed_buffers(G1CardTableEntryClosure* cl);
+
+ G1DirtyCardQueue* shared_dirty_card_queue() {
+ return &_shared_dirty_card_queue;
+ }
+
+ // If a full collection is happening, reset partial logs, and ignore
+ // completed ones: the full collection will make them all irrelevant.
+ void abandon_logs();
+
+ // If any threads have partial logs, add them to the global list of logs.
+ void concatenate_logs();
+
+ jint processed_buffers_mut() {
+ return _processed_buffers_mut;
+ }
+ jint processed_buffers_rs_thread() {
+ return _processed_buffers_rs_thread;
+ }
+
+};
+
+#endif // SHARE_GC_G1_G1DIRTYCARDQUEUE_HPP
--- a/src/hotspot/share/gc/g1/g1EvacFailure.cpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1EvacFailure.cpp Wed Feb 13 17:38:14 2019 -0500
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2012, 2018, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2012, 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
@@ -23,10 +23,10 @@
*/
#include "precompiled.hpp"
-#include "gc/g1/dirtyCardQueue.hpp"
#include "gc/g1/g1CollectedHeap.inline.hpp"
#include "gc/g1/g1CollectorState.hpp"
#include "gc/g1/g1ConcurrentMark.inline.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1EvacFailure.hpp"
#include "gc/g1/g1HeapVerifier.hpp"
#include "gc/g1/g1OopClosures.inline.hpp"
@@ -41,11 +41,11 @@
class UpdateRSetDeferred : public BasicOopIterateClosure {
private:
G1CollectedHeap* _g1h;
- DirtyCardQueue* _dcq;
+ G1DirtyCardQueue* _dcq;
G1CardTable* _ct;
public:
- UpdateRSetDeferred(DirtyCardQueue* dcq) :
+ UpdateRSetDeferred(G1DirtyCardQueue* dcq) :
_g1h(G1CollectedHeap::heap()), _dcq(dcq), _ct(_g1h->card_table()) {}
virtual void do_oop(narrowOop* p) { do_oop_work(p); }
@@ -196,7 +196,7 @@
uint _worker_id;
HeapRegionClaimer* _hrclaimer;
- DirtyCardQueue _dcq;
+ G1DirtyCardQueue _dcq;
UpdateRSetDeferred _update_rset_cl;
public:
--- a/src/hotspot/share/gc/g1/g1HotCardCache.cpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1HotCardCache.cpp Wed Feb 13 17:38:14 2019 -0500
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2013, 2017, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2013, 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
@@ -23,8 +23,8 @@
*/
#include "precompiled.hpp"
-#include "gc/g1/dirtyCardQueue.hpp"
#include "gc/g1/g1CollectedHeap.inline.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1HotCardCache.hpp"
#include "runtime/atomic.hpp"
@@ -83,7 +83,7 @@
return (previous_ptr == current_ptr) ? previous_ptr : card_ptr;
}
-void G1HotCardCache::drain(CardTableEntryClosure* cl, uint worker_i) {
+void G1HotCardCache::drain(G1CardTableEntryClosure* cl, uint worker_i) {
assert(default_use_cache(), "Drain only necessary if we use the hot card cache.");
assert(_hot_cache != NULL, "Logic");
--- a/src/hotspot/share/gc/g1/g1HotCardCache.hpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1HotCardCache.hpp Wed Feb 13 17:38:14 2019 -0500
@@ -32,8 +32,7 @@
#include "runtime/thread.hpp"
#include "utilities/globalDefinitions.hpp"
-class CardTableEntryClosure;
-class DirtyCardQueue;
+class G1CardTableEntryClosure;
class G1CollectedHeap;
class HeapRegion;
@@ -112,7 +111,7 @@
// Refine the cards that have delayed as a result of
// being in the cache.
- void drain(CardTableEntryClosure* cl, uint worker_i);
+ void drain(G1CardTableEntryClosure* cl, uint worker_i);
// Set up for parallel processing of the cards in the hot cache
void reset_hot_cache_claimed_index() {
--- a/src/hotspot/share/gc/g1/g1ParScanThreadState.hpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1ParScanThreadState.hpp Wed Feb 13 17:38:14 2019 -0500
@@ -25,9 +25,9 @@
#ifndef SHARE_GC_G1_G1PARSCANTHREADSTATE_HPP
#define SHARE_GC_G1_G1PARSCANTHREADSTATE_HPP
-#include "gc/g1/dirtyCardQueue.hpp"
#include "gc/g1/g1CardTable.hpp"
#include "gc/g1/g1CollectedHeap.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1OopClosures.hpp"
#include "gc/g1/g1Policy.hpp"
#include "gc/g1/g1RemSet.hpp"
@@ -46,7 +46,7 @@
class G1ParScanThreadState : public CHeapObj<mtGC> {
G1CollectedHeap* _g1h;
RefToScanQueue* _refs;
- DirtyCardQueue _dcq;
+ G1DirtyCardQueue _dcq;
G1CardTable* _ct;
G1EvacuationRootClosures* _closures;
@@ -77,7 +77,7 @@
#define PADDING_ELEM_NUM (DEFAULT_CACHE_LINE_SIZE / sizeof(size_t))
- DirtyCardQueue& dirty_card_queue() { return _dcq; }
+ G1DirtyCardQueue& dirty_card_queue() { return _dcq; }
G1CardTable* ct() { return _ct; }
InCSetState dest(InCSetState original) const {
--- a/src/hotspot/share/gc/g1/g1RemSet.cpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1RemSet.cpp Wed Feb 13 17:38:14 2019 -0500
@@ -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
@@ -23,12 +23,12 @@
*/
#include "precompiled.hpp"
-#include "gc/g1/dirtyCardQueue.hpp"
#include "gc/g1/g1BarrierSet.hpp"
#include "gc/g1/g1BlockOffsetTable.inline.hpp"
#include "gc/g1/g1CardTable.inline.hpp"
#include "gc/g1/g1CollectedHeap.inline.hpp"
#include "gc/g1/g1ConcurrentRefine.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1FromCardCache.hpp"
#include "gc/g1/g1GCPhaseTimes.hpp"
#include "gc/g1/g1HotCardCache.hpp"
@@ -300,7 +300,7 @@
}
uint G1RemSet::num_par_rem_sets() {
- return DirtyCardQueueSet::num_par_ids() + G1ConcurrentRefine::max_num_threads() + MAX2(ConcGCThreads, ParallelGCThreads);
+ return G1DirtyCardQueueSet::num_par_ids() + G1ConcurrentRefine::max_num_threads() + MAX2(ConcGCThreads, ParallelGCThreads);
}
void G1RemSet::initialize(size_t capacity, uint max_regions) {
@@ -456,7 +456,7 @@
}
// Closure used for updating rem sets. Only called during an evacuation pause.
-class G1RefineCardClosure: public CardTableEntryClosure {
+class G1RefineCardClosure: public G1CardTableEntryClosure {
G1RemSet* _g1rs;
G1ScanObjsDuringUpdateRSClosure* _update_rs_cl;
@@ -520,7 +520,7 @@
}
void G1RemSet::prepare_for_oops_into_collection_set_do() {
- DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
+ G1DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
dcqs.concatenate_logs();
_scan_state->reset();
@@ -677,7 +677,7 @@
*card_ptr = G1CardTable::dirty_card_val();
MutexLockerEx x(Shared_DirtyCardQ_lock,
Mutex::_no_safepoint_check_flag);
- DirtyCardQueue* sdcq =
+ G1DirtyCardQueue* sdcq =
G1BarrierSet::dirty_card_queue_set().shared_dirty_card_queue();
sdcq->enqueue(card_ptr);
}
--- a/src/hotspot/share/gc/g1/g1RemSet.hpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1RemSet.hpp Wed Feb 13 17:38:14 2019 -0500
@@ -25,7 +25,6 @@
#ifndef SHARE_GC_G1_G1REMSET_HPP
#define SHARE_GC_G1_G1REMSET_HPP
-#include "gc/g1/dirtyCardQueue.hpp"
#include "gc/g1/g1CardTable.hpp"
#include "gc/g1/g1OopClosures.hpp"
#include "gc/g1/g1GCPhaseTimes.hpp"
--- a/src/hotspot/share/gc/g1/g1RemSetSummary.cpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1RemSetSummary.cpp Wed Feb 13 17:38:14 2019 -0500
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2013, 2018, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2013, 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
@@ -26,6 +26,7 @@
#include "gc/g1/g1CollectedHeap.inline.hpp"
#include "gc/g1/g1ConcurrentRefine.hpp"
#include "gc/g1/g1ConcurrentRefineThread.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1RemSet.hpp"
#include "gc/g1/g1RemSetSummary.hpp"
#include "gc/g1/g1YoungRemSetSamplingThread.hpp"
@@ -53,7 +54,7 @@
void G1RemSetSummary::update() {
_num_conc_refined_cards = _rem_set->num_conc_refined_cards();
- DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
+ G1DirtyCardQueueSet& dcqs = G1BarrierSet::dirty_card_queue_set();
_num_processed_buf_mutator = dcqs.processed_buffers_mut();
_num_processed_buf_rs_threads = dcqs.processed_buffers_rs_thread();
--- a/src/hotspot/share/gc/g1/g1ThreadLocalData.hpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/g1ThreadLocalData.hpp Wed Feb 13 17:38:14 2019 -0500
@@ -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
@@ -24,8 +24,8 @@
#ifndef SHARE_GC_G1_G1THREADLOCALDATA_HPP
#define SHARE_GC_G1_G1THREADLOCALDATA_HPP
-#include "gc/g1/dirtyCardQueue.hpp"
#include "gc/g1/g1BarrierSet.hpp"
+#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/shared/satbMarkQueue.hpp"
#include "runtime/thread.hpp"
#include "utilities/debug.hpp"
@@ -33,8 +33,8 @@
class G1ThreadLocalData {
private:
- SATBMarkQueue _satb_mark_queue;
- DirtyCardQueue _dirty_card_queue;
+ SATBMarkQueue _satb_mark_queue;
+ G1DirtyCardQueue _dirty_card_queue;
G1ThreadLocalData() :
_satb_mark_queue(&G1BarrierSet::satb_mark_queue_set()),
@@ -66,7 +66,7 @@
return data(thread)->_satb_mark_queue;
}
- static DirtyCardQueue& dirty_card_queue(Thread* thread) {
+ static G1DirtyCardQueue& dirty_card_queue(Thread* thread) {
return data(thread)->_dirty_card_queue;
}
@@ -83,11 +83,11 @@
}
static ByteSize dirty_card_queue_index_offset() {
- return dirty_card_queue_offset() + DirtyCardQueue::byte_offset_of_index();
+ return dirty_card_queue_offset() + G1DirtyCardQueue::byte_offset_of_index();
}
static ByteSize dirty_card_queue_buffer_offset() {
- return dirty_card_queue_offset() + DirtyCardQueue::byte_offset_of_buf();
+ return dirty_card_queue_offset() + G1DirtyCardQueue::byte_offset_of_buf();
}
};
--- a/src/hotspot/share/gc/g1/vmStructs_g1.hpp Wed Feb 13 15:50:08 2019 -0500
+++ b/src/hotspot/share/gc/g1/vmStructs_g1.hpp Wed Feb 13 17:38:14 2019 -0500
@@ -102,7 +102,7 @@
declare_toplevel_type(PtrQueue) \
declare_toplevel_type(HeapRegionType) \
declare_toplevel_type(SATBMarkQueue) \
- declare_toplevel_type(DirtyCardQueue) \
+ declare_toplevel_type(G1DirtyCardQueue) \
\
declare_toplevel_type(G1CollectedHeap*) \
declare_toplevel_type(HeapRegion*) \