< prev index next >

src/hotspot/share/gc/g1/g1DirtyCardQueue.cpp

Print this page
rev 57716 : imported patch remove_cbl_mon
rev 57717 : [mq]: helper_classes

*** 1,7 **** /* ! * 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. --- 1,7 ---- /* ! * Copyright (c) 2001, 2020, 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.
*** 24,49 **** #include "precompiled.hpp" #include "gc/g1/g1BufferNodeList.hpp" #include "gc/g1/g1CardTableEntryClosure.hpp" #include "gc/g1/g1CollectedHeap.inline.hpp" #include "gc/g1/g1DirtyCardQueue.hpp" #include "gc/g1/g1FreeIdSet.hpp" #include "gc/g1/g1RedirtyCardsQueue.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 "memory/iterator.hpp" ! #include "runtime/flags/flagSetting.hpp" ! #include "runtime/mutexLocker.hpp" ! #include "runtime/orderAccess.hpp" #include "runtime/os.hpp" #include "runtime/safepoint.hpp" #include "runtime/thread.inline.hpp" #include "runtime/threadSMR.hpp" #include "utilities/quickSort.hpp" G1DirtyCardQueue::G1DirtyCardQueue(G1DirtyCardQueueSet* qset) : // Dirty card queues are always active, so we create them with their // active field set to true. --- 24,49 ---- #include "precompiled.hpp" #include "gc/g1/g1BufferNodeList.hpp" #include "gc/g1/g1CardTableEntryClosure.hpp" #include "gc/g1/g1CollectedHeap.inline.hpp" + #include "gc/g1/g1ConcurrentRefineThread.hpp" #include "gc/g1/g1DirtyCardQueue.hpp" #include "gc/g1/g1FreeIdSet.hpp" #include "gc/g1/g1RedirtyCardsQueue.hpp" #include "gc/g1/g1RemSet.hpp" #include "gc/g1/g1ThreadLocalData.hpp" #include "gc/g1/heapRegionRemSet.hpp" #include "gc/shared/suspendibleThreadSet.hpp" #include "memory/iterator.hpp" ! #include "runtime/atomic.hpp" #include "runtime/os.hpp" #include "runtime/safepoint.hpp" #include "runtime/thread.inline.hpp" #include "runtime/threadSMR.hpp" + #include "utilities/globalCounter.inline.hpp" + #include "utilities/macros.hpp" #include "utilities/quickSort.hpp" G1DirtyCardQueue::G1DirtyCardQueue(G1DirtyCardQueueSet* qset) : // Dirty card queues are always active, so we create them with their // active field set to true.
*** 66,87 **** } // Assumed to be zero by concurrent threads. static uint par_ids_start() { return 0; } ! G1DirtyCardQueueSet::G1DirtyCardQueueSet(Monitor* cbl_mon, ! BufferNode::Allocator* allocator) : PtrQueueSet(allocator), ! _cbl_mon(cbl_mon), ! _completed_buffers_head(NULL), ! _completed_buffers_tail(NULL), _num_cards(0), _process_cards_threshold(ProcessCardsThresholdNever), - _process_completed_buffers(false), _max_cards(MaxCardsUnlimited), _max_cards_padding(0), - _free_ids(par_ids_start(), num_par_ids()), _mutator_refined_cards_counters(NEW_C_HEAP_ARRAY(size_t, num_par_ids(), mtGC)) { ::memset(_mutator_refined_cards_counters, 0, num_par_ids() * sizeof(size_t)); _all_active = true; } --- 66,85 ---- } // Assumed to be zero by concurrent threads. static uint par_ids_start() { return 0; } ! G1DirtyCardQueueSet::G1DirtyCardQueueSet(BufferNode::Allocator* allocator) : PtrQueueSet(allocator), ! _primary_refinement_thread(NULL), _num_cards(0), + _completed(), + _paused(), + _free_ids(par_ids_start(), num_par_ids()), _process_cards_threshold(ProcessCardsThresholdNever), _max_cards(MaxCardsUnlimited), _max_cards_padding(0), _mutator_refined_cards_counters(NEW_C_HEAP_ARRAY(size_t, num_par_ids(), mtGC)) { ::memset(_mutator_refined_cards_counters, 0, num_par_ids() * sizeof(size_t)); _all_active = true; }
*** 106,233 **** void G1DirtyCardQueueSet::handle_zero_index_for_thread(Thread* t) { G1ThreadLocalData::dirty_card_queue(t).handle_zero_index(); } ! void G1DirtyCardQueueSet::enqueue_completed_buffer(BufferNode* cbn) { ! MonitorLocker ml(_cbl_mon, Mutex::_no_safepoint_check_flag); ! cbn->set_next(NULL); ! if (_completed_buffers_tail == NULL) { ! assert(_completed_buffers_head == NULL, "Well-formedness"); ! _completed_buffers_head = cbn; ! _completed_buffers_tail = cbn; } else { ! _completed_buffers_tail->set_next(cbn); ! _completed_buffers_tail = cbn; } ! _num_cards += buffer_size() - cbn->index(); ! if (!process_completed_buffers() && ! (num_cards() > process_cards_threshold())) { ! set_process_completed_buffers(true); ! ml.notify_all(); } - verify_num_cards(); } BufferNode* G1DirtyCardQueueSet::get_completed_buffer(size_t stop_at) { ! MutexLocker x(_cbl_mon, Mutex::_no_safepoint_check_flag); ! if (num_cards() <= stop_at) { return NULL; } ! assert(num_cards() > 0, "invariant"); ! assert(_completed_buffers_head != NULL, "invariant"); ! assert(_completed_buffers_tail != NULL, "invariant"); ! ! BufferNode* bn = _completed_buffers_head; ! _num_cards -= buffer_size() - bn->index(); ! _completed_buffers_head = bn->next(); ! if (_completed_buffers_head == NULL) { ! assert(num_cards() == 0, "invariant"); ! _completed_buffers_tail = NULL; ! set_process_completed_buffers(false); } ! verify_num_cards(); ! bn->set_next(NULL); ! return bn; } #ifdef ASSERT void G1DirtyCardQueueSet::verify_num_cards() const { size_t actual = 0; ! BufferNode* cur = _completed_buffers_head; ! while (cur != NULL) { actual += buffer_size() - cur->index(); - cur = cur->next(); } ! assert(actual == _num_cards, "Num entries in completed buffers should be " SIZE_FORMAT " but are " SIZE_FORMAT, ! _num_cards, actual); } ! #endif ! void G1DirtyCardQueueSet::abandon_completed_buffers() { ! BufferNode* buffers_to_delete = NULL; { ! MutexLocker x(_cbl_mon, Mutex::_no_safepoint_check_flag); ! buffers_to_delete = _completed_buffers_head; ! _completed_buffers_head = NULL; ! _completed_buffers_tail = NULL; ! _num_cards = 0; ! set_process_completed_buffers(false); } while (buffers_to_delete != NULL) { BufferNode* bn = buffers_to_delete; buffers_to_delete = bn->next(); bn->set_next(NULL); deallocate_buffer(bn); } } void G1DirtyCardQueueSet::notify_if_necessary() { ! MonitorLocker ml(_cbl_mon, Mutex::_no_safepoint_check_flag); ! if (num_cards() > process_cards_threshold()) { ! set_process_completed_buffers(true); ! ml.notify_all(); } } ! // Merge lists of buffers. Notify the processing threads. ! // The source queue is emptied as a result. The queues ! // must share the monitor. void G1DirtyCardQueueSet::merge_bufferlists(G1RedirtyCardsQueueSet* src) { assert(allocator() == src->allocator(), "precondition"); const G1BufferNodeList from = src->take_all_completed_buffers(); ! if (from._head == NULL) return; ! ! MutexLocker x(_cbl_mon, Mutex::_no_safepoint_check_flag); ! if (_completed_buffers_tail == NULL) { ! assert(_completed_buffers_head == NULL, "Well-formedness"); ! _completed_buffers_head = from._head; ! _completed_buffers_tail = from._tail; ! } else { ! assert(_completed_buffers_head != NULL, "Well formedness"); ! _completed_buffers_tail->set_next(from._head); ! _completed_buffers_tail = from._tail; ! } ! _num_cards += from._entry_count; ! ! assert(_completed_buffers_head == NULL && _completed_buffers_tail == NULL || ! _completed_buffers_head != NULL && _completed_buffers_tail != NULL, ! "Sanity"); ! verify_num_cards(); } G1BufferNodeList G1DirtyCardQueueSet::take_all_completed_buffers() { ! MutexLocker x(_cbl_mon, Mutex::_no_safepoint_check_flag); ! G1BufferNodeList result(_completed_buffers_head, _completed_buffers_tail, _num_cards); ! _completed_buffers_head = NULL; ! _completed_buffers_tail = NULL; ! _num_cards = 0; ! return result; } class G1RefineBufferedCards : public StackObj { BufferNode* const _node; CardTable::CardValue** const _node_buffer; --- 104,444 ---- void G1DirtyCardQueueSet::handle_zero_index_for_thread(Thread* t) { G1ThreadLocalData::dirty_card_queue(t).handle_zero_index(); } ! #ifdef ASSERT ! G1DirtyCardQueueSet::Queue::~Queue() { ! assert(_head == NULL, "precondition"); ! assert(_tail == NULL, "precondition"); ! } ! #endif // ASSERT ! ! BufferNode* G1DirtyCardQueueSet::Queue::top() const { ! return Atomic::load(&_head); ! } ! ! // An append operation atomically exchanges the new tail with the queue tail. ! // It then sets the "next" value of the old tail to the head of the list being ! // appended; it is an invariant that the old tail's "next" value is NULL. ! // But if the old tail is NULL then the queue was empty. In this case the ! // head of the list being appended is instead stored in the queue head; it is ! // an invariant that the queue head is NULL in this case. ! // ! // This means there is a period between the exchange and the old tail update ! // where the queue sequence is split into two parts, the list from the queue ! // head to the old tail, and the list being appended. If there are concurrent ! // push/append operations, each may introduce another such segment. But they ! // all eventually get resolved by their respective updates of their old tail's ! // "next" value. This also means that pop operations must handle a buffer ! // with a NULL "next" value specially. ! // ! // A push operation is just a degenerate append, where the buffer being pushed ! // is both the head and the tail of the list being appended. ! void G1DirtyCardQueueSet::Queue::append(BufferNode& first, BufferNode& last) { ! assert(last.next() == NULL, "precondition"); ! BufferNode* old_tail = Atomic::xchg(&_tail, &last); ! if (old_tail == NULL) { // Was empty. ! assert(Atomic::load(&_head) == NULL, "invariant"); ! Atomic::store(&_head, &first); } else { ! assert(old_tail->next() == NULL, "invariant"); ! old_tail->set_next(&first); } ! } ! // pop gets the queue head as the candidate result (returning NULL if the ! // queue head was NULL), and then gets that result node's "next" value. If ! // that "next" value is NULL and the queue head hasn't changed, then there ! // is only one element in the accessible part of the list (the sequence from ! // head to a node with a NULL "next" value). We can't return that element, ! // because it may be the old tail of a concurrent push/append that has not ! // yet had its "next" field set to the new tail. So return NULL in this case. ! // Otherwise, attempt to cmpxchg that "next" value into the queue head, ! // retrying the whole operation if that fails. This is the "usual" lock-free ! // pop from the head of a singly linked list, with the additional restriction ! // on taking the last element. ! BufferNode* G1DirtyCardQueueSet::Queue::pop() { ! Thread* current_thread = Thread::current(); ! while (true) { ! // Use a critical section per iteration, rather than over the whole ! // operation. We're not guaranteed to make progress, because of possible ! // contention on the queue head. Lingering in one CS the whole time could ! // lead to excessive allocation of buffers, because the CS blocks return ! // of released buffers to the free list for reuse. ! GlobalCounter::CriticalSection cs(current_thread); ! ! BufferNode* result = Atomic::load_acquire(&_head); ! // Check for empty queue. Only needs to be done on first iteration, ! // since we never take the last element, but it's messy to make use ! // of that and we expect one iteration to be the common case. ! if (result == NULL) return NULL; ! ! BufferNode* next = Atomic::load_acquire(BufferNode::next_ptr(*result)); ! if (next != NULL) { ! next = Atomic::cmpxchg(&_head, result, next); ! if (next == result) { ! // Former head successfully taken; it is not the last. ! assert(Atomic::load(&_tail) != result, "invariant"); ! assert(result->next() != NULL, "invariant"); ! result->set_next(NULL); ! return result; ! } ! // cmpxchg failed; try again. ! } else if (result == Atomic::load_acquire(&_head)) { ! // If follower of head is NULL and head hasn't changed, then only ! // the one element is currently accessible. We don't take the last ! // accessible element, because there may be a concurrent add using it. ! // The check for unchanged head isn't needed for correctness, but the ! // retry on change may sometimes let us get a buffer after all. ! return NULL; ! } ! // Head changed; try again. ! } ! } ! ! G1DirtyCardQueueSet::HeadTail G1DirtyCardQueueSet::Queue::take_all() { ! assert_at_safepoint(); ! HeadTail result(Atomic::load(&_head), Atomic::load(&_tail)); ! Atomic::store(&_head, (BufferNode*)NULL); ! Atomic::store(&_tail, (BufferNode*)NULL); ! return result; ! } ! ! void G1DirtyCardQueueSet::enqueue_completed_buffer(BufferNode* cbn) { ! assert(cbn != NULL, "precondition"); ! // Increment _num_cards before adding to queue, so queue removal doesn't ! // need to deal with _num_cards possibly going negative. ! size_t new_num_cards = Atomic::add(&_num_cards, buffer_size() - cbn->index()); ! _completed.push(*cbn); ! if ((new_num_cards > process_cards_threshold()) && ! (_primary_refinement_thread != NULL)) { ! _primary_refinement_thread->activate(); } } BufferNode* G1DirtyCardQueueSet::get_completed_buffer(size_t stop_at) { ! enqueue_previous_paused_buffers(); ! // Check for unsufficient cards to satisfy request. We only do this once, ! // up front, rather than on each iteration below, since the test is racy ! // regardless of when we do it. ! if (Atomic::load_acquire(&_num_cards) <= stop_at) { return NULL; } ! BufferNode* result = _completed.pop(); ! if (result != NULL) { ! Atomic::sub(&_num_cards, buffer_size() - result->index()); } ! return result; } #ifdef ASSERT void G1DirtyCardQueueSet::verify_num_cards() const { size_t actual = 0; ! BufferNode* cur = _completed.top(); ! for ( ; cur != NULL; cur = cur->next()) { actual += buffer_size() - cur->index(); } ! assert(actual == Atomic::load(&_num_cards), "Num entries in completed buffers should be " SIZE_FORMAT " but are " SIZE_FORMAT, ! Atomic::load(&_num_cards), actual); } ! #endif // ASSERT ! G1DirtyCardQueueSet::PausedBuffers::PausedList::PausedList() : ! _head(NULL), _tail(NULL), ! _safepoint_id(SafepointSynchronize::safepoint_id()) ! {} ! ! #ifdef ASSERT ! G1DirtyCardQueueSet::PausedBuffers::PausedList::~PausedList() { ! assert(Atomic::load(&_head) == NULL, "precondition"); ! assert(_tail == NULL, "precondition"); ! } ! #endif // ASSERT ! ! bool G1DirtyCardQueueSet::PausedBuffers::PausedList::is_next() const { ! assert_not_at_safepoint(); ! return _safepoint_id == SafepointSynchronize::safepoint_id(); ! } ! ! void G1DirtyCardQueueSet::PausedBuffers::PausedList::add(BufferNode* node) { ! assert_not_at_safepoint(); ! assert(is_next(), "precondition"); ! BufferNode* old_head = Atomic::xchg(&_head, node); ! if (old_head == NULL) { ! assert(_tail == NULL, "invariant"); ! _tail = node; ! } else { ! node->set_next(old_head); ! } ! } ! ! G1DirtyCardQueueSet::HeadTail G1DirtyCardQueueSet::PausedBuffers::PausedList::take() { ! BufferNode* head = Atomic::load(&_head); ! BufferNode* tail = _tail; ! Atomic::store(&_head, (BufferNode*)NULL); ! _tail = NULL; ! return HeadTail(head, tail); ! } ! ! G1DirtyCardQueueSet::PausedBuffers::PausedBuffers() : _plist(NULL) {} ! ! #ifdef ASSERT ! G1DirtyCardQueueSet::PausedBuffers::~PausedBuffers() { ! assert(empty(), "invariant"); ! } ! #endif // ASSERT ! ! bool G1DirtyCardQueueSet::PausedBuffers::empty() const { ! return Atomic::load(&_plist) == NULL; ! } ! ! void G1DirtyCardQueueSet::PausedBuffers::add(BufferNode* node) { ! assert_not_at_safepoint(); ! PausedList* plist = Atomic::load_acquire(&_plist); ! if (plist != NULL) { ! // Already have a next list, so use it. We know it's a next list because ! // of the precondition that take_previous() has already been called. ! assert(plist->is_next(), "invariant"); ! } else { ! // Try to install a new next list. ! plist = new PausedList(); ! PausedList* old_plist = Atomic::cmpxchg(&_plist, (PausedList*)NULL, plist); ! if (old_plist != NULL) { ! // Some other thread installed a new next list. Use it instead. ! delete plist; ! plist = old_plist; ! } ! } ! plist->add(node); ! } ! ! G1DirtyCardQueueSet::HeadTail G1DirtyCardQueueSet::PausedBuffers::take_previous() { ! assert_not_at_safepoint(); ! PausedList* previous; { ! // Deal with plist in a critical section, to prevent it from being ! // deleted out from under us by a concurrent take_previous(). ! GlobalCounter::CriticalSection cs(Thread::current()); ! previous = Atomic::load_acquire(&_plist); ! if ((previous == NULL) || // Nothing to take. ! previous->is_next() || // Not from a previous safepoint. ! // Some other thread stole it. ! (Atomic::cmpxchg(&_plist, previous, (PausedList*)NULL) != previous)) { ! return HeadTail(); ! } ! } ! // We now own previous. ! HeadTail result = previous->take(); ! // There might be other threads examining previous (in concurrent ! // take_previous()). Synchronize to wait until any such threads are ! // done with such examination before deleting. ! GlobalCounter::write_synchronize(); ! delete previous; ! return result; ! } ! ! G1DirtyCardQueueSet::HeadTail G1DirtyCardQueueSet::PausedBuffers::take_all() { ! assert_at_safepoint(); ! HeadTail result; ! PausedList* plist = Atomic::load(&_plist); ! if (plist != NULL) { ! Atomic::store(&_plist, (PausedList*)NULL); ! result = plist->take(); ! delete plist; } + return result; + } + + void G1DirtyCardQueueSet::record_paused_buffer(BufferNode* node) { + assert_not_at_safepoint(); + assert(node->next() == NULL, "precondition"); + // Cards for paused buffers are included in count, to contribute to + // notification checking after the coming safepoint if it doesn't GC. + // Note that this means the queue's _num_cards differs from the number + // of cards in the queued buffers when there are paused buffers. + Atomic::add(&_num_cards, buffer_size() - node->index()); + _paused.add(node); + } + + void G1DirtyCardQueueSet::enqueue_paused_buffers_aux(const HeadTail& paused) { + if (paused._head != NULL) { + assert(paused._tail != NULL, "invariant"); + // Cards from paused buffers are already recorded in the queue count. + _completed.append(*paused._head, *paused._tail); + } + } + + void G1DirtyCardQueueSet::enqueue_previous_paused_buffers() { + assert_not_at_safepoint(); + // The fast-path still satisfies the precondition for record_paused_buffer + // and PausedBuffers::add, even with a racy test. If there are paused + // buffers from a previous safepoint, empty() will return false; there will + // have been a safepoint between recording and test, so there can't be a + // false negative (empty() returns true) while such buffers are present. + // If empty() is false, there are two cases: + // + // (1) There were paused buffers from a previous safepoint. A concurrent + // caller may take and enqueue them first, but that's okay; the precondition + // for a possible later record_paused_buffer by this thread will still hold. + // + // (2) There are paused buffers for a requested next safepoint. + // + // In each of those cases some effort may be spent detecting and dealing + // with those circumstances; any wasted effort in such cases is expected to + // be well compensated by the fast path. + if (!_paused.empty()) { + enqueue_paused_buffers_aux(_paused.take_previous()); + } + } + + void G1DirtyCardQueueSet::enqueue_all_paused_buffers() { + assert_at_safepoint(); + enqueue_paused_buffers_aux(_paused.take_all()); + } + + void G1DirtyCardQueueSet::abandon_completed_buffers() { + enqueue_all_paused_buffers(); + verify_num_cards(); + G1BufferNodeList list = take_all_completed_buffers(); + BufferNode* buffers_to_delete = list._head; while (buffers_to_delete != NULL) { BufferNode* bn = buffers_to_delete; buffers_to_delete = bn->next(); bn->set_next(NULL); deallocate_buffer(bn); } } void G1DirtyCardQueueSet::notify_if_necessary() { ! if ((_primary_refinement_thread != NULL) && ! (num_cards() > process_cards_threshold())) { ! _primary_refinement_thread->activate(); } } ! // Merge lists of buffers. The source queue set is emptied as a ! // result. The queue sets must share the same allocator. void G1DirtyCardQueueSet::merge_bufferlists(G1RedirtyCardsQueueSet* src) { assert(allocator() == src->allocator(), "precondition"); const G1BufferNodeList from = src->take_all_completed_buffers(); ! if (from._head != NULL) { ! Atomic::add(&_num_cards, from._entry_count); ! _completed.append(*from._head, *from._tail); ! } } G1BufferNodeList G1DirtyCardQueueSet::take_all_completed_buffers() { ! enqueue_all_paused_buffers(); ! verify_num_cards(); ! HeadTail buffers = _completed.take_all(); ! size_t num_cards = Atomic::load(&_num_cards); ! Atomic::store(&_num_cards, size_t(0)); ! return G1BufferNodeList(buffers._head, buffers._tail, num_cards); } class G1RefineBufferedCards : public StackObj { BufferNode* const _node; CardTable::CardValue** const _node_buffer;
*** 366,383 **** #endif // ASSERT bool G1DirtyCardQueueSet::process_or_enqueue_completed_buffer(BufferNode* node) { if (Thread::current()->is_Java_thread()) { // If the number of buffers exceeds the limit, make this Java ! // thread do the processing itself. We don't lock to access ! // buffer count or padding; it is fine to be imprecise here. The ! // add of padding could overflow, which is treated as unlimited. size_t limit = max_cards() + max_cards_padding(); if ((num_cards() > limit) && (limit >= max_cards())) { if (mut_process_buffer(node)) { return true; } } } enqueue_completed_buffer(node); return false; } --- 577,600 ---- #endif // ASSERT bool G1DirtyCardQueueSet::process_or_enqueue_completed_buffer(BufferNode* node) { if (Thread::current()->is_Java_thread()) { // If the number of buffers exceeds the limit, make this Java ! // thread do the processing itself. Calculation is racy but we ! // don't need precision here. The add of padding could overflow, ! // which is treated as unlimited. size_t limit = max_cards() + max_cards_padding(); if ((num_cards() > limit) && (limit >= max_cards())) { if (mut_process_buffer(node)) { return true; } + // Buffer was incompletely processed because of a pending safepoint + // request. Unlike with refinement thread processing, for mutator + // processing the buffer did not come from the completed buffer queue, + // so it is okay to add it to the queue rather than to the paused set. + // Indeed, it can't be added to the paused set because we didn't pass + // through enqueue_previous_paused_buffers. } } enqueue_completed_buffer(node); return false; }
*** 405,422 **** assert_fully_consumed(node, buffer_size()); // Done with fully processed buffer. deallocate_buffer(node); return true; } else { ! // Return partially processed buffer to the queue. ! enqueue_completed_buffer(node); return true; } } 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. struct AbandonThreadLogClosure : public ThreadClosure { --- 622,640 ---- assert_fully_consumed(node, buffer_size()); // Done with fully processed buffer. deallocate_buffer(node); return true; } else { ! // Buffer incompletely processed because there is a pending safepoint. ! // Record partially processed buffer, to be finished later. ! record_paused_buffer(node); return true; } } void G1DirtyCardQueueSet::abandon_logs() { ! assert_at_safepoint(); abandon_completed_buffers(); // Since abandon is done only at safepoints, we can safely manipulate // these queues. struct AbandonThreadLogClosure : public ThreadClosure {
*** 431,441 **** 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_cards(); set_max_cards(MaxCardsUnlimited); struct ConcatenateThreadLogClosure : public ThreadClosure { virtual void do_thread(Thread* t) { --- 649,659 ---- 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_at_safepoint(); size_t old_limit = max_cards(); set_max_cards(MaxCardsUnlimited); struct ConcatenateThreadLogClosure : public ThreadClosure { virtual void do_thread(Thread* t) {
*** 446,452 **** --- 664,672 ---- } } closure; Threads::threads_do(&closure); G1BarrierSet::shared_dirty_card_queue().flush(); + enqueue_all_paused_buffers(); + verify_num_cards(); set_max_cards(old_limit); }
< prev index next >