src/share/vm/gc_implementation/g1/concurrentMark.cpp

Print this page
rev 4271 : 8009536: G1: Apache Lucene hang during reference processing
Summary: In CMTask::do_marking_step(), Skip offering termination and entering the first and second synchronization barriers if called from a serial context, i.e. the VM thread.
Reviewed-by:

*** 804,814 **** } else { // We currently assume that the concurrent flag has been set to // false before we start remark. At this point we should also be // in a STW phase. assert(!concurrent_marking_in_progress(), "invariant"); ! assert(_finger == _heap_end, "only way to get here"); update_g1_committed(true); } } void ConcurrentMark::set_non_marking_state() { --- 804,816 ---- } else { // We currently assume that the concurrent flag has been set to // false before we start remark. At this point we should also be // in a STW phase. assert(!concurrent_marking_in_progress(), "invariant"); ! assert(_finger == _heap_end, ! err_msg("only way to get here: _finger: "PTR_FORMAT", _heap_end: "PTR_FORMAT, ! _finger, _heap_end)); update_g1_committed(true); } } void ConcurrentMark::set_non_marking_state() {
*** 972,981 **** --- 974,989 ---- if (verbose_low()) { gclog_or_tty->print_cr("[%u] leaving first barrier", worker_id); } + // If we're during the concurrent phase of marking, reset the marking + // state; otherwise the marking state is reset after reference processing, + // during the remark pause. + // If we reset here as a result of an overflow during the remark we + // will see assertion failures from any subsequent set_phase() calls. + if (concurrent()) { // let the task associated with with worker 0 do this if (worker_id == 0) { // task 0 is responsible for clearing the global data structures // We should be here because of an overflow. During STW we should // not clear the overflow flag since we rely on it being true when
*** 988,997 **** --- 996,1006 ---- gclog_or_tty->date_stamp(PrintGCDateStamps); gclog_or_tty->stamp(PrintGCTimeStamps); gclog_or_tty->print_cr("[GC concurrent-mark-reset-for-overflow]"); } } + } // after this, each task should reset its own data structures then // then go into the second barrier }
*** 1064,1074 **** double start_time_sec = os::elapsedTime(); double mark_step_duration_ms = G1ConcMarkStepDurationMillis; the_task->do_marking_step(mark_step_duration_ms, true /* do_stealing */, ! true /* do_termination */); double end_time_sec = os::elapsedTime(); double end_vtime_sec = os::elapsedVTime(); double elapsed_vtime_sec = end_vtime_sec - start_vtime_sec; double elapsed_time_sec = end_time_sec - start_time_sec; --- 1073,1084 ---- double start_time_sec = os::elapsedTime(); double mark_step_duration_ms = G1ConcMarkStepDurationMillis; the_task->do_marking_step(mark_step_duration_ms, true /* do_stealing */, ! true /* do_termination */, ! false /* is_serial */); double end_time_sec = os::elapsedTime(); double end_vtime_sec = os::elapsedVTime(); double elapsed_vtime_sec = end_vtime_sec - start_vtime_sec; double elapsed_time_sec = end_time_sec - start_time_sec;
*** 2191,2204 **** class G1CMKeepAliveAndDrainClosure: public OopClosure { ConcurrentMark* _cm; CMTask* _task; int _ref_counter_limit; int _ref_counter; public: ! G1CMKeepAliveAndDrainClosure(ConcurrentMark* cm, CMTask* task) : ! _cm(cm), _task(task), _ref_counter_limit(G1RefProcDrainInterval) { assert(_ref_counter_limit > 0, "sanity"); _ref_counter = _ref_counter_limit; } virtual void do_oop(narrowOop* p) { do_oop_work(p); } virtual void do_oop( oop* p) { do_oop_work(p); } --- 2201,2217 ---- class G1CMKeepAliveAndDrainClosure: public OopClosure { ConcurrentMark* _cm; CMTask* _task; int _ref_counter_limit; int _ref_counter; + bool _is_serial; public: ! G1CMKeepAliveAndDrainClosure(ConcurrentMark* cm, CMTask* task, bool is_serial): ! _cm(cm), _task(task), _is_serial(is_serial), ! _ref_counter_limit(G1RefProcDrainInterval) { assert(_ref_counter_limit > 0, "sanity"); + assert(!_is_serial || _task->worker_id() == 0, "only task 0 for serial code"); _ref_counter = _ref_counter_limit; } virtual void do_oop(narrowOop* p) { do_oop_work(p); } virtual void do_oop( oop* p) { do_oop_work(p); }
*** 2234,2244 **** // has_aborted() flag that the marking step has completed. do { double mark_step_duration_ms = G1ConcMarkStepDurationMillis; _task->do_marking_step(mark_step_duration_ms, false /* do_stealing */, ! false /* do_termination */); } while (_task->has_aborted() && !_cm->has_overflown()); _ref_counter = _ref_counter_limit; } } else { if (_cm->verbose_high()) { --- 2247,2258 ---- // has_aborted() flag that the marking step has completed. do { double mark_step_duration_ms = G1ConcMarkStepDurationMillis; _task->do_marking_step(mark_step_duration_ms, false /* do_stealing */, ! false /* do_termination */, ! _is_serial); } while (_task->has_aborted() && !_cm->has_overflown()); _ref_counter = _ref_counter_limit; } } else { if (_cm->verbose_high()) {
*** 2258,2286 **** class G1CMDrainMarkingStackClosure: public VoidClosure { ConcurrentMark* _cm; CMTask* _task; bool _do_stealing; bool _do_termination; public: ! G1CMDrainMarkingStackClosure(ConcurrentMark* cm, CMTask* task, bool is_par) : ! _cm(cm), _task(task) { ! assert(is_par || _task->worker_id() == 0, ! "Only task for worker 0 should be used if ref processing is single threaded"); ! // We only allow stealing and only enter the termination protocol ! // in CMTask::do_marking_step() if this closure is being instantiated ! // for parallel reference processing. ! _do_stealing = _do_termination = is_par; } void do_void() { do { if (_cm->verbose_high()) { gclog_or_tty->print_cr("\t[%u] Drain: Calling do_marking_step - " ! "stealing: %s, termination: %s", _task->worker_id(), BOOL_TO_STR(_do_stealing), ! BOOL_TO_STR(_do_termination)); } // We call CMTask::do_marking_step() to completely drain the local // and global marking stacks of entries pushed by the 'keep alive' // oop closure (an instance of G1CMKeepAliveAndDrainClosure above). --- 2272,2302 ---- class G1CMDrainMarkingStackClosure: public VoidClosure { ConcurrentMark* _cm; CMTask* _task; bool _do_stealing; bool _do_termination; + bool _is_serial; public: ! G1CMDrainMarkingStackClosure(ConcurrentMark* cm, CMTask* task, bool is_serial): ! _cm(cm), _task(task), _is_serial(is_serial) { ! assert(!_is_serial || _task->worker_id() == 0, "only task 0 for serial code"); ! ! // It only makes sense to allow stealing in CMTask::do_marking_step() if ! // this closure is being instantiated for parallel reference processing. ! _do_stealing = !_is_serial; ! _do_termination = true; } void do_void() { do { if (_cm->verbose_high()) { gclog_or_tty->print_cr("\t[%u] Drain: Calling do_marking_step - " ! "stealing: %s, termination: %s, serial: %s", _task->worker_id(), BOOL_TO_STR(_do_stealing), ! BOOL_TO_STR(_do_termination), ! BOOL_TO_STR(_is_serial)); } // We call CMTask::do_marking_step() to completely drain the local // and global marking stacks of entries pushed by the 'keep alive' // oop closure (an instance of G1CMKeepAliveAndDrainClosure above).
*** 2298,2308 **** // when CMTask::do_marking_step() returns without setting the // has_aborted() flag that the marking step has completed. _task->do_marking_step(1000000000.0 /* something very large */, _do_stealing, ! _do_termination); } while (_task->has_aborted() && !_cm->has_overflown()); } }; // Implementation of AbstractRefProcTaskExecutor for parallel --- 2314,2325 ---- // when CMTask::do_marking_step() returns without setting the // has_aborted() flag that the marking step has completed. _task->do_marking_step(1000000000.0 /* something very large */, _do_stealing, ! _do_termination, ! _is_serial); } while (_task->has_aborted() && !_cm->has_overflown()); } }; // Implementation of AbstractRefProcTaskExecutor for parallel
*** 2331,2357 **** class G1CMRefProcTaskProxy: public AbstractGangTask { typedef AbstractRefProcTaskExecutor::ProcessTask ProcessTask; ProcessTask& _proc_task; G1CollectedHeap* _g1h; ConcurrentMark* _cm; - bool _processing_is_mt; public: G1CMRefProcTaskProxy(ProcessTask& proc_task, G1CollectedHeap* g1h, ConcurrentMark* cm) : AbstractGangTask("Process reference objects in parallel"), _proc_task(proc_task), _g1h(g1h), _cm(cm) { ReferenceProcessor* rp = _g1h->ref_processor_cm(); ! _processing_is_mt = rp->processing_is_mt(); } virtual void work(uint worker_id) { CMTask* marking_task = _cm->task(worker_id); G1CMIsAliveClosure g1_is_alive(_g1h); ! G1CMKeepAliveAndDrainClosure g1_par_keep_alive(_cm, marking_task); ! G1CMDrainMarkingStackClosure g1_par_drain(_cm, marking_task, _processing_is_mt); _proc_task.work(worker_id, g1_is_alive, g1_par_keep_alive, g1_par_drain); } }; --- 2348,2377 ---- class G1CMRefProcTaskProxy: public AbstractGangTask { typedef AbstractRefProcTaskExecutor::ProcessTask ProcessTask; ProcessTask& _proc_task; G1CollectedHeap* _g1h; ConcurrentMark* _cm; public: G1CMRefProcTaskProxy(ProcessTask& proc_task, G1CollectedHeap* g1h, ConcurrentMark* cm) : AbstractGangTask("Process reference objects in parallel"), _proc_task(proc_task), _g1h(g1h), _cm(cm) { ReferenceProcessor* rp = _g1h->ref_processor_cm(); ! assert(rp->processing_is_mt(), "shouldn't be here otherwise"); } virtual void work(uint worker_id) { CMTask* marking_task = _cm->task(worker_id); G1CMIsAliveClosure g1_is_alive(_g1h); ! G1CMKeepAliveAndDrainClosure g1_par_keep_alive(_cm, ! marking_task, ! false /* is_serial */); ! G1CMDrainMarkingStackClosure g1_par_drain(_cm, ! marking_task, ! false /* is_serial */); _proc_task.work(worker_id, g1_is_alive, g1_par_keep_alive, g1_par_drain); } };
*** 2359,2371 **** assert(_workers != NULL, "Need parallel worker threads."); assert(_g1h->ref_processor_cm()->processing_is_mt(), "processing is not MT"); G1CMRefProcTaskProxy proc_task_proxy(proc_task, _g1h, _cm); ! // We need to reset the phase for each task execution so that ! // the termination protocol of CMTask::do_marking_step works. ! _cm->set_phase(_active_workers, false /* concurrent */); _g1h->set_par_threads(_active_workers); _workers->run_task(&proc_task_proxy); _g1h->set_par_threads(0); } --- 2379,2393 ---- assert(_workers != NULL, "Need parallel worker threads."); assert(_g1h->ref_processor_cm()->processing_is_mt(), "processing is not MT"); G1CMRefProcTaskProxy proc_task_proxy(proc_task, _g1h, _cm); ! // We need to reset the number of workers in the parallel ! // task terminator, before each proxy task execution, so ! // that the termination protocol in CMTask::do_marking_step() ! // knows how many workers to wait for. ! _cm->terminator()->reset_for_reuse(_active_workers); _g1h->set_par_threads(_active_workers); _workers->run_task(&proc_task_proxy); _g1h->set_par_threads(0); }
*** 2393,2402 **** --- 2415,2434 ---- _workers->run_task(&enq_task_proxy); _g1h->set_par_threads(0); } void ConcurrentMark::weakRefsWork(bool clear_all_soft_refs) { + if (has_overflown()) { + // Skip processing the discovered references if we have + // overflown the global marking stack. Reference objects + // only get discovered once so it is OK to not + // de-populate the discovered reference lists. We could have, + // but the only benefit would be that, when marking restarts, + // less reference objects are discovered. + return; + } + ResourceMark rm; HandleMark hm; G1CollectedHeap* g1h = G1CollectedHeap::heap();
*** 2418,2447 **** // Set the soft reference policy rp->setup_policy(clear_all_soft_refs); assert(_markStack.isEmpty(), "mark stack should be empty"); ! // Non-MT instances 'Keep Alive' and 'Complete GC' oop closures. ! G1CMKeepAliveAndDrainClosure g1_keep_alive(this, task(0)); ! G1CMDrainMarkingStackClosure g1_drain_mark_stack(this, task(0), false); ! ! // We need at least one active thread. If reference processing is ! // not multi-threaded we use the current (ConcurrentMarkThread) thread, ! // otherwise we use the work gang from the G1CollectedHeap and we ! // utilize all the worker threads we can. ! uint active_workers = (rp->processing_is_mt() && g1h->workers() != NULL ! ? g1h->workers()->active_workers() ! : 1U); ! active_workers = MAX2(MIN2(active_workers, _max_worker_id), 1U); G1CMRefProcTaskExecutor par_task_executor(g1h, this, g1h->workers(), active_workers); ! AbstractRefProcTaskExecutor* executor = (rp->processing_is_mt() ! ? &par_task_executor ! : NULL); // Set the degree of MT processing here. If the discovery was done MT, // the number of threads involved during discovery could differ from // the number of active workers. This is OK as long as the discovered // Reference lists are balanced (see balance_all_queues() and balance_queues()). --- 2450,2480 ---- // Set the soft reference policy rp->setup_policy(clear_all_soft_refs); assert(_markStack.isEmpty(), "mark stack should be empty"); ! // We need at least one active thread. If reference processing ! // is not multi-threaded we use the current (VMThread) thread, ! // otherwise we use the work gang from the G1CollectedHeap and ! // we utilize all the worker threads we can. ! bool processing_is_mt = rp->processing_is_mt() && g1h->workers() != NULL; ! uint active_workers = (processing_is_mt ? g1h->workers()->active_workers() : 1U); active_workers = MAX2(MIN2(active_workers, _max_worker_id), 1U); + // Serial (i.e executed by VMThread) instances of the 'Keep Alive' + // and 'Complete GC' closures. + G1CMKeepAliveAndDrainClosure g1_keep_alive(this, task(0), true /* is_serial */); + G1CMDrainMarkingStackClosure g1_drain_mark_stack(this, task(0), true /* is_serial */); + + // Parallel processing task executor. G1CMRefProcTaskExecutor par_task_executor(g1h, this, g1h->workers(), active_workers); + AbstractRefProcTaskExecutor* executor = (processing_is_mt ? &par_task_executor : NULL); ! // Tell the parallel task terminator and the worker barrier syncs how ! // many workers to expect. ! set_phase(active_workers, concurrent()); // Set the degree of MT processing here. If the discovery was done MT, // the number of threads involved during discovery could differ from // the number of active workers. This is OK as long as the discovered // Reference lists are balanced (see balance_all_queues() and balance_queues()).
*** 2457,2466 **** --- 2490,2500 ---- // oop closures will set the has_overflown flag if we overflow the // global marking stack. assert(_markStack.overflow() || _markStack.isEmpty(), "mark stack should be empty (unless it overflowed)"); + if (_markStack.overflow()) { // This should have been done already when we tried to push an // entry on to the global mark stack. But let's do it again. set_has_overflown(); }
*** 2485,2495 **** _nextMarkBitMap = (CMBitMap*) temp; } class CMRemarkTask: public AbstractGangTask { private: ! ConcurrentMark *_cm; public: void work(uint worker_id) { // Since all available tasks are actually started, we should // only proceed if we're supposed to be actived. --- 2519,2530 ---- _nextMarkBitMap = (CMBitMap*) temp; } class CMRemarkTask: public AbstractGangTask { private: ! ConcurrentMark* _cm; ! bool _is_serial; public: void work(uint worker_id) { // Since all available tasks are actually started, we should // only proceed if we're supposed to be actived.
*** 2497,2516 **** CMTask* task = _cm->task(worker_id); task->record_start_time(); do { task->do_marking_step(1000000000.0 /* something very large */, true /* do_stealing */, ! true /* do_termination */); } while (task->has_aborted() && !_cm->has_overflown()); // If we overflow, then we do not want to restart. We instead // want to abort remark and do concurrent marking again. task->record_end_time(); } } ! CMRemarkTask(ConcurrentMark* cm, int active_workers) : ! AbstractGangTask("Par Remark"), _cm(cm) { _cm->terminator()->reset_for_reuse(active_workers); } }; void ConcurrentMark::checkpointRootsFinalWork() { --- 2532,2552 ---- CMTask* task = _cm->task(worker_id); task->record_start_time(); do { task->do_marking_step(1000000000.0 /* something very large */, true /* do_stealing */, ! true /* do_termination */, ! _is_serial); } while (task->has_aborted() && !_cm->has_overflown()); // If we overflow, then we do not want to restart. We instead // want to abort remark and do concurrent marking again. task->record_end_time(); } } ! CMRemarkTask(ConcurrentMark* cm, int active_workers, bool is_serial) : ! AbstractGangTask("Par Remark"), _cm(cm), _is_serial(is_serial) { _cm->terminator()->reset_for_reuse(active_workers); } }; void ConcurrentMark::checkpointRootsFinalWork() {
*** 2533,2556 **** // Leave _parallel_marking_threads at it's // value originally calculated in the ConcurrentMark // constructor and pass values of the active workers // through the gang in the task. ! CMRemarkTask remarkTask(this, active_workers); g1h->set_par_threads(active_workers); g1h->workers()->run_task(&remarkTask); g1h->set_par_threads(0); } else { G1CollectedHeap::StrongRootsScope srs(g1h); - // this is remark, so we'll use up all available threads uint active_workers = 1; set_phase(active_workers, false /* concurrent */); ! CMRemarkTask remarkTask(this, active_workers); ! // We will start all available threads, even if we decide that the ! // active_workers will be fewer. The extra ones will just bail out ! // immediately. remarkTask.work(0); } SATBMarkQueueSet& satb_mq_set = JavaThread::satb_mark_queue_set(); guarantee(satb_mq_set.completed_buffers_num() == 0, "invariant"); --- 2569,2599 ---- // Leave _parallel_marking_threads at it's // value originally calculated in the ConcurrentMark // constructor and pass values of the active workers // through the gang in the task. ! CMRemarkTask remarkTask(this, active_workers, false /* is_serial */); ! ! // We will start all available threads, even if we decide that the ! // active_workers will be fewer. The extra ones will just bail out ! // immediately. g1h->set_par_threads(active_workers); g1h->workers()->run_task(&remarkTask); g1h->set_par_threads(0); } else { G1CollectedHeap::StrongRootsScope srs(g1h); uint active_workers = 1; set_phase(active_workers, false /* concurrent */); ! // Note - if there's no work gang then the VMThread will be ! // the thread to execute the remark - serially. We have ! // to pass true for the is_serial parameter so that ! // CMTask::do_marking_step() doesn't enter the synch ! // barriers in the event of an overflow. Doing so will ! // cause an assert that the current thread is not a ! // concurrent GC thread. ! CMRemarkTask remarkTask(this, active_workers, true /* is_serial*/); remarkTask.work(0); } SATBMarkQueueSet& satb_mq_set = JavaThread::satb_mark_queue_set(); guarantee(satb_mq_set.completed_buffers_num() == 0, "invariant");
*** 3868,3889 **** #endif // _MARKING_STATS_ } /***************************************************************************** ! The do_marking_step(time_target_ms) method is the building block ! of the parallel marking framework. It can be called in parallel with other invocations of do_marking_step() on different tasks (but only one per task, obviously) and concurrently with the mutator threads, or during remark, hence it eliminates the need for two versions of the code. When called during remark, it will pick up from where the task left off during the concurrent marking phase. Interestingly, tasks are also claimable during evacuation pauses too, since do_marking_step() ensures that it aborts before it needs to yield. ! The data structures that is uses to do marking work are the following: (1) Marking Bitmap. If there are gray objects that appear only on the bitmap (this happens either when dealing with an overflow or when the initial marking phase has simply marked the roots --- 3911,3932 ---- #endif // _MARKING_STATS_ } /***************************************************************************** ! The do_marking_step(time_target_ms, ...) method is the building ! block of the parallel marking framework. It can be called in parallel with other invocations of do_marking_step() on different tasks (but only one per task, obviously) and concurrently with the mutator threads, or during remark, hence it eliminates the need for two versions of the code. When called during remark, it will pick up from where the task left off during the concurrent marking phase. Interestingly, tasks are also claimable during evacuation pauses too, since do_marking_step() ensures that it aborts before it needs to yield. ! The data structures that it uses to do marking work are the following: (1) Marking Bitmap. If there are gray objects that appear only on the bitmap (this happens either when dealing with an overflow or when the initial marking phase has simply marked the roots
*** 3965,3979 **** too. The initial reason for the clock method was to avoid calling vtime too regularly, as it is quite expensive. So, once it was in place, it was natural to piggy-back all the other conditions on it too and not constantly check them throughout the code. *****************************************************************************/ void CMTask::do_marking_step(double time_target_ms, bool do_stealing, ! bool do_termination) { assert(time_target_ms >= 1.0, "minimum granularity is 1ms"); assert(concurrent() == _cm->concurrent(), "they should be the same"); G1CollectorPolicy* g1_policy = _g1h->g1_policy(); assert(_task_queues != NULL, "invariant"); --- 4008,4036 ---- too. The initial reason for the clock method was to avoid calling vtime too regularly, as it is quite expensive. So, once it was in place, it was natural to piggy-back all the other conditions on it too and not constantly check them throughout the code. + If do_stealing is true then do_marking_step will attempt to steal + work from the other CMTasks. It only makes sense to enable + stealing when being called by multiple threads. + + If do_termination is true then do_marking_step will enter its + termination protocol. + + The value of is_serial should be true when do_marking_step is + being called by the serial reference processing closures. + In this case the calling thread is the VM thread and do_marking_step + should skip any synchronization in the termination and overflow + code. + *****************************************************************************/ void CMTask::do_marking_step(double time_target_ms, bool do_stealing, ! bool do_termination, ! bool is_serial) { assert(time_target_ms >= 1.0, "minimum granularity is 1ms"); assert(concurrent() == _cm->concurrent(), "they should be the same"); G1CollectorPolicy* g1_policy = _g1h->g1_policy(); assert(_task_queues != NULL, "invariant");
*** 4251,4264 **** if (_cm->verbose_low()) { gclog_or_tty->print_cr("[%u] starting termination protocol", _worker_id); } _termination_start_time_ms = os::elapsedVTime() * 1000.0; // The CMTask class also extends the TerminatorTerminator class, // hence its should_exit_termination() method will also decide // whether to exit the termination protocol or not. ! bool finished = _cm->terminator()->offer_termination(this); double termination_end_time_ms = os::elapsedVTime() * 1000.0; _termination_time_ms += termination_end_time_ms - _termination_start_time_ms; if (finished) { --- 4308,4324 ---- if (_cm->verbose_low()) { gclog_or_tty->print_cr("[%u] starting termination protocol", _worker_id); } _termination_start_time_ms = os::elapsedVTime() * 1000.0; + // The CMTask class also extends the TerminatorTerminator class, // hence its should_exit_termination() method will also decide // whether to exit the termination protocol or not. ! bool finished = (is_serial ? true ! :_cm->terminator()->offer_termination(this)); ! double termination_end_time_ms = os::elapsedVTime() * 1000.0; _termination_time_ms += termination_end_time_ms - _termination_start_time_ms; if (finished) {
*** 4334,4357 **** if (_cm->verbose_low()) { gclog_or_tty->print_cr("[%u] detected overflow", _worker_id); } _cm->enter_first_sync_barrier(_worker_id); // When we exit this sync barrier we know that all tasks have // stopped doing marking work. So, it's now safe to // re-initialise our data structures. At the end of this method, // task 0 will clear the global data structures. statsOnly( ++_aborted_overflow ); // We clear the local state of this task... clear_region_fields(); // ...and enter the second barrier. _cm->enter_second_sync_barrier(_worker_id); ! // At this point everything has bee re-initialised and we're // ready to restart. } if (_cm->verbose_low()) { gclog_or_tty->print_cr("[%u] <<<<<<<<<< ABORTING, target = %1.2lfms, " --- 4394,4425 ---- if (_cm->verbose_low()) { gclog_or_tty->print_cr("[%u] detected overflow", _worker_id); } + if (!is_serial) { + // We only need to enter the sync barrier if being called + // from a parallel context _cm->enter_first_sync_barrier(_worker_id); + // When we exit this sync barrier we know that all tasks have // stopped doing marking work. So, it's now safe to // re-initialise our data structures. At the end of this method, // task 0 will clear the global data structures. + } statsOnly( ++_aborted_overflow ); // We clear the local state of this task... clear_region_fields(); + if (!is_serial) { // ...and enter the second barrier. _cm->enter_second_sync_barrier(_worker_id); ! } ! // At this point, if we're during the concurrent phase of ! // marking, everything has been re-initialised and we're // ready to restart. } if (_cm->verbose_low()) { gclog_or_tty->print_cr("[%u] <<<<<<<<<< ABORTING, target = %1.2lfms, "