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

Print this page
rev 4773 : 8005849: JEP 167: Event-Based JVM Tracing
Reviewed-by: acorn, coleenp, sla
Contributed-by: Karen Kinnear <karen.kinnear@oracle.com>, Bengt Rutisson <bengt.rutisson@oracle.com>, Calvin Cheung <calvin.cheung@oracle.com>, Erik Gahlin <erik.gahlin@oracle.com>, Erik Helin <erik.helin@oracle.com>, Jesper Wilhelmsson <jesper.wilhelmsson@oracle.com>, Keith McGuigan <keith.mcguigan@oracle.com>, Mattias Tobiasson <mattias.tobiasson@oracle.com>, Markus Gronlund <markus.gronlund@oracle.com>, Mikael Auno <mikael.auno@oracle.com>, Nils Eliasson <nils.eliasson@oracle.com>, Nils Loodin <nils.loodin@oracle.com>, Rickard Backman <rickard.backman@oracle.com>, Staffan Larsen <staffan.larsen@oracle.com>, Stefan Karlsson <stefan.karlsson@oracle.com>, Yekaterina Kantserova <yekaterina.kantserova@oracle.com>

@@ -36,14 +36,19 @@
 #include "gc_implementation/g1/g1GCPhaseTimes.hpp"
 #include "gc_implementation/g1/g1Log.hpp"
 #include "gc_implementation/g1/g1MarkSweep.hpp"
 #include "gc_implementation/g1/g1OopClosures.inline.hpp"
 #include "gc_implementation/g1/g1RemSet.inline.hpp"
+#include "gc_implementation/g1/g1YCTypes.hpp"
 #include "gc_implementation/g1/heapRegion.inline.hpp"
 #include "gc_implementation/g1/heapRegionRemSet.hpp"
 #include "gc_implementation/g1/heapRegionSeq.inline.hpp"
 #include "gc_implementation/g1/vm_operations_g1.hpp"
+#include "gc_implementation/shared/gcHeapSummary.hpp"
+#include "gc_implementation/shared/gcTimer.hpp"
+#include "gc_implementation/shared/gcTrace.hpp"
+#include "gc_implementation/shared/gcTraceTime.hpp"
 #include "gc_implementation/shared/isGCActiveMark.hpp"
 #include "memory/gcLocker.inline.hpp"
 #include "memory/genOopClosures.inline.hpp"
 #include "memory/generationSpec.hpp"
 #include "memory/referenceProcessor.hpp"

@@ -74,11 +79,11 @@
 //
 // G1ParVerifyTask uses heap_region_par_iterate_chunked() for parallelism.
 // The number of GC workers is passed to heap_region_par_iterate_chunked().
 // It does use run_task() which sets _n_workers in the task.
 // G1ParTask executes g1_process_strong_roots() ->
-// SharedHeap::process_strong_roots() which calls eventuall to
+// SharedHeap::process_strong_roots() which calls eventually to
 // CardTableModRefBS::par_non_clean_card_iterate_work() which uses
 // SequentialSubTasksDone.  SharedHeap::process_strong_roots() also
 // directly uses SubTasksDone (_process_strong_tasks field in SharedHeap).
 //
 

@@ -455,11 +460,11 @@
   return hr != NULL && hr->in_collection_set();
 }
 #endif
 
 // Returns true if the reference points to an object that
-// can move in an incremental collecction.
+// can move in an incremental collection.
 bool G1CollectedHeap::is_scavengable(const void* p) {
   G1CollectedHeap* g1h = G1CollectedHeap::heap();
   G1CollectorPolicy* g1p = g1h->g1_policy();
   HeapRegion* hr = heap_region_containing(p);
   if (hr == NULL) {

@@ -546,11 +551,11 @@
                                HR_FORMAT_PARAMS(res));
       }
       return res;
     }
 
-    // Wait here until we get notifed either when (a) there are no
+    // Wait here until we get notified either when (a) there are no
     // more free regions coming or (b) some regions have been moved on
     // the secondary_free_list.
     SecondaryFreeList_lock->wait(Mutex::_no_safepoint_check_flag);
   }
 

