< prev index next >

src/hotspot/share/gc/shenandoah/shenandoahConcurrentGC.cpp

Print this page
*** 1,7 ***
--- 1,8 ---
  /*
   * Copyright (c) 2021, 2022, Red Hat, Inc. All rights reserved.
+  * Copyright Amazon.com Inc. 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.

*** 29,10 ***
--- 30,14 ---
  #include "gc/shared/continuationGCSupport.inline.hpp"
  #include "gc/shenandoah/shenandoahBreakpoint.hpp"
  #include "gc/shenandoah/shenandoahCollectorPolicy.hpp"
  #include "gc/shenandoah/shenandoahConcurrentGC.hpp"
  #include "gc/shenandoah/shenandoahFreeSet.hpp"
+ #include "gc/shenandoah/shenandoahGeneration.hpp"
+ #include "gc/shenandoah/shenandoahGenerationalHeap.hpp"
+ #include "gc/shenandoah/shenandoahOldGeneration.hpp"
+ #include "gc/shenandoah/shenandoahYoungGeneration.hpp"
  #include "gc/shenandoah/shenandoahLock.hpp"
  #include "gc/shenandoah/shenandoahMark.inline.hpp"
  #include "gc/shenandoah/shenandoahMonitoringSupport.hpp"
  #include "gc/shenandoah/shenandoahOopClosures.inline.hpp"
  #include "gc/shenandoah/shenandoahPhaseTimings.hpp"

*** 83,39 ***
        ShenandoahBreakpoint::at_before_marking_completed();
      }
    }
  };
  
! ShenandoahConcurrentGC::ShenandoahConcurrentGC() :
!   _mark(),
    _degen_point(ShenandoahDegenPoint::_degenerated_unset),
!   _abbreviated(false) {
  }
  
  ShenandoahGC::ShenandoahDegenPoint ShenandoahConcurrentGC::degen_point() const {
    return _degen_point;
  }
  
- void ShenandoahConcurrentGC::cancel() {
-   ShenandoahConcurrentMark::cancel();
- }
- 
  bool ShenandoahConcurrentGC::collect(GCCause::Cause cause) {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
    ShenandoahBreakpointGCScope breakpoint_gc_scope(cause);
  
    // Reset for upcoming marking
    entry_reset();
  
    // Start initial mark under STW
    vmop_entry_init_mark();
  
    {
      ShenandoahBreakpointMarkScope breakpoint_mark_scope(cause);
      // Concurrent mark roots
      entry_mark_roots();
!     if (check_cancellation_and_abort(ShenandoahDegenPoint::_degenerated_outside_cycle)) {
        return false;
      }
  
      // Continue concurrent mark
      entry_mark();
--- 88,48 ---
        ShenandoahBreakpoint::at_before_marking_completed();
      }
    }
  };
  
! ShenandoahConcurrentGC::ShenandoahConcurrentGC(ShenandoahGeneration* generation, bool do_old_gc_bootstrap) :
!   _mark(generation),
    _degen_point(ShenandoahDegenPoint::_degenerated_unset),
!   _abbreviated(false),
+   _do_old_gc_bootstrap(do_old_gc_bootstrap),
+   _generation(generation) {
  }
  
  ShenandoahGC::ShenandoahDegenPoint ShenandoahConcurrentGC::degen_point() const {
    return _degen_point;
  }
  
  bool ShenandoahConcurrentGC::collect(GCCause::Cause cause) {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
+ 
    ShenandoahBreakpointGCScope breakpoint_gc_scope(cause);
  
    // Reset for upcoming marking
    entry_reset();
  
    // Start initial mark under STW
    vmop_entry_init_mark();
  
    {
      ShenandoahBreakpointMarkScope breakpoint_mark_scope(cause);
+ 
+     // Reset task queue stats here, rather than in mark_concurrent_roots,
+     // because remembered set scan will `push` oops into the queues and
+     // resetting after this happens will lose those counts.
+     TASKQUEUE_STATS_ONLY(_mark.task_queues()->reset_taskqueue_stats());
+ 
+     // Concurrent remembered set scanning
+     entry_scan_remembered_set();
+     // TODO: When RS scanning yields, we will need a check_cancellation_and_abort() degeneration point here.
+ 
      // Concurrent mark roots
      entry_mark_roots();
!     if (check_cancellation_and_abort(ShenandoahDegenPoint::_degenerated_roots)) {
        return false;
      }
  
      // Continue concurrent mark
      entry_mark();

*** 125,10 ***
--- 139,23 ---
    }
  
    // Complete marking under STW, and start evacuation
    vmop_entry_final_mark();
  
+   // If GC was cancelled before final mark, then the safepoint operation will do nothing
+   // and the concurrent mark will still be in progress. In this case it is safe to resume
+   // the degenerated cycle from the marking phase. On the other hand, if the GC is cancelled
+   // after final mark (but before this check), then the final mark safepoint operation
+   // will have finished the mark (setting concurrent mark in progress to false). Final mark
+   // will also have setup state (in concurrent stack processing) that will not be safe to
+   // resume from the marking phase in the degenerated cycle. That is, if the cancellation
+   // occurred after final mark, we must resume the degenerated cycle after the marking phase.
+   if (_generation->is_concurrent_mark_in_progress() && check_cancellation_and_abort(ShenandoahDegenPoint::_degenerated_mark)) {
+     assert(!heap->is_concurrent_weak_root_in_progress(), "Weak roots should not be in progress when concurrent mark is in progress");
+     return false;
+   }
+ 
    // Concurrent stack processing
    if (heap->is_evacuation_in_progress()) {
      entry_thread_roots();
    }
  

*** 137,14 ***
      entry_weak_refs();
      entry_weak_roots();
    }
  
    // Final mark might have reclaimed some immediate garbage, kick cleanup to reclaim
!   // the space. This would be the last action if there is nothing to evacuate.
    entry_cleanup_early();
  
    {
      ShenandoahHeapLocker locker(heap->lock());
      heap->free_set()->log_status();
    }
  
    // Perform concurrent class unloading
--- 164,17 ---
      entry_weak_refs();
      entry_weak_roots();
    }
  
    // Final mark might have reclaimed some immediate garbage, kick cleanup to reclaim
!   // the space. This would be the last action if there is nothing to evacuate.  Note that
+   // we will not age young-gen objects in the case that we skip evacuation.
    entry_cleanup_early();
  
    {
+     // TODO: Not sure there is value in logging free-set status right here.  Note that whenever the free set is rebuilt,
+     // it logs the newly rebuilt status.
      ShenandoahHeapLocker locker(heap->lock());
      heap->free_set()->log_status();
    }
  
    // Perform concurrent class unloading

*** 167,11 ***
--- 197,13 ---
      // Concurrently evacuate
      entry_evacuate();
      if (check_cancellation_and_abort(ShenandoahDegenPoint::_degenerated_evac)) {
        return false;
      }
+   }
  
+   if (heap->has_forwarded_objects()) {
      // Perform update-refs phase.
      vmop_entry_init_updaterefs();
      entry_updaterefs();
      if (check_cancellation_and_abort(ShenandoahDegenPoint::_degenerated_updaterefs)) {
        return false;

*** 186,14 ***
--- 218,37 ---
      vmop_entry_final_updaterefs();
  
      // Update references freed up collection set, kick the cleanup to reclaim the space.
      entry_cleanup_complete();
    } else {
+     // We chose not to evacuate because we found sufficient immediate garbage. Note that we
+     // do not check for cancellation here because, at this point, the cycle is effectively
+     // complete. If the cycle has been cancelled here, the control thread will detect it
+     // on its next iteration and run a degenerated young cycle.
      vmop_entry_final_roots();
      _abbreviated = true;
    }
  
+   // We defer generation resizing actions until after cset regions have been recycled.  We do this even following an
+   // abbreviated cycle.
+   if (heap->mode()->is_generational()) {
+ 
+     ShenandoahGenerationalHeap::TransferResult result;
+     {
+       ShenandoahGenerationalHeap* gen_heap = ShenandoahGenerationalHeap::heap();
+       ShenandoahHeapLocker locker(gen_heap->lock());
+ 
+       result = gen_heap->balance_generations();
+       gen_heap->reset_generation_reserves();
+     }
+ 
+     LogTarget(Info, gc, ergo) lt;
+     if (lt.is_enabled()) {
+       LogStream ls(lt);
+       result.print_on("Concurrent GC", &ls);
+     }
+   }
    return true;
  }
  
  void ShenandoahConcurrentGC::vmop_entry_init_mark() {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();

*** 299,21 ***
    op_final_roots();
  }
  
  void ShenandoahConcurrentGC::entry_reset() {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
    TraceCollectorStats tcs(heap->monitoring_support()->concurrent_collection_counters());
!   static const char* msg = "Concurrent reset";
!   ShenandoahConcurrentPhase gc_phase(msg, ShenandoahPhaseTimings::conc_reset);
!   EventMark em("%s", msg);
  
!   ShenandoahWorkerScope scope(heap->workers(),
!                               ShenandoahWorkerPolicy::calc_workers_for_conc_reset(),
!                               "concurrent reset");
  
!   heap->try_inject_alloc_failure();
!   op_reset();
  }
  
  void ShenandoahConcurrentGC::entry_mark_roots() {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
    TraceCollectorStats tcs(heap->monitoring_support()->concurrent_collection_counters());
--- 354,51 ---
    op_final_roots();
  }
  
  void ShenandoahConcurrentGC::entry_reset() {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
+   heap->try_inject_alloc_failure();
+ 
    TraceCollectorStats tcs(heap->monitoring_support()->concurrent_collection_counters());
!   {
!     static const char* msg = "Concurrent reset";
!     ShenandoahConcurrentPhase gc_phase(msg, ShenandoahPhaseTimings::conc_reset);
+     EventMark em("%s", msg);
+ 
+     ShenandoahWorkerScope scope(heap->workers(),
+                                 ShenandoahWorkerPolicy::calc_workers_for_conc_reset(),
+                                 msg);
+     op_reset();
+   }
  
!   if (_do_old_gc_bootstrap) {
!     static const char* msg = "Concurrent reset (OLD)";
!     ShenandoahConcurrentPhase gc_phase(msg, ShenandoahPhaseTimings::conc_reset_old);
+     ShenandoahWorkerScope scope(ShenandoahHeap::heap()->workers(),
+                                 ShenandoahWorkerPolicy::calc_workers_for_conc_reset(),
+                                 msg);
+     EventMark em("%s", msg);
  
!     heap->old_generation()->prepare_gc();
!   }
+ }
+ 
+ void ShenandoahConcurrentGC::entry_scan_remembered_set() {
+   if (_generation->is_young()) {
+     ShenandoahHeap* const heap = ShenandoahHeap::heap();
+     TraceCollectorStats tcs(heap->monitoring_support()->concurrent_collection_counters());
+     const char* msg = "Concurrent remembered set scanning";
+     ShenandoahConcurrentPhase gc_phase(msg, ShenandoahPhaseTimings::init_scan_rset);
+     EventMark em("%s", msg);
+ 
+     ShenandoahWorkerScope scope(heap->workers(),
+                                 ShenandoahWorkerPolicy::calc_workers_for_rs_scanning(),
+                                 msg);
+ 
+     heap->try_inject_alloc_failure();
+     _generation->scan_remembered_set(true /* is_concurrent */);
+   }
  }
  
  void ShenandoahConcurrentGC::entry_mark_roots() {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
    TraceCollectorStats tcs(heap->monitoring_support()->concurrent_collection_counters());

*** 490,12 ***
  void ShenandoahConcurrentGC::op_reset() {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
    if (ShenandoahPacing) {
      heap->pacer()->setup_for_reset();
    }
! 
-   heap->prepare_gc();
  }
  
  class ShenandoahInitMarkUpdateRegionStateClosure : public ShenandoahHeapRegionClosure {
  private:
    ShenandoahMarkingContext* const _ctx;
--- 575,11 ---
  void ShenandoahConcurrentGC::op_reset() {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
    if (ShenandoahPacing) {
      heap->pacer()->setup_for_reset();
    }
!   _generation->prepare_gc();
  }
  
  class ShenandoahInitMarkUpdateRegionStateClosure : public ShenandoahHeapRegionClosure {
  private:
    ShenandoahMarkingContext* const _ctx;

*** 504,11 ***
  
    void heap_region_do(ShenandoahHeapRegion* r) {
      assert(!r->has_live(), "Region " SIZE_FORMAT " should have no live data", r->index());
      if (r->is_active()) {
        // Check if region needs updating its TAMS. We have updated it already during concurrent
!       // reset, so it is very likely we don't need to do another write here.
        if (_ctx->top_at_mark_start(r) != r->top()) {
          _ctx->capture_top_at_mark_start(r);
        }
      } else {
        assert(_ctx->top_at_mark_start(r) == r->top(),
--- 588,12 ---
  
    void heap_region_do(ShenandoahHeapRegion* r) {
      assert(!r->has_live(), "Region " SIZE_FORMAT " should have no live data", r->index());
      if (r->is_active()) {
        // Check if region needs updating its TAMS. We have updated it already during concurrent
!       // reset, so it is very likely we don't need to do another write here.  Since most regions
+       // are not "active", this path is relatively rare.
        if (_ctx->top_at_mark_start(r) != r->top()) {
          _ctx->capture_top_at_mark_start(r);
        }
      } else {
        assert(_ctx->top_at_mark_start(r) == r->top(),

*** 526,34 ***
  void ShenandoahConcurrentGC::op_init_mark() {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
    assert(ShenandoahSafepoint::is_at_shenandoah_safepoint(), "Should be at safepoint");
    assert(Thread::current()->is_VM_thread(), "can only do this in VMThread");
  
!   assert(heap->marking_context()->is_bitmap_clear(), "need clear marking bitmap");
!   assert(!heap->marking_context()->is_complete(), "should not be complete");
    assert(!heap->has_forwarded_objects(), "No forwarded objects on this path");
  
    if (ShenandoahVerify) {
      heap->verifier()->verify_before_concmark();
    }
  
    if (VerifyBeforeGC) {
      Universe::verify();
    }
  
!   heap->set_concurrent_mark_in_progress(true);
  
    start_mark();
  
!   {
      ShenandoahGCPhase phase(ShenandoahPhaseTimings::init_update_region_states);
      ShenandoahInitMarkUpdateRegionStateClosure cl;
      heap->parallel_heap_region_iterate(&cl);
    }
  
    // Weak reference processing
!   ShenandoahReferenceProcessor* rp = heap->ref_processor();
    rp->reset_thread_locals();
    rp->set_soft_reference_policy(heap->soft_ref_policy()->should_clear_all_soft_refs());
  
    // Make above changes visible to worker threads
    OrderAccess::fence();
--- 611,67 ---
  void ShenandoahConcurrentGC::op_init_mark() {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
    assert(ShenandoahSafepoint::is_at_shenandoah_safepoint(), "Should be at safepoint");
    assert(Thread::current()->is_VM_thread(), "can only do this in VMThread");
  
!   assert(_generation->is_bitmap_clear(), "need clear marking bitmap");
!   assert(!_generation->is_mark_complete(), "should not be complete");
    assert(!heap->has_forwarded_objects(), "No forwarded objects on this path");
  
+ 
+   if (heap->mode()->is_generational()) {
+     if (_generation->is_young() || (_generation->is_global() && ShenandoahVerify)) {
+       // The current implementation of swap_remembered_set() copies the write-card-table
+       // to the read-card-table. The remembered sets are also swapped for GLOBAL collections
+       // so that the verifier works with the correct copy of the card table when verifying.
+       // TODO: This path should not really depend on ShenandoahVerify.
+       ShenandoahGCPhase phase(ShenandoahPhaseTimings::init_swap_rset);
+       _generation->swap_remembered_set();
+     }
+ 
+     if (_generation->is_global()) {
+       heap->cancel_old_gc();
+     } else if (heap->is_concurrent_old_mark_in_progress()) {
+       // Purge the SATB buffers, transferring any valid, old pointers to the
+       // old generation mark queue. Any pointers in a young region will be
+       // abandoned.
+       ShenandoahGCPhase phase(ShenandoahPhaseTimings::init_transfer_satb);
+       heap->transfer_old_pointers_from_satb();
+     }
+   }
+ 
    if (ShenandoahVerify) {
      heap->verifier()->verify_before_concmark();
    }
  
    if (VerifyBeforeGC) {
      Universe::verify();
    }
  
!   _generation->set_concurrent_mark_in_progress(true);
  
    start_mark();
  
!   if (_do_old_gc_bootstrap) {
+     // Update region state for both young and old regions
+     // TODO: We should be able to pull this out of the safepoint for the bootstrap
+     // cycle. The top of an old region will only move when a GC cycle evacuates
+     // objects into it. When we start an old cycle, we know that nothing can touch
+     // the top of old regions.
      ShenandoahGCPhase phase(ShenandoahPhaseTimings::init_update_region_states);
      ShenandoahInitMarkUpdateRegionStateClosure cl;
      heap->parallel_heap_region_iterate(&cl);
+     heap->old_generation()->ref_processor()->reset_thread_locals();
+   } else {
+     // Update region state for only young regions
+     ShenandoahGCPhase phase(ShenandoahPhaseTimings::init_update_region_states);
+     ShenandoahInitMarkUpdateRegionStateClosure cl;
+     _generation->parallel_heap_region_iterate(&cl);
    }
  
    // Weak reference processing
!   ShenandoahReferenceProcessor* rp = _generation->ref_processor();
    rp->reset_thread_locals();
    rp->set_soft_reference_policy(heap->soft_ref_policy()->should_clear_all_soft_refs());
  
    // Make above changes visible to worker threads
    OrderAccess::fence();

*** 589,44 ***
      assert(!heap->cancelled_gc(), "STW mark cannot OOM");
  
      // Notify JVMTI that the tagmap table will need cleaning.
      JvmtiTagMap::set_needs_cleaning();
  
!     heap->prepare_regions_and_collection_set(true /*concurrent*/);
  
      // Has to be done after cset selection
      heap->prepare_concurrent_roots();
  
!     if (!heap->collection_set()->is_empty()) {
!       if (ShenandoahVerify) {
!         heap->verifier()->verify_before_evacuation();
!       }
! 
!       heap->set_evacuation_in_progress(true);
!       // From here on, we need to update references.
!       heap->set_has_forwarded_objects(true);
! 
!       // Verify before arming for concurrent processing.
!       // Otherwise, verification can trigger stack processing.
!       if (ShenandoahVerify) {
!         heap->verifier()->verify_during_evacuation();
!       }
! 
!       // Arm nmethods/stack for concurrent processing
!       ShenandoahCodeRoots::arm_nmethods_for_evac();
!       ShenandoahStackWatermark::change_epoch_id();
! 
!       if (ShenandoahPacing) {
!         heap->pacer()->setup_for_evac();
        }
      } else {
!       if (ShenandoahVerify) {
!         heap->verifier()->verify_after_concmark();
!       }
! 
!       if (VerifyAfterGC) {
!         Universe::verify();
        }
      }
    }
  }
  
--- 707,130 ---
      assert(!heap->cancelled_gc(), "STW mark cannot OOM");
  
      // Notify JVMTI that the tagmap table will need cleaning.
      JvmtiTagMap::set_needs_cleaning();
  
!     // The collection set is chosen by prepare_regions_and_collection_set().
+     //
+     // TODO: Under severe memory overload conditions that can be checked here, we may want to limit
+     // the inclusion of old-gen candidates within the collection set.  This would allow us to prioritize efforts on
+     // evacuating young-gen,  This remediation is most appropriate when old-gen availability is very high (so there
+     // are negligible negative impacts from delaying completion of old-gen evacuation) and when young-gen collections
+     // are "under duress" (as signalled by very low availability of memory within young-gen, indicating that/ young-gen
+     // collections are not triggering frequently enough).
+     _generation->prepare_regions_and_collection_set(true /*concurrent*/);
+ 
+     // Upon return from prepare_regions_and_collection_set(), certain parameters have been established to govern the
+     // evacuation efforts that are about to begin.  In particular:
+     //
+     // heap->get_promoted_reserve() represents the amount of memory within old-gen's available memory that has
+     //   been set aside to hold objects promoted from young-gen memory.  This represents an estimated percentage
+     //   of the live young-gen memory within the collection set.  If there is more data ready to be promoted than
+     //   can fit within this reserve, the promotion of some objects will be deferred until a subsequent evacuation
+     //   pass.
+     //
+     // heap->get_old_evac_reserve() represents the amount of memory within old-gen's available memory that has been
+     //  set aside to hold objects evacuated from the old-gen collection set.
+     //
+     // heap->get_young_evac_reserve() represents the amount of memory within young-gen's available memory that has
+     //  been set aside to hold objects evacuated from the young-gen collection set.  Conservatively, this value
+     //  equals the entire amount of live young-gen memory within the collection set, even though some of this memory
+     //  will likely be promoted.
  
      // Has to be done after cset selection
      heap->prepare_concurrent_roots();
  
!     if (heap->mode()->is_generational()) {
!       if (!heap->collection_set()->is_empty() || heap->old_generation()->has_in_place_promotions()) {
!         // Even if the collection set is empty, we need to do evacuation if there are regions to be promoted in place.
!         // Concurrent evacuation takes responsibility for registering objects and setting the remembered set cards to dirty.
! 
!         LogTarget(Debug, gc, cset) lt;
!         if (lt.is_enabled()) {
!           ResourceMark rm;
!           LogStream ls(lt);
!           heap->collection_set()->print_on(&ls);
!         }
! 
!         if (ShenandoahVerify) {
!           heap->verifier()->verify_before_evacuation();
!         }
! 
!         heap->set_evacuation_in_progress(true);
! 
!         // Verify before arming for concurrent processing.
!         // Otherwise, verification can trigger stack processing.
!         if (ShenandoahVerify) {
+           heap->verifier()->verify_during_evacuation();
+         }
+ 
+         // Generational mode may promote objects in place during the evacuation phase.
+         // If that is the only reason we are evacuating, we don't need to update references
+         // and there will be no forwarded objects on the heap.
+         heap->set_has_forwarded_objects(!heap->collection_set()->is_empty());
+ 
+         // Arm nmethods/stack for concurrent processing
+         if (!heap->collection_set()->is_empty()) {
+           // Iff objects will be evaluated, arm the nmethod barriers. These will be disarmed
+           // under the same condition (established in prepare_concurrent_roots) after strong
+           // root evacuation has completed (see op_strong_roots).
+           ShenandoahCodeRoots::arm_nmethods_for_evac();
+           ShenandoahStackWatermark::change_epoch_id();
+         }
+ 
+         if (ShenandoahPacing) {
+           heap->pacer()->setup_for_evac();
+         }
+       } else {
+         if (ShenandoahVerify) {
+           heap->verifier()->verify_after_concmark();
+         }
+ 
+         if (VerifyAfterGC) {
+           Universe::verify();
+         }
        }
      } else {
!       // Not is_generational()
!       if (!heap->collection_set()->is_empty()) {
!         LogTarget(Debug, gc, ergo) lt;
!         if (lt.is_enabled()) {
!           ResourceMark rm;
!           LogStream ls(lt);
+           heap->collection_set()->print_on(&ls);
+         }
+ 
+         if (ShenandoahVerify) {
+           heap->verifier()->verify_before_evacuation();
+         }
+ 
+         heap->set_evacuation_in_progress(true);
+ 
+         // Verify before arming for concurrent processing.
+         // Otherwise, verification can trigger stack processing.
+         if (ShenandoahVerify) {
+           heap->verifier()->verify_during_evacuation();
+         }
+ 
+         // From here on, we need to update references.
+         heap->set_has_forwarded_objects(true);
+ 
+         // Arm nmethods/stack for concurrent processing
+         ShenandoahCodeRoots::arm_nmethods_for_evac();
+         ShenandoahStackWatermark::change_epoch_id();
+ 
+         if (ShenandoahPacing) {
+           heap->pacer()->setup_for_evac();
+         }
+       } else {
+         if (ShenandoahVerify) {
+           heap->verifier()->verify_after_concmark();
+         }
+ 
+         if (VerifyAfterGC) {
+           Universe::verify();
+         }
        }
      }
    }
  }
  

*** 644,10 ***
--- 848,11 ---
  }
  
  void ShenandoahConcurrentEvacThreadClosure::do_thread(Thread* thread) {
    JavaThread* const jt = JavaThread::cast(thread);
    StackWatermarkSet::finish_processing(jt, _oops, StackWatermarkKind::gc);
+   ShenandoahThreadLocalData::enable_plab_promotions(thread);
  }
  
  class ShenandoahConcurrentEvacUpdateThreadTask : public WorkerTask {
  private:
    ShenandoahJavaThreadsIterator _java_threads;

*** 657,10 ***
--- 862,13 ---
      WorkerTask("Shenandoah Evacuate/Update Concurrent Thread Roots"),
      _java_threads(ShenandoahPhaseTimings::conc_thread_roots, n_workers) {
    }
  
    void work(uint worker_id) {
+     Thread* worker_thread = Thread::current();
+     ShenandoahThreadLocalData::enable_plab_promotions(worker_thread);
+ 
      // ShenandoahEvacOOMScope has to be setup by ShenandoahContextEvacuateUpdateRootsClosure.
      // Otherwise, may deadlock with watermark lock
      ShenandoahContextEvacuateUpdateRootsClosure oops_cl;
      ShenandoahConcurrentEvacThreadClosure thr_cl(&oops_cl);
      _java_threads.threads_do(&thr_cl, worker_id);

*** 681,11 ***
    // Concurrent weak refs processing
    ShenandoahGCWorkerPhase worker_phase(ShenandoahPhaseTimings::conc_weak_refs);
    if (heap->gc_cause() == GCCause::_wb_breakpoint) {
      ShenandoahBreakpoint::at_after_reference_processing_started();
    }
!   heap->ref_processor()->process_references(ShenandoahPhaseTimings::conc_weak_refs, heap->workers(), true /* concurrent */);
  }
  
  class ShenandoahEvacUpdateCleanupOopStorageRootsClosure : public BasicOopIterateClosure {
  private:
    ShenandoahHeap* const _heap;
--- 889,11 ---
    // Concurrent weak refs processing
    ShenandoahGCWorkerPhase worker_phase(ShenandoahPhaseTimings::conc_weak_refs);
    if (heap->gc_cause() == GCCause::_wb_breakpoint) {
      ShenandoahBreakpoint::at_after_reference_processing_started();
    }
!   _generation->ref_processor()->process_references(ShenandoahPhaseTimings::conc_weak_refs, heap->workers(), true /* concurrent */);
  }
  
  class ShenandoahEvacUpdateCleanupOopStorageRootsClosure : public BasicOopIterateClosure {
  private:
    ShenandoahHeap* const _heap;

*** 708,12 ***
  
  void ShenandoahEvacUpdateCleanupOopStorageRootsClosure::do_oop(oop* p) {
    const oop obj = RawAccess<>::oop_load(p);
    if (!CompressedOops::is_null(obj)) {
      if (!_mark_context->is_marked(obj)) {
!       shenandoah_assert_correct(p, obj);
!       ShenandoahHeap::atomic_clear_oop(p, obj);
      } else if (_evac_in_progress && _heap->in_collection_set(obj)) {
        oop resolved = ShenandoahBarrierSet::resolve_forwarded_not_null(obj);
        if (resolved == obj) {
          resolved = _heap->evacuate_object(obj, _thread);
        }
--- 916,19 ---
  
  void ShenandoahEvacUpdateCleanupOopStorageRootsClosure::do_oop(oop* p) {
    const oop obj = RawAccess<>::oop_load(p);
    if (!CompressedOops::is_null(obj)) {
      if (!_mark_context->is_marked(obj)) {
!       if (_heap->is_in_active_generation(obj)) {
!         // TODO: This worries me. Here we are asserting that an unmarked from-space object is 'correct'.
+         // Normally, I would call this a bogus assert, but there seems to be a legitimate use-case for
+         // accessing from-space objects during class unloading. However, the from-space object may have
+         // been "filled". We've made no effort to prevent old generation classes being unloaded by young
+         // gen (and vice-versa).
+         shenandoah_assert_correct(p, obj);
+         ShenandoahHeap::atomic_clear_oop(p, obj);
+       }
      } else if (_evac_in_progress && _heap->in_collection_set(obj)) {
        oop resolved = ShenandoahBarrierSet::resolve_forwarded_not_null(obj);
        if (resolved == obj) {
          resolved = _heap->evacuate_object(obj, _thread);
        }

*** 933,11 ***
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
    heap->set_evacuation_in_progress(false);
    heap->set_concurrent_weak_root_in_progress(false);
    heap->prepare_update_heap_references(true /*concurrent*/);
    heap->set_update_refs_in_progress(true);
! 
    if (ShenandoahPacing) {
      heap->pacer()->setup_for_updaterefs();
    }
  }
  
--- 1148,13 ---
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
    heap->set_evacuation_in_progress(false);
    heap->set_concurrent_weak_root_in_progress(false);
    heap->prepare_update_heap_references(true /*concurrent*/);
    heap->set_update_refs_in_progress(true);
!   if (ShenandoahVerify) {
+     heap->verifier()->verify_before_updaterefs();
+   }
    if (ShenandoahPacing) {
      heap->pacer()->setup_for_updaterefs();
    }
  }
  

*** 978,23 ***
    heap->finish_concurrent_roots();
  
    // Clear cancelled GC, if set. On cancellation path, the block before would handle
    // everything.
    if (heap->cancelled_gc()) {
!     heap->clear_cancelled_gc();
    }
  
    // Has to be done before cset is clear
    if (ShenandoahVerify) {
      heap->verifier()->verify_roots_in_to_space();
    }
  
    heap->update_heap_region_states(true /*concurrent*/);
  
    heap->set_update_refs_in_progress(false);
    heap->set_has_forwarded_objects(false);
  
    if (ShenandoahVerify) {
      heap->verifier()->verify_after_updaterefs();
    }
  
    if (VerifyAfterGC) {
--- 1195,45 ---
    heap->finish_concurrent_roots();
  
    // Clear cancelled GC, if set. On cancellation path, the block before would handle
    // everything.
    if (heap->cancelled_gc()) {
!     heap->clear_cancelled_gc(true /* clear oom handler */);
    }
  
    // Has to be done before cset is clear
    if (ShenandoahVerify) {
      heap->verifier()->verify_roots_in_to_space();
    }
  
+   if (heap->mode()->is_generational() && heap->is_concurrent_old_mark_in_progress()) {
+     // When the SATB barrier is left on to support concurrent old gen mark, it may pick up writes to
+     // objects in the collection set. After those objects are evacuated, the pointers in the
+     // SATB are no longer safe. Once we have finished update references, we are guaranteed that
+     // no more writes to the collection set are possible.
+     //
+     // This will transfer any old pointers in _active_ regions from the SATB to the old gen
+     // mark queues. All other pointers will be discarded. This would also discard any pointers
+     // in old regions that were included in a mixed evacuation. We aren't using the SATB filter
+     // methods here because we cannot control when they execute. If the SATB filter runs _after_
+     // a region has been recycled, we will not be able to detect the bad pointer.
+     //
+     // We are not concerned about skipping this step in abbreviated cycles because regions
+     // with no live objects cannot have been written to and so cannot have entries in the SATB
+     // buffers.
+     heap->transfer_old_pointers_from_satb();
+   }
+ 
    heap->update_heap_region_states(true /*concurrent*/);
  
    heap->set_update_refs_in_progress(false);
    heap->set_has_forwarded_objects(false);
  
+   // Aging_cycle is only relevant during evacuation cycle for individual objects and during final mark for
+   // entire regions.  Both of these relevant operations occur before final update refs.
+   heap->set_aging_cycle(false);
+ 
    if (ShenandoahVerify) {
      heap->verifier()->verify_after_updaterefs();
    }
  
    if (VerifyAfterGC) {

*** 1003,11 ***
  
    heap->rebuild_free_set(true /*concurrent*/);
  }
  
  void ShenandoahConcurrentGC::op_final_roots() {
!   ShenandoahHeap::heap()->set_concurrent_weak_root_in_progress(false);
  }
  
  void ShenandoahConcurrentGC::op_cleanup_complete() {
    ShenandoahHeap::heap()->free_set()->recycle_trash();
  }
--- 1242,37 ---
  
    heap->rebuild_free_set(true /*concurrent*/);
  }
  
  void ShenandoahConcurrentGC::op_final_roots() {
! 
+   ShenandoahHeap *heap = ShenandoahHeap::heap();
+   heap->set_concurrent_weak_root_in_progress(false);
+   heap->set_evacuation_in_progress(false);
+ 
+   if (heap->mode()->is_generational()) {
+     // If the cycle was shortened for having enough immediate garbage, this could be
+     // the last GC safepoint before concurrent marking of old resumes. We must be sure
+     // that old mark threads don't see any pointers to garbage in the SATB buffers.
+     if (heap->is_concurrent_old_mark_in_progress()) {
+       heap->transfer_old_pointers_from_satb();
+     }
+ 
+     ShenandoahMarkingContext *ctx = heap->complete_marking_context();
+     for (size_t i = 0; i < heap->num_regions(); i++) {
+       ShenandoahHeapRegion *r = heap->get_region(i);
+       if (r->is_active() && r->is_young()) {
+         HeapWord* tams = ctx->top_at_mark_start(r);
+         HeapWord* top = r->top();
+         if (top > tams) {
+           r->reset_age();
+         } else if (heap->is_aging_cycle()) {
+           r->increment_age();
+         }
+       }
+     }
+   }
  }
  
  void ShenandoahConcurrentGC::op_cleanup_complete() {
    ShenandoahHeap::heap()->free_set()->recycle_trash();
  }

*** 1022,30 ***
  
  const char* ShenandoahConcurrentGC::init_mark_event_message() const {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
    assert(!heap->has_forwarded_objects(), "Should not have forwarded objects here");
    if (heap->unload_classes()) {
!     return "Pause Init Mark (unload classes)";
    } else {
!     return "Pause Init Mark";
    }
  }
  
  const char* ShenandoahConcurrentGC::final_mark_event_message() const {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
!   assert(!heap->has_forwarded_objects(), "Should not have forwarded objects here");
    if (heap->unload_classes()) {
!     return "Pause Final Mark (unload classes)";
    } else {
!     return "Pause Final Mark";
    }
  }
  
  const char* ShenandoahConcurrentGC::conc_mark_event_message() const {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
!   assert(!heap->has_forwarded_objects(), "Should not have forwarded objects here");
    if (heap->unload_classes()) {
!     return "Concurrent marking (unload classes)";
    } else {
!     return "Concurrent marking";
    }
  }
--- 1287,33 ---
  
  const char* ShenandoahConcurrentGC::init_mark_event_message() const {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
    assert(!heap->has_forwarded_objects(), "Should not have forwarded objects here");
    if (heap->unload_classes()) {
!     SHENANDOAH_RETURN_EVENT_MESSAGE(_generation->type(), "Pause Init Mark", " (unload classes)");
    } else {
!     SHENANDOAH_RETURN_EVENT_MESSAGE(_generation->type(), "Pause Init Mark", "");
    }
  }
  
  const char* ShenandoahConcurrentGC::final_mark_event_message() const {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
!   assert(!heap->has_forwarded_objects() || heap->is_concurrent_old_mark_in_progress(),
+          "Should not have forwarded objects during final mark, unless old gen concurrent mark is running");
+ 
    if (heap->unload_classes()) {
!     SHENANDOAH_RETURN_EVENT_MESSAGE(_generation->type(), "Pause Final Mark", " (unload classes)");
    } else {
!     SHENANDOAH_RETURN_EVENT_MESSAGE(_generation->type(), "Pause Final Mark", "");
    }
  }
  
  const char* ShenandoahConcurrentGC::conc_mark_event_message() const {
    ShenandoahHeap* const heap = ShenandoahHeap::heap();
!   assert(!heap->has_forwarded_objects() || heap->is_concurrent_old_mark_in_progress(),
+          "Should not have forwarded objects concurrent mark, unless old gen concurrent mark is running");
    if (heap->unload_classes()) {
!     SHENANDOAH_RETURN_EVENT_MESSAGE(_generation->type(), "Concurrent marking", " (unload classes)");
    } else {
!     SHENANDOAH_RETURN_EVENT_MESSAGE(_generation->type(), "Concurrent marking", "");
    }
  }
< prev index next >