changeset 2035:0fa27f37d4d4

6977804: G1: remove the zero-filling thread Summary: This changeset removes the zero-filling thread from G1 and collapses the two free region lists we had before (the "free" and "unclean" lists) into one. The new free list uses the new heap region sets / lists abstractions that we'll ultimately use it to keep track of all regions in the heap. A heap region set was also introduced for the humongous regions. Finally, this change increases the concurrency between the thread that completes freeing regions (after a cleanup pause) and the rest of the system (before we'd have to wait for said thread to complete before allocating a new region). The changest also includes a lot of refactoring and code simplification. Reviewed-by: jcoomes, johnc
author tonyp
date Wed, 19 Jan 2011 19:30:42 -0500
parents cb913d743d09
children 377371490991
files src/share/vm/gc_implementation/g1/concurrentMark.cpp src/share/vm/gc_implementation/g1/concurrentMark.hpp src/share/vm/gc_implementation/g1/concurrentMarkThread.cpp src/share/vm/gc_implementation/g1/concurrentZFThread.cpp src/share/vm/gc_implementation/g1/concurrentZFThread.hpp src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp src/share/vm/gc_implementation/g1/g1CollectedHeap.hpp src/share/vm/gc_implementation/g1/g1CollectedHeap.inline.hpp src/share/vm/gc_implementation/g1/g1CollectorPolicy.cpp src/share/vm/gc_implementation/g1/g1MarkSweep.cpp src/share/vm/gc_implementation/g1/g1_globals.hpp src/share/vm/gc_implementation/g1/heapRegion.cpp src/share/vm/gc_implementation/g1/heapRegion.hpp src/share/vm/gc_implementation/g1/heapRegionSeq.cpp src/share/vm/gc_implementation/g1/heapRegionSeq.hpp src/share/vm/gc_implementation/g1/heapRegionSet.cpp src/share/vm/gc_implementation/g1/heapRegionSet.hpp src/share/vm/gc_implementation/g1/heapRegionSet.inline.hpp src/share/vm/gc_implementation/g1/heapRegionSets.cpp src/share/vm/gc_implementation/g1/heapRegionSets.hpp src/share/vm/runtime/mutexLocker.cpp src/share/vm/runtime/mutexLocker.hpp src/share/vm/utilities/debug.hpp src/share/vm/utilities/globalDefinitions.hpp
diffstat 24 files changed, 2192 insertions(+), 1924 deletions(-) [+]
line wrap: on
line diff
--- a/src/share/vm/gc_implementation/g1/concurrentMark.cpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/concurrentMark.cpp	Wed Jan 19 19:30:42 2011 -0500
@@ -458,6 +458,7 @@
   _marking_task_overhead(1.0),
   _cleanup_sleep_factor(0.0),
   _cleanup_task_overhead(1.0),