@@ -621,11 +626,11 @@
   assert(num_regions * HeapRegion::GrainWords >= word_size, "pre-condition");
 
   uint first = G1_NULL_HRS_INDEX;
   if (num_regions == 1) {
     // Only one region to allocate, no need to go through the slower
-    // path. The caller will attempt the expasion if this fails, so
+    // path. The caller will attempt the expansion if this fails, so
     // let's not try to expand here too.
     HeapRegion* hr = new_region(word_size, false /* do_expand */);
     if (hr != NULL) {
       first = hr->hrs_index();
     } else {

@@ -686,11 +691,11 @@
   HeapRegion* first_hr = region_at(first);
   // The header of the new object will be placed at the bottom of
   // the first region.
   HeapWord* new_obj = first_hr->bottom();
   // This will be the new end of the first region in the series that
-  // should also match the end of the last region in the seriers.
+  // should also match the end of the last region in the series.
   HeapWord* new_end = new_obj + word_size_sum;
   // This will be the new top of the first region that will reflect
   // this allocation.
   HeapWord* new_top = new_obj + word_size;
 

@@ -861,11 +866,11 @@
 HeapWord*
 G1CollectedHeap::mem_allocate(size_t word_size,
                               bool*  gc_overhead_limit_was_exceeded) {
   assert_heap_not_locked_and_not_at_safepoint();
 
-  // Loop until the allocation is satisified, or unsatisfied after GC.
+  // Loop until the allocation is satisfied, or unsatisfied after GC.
   for (int try_count = 1, gclocker_retry_count = 0; /* we'll return */; try_count += 1) {
     unsigned int gc_count_before;
 
     HeapWord* result = NULL;
     if (!isHumongous(word_size)) {

@@ -1001,11 +1006,11 @@
       // then retry the allocation.
       GC_locker::stall_until_clear();
       (*gclocker_retry_count_ret) += 1;
     }
 
-    // We can reach here if we were unsuccessul in scheduling a
+    // We can reach here if we were unsuccessful in scheduling a
     // collection (because another thread beat us to it) or if we were
     // stalled due to the GC locker. In either can we should retry the
     // allocation attempt in case another thread successfully
     // performed a collection and reclaimed enough space. We do the
     // first attempt (without holding the Heap_lock) here and the

@@ -1126,11 +1131,11 @@
       // then retry the allocation.
       GC_locker::stall_until_clear();
       (*gclocker_retry_count_ret) += 1;
     }
 
-    // We can reach here if we were unsuccessul in scheduling a
+    // We can reach here if we were unsuccessful in scheduling a
     // collection (because another thread beat us to it) or if we were
     // stalled due to the GC locker. In either can we should retry the
     // allocation attempt in case another thread successfully
     // performed a collection and reclaimed enough space.  Give a
     // warning if we seem to be looping forever.

@@ -1296,14 +1301,21 @@
 
   if (GC_locker::check_active_before_gc()) {
     return false;
   }
 
+  STWGCTimer* gc_timer = G1MarkSweep::gc_timer();
+  gc_timer->register_gc_start(os::elapsed_counter());
+
+  SerialOldTracer* gc_tracer = G1MarkSweep::gc_tracer();
+  gc_tracer->report_gc_start(gc_cause(), gc_timer->gc_start());
+
   SvcGCMarker sgcm(SvcGCMarker::FULL);
   ResourceMark rm;
 
   print_heap_before_gc();
+  trace_heap_before_gc(gc_tracer);
 
   size_t metadata_prev_used = MetaspaceAux::allocated_used_bytes();
 
   HRSPhaseSetter x(HRSPhaseFullGC);
   verify_region_sets_optional();

@@ -1320,11 +1332,11 @@
     assert(gc_cause() != GCCause::_java_lang_system_gc || explicit_gc, "invariant");
     gclog_or_tty->date_stamp(G1Log::fine() && PrintGCDateStamps);
     TraceCPUTime tcpu(G1Log::finer(), true, gclog_or_tty);
 
     {
-      TraceTime t(GCCauseString("Full GC", gc_cause()), G1Log::fine(), true, gclog_or_tty);
+      GCTraceTime t(GCCauseString("Full GC", gc_cause()), G1Log::fine(), true, NULL);
       TraceCollectorStats tcs(g1mm()->full_collection_counters());
       TraceMemoryManagerStats tms(true /* fullGC */, gc_cause());
 
       double start = os::elapsedTime();
       g1_policy()->record_full_collection_start();

@@ -1349,11 +1361,11 @@
 
       assert(used() == recalculate_used(), "Should be equal");
 
       verify_before_gc();
 
-      pre_full_gc_dump();
+      pre_full_gc_dump(gc_timer);
 
       COMPILER2_PRESENT(DerivedPointerTable::clear());
 
       // Disable discovery and empty the discovered lists
       // for the CM ref processor.

@@ -1433,11 +1445,11 @@
       assert(!ref_processor_cm()->discovery_enabled(), "Postcondition");
       ref_processor_cm()->verify_no_references_recorded();
 
       reset_gc_time_stamp();
       // Since everything potentially moved, we will clear all remembered
-      // sets, and clear all cards.  Later we will rebuild remebered
+      // sets, and clear all cards.  Later we will rebuild remembered
       // sets. We will also reset the GC time stamps of the regions.
       clear_rsets_post_compaction();
       check_gc_time_stamps();
 
       // Resize the heap if necessary.

@@ -1551,12 +1563,16 @@
     if (G1Log::finer()) {
       g1_policy()->print_detailed_heap_transition(true /* full */);
     }
 
     print_heap_after_gc();
+    trace_heap_after_gc(gc_tracer);
+
+    post_full_gc_dump(gc_timer);
 
-    post_full_gc_dump();
+    gc_timer->register_gc_end(os::elapsed_counter());
+    gc_tracer->report_gc_end(gc_timer->gc_end(), gc_timer->time_partitions());
   }
 
   return true;
 }
 

@@ -1917,11 +1933,11 @@
   _is_alive_closure_stw(this),
   _ref_processor_cm(NULL),
   _ref_processor_stw(NULL),
   _process_strong_tasks(new SubTasksDone(G1H_PS_NumElements)),
   _bot_shared(NULL),
-  _evac_failure_scan_stack(NULL) ,
+  _evac_failure_scan_stack(NULL),
   _mark_in_progress(false),
   _cg1r(NULL), _summary_bytes_used(0),
   _g1mm(NULL),
   _refine_cte_cl(NULL),
   _full_collection(false),

@@ -1937,16 +1953,22 @@
   _old_plab_stats(OldPLABSize, PLABWeight),
   _expand_heap_after_alloc_failure(true),
   _surviving_young_words(NULL),
   _old_marking_cycles_started(0),
   _old_marking_cycles_completed(0),
+  _concurrent_cycle_started(false),
   _in_cset_fast_test(NULL),
   _in_cset_fast_test_base(NULL),
   _dirty_cards_region_list(NULL),
   _worker_cset_start_region(NULL),
-  _worker_cset_start_region_time_stamp(NULL) {
-  _g1h = this; // To catch bugs.
+  _worker_cset_start_region_time_stamp(NULL),
+  _gc_timer_stw(new (ResourceObj::C_HEAP, mtGC) STWGCTimer()),
+  _gc_timer_cm(new (ResourceObj::C_HEAP, mtGC) ConcurrentGCTimer()),
+  _gc_tracer_stw(new (ResourceObj::C_HEAP, mtGC) G1NewTracer()),
+  _gc_tracer_cm(new (ResourceObj::C_HEAP, mtGC) G1OldTracer()) {
+
+  _g1h = this;
   if (_process_strong_tasks == NULL || !_process_strong_tasks->valid()) {
     vm_exit_during_initialization("Failed necessary allocation.");
   }
 
   _humongous_object_threshold_in_words = HeapRegion::GrainWords / 2;

@@ -1957,17 +1979,18 @@
   int n_rem_sets = HeapRegionRemSet::num_par_rem_sets();
   assert(n_rem_sets > 0, "Invariant.");
 
   _worker_cset_start_region = NEW_C_HEAP_ARRAY(HeapRegion*, n_queues, mtGC);
   _worker_cset_start_region_time_stamp = NEW_C_HEAP_ARRAY(unsigned int, n_queues, mtGC);
+  _evacuation_failed_info_array = NEW_C_HEAP_ARRAY(EvacuationFailedInfo, n_queues, mtGC);
 
   for (int i = 0; i < n_queues; i++) {
     RefToScanQueue* q = new RefToScanQueue();
     q->initialize();
     _task_queues->register_queue(i, q);
+    ::new (&_evacuation_failed_info_array[i]) EvacuationFailedInfo();
   }
-
   clear_cset_start_regions();
 
   // Initialize the G1EvacuationFailureALot counters and flags.
   NOT_PRODUCT(reset_evacuation_should_fail();)
 

@@ -2023,11 +2046,11 @@
 
   ReservedSpace heap_rs = Universe::reserve_heap(max_byte_size,
                                                  HeapRegion::GrainBytes);
 
   // It is important to do this in a way such that concurrent readers can't
-  // temporarily think somethings in the heap.  (I've actually seen this
+  // temporarily think something is in the heap.  (I've actually seen this
   // happen in asserts: DLD.)
   _reserved.set_word_size(0);
   _reserved.set_start((HeapWord*)heap_rs.base());
   _reserved.set_end((HeapWord*)(heap_rs.base() + heap_rs.size()));
 

@@ -2460,11 +2483,11 @@
   _old_marking_cycles_completed += 1;
 
   // We need to clear the "in_progress" flag in the CM thread before
   // we wake up any waiters (especially when ExplicitInvokesConcurrent
   // is set) so that if a waiter requests another System.gc() it doesn't
-  // incorrectly see that a marking cyle is still in progress.
+  // incorrectly see that a marking cycle is still in progress.
   if (concurrent) {
     _cmThread->clear_in_progress();
   }
 
   // This notify_all() will ensure that a thread that called

@@ -2472,10 +2495,53 @@
   // and it's waiting for a full GC to finish will be woken up. It is
   // waiting in VM_G1IncCollectionPause::doit_epilogue().
   FullGCCount_lock->notify_all();
 }
 
+void G1CollectedHeap::register_concurrent_cycle_start(jlong start_time) {
+  _concurrent_cycle_started = true;
+  _gc_timer_cm->register_gc_start(start_time);
+
+  _gc_tracer_cm->report_gc_start(gc_cause(), _gc_timer_cm->gc_start());
+  trace_heap_before_gc(_gc_tracer_cm);
+}
+
+void G1CollectedHeap::register_concurrent_cycle_end() {
+  if (_concurrent_cycle_started) {
+    _gc_timer_cm->register_gc_end(os::elapsed_counter());
+
+    if (_cm->has_aborted()) {
+      _gc_tracer_cm->report_concurrent_mode_failure();
+    }
+    _gc_tracer_cm->report_gc_end(_gc_timer_cm->gc_end(), _gc_timer_cm->time_partitions());
+
+    _concurrent_cycle_started = false;
+  }
+}
+
+void G1CollectedHeap::trace_heap_after_concurrent_cycle() {
+  if (_concurrent_cycle_started) {
+    trace_heap_after_gc(_gc_tracer_cm);
+  }
+}
+
+G1YCType G1CollectedHeap::yc_type() {
+  bool is_young = g1_policy()->gcs_are_young();
+  bool is_initial_mark = g1_policy()->during_initial_mark_pause();
+  bool is_during_mark = mark_in_progress();
+
+  if (is_initial_mark) {
+    return InitialMark;
+  } else if (is_during_mark) {
+    return DuringMark;
+  } else if (is_young) {
+    return Normal;
+  } else {
+    return Mixed;
+  }
+}
+
 void G1CollectedHeap::collect(GCCause::Cause cause) {
   assert_heap_not_locked();
 
   unsigned int gc_count_before;
   unsigned int old_marking_count_before;

@@ -2674,17 +2740,17 @@
           if (chr->claim_value() == claim_value ||
               !chr->continuesHumongous()) {
             break;
           }
 
-          // Noone should have claimed it directly. We can given
+          // No one should have claimed it directly. We can given
           // that we claimed its "starts humongous" region.
           assert(chr->claim_value() != claim_value, "sanity");
           assert(chr->humongous_start_region() == r, "sanity");
 
           if (chr->claimHeapRegion(claim_value)) {
-            // we should always be able to claim it; noone else should
+            // we should always be able to claim it; no one else should
             // be trying to claim this region
 
             bool res2 = cl->doHeapRegion(chr);
             assert(!res2, "Should not abort");
 

@@ -2974,11 +3040,11 @@
 size_t G1CollectedHeap::unsafe_max_tlab_alloc(Thread* ignored) const {
   // Return the remaining space in the cur alloc region, but not less than
   // the min TLAB size.
 
   // Also, this value can be at most the humongous object threshold,
-  // since we can't allow tlabs to grow big enough to accomodate
+  // since we can't allow tlabs to grow big enough to accommodate
   // humongous objects.
 
   HeapRegion* hr = _mutator_alloc_region.get();
   size_t max_tlab_size = _humongous_object_threshold_in_words * wordSize;
   if (hr == NULL) {

@@ -3733,14 +3799,19 @@
 
   if (GC_locker::check_active_before_gc()) {
     return false;
   }
 
+  _gc_timer_stw->register_gc_start(os::elapsed_counter());
+
+  _gc_tracer_stw->report_gc_start(gc_cause(), _gc_timer_stw->gc_start());
+
   SvcGCMarker sgcm(SvcGCMarker::MINOR);
   ResourceMark rm;
 
   print_heap_before_gc();
+  trace_heap_before_gc(_gc_tracer_stw);
 
   HRSPhaseSetter x(HRSPhaseEvacuation);
   verify_region_sets_optional();
   verify_dirty_young_regions();
 

@@ -3761,15 +3832,21 @@
   // the CM thread, the flag's value in the policy has been reset.
   bool should_start_conc_mark = g1_policy()->during_initial_mark_pause();
 
   // Inner scope for scope based logging, timers, and stats collection
   {
+    EvacuationInfo evacuation_info;
+
     if (g1_policy()->during_initial_mark_pause()) {
       // We are about to start a marking cycle, so we increment the
       // full collection counter.
       increment_old_marking_cycles_started();
+      register_concurrent_cycle_start(_gc_timer_stw->gc_start());
     }
+
+    _gc_tracer_stw->report_yc_type(yc_type());
+
     TraceCPUTime tcpu(G1Log::finer(), true, gclog_or_tty);
 
     int active_workers = (G1CollectedHeap::use_parallel_gc_threads() ?
                                 workers()->active_workers() : 1);
     double pause_start_sec = os::elapsedTime();

@@ -3875,11 +3952,11 @@
         gclog_or_tty->print_cr("\nBefore choosing collection set.\nYoung_list:");
         _young_list->print();
         g1_policy()->print_collection_set(g1_policy()->inc_cset_head(), gclog_or_tty);
 #endif // YOUNG_LIST_VERBOSE
 
-        g1_policy()->finalize_cset(target_pause_time_ms);
+        g1_policy()->finalize_cset(target_pause_time_ms, evacuation_info);
 
         _cm->note_start_of_gc();
         // We should not verify the per-thread SATB buffers given that
         // we have not filtered them yet (we'll do so during the
         // GC). We also call this after finalize_cset() to

@@ -3911,14 +3988,14 @@
 #endif // ASSERT
 
         setup_surviving_young_words();
 
         // Initialize the GC alloc regions.
-        init_gc_alloc_regions();
+        init_gc_alloc_regions(evacuation_info);
 
         // Actually do the work...
-        evacuate_collection_set();
+        evacuate_collection_set(evacuation_info);
 
         // We do this to mainly verify the per-thread SATB buffers
         // (which have been filtered by now) since we didn't verify
         // them earlier. No point in re-checking the stacks / enqueued
         // buffers given that the CSet has not changed since last time

@@ -3926,11 +4003,11 @@
         _cm->verify_no_cset_oops(false /* verify_stacks */,
                                  false /* verify_enqueued_buffers */,
                                  true  /* verify_thread_buffers */,
                                  true  /* verify_fingers */);
 
-        free_collection_set(g1_policy()->collection_set());
+        free_collection_set(g1_policy()->collection_set(), evacuation_info);
         g1_policy()->clear_collection_set();
 
         cleanup_surviving_young_words();
 
         // Start a new incremental collection set for the next pause.

@@ -3961,10 +4038,16 @@
 
         _young_list->reset_auxilary_lists();
 
         if (evacuation_failed()) {
           _summary_bytes_used = recalculate_used();
+          uint n_queues = MAX2((int)ParallelGCThreads, 1);
+          for (uint i = 0; i < n_queues; i++) {
+            if (_evacuation_failed_info_array[i].has_failed()) {
+              _gc_tracer_stw->report_evacuation_failed(_evacuation_failed_info_array[i]);
+            }
+          }
         } else {
           // The "used" of the the collection set have already been subtracted
           // when they were freed.  Add in the bytes evacuated.
           _summary_bytes_used += g1_policy()->bytes_copied_during_gc();
         }

@@ -4003,11 +4086,11 @@
               assert(max_capacity() == _g1_storage.reserved_size(), "reserved size mismatch");
             }
           }
         }
 
-        // We redo the verificaiton but now wrt to the new CSet which
+        // We redo the verification but now wrt to the new CSet which
         // has just got initialized after the previous CSet was freed.
         _cm->verify_no_cset_oops(true  /* verify_stacks */,
                                  true  /* verify_enqueued_buffers */,
                                  true  /* verify_thread_buffers */,
                                  true  /* verify_fingers */);

@@ -4016,11 +4099,11 @@
         // This timing is only used by the ergonomics to handle our pause target.
         // It is unclear why this should not include the full pause. We will
         // investigate this in CR 7178365.
         double sample_end_time_sec = os::elapsedTime();
         double pause_time_ms = (sample_end_time_sec - sample_start_time_sec) * MILLIUNITS;
-        g1_policy()->record_collection_pause_end(pause_time_ms);
+        g1_policy()->record_collection_pause_end(pause_time_ms, evacuation_info);
 
         MemoryService::track_memory_usage();
 
         // In prepare_for_verify() below we'll need to scan the deferred
         // update buffers to bring the RSets up-to-date if

@@ -4083,24 +4166,29 @@
 
     TASKQUEUE_STATS_ONLY(if (ParallelGCVerbose) print_taskqueue_stats());
     TASKQUEUE_STATS_ONLY(reset_taskqueue_stats());
 
     print_heap_after_gc();
+    trace_heap_after_gc(_gc_tracer_stw);
 
     // We must call G1MonitoringSupport::update_sizes() in the same scoping level
     // as an active TraceMemoryManagerStats object (i.e. before the destructor for the
     // TraceMemoryManagerStats is called) so that the G1 memory pools are updated
     // before any GC notifications are raised.
     g1mm()->update_sizes();
+
+    _gc_tracer_stw->report_evacuation_info(&evacuation_info);
+    _gc_tracer_stw->report_tenuring_threshold(_g1_policy->tenuring_threshold());
+    _gc_timer_stw->register_gc_end(os::elapsed_counter());
+    _gc_tracer_stw->report_gc_end(_gc_timer_stw->gc_end(), _gc_timer_stw->time_partitions());
   }
 
   if (G1SummarizeRSetStats &&
       (G1SummarizeRSetStatsPeriod > 0) &&
       (total_collections() % G1SummarizeRSetStatsPeriod == 0)) {
     g1_rem_set()->print_summary_info();
   }
-
   // It should now be safe to tell the concurrent mark thread to start
   // without its logging output interfering with the logging output
   // that came from the pause.
 
   if (should_start_conc_mark) {

@@ -4148,11 +4236,11 @@
 void G1CollectedHeap::release_mutator_alloc_region() {
   _mutator_alloc_region.release();
   assert(_mutator_alloc_region.get() == NULL, "post-condition");
 }
 
-void G1CollectedHeap::init_gc_alloc_regions() {
+void G1CollectedHeap::init_gc_alloc_regions(EvacuationInfo& evacuation_info) {
   assert_at_safepoint(true /* should_be_vm_thread */);
 
   _survivor_gc_alloc_region.init();
   _old_gc_alloc_region.init();
   HeapRegion* retained_region = _retained_old_gc_alloc_region;

@@ -4163,11 +4251,11 @@
   // b) it's already full (no point in using it),
   // c) it's empty (this means that it was emptied during
   // a cleanup and it should be on the free list now), or
   // d) it's humongous (this means that it was emptied
   // during a cleanup and was added to the free list, but
-  // has been subseqently used to allocate a humongous
+  // has been subsequently used to allocate a humongous
   // object that may be less than the region size).
   if (retained_region != NULL &&
       !retained_region->in_collection_set() &&
       !(retained_region->top() == retained_region->end()) &&
       !retained_region->is_empty() &&

@@ -4180,14 +4268,17 @@
     _old_set.remove(retained_region);
     bool during_im = g1_policy()->during_initial_mark_pause();
     retained_region->note_start_of_copying(during_im);
     _old_gc_alloc_region.set(retained_region);
     _hr_printer.reuse(retained_region);
+    evacuation_info.set_alloc_regions_used_before(retained_region->used());
   }
 }
 
-void G1CollectedHeap::release_gc_alloc_regions(uint no_of_gc_workers) {
+void G1CollectedHeap::release_gc_alloc_regions(uint no_of_gc_workers, EvacuationInfo& evacuation_info) {
+  evacuation_info.set_allocation_regions(_survivor_gc_alloc_region.count() +
+                                         _old_gc_alloc_region.count());
   _survivor_gc_alloc_region.release();
   // If we have an old GC alloc region to release, we'll save it in
   // _retained_old_gc_alloc_region. If we don't
   // _retained_old_gc_alloc_region will become NULL. This is what we
   // want either way so no reason to check explicitly for either

@@ -4266,20 +4357,25 @@
      obj->oop_iterate_backwards(_evac_failure_closure);
   }
 }
 
 oop
-G1CollectedHeap::handle_evacuation_failure_par(OopsInHeapRegionClosure* cl,
+G1CollectedHeap::handle_evacuation_failure_par(G1ParScanThreadState* _par_scan_state,
                                                oop old) {
   assert(obj_in_cs(old),
          err_msg("obj: "PTR_FORMAT" should still be in the CSet",
                  (HeapWord*) old));
   markOop m = old->mark();
   oop forward_ptr = old->forward_to_atomic(old);
   if (forward_ptr == NULL) {
     // Forward-to-self succeeded.
+    assert(_par_scan_state != NULL, "par scan state");
+    OopsInHeapRegionClosure* cl = _par_scan_state->evac_failure_closure();
+    uint queue_num = _par_scan_state->queue_num();
 
+    _evacuation_failed = true;
+    _evacuation_failed_info_array[queue_num].register_copy_failure(old->size());
     if (_evac_failure_closure != cl) {
       MutexLockerEx x(EvacFailureStack_lock, Mutex::_no_safepoint_check_flag);
       assert(!_drain_in_progress,
              "Should only be true while someone holds the lock.");
       // Set the global evac-failure closure to the current thread's.

@@ -4306,12 +4402,10 @@
     return forward_ptr;
   }
 }
 
 void G1CollectedHeap::handle_evacuation_failure_common(oop old, markOop m) {
-  set_evacuation_failed(true);
-
   preserve_mark_if_necessary(old, m);
 
   HeapRegion* r = heap_region_containing(old);
   if (!r->evacuation_failed()) {
     r->set_evacuation_failed(true);

@@ -4557,12 +4651,11 @@
 #endif // !PRODUCT
 
   if (obj_ptr == NULL) {
     // This will either forward-to-self, or detect that someone else has
     // installed a forwarding pointer.
-    OopsInHeapRegionClosure* cl = _par_scan_state->evac_failure_closure();
-    return _g1->handle_evacuation_failure_par(cl, old);
+    return _g1->handle_evacuation_failure_par(_par_scan_state, old);
   }
 
   oop obj = oop(obj_ptr);
 
   // We're going to allocate linearly, so might as well prefetch ahead.

@@ -5162,11 +5255,11 @@
       // on the PSS queue. When the queue is drained (after each
       // phase of reference processing) the object and it's followers
       // will be copied, the reference field set to point to the
       // new location, and the RSet updated. Otherwise we need to
       // use the the non-heap or metadata closures directly to copy
-      // the refernt object and update the pointer, while avoiding
+      // the referent object and update the pointer, while avoiding
       // updating the RSet.
 
       if (_g1h->is_in_g1_reserved(p)) {
         _par_scan_state->push_on_queue(p);
       } else {

@@ -5330,11 +5423,11 @@
   virtual void work(uint worker_id) {
     _enq_task.work(worker_id);
   }
 };
 
-// Driver routine for parallel reference enqueing.
+// Driver routine for parallel reference enqueueing.
 // Creates an instance of the ref enqueueing gang
 // task and has the worker threads execute it.
 
 void G1STWRefProcTaskExecutor::execute(EnqueueTask& enq_task) {
   assert(_workers != NULL, "Need parallel worker threads.");

@@ -5459,11 +5552,11 @@
   // But some of the referents, that are in the collection set, that these
   // reference objects point to may not have been copied: the STW ref
   // processor would have seen that the reference object had already
   // been 'discovered' and would have skipped discovering the reference,
   // but would not have treated the reference object as a regular oop.
-  // As a reult the copy closure would not have been applied to the
+  // As a result the copy closure would not have been applied to the
   // referent object.
   //
   // We need to explicitly copy these referent objects - the references
   // will be processed at the end of remarking.
   //

@@ -5535,25 +5628,32 @@
   G1STWDrainQueueClosure drain_queue(this, &pss);
 
   // Setup the soft refs policy...
   rp->setup_policy(false);
 
+  ReferenceProcessorStats stats;
   if (!rp->processing_is_mt()) {
     // Serial reference processing...
-    rp->process_discovered_references(&is_alive,
+    stats = rp->process_discovered_references(&is_alive,
                                       &keep_alive,
                                       &drain_queue,
-                                      NULL);
+                                              NULL,
+                                              _gc_timer_stw);
   } else {
     // Parallel reference processing
     assert(rp->num_q() == no_of_gc_workers, "sanity");
     assert(no_of_gc_workers <= rp->max_num_q(), "sanity");
 
     G1STWRefProcTaskExecutor par_task_executor(this, workers(), _task_queues, no_of_gc_workers);
-    rp->process_discovered_references(&is_alive, &keep_alive, &drain_queue, &par_task_executor);
+    stats = rp->process_discovered_references(&is_alive,
+                                              &keep_alive,
+                                              &drain_queue,
+                                              &par_task_executor,
+                                              _gc_timer_stw);
   }
 
+  _gc_tracer_stw->report_gc_reference_stats(stats);
   // We have completed copying any necessary live referent objects
   // (that were not copied during the actual pause) so we can
   // retire any active alloc buffers
   pss.retire_alloc_buffers();
   assert(pss.refs()->is_empty(), "both queue and overflow should be empty");

@@ -5573,11 +5673,11 @@
   // the pending list.
   if (!rp->processing_is_mt()) {
     // Serial reference processing...
     rp->enqueue_discovered_references();
   } else {
-    // Parallel reference enqueuing
+    // Parallel reference enqueueing
 
     assert(no_of_gc_workers == workers()->active_workers(),
            "Need to reset active workers");
     assert(rp->num_q() == no_of_gc_workers, "sanity");
     assert(no_of_gc_workers <= rp->max_num_q(), "sanity");

@@ -5590,19 +5690,19 @@
   assert(!rp->discovery_enabled(), "should have been disabled");
 
   // FIXME
   // CM's reference processing also cleans up the string and symbol tables.
   // Should we do that here also? We could, but it is a serial operation
-  // and could signicantly increase the pause time.
+  // and could significantly increase the pause time.
 
   double ref_enq_time = os::elapsedTime() - ref_enq_start;
   g1_policy()->phase_times()->record_ref_enq_time(ref_enq_time * 1000.0);
 }
 
-void G1CollectedHeap::evacuate_collection_set() {
+void G1CollectedHeap::evacuate_collection_set(EvacuationInfo& evacuation_info) {
   _expand_heap_after_alloc_failure = true;
-  set_evacuation_failed(false);
+  _evacuation_failed = false;
 
   // Should G1EvacuationFailureALot be in effect for this GC?
   NOT_PRODUCT(set_evacuation_failure_alot_for_current_gc();)
 
   g1_rem_set()->prepare_for_oops_into_collection_set_do();

@@ -5687,11 +5787,11 @@
     G1STWIsAliveClosure is_alive(this);
     G1KeepAliveClosure keep_alive(this);
     JNIHandles::weak_oops_do(&is_alive, &keep_alive);
   }
 
-  release_gc_alloc_regions(n_workers);
+  release_gc_alloc_regions(n_workers, evacuation_info);
   g1_rem_set()->cleanup_after_oops_into_collection_set_do();
 
   // Reset and re-enable the hot card cache.
   // Note the counts for the cards in the regions in the
   // collection set are reset when the collection set is freed.

@@ -5710,11 +5810,11 @@
   }
 
   // Enqueue any remaining references remaining on the STW
   // reference processor's discovered lists. We need to do
   // this after the card table is cleaned (and verified) as
-  // the act of enqueuing entries on to the pending list
+  // the act of enqueueing entries on to the pending list
   // will log these updates (and dirty their associated
   // cards). We need these updates logged to update any
   // RSets.
   enqueue_discovered_references(n_workers);
 

@@ -5938,11 +6038,11 @@
 
   double elapsed = os::elapsedTime() - start;
   g1_policy()->phase_times()->record_clear_ct_time(elapsed * 1000.0);
 }
 
-void G1CollectedHeap::free_collection_set(HeapRegion* cs_head) {
+void G1CollectedHeap::free_collection_set(HeapRegion* cs_head, EvacuationInfo& evacuation_info) {
   size_t pre_used = 0;
   FreeRegionList local_free_list("Local List for CSet Freeing");
 
   double young_time_ms     = 0.0;
   double non_young_time_ms = 0.0;

@@ -6024,14 +6124,16 @@
       }
       cur->set_not_young();
       cur->set_evacuation_failed(false);
       // The region is now considered to be old.
       _old_set.add(cur);
+      evacuation_info.increment_collectionset_used_after(cur->used());
     }
     cur = next;
   }
 
+  evacuation_info.set_regions_freed(local_free_list.length());
   policy->record_max_rs_lengths(rs_lengths);
   policy->cset_regions_freed();
 
   double end_sec = os::elapsedTime();
   double elapsed_ms = (end_sec - start_sec) * 1000.0;