mirror of
https://github.com/openjdk/jdk.git
synced 2026-01-28 12:09:14 +00:00
8344665: Refactor PartialArrayState allocation for reuse
Reviewed-by: tschatzl, ayang, iwalulya, zgu
This commit is contained in:
parent
5cc150c636
commit
dbf48a53ec
@ -86,6 +86,7 @@
|
||||
#include "gc/shared/isGCActiveMark.hpp"
|
||||
#include "gc/shared/locationPrinter.inline.hpp"
|
||||
#include "gc/shared/oopStorageParState.hpp"
|
||||
#include "gc/shared/partialArrayState.hpp"
|
||||
#include "gc/shared/referenceProcessor.inline.hpp"
|
||||
#include "gc/shared/suspendibleThreadSet.hpp"
|
||||
#include "gc/shared/taskqueue.inline.hpp"
|
||||
@ -1165,6 +1166,7 @@ G1CollectedHeap::G1CollectedHeap() :
|
||||
_cm_thread(nullptr),
|
||||
_cr(nullptr),
|
||||
_task_queues(nullptr),
|
||||
_partial_array_state_manager(nullptr),
|
||||
_ref_processor_stw(nullptr),
|
||||
_is_alive_closure_stw(this),
|
||||
_is_subject_to_discovery_stw(this),
|
||||
@ -1198,9 +1200,13 @@ G1CollectedHeap::G1CollectedHeap() :
|
||||
_task_queues->register_queue(i, q);
|
||||
}
|
||||
|
||||
_gc_tracer_stw->initialize();
|
||||
_partial_array_state_manager = new PartialArrayStateManager(n_queues);
|
||||
|
||||
guarantee(_task_queues != nullptr, "task_queues allocation failure.");
|
||||
_gc_tracer_stw->initialize();
|
||||
}
|
||||
|
||||
PartialArrayStateManager* G1CollectedHeap::partial_array_state_manager() const {
|
||||
return _partial_array_state_manager;
|
||||
}
|
||||
|
||||
G1RegionToSpaceMapper* G1CollectedHeap::create_aux_memory_mapper(const char* description,
|
||||
|
||||
@ -82,6 +82,7 @@ class GCMemoryManager;
|
||||
class G1HeapRegion;
|
||||
class MemoryPool;
|
||||
class nmethod;
|
||||
class PartialArrayStateManager;
|
||||
class ReferenceProcessor;
|
||||
class STWGCTimer;
|
||||
class WorkerThreads;
|
||||
@ -807,8 +808,9 @@ public:
|
||||
// The concurrent refiner.
|
||||
G1ConcurrentRefine* _cr;
|
||||
|
||||
// The parallel task queues
|
||||
G1ScannerTasksQueueSet *_task_queues;
|
||||
// Reusable parallel task queues and partial array manager.
|
||||
G1ScannerTasksQueueSet* _task_queues;
|
||||
PartialArrayStateManager* _partial_array_state_manager;
|
||||
|
||||
// ("Weak") Reference processing support.
|
||||
//
|
||||
@ -874,6 +876,8 @@ public:
|
||||
G1ScannerTasksQueueSet* task_queues() const;
|
||||
G1ScannerTasksQueue* task_queue(uint i) const;
|
||||
|
||||
PartialArrayStateManager* partial_array_state_manager() const;
|
||||
|
||||
// Create a G1CollectedHeap.
|
||||
// Must call the initialize method afterwards.
|
||||
// May not return if something goes wrong.
|
||||
|
||||
@ -105,6 +105,7 @@ G1GCPhaseTimes::G1GCPhaseTimes(STWGCTimer* gc_timer, uint max_gc_threads) :
|
||||
_gc_par_phases[UpdateDerivedPointers] = new WorkerDataArray<double>("UpdateDerivedPointers", "Update Derived Pointers (ms):", max_gc_threads);
|
||||
#endif
|
||||
_gc_par_phases[EagerlyReclaimHumongousObjects] = new WorkerDataArray<double>("EagerlyReclaimHumongousObjects", "Eagerly Reclaim Humongous Objects (ms):", max_gc_threads);
|
||||
_gc_par_phases[ResetPartialArrayStateManager] = new WorkerDataArray<double>("ResetPartialArrayStateManager", "Reset Partial Array State Manager (ms):", max_gc_threads);
|
||||
_gc_par_phases[ProcessEvacuationFailedRegions] = new WorkerDataArray<double>("ProcessEvacuationFailedRegions", "Process Evacuation Failed Regions (ms):", max_gc_threads);
|
||||
|
||||
_gc_par_phases[ScanHR]->create_thread_work_items("Scanned Cards:", ScanHRScannedCards);
|
||||
@ -517,6 +518,7 @@ double G1GCPhaseTimes::print_post_evacuate_collection_set(bool evacuation_failed
|
||||
debug_phase(_gc_par_phases[UpdateDerivedPointers], 1);
|
||||
#endif
|
||||
debug_phase(_gc_par_phases[EagerlyReclaimHumongousObjects], 1);
|
||||
trace_phase(_gc_par_phases[ResetPartialArrayStateManager]);
|
||||
|
||||
if (G1CollectedHeap::heap()->should_sample_collection_set_candidates()) {
|
||||
debug_phase(_gc_par_phases[SampleCollectionSetCandidates], 1);
|
||||
|
||||
@ -87,6 +87,7 @@ class G1GCPhaseTimes : public CHeapObj<mtGC> {
|
||||
UpdateDerivedPointers,
|
||||
#endif
|
||||
EagerlyReclaimHumongousObjects,
|
||||
ResetPartialArrayStateManager,
|
||||
ProcessEvacuationFailedRegions,
|
||||
ResetMarkingState,
|
||||
NoteStartOfMark,
|
||||
|
||||
@ -61,8 +61,7 @@ G1ParScanThreadState::G1ParScanThreadState(G1CollectedHeap* g1h,
|
||||
uint worker_id,
|
||||
uint num_workers,
|
||||
G1CollectionSet* collection_set,
|
||||
G1EvacFailureRegions* evac_failure_regions,
|
||||
PartialArrayStateAllocator* pas_allocator)
|
||||
G1EvacFailureRegions* evac_failure_regions)
|
||||
: _g1h(g1h),
|
||||
_task_queue(g1h->task_queue(worker_id)),
|
||||
_rdc_local_qset(rdcqs),
|
||||
@ -81,7 +80,7 @@ G1ParScanThreadState::G1ParScanThreadState(G1CollectedHeap* g1h,
|
||||
_surviving_young_words(nullptr),
|
||||
_surviving_words_length(collection_set->young_region_length() + 1),
|
||||
_old_gen_is_full(false),
|
||||
_partial_array_state_allocator(pas_allocator),
|
||||
_partial_array_state_allocator(g1h->partial_array_state_manager()),
|
||||
_partial_array_stepper(num_workers, ParGCArrayScanChunk),
|
||||
_string_dedup_requests(),
|
||||
_max_num_optional_regions(collection_set->optional_region_length()),
|
||||
@ -254,7 +253,7 @@ void G1ParScanThreadState::do_partial_array(PartialArrayState* state) {
|
||||
checked_cast<int>(step._index),
|
||||
checked_cast<int>(step._index + _partial_array_stepper.chunk_size()));
|
||||
// Release reference to the state, now that we're done with it.
|
||||
_partial_array_state_allocator->release(_worker_id, state);
|
||||
_partial_array_state_allocator.release(state);
|
||||
}
|
||||
|
||||
MAYBE_INLINE_EVACUATION
|
||||
@ -277,11 +276,10 @@ void G1ParScanThreadState::start_partial_objarray(G1HeapRegionAttr dest_attr,
|
||||
assert(((array_length - step._index) % _partial_array_stepper.chunk_size()) == 0,
|
||||
"invariant");
|
||||
PartialArrayState* state =
|
||||
_partial_array_state_allocator->allocate(_worker_id,
|
||||
from_obj, to_obj,
|
||||
step._index,
|
||||
array_length,
|
||||
step._ncreate);
|
||||
_partial_array_state_allocator.allocate(from_obj, to_obj,
|
||||
step._index,
|
||||
array_length,
|
||||
step._ncreate);
|
||||
for (uint i = 0; i < step._ncreate; ++i) {
|
||||
push_on_queue(ScannerTask(state));
|
||||
}
|
||||
@ -601,8 +599,7 @@ G1ParScanThreadState* G1ParScanThreadStateSet::state_for_worker(uint worker_id)
|
||||
worker_id,
|
||||
_num_workers,
|
||||
_collection_set,
|
||||
_evac_failure_regions,
|
||||
&_partial_array_state_allocator);
|
||||
_evac_failure_regions);
|
||||
}
|
||||
return _states[worker_id];
|
||||
}
|
||||
@ -732,8 +729,7 @@ G1ParScanThreadStateSet::G1ParScanThreadStateSet(G1CollectedHeap* g1h,
|
||||
_surviving_young_words_total(NEW_C_HEAP_ARRAY(size_t, collection_set->young_region_length() + 1, mtGC)),
|
||||
_num_workers(num_workers),
|
||||
_flushed(false),
|
||||
_evac_failure_regions(evac_failure_regions),
|
||||
_partial_array_state_allocator(num_workers)
|
||||
_evac_failure_regions(evac_failure_regions)
|
||||
{
|
||||
for (uint i = 0; i < num_workers; ++i) {
|
||||
_states[i] = nullptr;
|
||||
|
||||
@ -84,7 +84,7 @@ class G1ParScanThreadState : public CHeapObj<mtGC> {
|
||||
// Indicates whether in the last generation (old) there is no more space
|
||||
// available for allocation.
|
||||
bool _old_gen_is_full;
|
||||
PartialArrayStateAllocator* _partial_array_state_allocator;
|
||||
PartialArrayStateAllocator _partial_array_state_allocator;
|
||||
PartialArrayTaskStepper _partial_array_stepper;
|
||||
StringDedup::Requests _string_dedup_requests;
|
||||
|
||||
@ -124,8 +124,7 @@ public:
|
||||
uint worker_id,
|
||||
uint num_workers,
|
||||
G1CollectionSet* collection_set,
|
||||
G1EvacFailureRegions* evac_failure_regions,
|
||||
PartialArrayStateAllocator* partial_array_state_allocator);
|
||||
G1EvacFailureRegions* evac_failure_regions);
|
||||
virtual ~G1ParScanThreadState();
|
||||
|
||||
void set_ref_discoverer(ReferenceDiscoverer* rd) { _scanner.set_ref_discoverer(rd); }
|
||||
@ -247,7 +246,6 @@ class G1ParScanThreadStateSet : public StackObj {
|
||||
uint _num_workers;
|
||||
bool _flushed;
|
||||
G1EvacFailureRegions* _evac_failure_regions;
|
||||
PartialArrayStateAllocator _partial_array_state_allocator;
|
||||
|
||||
public:
|
||||
G1ParScanThreadStateSet(G1CollectedHeap* g1h,
|
||||
|
||||
@ -42,6 +42,7 @@
|
||||
#include "gc/g1/g1RemSet.hpp"
|
||||
#include "gc/g1/g1YoungGCPostEvacuateTasks.hpp"
|
||||
#include "gc/shared/bufferNode.hpp"
|
||||
#include "gc/shared/partialArrayState.hpp"
|
||||
#include "jfr/jfrEvents.hpp"
|
||||
#include "oops/access.inline.hpp"
|
||||
#include "oops/compressedOops.inline.hpp"
|
||||
@ -944,6 +945,25 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
class G1PostEvacuateCollectionSetCleanupTask2::ResetPartialArrayStateManagerTask
|
||||
: public G1AbstractSubTask
|
||||
{
|
||||
public:
|
||||
ResetPartialArrayStateManagerTask()
|
||||
: G1AbstractSubTask(G1GCPhaseTimes::ResetPartialArrayStateManager)
|
||||
{}
|
||||
|
||||
double worker_cost() const override {
|
||||
return AlmostNoWork;
|
||||
}
|
||||
|
||||
void do_work(uint worker_id) override {
|
||||
// This must be in phase2 cleanup, after phase1 has destroyed all of the
|
||||
// associated allocators.
|
||||
G1CollectedHeap::heap()->partial_array_state_manager()->reset();
|
||||
}
|
||||
};
|
||||
|
||||
G1PostEvacuateCollectionSetCleanupTask2::G1PostEvacuateCollectionSetCleanupTask2(G1ParScanThreadStateSet* per_thread_states,
|
||||
G1EvacInfo* evacuation_info,
|
||||
G1EvacFailureRegions* evac_failure_regions) :
|
||||
@ -955,6 +975,7 @@ G1PostEvacuateCollectionSetCleanupTask2::G1PostEvacuateCollectionSetCleanupTask2
|
||||
if (G1CollectedHeap::heap()->has_humongous_reclaim_candidates()) {
|
||||
add_serial_task(new EagerlyReclaimHumongousObjectsTask());
|
||||
}
|
||||
add_serial_task(new ResetPartialArrayStateManagerTask());
|
||||
|
||||
if (evac_failure_regions->has_regions_evac_failed()) {
|
||||
add_parallel_task(new ProcessEvacuationFailedRegionsTask(evac_failure_regions));
|
||||
|
||||
@ -58,6 +58,7 @@ public:
|
||||
// - Redirty Logged Cards
|
||||
// - Free Collection Set
|
||||
// - Resize TLABs
|
||||
// - Reset the reusable PartialArrayStateManager.
|
||||
class G1PostEvacuateCollectionSetCleanupTask2 : public G1BatchedTask {
|
||||
class EagerlyReclaimHumongousObjectsTask;
|
||||
#if COMPILER2_OR_JVMCI
|
||||
@ -68,6 +69,7 @@ class G1PostEvacuateCollectionSetCleanupTask2 : public G1BatchedTask {
|
||||
class RedirtyLoggedCardsTask;
|
||||
class FreeCollectionSetTask;
|
||||
class ResizeTLABsTask;
|
||||
class ResetPartialArrayStateManagerTask;
|
||||
|
||||
public:
|
||||
G1PostEvacuateCollectionSetCleanupTask2(G1ParScanThreadStateSet* per_thread_states,
|
||||
|
||||
@ -51,7 +51,7 @@ PSPromotionManager::PSScannerTasksQueueSet* PSPromotionManager::_stack_array_dep
|
||||
PreservedMarksSet* PSPromotionManager::_preserved_marks_set = nullptr;
|
||||
PSOldGen* PSPromotionManager::_old_gen = nullptr;
|
||||
MutableSpace* PSPromotionManager::_young_space = nullptr;
|
||||
PartialArrayStateAllocator* PSPromotionManager::_partial_array_state_allocator = nullptr;
|
||||
PartialArrayStateManager* PSPromotionManager::_partial_array_state_manager = nullptr;
|
||||
|
||||
void PSPromotionManager::initialize() {
|
||||
ParallelScavengeHeap* heap = ParallelScavengeHeap::heap();
|
||||
@ -61,21 +61,20 @@ void PSPromotionManager::initialize() {
|
||||
|
||||
const uint promotion_manager_num = ParallelGCThreads;
|
||||
|
||||
assert(_partial_array_state_manager == nullptr, "Attempt to initialize twice");
|
||||
_partial_array_state_manager
|
||||
= new PartialArrayStateManager(promotion_manager_num);
|
||||
|
||||
// To prevent false sharing, we pad the PSPromotionManagers
|
||||
// and make sure that the first instance starts at a cache line.
|
||||
assert(_manager_array == nullptr, "Attempt to initialize twice");
|
||||
_manager_array = PaddedArray<PSPromotionManager, mtGC>::create_unfreeable(promotion_manager_num);
|
||||
|
||||
assert(_partial_array_state_allocator == nullptr, "Attempt to initialize twice");
|
||||
_partial_array_state_allocator
|
||||
= new PartialArrayStateAllocator(ParallelGCThreads);
|
||||
|
||||
_stack_array_depth = new PSScannerTasksQueueSet(ParallelGCThreads);
|
||||
_stack_array_depth = new PSScannerTasksQueueSet(promotion_manager_num);
|
||||
|
||||
// Create and register the PSPromotionManager(s) for the worker threads.
|
||||
for(uint i=0; i<ParallelGCThreads; i++) {
|
||||
stack_array_depth()->register_queue(i, _manager_array[i].claimed_stack_depth());
|
||||
_manager_array[i]._partial_array_state_allocator_index = i;
|
||||
}
|
||||
// The VMThread gets its own PSPromotionManager, which is not available
|
||||
// for work stealing.
|
||||
@ -187,7 +186,8 @@ void PSPromotionManager::reset_stats() {
|
||||
|
||||
// Most members are initialized either by initialize() or reset().
|
||||
PSPromotionManager::PSPromotionManager()
|
||||
: _partial_array_stepper(ParallelGCThreads, ParGCArrayScanChunk)
|
||||
: _partial_array_state_allocator(_partial_array_state_manager),
|
||||
_partial_array_stepper(ParallelGCThreads, ParGCArrayScanChunk)
|
||||
{
|
||||
// We set the old lab's start array.
|
||||
_old_lab.set_start_array(old_gen()->start_array());
|
||||
@ -198,9 +198,6 @@ PSPromotionManager::PSPromotionManager()
|
||||
_target_stack_size = GCDrainStackTargetSize;
|
||||
}
|
||||
|
||||
// Initialize to a bad value; fixed by initialize().
|
||||
_partial_array_state_allocator_index = UINT_MAX;
|
||||
|
||||
// let's choose 1.5x the chunk size
|
||||
_min_array_size_for_chunking = (3 * ParGCArrayScanChunk / 2);
|
||||
|
||||
@ -317,7 +314,7 @@ void PSPromotionManager::process_array_chunk(PartialArrayState* state) {
|
||||
process_array_chunk_work<oop>(state->destination(), start, end);
|
||||
}
|
||||
// Release reference to state, now that we're done with it.
|
||||
_partial_array_state_allocator->release(_partial_array_state_allocator_index, state);
|
||||
_partial_array_state_allocator.release(state);
|
||||
}
|
||||
|
||||
void PSPromotionManager::push_objArray(oop old_obj, oop new_obj) {
|
||||
@ -331,11 +328,10 @@ void PSPromotionManager::push_objArray(oop old_obj, oop new_obj) {
|
||||
if (step._ncreate > 0) {
|
||||
TASKQUEUE_STATS_ONLY(++_arrays_chunked);
|
||||
PartialArrayState* state =
|
||||
_partial_array_state_allocator->allocate(_partial_array_state_allocator_index,
|
||||
old_obj, new_obj,
|
||||
step._index,
|
||||
array_length,
|
||||
step._ncreate);
|
||||
_partial_array_state_allocator.allocate(old_obj, new_obj,
|
||||
step._index,
|
||||
array_length,
|
||||
step._ncreate);
|
||||
for (uint i = 0; i < step._ncreate; ++i) {
|
||||
push_depth(ScannerTask(state));
|
||||
}
|
||||
|
||||
@ -28,6 +28,7 @@
|
||||
#include "gc/parallel/psPromotionLAB.hpp"
|
||||
#include "gc/shared/copyFailedInfo.hpp"
|
||||
#include "gc/shared/gcTrace.hpp"
|
||||
#include "gc/shared/partialArrayState.hpp"
|
||||
#include "gc/shared/partialArrayTaskStepper.hpp"
|
||||
#include "gc/shared/preservedMarks.hpp"
|
||||
#include "gc/shared/stringdedup/stringDedup.hpp"
|
||||
@ -50,8 +51,6 @@
|
||||
class MutableSpace;
|
||||
class PSOldGen;
|
||||
class ParCompactionManager;
|
||||
class PartialArrayState;
|
||||
class PartialArrayStateAllocator;
|
||||
|
||||
class PSPromotionManager {
|
||||
friend class PSScavenge;
|
||||
@ -88,9 +87,9 @@ class PSPromotionManager {
|
||||
|
||||
uint _target_stack_size;
|
||||
|
||||
static PartialArrayStateAllocator* _partial_array_state_allocator;
|
||||
static PartialArrayStateManager* _partial_array_state_manager;
|
||||
PartialArrayStateAllocator _partial_array_state_allocator;
|
||||
PartialArrayTaskStepper _partial_array_stepper;
|
||||
uint _partial_array_state_allocator_index;
|
||||
uint _min_array_size_for_chunking;
|
||||
|
||||
PreservedMarks* _preserved_marks;
|
||||
|
||||
@ -52,27 +52,8 @@ void PartialArrayState::add_references(size_t count) {
|
||||
assert(new_count >= count, "reference count overflow");
|
||||
}
|
||||
|
||||
class PartialArrayStateAllocator::Impl : public CHeapObj<mtGC> {
|
||||
struct FreeListEntry;
|
||||
|
||||
Arena* _arenas;
|
||||
FreeListEntry** _free_lists;
|
||||
uint _num_workers;
|
||||
|
||||
class PartialArrayStateAllocator::FreeListEntry {
|
||||
public:
|
||||
Impl(uint num_workers);
|
||||
~Impl();
|
||||
|
||||
NONCOPYABLE(Impl);
|
||||
|
||||
PartialArrayState* allocate(uint worker_id,
|
||||
oop src, oop dst,
|
||||
size_t index, size_t length,
|
||||
size_t initial_refcount);
|
||||
void release(uint worker_id, PartialArrayState* state);
|
||||
};
|
||||
|
||||
struct PartialArrayStateAllocator::Impl::FreeListEntry {
|
||||
FreeListEntry* _next;
|
||||
|
||||
FreeListEntry(FreeListEntry* next) : _next(next) {}
|
||||
@ -81,73 +62,80 @@ struct PartialArrayStateAllocator::Impl::FreeListEntry {
|
||||
NONCOPYABLE(FreeListEntry);
|
||||
};
|
||||
|
||||
PartialArrayStateAllocator::Impl::Impl(uint num_workers)
|
||||
: _arenas(NEW_C_HEAP_ARRAY(Arena, num_workers, mtGC)),
|
||||
_free_lists(NEW_C_HEAP_ARRAY(FreeListEntry*, num_workers, mtGC)),
|
||||
_num_workers(num_workers)
|
||||
{
|
||||
for (uint i = 0; i < _num_workers; ++i) {
|
||||
::new (&_arenas[i]) Arena(mtGC);
|
||||
_free_lists[i] = nullptr;
|
||||
}
|
||||
}
|
||||
PartialArrayStateAllocator::PartialArrayStateAllocator(PartialArrayStateManager* manager)
|
||||
: _manager(manager),
|
||||
_free_list(),
|
||||
_arena(manager->register_allocator())
|
||||
{}
|
||||
|
||||
PartialArrayStateAllocator::Impl::~Impl() {
|
||||
// We don't need to clean up the free lists. Deallocating the entries
|
||||
PartialArrayStateAllocator::~PartialArrayStateAllocator() {
|
||||
// We don't need to clean up the free list. Deallocating the entries
|
||||
// does nothing, since we're using arena allocation. Instead, leave it
|
||||
// to the arena destructor to release the memory.
|
||||
FREE_C_HEAP_ARRAY(FreeListEntry*, _free_lists);
|
||||
for (uint i = 0; i < _num_workers; ++i) {
|
||||
_arenas[i].~Arena();
|
||||
}
|
||||
FREE_C_HEAP_ARRAY(Arena*, _arenas);
|
||||
// to the manager to release the memory.
|
||||
// Inform the manager that an allocator is no longer in use.
|
||||
_manager->release_allocator();
|
||||
}
|
||||
|
||||
PartialArrayState* PartialArrayStateAllocator::Impl::allocate(uint worker_id,
|
||||
oop src, oop dst,
|
||||
size_t index,
|
||||
size_t length,
|
||||
size_t initial_refcount) {
|
||||
PartialArrayState* PartialArrayStateAllocator::allocate(oop src, oop dst,
|
||||
size_t index,
|
||||
size_t length,
|
||||
size_t initial_refcount) {
|
||||
void* p;
|
||||
FreeListEntry* head = _free_lists[worker_id];
|
||||
FreeListEntry* head = _free_list;
|
||||
if (head == nullptr) {
|
||||
p = NEW_ARENA_OBJ(&_arenas[worker_id], PartialArrayState);
|
||||
p = NEW_ARENA_OBJ(_arena, PartialArrayState);
|
||||
} else {
|
||||
_free_lists[worker_id] = head->_next;
|
||||
_free_list = head->_next;
|
||||
head->~FreeListEntry();
|
||||
p = head;
|
||||
}
|
||||
return ::new (p) PartialArrayState(src, dst, index, length, initial_refcount);
|
||||
}
|
||||
|
||||
void PartialArrayStateAllocator::Impl::release(uint worker_id, PartialArrayState* state) {
|
||||
void PartialArrayStateAllocator::release(PartialArrayState* state) {
|
||||
size_t refcount = Atomic::sub(&state->_refcount, size_t(1), memory_order_release);
|
||||
if (refcount != 0) {
|
||||
assert(refcount + 1 != 0, "refcount underflow");
|
||||
} else {
|
||||
OrderAccess::acquire();
|
||||
state->~PartialArrayState();
|
||||
_free_lists[worker_id] = ::new (state) FreeListEntry(_free_lists[worker_id]);
|
||||
// Don't need to call destructor; can't if not destructible.
|
||||
static_assert(!std::is_destructible<PartialArrayState>::value, "expected");
|
||||
_free_list = ::new (state) FreeListEntry(_free_list);
|
||||
}
|
||||
}
|
||||
|
||||
PartialArrayStateAllocator::PartialArrayStateAllocator(uint num_workers)
|
||||
: _impl(new Impl(num_workers))
|
||||
PartialArrayStateManager::PartialArrayStateManager(uint max_allocators)
|
||||
: _arenas(NEW_C_HEAP_ARRAY(Arena, max_allocators, mtGC)),
|
||||
_max_allocators(max_allocators),
|
||||
_registered_allocators(0)
|
||||
DEBUG_ONLY(COMMA _released_allocators(0))
|
||||
{}
|
||||
|
||||
PartialArrayStateAllocator::~PartialArrayStateAllocator() {
|
||||
delete _impl;
|
||||
PartialArrayStateManager::~PartialArrayStateManager() {
|
||||
reset();
|
||||
FREE_C_HEAP_ARRAY(Arena, _arenas);
|
||||
}
|
||||
|
||||
PartialArrayState* PartialArrayStateAllocator::allocate(uint worker_id,
|
||||
oop src, oop dst,
|
||||
size_t index,
|
||||
size_t length,
|
||||
size_t initial_refcount) {
|
||||
return _impl->allocate(worker_id, src, dst, index, length, initial_refcount);
|
||||
Arena* PartialArrayStateManager::register_allocator() {
|
||||
uint idx = Atomic::fetch_then_add(&_registered_allocators, 1u, memory_order_relaxed);
|
||||
assert(idx < _max_allocators, "exceeded configured max number of allocators");
|
||||
return ::new (&_arenas[idx]) Arena(mtGC);
|
||||
}
|
||||
|
||||
void PartialArrayStateAllocator::release(uint worker_id, PartialArrayState* state) {
|
||||
_impl->release(worker_id, state);
|
||||
#ifdef ASSERT
|
||||
void PartialArrayStateManager::release_allocator() {
|
||||
uint old = Atomic::fetch_then_add(&_released_allocators, 1u, memory_order_relaxed);
|
||||
assert(old < Atomic::load(&_registered_allocators), "too many releases");
|
||||
}
|
||||
#endif // ASSERT
|
||||
|
||||
void PartialArrayStateManager::reset() {
|
||||
uint count = Atomic::load(&_registered_allocators);
|
||||
assert(count == Atomic::load(&_released_allocators),
|
||||
"some allocators still active");
|
||||
for (uint i = 0; i < count; ++i) {
|
||||
_arenas[i].~Arena();
|
||||
}
|
||||
Atomic::store(&_registered_allocators, 0u);
|
||||
DEBUG_ONLY(Atomic::store(&_released_allocators, 0u);)
|
||||
}
|
||||
|
||||
@ -30,7 +30,9 @@
|
||||
#include "utilities/globalDefinitions.hpp"
|
||||
#include "utilities/macros.hpp"
|
||||
|
||||
class Arena;
|
||||
class PartialArrayStateAllocator;
|
||||
class PartialArrayStateManager;
|
||||
|
||||
// Instances of this class are used to represent processing progress for an
|
||||
// array task in a taskqueue. When a sufficiently large array needs to be
|
||||
@ -52,8 +54,8 @@ class PartialArrayStateAllocator;
|
||||
// referring to a given state that is added to a taskqueue must increase the
|
||||
// reference count by one. When the processing of a task referring to a state
|
||||
// is complete, the reference count must be decreased by one. When the
|
||||
// reference count reaches zero the state should be released to the allocator
|
||||
// for later reuse.
|
||||
// reference count reaches zero the state is released to the allocator for
|
||||
// later reuse.
|
||||
class PartialArrayState {
|
||||
oop _source;
|
||||
oop _destination;
|
||||
@ -66,11 +68,13 @@ class PartialArrayState {
|
||||
PartialArrayState(oop src, oop dst,
|
||||
size_t index, size_t length,
|
||||
size_t initial_refcount);
|
||||
~PartialArrayState() = default;
|
||||
|
||||
public:
|
||||
// Deleted to require management by allocator object.
|
||||
~PartialArrayState() = delete;
|
||||
|
||||
NONCOPYABLE(PartialArrayState);
|
||||
|
||||
public:
|
||||
// Add count references, one per referring task being added to a taskqueue.
|
||||
void add_references(size_t count);
|
||||
|
||||
@ -91,39 +95,39 @@ public:
|
||||
|
||||
// This class provides memory management for PartialArrayStates.
|
||||
//
|
||||
// States are initially allocated from a set of arenas owned by the allocator.
|
||||
// This allows the entire set of allocated states to be discarded without the
|
||||
// need to keep track of or find them under some circumstances. For example,
|
||||
// if G1 concurrent marking is aborted and needs to restart because of a full
|
||||
// marking queue, the queue doesn't need to be searched for tasks referring to
|
||||
// states to allow releasing them. Instead the queue contents can just be
|
||||
// discarded, and the memory for the no longer referenced states will
|
||||
// eventually be reclaimed when the arenas are reset.
|
||||
// States are initially arena allocated from the manager, using a per-thread
|
||||
// allocator. This allows the entire set of allocated states to be discarded
|
||||
// without the need to keep track of or find them under some circumstances.
|
||||
// For example, if G1 concurrent marking is aborted and needs to restart
|
||||
// because of a full marking queue, the queue doesn't need to be searched for
|
||||
// tasks referring to states to allow releasing them. Instead the queue
|
||||
// contents can just be discarded, and the memory for the no longer referenced
|
||||
// states will eventually be reclaimed when the arena is reset.
|
||||
//
|
||||
// A set of free-lists is placed in front of the arena allocators. This
|
||||
// causes the maximum number of allocated states to be based on the number of
|
||||
// The allocators each provide a free-list of states. When a state is
|
||||
// released and its reference count has reached zero, it is added to the
|
||||
// allocator's free-list, for use by future allocation requests. This causes
|
||||
// the maximum number of allocated states to be based on the number of
|
||||
// in-progress arrays, rather than the total number of arrays that need to be
|
||||
// processed. The use of free-list allocators is the reason for reference
|
||||
// counting states.
|
||||
// processed.
|
||||
//
|
||||
// The arena and free-list to use for an allocation operation is designated by
|
||||
// the worker_id used in the operation. This avoids locking and such on those
|
||||
// data structures, at the cost of possibly doing more total arena allocation
|
||||
// that would be needed with a single shared arena and free-list.
|
||||
// An allocator object is not thread-safe.
|
||||
class PartialArrayStateAllocator : public CHeapObj<mtGC> {
|
||||
class Impl;
|
||||
Impl* _impl;
|
||||
class FreeListEntry;
|
||||
|
||||
PartialArrayStateManager* _manager;
|
||||
FreeListEntry* _free_list;
|
||||
Arena* _arena; // Obtained from _manager.
|
||||
|
||||
public:
|
||||
PartialArrayStateAllocator(uint num_workers);
|
||||
explicit PartialArrayStateAllocator(PartialArrayStateManager* manager);
|
||||
~PartialArrayStateAllocator();
|
||||
|
||||
NONCOPYABLE(PartialArrayStateAllocator);
|
||||
|
||||
// Create a new state, obtaining the memory for it from the free-list or
|
||||
// arena associated with worker_id.
|
||||
PartialArrayState* allocate(uint worker_id,
|
||||
oop src, oop dst,
|
||||
// from the associated manager.
|
||||
PartialArrayState* allocate(oop src, oop dst,
|
||||
size_t index, size_t length,
|
||||
size_t initial_refcount);
|
||||
|
||||
@ -131,7 +135,70 @@ public:
|
||||
// state to the free-list associated with worker_id. The state must have
|
||||
// been allocated by this allocator, but that allocation doesn't need to
|
||||
// have been associated with worker_id.
|
||||
void release(uint worker_id, PartialArrayState* state);
|
||||
void release(PartialArrayState* state);
|
||||
};
|
||||
|
||||
// This class provides memory management for PartialArrayStates.
|
||||
//
|
||||
// States are allocated using an allocator object. Those allocators in turn
|
||||
// may request memory for a state from their associated manager. The manager
|
||||
// is responsible for obtaining and releasing memory used for states by the
|
||||
// associated allocators.
|
||||
//
|
||||
// A state may be allocated by one allocator, but end up on the free-list of a
|
||||
// different allocator. This can happen because a task referring to the state
|
||||
// may be stolen from the queue where it was initially added. This is permitted
|
||||
// because a state's memory won't be reclaimed until all of the allocators
|
||||
// associated with the manager that is ultimately providing the memory have
|
||||
// been deleted and the manager is reset.
|
||||
//
|
||||
// A manager is used in two distinct and non-overlapping phases.
|
||||
//
|
||||
// - allocating: This is the initial phase. During this phase, new allocators
|
||||
// may be created, and allocators may request memory from the manager.
|
||||
//
|
||||
// - releasing: When an allocator is destroyed the manager transitions to this
|
||||
// phase. It remains in this phase until all extent allocators associated with
|
||||
// this manager have been destroyed. During this phase, new allocators may not
|
||||
// be created, nor may extent allocators request memory from this manager.
|
||||
//
|
||||
// Once all the associated allocators have been destroyed the releasing phase
|
||||
// ends and the manager may be reset or deleted. Resetting transitions back
|
||||
// to the allocating phase.
|
||||
class PartialArrayStateManager : public CHeapObj<mtGC> {
|
||||
friend class PartialArrayStateAllocator;
|
||||
|
||||
// Use an arena for each allocator, for thread-safe concurrent allocation by
|
||||
// different allocators.
|
||||
Arena* _arenas;
|
||||
|
||||
// Limit on the number of allocators this manager supports.
|
||||
uint _max_allocators;
|
||||
|
||||
// The number of allocators that have been registered/released.
|
||||
// Atomic to support concurrent registration, and concurrent release.
|
||||
// Phasing restriction forbids registration concurrent with release.
|
||||
volatile uint _registered_allocators;
|
||||
DEBUG_ONLY(volatile uint _released_allocators;)
|
||||
|
||||
// These are all for sole use of the befriended allocator class.
|
||||
Arena* register_allocator();
|
||||
void release_allocator() NOT_DEBUG_RETURN;
|
||||
|
||||
public:
|
||||
explicit PartialArrayStateManager(uint max_allocators);
|
||||
|
||||
// Release the memory that has been requested by allocators associated with
|
||||
// this manager.
|
||||
// precondition: all associated allocators have been deleted.
|
||||
~PartialArrayStateManager();
|
||||
|
||||
NONCOPYABLE(PartialArrayStateManager);
|
||||
|
||||
// Recycle the memory that has been requested by allocators associated with
|
||||
// this manager.
|
||||
// precondition: all associated allocators have been deleted.
|
||||
void reset();
|
||||
};
|
||||
|
||||
#endif // SHARE_GC_SHARED_PARTIALARRAYSTATE_HPP
|
||||
|
||||
@ -187,6 +187,7 @@ public class TestGCLogMessages {
|
||||
new LogMessageWithLevel("Serial Free Collection Set:", Level.TRACE),
|
||||
new LogMessageWithLevel("Young Free Collection Set \\(ms\\):", Level.TRACE),
|
||||
new LogMessageWithLevel("Non-Young Free Collection Set \\(ms\\):", Level.TRACE),
|
||||
new LogMessageWithLevel("Reset Partial Array State Manager \\(ms\\)", Level.TRACE),
|
||||
|
||||
// Misc Top-level
|
||||
new LogMessageWithLevel("Rebuild Free List:", Level.DEBUG),
|
||||
|
||||
@ -111,6 +111,7 @@ public class TestG1ParallelPhases {
|
||||
"FreeCSet",
|
||||
"UpdateDerivedPointers",
|
||||
"EagerlyReclaimHumongousObjects",
|
||||
"ResetPartialArrayStateManager",
|
||||
"ClearLoggedCards",
|
||||
"MergePSS",
|
||||
"NonYoungFreeCSet",
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user