8302122: Parallelize TLAB retirement in prologue in G1

8297611: G1: Merge tlab and per-thread dirty card log flushing

Reviewed-by: kbarrett, ayang
This commit is contained in:
Thomas Schatzl 2023-02-20 10:23:00 +00:00
parent e971f90a0b
commit 593bec685e
17 changed files with 328 additions and 109 deletions

View File

@ -101,7 +101,7 @@ void G1BarrierSet::write_ref_field_post_slow(volatile CardValue* byte) {
}
}
void G1BarrierSet::invalidate(MemRegion mr) {
void G1BarrierSet::invalidate(JavaThread* thread, MemRegion mr) {
if (mr.is_empty()) {
return;
}
@ -120,9 +120,8 @@ void G1BarrierSet::invalidate(MemRegion mr) {
OrderAccess::storeload();
// Enqueue if necessary.
Thread* thr = Thread::current();
G1DirtyCardQueueSet& qset = G1BarrierSet::dirty_card_queue_set();
G1DirtyCardQueue& queue = G1ThreadLocalData::dirty_card_queue(thr);
G1DirtyCardQueue& queue = G1ThreadLocalData::dirty_card_queue(thread);
for (; byte <= last_byte; byte++) {
CardValue bv = *byte;
assert(bv != G1CardTable::g1_young_card_val(), "Invalid card");

View File

@ -47,6 +47,8 @@ class G1BarrierSet: public CardTableBarrierSet {
return barrier_set_cast<G1BarrierSet>(BarrierSet::barrier_set());
}
void invalidate(JavaThread* thread, MemRegion mr);
public:
G1BarrierSet(G1CardTable* table);
~G1BarrierSet() { }
@ -70,12 +72,10 @@ class G1BarrierSet: public CardTableBarrierSet {
template <DecoratorSet decorators, typename T>
void write_ref_field_pre(T* field);
// NB: if you do a whole-heap invalidation, the "usual invariant" defined
// above no longer applies.
void invalidate(MemRegion mr);
inline void invalidate(MemRegion mr);
inline void write_region(JavaThread* thread, MemRegion mr);
void write_region(MemRegion mr) { invalidate(mr); }
void write_ref_array_work(MemRegion mr) { invalidate(mr); }
inline void write_ref_array_work(MemRegion mr);
template <DecoratorSet decorators, typename T>
void write_ref_field_post(T* field);

View File

@ -33,6 +33,7 @@
#include "oops/access.inline.hpp"
#include "oops/compressedOops.inline.hpp"
#include "oops/oop.hpp"
#include "runtime/thread.hpp"
inline void G1BarrierSet::enqueue_preloaded(oop pre_val) {
// Nulls should have been already filtered.
@ -67,6 +68,18 @@ inline void G1BarrierSet::write_ref_field_pre(T* field) {
enqueue(field);
}
inline void G1BarrierSet::invalidate(MemRegion mr) {
invalidate(JavaThread::current(), mr);
}
inline void G1BarrierSet::write_region(JavaThread* thread, MemRegion mr) {
invalidate(thread, mr);
}
inline void G1BarrierSet::write_ref_array_work(MemRegion mr) {
invalidate(mr);
}
template <DecoratorSet decorators, typename T>
inline void G1BarrierSet::write_ref_field_post(T* field) {
volatile CardValue* byte = _card_table->byte_for(field);

View File

@ -1,5 +1,5 @@
/*
* Copyright (c) 2001, 2022, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2001, 2023, 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
@ -556,36 +556,10 @@ void G1DirtyCardQueueSet::abandon_logs_and_stats() {
_detached_refinement_stats.reset();
}
void G1DirtyCardQueueSet::concatenate_logs_and_stats() {
void G1DirtyCardQueueSet::update_refinement_stats(G1ConcurrentRefineStats& stats) {
assert_at_safepoint();
// Disable mutator refinement until concurrent refinement decides otherwise.
set_mutator_refinement_threshold(SIZE_MAX);
// Iterate over all the threads, if we find a partial log add it to
// the global list of logs.
struct ConcatenateThreadLogClosure : public ThreadClosure {
G1DirtyCardQueueSet& _qset;
G1ConcurrentRefineStats _total_stats;
ConcatenateThreadLogClosure(G1DirtyCardQueueSet& qset) :
_qset{qset}, _total_stats{} {}
virtual void do_thread(Thread* t) {
G1DirtyCardQueue& queue = G1ThreadLocalData::dirty_card_queue(t);
// Flush the buffer if non-empty. Flush before accumulating and
// resetting stats, since flushing may modify the stats.
if ((queue.buffer() != nullptr) &&
(queue.index() != _qset.buffer_size())) {
_qset.flush_queue(queue);
}
G1ConcurrentRefineStats& qstats = *queue.refinement_stats();
_total_stats += qstats;
qstats.reset();
}
} closure(*this);
Threads::threads_do(&closure);
_concatenated_refinement_stats = closure._total_stats;
_concatenated_refinement_stats = stats;
enqueue_all_paused_buffers();
verify_num_cards();
@ -596,6 +570,22 @@ void G1DirtyCardQueueSet::concatenate_logs_and_stats() {
_detached_refinement_stats.reset();
}
G1ConcurrentRefineStats G1DirtyCardQueueSet::concatenate_log_and_stats(Thread* thread) {
assert_at_safepoint();
G1DirtyCardQueue& queue = G1ThreadLocalData::dirty_card_queue(thread);
// Flush the buffer if non-empty. Flush before accumulating and
// resetting stats, since flushing may modify the stats.
if ((queue.buffer() != nullptr) &&
(queue.index() != buffer_size())) {
flush_queue(queue);
}
G1ConcurrentRefineStats result = *queue.refinement_stats();
queue.refinement_stats()->reset();
return result;
}
G1ConcurrentRefineStats G1DirtyCardQueueSet::concatenated_refinement_stats() const {
assert_at_safepoint();
return _concatenated_refinement_stats;

View File

@ -1,5 +1,5 @@
/*
* Copyright (c) 2001, 2022, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2001, 2023, 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
@ -274,10 +274,14 @@ public:
// precondition: at safepoint.
void abandon_logs_and_stats();
// Collect and reset all the per-thread refinement stats. If any threads
// have partial logs then add them to the global list.
// Update global refinement statistics with the ones given and the ones from
// detached threads.
// precondition: at safepoint.
void concatenate_logs_and_stats();
void update_refinement_stats(G1ConcurrentRefineStats& stats);
// Add the given thread's partial logs to the global list and return and reset
// its refinement stats.
// precondition: at safepoint.
G1ConcurrentRefineStats concatenate_log_and_stats(Thread* thread);
// Return the total of mutator refinement stats for all threads.
// precondition: at safepoint.

View File

@ -51,6 +51,9 @@ G1GCPhaseTimes::G1GCPhaseTimes(STWGCTimer* gc_timer, uint max_gc_threads) :
{
assert(max_gc_threads > 0, "Must have some GC threads");
_gc_par_phases[RetireTLABsAndFlushLogs] = new WorkerDataArray<double>("RetireTLABsAndFlushLogs", "JT Retire TLABs And Flush Logs (ms):", max_gc_threads);
_gc_par_phases[NonJavaThreadFlushLogs] = new WorkerDataArray<double>("NonJavaThreadFlushLogs", "Non-JT Flush Logs (ms):", max_gc_threads);
_gc_par_phases[GCWorkerStart] = new WorkerDataArray<double>("GCWorkerStart", "GC Worker Start (ms):", max_gc_threads);
_gc_par_phases[ExtRootScan] = new WorkerDataArray<double>("ExtRootScan", "Ext Root Scanning (ms):", max_gc_threads);
@ -165,7 +168,7 @@ void G1GCPhaseTimes::reset() {
_cur_optional_merge_heap_roots_time_ms = 0.0;
_cur_prepare_merge_heap_roots_time_ms = 0.0;
_cur_optional_prepare_merge_heap_roots_time_ms = 0.0;
_cur_prepare_tlab_time_ms = 0.0;
_cur_pre_evacuate_prepare_time_ms = 0.0;
_cur_post_evacuate_cleanup_1_time_ms = 0.0;
_cur_post_evacuate_cleanup_2_time_ms = 0.0;
_cur_expand_heap_time_ms = 0.0;
@ -402,8 +405,7 @@ double G1GCPhaseTimes::print_pre_evacuate_collection_set() const {
const double pre_concurrent_start_ms = average_time_ms(ResetMarkingState) +
average_time_ms(NoteStartOfMark);
const double sum_ms = _cur_prepare_tlab_time_ms +
_cur_concatenate_dirty_card_logs_time_ms +
const double sum_ms = _cur_pre_evacuate_prepare_time_ms +
_recorded_young_cset_choice_time_ms +
_recorded_non_young_cset_choice_time_ms +
_cur_region_register_time +
@ -412,8 +414,9 @@ double G1GCPhaseTimes::print_pre_evacuate_collection_set() const {
info_time("Pre Evacuate Collection Set", sum_ms);
debug_time("Prepare TLABs", _cur_prepare_tlab_time_ms);
debug_time("Concatenate Dirty Card Logs", _cur_concatenate_dirty_card_logs_time_ms);
debug_time("Pre Evacuate Prepare", _cur_pre_evacuate_prepare_time_ms);
debug_phase(_gc_par_phases[RetireTLABsAndFlushLogs], 1);
debug_phase(_gc_par_phases[NonJavaThreadFlushLogs], 1);
debug_time("Choose Collection Set", (_recorded_young_cset_choice_time_ms + _recorded_non_young_cset_choice_time_ms));
debug_time("Region Register", _cur_region_register_time);

View File

@ -46,6 +46,8 @@ class G1GCPhaseTimes : public CHeapObj<mtGC> {
public:
enum GCParPhases {
RetireTLABsAndFlushLogs,
NonJavaThreadFlushLogs,
GCWorkerStart,
ExtRootScan,
ThreadRoots,
@ -172,9 +174,7 @@ class G1GCPhaseTimes : public CHeapObj<mtGC> {
double _cur_prepare_merge_heap_roots_time_ms;
double _cur_optional_prepare_merge_heap_roots_time_ms;
double _cur_prepare_tlab_time_ms;
double _cur_concatenate_dirty_card_logs_time_ms;
double _cur_pre_evacuate_prepare_time_ms;
double _cur_post_evacuate_cleanup_1_time_ms;
double _cur_post_evacuate_cleanup_2_time_ms;
@ -265,12 +265,8 @@ class G1GCPhaseTimes : public CHeapObj<mtGC> {
size_t sum_thread_work_items(GCParPhases phase, uint index = 0);
void record_prepare_tlab_time_ms(double ms) {
_cur_prepare_tlab_time_ms = ms;
}
void record_concatenate_dirty_card_logs_time_ms(double ms) {
_cur_concatenate_dirty_card_logs_time_ms = ms;
void record_pre_evacuate_prepare_time_ms(double ms) {
_cur_pre_evacuate_prepare_time_ms = ms;
}
void record_expand_heap_time(double ms) {

View File

@ -1,5 +1,5 @@
/*
* Copyright (c) 2021, 2022, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2021, 2023, 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
@ -45,6 +45,7 @@
#include "gc/g1/g1Trace.hpp"
#include "gc/g1/g1YoungCollector.hpp"
#include "gc/g1/g1YoungGCPostEvacuateTasks.hpp"
#include "gc/g1/g1YoungGCPreEvacuateTasks.hpp"
#include "gc/g1/g1_globals.hpp"
#include "gc/shared/concurrentGCBreakpoints.hpp"
#include "gc/shared/gcTraceTime.inline.hpp"
@ -462,48 +463,15 @@ void G1YoungCollector::set_young_collection_default_active_worker_threads(){
log_info(gc,task)("Using %u workers of %u for evacuation", active_workers, workers()->max_workers());
}
void G1YoungCollector::retire_tlabs() {
Ticks start = Ticks::now();
_g1h->retire_tlabs();
double retire_time = (Ticks::now() - start).seconds() * MILLIUNITS;
phase_times()->record_prepare_tlab_time_ms(retire_time);
}
void G1YoungCollector::concatenate_dirty_card_logs_and_stats() {
Ticks start = Ticks::now();
G1DirtyCardQueueSet& qset = G1BarrierSet::dirty_card_queue_set();
size_t old_cards = qset.num_cards();
qset.concatenate_logs_and_stats();
size_t pending_cards = qset.num_cards();
size_t thread_buffer_cards = pending_cards - old_cards;
policy()->record_concurrent_refinement_stats(pending_cards, thread_buffer_cards);
double concat_time = (Ticks::now() - start).seconds() * MILLIUNITS;
phase_times()->record_concatenate_dirty_card_logs_time_ms(concat_time);
}
#ifdef ASSERT
void G1YoungCollector::verify_empty_dirty_card_logs() const {
struct Verifier : public ThreadClosure {
size_t _buffer_size;
Verifier() : _buffer_size(G1BarrierSet::dirty_card_queue_set().buffer_size()) {}
void do_thread(Thread* t) override {
G1DirtyCardQueue& queue = G1ThreadLocalData::dirty_card_queue(t);
assert((queue.buffer() == nullptr) || (queue.index() == _buffer_size),
"non-empty dirty card queue for thread");
}
} verifier;
Threads::threads_do(&verifier);
}
#endif // ASSERT
void G1YoungCollector::pre_evacuate_collection_set(G1EvacInfo* evacuation_info) {
// Flush early, so later phases don't need to account for per-thread stuff.
// Flushes deferred card marks, so must precede concatenating logs.
retire_tlabs();
// Flush early, so later phases don't need to account for per-thread stuff.
concatenate_dirty_card_logs_and_stats();
{
Ticks start = Ticks::now();
G1PreEvacuateCollectionSetBatchTask cl;
G1CollectedHeap::heap()->run_batch_task(&cl);
phase_times()->record_pre_evacuate_prepare_time_ms((Ticks::now() - start).seconds() * 1000.0);
}
// Needs log buffers flushed.
calculate_collection_set(evacuation_info, policy()->max_pause_time_ms());
// Please see comment in g1CollectedHeap.hpp and
@ -535,7 +503,6 @@ void G1YoungCollector::pre_evacuate_collection_set(G1EvacInfo* evacuation_info)
}
assert(_g1h->verifier()->check_region_attr_table(), "Inconsistency in the region attributes table.");
verify_empty_dirty_card_logs();
#if COMPILER2_OR_JVMCI
DerivedPointerTable::clear();

View File

@ -1,5 +1,5 @@
/*
* Copyright (c) 2021, 2022, Oracle and/or its affiliates. All rights reserved.
* Copyright (c) 2021, 2023, 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
@ -97,10 +97,6 @@ class G1YoungCollector {
void set_young_collection_default_active_worker_threads();
void retire_tlabs();
void concatenate_dirty_card_logs_and_stats();
void verify_empty_dirty_card_logs() const NOT_DEBUG_RETURN;
void pre_evacuate_collection_set(G1EvacInfo* evacuation_info);
// Actually do the work of evacuating the parts of the collection set.
// The has_optional_evacuation_work flag for the initial collection set

View File

@ -0,0 +1,198 @@
/*
* Copyright (c) 2023, 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/g1ConcurrentRefineStats.hpp"
#include "gc/g1/g1DirtyCardQueue.hpp"
#include "gc/g1/g1YoungGCPreEvacuateTasks.hpp"
#include "gc/shared/barrierSet.inline.hpp"
#include "gc/shared/threadLocalAllocBuffer.inline.hpp"
#include "memory/allocation.inline.hpp"
#include "memory/iterator.hpp"
#include "runtime/thread.inline.hpp"
#include "runtime/threads.hpp"
class G1PreEvacuateCollectionSetBatchTask::JavaThreadRetireTLABAndFlushLogs : public G1AbstractSubTask {
G1JavaThreadsListClaimer _claimer;
// Per worker thread statistics.
ThreadLocalAllocStats* _local_tlab_stats;
G1ConcurrentRefineStats* _local_refinement_stats;
uint _num_workers;
// There is relatively little work to do per thread.
static const uint ThreadsPerWorker = 250;
struct RetireTLABAndFlushLogsClosure : public ThreadClosure {
ThreadLocalAllocStats _tlab_stats;
G1ConcurrentRefineStats _refinement_stats;
RetireTLABAndFlushLogsClosure() : _tlab_stats(), _refinement_stats() { }
void do_thread(Thread* thread) override {
assert(thread->is_Java_thread(), "must be");
// Flushes deferred card marks, so must precede concatenating logs.
BarrierSet::barrier_set()->make_parsable((JavaThread*)thread);
if (UseTLAB) {
thread->tlab().retire(&_tlab_stats);
}
G1DirtyCardQueueSet& qset = G1BarrierSet::dirty_card_queue_set();
_refinement_stats += qset.concatenate_log_and_stats(thread);
}
};
public:
JavaThreadRetireTLABAndFlushLogs() :
G1AbstractSubTask(G1GCPhaseTimes::RetireTLABsAndFlushLogs),
_claimer(ThreadsPerWorker),
_local_tlab_stats(nullptr),
_local_refinement_stats(nullptr),
_num_workers(0) {
}
~JavaThreadRetireTLABAndFlushLogs() {
static_assert(std::is_trivially_destructible<G1ConcurrentRefineStats>::value, "must be");
FREE_C_HEAP_ARRAY(G1ConcurrentRefineStats, _local_refinement_stats);
static_assert(std::is_trivially_destructible<ThreadLocalAllocStats>::value, "must be");
FREE_C_HEAP_ARRAY(ThreadLocalAllocStats, _local_tlab_stats);
}
void do_work(uint worker_id) override {
RetireTLABAndFlushLogsClosure tc;
_claimer.apply(&tc);
_local_tlab_stats[worker_id] = tc._tlab_stats;
_local_refinement_stats[worker_id] = tc._refinement_stats;
}
double worker_cost() const override {
return (double)_claimer.length() / ThreadsPerWorker;
}
void set_max_workers(uint max_workers) override {
_num_workers = max_workers;
_local_tlab_stats = NEW_C_HEAP_ARRAY(ThreadLocalAllocStats, _num_workers, mtGC);
_local_refinement_stats = NEW_C_HEAP_ARRAY(G1ConcurrentRefineStats, _num_workers, mtGC);
for (uint i = 0; i < _num_workers; i++) {
::new (&_local_tlab_stats[i]) ThreadLocalAllocStats();
::new (&_local_refinement_stats[i]) G1ConcurrentRefineStats();
}
}
ThreadLocalAllocStats tlab_stats() const {
ThreadLocalAllocStats result;
for (uint i = 0; i < _num_workers; i++) {
result.update(_local_tlab_stats[i]);
}
return result;
}
G1ConcurrentRefineStats refinement_stats() const {
G1ConcurrentRefineStats result;
for (uint i = 0; i < _num_workers; i++) {
result += _local_refinement_stats[i];
}
return result;
}
};
class G1PreEvacuateCollectionSetBatchTask::NonJavaThreadFlushLogs : public G1AbstractSubTask {
struct FlushLogsClosure : public ThreadClosure {
G1ConcurrentRefineStats _refinement_stats;
FlushLogsClosure() : _refinement_stats() { }
void do_thread(Thread* thread) override {
G1DirtyCardQueueSet& qset = G1BarrierSet::dirty_card_queue_set();
_refinement_stats += qset.concatenate_log_and_stats(thread);
}
} _tc;
public:
NonJavaThreadFlushLogs() : G1AbstractSubTask(G1GCPhaseTimes::NonJavaThreadFlushLogs), _tc() { }
void do_work(uint worker_id) override {
Threads::non_java_threads_do(&_tc);
}
double worker_cost() const override {
return 1.0;
}
G1ConcurrentRefineStats refinement_stats() const { return _tc._refinement_stats; }
};
G1PreEvacuateCollectionSetBatchTask::G1PreEvacuateCollectionSetBatchTask() :
G1BatchedTask("Pre Evacuate Prepare", G1CollectedHeap::heap()->phase_times()),
_old_pending_cards(G1BarrierSet::dirty_card_queue_set().num_cards()),
_java_retire_task(new JavaThreadRetireTLABAndFlushLogs()),
_non_java_retire_task(new NonJavaThreadFlushLogs()) {
// Disable mutator refinement until concurrent refinement decides otherwise.
G1BarrierSet::dirty_card_queue_set().set_mutator_refinement_threshold(SIZE_MAX);
add_serial_task(_non_java_retire_task);
add_parallel_task(_java_retire_task);
}
static void verify_empty_dirty_card_logs() {
#ifdef ASSERT
ResourceMark rm;
struct Verifier : public ThreadClosure {
size_t _buffer_size;
Verifier() : _buffer_size(G1BarrierSet::dirty_card_queue_set().buffer_size()) {}
void do_thread(Thread* t) override {
G1DirtyCardQueue& queue = G1ThreadLocalData::dirty_card_queue(t);
assert((queue.buffer() == nullptr) || (queue.index() == _buffer_size),
"non-empty dirty card queue for thread %s", t->name());
}
} verifier;
Threads::threads_do(&verifier);
#endif
}
G1PreEvacuateCollectionSetBatchTask::~G1PreEvacuateCollectionSetBatchTask() {
_java_retire_task->tlab_stats().publish();
G1DirtyCardQueueSet& qset = G1BarrierSet::dirty_card_queue_set();
G1ConcurrentRefineStats total_refinement_stats;
total_refinement_stats += _java_retire_task->refinement_stats();
total_refinement_stats += _non_java_retire_task->refinement_stats();
qset.update_refinement_stats(total_refinement_stats);
verify_empty_dirty_card_logs();
size_t pending_cards = qset.num_cards();
size_t thread_buffer_cards = pending_cards - _old_pending_cards;
G1CollectedHeap::heap()->policy()->record_concurrent_refinement_stats(pending_cards, thread_buffer_cards);
}

View File

@ -0,0 +1,49 @@
/*
* Copyright (c) 2023, 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_G1YOUNGGCPREEVACUATETASKS_HPP
#define SHARE_GC_G1_G1YOUNGGCPREEVACUATETASKS_HPP
#include "gc/g1/g1BatchedTask.hpp"
// Set of pre evacuate collection set tasks containing ("s" means serial):
// - Retire TLAB and Flush Logs (Java threads)
// - Flush Logs (s) (Non-Java threads)
class G1PreEvacuateCollectionSetBatchTask : public G1BatchedTask {
class JavaThreadRetireTLABAndFlushLogs;
class NonJavaThreadFlushLogs;
size_t _old_pending_cards;
// References to the tasks to retain access to statistics.
JavaThreadRetireTLABAndFlushLogs* _java_retire_task;
NonJavaThreadFlushLogs* _non_java_retire_task;
public:
G1PreEvacuateCollectionSetBatchTask();
~G1PreEvacuateCollectionSetBatchTask();
};
#endif // SHARE_GC_G1_G1YOUNGGCPREEVACUATETASKS_HPP

View File

@ -172,7 +172,7 @@ void CardTableBarrierSet::flush_deferred_card_mark_barrier(JavaThread* thread) {
assert(deferred.word_size() == old_obj->size(),
"Mismatch: multiple objects?");
}
write_region(deferred);
write_region(thread, deferred);
// "Clear" the deferred_card_mark field
thread->set_deferred_card_mark(MemRegion());
}

View File

@ -68,7 +68,7 @@ protected:
virtual void initialize();
void write_region(MemRegion mr) {
void write_region(JavaThread* thread, MemRegion mr) {
invalidate(mr);
}

View File

@ -51,9 +51,10 @@ public:
template <DecoratorSet decorators, typename T>
inline void write_ref_field_post(T *addr) {}
// Causes all refs in "mr" to be assumed to be modified.
// Causes all refs in "mr" to be assumed to be modified (by this JavaThread).
virtual void invalidate(MemRegion mr) = 0;
virtual void write_region(MemRegion mr) = 0;
// Causes all refs in "mr" to be assumed to be modified by the given JavaThread.
virtual void write_region(JavaThread* thread, MemRegion mr) = 0;
// Operations on arrays, or general regions (e.g., for "clone") may be
// optimized by some barriers.

View File

@ -31,6 +31,7 @@
#include "oops/compressedOops.inline.hpp"
#include "oops/objArrayOop.hpp"
#include "oops/oop.hpp"
#include "runtime/thread.hpp"
class Klass;
@ -137,7 +138,7 @@ inline void ModRefBarrierSet::AccessBarrier<decorators, BarrierSetT>::
clone_in_heap(oop src, oop dst, size_t size) {
Raw::clone(src, dst, size);
BarrierSetT *bs = barrier_set_cast<BarrierSetT>(barrier_set());
bs->write_region(MemRegion((HeapWord*)(void*)dst, size));
bs->invalidate(MemRegion((HeapWord*)(void*)dst, size));
}
#endif // SHARE_GC_SHARED_MODREFBARRIERSET_INLINE_HPP

View File

@ -108,8 +108,8 @@ public class TestGCLogMessages {
new LogMessageWithLevel("Other", Level.INFO),
// Pre Evacuate Collection Set
new LogMessageWithLevel("Prepare TLABs", Level.DEBUG),
new LogMessageWithLevel("Concatenate Dirty Card Logs", Level.DEBUG),
new LogMessageWithLevel("JT Retire TLABs And Flush Logs", Level.DEBUG),
new LogMessageWithLevel("Non-JT Flush Logs", Level.DEBUG),
new LogMessageWithLevel("Choose Collection Set", Level.DEBUG),
new LogMessageWithLevel("Region Register", Level.DEBUG),
new LogMessageWithLevel("Prepare Heap Roots", Level.DEBUG),

View File

@ -87,6 +87,8 @@ public class TestG1ParallelPhases {
.collect(toSet());
Set<String> allPhases = of(
"RetireTLABsAndFlushLogs",
"NonJavaThreadFlushLogs",
"ExtRootScan",
"ThreadRoots",
"VM Global",