+  _cleanup_list("Cleanup List"),
   _region_bm(max_regions, false /* in_resource_area*/),
   _card_bm((rs.size() + CardTableModRefBS::card_size - 1) >>
            CardTableModRefBS::card_shift,
@@ -521,12 +522,6 @@
   SATBMarkQueueSet& satb_qs = JavaThread::satb_mark_queue_set();
   satb_qs.set_buffer_size(G1SATBBufferSize);
 
-  int size = (int) MAX2(ParallelGCThreads, (size_t)1);
-  _par_cleanup_thread_state = NEW_C_HEAP_ARRAY(ParCleanupThreadState*, size);
-  for (int i = 0 ; i < size; i++) {
-    _par_cleanup_thread_state[i] = new ParCleanupThreadState;
-  }
-
   _tasks = NEW_C_HEAP_ARRAY(CMTask*, _max_task_num);
   _accum_task_vtime = NEW_C_HEAP_ARRAY(double, _max_task_num);
 
@@ -711,11 +706,6 @@
 }
 
 ConcurrentMark::~ConcurrentMark() {
-  int size = (int) MAX2(ParallelGCThreads, (size_t)1);
-  for (int i = 0; i < size; i++) delete _par_cleanup_thread_state[i];
-  FREE_C_HEAP_ARRAY(ParCleanupThreadState*,
-                    _par_cleanup_thread_state);
-
   for (int i = 0; i < (int) _max_task_num; ++i) {
     delete _task_queues->queue(i);
     delete _tasks[i];
@@ -1510,21 +1500,20 @@
   size_t _max_live_bytes;
   size_t _regions_claimed;
   size_t _freed_bytes;
-  size_t _cleared_h_regions;
-  size_t _freed_regions;
-  UncleanRegionList* _unclean_region_list;
+  FreeRegionList _local_cleanup_list;
+  HumongousRegionSet _humongous_proxy_set;
   double _claimed_region_time;
   double _max_region_time;
 
 public:
   G1NoteEndOfConcMarkClosure(G1CollectedHeap* g1,
-                             UncleanRegionList* list,
                              int worker_num);
   size_t freed_bytes() { return _freed_bytes; }
-  size_t cleared_h_regions() { return _cleared_h_regions; }
-  size_t freed_regions() { return  _freed_regions; }
-  UncleanRegionList* unclean_region_list() {
-    return _unclean_region_list;
+  FreeRegionList* local_cleanup_list() {
+    return &_local_cleanup_list;
+  }
+  HumongousRegionSet* humongous_proxy_set() {
+    return &_humongous_proxy_set;
   }
 
   bool doHeapRegion(HeapRegion *r);
@@ -1537,25 +1526,22 @@
 
 class G1ParNoteEndTask: public AbstractGangTask {
   friend class G1NoteEndOfConcMarkClosure;
+
 protected:
   G1CollectedHeap* _g1h;
   size_t _max_live_bytes;
   size_t _freed_bytes;
-  ConcurrentMark::ParCleanupThreadState** _par_cleanup_thread_state;
+  FreeRegionList* _cleanup_list;
+
 public:
   G1ParNoteEndTask(G1CollectedHeap* g1h,
-                   ConcurrentMark::ParCleanupThreadState**
-                   par_cleanup_thread_state) :
+                   FreeRegionList* cleanup_list) :
     AbstractGangTask("G1 note end"), _g1h(g1h),
-    _max_live_bytes(0), _freed_bytes(0),
-    _par_cleanup_thread_state(par_cleanup_thread_state)
-  {}
+    _max_live_bytes(0), _freed_bytes(0), _cleanup_list(cleanup_list) { }
 
   void work(int i) {
     double start = os::elapsedTime();
-    G1NoteEndOfConcMarkClosure g1_note_end(_g1h,
-                                           &_par_cleanup_thread_state[i]->list,
-                                           i);
+    G1NoteEndOfConcMarkClosure g1_note_end(_g1h, i);
     if (G1CollectedHeap::use_parallel_gc_threads()) {
       _g1h->heap_region_par_iterate_chunked(&g1_note_end, i,
                                             HeapRegion::NoteEndClaimValue);
@@ -1564,14 +1550,18 @@
     }
     assert(g1_note_end.complete(), "Shouldn't have yielded!");
 
-    // Now finish up freeing the current thread's regions.
-    _g1h->finish_free_region_work(g1_note_end.freed_bytes(),
-                                  g1_note_end.cleared_h_regions(),
-                                  0, NULL);
+    // Now update the lists
+    _g1h->update_sets_after_freeing_regions(g1_note_end.freed_bytes(),
+                                            NULL /* free_list */,
+                                            g1_note_end.humongous_proxy_set(),
+                                            true /* par */);
     {
       MutexLockerEx x(ParGCRareEvent_lock, Mutex::_no_safepoint_check_flag);
       _max_live_bytes += g1_note_end.max_live_bytes();
       _freed_bytes += g1_note_end.freed_bytes();
+
+      _cleanup_list->add_as_tail(g1_note_end.local_cleanup_list());
+      assert(g1_note_end.local_cleanup_list()->is_empty(), "post-condition");
     }
     double end = os::elapsedTime();
     if (G1PrintParCleanupStats) {
@@ -1612,30 +1602,28 @@
 
 G1NoteEndOfConcMarkClosure::
 G1NoteEndOfConcMarkClosure(G1CollectedHeap* g1,
-                           UncleanRegionList* list,
                            int worker_num)
   : _g1(g1), _worker_num(worker_num),
     _max_live_bytes(0), _regions_claimed(0),
-    _freed_bytes(0), _cleared_h_regions(0), _freed_regions(0),
+    _freed_bytes(0),
     _claimed_region_time(0.0), _max_region_time(0.0),
-    _unclean_region_list(list)
-{}
-
-bool G1NoteEndOfConcMarkClosure::doHeapRegion(HeapRegion *r) {
+    _local_cleanup_list("Local Cleanup List"),
+    _humongous_proxy_set("Local Cleanup Humongous Proxy Set") { }
+
+bool G1NoteEndOfConcMarkClosure::doHeapRegion(HeapRegion *hr) {
   // We use a claim value of zero here because all regions
   // were claimed with value 1 in the FinalCount task.
-  r->reset_gc_time_stamp();
-  if (!r->continuesHumongous()) {
+  hr->reset_gc_time_stamp();
+  if (!hr->continuesHumongous()) {
     double start = os::elapsedTime();
     _regions_claimed++;
-    r->note_end_of_marking();
-    _max_live_bytes += r->max_live_bytes();
-    _g1->free_region_if_totally_empty_work(r,
-                                           _freed_bytes,
-                                           _cleared_h_regions,
-                                           _freed_regions,
-                                           _unclean_region_list,
-                                           true /*par*/);
+    hr->note_end_of_marking();
+    _max_live_bytes += hr->max_live_bytes();
+    _g1->free_region_if_totally_empty(hr,
+                                      &_freed_bytes,
+                                      &_local_cleanup_list,
+                                      &_humongous_proxy_set,
+                                      true /* par */);
     double region_time = (os::elapsedTime() - start);
     _claimed_region_time += region_time;
     if (region_time > _max_region_time) _max_region_time = region_time;
@@ -1655,6 +1643,8 @@
     return;
   }
 
+  g1h->verify_region_sets_optional();
+
   if (VerifyDuringGC) {
     HandleMark hm;  // handle scope
     gclog_or_tty->print(" VerifyDuringGC:(before)");
@@ -1719,7 +1709,7 @@
 
   // Note end of marking in all heap regions.
   double note_end_start = os::elapsedTime();
-  G1ParNoteEndTask g1_par_note_end_task(g1h, _par_cleanup_thread_state);
+  G1ParNoteEndTask g1_par_note_end_task(g1h, &_cleanup_list);
   if (G1CollectedHeap::use_parallel_gc_threads()) {
     int n_workers = g1h->workers()->total_workers();
     g1h->set_par_threads(n_workers);
@@ -1731,9 +1721,14 @@
   } else {
     g1_par_note_end_task.work(0);
   }
-  g1h->set_unclean_regions_coming(true);
+
+  if (!cleanup_list_is_empty()) {
+    // The cleanup list is not empty, so we'll have to process it
+    // concurrently. Notify anyone else that might be wanting free
+    // regions that there will be more free regions coming soon.
+    g1h->set_free_regions_coming();
+  }
   double note_end_end = os::elapsedTime();
-  // Tell the mutators that there might be unclean regions coming...
   if (G1PrintParCleanupStats) {
     gclog_or_tty->print_cr("  note end of marking: %8.3f ms.",
                            (note_end_end - note_end_start)*1000.0);
@@ -1799,33 +1794,63 @@
                      /* silent       */ false,
                      /* prev marking */ true);
   }
+
+  g1h->verify_region_sets_optional();
 }
 
 void ConcurrentMark::completeCleanup() {
-  // A full collection intervened.
   if (has_aborted()) return;
 
-  int first = 0;
-  int last = (int)MAX2(ParallelGCThreads, (size_t)1);
-  for (int t = 0; t < last; t++) {
-    UncleanRegionList* list = &_par_cleanup_thread_state[t]->list;
-    assert(list->well_formed(), "Inv");
-    HeapRegion* hd = list->hd();
-    while (hd != NULL) {
-      // Now finish up the other stuff.
-      hd->rem_set()->clear();
-      HeapRegion* next_hd = hd->next_from_unclean_list();
-      (void)list->pop();
-      assert(list->hd() == next_hd, "how not?");
-      _g1h->put_region_on_unclean_list(hd);
-      if (!hd->isHumongous()) {
-        // Add this to the _free_regions count by 1.
-        _g1h->finish_free_region_work(0, 0, 1, NULL);
+  G1CollectedHeap* g1h = G1CollectedHeap::heap();
+
+  _cleanup_list.verify_optional();
+  FreeRegionList local_free_list("Local Cleanup List");
+
+  if (G1ConcRegionFreeingVerbose) {
+    gclog_or_tty->print_cr("G1ConcRegionFreeing [complete cleanup] : "
+                           "cleanup list has "SIZE_FORMAT" entries",
+                           _cleanup_list.length());
+  }
+
+  // Noone else should be accessing the _cleanup_list at this point,
+  // so it's not necessary to take any locks
+  while (!_cleanup_list.is_empty()) {
+    HeapRegion* hr = _cleanup_list.remove_head();
+    assert(hr != NULL, "the list was not empty");
+    hr->rem_set()->clear();
+    local_free_list.add_as_tail(hr);
+
+    // Instead of adding one region at a time to the secondary_free_list,
+    // we accumulate them in the local list and move them a few at a
+    // time. This also cuts down on the number of notify_all() calls
+    // we do during this process. We'll also append the local list when
+    // _cleanup_list is empty (which means we just removed the last
+    // region from the _cleanup_list).
+    if ((local_free_list.length() % G1SecondaryFreeListAppendLength == 0) ||
+        _cleanup_list.is_empty()) {
+      if (G1ConcRegionFreeingVerbose) {
+        gclog_or_tty->print_cr("G1ConcRegionFreeing [complete cleanup] : "
+                               "appending "SIZE_FORMAT" entries to the "
+                               "secondary_free_list, clean list still has "
+                               SIZE_FORMAT" entries",
+                               local_free_list.length(),
+                               _cleanup_list.length());
       }
-      hd = list->hd();
-      assert(hd == next_hd, "how not?");
+
+      {
+        MutexLockerEx x(SecondaryFreeList_lock, Mutex::_no_safepoint_check_flag);
+        g1h->secondary_free_list_add_as_tail(&local_free_list);
+        SecondaryFreeList_lock->notify_all();
+      }
+
+      if (G1StressConcRegionFreeing) {
+        for (uintx i = 0; i < G1StressConcRegionFreeingDelayMillis; ++i) {
+          os::sleep(Thread::current(), (jlong) 1, false);
+        }
+      }
     }
   }
+  assert(local_free_list.is_empty(), "post-condition");
 }
 
 bool G1CMIsAliveClosure::do_object_b(oop obj) {
@@ -2897,9 +2922,9 @@
   virtual void do_oop(      oop* p) { do_oop_work(p); }
 
   template <class T> void do_oop_work(T* p) {
-    assert(_g1h->is_in_g1_reserved((HeapWord*) p), "invariant");
-    assert(!_g1h->heap_region_containing((HeapWord*) p)->is_on_free_list(),
-           "invariant");
+    assert( _g1h->is_in_g1_reserved((HeapWord*) p), "invariant");
+    assert(!_g1h->is_on_free_list(
+                    _g1h->heap_region_containing((HeapWord*) p)), "invariant");
 
     oop obj = oopDesc::load_decode_heap_oop(p);
     if (_cm->verbose_high())
@@ -3119,8 +3144,8 @@
 void CMTask::push(oop obj) {
   HeapWord* objAddr = (HeapWord*) obj;
   assert(_g1h->is_in_g1_reserved(objAddr), "invariant");
-  assert(!_g1h->heap_region_containing(objAddr)->is_on_free_list(),
-         "invariant");
+  assert(!_g1h->is_on_free_list(
+              _g1h->heap_region_containing((HeapWord*) objAddr)), "invariant");
   assert(!_g1h->is_obj_ill(obj), "invariant");
   assert(_nextMarkBitMap->isMarked(objAddr), "invariant");
 
@@ -3365,8 +3390,8 @@
                                (void*) obj);
 
       assert(_g1h->is_in_g1_reserved((HeapWord*) obj), "invariant" );
-      assert(!_g1h->heap_region_containing(obj)->is_on_free_list(),
-             "invariant");
+      assert(!_g1h->is_on_free_list(
+                  _g1h->heap_region_containing((HeapWord*) obj)), "invariant");
 
       scan_object(obj);
 
--- a/src/share/vm/gc_implementation/g1/concurrentMark.hpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/concurrentMark.hpp	Wed Jan 19 19:30:42 2011 -0500
@@ -1,5 +1,5 @@
 /*
- * Copyright (c) 2001, 2010, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2001, 2011, Oracle and/or its affiliates. All rights reserved.
  * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
  *
  * This code is free software; you can redistribute it and/or modify it
@@ -25,7 +25,7 @@
 #ifndef SHARE_VM_GC_IMPLEMENTATION_G1_CONCURRENTMARK_HPP
 #define SHARE_VM_GC_IMPLEMENTATION_G1_CONCURRENTMARK_HPP
 
-#include "gc_implementation/g1/heapRegion.hpp"
+#include "gc_implementation/g1/heapRegionSets.hpp"
 #include "utilities/taskqueue.hpp"
 
 class G1CollectedHeap;
@@ -369,13 +369,7 @@
   double                _cleanup_sleep_factor;
   double                _cleanup_task_overhead;
 
-  // Stuff related to age cohort processing.
-  struct ParCleanupThreadState {
-    char _pre[64];
-    UncleanRegionList list;
-    char _post[64];
-  };
-  ParCleanupThreadState** _par_cleanup_thread_state;
+  FreeRegionList        _cleanup_list;
 
   // CMS marking support structures
   CMBitMap                _markBitMap1;
@@ -484,6 +478,10 @@
   // prints all gathered CM-related statistics
   void print_stats();
 
+  bool cleanup_list_is_empty() {
+    return _cleanup_list.is_empty();
+  }
+
   // accessor methods
   size_t parallel_marking_threads() { return _parallel_marking_threads; }
   double sleep_factor()             { return _sleep_factor; }
--- a/src/share/vm/gc_implementation/g1/concurrentMarkThread.cpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/concurrentMarkThread.cpp	Wed Jan 19 19:30:42 2011 -0500
@@ -1,5 +1,5 @@
 /*
- * Copyright (c) 2001, 2010, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2001, 2011, Oracle and/or its affiliates. All rights reserved.
  * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
  *
  * This code is free software; you can redistribute it and/or modify it
@@ -95,8 +95,8 @@
   _vtime_start = os::elapsedVTime();
   wait_for_universe_init();
 
-  G1CollectedHeap* g1 = G1CollectedHeap::heap();
-  G1CollectorPolicy* g1_policy = g1->g1_policy();
+  G1CollectedHeap* g1h = G1CollectedHeap::heap();
+  G1CollectorPolicy* g1_policy = g1h->g1_policy();
   G1MMUTracker *mmu_tracker = g1_policy->mmu_tracker();
   Thread *current_thread = Thread::current();
 
@@ -119,7 +119,7 @@
       if (!g1_policy->in_young_gc_mode()) {
         // this ensures the flag is not set if we bail out of the marking
         // cycle; normally the flag is cleared immediately after cleanup
-        g1->set_marking_complete();
+        g1h->set_marking_complete();
 
         if (g1_policy->adaptive_young_list_length()) {
           double now = os::elapsedTime();
@@ -228,10 +228,20 @@
         VM_CGC_Operation op(&cl_cl, verbose_str);
         VMThread::execute(&op);
       } else {
-        G1CollectedHeap::heap()->set_marking_complete();
+        g1h->set_marking_complete();
       }
 
-      if (!cm()->has_aborted()) {
+      // Check if cleanup set the free_regions_coming flag. If it
+      // hasn't, we can just skip the next step.
+      if (g1h->free_regions_coming()) {
+        // The following will finish freeing up any regions that we
+        // found to be empty during cleanup. We'll do this part
+        // without joining the suspendible set. If an evacuation pause
+        // takes places, then we would carry on freeing regions in
+        // case they are needed by the pause. If a Full GC takes
+        // places, it would wait for us to process the regions
+        // reclaimed by cleanup.
+
         double cleanup_start_sec = os::elapsedTime();
         if (PrintGC) {
           gclog_or_tty->date_stamp(PrintGCDateStamps);
@@ -240,23 +250,22 @@
         }
 
         // Now do the remainder of the cleanup operation.
-        _sts.join();
         _cm->completeCleanup();
-        if (!cm()->has_aborted()) {
-          g1_policy->record_concurrent_mark_cleanup_completed();
+        g1_policy->record_concurrent_mark_cleanup_completed();
 
-          double cleanup_end_sec = os::elapsedTime();
-          if (PrintGC) {
-            gclog_or_tty->date_stamp(PrintGCDateStamps);
-            gclog_or_tty->stamp(PrintGCTimeStamps);
-            gclog_or_tty->print_cr("[GC concurrent-cleanup-end, %1.7lf]",
-                                   cleanup_end_sec - cleanup_start_sec);
-          }
+        double cleanup_end_sec = os::elapsedTime();
+        if (PrintGC) {
+          gclog_or_tty->date_stamp(PrintGCDateStamps);
+          gclog_or_tty->stamp(PrintGCTimeStamps);
+          gclog_or_tty->print_cr("[GC concurrent-cleanup-end, %1.7lf]",
+                                 cleanup_end_sec - cleanup_start_sec);
         }
-        _sts.leave();
+
+        // We're done: no more free regions coming.
+        g1h->reset_free_regions_coming();
       }
-      // We're done: no more unclean regions coming.
-      G1CollectedHeap::heap()->set_unclean_regions_coming(false);
+      guarantee(cm()->cleanup_list_is_empty(),
+                "at this point there should be no regions on the cleanup list");
 
       if (cm()->has_aborted()) {
         if (PrintGC) {
@@ -278,7 +287,7 @@
     // Java thread is waiting for a full GC to happen (e.g., it
     // called System.gc() with +ExplicitGCInvokesConcurrent).
     _sts.join();
-    g1->increment_full_collections_completed(true /* concurrent */);
+    g1h->increment_full_collections_completed(true /* concurrent */);
     _sts.leave();
   }
   assert(_should_terminate, "just checking");
--- a/src/share/vm/gc_implementation/g1/concurrentZFThread.cpp	Wed Jan 19 13:04:37 2011 -0800
+++ /dev/null	Thu Jan 01 00:00:00 1970 +0000
@@ -1,194 +0,0 @@
-/*
- * Copyright (c) 2001, 2010, Oracle and/or its affiliates. All rights reserved.
- * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
- *
- * This code is free software; you can redistribute it and/or modify it
- * under the terms of the GNU General Public License version 2 only, as
- * published by the Free Software Foundation.
- *
- * This code is distributed in the hope that it will be useful, but WITHOUT
- * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
- * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License
- * version 2 for more details (a copy is included in the LICENSE file that
- * accompanied this code).
- *
- * You should have received a copy of the GNU General Public License version
- * 2 along with this work; if not, write to the Free Software Foundation,
- * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
- *
- * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
- * or visit www.oracle.com if you need additional information or have any
- * questions.
- *
- */
-
-#include "precompiled.hpp"
-#include "gc_implementation/g1/concurrentZFThread.hpp"
-#include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
-#include "gc_implementation/g1/heapRegion.hpp"
-#include "memory/space.inline.hpp"
-#include "runtime/mutexLocker.hpp"
-#include "utilities/copy.hpp"
-
-// ======= Concurrent Zero-Fill Thread ========
-
-// The CM thread is created when the G1 garbage collector is used
-
-int ConcurrentZFThread::_region_allocs = 0;
-int ConcurrentZFThread::_sync_zfs = 0;
-int ConcurrentZFThread::_zf_waits = 0;
-int ConcurrentZFThread::_regions_filled = 0;
-
-ConcurrentZFThread::ConcurrentZFThread() :
-  ConcurrentGCThread()
-{
-  create_and_start();
-}
-
-void ConcurrentZFThread::wait_for_ZF_completed(HeapRegion* hr) {
-  assert(ZF_mon->owned_by_self(), "Precondition.");
-  note_zf_wait();
-  while (hr->zero_fill_state() == HeapRegion::ZeroFilling) {
-    ZF_mon->wait(Mutex::_no_safepoint_check_flag);
-  }
-}
-
-void ConcurrentZFThread::processHeapRegion(HeapRegion* hr) {
-  assert(!Universe::heap()->is_gc_active(),
-         "This should not happen during GC.");
-  assert(hr != NULL, "Precondition");
-  // These are unlocked reads, but if this test is successful, then no
-  // other thread will attempt this zero filling.  Only a GC thread can
-  // modify the ZF state of a region whose state is zero-filling, and this
-  // should only happen while the ZF thread is locking out GC.
-  if (hr->zero_fill_state() == HeapRegion::ZeroFilling
-      && hr->zero_filler() == Thread::current()) {
-    assert(hr->top() == hr->bottom(), "better be empty!");
-    assert(!hr->isHumongous(), "Only free regions on unclean list.");
-    Copy::fill_to_words(hr->bottom(), hr->capacity()/HeapWordSize);
-    note_region_filled();
-  }
-}
-
-void ConcurrentZFThread::run() {
-  initialize_in_thread();
-  Thread* thr_self = Thread::current();
-  _vtime_start = os::elapsedVTime();
-  wait_for_universe_init();
-
-  G1CollectedHeap* g1 = G1CollectedHeap::heap();
-  _sts.join();
-  while (!_should_terminate) {
-    _sts.leave();
-
-    {
-      MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-
-      // This local variable will hold a region being zero-filled.  This
-      // region will neither be on the unclean or zero-filled lists, and
-      // will not be available for allocation; thus, we might have an
-      // allocation fail, causing a full GC, because of this, but this is a
-      // price we will pay.  (In future, we might want to make the fact
-      // that there's a region being zero-filled apparent to the G1 heap,
-      // which could then wait for it in this extreme case...)
-      HeapRegion* to_fill;
-
-      while (!g1->should_zf()
-             || (to_fill = g1->pop_unclean_region_list_locked()) == NULL)
-        ZF_mon->wait(Mutex::_no_safepoint_check_flag);
-      while (to_fill->zero_fill_state() == HeapRegion::ZeroFilling)
-        ZF_mon->wait(Mutex::_no_safepoint_check_flag);
-
-      // So now to_fill is non-NULL and is not ZeroFilling.  It might be
-      // Allocated or ZeroFilled.  (The latter could happen if this thread
-      // starts the zero-filling of a region, but a GC intervenes and
-      // pushes new regions needing on the front of the filling on the
-      // front of the list.)
-
-      switch (to_fill->zero_fill_state()) {
-      case HeapRegion::Allocated:
-        to_fill = NULL;
-        break;
-
-      case HeapRegion::NotZeroFilled:
-        to_fill->set_zero_fill_in_progress(thr_self);
-
-        ZF_mon->unlock();
-        _sts.join();
-        processHeapRegion(to_fill);
-        _sts.leave();
-        ZF_mon->lock_without_safepoint_check();
-
-        if (to_fill->zero_fill_state() == HeapRegion::ZeroFilling
-            && to_fill->zero_filler() == thr_self) {
-          to_fill->set_zero_fill_complete();
-          (void)g1->put_free_region_on_list_locked(to_fill);
-        }
-        break;
-
-      case HeapRegion::ZeroFilled:
-        (void)g1->put_free_region_on_list_locked(to_fill);
-        break;
-
-      case HeapRegion::ZeroFilling:
-        ShouldNotReachHere();
-        break;
-      }
-    }
-    _vtime_accum = (os::elapsedVTime() - _vtime_start);
-    _sts.join();
-  }
-  _sts.leave();
-
-  assert(_should_terminate, "just checking");
-  terminate();
-}
-
-bool ConcurrentZFThread::offer_yield() {
-  if (_sts.should_yield()) {
-    _sts.yield("Concurrent ZF");
-    return true;
-  } else {
-    return false;
-  }
-}
-
-void ConcurrentZFThread::stop() {
-  // it is ok to take late safepoints here, if needed
-  MutexLockerEx mu(Terminator_lock);
-  _should_terminate = true;
-  while (!_has_terminated) {
-    Terminator_lock->wait();
-  }
-}
-
-void ConcurrentZFThread::print() const {
-  print_on(tty);
-}
-
-void ConcurrentZFThread::print_on(outputStream* st) const {
-  st->print("\"G1 Concurrent Zero-Fill Thread\" ");
-  Thread::print_on(st);
-  st->cr();
-}
-
-
-double ConcurrentZFThread::_vtime_accum;
-
-void ConcurrentZFThread::print_summary_info() {
-  gclog_or_tty->print("\nConcurrent Zero-Filling:\n");
-  gclog_or_tty->print("  Filled %d regions, used %5.2fs.\n",
-                      _regions_filled,
-                      vtime_accum());
-  gclog_or_tty->print("  Of %d region allocs, %d (%5.2f%%) required sync ZF,\n",
-                      _region_allocs, _sync_zfs,
-                      (_region_allocs > 0 ?
-                       (float)_sync_zfs/(float)_region_allocs*100.0 :
-                       0.0));
-  gclog_or_tty->print("     and %d (%5.2f%%) required a ZF wait.\n",
-                      _zf_waits,
-                      (_region_allocs > 0 ?
-                       (float)_zf_waits/(float)_region_allocs*100.0 :
-                       0.0));
-
-}
--- a/src/share/vm/gc_implementation/g1/concurrentZFThread.hpp	Wed Jan 19 13:04:37 2011 -0800
+++ /dev/null	Thu Jan 01 00:00:00 1970 +0000
@@ -1,91 +0,0 @@
-/*
- * Copyright (c) 2001, 2010, Oracle and/or its affiliates. All rights reserved.
- * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
- *
- * This code is free software; you can redistribute it and/or modify it
- * under the terms of the GNU General Public License version 2 only, as
- * published by the Free Software Foundation.
- *
- * This code is distributed in the hope that it will be useful, but WITHOUT
- * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
- * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License
- * version 2 for more details (a copy is included in the LICENSE file that
- * accompanied this code).
- *
- * You should have received a copy of the GNU General Public License version
- * 2 along with this work; if not, write to the Free Software Foundation,
- * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
- *
- * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
- * or visit www.oracle.com if you need additional information or have any
- * questions.
- *
- */
-
-#ifndef SHARE_VM_GC_IMPLEMENTATION_G1_CONCURRENTZFTHREAD_HPP
-#define SHARE_VM_GC_IMPLEMENTATION_G1_CONCURRENTZFTHREAD_HPP
-
-#include "gc_implementation/shared/concurrentGCThread.hpp"
-
-// The Concurrent ZF Thread.  Performs concurrent zero-filling.
-
-class ConcurrentZFThread: public ConcurrentGCThread {
-  friend class VMStructs;
-  friend class ZeroFillRegionClosure;
-
- private:
-
-  // Zero fill the heap region.
-  void processHeapRegion(HeapRegion* r);
-
-  // Stats
-  //   Allocation (protected by heap lock).
-  static int _region_allocs;  // Number of regions allocated
-  static int _sync_zfs;       //   Synchronous zero-fills +
-  static int _zf_waits;      //   Wait for conc zero-fill completion.
-
-  // Number of regions CFZ thread fills.
-  static int _regions_filled;
-
-  double _vtime_start;  // Initial virtual time.
-
-  // These are static because the "print_summary_info" method is, and
-  // it currently assumes there is only one ZF thread.  We'll change when
-  // we need to.
-  static double _vtime_accum;  // Initial virtual time.
-  static double vtime_accum() { return _vtime_accum; }
-
-  // Offer yield for GC.  Returns true if yield occurred.
-  bool offer_yield();
-
- public:
-  // Constructor
-  ConcurrentZFThread();
-
-  // Main loop.
-  virtual void run();
-
-  // Printing
-  void print_on(outputStream* st) const;
-  void print() const;
-
-  // Waits until "r" has been zero-filled.  Requires caller to hold the
-  // ZF_mon.
-  static void wait_for_ZF_completed(HeapRegion* r);
-
-  // Get or clear the current unclean region.  Should be done
-  // while holding the ZF_needed_mon lock.
-
-  // shutdown
-  void stop();
-
-  // Stats
-  static void note_region_alloc() {_region_allocs++; }
-  static void note_sync_zfs() { _sync_zfs++; }
-  static void note_zf_wait() { _zf_waits++; }
-  static void note_region_filled() { _regions_filled++; }
-
-  static void print_summary_info();
-};
-
-#endif // SHARE_VM_GC_IMPLEMENTATION_G1_CONCURRENTZFTHREAD_HPP
--- a/src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp	Wed Jan 19 19:30:42 2011 -0500
@@ -28,7 +28,6 @@
 #include "gc_implementation/g1/concurrentG1Refine.hpp"
 #include "gc_implementation/g1/concurrentG1RefineThread.hpp"
 #include "gc_implementation/g1/concurrentMarkThread.inline.hpp"
-#include "gc_implementation/g1/concurrentZFThread.hpp"
 #include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
 #include "gc_implementation/g1/g1CollectorPolicy.hpp"
 #include "gc_implementation/g1/g1MarkSweep.hpp"
@@ -425,11 +424,9 @@
 
 void G1CollectedHeap::stop_conc_gc_threads() {
   _cg1r->stop();
-  _czft->stop();
   _cmThread->stop();
 }
 
-
 void G1CollectedHeap::check_ct_logs_at_safepoint() {
   DirtyCardQueueSet& dcqs = JavaThread::dirty_card_queue_set();
   CardTableModRefBS* ct_bs = (CardTableModRefBS*)barrier_set();
@@ -481,49 +478,92 @@
 
 // Private methods.
 
-// Finds a HeapRegion that can be used to allocate a given size of block.
-
-
-HeapRegion* G1CollectedHeap::newAllocRegion_work(size_t word_size,
-                                                 bool do_expand,
-                                                 bool zero_filled) {
-  ConcurrentZFThread::note_region_alloc();
-  HeapRegion* res = alloc_free_region_from_lists(zero_filled);
+HeapRegion*
+G1CollectedHeap::new_region_try_secondary_free_list(size_t word_size) {
+  MutexLockerEx x(SecondaryFreeList_lock, Mutex::_no_safepoint_check_flag);
+  while (!_secondary_free_list.is_empty() || free_regions_coming()) {
+    if (!_secondary_free_list.is_empty()) {
+      if (G1ConcRegionFreeingVerbose) {
+        gclog_or_tty->print_cr("G1ConcRegionFreeing [region alloc] : "
+                               "secondary_free_list has "SIZE_FORMAT" entries",
+                               _secondary_free_list.length());
+      }
+      // It looks as if there are free regions available on the
+      // secondary_free_list. Let's move them to the free_list and try
+      // again to allocate from it.
+      append_secondary_free_list();
+
+      assert(!_free_list.is_empty(), "if the secondary_free_list was not "
+             "empty we should have moved at least one entry to the free_list");
+      HeapRegion* res = _free_list.remove_head();
+      if (G1ConcRegionFreeingVerbose) {
+        gclog_or_tty->print_cr("G1ConcRegionFreeing [region alloc] : "
+                               "allocated "HR_FORMAT" from secondary_free_list",
+                               HR_FORMAT_PARAMS(res));
+      }
+      return res;
+    }
+
+    // Wait here until we get notifed 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);
+  }
+
+  if (G1ConcRegionFreeingVerbose) {
+    gclog_or_tty->print_cr("G1ConcRegionFreeing [region alloc] : "
+                           "could not allocate from secondary_free_list");
+  }
+  return NULL;
+}
+
+HeapRegion* G1CollectedHeap::new_region_work(size_t word_size,
+                                             bool do_expand) {
+  assert(!isHumongous(word_size) ||
+                                  word_size <= (size_t) HeapRegion::GrainWords,
+         "the only time we use this to allocate a humongous region is "
+         "when we are allocating a single humongous region");
+
+  HeapRegion* res;
+  if (G1StressConcRegionFreeing) {
+    if (!_secondary_free_list.is_empty()) {
+      if (G1ConcRegionFreeingVerbose) {
+        gclog_or_tty->print_cr("G1ConcRegionFreeing [region alloc] : "
+                               "forced to look at the secondary_free_list");
+      }
+      res = new_region_try_secondary_free_list(word_size);
+      if (res != NULL) {
+        return res;
+      }
+    }
+  }
+  res = _free_list.remove_head_or_null();
+  if (res == NULL) {
+    if (G1ConcRegionFreeingVerbose) {
+      gclog_or_tty->print_cr("G1ConcRegionFreeing [region alloc] : "
+                             "res == NULL, trying the secondary_free_list");
+    }
+    res = new_region_try_secondary_free_list(word_size);
+  }
   if (res == NULL && do_expand) {
     expand(word_size * HeapWordSize);
-    res = alloc_free_region_from_lists(zero_filled);
-    assert(res == NULL ||
-           (!res->isHumongous() &&
-            (!zero_filled ||
-             res->zero_fill_state() == HeapRegion::Allocated)),
-           "Alloc Regions must be zero filled (and non-H)");
+    res = _free_list.remove_head_or_null();
   }
   if (res != NULL) {
-    if (res->is_empty()) {
-      _free_regions--;
-    }
-    assert(!res->isHumongous() &&
-           (!zero_filled || res->zero_fill_state() == HeapRegion::Allocated),
-           err_msg("Non-young alloc Regions must be zero filled (and non-H):"
-                   " res->isHumongous()=%d, zero_filled=%d, res->zero_fill_state()=%d",
-                   res->isHumongous(), zero_filled, res->zero_fill_state()));
-    assert(!res->is_on_unclean_list(),
-           "Alloc Regions must not be on the unclean list");
     if (G1PrintHeapRegions) {
-      gclog_or_tty->print_cr("new alloc region %d:["PTR_FORMAT", "PTR_FORMAT"], "
-                             "top "PTR_FORMAT,
-                             res->hrs_index(), res->bottom(), res->end(), res->top());
+      gclog_or_tty->print_cr("new alloc region %d:["PTR_FORMAT","PTR_FORMAT"], "
+                             "top "PTR_FORMAT, res->hrs_index(),
+                             res->bottom(), res->end(), res->top());
     }
   }
   return res;
 }
 
-HeapRegion* G1CollectedHeap::newAllocRegionWithExpansion(int purpose,
-                                                         size_t word_size,
-                                                         bool zero_filled) {
+HeapRegion* G1CollectedHeap::new_gc_alloc_region(int purpose,
+                                                 size_t word_size) {
   HeapRegion* alloc_region = NULL;
   if (_gc_alloc_region_counts[purpose] < g1_policy()->max_regions(purpose)) {
-    alloc_region = newAllocRegion_work(word_size, true, zero_filled);
+    alloc_region = new_region_work(word_size, true /* do_expand */);
     if (purpose == GCAllocForSurvived && alloc_region != NULL) {
       alloc_region->set_survivor();
     }
@@ -534,82 +574,188 @@
   return alloc_region;
 }
 
+int G1CollectedHeap::humongous_obj_allocate_find_first(size_t num_regions,
+                                                       size_t word_size) {
+  int first = -1;
+  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
+    // let's not try to expand here too.
+    HeapRegion* hr = new_region_work(word_size, false /* do_expand */);
+    if (hr != NULL) {
+      first = hr->hrs_index();
+    } else {
+      first = -1;
+    }
+  } else {
+    // We can't allocate humongous regions while cleanupComplete() is
+    // running, since some of the regions we find to be empty might not
+    // yet be added to the free list and it is not straightforward to
+    // know which list they are on so that we can remove them. Note
+    // that we only need to do this if we need to allocate more than
+    // one region to satisfy the current humongous allocation
+    // request. If we are only allocating one region we use the common
+    // region allocation code (see above).
+    wait_while_free_regions_coming();
+    append_secondary_free_list_if_not_empty();
+
+    if (free_regions() >= num_regions) {
+      first = _hrs->find_contiguous(num_regions);
+      if (first != -1) {
+        for (int i = first; i < first + (int) num_regions; ++i) {
+          HeapRegion* hr = _hrs->at(i);
+          assert(hr->is_empty(), "sanity");
+          assert(is_on_free_list(hr), "sanity");
+          hr->set_pending_removal(true);
+        }
+        _free_list.remove_all_pending(num_regions);
+      }
+    }
+  }
+  return first;
+}
+
 // If could fit into free regions w/o expansion, try.
 // Otherwise, if can expand, do so.
 // Otherwise, if using ex regions might help, try with ex given back.
 HeapWord* G1CollectedHeap::humongous_obj_allocate(size_t word_size) {
-  assert_heap_locked_or_at_safepoint();
-  assert(regions_accounted_for(), "Region leakage!");
-
-  // We can't allocate humongous regions while cleanupComplete is
-  // running, since some of the regions we find to be empty might not
-  // yet be added to the unclean list. If we're already at a
-  // safepoint, this call is unnecessary, not to mention wrong.
-  if (!SafepointSynchronize::is_at_safepoint()) {
-    wait_for_cleanup_complete();
-  }
+  assert_heap_locked_or_at_safepoint(true /* should_be_vm_thread */);
+
+  verify_region_sets_optional();
 
   size_t num_regions =
          round_to(word_size, HeapRegion::GrainWords) / HeapRegion::GrainWords;
-
-  // Special case if < one region???
-
-  // Remember the ft size.
   size_t x_size = expansion_regions();
-
-  HeapWord* res = NULL;
-  bool eliminated_allocated_from_lists = false;
-
-  // Can the allocation potentially fit in the free regions?
-  if (free_regions() >= num_regions) {
-    res = _hrs->obj_allocate(word_size);
-  }
-  if (res == NULL) {
-    // Try expansion.
-    size_t fs = _hrs->free_suffix();
+  size_t fs = _hrs->free_suffix();
+  int first = humongous_obj_allocate_find_first(num_regions, word_size);
+  if (first == -1) {
+    // The only thing we can do now is attempt expansion.
     if (fs + x_size >= num_regions) {
       expand((num_regions - fs) * HeapRegion::GrainBytes);
-      res = _hrs->obj_allocate(word_size);
-      assert(res != NULL, "This should have worked.");
-    } else {
-      // Expansion won't help.  Are there enough free regions if we get rid
-      // of reservations?
-      size_t avail = free_regions();
-      if (avail >= num_regions) {
-        res = _hrs->obj_allocate(word_size);
-        if (res != NULL) {
-          remove_allocated_regions_from_lists();
-          eliminated_allocated_from_lists = true;
-        }
+      first = humongous_obj_allocate_find_first(num_regions, word_size);
+      assert(first != -1, "this should have worked");
+    }
+  }
+
+  if (first != -1) {
+    // Index of last region in the series + 1.
+    int last = first + (int) num_regions;
+
+    // We need to initialize the region(s) we just discovered. This is
+    // a bit tricky given that it can happen concurrently with
+    // refinement threads refining cards on these regions and
+    // potentially wanting to refine the BOT as they are scanning
+    // those cards (this can happen shortly after a cleanup; see CR
+    // 6991377). So we have to set up the region(s) carefully and in
+    // a specific order.
+
+    // The word size sum of all the regions we will allocate.
+    size_t word_size_sum = num_regions * HeapRegion::GrainWords;
+    assert(word_size <= word_size_sum, "sanity");
+
+    // This will be the "starts humongous" region.
+    HeapRegion* first_hr = _hrs->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.
+    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;
+
+    // First, we need to zero the header of the space that we will be
+    // allocating. When we update top further down, some refinement
+    // threads might try to scan the region. By zeroing the header we
+    // ensure that any thread that will try to scan the region will
+    // come across the zero klass word and bail out.
+    //
+    // NOTE: It would not have been correct to have used
+    // CollectedHeap::fill_with_object() and make the space look like
+    // an int array. The thread that is doing the allocation will
+    // later update the object header to a potentially different array
+    // type and, for a very short period of time, the klass and length
+    // fields will be inconsistent. This could cause a refinement
+    // thread to calculate the object size incorrectly.
+    Copy::fill_to_words(new_obj, oopDesc::header_size(), 0);
+
+    // We will set up the first region as "starts humongous". This
+    // will also update the BOT covering all the regions to reflect
+    // that there is a single object that starts at the bottom of the
+    // first region.
+    first_hr->set_startsHumongous(new_top, new_end);
+
+    // Then, if there are any, we will set up the "continues
+    // humongous" regions.
+    HeapRegion* hr = NULL;
+    for (int i = first + 1; i < last; ++i) {
+      hr = _hrs->at(i);
+      hr->set_continuesHumongous(first_hr);
+    }
+    // If we have "continues humongous" regions (hr != NULL), then the
+    // end of the last one should match new_end.
+    assert(hr == NULL || hr->end() == new_end, "sanity");
+
+    // Up to this point no concurrent thread would have been able to
+    // do any scanning on any region in this series. All the top
+    // fields still point to bottom, so the intersection between
+    // [bottom,top] and [card_start,card_end] will be empty. Before we
+    // update the top fields, we'll do a storestore to make sure that
+    // no thread sees the update to top before the zeroing of the
+    // object header and the BOT initialization.
+    OrderAccess::storestore();
+
+    // Now that the BOT and the object header have been initialized,
+    // we can update top of the "starts humongous" region.
+    assert(first_hr->bottom() < new_top && new_top <= first_hr->end(),
+           "new_top should be in this region");
+    first_hr->set_top(new_top);
+
+    // Now, we will update the top fields of the "continues humongous"
+    // regions. The reason we need to do this is that, otherwise,
+    // these regions would look empty and this will confuse parts of
+    // G1. For example, the code that looks for a consecutive number
+    // of empty regions will consider them empty and try to
+    // re-allocate them. We can extend is_empty() to also include
+    // !continuesHumongous(), but it is easier to just update the top
+    // fields here. The way we set top for all regions (i.e., top ==
+    // end for all regions but the last one, top == new_top for the
+    // last one) is actually used when we will free up the humongous
+    // region in free_humongous_region().
+    hr = NULL;
+    for (int i = first + 1; i < last; ++i) {
+      hr = _hrs->at(i);
+      if ((i + 1) == last) {
+        // last continues humongous region
+        assert(hr->bottom() < new_top && new_top <= hr->end(),
+               "new_top should fall on this region");
+        hr->set_top(new_top);
+      } else {
+        // not last one
+        assert(new_top > hr->end(), "new_top should be above this region");
+        hr->set_top(hr->end());
       }
     }
-  }
-  if (res != NULL) {
-    // Increment by the number of regions allocated.
-    // FIXME: Assumes regions all of size GrainBytes.
-#ifndef PRODUCT
-    mr_bs()->verify_clean_region(MemRegion(res, res + num_regions *
-                                           HeapRegion::GrainWords));
-#endif
-    if (!eliminated_allocated_from_lists)
-      remove_allocated_regions_from_lists();
-    _summary_bytes_used += word_size * HeapWordSize;
-    _free_regions -= num_regions;
-    _num_humongous_regions += (int) num_regions;
-  }
-  assert(regions_accounted_for(), "Region Leakage");
-  return res;
+    // If we have continues humongous regions (hr != NULL), then the
+    // end of the last one should match new_end and its top should
+    // match new_top.
+    assert(hr == NULL ||
+           (hr->end() == new_end && hr->top() == new_top), "sanity");
+
+    assert(first_hr->used() == word_size * HeapWordSize, "invariant");
+    _summary_bytes_used += first_hr->used();
+    _humongous_set.add(first_hr);
+
+    return new_obj;
+  }
+
+  verify_region_sets_optional();
+  return NULL;
 }
 
 void
 G1CollectedHeap::retire_cur_alloc_region(HeapRegion* cur_alloc_region) {
-  // The cleanup operation might update _summary_bytes_used
-  // concurrently with this method. So, right now, if we don't wait
-  // for it to complete, updates to _summary_bytes_used might get
-  // lost. This will be resolved in the near future when the operation
-  // of the free region list is revamped as part of CR 6977804.
-  wait_for_cleanup_complete();
-
   // Other threads might still be trying to allocate using CASes out
   // of the region we are retiring, as they can do so without holding
   // the Heap_lock. So we first have to make sure that noone else can
@@ -654,7 +800,7 @@
                                                        bool at_safepoint,
                                                        bool do_dirtying,
                                                        bool can_expand) {
-  assert_heap_locked_or_at_safepoint();
+  assert_heap_locked_or_at_safepoint(true /* should_be_vm_thread */);
   assert(_cur_alloc_region == NULL,
          "replace_cur_alloc_region_and_allocate() should only be called "
          "after retiring the previous current alloc region");
@@ -665,25 +811,12 @@
          "we are not allowed to expand the young gen");
 
   if (can_expand || !g1_policy()->is_young_list_full()) {
-    if (!at_safepoint) {
-      // The cleanup operation might update _summary_bytes_used
-      // concurrently with this method. So, right now, if we don't
-      // wait for it to complete, updates to _summary_bytes_used might
-      // get lost. This will be resolved in the near future when the
-      // operation of the free region list is revamped as part of
-      // CR 6977804. If we're already at a safepoint, this call is
-      // unnecessary, not to mention wrong.
-      wait_for_cleanup_complete();
-    }
-
-    HeapRegion* new_cur_alloc_region = newAllocRegion(word_size,
-                                                      false /* zero_filled */);
+    HeapRegion* new_cur_alloc_region = new_alloc_region(word_size);
     if (new_cur_alloc_region != NULL) {
       assert(new_cur_alloc_region->is_empty(),
              "the newly-allocated region should be empty, "
              "as right now we only allocate new regions out of the free list");
       g1_policy()->update_region_num(true /* next_is_young */);
-      _summary_bytes_used -= new_cur_alloc_region->used();
       set_region_short_lived_locked(new_cur_alloc_region);
 
       assert(!new_cur_alloc_region->isHumongous(),
@@ -733,7 +866,7 @@
 
   assert(_cur_alloc_region == NULL, "we failed to allocate a new current "
          "alloc region, it should still be NULL");
-  assert_heap_locked_or_at_safepoint();
+  assert_heap_locked_or_at_safepoint(true /* should_be_vm_thread */);
   return NULL;
 }
 
@@ -745,6 +878,10 @@
   assert(!isHumongous(word_size), "attempt_allocation_slow() should not be "
          "used for humongous allocations");
 
+  // We should only reach here when we were unable to allocate
+  // otherwise. So, we should have not active current alloc region.
+  assert(_cur_alloc_region == NULL, "current alloc region should be NULL");
+
   // We will loop while succeeded is false, which means that we tried
   // to do a collection, but the VM op did not succeed. So, when we
   // exit the loop, either one of the allocation attempts was
@@ -756,30 +893,6 @@
     // Every time we go round the loop we should be holding the Heap_lock.
     assert_heap_locked();
 
-    {
-      // We may have concurrent cleanup working at the time. Wait for
-      // it to complete. In the future we would probably want to make
-      // the concurrent cleanup truly concurrent by decoupling it from
-      // the allocation. This will happen in the near future as part
-      // of CR 6977804 which will revamp the operation of the free
-      // region list. The fact that wait_for_cleanup_complete() will
-      // do a wait() means that we'll give up the Heap_lock. So, it's
-      // possible that when we exit wait_for_cleanup_complete() we
-      // might be able to allocate successfully (since somebody else
-      // might have done a collection meanwhile). So, we'll attempt to
-      // allocate again, just in case. When we make cleanup truly
-      // concurrent with allocation, we should remove this allocation
-      // attempt as it's redundant (we only reach here after an
-      // allocation attempt has been unsuccessful).
-      wait_for_cleanup_complete();
-
-      HeapWord* result = attempt_allocation_locked(word_size);
-      if (result != NULL) {
-        assert_heap_not_locked();
-        return result;
-      }
-    }
-
     if (GC_locker::is_active_and_needs_gc()) {
       // We are locked out of GC because of the GC locker. We can
       // allocate a new region only if we can expand the young gen.
@@ -894,7 +1007,7 @@
   // allocation paths that attempt to allocate a humongous object
   // should eventually reach here. Currently, the only paths are from
   // mem_allocate() and attempt_allocation_at_safepoint().
-  assert_heap_locked_or_at_safepoint();
+  assert_heap_locked_or_at_safepoint(true /* should_be_vm_thread */);
   assert(isHumongous(word_size), "attempt_allocation_humongous() "
          "should only be used for humongous allocations");
   assert(SafepointSynchronize::is_at_safepoint() == at_safepoint,
@@ -971,13 +1084,13 @@
     }
   }
 
-  assert_heap_locked_or_at_safepoint();
+  assert_heap_locked_or_at_safepoint(true /* should_be_vm_thread */);
   return NULL;
 }
 
 HeapWord* G1CollectedHeap::attempt_allocation_at_safepoint(size_t word_size,
                                            bool expect_null_cur_alloc_region) {
-  assert_at_safepoint();
+  assert_at_safepoint(true /* should_be_vm_thread */);
   assert(_cur_alloc_region == NULL || !expect_null_cur_alloc_region,
          err_msg("the current alloc region was unexpectedly found "
                  "to be non-NULL, cur alloc region: "PTR_FORMAT" "
@@ -1131,22 +1244,18 @@
 }
 
 void G1CollectedHeap::abandon_cur_alloc_region() {
-  if (_cur_alloc_region != NULL) {
-    // We're finished with the _cur_alloc_region.
-    if (_cur_alloc_region->is_empty()) {
-      _free_regions++;
-      free_region(_cur_alloc_region);
-    } else {
-      // As we're builing (at least the young portion) of the collection
-      // set incrementally we'll add the current allocation region to
-      // the collection set here.
-      if (_cur_alloc_region->is_young()) {
-        g1_policy()->add_region_to_incremental_cset_lhs(_cur_alloc_region);
-      }
-      _summary_bytes_used += _cur_alloc_region->used();
-    }
-    _cur_alloc_region = NULL;
-  }
+  assert_at_safepoint(true /* should_be_vm_thread */);
+
+  HeapRegion* cur_alloc_region = _cur_alloc_region;
+  if (cur_alloc_region != NULL) {
+    assert(!cur_alloc_region->is_empty(),
+           "the current alloc region can never be empty");
+    assert(cur_alloc_region->is_young(),
+           "the current alloc region should be young");
+
+    retire_cur_alloc_region_common(cur_alloc_region);
+  }
+  assert(_cur_alloc_region == NULL, "post-condition");
 }
 
 void G1CollectedHeap::abandon_gc_alloc_regions() {
@@ -1227,6 +1336,8 @@
 bool G1CollectedHeap::do_collection(bool explicit_gc,
                                     bool clear_all_soft_refs,
                                     size_t word_size) {
+  assert_at_safepoint(true /* should_be_vm_thread */);
+
   if (GC_locker::check_active_before_gc()) {
     return false;
   }
@@ -1238,8 +1349,7 @@
     Universe::print_heap_before_gc();
   }
 
-  assert(SafepointSynchronize::is_at_safepoint(), "should be at safepoint");
-  assert(Thread::current() == VMThread::vm_thread(), "should be in vm thread");
+  verify_region_sets_optional();
 
   const bool do_clear_all_soft_refs = clear_all_soft_refs ||
                            collector_policy()->should_clear_all_soft_refs();
@@ -1262,6 +1372,9 @@
     double start = os::elapsedTime();
     g1_policy()->record_full_collection_start();
 
+    wait_while_free_regions_coming();
+    append_secondary_free_list_if_not_empty();
+
     gc_prologue(true);
     increment_total_collections(true /* full gc */);
 
@@ -1274,7 +1387,6 @@
       gclog_or_tty->print(" VerifyBeforeGC:");
       Universe::verify(true);
     }
-    assert(regions_accounted_for(), "Region leakage!");
 
     COMPILER2_PRESENT(DerivedPointerTable::clear());
 
@@ -1298,7 +1410,6 @@
     assert(_cur_alloc_region == NULL, "Invariant.");
     g1_rem_set()->cleanupHRRS();
     tear_down_region_lists();
-    set_used_regions_to_need_zero_fill();
 
     // We may have added regions to the current incremental collection
     // set between the last GC or pause and now. We need to clear the
@@ -1333,9 +1444,7 @@
       HandleMark hm;  // Discard invalid handles created during gc
       G1MarkSweep::invoke_at_safepoint(ref_processor(), do_clear_all_soft_refs);
     }
-    // Because freeing humongous regions may have added some unclean
-    // regions, it is necessary to tear down again before rebuilding.
-    tear_down_region_lists();
+    assert(free_regions() == 0, "we should not have added any free regions");
     rebuild_region_lists();
 
     _summary_bytes_used = recalculate_used();
@@ -1417,7 +1526,6 @@
     JavaThread::dirty_card_queue_set().abandon_logs();
     assert(!G1DeferredRSUpdate
            || (G1DeferredRSUpdate && (dirty_card_queue_set().completed_buffers_num() == 0)), "Should not be any");
-    assert(regions_accounted_for(), "Region leakage!");
   }
 
   if (g1_policy()->in_young_gc_mode()) {
@@ -1431,6 +1539,8 @@
   // Update the number of full collections that have been completed.
   increment_full_collections_completed(false /* concurrent */);
 
+  verify_region_sets_optional();
+
   if (PrintHeapAtGC) {
     Universe::print_heap_after_gc();
   }
@@ -1571,10 +1681,7 @@
 HeapWord*
 G1CollectedHeap::satisfy_failed_allocation(size_t word_size,
                                            bool* succeeded) {
-  assert(SafepointSynchronize::is_at_safepoint(),
-         "satisfy_failed_allocation() should only be called at a safepoint");
-  assert(Thread::current()->is_VM_thread(),
-         "satisfy_failed_allocation() should only be called by the VM thread");
+  assert_at_safepoint(true /* should_be_vm_thread */);
 
   *succeeded = true;
   // Let's attempt the allocation first.
@@ -1646,53 +1753,22 @@
 // allocated block, or else "NULL".
 
 HeapWord* G1CollectedHeap::expand_and_allocate(size_t word_size) {
-  assert(SafepointSynchronize::is_at_safepoint(),
-         "expand_and_allocate() should only be called at a safepoint");
-  assert(Thread::current()->is_VM_thread(),
-         "expand_and_allocate() should only be called by the VM thread");
+  assert_at_safepoint(true /* should_be_vm_thread */);
+
+  verify_region_sets_optional();
 
   size_t expand_bytes = word_size * HeapWordSize;
   if (expand_bytes < MinHeapDeltaBytes) {
     expand_bytes = MinHeapDeltaBytes;
   }
   expand(expand_bytes);
-  assert(regions_accounted_for(), "Region leakage!");
+
+  verify_region_sets_optional();
 
   return attempt_allocation_at_safepoint(word_size,
                                      false /* expect_null_cur_alloc_region */);
 }
 
-size_t G1CollectedHeap::free_region_if_totally_empty(HeapRegion* hr) {
-  size_t pre_used = 0;
-  size_t cleared_h_regions = 0;
-  size_t freed_regions = 0;
-  UncleanRegionList local_list;
-  free_region_if_totally_empty_work(hr, pre_used, cleared_h_regions,
-                                    freed_regions, &local_list);
-
-  finish_free_region_work(pre_used, cleared_h_regions, freed_regions,
-                          &local_list);
-  return pre_used;
-}
-
-void
-G1CollectedHeap::free_region_if_totally_empty_work(HeapRegion* hr,
-                                                   size_t& pre_used,
-                                                   size_t& cleared_h,
-                                                   size_t& freed_regions,
-                                                   UncleanRegionList* list,
-                                                   bool par) {
-  assert(!hr->continuesHumongous(), "should have filtered these out");
-  size_t res = 0;
-  if (hr->used() > 0 && hr->garbage_bytes() == hr->used() &&
-      !hr->is_young()) {
-    if (G1PolicyVerbose > 0)
-      gclog_or_tty->print_cr("Freeing empty region "PTR_FORMAT "(" SIZE_FORMAT " bytes)"
-                                                                               " during cleanup", hr, hr->used());
-    free_region_work(hr, pre_used, cleared_h, freed_regions, list, par);
-  }
-}
-
 // FIXME: both this and shrink could probably be more efficient by
 // doing one "VirtualSpace::expand_by" call rather than several.
 void G1CollectedHeap::expand(size_t expand_bytes) {
@@ -1725,19 +1801,7 @@
 
       // Add it to the HeapRegionSeq.
       _hrs->insert(hr);
-      // Set the zero-fill state, according to whether it's already
-      // zeroed.
-      {
-        MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-        if (is_zeroed) {
-          hr->set_zero_fill_complete();
-          put_free_region_on_list_locked(hr);
-        } else {
-          hr->set_zero_fill_needed();
-          put_region_on_unclean_list_locked(hr);
-        }
-      }
-      _free_regions++;
+      _free_list.add_as_tail(hr);
       // And we used up an expansion region to create it.
       _expansion_regions--;
       // Tell the cardtable about it.
@@ -1746,6 +1810,7 @@
       _bot_shared->resize(_g1_committed.word_size());
     }
   }
+
   if (Verbose && PrintGC) {
     size_t new_mem_size = _g1_storage.committed_size();
     gclog_or_tty->print_cr("Expanding garbage-first heap from %ldK by %ldK to %ldK",
@@ -1770,7 +1835,6 @@
   assert(mr.start() == (HeapWord*)_g1_storage.high(), "Bad shrink!");
 
   _g1_committed.set_end(mr.start());
-  _free_regions -= num_regions_deleted;
   _expansion_regions += num_regions_deleted;
 
   // Tell the cardtable about it.
@@ -1790,10 +1854,17 @@
 }
 
 void G1CollectedHeap::shrink(size_t shrink_bytes) {
+  verify_region_sets_optional();
+
   release_gc_alloc_regions(true /* totally */);
+  // Instead of tearing down / rebuilding the free lists here, we
+  // could instead use the remove_all_pending() method on free_list to
+  // remove only the ones that we need to remove.
   tear_down_region_lists();  // We will rebuild them in a moment.
   shrink_helper(shrink_bytes);
   rebuild_region_lists();
+
+  verify_region_sets_optional();
 }
 
 // Public methods.
@@ -1812,18 +1883,17 @@
   _ref_processor(NULL),
   _process_strong_tasks(new SubTasksDone(G1H_PS_NumElements)),
   _bot_shared(NULL),
-  _par_alloc_during_gc_lock(Mutex::leaf, "par alloc during GC lock"),
   _objs_with_preserved_marks(NULL), _preserved_marks_of_objs(NULL),
   _evac_failure_scan_stack(NULL) ,
   _mark_in_progress(false),
-  _cg1r(NULL), _czft(NULL), _summary_bytes_used(0),
+  _cg1r(NULL), _summary_bytes_used(0),
   _cur_alloc_region(NULL),
   _refine_cte_cl(NULL),
-  _free_region_list(NULL), _free_region_list_size(0),
-  _free_regions(0),
   _full_collection(false),
-  _unclean_region_list(),
-  _unclean_regions_coming(false),
+  _free_list("Master Free List"),
+  _secondary_free_list("Secondary Free List"),
+  _humongous_set("Master Humongous Set"),
+  _free_regions_coming(false),
   _young_list(new YoungList(this)),
   _gc_time_stamp(0),
   _surviving_young_words(NULL),
@@ -1944,8 +2014,6 @@
 
   _expansion_regions = max_byte_size/HeapRegion::GrainBytes;
 
-  _num_humongous_regions = 0;
-
   // Create the gen rem set (and barrier set) for the entire reserved region.
   _rem_set = collector_policy()->create_rem_set(_reserved, 2);
   set_barrier_set(rem_set()->bs());
@@ -1990,6 +2058,8 @@
   guarantee((size_t) HeapRegion::CardsPerRegion < max_cards_per_region,
             "too many cards per region");
 
+  HeapRegionSet::set_unrealistically_long_length(max_regions() + 1);
+
   _bot_shared = new G1BlockOffsetSharedArray(_reserved,
                                              heap_word_size(init_byte_size));
 
@@ -2014,11 +2084,6 @@
   _cm       = new ConcurrentMark(heap_rs, (int) max_regions());
   _cmThread = _cm->cmThread();
 
-  // ...and the concurrent zero-fill thread, if necessary.
-  if (G1ConcZeroFill) {
-    _czft = new ConcurrentZFThread();
-  }
-
   // Initialize the from_card cache structure of HeapRegionRemSet.
   HeapRegionRemSet::init_heap(max_regions());
 
@@ -2192,7 +2257,7 @@
 #endif // PRODUCT
 
 size_t G1CollectedHeap::unsafe_max_alloc() {
-  if (_free_regions > 0) return HeapRegion::GrainBytes;
+  if (free_regions() > 0) return HeapRegion::GrainBytes;
   // otherwise, is there space in the current allocation region?
 
   // We need to store the current allocation region in a local variable
@@ -2272,8 +2337,7 @@
 }
 
 void G1CollectedHeap::collect_as_vm_thread(GCCause::Cause cause) {
-  assert(Thread::current()->is_VM_thread(), "Precondition#1");
-  assert(Heap_lock->is_locked(), "Precondition#2");
+  assert_at_safepoint(true /* should_be_vm_thread */);
   GCCauseSetter gcs(this, cause);
   switch (cause) {
     case GCCause::_heap_inspection:
@@ -2296,12 +2360,6 @@
   {
     MutexLocker ml(Heap_lock);
 
-    // Don't want to do a GC until cleanup is completed. This
-    // limitation will be removed in the near future when the
-    // operation of the free region list is revamped as part of
-    // CR 6977804.
-    wait_for_cleanup_complete();
-
     // Read the GC count while holding the Heap_lock
     gc_count_before = SharedHeap::heap()->total_collections();
     full_gc_count_before = SharedHeap::heap()->total_full_collections();
@@ -2680,10 +2738,6 @@
   }
 }
 
-bool G1CollectedHeap::allocs_are_zero_filled() {
-  return false;
-}
-
 size_t G1CollectedHeap::large_typearray_limit() {
   // FIXME
   return HeapRegion::GrainBytes/HeapWordSize;
@@ -2698,7 +2752,6 @@
   return 0;
 }
 
-
 void G1CollectedHeap::prepare_for_verify() {
   if (SafepointSynchronize::is_at_safepoint() || ! UseTLAB) {
     ensure_parsability(false);
@@ -2909,7 +2962,9 @@
                          &rootsCl);
     bool failures = rootsCl.failures();
     rem_set()->invalidate(perm_gen()->used_region(), false);
-    if (!silent) { gclog_or_tty->print("heapRegions "); }
+    if (!silent) { gclog_or_tty->print("HeapRegionSets "); }
+    verify_region_sets();
+    if (!silent) { gclog_or_tty->print("HeapRegions "); }
     if (GCParallelVerificationEnabled && ParallelGCThreads > 1) {
       assert(check_heap_region_claim_values(HeapRegion::InitialClaimValue),
              "sanity check");
@@ -2937,7 +2992,7 @@
         failures = true;
       }
     }
-    if (!silent) gclog_or_tty->print("remset ");
+    if (!silent) gclog_or_tty->print("RemSet ");
     rem_set()->verify();
 
     if (failures) {
@@ -3008,15 +3063,10 @@
   if (G1CollectedHeap::use_parallel_gc_threads()) {
     workers()->print_worker_threads_on(st);
   }
-
   _cmThread->print_on(st);
   st->cr();
-
   _cm->print_worker_threads_on(st);
-
   _cg1r->print_worker_threads_on(st);
-
-  _czft->print_on(st);
   st->cr();
 }
 
@@ -3026,7 +3076,6 @@
   }
   tc->do_thread(_cmThread);
   _cg1r->threads_do(tc);
-  tc->do_thread(_czft);
 }
 
 void G1CollectedHeap::print_tracing_info() const {
@@ -3042,15 +3091,10 @@
   if (G1SummarizeConcMark) {
     concurrent_mark()->print_summary_info();
   }
-  if (G1SummarizeZFStats) {
-    ConcurrentZFThread::print_summary_info();
-  }
   g1_policy()->print_yg_surv_rate_info();
-
   SpecializationStats::print();
 }
 
-
 int G1CollectedHeap::addr_to_arena_id(void* addr) const {
   HeapRegion* hr = heap_region_containing(addr);
   if (hr == NULL) {
@@ -3249,6 +3293,9 @@
 
 bool
 G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
+  assert_at_safepoint(true /* should_be_vm_thread */);
+  guarantee(!is_gc_active(), "collection is not reentrant");
+
   if (GC_locker::check_active_before_gc()) {
     return false;
   }
@@ -3260,6 +3307,8 @@
     Universe::print_heap_before_gc();
   }
 
+  verify_region_sets_optional();
+
   {
     // This call will decide whether this pause is an initial-mark
     // pause. If it is, during_initial_mark_pause() will return true
@@ -3290,10 +3339,16 @@
 
     TraceMemoryManagerStats tms(false /* fullGC */);
 
-    assert(SafepointSynchronize::is_at_safepoint(), "should be at safepoint");
-    assert(Thread::current() == VMThread::vm_thread(), "should be in vm thread");
-    guarantee(!is_gc_active(), "collection is not reentrant");
-    assert(regions_accounted_for(), "Region leakage!");
+    // If there are any free regions available on the secondary_free_list
+    // make sure we append them to the free_list. However, we don't
+    // have to wait for the rest of the cleanup operation to
+    // finish. If it's still going on that's OK. If we run out of
+    // regions, the region allocation code will check the
+    // secondary_free_list and potentially wait if more free regions
+    // are coming (see new_region_try_secondary_free_list()).
+    if (!G1StressConcRegionFreeing) {
+      append_secondary_free_list_if_not_empty();
+    }
 
     increment_gc_time_stamp();
 
@@ -3373,8 +3428,6 @@
       // progress, this will be zero.
       _cm->set_oops_do_bound();
 
-      assert(regions_accounted_for(), "Region leakage.");
-
       if (mark_in_progress())
         concurrent_mark()->newCSet();
 
@@ -3470,8 +3523,6 @@
       g1_policy()->record_pause_time_ms(pause_time_ms);
       g1_policy()->record_collection_pause_end();
 
-      assert(regions_accounted_for(), "Region leakage.");
-
       MemoryService::track_memory_usage();
 
       if (VerifyAfterGC && total_collections() >= VerifyGCStartAt) {
@@ -3502,8 +3553,6 @@
       gc_epilogue(false);
     }
 
-    assert(verify_region_lists(), "Bad region lists.");
-
     if (ExitAfterGCNum > 0 && total_collections() == ExitAfterGCNum) {
       gclog_or_tty->print_cr("Stopping after GC #%d", ExitAfterGCNum);
       print_tracing_info();
@@ -3511,6 +3560,8 @@
     }
   }
 
+  verify_region_sets_optional();
+
   TASKQUEUE_STATS_ONLY(if (ParallelGCVerbose) print_taskqueue_stats());
   TASKQUEUE_STATS_ONLY(reset_taskqueue_stats());
 
@@ -3617,7 +3668,7 @@
 
 void G1CollectedHeap::push_gc_alloc_region(HeapRegion* hr) {
   assert(Thread::current()->is_VM_thread() ||
-         par_alloc_during_gc_lock()->owned_by_self(), "Precondition");
+         FreeList_lock->owned_by_self(), "Precondition");
   assert(!hr->is_gc_alloc_region() && !hr->in_collection_set(),
          "Precondition.");
   hr->set_is_gc_alloc_region(true);
@@ -3639,7 +3690,7 @@
 #endif // G1_DEBUG
 
 void G1CollectedHeap::forget_alloc_region_list() {
-  assert(Thread::current()->is_VM_thread(), "Precondition");
+  assert_at_safepoint(true /* should_be_vm_thread */);
   while (_gc_alloc_region_list != NULL) {
     HeapRegion* r = _gc_alloc_region_list;
     assert(r->is_gc_alloc_region(), "Invariant.");
@@ -3659,9 +3710,6 @@
         _young_list->add_survivor_region(r);
       }
     }
-    if (r->is_empty()) {
-      ++_free_regions;
-    }
   }
 #ifdef G1_DEBUG
   FindGCAllocRegion fa;
@@ -3714,7 +3762,7 @@
 
     if (alloc_region == NULL) {
       // we will get a new GC alloc region
-      alloc_region = newAllocRegionWithExpansion(ap, 0);
+      alloc_region = new_gc_alloc_region(ap, 0);
     } else {
       // the region was retained from the last collection
       ++_gc_alloc_region_counts[ap];
@@ -3769,11 +3817,9 @@
       set_gc_alloc_region(ap, NULL);
 
       if (r->is_empty()) {
-        // we didn't actually allocate anything in it; let's just put
-        // it on the free list
-        MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-        r->set_zero_fill_complete();
-        put_free_region_on_list_locked(r);
+        // We didn't actually allocate anything in it; let's just put
+        // it back on the free list.
+        _free_list.add_as_tail(r);
       } else if (_retain_gc_alloc_region[ap] && !totally) {
         // retain it so that we can use it at the beginning of the next GC
         _retained_gc_alloc_regions[ap] = r;
@@ -4128,8 +4174,6 @@
 
   HeapWord* block = alloc_region->par_allocate(word_size);
   if (block == NULL) {
-    MutexLockerEx x(par_alloc_during_gc_lock(),
-                    Mutex::_no_safepoint_check_flag);
     block = allocate_during_gc_slow(purpose, alloc_region, true, word_size);
   }
   return block;
@@ -4158,6 +4202,12 @@
          err_msg("we should not be seeing humongous allocation requests "
                  "during GC, word_size = "SIZE_FORMAT, word_size));
 
+  // We need to make sure we serialize calls to this method. Given
+  // that the FreeList_lock guards accesses to the free_list anyway,
+  // and we need to potentially remove a region from it, we'll use it
+  // to protect the whole call.
+  MutexLockerEx x(FreeList_lock, Mutex::_no_safepoint_check_flag);
+
   HeapWord* block = NULL;
   // In the parallel case, a previous thread to obtain the lock may have
   // already assigned a new gc_alloc_region.
@@ -4203,7 +4253,7 @@
   }
 
   // Now allocate a new region for allocation.
-  alloc_region = newAllocRegionWithExpansion(purpose, word_size, false /*zero_filled*/);
+  alloc_region = new_gc_alloc_region(purpose, word_size);
 
   // let the caller handle alloc failure
   if (alloc_region != NULL) {
@@ -4211,9 +4261,6 @@
     assert(check_gc_alloc_regions(), "alloc regions messed up");
     assert(alloc_region->saved_mark_at_top(),
            "Mark should have been saved already.");
-    // We used to assert that the region was zero-filled here, but no
-    // longer.
-
     // This must be done last: once it's installed, other regions may
     // allocate in it (without holding the lock.)
     set_gc_alloc_region(purpose, alloc_region);
@@ -4878,91 +4925,91 @@
   COMPILER2_PRESENT(DerivedPointerTable::update_pointers());
 }
 
-void G1CollectedHeap::free_region(HeapRegion* hr) {
-  size_t pre_used = 0;
-  size_t cleared_h_regions = 0;
-  size_t freed_regions = 0;
-  UncleanRegionList local_list;
-
-  HeapWord* start = hr->bottom();
-  HeapWord* end   = hr->prev_top_at_mark_start();
-  size_t used_bytes = hr->used();
-  size_t live_bytes = hr->max_live_bytes();
-  if (used_bytes > 0) {
-    guarantee( live_bytes <= used_bytes, "invariant" );
-  } else {
-    guarantee( live_bytes == 0, "invariant" );
-  }
-
-  size_t garbage_bytes = used_bytes - live_bytes;
-  if (garbage_bytes > 0)
-    g1_policy()->decrease_known_garbage_bytes(garbage_bytes);
-
-  free_region_work(hr, pre_used, cleared_h_regions, freed_regions,
-                   &local_list);
-  finish_free_region_work(pre_used, cleared_h_regions, freed_regions,
-                          &local_list);
-}
-
-void
-G1CollectedHeap::free_region_work(HeapRegion* hr,
-                                  size_t& pre_used,
-                                  size_t& cleared_h_regions,
-                                  size_t& freed_regions,
-                                  UncleanRegionList* list,
-                                  bool par) {
-  pre_used += hr->used();
-  if (hr->isHumongous()) {
-    assert(hr->startsHumongous(),
-           "Only the start of a humongous region should be freed.");
-    int ind = _hrs->find(hr);
-    assert(ind != -1, "Should have an index.");
-    // Clear the start region.
-    hr->hr_clear(par, true /*clear_space*/);
-    list->insert_before_head(hr);
-    cleared_h_regions++;
-    freed_regions++;
-    // Clear any continued regions.
-    ind++;
-    while ((size_t)ind < n_regions()) {
-      HeapRegion* hrc = _hrs->at(ind);
-      if (!hrc->continuesHumongous()) break;
-      // Otherwise, does continue the H region.
-      assert(hrc->humongous_start_region() == hr, "Huh?");
-      hrc->hr_clear(par, true /*clear_space*/);
-      cleared_h_regions++;
-      freed_regions++;
-      list->insert_before_head(hrc);
-      ind++;
+void G1CollectedHeap::free_region_if_totally_empty(HeapRegion* hr,
+                                     size_t* pre_used,
+                                     FreeRegionList* free_list,
+                                     HumongousRegionSet* humongous_proxy_set,
+                                     bool par) {
+  if (hr->used() > 0 && hr->max_live_bytes() == 0 && !hr->is_young()) {
+    if (hr->isHumongous()) {
+      assert(hr->startsHumongous(), "we should only see starts humongous");
+      free_humongous_region(hr, pre_used, free_list, humongous_proxy_set, par);
+    } else {
+      free_region(hr, pre_used, free_list, par);
     }
-  } else {
-    hr->hr_clear(par, true /*clear_space*/);
-    list->insert_before_head(hr);
-    freed_regions++;
-    // If we're using clear2, this should not be enabled.
-    // assert(!hr->in_cohort(), "Can't be both free and in a cohort.");
   }
 }
 
-void G1CollectedHeap::finish_free_region_work(size_t pre_used,
-                                              size_t cleared_h_regions,
-                                              size_t freed_regions,
-                                              UncleanRegionList* list) {
-  if (list != NULL && list->sz() > 0) {
-    prepend_region_list_on_unclean_list(list);
-  }
-  // Acquire a lock, if we're parallel, to update possibly-shared
-  // variables.
-  Mutex* lock = (n_par_threads() > 0) ? ParGCRareEvent_lock : NULL;
-  {
+void G1CollectedHeap::free_region(HeapRegion* hr,
+                                  size_t* pre_used,
+                                  FreeRegionList* free_list,
+                                  bool par) {
+  assert(!hr->isHumongous(), "this is only for non-humongous regions");
+  assert(!hr->is_empty(), "the region should not be empty");
+  assert(free_list != NULL, "pre-condition");
+
+  *pre_used += hr->used();
+  hr->hr_clear(par, true /* clear_space */);
+  free_list->add_as_tail(hr);
+}
+
+void G1CollectedHeap::free_humongous_region(HeapRegion* hr,
+                                     size_t* pre_used,
+                                     FreeRegionList* free_list,
+                                     HumongousRegionSet* humongous_proxy_set,
+                                     bool par) {
+  assert(hr->startsHumongous(), "this is only for starts humongous regions");
+  assert(free_list != NULL, "pre-condition");
+  assert(humongous_proxy_set != NULL, "pre-condition");
+
+  size_t hr_used = hr->used();
+  size_t hr_capacity = hr->capacity();
+  size_t hr_pre_used = 0;
+  _humongous_set.remove_with_proxy(hr, humongous_proxy_set);
+  hr->set_notHumongous();
+  free_region(hr, &hr_pre_used, free_list, par);
+
+  int i = hr->hrs_index() + 1;
+  size_t num = 1;
+  while ((size_t) i < n_regions()) {
+    HeapRegion* curr_hr = _hrs->at(i);
+    if (!curr_hr->continuesHumongous()) {
+      break;
+    }
+    curr_hr->set_notHumongous();
+    free_region(curr_hr, &hr_pre_used, free_list, par);
+    num += 1;
+    i += 1;
+  }
+  assert(hr_pre_used == hr_used,
+         err_msg("hr_pre_used: "SIZE_FORMAT" and hr_used: "SIZE_FORMAT" "
+                 "should be the same", hr_pre_used, hr_used));
+  *pre_used += hr_pre_used;
+}
+
+void G1CollectedHeap::update_sets_after_freeing_regions(size_t pre_used,
+                                       FreeRegionList* free_list,
+                                       HumongousRegionSet* humongous_proxy_set,
+                                       bool par) {
+  if (pre_used > 0) {
+    Mutex* lock = (par) ? ParGCRareEvent_lock : NULL;
     MutexLockerEx x(lock, Mutex::_no_safepoint_check_flag);
+    assert(_summary_bytes_used >= pre_used,
+           err_msg("invariant: _summary_bytes_used: "SIZE_FORMAT" "
+                   "should be >= pre_used: "SIZE_FORMAT,
+                   _summary_bytes_used, pre_used));
     _summary_bytes_used -= pre_used;
-    _num_humongous_regions -= (int) cleared_h_regions;
-    _free_regions += freed_regions;
+  }
+  if (free_list != NULL && !free_list->is_empty()) {
+    MutexLockerEx x(FreeList_lock, Mutex::_no_safepoint_check_flag);
+    _free_list.add_as_tail(free_list);
+  }
+  if (humongous_proxy_set != NULL && !humongous_proxy_set->is_empty()) {
+    MutexLockerEx x(OldSets_lock, Mutex::_no_safepoint_check_flag);
+    _humongous_set.update_from_proxy(humongous_proxy_set);
   }
 }
 
-
 void G1CollectedHeap::dirtyCardsForYoungRegions(CardTableModRefBS* ct_bs, HeapRegion* list) {
   while (list != NULL) {
     guarantee( list->is_young(), "invariant" );
@@ -5085,6 +5132,9 @@
 }
 
 void G1CollectedHeap::free_collection_set(HeapRegion* cs_head) {
+  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;
 
@@ -5103,6 +5153,8 @@
   size_t rs_lengths = 0;
 
   while (cur != NULL) {
+    assert(!is_on_free_list(cur), "sanity");
+
     if (non_young) {
       if (cur->is_young()) {
         double end_sec = os::elapsedTime();
@@ -5113,14 +5165,12 @@
         non_young = false;
       }
     } else {
-      if (!cur->is_on_free_list()) {
-        double end_sec = os::elapsedTime();
-        double elapsed_ms = (end_sec - start_sec) * 1000.0;
-        young_time_ms += elapsed_ms;
-
-        start_sec = os::elapsedTime();
-        non_young = true;
-      }
+      double end_sec = os::elapsedTime();
+      double elapsed_ms = (end_sec - start_sec) * 1000.0;
+      young_time_ms += elapsed_ms;
+
+      start_sec = os::elapsedTime();
+      non_young = true;
     }
 
     rs_lengths += cur->rem_set()->occupied();
@@ -5153,9 +5203,8 @@
 
     if (!cur->evacuation_failed()) {
       // And the region is empty.
-      assert(!cur->is_empty(),
-             "Should not have empty regions in a CS.");
-      free_region(cur);
+      assert(!cur->is_empty(), "Should not have empty regions in a CS.");
+      free_region(cur, &pre_used, &local_free_list, false /* par */);
     } else {
       cur->uninstall_surv_rate_group();
       if (cur->is_young())
@@ -5176,6 +5225,9 @@
   else
     young_time_ms += elapsed_ms;
 
+  update_sets_after_freeing_regions(pre_used, &local_free_list,
+                                    NULL /* humongous_proxy_set */,
+                                    false /* par */);
   policy->record_young_free_cset_time_ms(young_time_ms);
   policy->record_non_young_free_cset_time_ms(non_young_time_ms);
 }
@@ -5201,291 +5253,53 @@
   }
 }
 
-HeapRegion*
-G1CollectedHeap::alloc_region_from_unclean_list_locked(bool zero_filled) {
-  assert(ZF_mon->owned_by_self(), "Precondition");
-  HeapRegion* res = pop_unclean_region_list_locked();
-  if (res != NULL) {
-    assert(!res->continuesHumongous() &&
-           res->zero_fill_state() != HeapRegion::Allocated,
-           "Only free regions on unclean list.");
-    if (zero_filled) {
-      res->ensure_zero_filled_locked();
-      res->set_zero_fill_allocated();
-    }
-  }
-  return res;
-}
-
-HeapRegion* G1CollectedHeap::alloc_region_from_unclean_list(bool zero_filled) {
-  MutexLockerEx zx(ZF_mon, Mutex::_no_safepoint_check_flag);
-  return alloc_region_from_unclean_list_locked(zero_filled);
+void G1CollectedHeap::set_free_regions_coming() {
+  if (G1ConcRegionFreeingVerbose) {
+    gclog_or_tty->print_cr("G1ConcRegionFreeing [cm thread] : "
+                           "setting free regions coming");
+  }
+
+  assert(!free_regions_coming(), "pre-condition");
+  _free_regions_coming = true;
 }
 
-void G1CollectedHeap::put_region_on_unclean_list(HeapRegion* r) {
-  MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-  put_region_on_unclean_list_locked(r);
-  if (should_zf()) ZF_mon->notify_all(); // Wake up ZF thread.
-}
-
-void G1CollectedHeap::set_unclean_regions_coming(bool b) {
-  MutexLockerEx x(Cleanup_mon);
-  set_unclean_regions_coming_locked(b);
-}
-
-void G1CollectedHeap::set_unclean_regions_coming_locked(bool b) {
-  assert(Cleanup_mon->owned_by_self(), "Precondition");
-  _unclean_regions_coming = b;
-  // Wake up mutator threads that might be waiting for completeCleanup to
-  // finish.
-  if (!b) Cleanup_mon->notify_all();
-}
-
-void G1CollectedHeap::wait_for_cleanup_complete() {
-  assert_not_at_safepoint();
-  MutexLockerEx x(Cleanup_mon);
-  wait_for_cleanup_complete_locked();
-}
-
-void G1CollectedHeap::wait_for_cleanup_complete_locked() {
-  assert(Cleanup_mon->owned_by_self(), "precondition");
-  while (_unclean_regions_coming) {
-    Cleanup_mon->wait();
+void G1CollectedHeap::reset_free_regions_coming() {
+  {
+    assert(free_regions_coming(), "pre-condition");
+    MutexLockerEx x(SecondaryFreeList_lock, Mutex::_no_safepoint_check_flag);
+    _free_regions_coming = false;
+    SecondaryFreeList_lock->notify_all();
+  }
+
+  if (G1ConcRegionFreeingVerbose) {
+    gclog_or_tty->print_cr("G1ConcRegionFreeing [cm thread] : "
+                           "reset free regions coming");
   }
 }
 
-void
-G1CollectedHeap::put_region_on_unclean_list_locked(HeapRegion* r) {
-  assert(ZF_mon->owned_by_self(), "precondition.");
-#ifdef ASSERT
-  if (r->is_gc_alloc_region()) {
-    ResourceMark rm;
-    stringStream region_str;
-    print_on(&region_str);
-    assert(!r->is_gc_alloc_region(), err_msg("Unexpected GC allocation region: %s",
-                                             region_str.as_string()));
-  }
-#endif
-  _unclean_region_list.insert_before_head(r);
-}
-
-void
-G1CollectedHeap::prepend_region_list_on_unclean_list(UncleanRegionList* list) {
-  MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-  prepend_region_list_on_unclean_list_locked(list);
-  if (should_zf()) ZF_mon->notify_all(); // Wake up ZF thread.
-}
-
-void
-G1CollectedHeap::
-prepend_region_list_on_unclean_list_locked(UncleanRegionList* list) {
-  assert(ZF_mon->owned_by_self(), "precondition.");
-  _unclean_region_list.prepend_list(list);
-}
-
-HeapRegion* G1CollectedHeap::pop_unclean_region_list_locked() {
-  assert(ZF_mon->owned_by_self(), "precondition.");
-  HeapRegion* res = _unclean_region_list.pop();
-  if (res != NULL) {
-    // Inform ZF thread that there's a new unclean head.
-    if (_unclean_region_list.hd() != NULL && should_zf())
-      ZF_mon->notify_all();
-  }
-  return res;
-}
-
-HeapRegion* G1CollectedHeap::peek_unclean_region_list_locked() {
-  assert(ZF_mon->owned_by_self(), "precondition.");
-  return _unclean_region_list.hd();
-}
-
-
-bool G1CollectedHeap::move_cleaned_region_to_free_list_locked() {
-  assert(ZF_mon->owned_by_self(), "Precondition");
-  HeapRegion* r = peek_unclean_region_list_locked();
-  if (r != NULL && r->zero_fill_state() == HeapRegion::ZeroFilled) {
-    // Result of below must be equal to "r", since we hold the lock.
-    (void)pop_unclean_region_list_locked();
-    put_free_region_on_list_locked(r);
-    return true;
-  } else {
-    return false;
-  }
-}
-
-bool G1CollectedHeap::move_cleaned_region_to_free_list() {
-  MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-  return move_cleaned_region_to_free_list_locked();
-}
-
-
-void G1CollectedHeap::put_free_region_on_list_locked(HeapRegion* r) {
-  assert(ZF_mon->owned_by_self(), "precondition.");
-  assert(_free_region_list_size == free_region_list_length(), "Inv");
-  assert(r->zero_fill_state() == HeapRegion::ZeroFilled,
-        "Regions on free list must be zero filled");
-  assert(!r->isHumongous(), "Must not be humongous.");
-  assert(r->is_empty(), "Better be empty");
-  assert(!r->is_on_free_list(),
-         "Better not already be on free list");
-  assert(!r->is_on_unclean_list(),
-         "Better not already be on unclean list");
-  r->set_on_free_list(true);
-  r->set_next_on_free_list(_free_region_list);
-  _free_region_list = r;
-  _free_region_list_size++;
-  assert(_free_region_list_size == free_region_list_length(), "Inv");
-}
-
-void G1CollectedHeap::put_free_region_on_list(HeapRegion* r) {
-  MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-  put_free_region_on_list_locked(r);
-}
-
-HeapRegion* G1CollectedHeap::pop_free_region_list_locked() {
-  assert(ZF_mon->owned_by_self(), "precondition.");
-  assert(_free_region_list_size == free_region_list_length(), "Inv");
-  HeapRegion* res = _free_region_list;
-  if (res != NULL) {
-    _free_region_list = res->next_from_free_list();
-    _free_region_list_size--;
-    res->set_on_free_list(false);
-    res->set_next_on_free_list(NULL);
-    assert(_free_region_list_size == free_region_list_length(), "Inv");
-  }
-  return res;
-}
-
-
-HeapRegion* G1CollectedHeap::alloc_free_region_from_lists(bool zero_filled) {
-  // By self, or on behalf of self.
-  assert(Heap_lock->is_locked(), "Precondition");
-  HeapRegion* res = NULL;
-  bool first = true;
-  while (res == NULL) {
-    if (zero_filled || !first) {
-      MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-      res = pop_free_region_list_locked();
-      if (res != NULL) {
-        assert(!res->zero_fill_is_allocated(),
-               "No allocated regions on free list.");
-        res->set_zero_fill_allocated();
-      } else if (!first) {
-        break;  // We tried both, time to return NULL.
-      }
-    }
-
-    if (res == NULL) {
-      res = alloc_region_from_unclean_list(zero_filled);
-    }
-    assert(res == NULL ||
-           !zero_filled ||
-           res->zero_fill_is_allocated(),
-           "We must have allocated the region we're returning");
-    first = false;
-  }
-  return res;
-}
-
-void G1CollectedHeap::remove_allocated_regions_from_lists() {
-  MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-  {
-    HeapRegion* prev = NULL;
-    HeapRegion* cur = _unclean_region_list.hd();
-    while (cur != NULL) {
-      HeapRegion* next = cur->next_from_unclean_list();
-      if (cur->zero_fill_is_allocated()) {
-        // Remove from the list.
-        if (prev == NULL) {
-          (void)_unclean_region_list.pop();
-        } else {
-          _unclean_region_list.delete_after(prev);
-        }
-        cur->set_on_unclean_list(false);
-        cur->set_next_on_unclean_list(NULL);
-      } else {
-        prev = cur;
-      }
-      cur = next;
-    }
-    assert(_unclean_region_list.sz() == unclean_region_list_length(),
-           "Inv");
+void G1CollectedHeap::wait_while_free_regions_coming() {
+  // Most of the time we won't have to wait, so let's do a quick test
+  // first before we take the lock.
+  if (!free_regions_coming()) {
+    return;
+  }
+
+  if (G1ConcRegionFreeingVerbose) {
+    gclog_or_tty->print_cr("G1ConcRegionFreeing [other] : "
+                           "waiting for free regions");
   }
 
   {
-    HeapRegion* prev = NULL;
-    HeapRegion* cur = _free_region_list;
-    while (cur != NULL) {
-      HeapRegion* next = cur->next_from_free_list();
-      if (cur->zero_fill_is_allocated()) {
-        // Remove from the list.
-        if (prev == NULL) {
-          _free_region_list = cur->next_from_free_list();
-        } else {
-          prev->set_next_on_free_list(cur->next_from_free_list());
-        }
-        cur->set_on_free_list(false);
-        cur->set_next_on_free_list(NULL);
-        _free_region_list_size--;
-      } else {
-        prev = cur;
-      }
-      cur = next;
+    MutexLockerEx x(SecondaryFreeList_lock, Mutex::_no_safepoint_check_flag);
+    while (free_regions_coming()) {
+      SecondaryFreeList_lock->wait(Mutex::_no_safepoint_check_flag);
     }
-    assert(_free_region_list_size == free_region_list_length(), "Inv");
-  }
-}
-
-bool G1CollectedHeap::verify_region_lists() {
-  MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-  return verify_region_lists_locked();
-}
-
-bool G1CollectedHeap::verify_region_lists_locked() {
-  HeapRegion* unclean = _unclean_region_list.hd();
-  while (unclean != NULL) {
-    guarantee(unclean->is_on_unclean_list(), "Well, it is!");
-    guarantee(!unclean->is_on_free_list(), "Well, it shouldn't be!");
-    guarantee(unclean->zero_fill_state() != HeapRegion::Allocated,
-              "Everything else is possible.");
-    unclean = unclean->next_from_unclean_list();
-  }
-  guarantee(_unclean_region_list.sz() == unclean_region_list_length(), "Inv");
-
-  HeapRegion* free_r = _free_region_list;
-  while (free_r != NULL) {
-    assert(free_r->is_on_free_list(), "Well, it is!");
-    assert(!free_r->is_on_unclean_list(), "Well, it shouldn't be!");
-    switch (free_r->zero_fill_state()) {
-    case HeapRegion::NotZeroFilled:
-    case HeapRegion::ZeroFilling:
-      guarantee(false, "Should not be on free list.");
-      break;
-    default:
-      // Everything else is possible.
-      break;
-    }
-    free_r = free_r->next_from_free_list();
-  }
-  guarantee(_free_region_list_size == free_region_list_length(), "Inv");
-  // If we didn't do an assertion...
-  return true;
-}
-
-size_t G1CollectedHeap::free_region_list_length() {
-  assert(ZF_mon->owned_by_self(), "precondition.");
-  size_t len = 0;
-  HeapRegion* cur = _free_region_list;
-  while (cur != NULL) {
-    len++;
-    cur = cur->next_from_free_list();
-  }
-  return len;
-}
-
-size_t G1CollectedHeap::unclean_region_list_length() {
-  assert(ZF_mon->owned_by_self(), "precondition.");
-  return _unclean_region_list.length();
+  }
+
+  if (G1ConcRegionFreeingVerbose) {
+    gclog_or_tty->print_cr("G1ConcRegionFreeing [other] : "
+                           "done waiting for free regions");
+  }
 }
 
 size_t G1CollectedHeap::n_regions() {
@@ -5498,55 +5312,6 @@
     HeapRegion::GrainBytes;
 }
 
-size_t G1CollectedHeap::free_regions() {
-  /* Possibly-expensive assert.
-  assert(_free_regions == count_free_regions(),
-         "_free_regions is off.");
-  */
-  return _free_regions;
-}
-
-bool G1CollectedHeap::should_zf() {
-  return _free_region_list_size < (size_t) G1ConcZFMaxRegions;
-}
-
-class RegionCounter: public HeapRegionClosure {
-  size_t _n;
-public:
-  RegionCounter() : _n(0) {}
-  bool doHeapRegion(HeapRegion* r) {
-    if (r->is_empty()) {
-      assert(!r->isHumongous(), "H regions should not be empty.");
-      _n++;
-    }
-    return false;
-  }
-  int res() { return (int) _n; }
-};
-
-size_t G1CollectedHeap::count_free_regions() {
-  RegionCounter rc;
-  heap_region_iterate(&rc);
-  size_t n = rc.res();
-  if (_cur_alloc_region != NULL && _cur_alloc_region->is_empty())
-    n--;
-  return n;
-}
-
-size_t G1CollectedHeap::count_free_regions_list() {
-  size_t n = 0;
-  size_t o = 0;
-  ZF_mon->lock_without_safepoint_check();
-  HeapRegion* cur = _free_region_list;
-  while (cur != NULL) {
-    cur = cur->next_from_free_list();
-    n++;
-  }
-  size_t m = unclean_region_list_length();
-  ZF_mon->unlock();
-  return n + m;
-}
-
 void G1CollectedHeap::set_region_short_lived_locked(HeapRegion* hr) {
   assert(heap_lock_held_for_gc(),
               "the heap lock should already be held by or for this thread");
@@ -5618,28 +5383,19 @@
   }
 }
 
-
 // Done at the start of full GC.
 void G1CollectedHeap::tear_down_region_lists() {
-  MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-  while (pop_unclean_region_list_locked() != NULL) ;
-  assert(_unclean_region_list.hd() == NULL && _unclean_region_list.sz() == 0,
-         "Postconditions of loop.");
-  while (pop_free_region_list_locked() != NULL) ;
-  assert(_free_region_list == NULL, "Postcondition of loop.");
-  if (_free_region_list_size != 0) {
-    gclog_or_tty->print_cr("Size is %d.", _free_region_list_size);
-    print_on(gclog_or_tty, true /* extended */);
-  }
-  assert(_free_region_list_size == 0, "Postconditions of loop.");
+  _free_list.remove_all();
 }
 
-
 class RegionResetter: public HeapRegionClosure {
-  G1CollectedHeap* _g1;
-  int _n;
+  G1CollectedHeap* _g1h;
+  FreeRegionList _local_free_list;
+
 public:
-  RegionResetter() : _g1(G1CollectedHeap::heap()), _n(0) {}
+  RegionResetter() : _g1h(G1CollectedHeap::heap()),
+                     _local_free_list("Local Free List for RegionResetter") { }
+
   bool doHeapRegion(HeapRegion* r) {
     if (r->continuesHumongous()) return false;
     if (r->top() > r->bottom()) {
@@ -5647,152 +5403,32 @@
         Copy::fill_to_words(r->top(),
                           pointer_delta(r->end(), r->top()));
       }
-      r->set_zero_fill_allocated();
     } else {
       assert(r->is_empty(), "tautology");
-      _n++;
-      switch (r->zero_fill_state()) {
-        case HeapRegion::NotZeroFilled:
-        case HeapRegion::ZeroFilling:
-          _g1->put_region_on_unclean_list_locked(r);
-          break;
-        case HeapRegion::Allocated:
-          r->set_zero_fill_complete();
-          // no break; go on to put on free list.
-        case HeapRegion::ZeroFilled:
-          _g1->put_free_region_on_list_locked(r);
-          break;
-      }
+      _local_free_list.add_as_tail(r);
     }
     return false;
   }
 
-  int getFreeRegionCount() {return _n;}
+  void update_free_lists() {
+    _g1h->update_sets_after_freeing_regions(0, &_local_free_list, NULL,
+                                            false /* par */);
+  }
 };
 
 // Done at the end of full GC.
 void G1CollectedHeap::rebuild_region_lists() {
-  MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
   // This needs to go at the end of the full GC.
   RegionResetter rs;
   heap_region_iterate(&rs);
-  _free_regions = rs.getFreeRegionCount();
-  // Tell the ZF thread it may have work to do.
-  if (should_zf()) ZF_mon->notify_all();
-}
-
-class UsedRegionsNeedZeroFillSetter: public HeapRegionClosure {
-  G1CollectedHeap* _g1;
-  int _n;
-public:
-  UsedRegionsNeedZeroFillSetter() : _g1(G1CollectedHeap::heap()), _n(0) {}
-  bool doHeapRegion(HeapRegion* r) {
-    if (r->continuesHumongous()) return false;
-    if (r->top() > r->bottom()) {
-      // There are assertions in "set_zero_fill_needed()" below that
-      // require top() == bottom(), so this is technically illegal.
-      // We'll skirt the law here, by making that true temporarily.
-      DEBUG_ONLY(HeapWord* save_top = r->top();
-                 r->set_top(r->bottom()));
-      r->set_zero_fill_needed();
-      DEBUG_ONLY(r->set_top(save_top));
-    }
-    return false;
-  }
-};
-
-// Done at the start of full GC.
-void G1CollectedHeap::set_used_regions_to_need_zero_fill() {
-  MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-  // This needs to go at the end of the full GC.
-  UsedRegionsNeedZeroFillSetter rs;
-  heap_region_iterate(&rs);
+  rs.update_free_lists();
 }
 
 void G1CollectedHeap::set_refine_cte_cl_concurrency(bool concurrent) {
   _refine_cte_cl->set_concurrent(concurrent);
 }
 
-#ifndef PRODUCT
-
-class PrintHeapRegionClosure: public HeapRegionClosure {
-public:
-  bool doHeapRegion(HeapRegion *r) {
-    gclog_or_tty->print("Region: "PTR_FORMAT":", r);
-    if (r != NULL) {
-      if (r->is_on_free_list())
-        gclog_or_tty->print("Free ");
-      if (r->is_young())
-        gclog_or_tty->print("Young ");
-      if (r->isHumongous())
-        gclog_or_tty->print("Is Humongous ");
-      r->print();
-    }
-    return false;
-  }
-};
-
-class SortHeapRegionClosure : public HeapRegionClosure {
-  size_t young_regions,free_regions, unclean_regions;
-  size_t hum_regions, count;
-  size_t unaccounted, cur_unclean, cur_alloc;
-  size_t total_free;
-  HeapRegion* cur;
-public:
-  SortHeapRegionClosure(HeapRegion *_cur) : cur(_cur), young_regions(0),
-    free_regions(0), unclean_regions(0),
-    hum_regions(0),
-    count(0), unaccounted(0),
-    cur_alloc(0), total_free(0)
-  {}
-  bool doHeapRegion(HeapRegion *r) {
-    count++;
-    if (r->is_on_free_list()) free_regions++;
-    else if (r->is_on_unclean_list()) unclean_regions++;
-    else if (r->isHumongous())  hum_regions++;
-    else if (r->is_young()) young_regions++;
-    else if (r == cur) cur_alloc++;
-    else unaccounted++;
-    return false;
-  }
-  void print() {
-    total_free = free_regions + unclean_regions;
-    gclog_or_tty->print("%d regions\n", count);
-    gclog_or_tty->print("%d free: free_list = %d unclean = %d\n",
-                        total_free, free_regions, unclean_regions);
-    gclog_or_tty->print("%d humongous %d young\n",
-                        hum_regions, young_regions);
-    gclog_or_tty->print("%d cur_alloc\n", cur_alloc);
-    gclog_or_tty->print("UHOH unaccounted = %d\n", unaccounted);
-  }
-};
-
-void G1CollectedHeap::print_region_counts() {
-  SortHeapRegionClosure sc(_cur_alloc_region);
-  PrintHeapRegionClosure cl;
-  heap_region_iterate(&cl);
-  heap_region_iterate(&sc);
-  sc.print();
-  print_region_accounting_info();
-};
-
-bool G1CollectedHeap::regions_accounted_for() {
-  // TODO: regions accounting for young/survivor/tenured
-  return true;
-}
-
-bool G1CollectedHeap::print_region_accounting_info() {
-  gclog_or_tty->print_cr("Free regions: %d (count: %d count list %d) (clean: %d unclean: %d).",
-                         free_regions(),
-                         count_free_regions(), count_free_regions_list(),
-                         _free_region_list_size, _unclean_region_list.sz());
-  gclog_or_tty->print_cr("cur_alloc: %d.",
-                         (_cur_alloc_region == NULL ? 0 : 1));
-  gclog_or_tty->print_cr("H regions: %d.", _num_humongous_regions);
-
-  // TODO: check regions accounting for young/survivor/tenured
-  return true;
-}
+#ifdef ASSERT
 
 bool G1CollectedHeap::is_in_closed_subset(const void* p) const {
   HeapRegion* hr = heap_region_containing(p);
@@ -5802,8 +5438,84 @@
     return hr->is_in(p);
   }
 }
-#endif // !PRODUCT
-
-void G1CollectedHeap::g1_unimplemented() {
-  // Unimplemented();
+#endif // ASSERT
+
+class VerifyRegionListsClosure : public HeapRegionClosure {
+private:
+  HumongousRegionSet* _humongous_set;
+  FreeRegionList*     _free_list;
+  size_t              _region_count;
+
+public:
+  VerifyRegionListsClosure(HumongousRegionSet* humongous_set,
+                           FreeRegionList* free_list) :
+    _humongous_set(humongous_set), _free_list(free_list),
+    _region_count(0) { }
+
+  size_t region_count()      { return _region_count;      }
+
+  bool doHeapRegion(HeapRegion* hr) {
+    _region_count += 1;
+
+    if (hr->continuesHumongous()) {
+      return false;
+    }
+
+    if (hr->is_young()) {
+      // TODO
+    } else if (hr->startsHumongous()) {
+      _humongous_set->verify_next_region(hr);
+    } else if (hr->is_empty()) {
+      _free_list->verify_next_region(hr);
+    }
+    return false;
+  }
+};
+
+void G1CollectedHeap::verify_region_sets() {
+  assert_heap_locked_or_at_safepoint(true /* should_be_vm_thread */);
+
+  // First, check the explicit lists.
+  _free_list.verify();
+  {
+    // Given that a concurrent operation might be adding regions to
+    // the secondary free list we have to take the lock before
+    // verifying it.
+    MutexLockerEx x(SecondaryFreeList_lock, Mutex::_no_safepoint_check_flag);
+    _secondary_free_list.verify();
+  }
+  _humongous_set.verify();
+
+  // If a concurrent region freeing operation is in progress it will
+  // be difficult to correctly attributed any free regions we come
+  // across to the correct free list given that they might belong to
+  // one of several (free_list, secondary_free_list, any local lists,
+  // etc.). So, if that's the case we will skip the rest of the
+  // verification operation. Alternatively, waiting for the concurrent
+  // operation to complete will have a non-trivial effect on the GC's
+  // operation (no concurrent operation will last longer than the
+  // interval between two calls to verification) and it might hide
+  // any issues that we would like to catch during testing.
+  if (free_regions_coming()) {
+    return;
+  }
+
+  {
+    MutexLockerEx x(SecondaryFreeList_lock, Mutex::_no_safepoint_check_flag);
+    // Make sure we append the secondary_free_list on the free_list so
+    // that all free regions we will come across can be safely
+    // attributed to the free_list.
+    append_secondary_free_list();
+  }
+
+  // Finally, make sure that the region accounting in the lists is
+  // consistent with what we see in the heap.
+  _humongous_set.verify_start();
+  _free_list.verify_start();
+
+  VerifyRegionListsClosure cl(&_humongous_set, &_free_list);
+  heap_region_iterate(&cl);
+
+  _humongous_set.verify_end();
+  _free_list.verify_end();
 }
--- a/src/share/vm/gc_implementation/g1/g1CollectedHeap.hpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/g1CollectedHeap.hpp	Wed Jan 19 19:30:42 2011 -0500
@@ -27,7 +27,7 @@
 
 #include "gc_implementation/g1/concurrentMark.hpp"
 #include "gc_implementation/g1/g1RemSet.hpp"
-#include "gc_implementation/g1/heapRegion.hpp"
+#include "gc_implementation/g1/heapRegionSets.hpp"
 #include "gc_implementation/parNew/parGCAllocBuffer.hpp"
 #include "memory/barrierSet.hpp"
 #include "memory/memRegion.hpp"
@@ -66,8 +66,7 @@
 enum G1GCThreadGroups {
   G1CRGroup = 0,
   G1ZFGroup = 1,
-  G1CMGroup = 2,
-  G1CLGroup = 3
+  G1CMGroup = 2
 };
 
 enum GCAllocPurpose {
@@ -155,6 +154,7 @@
   friend class RefineCardTableEntryClosure;
   friend class G1PrepareCompactClosure;
   friend class RegionSorter;
+  friend class RegionResetter;
   friend class CountRCClosure;
   friend class EvacPopObjClosure;
   friend class G1ParCleanupCTTask;
@@ -178,17 +178,20 @@
   // The maximum part of _g1_storage that has ever been committed.
   MemRegion _g1_max_committed;
 
-  // The number of regions that are completely free.
-  size_t _free_regions;
+  // The master free list. It will satisfy all new region allocations.
+  MasterFreeRegionList      _free_list;
+
+  // The secondary free list which contains regions that have been
+  // freed up during the cleanup process. This will be appended to the
+  // master free list when appropriate.
+  SecondaryFreeRegionList   _secondary_free_list;
+
+  // It keeps track of the humongous regions.
+  MasterHumongousRegionSet  _humongous_set;
 
   // The number of regions we could create by expansion.
   size_t _expansion_regions;
 
-  // Return the number of free regions in the heap (by direct counting.)
-  size_t count_free_regions();
-  // Return the number of free regions on the free and unclean lists.
-  size_t count_free_regions_list();
-
   // The block offset table for the G1 heap.
   G1BlockOffsetSharedArray* _bot_shared;
 
@@ -196,9 +199,6 @@
   // lists, before and after full GC.
   void tear_down_region_lists();
   void rebuild_region_lists();
-  // This sets all non-empty regions to need zero-fill (which they will if
-  // they are empty after full collection.)
-  void set_used_regions_to_need_zero_fill();
 
   // The sequence of all heap regions in the heap.
   HeapRegionSeq* _hrs;
@@ -231,7 +231,7 @@
   // Determines PLAB size for a particular allocation purpose.
   static size_t desired_plab_sz(GCAllocPurpose purpose);
 
-  // When called by par thread, require par_alloc_during_gc_lock() to be held.
+  // When called by par thread, requires the FreeList_lock to be held.
   void push_gc_alloc_region(HeapRegion* hr);
 
   // This should only be called single-threaded.  Undeclares all GC alloc
@@ -294,10 +294,11 @@
   // line number, file, etc.
 
 #define heap_locking_asserts_err_msg(__extra_message)                         \
-  err_msg("%s : Heap_lock %slocked, %sat a safepoint",                        \
+  err_msg("%s : Heap_lock locked: %s, at safepoint: %s, is VM thread: %s",    \
           (__extra_message),                                                  \
-          (!Heap_lock->owned_by_self()) ? "NOT " : "",                        \
-          (!SafepointSynchronize::is_at_safepoint()) ? "NOT " : "")
+          BOOL_TO_STR(Heap_lock->owned_by_self()),                            \
+          BOOL_TO_STR(SafepointSynchronize::is_at_safepoint()),               \
+          BOOL_TO_STR(Thread::current()->is_VM_thread()))
 
 #define assert_heap_locked()                                                  \
   do {                                                                        \
@@ -305,10 +306,11 @@
            heap_locking_asserts_err_msg("should be holding the Heap_lock"));  \
   } while (0)
 
-#define assert_heap_locked_or_at_safepoint()                                  \
+#define assert_heap_locked_or_at_safepoint(__should_be_vm_thread)             \
   do {                                                                        \
     assert(Heap_lock->owned_by_self() ||                                      \
-                                     SafepointSynchronize::is_at_safepoint(), \
+           (SafepointSynchronize::is_at_safepoint() &&                        \
+             ((__should_be_vm_thread) == Thread::current()->is_VM_thread())), \
            heap_locking_asserts_err_msg("should be holding the Heap_lock or " \
                                         "should be at a safepoint"));         \
   } while (0)
@@ -335,9 +337,10 @@
                                    "should not be at a safepoint"));          \
   } while (0)
 
-#define assert_at_safepoint()                                                 \
+#define assert_at_safepoint(__should_be_vm_thread)                            \
   do {                                                                        \
-    assert(SafepointSynchronize::is_at_safepoint(),                           \
+    assert(SafepointSynchronize::is_at_safepoint() &&                         \
+              ((__should_be_vm_thread) == Thread::current()->is_VM_thread()), \
            heap_locking_asserts_err_msg("should be at a safepoint"));         \
   } while (0)
 
@@ -362,31 +365,41 @@
   // The current policy object for the collector.
   G1CollectorPolicy* _g1_policy;
 
-  // Parallel allocation lock to protect the current allocation region.
-  Mutex  _par_alloc_during_gc_lock;
-  Mutex* par_alloc_during_gc_lock() { return &_par_alloc_during_gc_lock; }
+  // This is the second level of trying to allocate a new region. If
+  // new_region_work didn't find a region in the free_list, this call
+  // will check whether there's anything available in the
+  // secondary_free_list and/or wait for more regions to appear in that
+  // list, if _free_regions_coming is set.
+  HeapRegion* new_region_try_secondary_free_list(size_t word_size);
 
-  // If possible/desirable, allocate a new HeapRegion for normal object
-  // allocation sufficient for an allocation of the given "word_size".
-  // If "do_expand" is true, will attempt to expand the heap if necessary
-  // to to satisfy the request.  If "zero_filled" is true, requires a
-  // zero-filled region.
-  // (Returning NULL will trigger a GC.)
-  virtual HeapRegion* newAllocRegion_work(size_t word_size,
-                                          bool do_expand,
-                                          bool zero_filled);
+  // It will try to allocate a single non-humongous HeapRegion
+  // sufficient for an allocation of the given word_size.  If
+  // do_expand is true, it will attempt to expand the heap if
+  // necessary to satisfy the allocation request. Note that word_size
+  // is only used to make sure that we expand sufficiently but, given
+  // that the allocation request is assumed not to be humongous,
+  // having word_size is not strictly necessary (expanding by a single
+  // region will always be sufficient). But let's keep that parameter
+  // in case we need it in the future.
+  HeapRegion* new_region_work(size_t word_size, bool do_expand);
 
-  virtual HeapRegion* newAllocRegion(size_t word_size,
-                                     bool zero_filled = true) {
-    return newAllocRegion_work(word_size, false, zero_filled);
+  // It will try to allocate a new region to be used for allocation by
+  // mutator threads. It will not try to expand the heap if not region
+  // is available.
+  HeapRegion* new_alloc_region(size_t word_size) {
+    return new_region_work(word_size, false /* do_expand */);
   }
-  virtual HeapRegion* newAllocRegionWithExpansion(int purpose,
-                                                  size_t word_size,
-                                                  bool zero_filled = true);
+
+  // It will try to allocate a new region to be used for allocation by
+  // a GC thread. It will try to expand the heap if no region is
+  // available.
+  HeapRegion* new_gc_alloc_region(int purpose, size_t word_size);
+
+  int humongous_obj_allocate_find_first(size_t num_regions, size_t word_size);
 
   // Attempt to allocate an object of the given (very large) "word_size".
   // Returns "NULL" on failure.
-  virtual HeapWord* humongous_obj_allocate(size_t word_size);
+  HeapWord* humongous_obj_allocate(size_t word_size);
 
   // The following two methods, allocate_new_tlab() and
   // mem_allocate(), are the two main entry points from the runtime
@@ -760,20 +773,29 @@
   // Invoke "save_marks" on all heap regions.
   void save_marks();
 
-  // Free a heap region.
-  void free_region(HeapRegion* hr);
-  // A component of "free_region", exposed for 'batching'.
-  // All the params after "hr" are out params: the used bytes of the freed
-  // region(s), the number of H regions cleared, the number of regions
-  // freed, and pointers to the head and tail of a list of freed contig
-  // regions, linked throught the "next_on_unclean_list" field.
-  void free_region_work(HeapRegion* hr,
-                        size_t& pre_used,
-                        size_t& cleared_h,
-                        size_t& freed_regions,
-                        UncleanRegionList* list,
-                        bool par = false);
+  // It frees a non-humongous region by initializing its contents and
+  // adding it to the free list that's passed as a parameter (this is
+  // usually a local list which will be appended to the master free
+  // list later). The used bytes of freed regions are accumulated in
+  // pre_used. If par is true, the region's RSet will not be freed
+  // up. The assumption is that this will be done later.
+  void free_region(HeapRegion* hr,
+                   size_t* pre_used,
+                   FreeRegionList* free_list,
+                   bool par);
 
+  // It frees a humongous region by collapsing it into individual
+  // regions and calling free_region() for each of them. The freed
+  // regions will be added to the free list that's passed as a parameter
+  // (this is usually a local list which will be appended to the
+  // master free list later). The used bytes of freed regions are
+  // accumulated in pre_used. If par is true, the region's RSet will
+  // not be freed up. The assumption is that this will be done later.
+  void free_humongous_region(HeapRegion* hr,
+                             size_t* pre_used,
+                             FreeRegionList* free_list,
+                             HumongousRegionSet* humongous_proxy_set,
+                             bool par);
 
   // The concurrent marker (and the thread it runs in.)
   ConcurrentMark* _cm;
@@ -783,9 +805,6 @@
   // The concurrent refiner.
   ConcurrentG1Refine* _cg1r;
 
-  // The concurrent zero-fill thread.
-  ConcurrentZFThread* _czft;
-
   // The parallel task queues
   RefToScanQueueSet *_task_queues;
 
@@ -877,9 +896,7 @@
 
   SubTasksDone* _process_strong_tasks;
 
-  // List of regions which require zero filling.
-  UncleanRegionList _unclean_region_list;
-  bool _unclean_regions_coming;
+  volatile bool _free_regions_coming;
 
 public:
 
@@ -1002,71 +1019,64 @@
   size_t max_regions();
 
   // The number of regions that are completely free.
-  size_t free_regions();
+  size_t free_regions() {
+    return _free_list.length();
+  }
 
   // The number of regions that are not completely free.
   size_t used_regions() { return n_regions() - free_regions(); }
 
-  // True iff the ZF thread should run.
-  bool should_zf();
-
   // The number of regions available for "regular" expansion.
   size_t expansion_regions() { return _expansion_regions; }
 
-#ifndef PRODUCT
-  bool regions_accounted_for();
-  bool print_region_accounting_info();
-  void print_region_counts();
-#endif
-
-  HeapRegion* alloc_region_from_unclean_list(bool zero_filled);
-  HeapRegion* alloc_region_from_unclean_list_locked(bool zero_filled);
-
-  void put_region_on_unclean_list(HeapRegion* r);
-  void put_region_on_unclean_list_locked(HeapRegion* r);
+  // verify_region_sets() performs verification over the region
+  // lists. It will be compiled in the product code to be used when
+  // necessary (i.e., during heap verification).
+  void verify_region_sets();
 
-  void prepend_region_list_on_unclean_list(UncleanRegionList* list);
-  void prepend_region_list_on_unclean_list_locked(UncleanRegionList* list);
+  // verify_region_sets_optional() is planted in the code for
+  // list verification in non-product builds (and it can be enabled in
+  // product builds by definning HEAP_REGION_SET_FORCE_VERIFY to be 1).
+#if HEAP_REGION_SET_FORCE_VERIFY
+  void verify_region_sets_optional() {
+    verify_region_sets();
+  }
+#else // HEAP_REGION_SET_FORCE_VERIFY
+  void verify_region_sets_optional() { }
+#endif // HEAP_REGION_SET_FORCE_VERIFY
 
-  void set_unclean_regions_coming(bool b);
-  void set_unclean_regions_coming_locked(bool b);
-  // Wait for cleanup to be complete.
-  void wait_for_cleanup_complete();
-  // Like above, but assumes that the calling thread owns the Heap_lock.
-  void wait_for_cleanup_complete_locked();
-
-  // Return the head of the unclean list.
-  HeapRegion* peek_unclean_region_list_locked();
-  // Remove and return the head of the unclean list.
-  HeapRegion* pop_unclean_region_list_locked();
+#ifdef ASSERT
+  bool is_on_free_list(HeapRegion* hr) {
+    return hr->containing_set() == &_free_list;
+  }
 
-  // List of regions which are zero filled and ready for allocation.
-  HeapRegion* _free_region_list;
-  // Number of elements on the free list.
-  size_t _free_region_list_size;
+  bool is_on_humongous_set(HeapRegion* hr) {
+    return hr->containing_set() == &_humongous_set;
+}
+#endif // ASSERT
 
-  // If the head of the unclean list is ZeroFilled, move it to the free
-  // list.
-  bool move_cleaned_region_to_free_list_locked();
-  bool move_cleaned_region_to_free_list();
+  // Wrapper for the region list operations that can be called from
+  // methods outside this class.
 
-  void put_free_region_on_list_locked(HeapRegion* r);
-  void put_free_region_on_list(HeapRegion* r);
+  void secondary_free_list_add_as_tail(FreeRegionList* list) {
+    _secondary_free_list.add_as_tail(list);
+  }
 
-  // Remove and return the head element of the free list.
-  HeapRegion* pop_free_region_list_locked();
+  void append_secondary_free_list() {
+    _free_list.add_as_tail(&_secondary_free_list);
+  }
 
-  // If "zero_filled" is true, we first try the free list, then we try the
-  // unclean list, zero-filling the result.  If "zero_filled" is false, we
-  // first try the unclean list, then the zero-filled list.
-  HeapRegion* alloc_free_region_from_lists(bool zero_filled);
+  void append_secondary_free_list_if_not_empty() {
+    if (!_secondary_free_list.is_empty()) {
+      MutexLockerEx x(SecondaryFreeList_lock, Mutex::_no_safepoint_check_flag);
+      append_secondary_free_list();
+    }
+  }
 
-  // Verify the integrity of the region lists.
-  void remove_allocated_regions_from_lists();
-  bool verify_region_lists();
-  bool verify_region_lists_locked();
-  size_t unclean_region_list_length();
-  size_t free_region_list_length();
+  void set_free_regions_coming();
+  void reset_free_regions_coming();
+  bool free_regions_coming() { return _free_regions_coming; }
+  void wait_while_free_regions_coming();
 
   // Perform a collection of the heap; intended for use in implementing
   // "System.gc".  This probably implies as full a collection as the
@@ -1085,23 +1095,24 @@
   // True iff a evacuation has failed in the most-recent collection.
   bool evacuation_failed() { return _evacuation_failed; }
 
-  // Free a region if it is totally full of garbage.  Returns the number of
-  // bytes freed (0 ==> didn't free it).
-  size_t free_region_if_totally_empty(HeapRegion *hr);
-  void free_region_if_totally_empty_work(HeapRegion *hr,
-                                         size_t& pre_used,
-                                         size_t& cleared_h_regions,
-                                         size_t& freed_regions,
-                                         UncleanRegionList* list,
-                                         bool par = false);
+  // It will free a region if it has allocated objects in it that are
+  // all dead. It calls either free_region() or
+  // free_humongous_region() depending on the type of the region that
+  // is passed to it.
+  void free_region_if_totally_empty(HeapRegion* hr,
+                                    size_t* pre_used,
+                                    FreeRegionList* free_list,
+                                    HumongousRegionSet* humongous_proxy_set,
+                                    bool par);
 
-  // If we've done free region work that yields the given changes, update
-  // the relevant global variables.
-  void finish_free_region_work(size_t pre_used,
-                               size_t cleared_h_regions,
-                               size_t freed_regions,
-                               UncleanRegionList* list);
-
+  // It appends the free list to the master free list and updates the
+  // master humongous list according to the contents of the proxy
+  // list. It also adjusts the total used bytes according to pre_used
+  // (if par is true, it will do so by taking the ParGCRareEvent_lock).
+  void update_sets_after_freeing_regions(size_t pre_used,
+                                       FreeRegionList* free_list,
+                                       HumongousRegionSet* humongous_proxy_set,
+                                       bool par);
 
   // Returns "TRUE" iff "p" points into the allocated area of the heap.
   virtual bool is_in(const void* p) const;
@@ -1314,8 +1325,6 @@
     return true;
   }
 
-  virtual bool allocs_are_zero_filled();
-
   // The boundary between a "large" and "small" array of primitives, in
   // words.
   virtual size_t large_typearray_limit();
@@ -1546,13 +1555,6 @@
 
 protected:
   size_t _max_heap_capacity;
-
-public:
-  // Temporary: call to mark things unimplemented for the G1 heap (e.g.,
-  // MemoryService).  In productization, we can make this assert false
-  // to catch such places (as well as searching for calls to this...)
-  static void g1_unimplemented();
-
 };
 
 #define use_local_bitmaps         1
--- a/src/share/vm/gc_implementation/g1/g1CollectedHeap.inline.hpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/g1CollectedHeap.inline.hpp	Wed Jan 19 19:30:42 2011 -0500
@@ -135,7 +135,7 @@
 
 inline void
 G1CollectedHeap::retire_cur_alloc_region_common(HeapRegion* cur_alloc_region) {
-  assert_heap_locked_or_at_safepoint();
+  assert_heap_locked_or_at_safepoint(true /* should_be_vm_thread */);
   assert(cur_alloc_region != NULL && cur_alloc_region == _cur_alloc_region,
          "pre-condition of the call");
   assert(cur_alloc_region->is_young(),
--- a/src/share/vm/gc_implementation/g1/g1CollectorPolicy.cpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/g1CollectorPolicy.cpp	Wed Jan 19 19:30:42 2011 -0500
@@ -1,5 +1,5 @@
 /*
- * Copyright (c) 2001, 2010, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2001, 2011, Oracle and/or its affiliates. All rights reserved.
  * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
  *
  * This code is free software; you can redistribute it and/or modify it
@@ -2875,8 +2875,6 @@
   // Adjust for expansion and slop.
   max_live_bytes = max_live_bytes + expansion_bytes;
 
-  assert(_g1->regions_accounted_for(), "Region leakage!");
-
   HeapRegion* hr;
   if (in_young_gc_mode()) {
     double young_start_time_sec = os::elapsedTime();
--- a/src/share/vm/gc_implementation/g1/g1MarkSweep.cpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/g1MarkSweep.cpp	Wed Jan 19 19:30:42 2011 -0500
@@ -1,5 +1,5 @@
 /*
- * Copyright (c) 2001, 2010, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2001, 2011, Oracle and/or its affiliates. All rights reserved.
  * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
  *
  * This code is free software; you can redistribute it and/or modify it
@@ -180,26 +180,46 @@
 }
 
 class G1PrepareCompactClosure: public HeapRegionClosure {
+  G1CollectedHeap* _g1h;
   ModRefBarrierSet* _mrbs;
   CompactPoint _cp;
+  size_t _pre_used;
+  FreeRegionList _free_list;
+  HumongousRegionSet _humongous_proxy_set;
 
   void free_humongous_region(HeapRegion* hr) {
-    HeapWord* bot = hr->bottom();
     HeapWord* end = hr->end();
     assert(hr->startsHumongous(),
            "Only the start of a humongous region should be freed.");
-    G1CollectedHeap::heap()->free_region(hr);
+    _g1h->free_humongous_region(hr, &_pre_used, &_free_list,
+                                &_humongous_proxy_set, false /* par */);
+    // Do we also need to do this for the continues humongous regions
+    // we just collapsed?
     hr->prepare_for_compaction(&_cp);
     // Also clear the part of the card table that will be unused after
     // compaction.
-    _mrbs->clear(MemRegion(hr->compaction_top(), hr->end()));
+    _mrbs->clear(MemRegion(hr->compaction_top(), end));
   }
 
 public:
-  G1PrepareCompactClosure(CompactibleSpace* cs) :
+  G1PrepareCompactClosure(CompactibleSpace* cs)
+  : _g1h(G1CollectedHeap::heap()),
+    _mrbs(G1CollectedHeap::heap()->mr_bs()),
     _cp(NULL, cs, cs->initialize_threshold()),
-    _mrbs(G1CollectedHeap::heap()->mr_bs())
-  {}
+    _pre_used(0),
+    _free_list("Local Free List for G1MarkSweep"),
+    _humongous_proxy_set("G1MarkSweep Humongous Proxy Set") { }
+
+  void update_sets() {
+    // We'll recalculate total used bytes and recreate the free list
+    // at the end of the GC, so no point in updating those values here.
+    _g1h->update_sets_after_freeing_regions(0, /* pre_used */
+                                            NULL, /* free_list */
+                                            &_humongous_proxy_set,
+                                            false /* par */);
+    _free_list.remove_all();
+  }
+
   bool doHeapRegion(HeapRegion* hr) {
     if (hr->isHumongous()) {
       if (hr->startsHumongous()) {
@@ -265,6 +285,7 @@
 
   G1PrepareCompactClosure blk(sp);
   g1h->heap_region_iterate(&blk);
+  blk.update_sets();
 
   CompactPoint perm_cp(pg, NULL, NULL);
   pg->prepare_for_compaction(&perm_cp);
--- a/src/share/vm/gc_implementation/g1/g1_globals.hpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/g1_globals.hpp	Wed Jan 19 19:30:42 2011 -0500
@@ -75,21 +75,12 @@
           "(0 means do not periodically generate this info); "              \
           "it also requires -XX:+G1SummarizeRSetStats")                     \
                                                                             \
-  diagnostic(bool, G1SummarizeZFStats, false,                               \
-          "Summarize zero-filling info")                                    \
-                                                                            \
   diagnostic(bool, G1TraceConcRefinement, false,                            \
           "Trace G1 concurrent refinement")                                 \
                                                                             \
   product(intx, G1MarkRegionStackSize, 1024 * 1024,                         \
           "Size of the region stack for concurrent marking.")               \
                                                                             \
-  develop(bool, G1ConcZeroFill, true,                                       \
-          "If true, run concurrent zero-filling thread")                    \
-                                                                            \
-  develop(intx, G1ConcZFMaxRegions, 1,                                      \
-          "Stop zero-filling when # of zf'd regions reaches")               \
-                                                                            \
   develop(bool, G1SATBBarrierPrintNullPreVals, false,                       \
           "If true, count frac of ptr writes with null pre-vals.")          \
                                                                             \
@@ -289,7 +280,20 @@
           "Size of a work unit of cards claimed by a worker thread"         \
           "during RSet scanning.")                                          \
                                                                             \
-  develop(bool, ReduceInitialCardMarksForG1, false,                         \
+  develop(uintx, G1SecondaryFreeListAppendLength, 5,                        \
+          "The number of regions we will add to the secondary free list "   \
+          "at every append operation")                                      \
+                                                                            \
+  develop(bool, G1ConcRegionFreeingVerbose, false,                          \
+          "Enables verboseness during concurrent region freeing")           \
+                                                                            \
+  develop(bool, G1StressConcRegionFreeing, false,                           \
+          "It stresses the concurrent region freeing operation")            \
+                                                                            \
+  develop(uintx, G1StressConcRegionFreeingDelayMillis, 0,                   \
+          "Artificial delay during concurrent region freeing")              \
+                                                                            \
+   develop(bool, ReduceInitialCardMarksForG1, false,                        \
           "When ReduceInitialCardMarks is true, this flag setting "         \
           " controls whether G1 allows the RICM optimization")
 
--- a/src/share/vm/gc_implementation/g1/heapRegion.cpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/heapRegion.cpp	Wed Jan 19 19:30:42 2011 -0500
@@ -23,7 +23,6 @@
  */
 
 #include "precompiled.hpp"
-#include "gc_implementation/g1/concurrentZFThread.hpp"
 #include "gc_implementation/g1/g1BlockOffsetTable.inline.hpp"
 #include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
 #include "gc_implementation/g1/g1OopClosures.inline.hpp"
@@ -348,22 +347,20 @@
 }
 
 void HeapRegion::hr_clear(bool par, bool clear_space) {
-  _humongous_type = NotHumongous;
-  _humongous_start_region = NULL;
+  assert(_humongous_type == NotHumongous,
+         "we should have already filtered out humongous regions");
+  assert(_humongous_start_region == NULL,
+         "we should have already filtered out humongous regions");
+  assert(_end == _orig_end,
+         "we should have already filtered out humongous regions");
+
   _in_collection_set = false;
   _is_gc_alloc_region = false;
 
-  // Age stuff (if parallel, this will be done separately, since it needs
-  // to be sequential).
-  G1CollectedHeap* g1h = G1CollectedHeap::heap();
-
   set_young_index_in_cset(-1);
   uninstall_surv_rate_group();
   set_young_type(NotYoung);
 
-  // In case it had been the start of a humongous sequence, reset its end.
-  set_end(_orig_end);
-
   if (!par) {
     // If this is parallel, this will be done later.
     HeapRegionRemSet* hrrs = rem_set();
@@ -387,6 +384,7 @@
 // </PREDICTION>
 
 void HeapRegion::set_startsHumongous(HeapWord* new_top, HeapWord* new_end) {
+  assert(!isHumongous(), "sanity / pre-condition");
   assert(end() == _orig_end,
          "Should be normal before the humongous object allocation");
   assert(top() == bottom(), "should be empty");
@@ -400,6 +398,7 @@
 }
 
 void HeapRegion::set_continuesHumongous(HeapRegion* first_hr) {
+  assert(!isHumongous(), "sanity / pre-condition");
   assert(end() == _orig_end,
          "Should be normal before the humongous object allocation");
   assert(top() == bottom(), "should be empty");
@@ -409,6 +408,26 @@
   _humongous_start_region = first_hr;
 }
 
+void HeapRegion::set_notHumongous() {
+  assert(isHumongous(), "pre-condition");
+
+  if (startsHumongous()) {
+    assert(top() <= end(), "pre-condition");
+    set_end(_orig_end);
+    if (top() > end()) {
+      // at least one "continues humongous" region after it
+      set_top(end());
+    }
+  } else {
+    // continues humongous
+    assert(end() == _orig_end, "sanity");
+  }
+
+  assert(capacity() == (size_t) HeapRegion::GrainBytes, "pre-condition");
+  _humongous_type = NotHumongous;
+  _humongous_start_region = NULL;
+}
+
 bool HeapRegion::claimHeapRegion(jint claimValue) {
   jint current = _claimed;
   if (current != claimValue) {
@@ -443,15 +462,6 @@
   return low;
 }
 
-void HeapRegion::set_next_on_unclean_list(HeapRegion* r) {
-  assert(r == NULL || r->is_on_unclean_list(), "Malformed unclean list.");
-  _next_in_special_set = r;
-}
-
-void HeapRegion::set_on_unclean_list(bool b) {
-  _is_on_unclean_list = b;
-}
-
 void HeapRegion::initialize(MemRegion mr, bool clear_space, bool mangle_space) {
   G1OffsetTableContigSpace::initialize(mr, false, mangle_space);
   hr_clear(false/*par*/, clear_space);
@@ -469,15 +479,16 @@
     _hrs_index(-1),
     _humongous_type(NotHumongous), _humongous_start_region(NULL),
     _in_collection_set(false), _is_gc_alloc_region(false),
-    _is_on_free_list(false), _is_on_unclean_list(false),
     _next_in_special_set(NULL), _orig_end(NULL),
     _claimed(InitialClaimValue), _evacuation_failed(false),
     _prev_marked_bytes(0), _next_marked_bytes(0), _sort_index(-1),
     _young_type(NotYoung), _next_young_region(NULL),
-    _next_dirty_cards_region(NULL),
-    _young_index_in_cset(-1), _surv_rate_group(NULL), _age_index(-1),
-    _rem_set(NULL), _zfs(NotZeroFilled),
-    _recorded_rs_length(0), _predicted_elapsed_time_ms(0),
+    _next_dirty_cards_region(NULL), _next(NULL), _pending_removal(false),
+#ifdef ASSERT
+    _containing_set(NULL),
+#endif // ASSERT
+     _young_index_in_cset(-1), _surv_rate_group(NULL), _age_index(-1),
+    _rem_set(NULL), _recorded_rs_length(0), _predicted_elapsed_time_ms(0),
     _predicted_bytes_to_copy(0)
 {
   _orig_end = mr.end();
@@ -552,86 +563,6 @@
   oops_in_mr_iterate(MemRegion(bottom(), saved_mark_word()), cl);
 }
 
-#ifdef DEBUG
-HeapWord* HeapRegion::allocate(size_t size) {
-  jint state = zero_fill_state();
-  assert(!G1CollectedHeap::heap()->allocs_are_zero_filled() ||
-         zero_fill_is_allocated(),
-         "When ZF is on, only alloc in ZF'd regions");
-  return G1OffsetTableContigSpace::allocate(size);
-}
-#endif
-
-void HeapRegion::set_zero_fill_state_work(ZeroFillState zfs) {
-  assert(ZF_mon->owned_by_self() ||
-         Universe::heap()->is_gc_active(),
-         "Must hold the lock or be a full GC to modify.");
-#ifdef ASSERT
-  if (top() != bottom() && zfs != Allocated) {
-    ResourceMark rm;
-    stringStream region_str;
-    print_on(&region_str);
-    assert(top() == bottom() || zfs == Allocated,
-           err_msg("Region must be empty, or we must be setting it to allocated. "
-                   "_zfs=%d, zfs=%d, region: %s", _zfs, zfs, region_str.as_string()));
-  }
-#endif
-  _zfs = zfs;
-}
-
-void HeapRegion::set_zero_fill_complete() {
-  set_zero_fill_state_work(ZeroFilled);
-  if (ZF_mon->owned_by_self()) {
-    ZF_mon->notify_all();
-  }
-}
-
-
-void HeapRegion::ensure_zero_filled() {
-  MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-  ensure_zero_filled_locked();
-}
-
-void HeapRegion::ensure_zero_filled_locked() {
-  assert(ZF_mon->owned_by_self(), "Precondition");
-  bool should_ignore_zf = SafepointSynchronize::is_at_safepoint();
-  assert(should_ignore_zf || Heap_lock->is_locked(),
-         "Either we're in a GC or we're allocating a region.");
-  switch (zero_fill_state()) {
-  case HeapRegion::NotZeroFilled:
-    set_zero_fill_in_progress(Thread::current());
-    {
-      ZF_mon->unlock();
-      Copy::fill_to_words(bottom(), capacity()/HeapWordSize);
-      ZF_mon->lock_without_safepoint_check();
-    }
-    // A trap.
-    guarantee(zero_fill_state() == HeapRegion::ZeroFilling
-              && zero_filler() == Thread::current(),
-              "AHA!  Tell Dave D if you see this...");
-    set_zero_fill_complete();
-    // gclog_or_tty->print_cr("Did sync ZF.");
-    ConcurrentZFThread::note_sync_zfs();
-    break;
-  case HeapRegion::ZeroFilling:
-    if (should_ignore_zf) {
-      // We can "break" the lock and take over the work.
-      Copy::fill_to_words(bottom(), capacity()/HeapWordSize);
-      set_zero_fill_complete();
-      ConcurrentZFThread::note_sync_zfs();
-      break;
-    } else {
-      ConcurrentZFThread::wait_for_ZF_completed(this);
-    }
-  case HeapRegion::ZeroFilled:
-    // Nothing to do.
-    break;
-  case HeapRegion::Allocated:
-    guarantee(false, "Should not call on allocated regions.");
-  }
-  assert(zero_fill_state() == HeapRegion::ZeroFilled, "Post");
-}
-
 HeapWord*
 HeapRegion::object_iterate_mem_careful(MemRegion mr,
                                                  ObjectClosure* cl) {
@@ -1010,67 +941,3 @@
   _offsets.set_space(this);
   initialize(mr, !is_zeroed, SpaceDecorator::Mangle);
 }
-
-size_t RegionList::length() {
-  size_t len = 0;
-  HeapRegion* cur = hd();
-  DEBUG_ONLY(HeapRegion* last = NULL);
-  while (cur != NULL) {
-    len++;
-    DEBUG_ONLY(last = cur);
-    cur = get_next(cur);
-  }
-  assert(last == tl(), "Invariant");
-  return len;
-}
-
-void RegionList::insert_before_head(HeapRegion* r) {
-  assert(well_formed(), "Inv");
-  set_next(r, hd());
-  _hd = r;
-  _sz++;
-  if (tl() == NULL) _tl = r;
-  assert(well_formed(), "Inv");
-}
-
-void RegionList::prepend_list(RegionList* new_list) {
-  assert(well_formed(), "Precondition");
-  assert(new_list->well_formed(), "Precondition");
-  HeapRegion* new_tl = new_list->tl();
-  if (new_tl != NULL) {
-    set_next(new_tl, hd());
-    _hd = new_list->hd();
-    _sz += new_list->sz();
-    if (tl() == NULL) _tl = new_list->tl();
-  } else {
-    assert(new_list->hd() == NULL && new_list->sz() == 0, "Inv");
-  }
-  assert(well_formed(), "Inv");
-}
-
-void RegionList::delete_after(HeapRegion* r) {
-  assert(well_formed(), "Precondition");
-  HeapRegion* next = get_next(r);
-  assert(r != NULL, "Precondition");
-  HeapRegion* next_tl = get_next(next);
-  set_next(r, next_tl);
-  dec_sz();
-  if (next == tl()) {
-    assert(next_tl == NULL, "Inv");
-    _tl = r;
-  }
-  assert(well_formed(), "Inv");
-}
-
-HeapRegion* RegionList::pop() {
-  assert(well_formed(), "Inv");
-  HeapRegion* res = hd();
-  if (res != NULL) {
-    _hd = get_next(res);
-    _sz--;
-    set_next(res, NULL);
-    if (sz() == 0) _tl = NULL;
-  }
-  assert(well_formed(), "Inv");
-  return res;
-}
--- a/src/share/vm/gc_implementation/g1/heapRegion.hpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/heapRegion.hpp	Wed Jan 19 19:30:42 2011 -0500
@@ -50,6 +50,11 @@
 class HeapRegionRemSet;
 class HeapRegionRemSetIterator;
 class HeapRegion;
+class HeapRegionSetBase;
+
+#define HR_FORMAT "%d:["PTR_FORMAT","PTR_FORMAT","PTR_FORMAT"]"
+#define HR_FORMAT_PARAMS(__hr) (__hr)->hrs_index(), (__hr)->bottom(), \
+                               (__hr)->top(), (__hr)->end()
 
 // A dirty card to oop closure for heap regions. It
 // knows how to get the G1 heap and how to use the bitmap
@@ -227,12 +232,6 @@
   // True iff the region is in current collection_set.
   bool _in_collection_set;
 
-    // True iff the region is on the unclean list, waiting to be zero filled.
-  bool _is_on_unclean_list;
-
-  // True iff the region is on the free list, ready for allocation.
-  bool _is_on_free_list;
-
   // Is this or has it been an allocation region in the current collection
   // pause.
   bool _is_gc_alloc_region;
@@ -254,6 +253,13 @@
   // Next region whose cards need cleaning
   HeapRegion* _next_dirty_cards_region;
 
+  // Fields used by the HeapRegionSetBase class and subclasses.
+  HeapRegion* _next;
+#ifdef ASSERT
+  HeapRegionSetBase* _containing_set;
+#endif // ASSERT
+  bool _pending_removal;
+
   // For parallel heapRegion traversal.
   jint _claimed;
 
@@ -305,10 +311,6 @@
     _top_at_conc_mark_count = bot;
   }
 
-  jint _zfs;  // A member of ZeroFillState.  Protected by ZF_lock.
-  Thread* _zero_filler; // If _zfs is ZeroFilling, the thread that (last)
-                        // made it so.
-
   void set_young_type(YoungType new_type) {
     //assert(_young_type != new_type, "setting the same type" );
     // TODO: add more assertions here
@@ -362,16 +364,6 @@
     RebuildRSClaimValue   = 5
   };
 
-  // Concurrent refinement requires contiguous heap regions (in which TLABs
-  // might be allocated) to be zero-filled.  Each region therefore has a
-  // zero-fill-state.
-  enum ZeroFillState {
-    NotZeroFilled,
-    ZeroFilling,
-    ZeroFilled,
-    Allocated
-  };
-
   inline HeapWord* par_allocate_no_bot_updates(size_t word_size) {
     assert(is_young(), "we can only skip BOT updates on young regions");
     return ContiguousSpace::par_allocate(word_size);
@@ -456,6 +448,9 @@
   // which this region will be part of.
   void set_continuesHumongous(HeapRegion* first_hr);
 
+  // Unsets the humongous-related fields on the region.
+  void set_notHumongous();
+
   // If the region has a remembered set, return a pointer to it.
   HeapRegionRemSet* rem_set() const {
     return _rem_set;
@@ -502,45 +497,56 @@
     _next_in_special_set = r;
   }
 
-  bool is_on_free_list() {
-    return _is_on_free_list;
-  }
+  // Methods used by the HeapRegionSetBase class and subclasses.
 
-  void set_on_free_list(bool b) {
-    _is_on_free_list = b;
-  }
+  // Getter and setter for the next field used to link regions into
+  // linked lists.
+  HeapRegion* next()              { return _next; }
+
+  void set_next(HeapRegion* next) { _next = next; }
 
-  HeapRegion* next_from_free_list() {
-    assert(is_on_free_list(),
-           "Should only invoke on free space.");
-    assert(_next_in_special_set == NULL ||
-           _next_in_special_set->is_on_free_list(),
-           "Malformed Free List.");
-    return _next_in_special_set;
-  }
+  // Every region added to a set is tagged with a reference to that
+  // set. This is used for doing consistency checking to make sure that
+  // the contents of a set are as they should be and it's only
+  // available in non-product builds.
+#ifdef ASSERT
+  void set_containing_set(HeapRegionSetBase* containing_set) {
+    assert((containing_set == NULL && _containing_set != NULL) ||
+           (containing_set != NULL && _containing_set == NULL),
+           err_msg("containing_set: "PTR_FORMAT" "
+                   "_containing_set: "PTR_FORMAT,
+                   containing_set, _containing_set));
+
+    _containing_set = containing_set;
+}
 
-  void set_next_on_free_list(HeapRegion* r) {
-    assert(r == NULL || r->is_on_free_list(), "Malformed free list.");
-    _next_in_special_set = r;
-  }
+  HeapRegionSetBase* containing_set() { return _containing_set; }
+#else // ASSERT
+  void set_containing_set(HeapRegionSetBase* containing_set) { }
 
-  bool is_on_unclean_list() {
-    return _is_on_unclean_list;
-  }
+  // containing_set() is only used in asserts so there's not reason
+  // to provide a dummy version of it.
+#endif // ASSERT
 
-  void set_on_unclean_list(bool b);
+  // If we want to remove regions from a list in bulk we can simply tag
+  // them with the pending_removal tag and call the
+  // remove_all_pending() method on the list.
 
-  HeapRegion* next_from_unclean_list() {
-    assert(is_on_unclean_list(),
-           "Should only invoke on unclean space.");
-    assert(_next_in_special_set == NULL ||
-           _next_in_special_set->is_on_unclean_list(),
-           "Malformed unclean List.");
-    return _next_in_special_set;
+  bool pending_removal() { return _pending_removal; }
+
+  void set_pending_removal(bool pending_removal) {
+    // We can only set pending_removal to true, if it's false and the
+    // region belongs to a set.
+    assert(!pending_removal ||
+           (!_pending_removal && containing_set() != NULL), "pre-condition");
+    // We can only set pending_removal to false, if it's true and the
+    // region does not belong to a set.
+    assert( pending_removal ||
+           ( _pending_removal && containing_set() == NULL), "pre-condition");
+
+    _pending_removal = pending_removal;
   }
 
-  void set_next_on_unclean_list(HeapRegion* r);
-
   HeapRegion* get_next_young_region() { return _next_young_region; }
   void set_next_young_region(HeapRegion* hr) {
     _next_young_region = hr;
@@ -559,11 +565,6 @@
 
   void initialize(MemRegion mr, bool clear_space, bool mangle_space);
 
-  // Ensure that "this" is zero-filled.
-  void ensure_zero_filled();
-  // This one requires that the calling thread holds ZF_mon.
-  void ensure_zero_filled_locked();
-
   // Get the start of the unmarked area in this region.
   HeapWord* prev_top_at_mark_start() const { return _prev_top_at_mark_start; }
   HeapWord* next_top_at_mark_start() const { return _next_top_at_mark_start; }
@@ -798,36 +799,6 @@
   // "end" of the region if there is no such block.
   HeapWord* next_block_start_careful(HeapWord* addr);
 
-  // Returns the zero-fill-state of the current region.
-  ZeroFillState zero_fill_state() { return (ZeroFillState)_zfs; }
-  bool zero_fill_is_allocated() { return _zfs == Allocated; }
-  Thread* zero_filler() { return _zero_filler; }
-
-  // Indicate that the contents of the region are unknown, and therefore
-  // might require zero-filling.
-  void set_zero_fill_needed() {
-    set_zero_fill_state_work(NotZeroFilled);
-  }
-  void set_zero_fill_in_progress(Thread* t) {
-    set_zero_fill_state_work(ZeroFilling);
-    _zero_filler = t;
-  }
-  void set_zero_fill_complete();
-  void set_zero_fill_allocated() {
-    set_zero_fill_state_work(Allocated);
-  }
-
-  void set_zero_fill_state_work(ZeroFillState zfs);
-
-  // This is called when a full collection shrinks the heap.
-  // We want to set the heap region to a value which says
-  // it is no longer part of the heap.  For now, we'll let "NotZF" fill
-  // that role.
-  void reset_zero_fill() {
-    set_zero_fill_state_work(NotZeroFilled);
-    _zero_filler = NULL;
-  }
-
   size_t recorded_rs_length() const        { return _recorded_rs_length; }
   double predicted_elapsed_time_ms() const { return _predicted_elapsed_time_ms; }
   size_t predicted_bytes_to_copy() const   { return _predicted_bytes_to_copy; }
@@ -866,10 +837,6 @@
 
   // Override; it uses the "prev" marking information
   virtual void verify(bool allow_dirty) const;
-
-#ifdef DEBUG
-  HeapWord* allocate(size_t size);
-#endif
 };
 
 // HeapRegionClosure is used for iterating over regions.
@@ -892,113 +859,6 @@
   bool complete() { return _complete; }
 };
 
-// A linked lists of heap regions.  It leaves the "next" field
-// unspecified; that's up to subtypes.
-class RegionList VALUE_OBJ_CLASS_SPEC {
-protected:
-  virtual HeapRegion* get_next(HeapRegion* chr) = 0;
-  virtual void set_next(HeapRegion* chr,
-                        HeapRegion* new_next) = 0;
-
-  HeapRegion* _hd;
-  HeapRegion* _tl;
-  size_t _sz;
-
-  // Protected constructor because this type is only meaningful
-  // when the _get/_set next functions are defined.
-  RegionList() : _hd(NULL), _tl(NULL), _sz(0) {}
-public:
-  void reset() {
-    _hd = NULL;
-    _tl = NULL;
-    _sz = 0;
-  }
-  HeapRegion* hd() { return _hd; }
-  HeapRegion* tl() { return _tl; }
-  size_t sz() { return _sz; }
-  size_t length();
-
-  bool well_formed() {
-    return
-      ((hd() == NULL && tl() == NULL && sz() == 0)
-       || (hd() != NULL && tl() != NULL && sz() > 0))
-      && (sz() == length());
-  }
-  virtual void insert_before_head(HeapRegion* r);
-  void prepend_list(RegionList* new_list);
-  virtual HeapRegion* pop();
-  void dec_sz() { _sz--; }
-  // Requires that "r" is an element of the list, and is not the tail.
-  void delete_after(HeapRegion* r);
-};
-
-class EmptyNonHRegionList: public RegionList {
-protected:
-  // Protected constructor because this type is only meaningful
-  // when the _get/_set next functions are defined.
-  EmptyNonHRegionList() : RegionList() {}
-
-public:
-  void insert_before_head(HeapRegion* r) {
-    //    assert(r->is_empty(), "Better be empty");
-    assert(!r->isHumongous(), "Better not be humongous.");
-    RegionList::insert_before_head(r);
-  }
-  void prepend_list(EmptyNonHRegionList* new_list) {
-    //    assert(new_list->hd() == NULL || new_list->hd()->is_empty(),
-    //     "Better be empty");
-    assert(new_list->hd() == NULL || !new_list->hd()->isHumongous(),
-           "Better not be humongous.");
-    //    assert(new_list->tl() == NULL || new_list->tl()->is_empty(),
-    //     "Better be empty");
-    assert(new_list->tl() == NULL || !new_list->tl()->isHumongous(),
-           "Better not be humongous.");
-    RegionList::prepend_list(new_list);
-  }
-};
-
-class UncleanRegionList: public EmptyNonHRegionList {
-public:
-  HeapRegion* get_next(HeapRegion* hr) {
-    return hr->next_from_unclean_list();
-  }
-  void set_next(HeapRegion* hr, HeapRegion* new_next) {
-    hr->set_next_on_unclean_list(new_next);
-  }
-
-  UncleanRegionList() : EmptyNonHRegionList() {}
-
-  void insert_before_head(HeapRegion* r) {
-    assert(!r->is_on_free_list(),
-           "Better not already be on free list");
-    assert(!r->is_on_unclean_list(),
-           "Better not already be on unclean list");
-    r->set_zero_fill_needed();
-    r->set_on_unclean_list(true);
-    EmptyNonHRegionList::insert_before_head(r);
-  }
-  void prepend_list(UncleanRegionList* new_list) {
-    assert(new_list->tl() == NULL || !new_list->tl()->is_on_free_list(),
-           "Better not already be on free list");
-    assert(new_list->tl() == NULL || new_list->tl()->is_on_unclean_list(),
-           "Better already be marked as on unclean list");
-    assert(new_list->hd() == NULL || !new_list->hd()->is_on_free_list(),
-           "Better not already be on free list");
-    assert(new_list->hd() == NULL || new_list->hd()->is_on_unclean_list(),
-           "Better already be marked as on unclean list");
-    EmptyNonHRegionList::prepend_list(new_list);
-  }
-  HeapRegion* pop() {
-    HeapRegion* res = RegionList::pop();
-    if (res != NULL) res->set_on_unclean_list(false);
-    return res;
-  }
-};
-
-// Local Variables: ***
-// c-indentation-style: gnu ***
-// End: ***
-
 #endif // SERIALGC
 
 #endif // SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGION_HPP
--- a/src/share/vm/gc_implementation/g1/heapRegionSeq.cpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/heapRegionSeq.cpp	Wed Jan 19 19:30:42 2011 -0500
@@ -65,152 +65,6 @@
 
 // Private methods.
 
-HeapWord*
-HeapRegionSeq::alloc_obj_from_region_index(int ind, size_t word_size) {
-  assert(G1CollectedHeap::isHumongous(word_size),
-         "Allocation size should be humongous");
-  int cur = ind;
-  int first = cur;
-  size_t sumSizes = 0;
-  while (cur < _regions.length() && sumSizes < word_size) {
-    // Loop invariant:
-    //  For all i in [first, cur):
-    //       _regions.at(i)->is_empty()
-    //    && _regions.at(i) is contiguous with its predecessor, if any
-    //  && sumSizes is the sum of the sizes of the regions in the interval
-    //       [first, cur)
-    HeapRegion* curhr = _regions.at(cur);
-    if (curhr->is_empty()
-        && (first == cur
-            || (_regions.at(cur-1)->end() ==
-                curhr->bottom()))) {
-      sumSizes += curhr->capacity() / HeapWordSize;
-    } else {
-      first = cur + 1;
-      sumSizes = 0;
-    }
-    cur++;
-  }
-  if (sumSizes >= word_size) {
-    _alloc_search_start = cur;
-
-    // We need to initialize the region(s) we just discovered. This is
-    // a bit tricky given that it can happen concurrently with
-    // refinement threads refining cards on these regions and
-    // potentially wanting to refine the BOT as they are scanning
-    // those cards (this can happen shortly after a cleanup; see CR
-    // 6991377). So we have to set up the region(s) carefully and in
-    // a specific order.
-
-    // Currently, allocs_are_zero_filled() returns false. The zero
-    // filling infrastructure will be going away soon (see CR 6977804).
-    // So no need to do anything else here.
-    bool zf = G1CollectedHeap::heap()->allocs_are_zero_filled();
-    assert(!zf, "not supported");
-
-    // This will be the "starts humongous" region.
-    HeapRegion* first_hr = _regions.at(first);
-    {
-      MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-      first_hr->set_zero_fill_allocated();
-    }
-    // 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.
-    // (Note: sumSizes = "region size" x "number of regions we found").
-    HeapWord* new_end = new_obj + sumSizes;
-    // This will be the new top of the first region that will reflect
-    // this allocation.
-    HeapWord* new_top = new_obj + word_size;
-
-    // First, we need to zero the header of the space that we will be
-    // allocating. When we update top further down, some refinement
-    // threads might try to scan the region. By zeroing the header we
-    // ensure that any thread that will try to scan the region will
-    // come across the zero klass word and bail out.
-    //
-    // NOTE: It would not have been correct to have used
-    // CollectedHeap::fill_with_object() and make the space look like
-    // an int array. The thread that is doing the allocation will
-    // later update the object header to a potentially different array
-    // type and, for a very short period of time, the klass and length
-    // fields will be inconsistent. This could cause a refinement
-    // thread to calculate the object size incorrectly.
-    Copy::fill_to_words(new_obj, oopDesc::header_size(), 0);
-
-    // We will set up the first region as "starts humongous". This
-    // will also update the BOT covering all the regions to reflect
-    // that there is a single object that starts at the bottom of the
-    // first region.
-    first_hr->set_startsHumongous(new_top, new_end);
-
-    // Then, if there are any, we will set up the "continues
-    // humongous" regions.
-    HeapRegion* hr = NULL;
-    for (int i = first + 1; i < cur; ++i) {
-      hr = _regions.at(i);
-      {
-        MutexLockerEx x(ZF_mon, Mutex::_no_safepoint_check_flag);
-        hr->set_zero_fill_allocated();
-      }
-      hr->set_continuesHumongous(first_hr);
-    }
-    // If we have "continues humongous" regions (hr != NULL), then the
-    // end of the last one should match new_end.
-    assert(hr == NULL || hr->end() == new_end, "sanity");
-
-    // Up to this point no concurrent thread would have been able to
-    // do any scanning on any region in this series. All the top
-    // fields still point to bottom, so the intersection between
-    // [bottom,top] and [card_start,card_end] will be empty. Before we
-    // update the top fields, we'll do a storestore to make sure that
-    // no thread sees the update to top before the zeroing of the
-    // object header and the BOT initialization.
-    OrderAccess::storestore();
-
-    // Now that the BOT and the object header have been initialized,
-    // we can update top of the "starts humongous" region.
-    assert(first_hr->bottom() < new_top && new_top <= first_hr->end(),
-           "new_top should be in this region");
-    first_hr->set_top(new_top);
-
-    // Now, we will update the top fields of the "continues humongous"
-    // regions. The reason we need to do this is that, otherwise,
-    // these regions would look empty and this will confuse parts of
-    // G1. For example, the code that looks for a consecutive number
-    // of empty regions will consider them empty and try to
-    // re-allocate them. We can extend is_empty() to also include
-    // !continuesHumongous(), but it is easier to just update the top
-    // fields here.
-    hr = NULL;
-    for (int i = first + 1; i < cur; ++i) {
-      hr = _regions.at(i);
-      if ((i + 1) == cur) {
-        // last continues humongous region
-        assert(hr->bottom() < new_top && new_top <= hr->end(),
-               "new_top should fall on this region");
-        hr->set_top(new_top);
-      } else {
-        // not last one
-        assert(new_top > hr->end(), "new_top should be above this region");
-        hr->set_top(hr->end());
-      }
-    }
-    // If we have continues humongous regions (hr != NULL), then the
-    // end of the last one should match new_end and its top should
-    // match new_top.
-    assert(hr == NULL ||
-           (hr->end() == new_end && hr->top() == new_top), "sanity");
-
-    return new_obj;
-  } else {
-    // If we started from the beginning, we want to know why we can't alloc.
-    return NULL;
-  }
-}
-
 void HeapRegionSeq::print_empty_runs() {
   int empty_run = 0;
   int n_empty = 0;
@@ -284,13 +138,67 @@
   return res;
 }
 
-HeapWord* HeapRegionSeq::obj_allocate(size_t word_size) {
-  int cur = _alloc_search_start;
-  // Make sure "cur" is a valid index.
-  assert(cur >= 0, "Invariant.");
-  HeapWord* res = alloc_obj_from_region_index(cur, word_size);
-  if (res == NULL)
-    res = alloc_obj_from_region_index(0, word_size);
+int HeapRegionSeq::find_contiguous_from(int from, size_t num) {
+  assert(num > 1, "pre-condition");
+  assert(0 <= from && from <= _regions.length(),
+         err_msg("from: %d should be valid and <= than %d",
+                 from, _regions.length()));
+
+  int curr = from;
+  int first = -1;
+  size_t num_so_far = 0;
+  while (curr < _regions.length() && num_so_far < num) {
+    HeapRegion* curr_hr = _regions.at(curr);
+    if (curr_hr->is_empty()) {
+      if (first == -1) {
+        first = curr;
+        num_so_far = 1;
+      } else {
+        num_so_far += 1;
+      }
+    } else {
+      first = -1;
+      num_so_far = 0;
+    }
+    curr += 1;
+  }
+
+  assert(num_so_far <= num, "post-condition");
+  if (num_so_far == num) {
+    // we find enough space for the humongous object
+    assert(from <= first && first < _regions.length(), "post-condition");
+    assert(first < curr && (curr - first) == (int) num, "post-condition");
+    for (int i = first; i < first + (int) num; ++i) {
+      assert(_regions.at(i)->is_empty(), "post-condition");
+    }
+    return first;
+  } else {
+    // we failed to find enough space for the humongous object
+    return -1;
+  }
+}
+
+int HeapRegionSeq::find_contiguous(size_t num) {
+  assert(num > 1, "otherwise we should not be calling this");
+  assert(0 <= _alloc_search_start && _alloc_search_start <= _regions.length(),
+         err_msg("_alloc_search_start: %d should be valid and <= than %d",
+                 _alloc_search_start, _regions.length()));
+
+  int start = _alloc_search_start;
+  int res = find_contiguous_from(start, num);
+  if (res == -1 && start != 0) {
+    // Try starting from the beginning. If _alloc_search_start was 0,
+    // no point in doing this again.
+    res = find_contiguous_from(0, num);
+  }
+  if (res != -1) {
+    assert(0 <= res && res < _regions.length(),
+           err_msg("res: %d should be valid", res));
+    _alloc_search_start = res + (int) num;
+  }
+  assert(0 < _alloc_search_start && _alloc_search_start <= _regions.length(),
+         err_msg("_alloc_search_start: %d should be valid",
+                 _alloc_search_start));
   return res;
 }
 
@@ -376,6 +284,10 @@
 
 MemRegion HeapRegionSeq::shrink_by(size_t shrink_bytes,
                                    size_t& num_regions_deleted) {
+  // Reset this in case it's currently pointing into the regions that
+  // we just removed.
+  _alloc_search_start = 0;
+
   assert(shrink_bytes % os::vm_page_size() == 0, "unaligned");
   assert(shrink_bytes % HeapRegion::GrainBytes == 0, "unaligned");
 
@@ -395,7 +307,6 @@
     }
     assert(cur == _regions.top(), "Should be top");
     if (!cur->is_empty()) break;
-    cur->reset_zero_fill();
     shrink_bytes -= cur->capacity();
     num_regions_deleted++;
     _regions.pop();
@@ -410,7 +321,6 @@
   return MemRegion(last_start, end);
 }
 
-
 class PrintHeapRegionClosure : public  HeapRegionClosure {
 public:
   bool doHeapRegion(HeapRegion* r) {
--- a/src/share/vm/gc_implementation/g1/heapRegionSeq.hpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/gc_implementation/g1/heapRegionSeq.hpp	Wed Jan 19 19:30:42 2011 -0500
@@ -41,9 +41,9 @@
   // (For efficiency only; private to obj_allocate after initialization.)
   int _alloc_search_start;
 
-  // Attempts to allocate a block of the (assumed humongous) word_size,
-  // starting at the region "ind".
-  HeapWord* alloc_obj_from_region_index(int ind, size_t word_size);
+  // Finds a contiguous set of empty regions of length num, starting
+  // from a given index.
+  int find_contiguous_from(int from, size_t num);
 
   // Currently, we're choosing collection sets in a round-robin fashion,
   // starting here.
@@ -76,11 +76,8 @@
   // that are available for allocation.
   size_t free_suffix();
 
-  // Requires "word_size" to be humongous (in the technical sense).  If
-  // possible, allocates a contiguous subsequence of the heap regions to
-  // satisfy the allocation, and returns the address of the beginning of
-  // that sequence, otherwise returns NULL.
-  HeapWord* obj_allocate(size_t word_size);
+  // Finds a contiguous set of empty regions of length num.
+  int find_contiguous(size_t num);
 
   // Apply the "doHeapRegion" method of "blk" to all regions in "this",
   // in address order, terminating the iteration early
--- /dev/null	Thu Jan 01 00:00:00 1970 +0000
+++ b/src/share/vm/gc_implementation/g1/heapRegionSet.cpp	Wed Jan 19 19:30:42 2011 -0500
@@ -0,0 +1,438 @@
+/*
+ * Copyright (c) 2011, Oracle and/or its affiliates. All rights reserved.
+ * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
+ *
+ * This code is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This code is distributed in the hope that it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License
+ * version 2 for more details (a copy is included in the LICENSE file that
+ * accompanied this code).
+ *
+ * You should have received a copy of the GNU General Public License version
+ * 2 along with this work; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
+ *
+ * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
+ * or visit www.oracle.com if you need additional information or have any
+ * questions.
+ *
+ */
+
+#include "precompiled.hpp"
+#include "gc_implementation/g1/heapRegionSet.inline.hpp"
+
+size_t HeapRegionSetBase::_unrealistically_long_length = 0;
+
+//////////////////// HeapRegionSetBase ////////////////////
+
+void HeapRegionSetBase::set_unrealistically_long_length(size_t len) {
+  guarantee(_unrealistically_long_length == 0, "should only be set once");
+  _unrealistically_long_length = len;
+}
+
+size_t HeapRegionSetBase::calculate_region_num(HeapRegion* hr) {
+  assert(hr->startsHumongous(), "pre-condition");
+  assert(hr->capacity() % HeapRegion::GrainBytes == 0, "invariant");
+  size_t region_num = hr->capacity() >> HeapRegion::LogOfHRGrainBytes;
+  assert(region_num > 0, "sanity");
+  return region_num;
+}
+
+void HeapRegionSetBase::fill_in_ext_msg(hrl_ext_msg* msg, const char* message) {
+  msg->append("[%s] %s "
+              "ln: "SIZE_FORMAT" rn: "SIZE_FORMAT" "
+              "cy: "SIZE_FORMAT" ud: "SIZE_FORMAT,
+              name(), message, length(), region_num(),
+              total_capacity_bytes(), total_used_bytes());
+  fill_in_ext_msg_extra(msg);
+}
+
+bool HeapRegionSetBase::verify_region(HeapRegion* hr,
+                                  HeapRegionSetBase* expected_containing_set) {
+  const char* error_message = NULL;
+
+  if (!regions_humongous()) {
+    if (hr->isHumongous()) {
+      error_message = "the region should not be humongous";
+    }
+  } else {
+    if (!hr->isHumongous() || !hr->startsHumongous()) {
+      error_message = "the region should be 'starts humongous'";
+    }
+  }
+
+  if (!regions_empty()) {
+    if (hr->is_empty()) {
+      error_message = "the region should not be empty";
+    }
+  } else {
+    if (!hr->is_empty()) {
+      error_message = "the region should be empty";
+    }
+  }
+
+#ifdef ASSERT
+  // The _containing_set field is only available when ASSERT is defined.
+  if (hr->containing_set() != expected_containing_set) {
+    error_message = "inconsistent containing set found";
+  }
+#endif // ASSERT
+
+  const char* extra_error_message = verify_region_extra(hr);
+  if (extra_error_message != NULL) {
+    error_message = extra_error_message;
+  }
+
+  if (error_message != NULL) {
+    outputStream* out = tty;
+    out->cr();
+    out->print_cr("## [%s] %s", name(), error_message);
+    out->print_cr("## Offending Region: "PTR_FORMAT, hr);
+    out->print_cr("   "HR_FORMAT, HR_FORMAT_PARAMS(hr));
+#ifdef ASSERT
+    out->print_cr("   containing set: "PTR_FORMAT, hr->containing_set());
+#endif // ASSERT
+    out->print_cr("## Offending Region Set: "PTR_FORMAT, this);
+    print_on(out);
+    return false;
+  } else {
+    return true;
+  }
+}
+
+void HeapRegionSetBase::verify() {
+  // It's important that we also observe the MT safety protocol even
+  // for the verification calls. If we do verification without the
+  // appropriate locks and the set changes underneath our feet
+  // verification might fail and send us on a wild goose chase.
+  hrl_assert_mt_safety_ok(this);
+
+  guarantee(( is_empty() && length() == 0 && region_num() == 0 &&
+              total_used_bytes() == 0 && total_capacity_bytes() == 0) ||
+            (!is_empty() && length() >= 0 && region_num() >= 0 &&
+              total_used_bytes() >= 0 && total_capacity_bytes() >= 0),
+            hrl_ext_msg(this, "invariant"));
+
+  guarantee((!regions_humongous() && region_num() == length()) ||
+            ( regions_humongous() && region_num() >= length()),
+            hrl_ext_msg(this, "invariant"));
+
+  guarantee(!regions_empty() || total_used_bytes() == 0,
+            hrl_ext_msg(this, "invariant"));
+
+  guarantee(total_used_bytes() <= total_capacity_bytes(),
+            hrl_ext_msg(this, "invariant"));
+}
+
+void HeapRegionSetBase::verify_start() {
+  // See comment in verify() about MT safety and verification.
+  hrl_assert_mt_safety_ok(this);
+  assert(!_verify_in_progress,
+         hrl_ext_msg(this, "verification should not be in progress"));
+
+  // Do the basic verification first before we do the checks over the regions.
+  HeapRegionSetBase::verify();
+
+  _calc_length               = 0;
+  _calc_region_num           = 0;
+  _calc_total_capacity_bytes = 0;
+  _calc_total_used_bytes     = 0;
+  _verify_in_progress        = true;
+}
+
+void HeapRegionSetBase::verify_next_region(HeapRegion* hr) {
+  // See comment in verify() about MT safety and verification.
+  hrl_assert_mt_safety_ok(this);
+  assert(_verify_in_progress,
+         hrl_ext_msg(this, "verification should be in progress"));
+
+  guarantee(verify_region(hr, this), hrl_ext_msg(this, "region verification"));
+
+  _calc_length               += 1;
+  if (!hr->isHumongous()) {
+    _calc_region_num         += 1;
+  } else {
+    _calc_region_num         += calculate_region_num(hr);
+  }
+  _calc_total_capacity_bytes += hr->capacity();
+  _calc_total_used_bytes     += hr->used();
+}
+
+void HeapRegionSetBase::verify_end() {
+  // See comment in verify() about MT safety and verification.
+  hrl_assert_mt_safety_ok(this);
+  assert(_verify_in_progress,
+         hrl_ext_msg(this, "verification should be in progress"));
+
+  guarantee(length() == _calc_length,
+            hrl_err_msg("[%s] length: "SIZE_FORMAT" should be == "
+                        "calc length: "SIZE_FORMAT,
+                        name(), length(), _calc_length));
+
+  guarantee(region_num() == _calc_region_num,
+            hrl_err_msg("[%s] region num: "SIZE_FORMAT" should be == "
+                        "calc region num: "SIZE_FORMAT,
+                        name(), region_num(), _calc_region_num));
+
+  guarantee(total_capacity_bytes() == _calc_total_capacity_bytes,
+            hrl_err_msg("[%s] capacity bytes: "SIZE_FORMAT" should be == "
+                        "calc capacity bytes: "SIZE_FORMAT,
+                        name(),
+                        total_capacity_bytes(), _calc_total_capacity_bytes));
+
+  guarantee(total_used_bytes() == _calc_total_used_bytes,
+            hrl_err_msg("[%s] used bytes: "SIZE_FORMAT" should be == "
+                        "calc used bytes: "SIZE_FORMAT,
+                        name(), total_used_bytes(), _calc_total_used_bytes));
+
+  _verify_in_progress = false;
+}
+
+void HeapRegionSetBase::print_on(outputStream* out, bool print_contents) {
+  out->cr();
+  out->print_cr("Set: %s ("PTR_FORMAT")", name(), this);
+  out->print_cr("  Region Assumptions");
+  out->print_cr("    humongous         : %s", BOOL_TO_STR(regions_humongous()));
+  out->print_cr("    empty             : %s", BOOL_TO_STR(regions_empty()));
+  out->print_cr("  Attributes");
+  out->print_cr("    length            : "SIZE_FORMAT_W(14), length());
+  out->print_cr("    region num        : "SIZE_FORMAT_W(14), region_num());
+  out->print_cr("    total capacity    : "SIZE_FORMAT_W(14)" bytes",
+                total_capacity_bytes());
+  out->print_cr("    total used        : "SIZE_FORMAT_W(14)" bytes",
+                total_used_bytes());
+}
+
+void HeapRegionSetBase::clear() {
+  _length           = 0;
+  _region_num       = 0;
+  _total_used_bytes = 0;
+}
+
+HeapRegionSetBase::HeapRegionSetBase(const char* name)
+  : _name(name), _verify_in_progress(false),
+    _calc_length(0), _calc_region_num(0),
+    _calc_total_capacity_bytes(0), _calc_total_used_bytes(0) { }
+
+//////////////////// HeapRegionSet ////////////////////
+
+void HeapRegionSet::update_from_proxy(HeapRegionSet* proxy_set) {
+  hrl_assert_mt_safety_ok(this);
+  hrl_assert_mt_safety_ok(proxy_set);
+  hrl_assert_sets_match(this, proxy_set);
+
+  verify_optional();
+  proxy_set->verify_optional();
+
+  if (proxy_set->is_empty()) return;
+
+  assert(proxy_set->length() <= _length,
+         hrl_err_msg("[%s] proxy set length: "SIZE_FORMAT" "
+                     "should be <= length: "SIZE_FORMAT,
+                     name(), proxy_set->length(), _length));
+  _length -= proxy_set->length();
+
+  assert(proxy_set->region_num() <= _region_num,
+         hrl_err_msg("[%s] proxy set region num: "SIZE_FORMAT" "
+                     "should be <= region num: "SIZE_FORMAT,
+                     name(), proxy_set->region_num(), _region_num));
+  _region_num -= proxy_set->region_num();
+
+  assert(proxy_set->total_used_bytes() <= _total_used_bytes,
+         hrl_err_msg("[%s] proxy set used bytes: "SIZE_FORMAT" "
+                     "should be <= used bytes: "SIZE_FORMAT,
+                     name(), proxy_set->total_used_bytes(),
+                     _total_used_bytes));
+  _total_used_bytes -= proxy_set->total_used_bytes();
+
+  proxy_set->clear();
+
+  verify_optional();
+  proxy_set->verify_optional();
+}
+
+//////////////////// HeapRegionLinkedList ////////////////////
+
+void HeapRegionLinkedList::fill_in_ext_msg_extra(hrl_ext_msg* msg) {
+  msg->append(" hd: "PTR_FORMAT" tl: "PTR_FORMAT, head(), tail());
+}
+
+void HeapRegionLinkedList::add_as_tail(HeapRegionLinkedList* from_list) {
+  hrl_assert_mt_safety_ok(this);
+  hrl_assert_mt_safety_ok(from_list);
+
+  verify_optional();
+  from_list->verify_optional();
+
+  if (from_list->is_empty()) return;
+
+#ifdef ASSERT
+  HeapRegionLinkedListIterator iter(from_list);
+  while (iter.more_available()) {
+    HeapRegion* hr = iter.get_next();
+    // In set_containing_set() we check that we either set the value
+    // from NULL to non-NULL or vice versa to catch bugs. So, we have
+    // to NULL it first before setting it to the value.
+    hr->set_containing_set(NULL);
+    hr->set_containing_set(this);
+  }
+#endif // ASSERT
+
+  if (_tail != NULL) {
+    assert(length() >  0 && _head != NULL, hrl_ext_msg(this, "invariant"));
+    _tail->set_next(from_list->_head);
+  } else {
+    assert(length() == 0 && _head == NULL, hrl_ext_msg(this, "invariant"));
+    _head = from_list->_head;
+  }
+  _tail = from_list->_tail;
+
+  _length           += from_list->length();
+  _region_num       += from_list->region_num();
+  _total_used_bytes += from_list->total_used_bytes();
+  from_list->clear();
+
+  verify_optional();
+  from_list->verify_optional();
+}
+
+void HeapRegionLinkedList::remove_all() {
+  hrl_assert_mt_safety_ok(this);
+  verify_optional();
+
+  HeapRegion* curr = _head;
+  while (curr != NULL) {
+    hrl_assert_region_ok(this, curr, this);
+
+    HeapRegion* next = curr->next();
+    curr->set_next(NULL);
+    curr->set_containing_set(NULL);
+    curr = next;
+  }
+  clear();
+
+  verify_optional();
+}
+
+void HeapRegionLinkedList::remove_all_pending(size_t target_count) {
+  hrl_assert_mt_safety_ok(this);
+  assert(target_count > 1, hrl_ext_msg(this, "pre-condition"));
+  assert(!is_empty(), hrl_ext_msg(this, "pre-condition"));
+
+  verify_optional();
+  DEBUG_ONLY(size_t old_length = length();)
+
+  HeapRegion* curr = _head;
+  HeapRegion* prev = NULL;
+  size_t count = 0;
+  while (curr != NULL) {
+    hrl_assert_region_ok(this, curr, this);
+    HeapRegion* next = curr->next();
+
+    if (curr->pending_removal()) {
+      assert(count < target_count,
+             hrl_err_msg("[%s] should not come across more regions "
+                         "pending for removal than target_count: "SIZE_FORMAT,
+                         name(), target_count));
+
+      if (prev == NULL) {
+        assert(_head == curr, hrl_ext_msg(this, "invariant"));
+        _head = next;
+      } else {
+        assert(_head != curr, hrl_ext_msg(this, "invariant"));
+        prev->set_next(next);
+      }
+      if (next == NULL) {
+        assert(_tail == curr, hrl_ext_msg(this, "invariant"));
+        _tail = prev;
+      } else {
+        assert(_tail != curr, hrl_ext_msg(this, "invariant"));
+      }
+
+      curr->set_next(NULL);
+      remove_internal(curr);
+      curr->set_pending_removal(false);
+
+      count += 1;
+
+      // If we have come across the target number of regions we can
+      // just bail out. However, for debugging purposes, we can just
+      // carry on iterating to make sure there are not more regions
+      // tagged with pending removal.
+      DEBUG_ONLY(if (count == target_count) break;)
+    } else {
+      prev = curr;
+    }
+    curr = next;
+  }
+
+  assert(count == target_count,
+         hrl_err_msg("[%s] count: "SIZE_FORMAT" should be == "
+                     "target_count: "SIZE_FORMAT, name(), count, target_count));
+  assert(length() + target_count == old_length,
+         hrl_err_msg("[%s] new length should be consistent "
+                     "new length: "SIZE_FORMAT" old length: "SIZE_FORMAT" "
+                     "target_count: "SIZE_FORMAT,
+                     name(), length(), old_length, target_count));
+
+  verify_optional();
+}
+
+void HeapRegionLinkedList::verify() {
+  // See comment in HeapRegionSetBase::verify() about MT safety and
+  // verification.
+  hrl_assert_mt_safety_ok(this);
+
+  // This will also do the basic verification too.
+  verify_start();
+
+  HeapRegion* curr  = _head;
+  HeapRegion* prev1 = NULL;
+  HeapRegion* prev0 = NULL;
+  size_t      count = 0;
+  while (curr != NULL) {
+    verify_next_region(curr);
+
+    count += 1;
+    guarantee(count < _unrealistically_long_length,
+              hrl_err_msg("[%s] the calculated length: "SIZE_FORMAT" "
+                          "seems very long, is there maybe a cycle? "
+                          "curr: "PTR_FORMAT" prev0: "PTR_FORMAT" "
+                          "prev1: "PTR_FORMAT" length: "SIZE_FORMAT,
+                          name(), count, curr, prev0, prev1, length()));
+
+    prev1 = prev0;
+    prev0 = curr;
+    curr  = curr->next();
+  }
+
+  guarantee(_tail == prev0, hrl_ext_msg(this, "post-condition"));
+
+  verify_end();
+}
+
+void HeapRegionLinkedList::clear() {
+  HeapRegionSetBase::clear();
+  _head = NULL;
+  _tail = NULL;
+}
+
+void HeapRegionLinkedList::print_on(outputStream* out, bool print_contents) {
+  HeapRegionSetBase::print_on(out, print_contents);
+  out->print_cr("  Linking");
+  out->print_cr("    head              : "PTR_FORMAT, _head);
+  out->print_cr("    tail              : "PTR_FORMAT, _tail);
+
+  if (print_contents) {
+    out->print_cr("  Contents");
+    HeapRegionLinkedListIterator iter(this);
+    while (iter.more_available()) {
+      HeapRegion* hr = iter.get_next();
+      hr->print_on(out);
+    }
+  }
+}
--- /dev/null	Thu Jan 01 00:00:00 1970 +0000
+++ b/src/share/vm/gc_implementation/g1/heapRegionSet.hpp	Wed Jan 19 19:30:42 2011 -0500
@@ -0,0 +1,346 @@
+/*
+ * copyright (c) 2011, Oracle and/or its affiliates. All rights reserved.
+ * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
+ *
+ * This code is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This code is distributed in the hope that it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License
+ * version 2 for more details (a copy is included in the LICENSE file that
+ * accompanied this code).
+ *
+ * You should have received a copy of the GNU General Public License version
+ * 2 along with this work; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
+ *
+ * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
+ * or visit www.oracle.com if you need additional information or have any
+ * questions.
+ *
+ */
+
+#ifndef SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSET_HPP
+#define SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSET_HPP
+
+#include "gc_implementation/g1/heapRegion.hpp"
+
+// Large buffer for some cases where the output might be larger than normal.
+#define HRL_ERR_MSG_BUFSZ 512
+typedef FormatBuffer<HRL_ERR_MSG_BUFSZ> hrl_err_msg;
+
+// Set verification will be forced either if someone defines
+// HEAP_REGION_SET_FORCE_VERIFY to be 1, or in builds in which
+// asserts are compiled in.
+#ifndef HEAP_REGION_SET_FORCE_VERIFY
+#define HEAP_REGION_SET_FORCE_VERIFY defined(ASSERT)
+#endif // HEAP_REGION_SET_FORCE_VERIFY
+
+//////////////////// HeapRegionSetBase ////////////////////
+
+// Base class for all the classes that represent heap region sets. It
+// contains the basic attributes that each set needs to maintain
+// (e.g., length, region num, used bytes sum) plus any shared
+// functionality (e.g., verification).
+
+class hrl_ext_msg;
+
+class HeapRegionSetBase VALUE_OBJ_CLASS_SPEC {
+  friend class hrl_ext_msg;
+
+protected:
+  static size_t calculate_region_num(HeapRegion* hr);
+
+  static size_t _unrealistically_long_length;
+
+  // The number of regions added to the set. If the set contains
+  // only humongous regions, this reflects only 'starts humongous'
+  // regions and does not include 'continues humongous' ones.
+  size_t _length;
+
+  // The total number of regions represented by the set. If the set
+  // does not contain humongous regions, this should be the same as
+  // _length. If the set contains only humongous regions, this will
+  // include the 'continues humongous' regions.
+  size_t _region_num;
+
+  // We don't keep track of the total capacity explicitly, we instead
+  // recalculate it based on _region_num and the heap region size.
+
+  // The sum of used bytes in the all the regions in the set.
+  size_t _total_used_bytes;
+
+  const char* _name;
+
+  bool        _verify_in_progress;
+  size_t      _calc_length;
+  size_t      _calc_region_num;
+  size_t      _calc_total_capacity_bytes;
+  size_t      _calc_total_used_bytes;
+
+  // verify_region() is used to ensure that the contents of a region
+  // added to / removed from a set are consistent. Different sets
+  // make different assumptions about the regions added to them. So
+  // each set can override verify_region_extra(), which is called
+  // from verify_region(), and do any extra verification it needs to
+  // perform in that.
+  virtual const char* verify_region_extra(HeapRegion* hr) { return NULL; }
+  bool verify_region(HeapRegion* hr,
+                     HeapRegionSetBase* expected_containing_set);
+
+  // Indicates whether all regions in the set should be humongous or
+  // not. Only used during verification.
+  virtual bool regions_humongous() = 0;
+
+  // Indicates whether all regions in the set should be empty or
+  // not. Only used during verification.
+  virtual bool regions_empty() = 0;
+
+  // Subclasses can optionally override this to do MT safety protocol
+  // checks. It is called in an assert from all methods that perform
+  // updates on the set (and subclasses should also call it too).
+  virtual bool check_mt_safety() { return true; }
+
+  // fill_in_ext_msg() writes the the values of the set's attributes
+  // in the custom err_msg (hrl_ext_msg). fill_in_ext_msg_extra()
+  // allows subclasses to append further information.
+  virtual void fill_in_ext_msg_extra(hrl_ext_msg* msg) { }
+  void fill_in_ext_msg(hrl_ext_msg* msg, const char* message);
+
+  // It updates the fields of the set to reflect hr being added to
+  // the set.
+  inline void update_for_addition(HeapRegion* hr);
+
+  // It updates the fields of the set to reflect hr being added to
+  // the set and tags the region appropriately.
+  inline void add_internal(HeapRegion* hr);
+
+  // It updates the fields of the set to reflect hr being removed
+  // from the set.
+  inline void update_for_removal(HeapRegion* hr);
+
+  // It updates the fields of the set to reflect hr being removed
+  // from the set and tags the region appropriately.
+  inline void remove_internal(HeapRegion* hr);
+
+  // It clears all the fields of the sets. Note: it will not iterate
+  // over the set and remove regions from it. It assumes that the
+  // caller has already done so. It will literally just clear the fields.
+  virtual void clear();
+
+  HeapRegionSetBase(const char* name);
+
+public:
+  static void set_unrealistically_long_length(size_t len);
+
+  const char* name() { return _name; }
+
+  size_t length() { return _length; }
+
+  bool is_empty() { return _length == 0; }
+
+  size_t region_num() { return _region_num; }
+
+  size_t total_capacity_bytes() {
+    return region_num() << HeapRegion::LogOfHRGrainBytes;
+  }
+
+  size_t total_used_bytes() { return _total_used_bytes; }
+
+  virtual void verify();
+  void verify_start();
+  void verify_next_region(HeapRegion* hr);
+  void verify_end();
+
+#if HEAP_REGION_SET_FORCE_VERIFY
+  void verify_optional() {
+    verify();
+  }
+#else // HEAP_REGION_SET_FORCE_VERIFY
+  void verify_optional() { }
+#endif // HEAP_REGION_SET_FORCE_VERIFY
+
+  virtual void print_on(outputStream* out, bool print_contents = false);
+};
+
+// Customized err_msg for heap region sets. Apart from a
+// assert/guarantee-specific message it also prints out the values of
+// the fields of the associated set. This can be very helpful in
+// diagnosing failures.
+
+class hrl_ext_msg : public hrl_err_msg {
+public:
+  hrl_ext_msg(HeapRegionSetBase* set, const char* message) : hrl_err_msg("") {
+    set->fill_in_ext_msg(this, message);
+  }
+};
+
+// These two macros are provided for convenience, to keep the uses of
+// these two asserts a bit more concise.
+
+#define hrl_assert_mt_safety_ok(_set_)                                        \
+  do {                                                                        \
+    assert((_set_)->check_mt_safety(), hrl_ext_msg((_set_), "MT safety"));    \
+  } while (0)
+
+#define hrl_assert_region_ok(_set_, _hr_, _expected_)                         \
+  do {                                                                        \
+    assert((_set_)->verify_region((_hr_), (_expected_)),                      \
+           hrl_ext_msg((_set_), "region verification"));                      \
+  } while (0)
+
+//////////////////// HeapRegionSet ////////////////////
+
+#define hrl_assert_sets_match(_set1_, _set2_)                                 \
+  do {                                                                        \
+    assert(((_set1_)->regions_humongous() ==                                  \
+                                            (_set2_)->regions_humongous()) && \
+           ((_set1_)->regions_empty() == (_set2_)->regions_empty()),          \
+           hrl_err_msg("the contents of set %s and set %s should match",      \
+                       (_set1_)->name(), (_set2_)->name()));                  \
+  } while (0)
+
+// This class represents heap region sets whose members are not
+// explicitly tracked. It's helpful to group regions using such sets
+// so that we can reason about all the region groups in the heap using
+// the same interface (namely, the HeapRegionSetBase API).
+
+class HeapRegionSet : public HeapRegionSetBase {
+protected:
+  virtual const char* verify_region_extra(HeapRegion* hr) {
+    if (hr->next() != NULL) {
+      return "next() should always be NULL as we do not link the regions";
+    }
+
+    return HeapRegionSetBase::verify_region_extra(hr);
+  }
+
+  HeapRegionSet(const char* name) : HeapRegionSetBase(name) {
+    clear();
+  }
+
+public:
+  // It adds hr to the set. The region should not be a member of
+  // another set.
+  inline void add(HeapRegion* hr);
+
+  // It removes hr from the set. The region should be a member of
+  // this set.
+  inline void remove(HeapRegion* hr);
+
+  // It removes a region from the set. Instead of updating the fields
+  // of the set to reflect this removal, it accumulates the updates
+  // in proxy_set. The idea is that proxy_set is thread-local to
+  // avoid multiple threads updating the fields of the set
+  // concurrently and having to synchronize. The method
+  // update_from_proxy() will update the fields of the set from the
+  // proxy_set.
+  inline void remove_with_proxy(HeapRegion* hr, HeapRegionSet* proxy_set);
+
+  // After multiple calls to remove_with_proxy() the updates to the
+  // fields of the set are accumulated in proxy_set. This call
+  // updates the fields of the set from proxy_set.
+  void update_from_proxy(HeapRegionSet* proxy_set);
+};
+
+//////////////////// HeapRegionLinkedList ////////////////////
+
+// A set that links all the regions added to it in a singly-linked
+// list. We should try to avoid doing operations that iterate over
+// such lists in performance critical paths. Typically we should
+// add / remove one region at a time or concatenate two lists. All
+// those operations are done in constant time.
+
+class HeapRegionLinkedListIterator;
+
+class HeapRegionLinkedList : public HeapRegionSetBase {
+  friend class HeapRegionLinkedListIterator;
+
+private:
+  HeapRegion* _head;
+  HeapRegion* _tail;
+
+  // These are provided for use by the friend classes.
+  HeapRegion* head() { return _head; }
+  HeapRegion* tail() { return _tail; }
+
+protected:
+  virtual void fill_in_ext_msg_extra(hrl_ext_msg* msg);
+
+  // See the comment for HeapRegionSetBase::clear()
+  virtual void clear();
+
+  HeapRegionLinkedList(const char* name) : HeapRegionSetBase(name) {
+    clear();
+  }
+
+public:
+  // It adds hr to the list as the new tail. The region should not be
+  // a member of another set.
+  inline void add_as_tail(HeapRegion* hr);
+
+  // It removes and returns the head of the list. It assumes that the
+  // list is not empty so it will return a non-NULL value.
+  inline HeapRegion* remove_head();
+
+  // Convenience method.
+  inline HeapRegion* remove_head_or_null();
+
+  // It moves the regions from from_list to this list and empties
+  // from_list. The new regions will appear in the same order as they
+  // were in from_list and be linked in the end of this list.
+  void add_as_tail(HeapRegionLinkedList* from_list);
+
+  // It empties the list by removing all regions from it.
+  void remove_all();
+
+  // It removes all regions in the list that are pending for removal
+  // (i.e., they have been tagged with "pending_removal"). The list
+  // must not be empty, target_count should reflect the exact number
+  // of regions that are pending for removal in the list, and
+  // target_count should be > 1 (currently, we never need to remove a
+  // single region using this).
+  void remove_all_pending(size_t target_count);
+
+  virtual void verify();
+
+  virtual void print_on(outputStream* out, bool print_contents = false);
+};
+
+//////////////////// HeapRegionLinkedList ////////////////////
+
+// Iterator class that provides a convenient way to iterator over the
+// regions in a HeapRegionLinkedList instance.
+
+class HeapRegionLinkedListIterator : public StackObj {
+private:
+  HeapRegionLinkedList* _list;
+  HeapRegion*           _curr;
+
+public:
+  bool more_available() {
+    return _curr != NULL;
+  }
+
+  HeapRegion* get_next() {
+    assert(more_available(),
+           "get_next() should be called when more regions are available");
+
+    // If we are going to introduce a count in the iterator we should
+    // do the "cycle" check.
+
+    HeapRegion* hr = _curr;
+    assert(_list->verify_region(hr, _list), "region verification");
+    _curr = hr->next();
+    return hr;
+  }
+
+  HeapRegionLinkedListIterator(HeapRegionLinkedList* list)
+    : _curr(NULL), _list(list) {
+    _curr = list->head();
+  }
+};
+
+#endif // SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSET_HPP
--- /dev/null	Thu Jan 01 00:00:00 1970 +0000
+++ b/src/share/vm/gc_implementation/g1/heapRegionSet.inline.hpp	Wed Jan 19 19:30:42 2011 -0500
@@ -0,0 +1,159 @@
+/*
+ * copyright (c) 2011, Oracle and/or its affiliates. All rights reserved.
+ * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
+ *
+ * This code is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This code is distributed in the hope that it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License
+ * version 2 for more details (a copy is included in the LICENSE file that
+ * accompanied this code).
+ *
+ * You should have received a copy of the GNU General Public License version
+ * 2 along with this work; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
+ *
+ * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
+ * or visit www.oracle.com if you need additional information or have any
+ * questions.
+ *
+ */
+
+#ifndef SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSET_INLINE_HPP
+#define SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSET_INLINE_HPP
+
+#include "gc_implementation/g1/heapRegionSet.hpp"
+
+//////////////////// HeapRegionSetBase ////////////////////
+
+inline void HeapRegionSetBase::update_for_addition(HeapRegion* hr) {
+  // Assumes the caller has already verified the region.
+
+  _length           += 1;
+  if (!hr->isHumongous()) {
+    _region_num     += 1;
+  } else {
+    _region_num     += calculate_region_num(hr);
+  }
+  _total_used_bytes += hr->used();
+}
+
+inline void HeapRegionSetBase::add_internal(HeapRegion* hr) {
+  hrl_assert_region_ok(this, hr, NULL);
+  assert(hr->next() == NULL, hrl_ext_msg(this, "should not already be linked"));
+
+  update_for_addition(hr);
+  hr->set_containing_set(this);
+}
+
+inline void HeapRegionSetBase::update_for_removal(HeapRegion* hr) {
+  // Assumes the caller has already verified the region.
+  assert(_length > 0, hrl_ext_msg(this, "pre-condition"));
+  _length -= 1;
+
+  size_t region_num_diff;
+  if (!hr->isHumongous()) {
+    region_num_diff = 1;
+  } else {
+    region_num_diff = calculate_region_num(hr);
+  }
+  assert(region_num_diff <= _region_num,
+         hrl_err_msg("[%s] region's region num: "SIZE_FORMAT" "
+                     "should be <= region num: "SIZE_FORMAT,
+                     name(), region_num_diff, _region_num));
+  _region_num -= region_num_diff;
+
+  size_t used_bytes = hr->used();
+  assert(used_bytes <= _total_used_bytes,
+         hrl_err_msg("[%s] region's used bytes: "SIZE_FORMAT" "
+                     "should be <= used bytes: "SIZE_FORMAT,
+                     name(), used_bytes, _total_used_bytes));
+  _total_used_bytes -= used_bytes;
+}
+
+inline void HeapRegionSetBase::remove_internal(HeapRegion* hr) {
+  hrl_assert_region_ok(this, hr, this);
+  assert(hr->next() == NULL, hrl_ext_msg(this, "should already be unlinked"));
+
+  hr->set_containing_set(NULL);
+  update_for_removal(hr);
+}
+
+//////////////////// HeapRegionSet ////////////////////
+
+inline void HeapRegionSet::add(HeapRegion* hr) {
+  hrl_assert_mt_safety_ok(this);
+  // add_internal() will verify the region.
+  add_internal(hr);
+}
+
+inline void HeapRegionSet::remove(HeapRegion* hr) {
+  hrl_assert_mt_safety_ok(this);
+  // remove_internal() will verify the region.
+  remove_internal(hr);
+}
+
+inline void HeapRegionSet::remove_with_proxy(HeapRegion* hr,
+                                             HeapRegionSet* proxy_set) {
+  // No need to fo the MT safety check here given that this method
+  // does not update the contents of the set but instead accumulates
+  // the changes in proxy_set which is assumed to be thread-local.
+  hrl_assert_sets_match(this, proxy_set);
+  hrl_assert_region_ok(this, hr, this);
+
+  hr->set_containing_set(NULL);
+  proxy_set->update_for_addition(hr);
+}
+
+//////////////////// HeapRegionLinkedList ////////////////////
+
+inline void HeapRegionLinkedList::add_as_tail(HeapRegion* hr) {
+  hrl_assert_mt_safety_ok(this);
+  assert((length() == 0 && _head == NULL && _tail == NULL) ||
+         (length() >  0 && _head != NULL && _tail != NULL),
+         hrl_ext_msg(this, "invariant"));
+  // add_internal() will verify the region.
+  add_internal(hr);
+
+  // Now link the region.
+  if (_tail != NULL) {
+    _tail->set_next(hr);
+  } else {
+    _head = hr;
+  }
+  _tail = hr;
+}
+
+inline HeapRegion* HeapRegionLinkedList::remove_head() {
+  hrl_assert_mt_safety_ok(this);
+  assert(!is_empty(), hrl_ext_msg(this, "the list should not be empty"));
+  assert(length() > 0 && _head != NULL && _tail != NULL,
+         hrl_ext_msg(this, "invariant"));
+
+  // We need to unlink it first.
+  HeapRegion* hr = _head;
+  _head = hr->next();
+  if (_head == NULL) {
+    _tail = NULL;
+  }
+  hr->set_next(NULL);
+
+  // remove_internal() will verify the region.
+  remove_internal(hr);
+  return hr;
+}
+
+inline HeapRegion* HeapRegionLinkedList::remove_head_or_null() {
+  hrl_assert_mt_safety_ok(this);
+
+  if (!is_empty()) {
+    return remove_head();
+  } else {
+    return NULL;
+  }
+}
+
+#endif // SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSET_INLINE_HPP
--- /dev/null	Thu Jan 01 00:00:00 1970 +0000
+++ b/src/share/vm/gc_implementation/g1/heapRegionSets.cpp	Wed Jan 19 19:30:42 2011 -0500
@@ -0,0 +1,102 @@
+/*
+ * Copyright (c) 2011, Oracle and/or its affiliates. All rights reserved.
+ * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
+ *
+ * This code is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This code is distributed in the hope that it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License
+ * version 2 for more details (a copy is included in the LICENSE file that
+ * accompanied this code).
+ *
+ * You should have received a copy of the GNU General Public License version
+ * 2 along with this work; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
+ *
+ * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
+ * or visit www.oracle.com if you need additional information or have any
+ * questions.
+ *
+ */
+
+#include "precompiled.hpp"
+#include "gc_implementation/g1/heapRegionSets.hpp"
+
+//////////////////// FreeRegionList ////////////////////
+
+const char* FreeRegionList::verify_region_extra(HeapRegion* hr) {
+  if (hr->is_young()) {
+    return "the region should not be young";
+  }
+  // The superclass will check that the region is empty and
+  // not-humongous.
+  return HeapRegionLinkedList::verify_region_extra(hr);
+}
+
+//////////////////// MasterFreeRegionList ////////////////////
+
+bool MasterFreeRegionList::check_mt_safety() {
+  // Master Free List MT safety protocol:
+  // (a) If we're at a safepoint, operations on the master free list
+  // should be invoked by either the VM thread (which will serialize
+  // them) or by the GC workers while holding the
+  // FreeList_lock.
+  // (b) If we're not at a safepoint, operations on the master free
+  // list should be invoked while holding the Heap_lock.
+
+  guarantee((SafepointSynchronize::is_at_safepoint() &&
+               (Thread::current()->is_VM_thread() ||
+                                            FreeList_lock->owned_by_self())) ||
+            (!SafepointSynchronize::is_at_safepoint() &&
+                                                Heap_lock->owned_by_self()),
+            hrl_ext_msg(this, "master free list MT safety protocol"));
+
+  return FreeRegionList::check_mt_safety();
+}
+
+//////////////////// SecondaryFreeRegionList ////////////////////
+
+bool SecondaryFreeRegionList::check_mt_safety() {
+  // Secondary Free List MT safety protocol:
+  // Operations on the secondary free list should always be invoked
+  // while holding the SecondaryFreeList_lock.
+
+  guarantee(SecondaryFreeList_lock->owned_by_self(),
+            hrl_ext_msg(this, "secondary free list MT safety protocol"));
+
+  return FreeRegionList::check_mt_safety();
+}
+
+//////////////////// HumongousRegionSet ////////////////////
+
+const char* HumongousRegionSet::verify_region_extra(HeapRegion* hr) {
+  if (hr->is_young()) {
+    return "the region should not be young";
+  }
+  // The superclass will check that the region is not empty and
+  // humongous.
+  return HeapRegionSet::verify_region_extra(hr);
+}
+
+//////////////////// HumongousRegionSet ////////////////////
+
+bool MasterHumongousRegionSet::check_mt_safety() {
+  // Master Humongous Set MT safety protocol:
+  // (a) If we're at a safepoint, operations on the master humongous
+  // set should be invoked by either the VM thread (which will
+  // serialize them) or by the GC workers while holding the
+  // OldSets_lock.
+  // (b) If we're not at a safepoint, operations on the master
+  // humongous set should be invoked while holding the Heap_lock.
+
+  guarantee((SafepointSynchronize::is_at_safepoint() &&
+               (Thread::current()->is_VM_thread() ||
+                                             OldSets_lock->owned_by_self())) ||
+            (!SafepointSynchronize::is_at_safepoint() &&
+                                                 Heap_lock->owned_by_self()),
+            hrl_ext_msg(this, "master humongous set MT safety protocol"));
+  return HumongousRegionSet::check_mt_safety();
+}
--- /dev/null	Thu Jan 01 00:00:00 1970 +0000
+++ b/src/share/vm/gc_implementation/g1/heapRegionSets.hpp	Wed Jan 19 19:30:42 2011 -0500
@@ -0,0 +1,86 @@
+/*
+ * copyright (c) 2011, Oracle and/or its affiliates. All rights reserved.
+ * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
+ *
+ * This code is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This code is distributed in the hope that it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License
+ * version 2 for more details (a copy is included in the LICENSE file that
+ * accompanied this code).
+ *
+ * You should have received a copy of the GNU General Public License version
+ * 2 along with this work; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
+ *
+ * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA
+ * or visit www.oracle.com if you need additional information or have any
+ * questions.
+ *
+ */
+
+#ifndef SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSETS_HPP
+#define SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSETS_HPP
+
+#include "gc_implementation/g1/heapRegionSet.inline.hpp"
+
+//////////////////// FreeRegionList ////////////////////
+
+class FreeRegionList : public HeapRegionLinkedList {
+protected:
+  virtual const char* verify_region_extra(HeapRegion* hr);
+
+  virtual bool regions_humongous() { return false; }
+  virtual bool regions_empty()     { return true;  }
+
+public:
+  FreeRegionList(const char* name) : HeapRegionLinkedList(name) { }
+};
+
+//////////////////// MasterFreeRegionList ////////////////////
+
+class MasterFreeRegionList : public FreeRegionList {
+protected:
+  virtual bool check_mt_safety();
+
+public:
+  MasterFreeRegionList(const char* name) : FreeRegionList(name) { }
+};
+
+//////////////////// SecondaryFreeRegionList ////////////////////
+
+class SecondaryFreeRegionList : public FreeRegionList {
+protected:
+  virtual bool check_mt_safety();
+
+public:
+  SecondaryFreeRegionList(const char* name) : FreeRegionList(name) { }
+};
+
+//////////////////// HumongousRegionSet ////////////////////
+
+class HumongousRegionSet : public HeapRegionSet {
+protected:
+  virtual const char* verify_region_extra(HeapRegion* hr);
+
+  virtual bool regions_humongous() { return true;  }
+  virtual bool regions_empty()     { return false; }
+
+public:
+  HumongousRegionSet(const char* name) : HeapRegionSet(name) { }
+};
+
+//////////////////// MasterHumongousRegionSet ////////////////////
+
+class MasterHumongousRegionSet : public HumongousRegionSet {
+protected:
+  virtual bool check_mt_safety();
+
+public:
+  MasterHumongousRegionSet(const char* name) : HumongousRegionSet(name) { }
+};
+
+#endif // SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSETS_HPP
--- a/src/share/vm/runtime/mutexLocker.cpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/runtime/mutexLocker.cpp	Wed Jan 19 19:30:42 2011 -0500
@@ -1,5 +1,5 @@
 /*
- * Copyright (c) 1997, 2010, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 1997, 2011, Oracle and/or its affiliates. All rights reserved.
  * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
  *
  * This code is free software; you can redistribute it and/or modify it
@@ -80,8 +80,6 @@
 Monitor* iCMS_lock                    = NULL;
 Monitor* FullGCCount_lock             = NULL;
 Monitor* CMark_lock                   = NULL;
-Monitor* ZF_mon                       = NULL;
-Monitor* Cleanup_mon                  = NULL;
 Mutex*   CMRegionStack_lock           = NULL;
 Mutex*   SATB_Q_FL_lock               = NULL;
 Monitor* SATB_Q_CBL_mon               = NULL;
@@ -122,6 +120,9 @@
 Mutex*   PerfDataManager_lock         = NULL;
 Mutex*   OopMapCacheAlloc_lock        = NULL;
 
+Mutex*   FreeList_lock                = NULL;
+Monitor* SecondaryFreeList_lock       = NULL;
+Mutex*   OldSets_lock                 = NULL;
 Mutex*   MMUTracker_lock              = NULL;
 Mutex*   HotCardCache_lock            = NULL;
 
@@ -177,8 +178,6 @@
   }
   if (UseG1GC) {
     def(CMark_lock                 , Monitor, nonleaf,     true ); // coordinate concurrent mark thread
-    def(ZF_mon                     , Monitor, leaf,        true );
-    def(Cleanup_mon                , Monitor, nonleaf,     true );
     def(CMRegionStack_lock         , Mutex,   leaf,        true );
     def(SATB_Q_FL_lock             , Mutex  , special,     true );
     def(SATB_Q_CBL_mon             , Monitor, nonleaf,     true );
@@ -188,6 +187,9 @@
     def(DirtyCardQ_CBL_mon         , Monitor, nonleaf,     true );
     def(Shared_DirtyCardQ_lock     , Mutex,   nonleaf,     true );
 
+    def(FreeList_lock              , Mutex,   leaf     ,   true );
+    def(SecondaryFreeList_lock     , Monitor, leaf     ,   true );
+    def(OldSets_lock               , Mutex  , leaf     ,   true );
     def(MMUTracker_lock            , Mutex  , leaf     ,   true );
     def(HotCardCache_lock          , Mutex  , special  ,   true );
     def(EvacFailureStack_lock      , Mutex  , nonleaf  ,   true );
--- a/src/share/vm/runtime/mutexLocker.hpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/runtime/mutexLocker.hpp	Wed Jan 19 19:30:42 2011 -0500
@@ -1,5 +1,5 @@
 /*
- * Copyright (c) 1997, 2010, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 1997, 2011, Oracle and/or its affiliates. All rights reserved.
  * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
  *
  * This code is free software; you can redistribute it and/or modify it
@@ -76,8 +76,6 @@
 extern Monitor* iCMS_lock;                       // CMS incremental mode start/stop notification
 extern Monitor* FullGCCount_lock;                // in support of "concurrent" full gc
 extern Monitor* CMark_lock;                      // used for concurrent mark thread coordination
-extern Monitor* ZF_mon;                          // used for G1 conc zero-fill.
-extern Monitor* Cleanup_mon;                     // used for G1 conc cleanup.
 extern Mutex*   CMRegionStack_lock;              // used for protecting accesses to the CM region stack
 extern Mutex*   SATB_Q_FL_lock;                  // Protects SATB Q
                                                  // buffer free list.
@@ -125,6 +123,9 @@
 extern Mutex*   ParkerFreeList_lock;
 extern Mutex*   OopMapCacheAlloc_lock;           // protects allocation of oop_map caches
 
+extern Mutex*   FreeList_lock;                   // protects the free region list during safepoints
+extern Monitor* SecondaryFreeList_lock;          // protects the secondary free region list
+extern Mutex*   OldSets_lock;                    // protects the old region sets
 extern Mutex*   MMUTracker_lock;                 // protects the MMU
                                                  // tracker data structures
 extern Mutex*   HotCardCache_lock;               // protects the hot card cache
--- a/src/share/vm/utilities/debug.hpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/utilities/debug.hpp	Wed Jan 19 19:30:42 2011 -0500
@@ -34,6 +34,7 @@
 class FormatBuffer {
 public:
   inline FormatBuffer(const char * format, ...);
+  inline void append(const char* format, ...);
   operator const char *() const { return _buf; }
 
 private:
@@ -51,6 +52,19 @@
   va_end(argp);
 }
 
+template <size_t bufsz>
+void FormatBuffer<bufsz>::append(const char* format, ...) {
+  // Given that the constructor does a vsnprintf we can assume that
+  // _buf is already initialized.
+  size_t len = strlen(_buf);
+  char* buf_end = _buf + len;
+
+  va_list argp;
+  va_start(argp, format);
+  vsnprintf(buf_end, bufsz - len, format, argp);
+  va_end(argp);
+}
+
 // Used to format messages for assert(), guarantee(), fatal(), etc.
 typedef FormatBuffer<> err_msg;
 
--- a/src/share/vm/utilities/globalDefinitions.hpp	Wed Jan 19 13:04:37 2011 -0800
+++ b/src/share/vm/utilities/globalDefinitions.hpp	Wed Jan 19 19:30:42 2011 -0500
@@ -1,5 +1,5 @@
 /*
- * Copyright (c) 1997, 2010, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 1997, 2011, Oracle and/or its affiliates. All rights reserved.
  * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
  *
  * This code is free software; you can redistribute it and/or modify it
@@ -1179,6 +1179,8 @@
 // '%d' formats to indicate a 64-bit quantity; commonly "l" (in LP64) or "ll"
 // (in ILP32).
 
+#define BOOL_TO_STR(__b) (__b) ? "true" : "false"
+
 // Format 32-bit quantities.
 #define INT32_FORMAT  "%d"
 #define UINT32_FORMAT "%u"