diff src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp @ 1973:631f79e71e90

6974966: G1: unnecessary direct-to-old allocations Summary: This change revamps the slow allocation path of G1. Improvements include the following: a) Allocations directly to old regions are now totally banned. G1 now only allows allocations out of young regions (with the only exception being humongous regions). b) The thread that allocates a new region (which is now guaranteed to be young) does not dirty all its cards. Each thread that successfully allocates out of a young region is now responsible for dirtying the cards that corresponding to the "block" that just got allocated. c) allocate_new_tlab() and mem_allocate() are now implemented differently and TLAB allocations are only done by allocate_new_tlab(). d) If a thread schedules an evacuation pause in order to satisfy an allocation request, it will perform the allocation at the end of the safepoint so that the thread that initiated the GC also gets "first pick" of any space made available by the GC. e) If a thread is unable to allocate a humongous object it will schedule an evacuation pause in case it reclaims enough regions so that the humongous allocation can be satisfied aftewards. f) The G1 policy is more careful to set the young list target length to be the survivor number +1. g) Lots of code tidy up, removal, refactoring to make future changes easier. Reviewed-by: johnc, ysr
author tonyp
date Tue, 24 Aug 2010 17:24:33 -0400
parents f95d63e2154a
children fd1d227ef1b9
line wrap: on
line diff
--- a/src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp	Tue Nov 23 13:22:55 2010 -0800
+++ b/src/share/vm/gc_implementation/g1/g1CollectedHeap.cpp	Tue Aug 24 17:24:33 2010 -0400
@@ -58,10 +58,11 @@
 // INVARIANTS/NOTES
 //
 // All allocation activity covered by the G1CollectedHeap interface is
-//   serialized by acquiring the HeapLock.  This happens in
-//   mem_allocate_work, which all such allocation functions call.
-//   (Note that this does not apply to TLAB allocation, which is not part
-//   of this interface: it is done by clients of this interface.)
+// serialized by acquiring the HeapLock.  This happens in mem_allocate
+// and allocate_new_tlab, which are the "entry" points to the
+// allocation code from the rest of the JVM.  (Note that this does not
+// apply to TLAB allocation, which is not part of this interface: it
+// is done by clients of this interface.)
 
 // Local to this file.
 
@@ -536,18 +537,20 @@
 // 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::humongousObjAllocate(size_t word_size) {
+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 H 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())
+  // 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();
+  }
 
   size_t num_regions =
-    round_to(word_size, HeapRegion::GrainWords) / HeapRegion::GrainWords;
+         round_to(word_size, HeapRegion::GrainWords) / HeapRegion::GrainWords;
 
   // Special case if < one region???
 
@@ -598,153 +601,472 @@
   return res;
 }
 
+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();
+
+  retire_cur_alloc_region_common(cur_alloc_region);
+  assert(_cur_alloc_region == NULL, "post-condition");
+}
+
+// See the comment in the .hpp file about the locking protocol and
+// assumptions of this method (and other related ones).
 HeapWord*
-G1CollectedHeap::attempt_allocation_slow(size_t word_size,
-                                         bool permit_collection_pause) {
-  HeapWord* res = NULL;
-  HeapRegion* allocated_young_region = NULL;
-
-  assert( SafepointSynchronize::is_at_safepoint() ||
-          Heap_lock->owned_by_self(), "pre condition of the call" );
-
-  if (isHumongous(word_size)) {
-    // Allocation of a humongous object can, in a sense, complete a
-    // partial region, if the previous alloc was also humongous, and
-    // caused the test below to succeed.
-    if (permit_collection_pause)
-      do_collection_pause_if_appropriate(word_size);
-    res = humongousObjAllocate(word_size);
-    assert(_cur_alloc_region == NULL
-           || !_cur_alloc_region->isHumongous(),
-           "Prevent a regression of this bug.");
-
-  } else {
-    // 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.
-    if (!SafepointSynchronize::is_at_safepoint())
+G1CollectedHeap::replace_cur_alloc_region_and_allocate(size_t word_size,
+                                                       bool at_safepoint,
+                                                       bool do_dirtying) {
+  assert_heap_locked_or_at_safepoint();
+  assert(_cur_alloc_region == NULL,
+         "replace_cur_alloc_region_and_allocate() should only be called "
+         "after retiring the previous current alloc region");
+  assert(SafepointSynchronize::is_at_safepoint() == at_safepoint,
+         "at_safepoint and is_at_safepoint() should be a tautology");
+
+  if (!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();
-    // If we do a collection pause, this will be reset to a non-NULL
-    // value.  If we don't, nulling here ensures that we allocate a new
-    // region below.
-    if (_cur_alloc_region != NULL) {
-      // We're finished with the _cur_alloc_region.
-      // 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(_cur_alloc_region == NULL, "Invariant.");
-    // Completion of a heap region is perhaps a good point at which to do
-    // a collection pause.
-    if (permit_collection_pause)
-      do_collection_pause_if_appropriate(word_size);
-    // Make sure we have an allocation region available.
-    if (_cur_alloc_region == NULL) {
-      if (!SafepointSynchronize::is_at_safepoint())
-        wait_for_cleanup_complete();
-      bool next_is_young = should_set_young_locked();
-      // If the next region is not young, make sure it's zero-filled.
-      _cur_alloc_region = newAllocRegion(word_size, !next_is_young);
-      if (_cur_alloc_region != NULL) {
-        _summary_bytes_used -= _cur_alloc_region->used();
-        if (next_is_young) {
-          set_region_short_lived_locked(_cur_alloc_region);
-          allocated_young_region = _cur_alloc_region;
-        }
+
+    HeapRegion* new_cur_alloc_region = newAllocRegion(word_size,
+                                                      false /* zero_filled */);
+    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(),
+             "Catch a regression of this bug.");
+
+      // We need to ensure that the stores to _cur_alloc_region and,
+      // subsequently, to top do not float above the setting of the
+      // young type.
+      OrderAccess::storestore();
+
+      // Now allocate out of the new current alloc region. We could
+      // have re-used allocate_from_cur_alloc_region() but its
+      // operation is slightly different to what we need here. First,
+      // allocate_from_cur_alloc_region() is only called outside a
+      // safepoint and will always unlock the Heap_lock if it returns
+      // a non-NULL result. Second, it assumes that the current alloc
+      // region is what's already assigned in _cur_alloc_region. What
+      // we want here is to actually do the allocation first before we
+      // assign the new region to _cur_alloc_region. This ordering is
+      // not currently important, but it will be essential when we
+      // change the code to support CAS allocation in the future (see
+      // CR 6994297).
+      //
+      // This allocate method does BOT updates and we don't need them in
+      // the young generation. This will be fixed in the near future by
+      // CR 6994297.
+      HeapWord* result = new_cur_alloc_region->allocate(word_size);
+      assert(result != NULL, "we just allocate out of an empty region "
+             "so allocation should have been successful");
+      assert(is_in(result), "result should be in the heap");
+
+      _cur_alloc_region = new_cur_alloc_region;
+
+      if (!at_safepoint) {
+        Heap_lock->unlock();
+      }
+
+      // do the dirtying, if necessary, after we release the Heap_lock
+      if (do_dirtying) {
+        dirty_young_block(result, word_size);
+      }
+      return result;
+    }
+  }
+
+  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();
+  return NULL;
+}
+
+// See the comment in the .hpp file about the locking protocol and
+// assumptions of this method (and other related ones).
+HeapWord*
+G1CollectedHeap::attempt_allocation_slow(size_t word_size) {
+  assert_heap_locked_and_not_at_safepoint();
+  assert(!isHumongous(word_size), "attempt_allocation_slow() should not be "
+         "used for humongous allocations");
+
+  // 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
+  // successful, or we succeeded in doing the VM op but which was
+  // unable to allocate after the collection.
+  for (int try_count = 1; /* we'll return or break */; try_count += 1) {
+    bool succeeded = true;
+
+    {
+      // 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(word_size);
+      if (result != NULL) {
+        assert_heap_not_locked();
+        return result;
       }
     }
-    assert(_cur_alloc_region == NULL || !_cur_alloc_region->isHumongous(),
-           "Prevent a regression of this bug.");
-
-    // Now retry the allocation.
-    if (_cur_alloc_region != NULL) {
-      if (allocated_young_region != NULL) {
-        // We need to ensure that the store to top does not
-        // float above the setting of the young type.
-        OrderAccess::storestore();
+
+    if (GC_locker::is_active_and_needs_gc()) {
+      // We are locked out of GC because of the GC locker. Right now,
+      // we'll just stall until the GC locker-induced GC
+      // completes. This will be fixed in the near future by extending
+      // the eden while waiting for the GC locker to schedule the GC
+      // (see CR 6994056).
+
+      // If this thread is not in a jni critical section, we stall
+      // the requestor until the critical section has cleared and
+      // GC allowed. When the critical section clears, a GC is
+      // initiated by the last thread exiting the critical section; so
+      // we retry the allocation sequence from the beginning of the loop,
+      // rather than causing more, now probably unnecessary, GC attempts.
+      JavaThread* jthr = JavaThread::current();
+      assert(jthr != NULL, "sanity");
+      if (!jthr->in_critical()) {
+        MutexUnlocker mul(Heap_lock);
+        GC_locker::stall_until_clear();
+
+        // We'll then fall off the end of the ("if GC locker active")
+        // if-statement and retry the allocation further down in the
+        // loop.
+      } else {
+        if (CheckJNICalls) {
+          fatal("Possible deadlock due to allocating while"
+                " in jni critical section");
+        }
+        return NULL;
       }
-      res = _cur_alloc_region->allocate(word_size);
+    } else {
+      // We are not locked out. So, let's try to do a GC. The VM op
+      // will retry the allocation before it completes.
+
+      // Read the GC count while holding the Heap_lock
+      unsigned int gc_count_before = SharedHeap::heap()->total_collections();
+
+      Heap_lock->unlock();
+
+      HeapWord* result =
+        do_collection_pause(word_size, gc_count_before, &succeeded);
+      assert_heap_not_locked();
+      if (result != NULL) {
+        assert(succeeded, "the VM op should have succeeded");
+
+        // Allocations that take place on VM operations do not do any
+        // card dirtying and we have to do it here.
+        dirty_young_block(result, word_size);
+        return result;
+      }
+
+      Heap_lock->lock();
+    }
+
+    assert_heap_locked();
+
+    // We can reach here when we were unsuccessful in doing a GC,
+    // because another thread beat us to it, or because we were locked
+    // out of GC due to the GC locker. In either case a new alloc
+    // region might be available so we will retry the allocation.
+    HeapWord* result = attempt_allocation(word_size);
+    if (result != NULL) {
+      assert_heap_not_locked();
+      return result;
+    }
+
+    // So far our attempts to allocate failed. The only time we'll go
+    // around the loop and try again is if we tried to do a GC and the
+    // VM op that we tried to schedule was not successful because
+    // another thread beat us to it. If that happened it's possible
+    // that by the time we grabbed the Heap_lock again and tried to
+    // allocate other threads filled up the young generation, which
+    // means that the allocation attempt after the GC also failed. So,
+    // it's worth trying to schedule another GC pause.
+    if (succeeded) {
+      break;
+    }
+
+    // Give a warning if we seem to be looping forever.
+    if ((QueuedAllocationWarningCount > 0) &&
+        (try_count % QueuedAllocationWarningCount == 0)) {
+      warning("G1CollectedHeap::attempt_allocation_slow() "
+              "retries %d times", try_count);
     }
   }
 
-  // NOTE: fails frequently in PRT
-  assert(regions_accounted_for(), "Region leakage!");
-
-  if (res != NULL) {
-    if (!SafepointSynchronize::is_at_safepoint()) {
-      assert( permit_collection_pause, "invariant" );
-      assert( Heap_lock->owned_by_self(), "invariant" );
+  assert_heap_locked();
+  return NULL;
+}
+
+// See the comment in the .hpp file about the locking protocol and
+// assumptions of this method (and other related ones).
+HeapWord*
+G1CollectedHeap::attempt_allocation_humongous(size_t word_size,
+                                              bool at_safepoint) {
+  // This is the method that will allocate a humongous object. All
+  // 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(isHumongous(word_size), "attempt_allocation_humongous() "
+         "should only be used for humongous allocations");
+  assert(SafepointSynchronize::is_at_safepoint() == at_safepoint,
+         "at_safepoint and is_at_safepoint() should be a tautology");
+
+  HeapWord* result = 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
+  // successful, or we succeeded in doing the VM op but which was
+  // unable to allocate after the collection.
+  for (int try_count = 1; /* we'll return or break */; try_count += 1) {
+    bool succeeded = true;
+
+    // Given that humongous objects are not allocated in young
+    // regions, we'll first try to do the allocation without doing a
+    // collection hoping that there's enough space in the heap.
+    result = humongous_obj_allocate(word_size);
+    assert(_cur_alloc_region == NULL || !_cur_alloc_region->isHumongous(),
+           "catch a regression of this bug.");
+    if (result != NULL) {
+      if (!at_safepoint) {
+        // If we're not at a safepoint, unlock the Heap_lock.
+        Heap_lock->unlock();
+      }
+      return result;
+    }
+
+    // If we failed to allocate the humongous object, we should try to
+    // do a collection pause (if we're allowed) in case it reclaims
+    // enough space for the allocation to succeed after the pause.
+    if (!at_safepoint) {
+      // Read the GC count while holding the Heap_lock
+      unsigned int gc_count_before = SharedHeap::heap()->total_collections();
+
+      // If we're allowed to do a collection we're not at a
+      // safepoint, so it is safe to unlock the Heap_lock.
       Heap_lock->unlock();
+
+      result = do_collection_pause(word_size, gc_count_before, &succeeded);
+      assert_heap_not_locked();
+      if (result != NULL) {
+        assert(succeeded, "the VM op should have succeeded");
+        return result;
+      }
+
+      // If we get here, the VM operation either did not succeed
+      // (i.e., another thread beat us to it) or it succeeded but
+      // failed to allocate the object.
+
+      // If we're allowed to do a collection we're not at a
+      // safepoint, so it is safe to lock the Heap_lock.
+      Heap_lock->lock();
+    }
+
+    assert(result == NULL, "otherwise we should have exited the loop earlier");
+
+    // So far our attempts to allocate failed. The only time we'll go
+    // around the loop and try again is if we tried to do a GC and the
+    // VM op that we tried to schedule was not successful because
+    // another thread beat us to it. That way it's possible that some
+    // space was freed up by the thread that successfully scheduled a
+    // GC. So it's worth trying to allocate again.
+    if (succeeded) {
+      break;
     }
 
-    if (allocated_young_region != NULL) {
-      HeapRegion* hr = allocated_young_region;
-      HeapWord* bottom = hr->bottom();
-      HeapWord* end = hr->end();
-      MemRegion mr(bottom, end);
-      ((CardTableModRefBS*)_g1h->barrier_set())->dirty(mr);
+    // Give a warning if we seem to be looping forever.
+    if ((QueuedAllocationWarningCount > 0) &&
+        (try_count % QueuedAllocationWarningCount == 0)) {
+      warning("G1CollectedHeap::attempt_allocation_humongous "
+              "retries %d times", try_count);
+    }
+  }
+
+  assert_heap_locked_or_at_safepoint();
+  return NULL;
+}
+
+HeapWord* G1CollectedHeap::attempt_allocation_at_safepoint(size_t word_size,
+                                           bool expect_null_cur_alloc_region) {
+  assert_at_safepoint();
+  assert(_cur_alloc_region == NULL || !expect_null_cur_alloc_region,
+         "The current alloc region should only be non-NULL if we're "
+         "expecting it not to be NULL");
+
+  if (!isHumongous(word_size)) {
+    if (!expect_null_cur_alloc_region) {
+      HeapRegion* cur_alloc_region = _cur_alloc_region;
+      if (cur_alloc_region != NULL) {
+        // This allocate method does BOT updates and we don't need them in
+        // the young generation. This will be fixed in the near future by
+        // CR 6994297.
+        HeapWord* result = cur_alloc_region->allocate(word_size);
+        if (result != NULL) {
+          assert(is_in(result), "result should be in the heap");
+
+          // We will not do any dirtying here. This is guaranteed to be
+          // called during a safepoint and the thread that scheduled the
+          // pause will do the dirtying if we return a non-NULL result.
+          return result;
+        }
+
+        retire_cur_alloc_region_common(cur_alloc_region);
+      }
     }
-  }
-
-  assert( SafepointSynchronize::is_at_safepoint() ||
-          (res == NULL && Heap_lock->owned_by_self()) ||
-          (res != NULL && !Heap_lock->owned_by_self()),
-          "post condition of the call" );
-
-  return res;
+
+    assert(_cur_alloc_region == NULL,
+           "at this point we should have no cur alloc region");
+    return replace_cur_alloc_region_and_allocate(word_size,
+                                                 true, /* at_safepoint */
+                                                 false /* do_dirtying */);
+  } else {
+    return attempt_allocation_humongous(word_size,
+                                        true /* at_safepoint */);
+  }
+
+  ShouldNotReachHere();
+}
+
+HeapWord* G1CollectedHeap::allocate_new_tlab(size_t word_size) {
+  assert_heap_not_locked_and_not_at_safepoint();
+  assert(!isHumongous(word_size), "we do not allow TLABs of humongous size");
+
+  Heap_lock->lock();
+
+  // First attempt: try allocating out of the current alloc region or
+  // after replacing the current alloc region.
+  HeapWord* result = attempt_allocation(word_size);
+  if (result != NULL) {
+    assert_heap_not_locked();
+    return result;
+  }
+
+  assert_heap_locked();
+
+  // Second attempt: go into the even slower path where we might
+  // try to schedule a collection.
+  result = attempt_allocation_slow(word_size);
+  if (result != NULL) {
+    assert_heap_not_locked();
+    return result;
+  }
+
+  assert_heap_locked();
+  Heap_lock->unlock();
+  return NULL;
 }
 
 HeapWord*
 G1CollectedHeap::mem_allocate(size_t word_size,
                               bool   is_noref,
                               bool   is_tlab,
-                              bool* gc_overhead_limit_was_exceeded) {
-  debug_only(check_for_valid_allocation_state());
-  assert(no_gc_in_progress(), "Allocation during gc not allowed");
-  HeapWord* result = NULL;
+                              bool*  gc_overhead_limit_was_exceeded) {
+  assert_heap_not_locked_and_not_at_safepoint();
+  assert(!is_tlab, "mem_allocate() this should not be called directly "
+         "to allocate TLABs");
 
   // Loop until the allocation is satisified,
   // or unsatisfied after GC.
-  for (int try_count = 1; /* return or throw */; try_count += 1) {
-    int gc_count_before;
+  for (int try_count = 1; /* we'll return */; try_count += 1) {
+    unsigned int gc_count_before;
     {
       Heap_lock->lock();
-      result = attempt_allocation(word_size);
-      if (result != NULL) {
-        // attempt_allocation should have unlocked the heap lock
-        assert(is_in(result), "result not in heap");
-        return result;
+
+      if (!isHumongous(word_size)) {
+        // First attempt: try allocating out of the current alloc
+        // region or after replacing the current alloc region.
+        HeapWord* result = attempt_allocation(word_size);
+        if (result != NULL) {
+          assert_heap_not_locked();
+          return result;
+        }
+
+        assert_heap_locked();
+
+        // Second attempt: go into the even slower path where we might
+        // try to schedule a collection.
+        result = attempt_allocation_slow(word_size);
+        if (result != NULL) {
+          assert_heap_not_locked();
+          return result;
+        }
+      } else {
+        HeapWord* result = attempt_allocation_humongous(word_size,
+                                                     false /* at_safepoint */);
+        if (result != NULL) {
+          assert_heap_not_locked();
+          return result;
+        }
       }
+
+      assert_heap_locked();
       // Read the gc count while the heap lock is held.
       gc_count_before = SharedHeap::heap()->total_collections();
+      // We cannot be at a safepoint, so it is safe to unlock the Heap_lock
       Heap_lock->unlock();
     }
 
     // Create the garbage collection operation...
-    VM_G1CollectForAllocation op(word_size,
-                                 gc_count_before);
-
+    VM_G1CollectForAllocation op(gc_count_before, word_size);
     // ...and get the VM thread to execute it.
     VMThread::execute(&op);
-    if (op.prologue_succeeded()) {
-      result = op.result();
-      assert(result == NULL || is_in(result), "result not in heap");
+
+    assert_heap_not_locked();
+    if (op.prologue_succeeded() && op.pause_succeeded()) {
+      // If the operation was successful we'll return the result even
+      // if it is NULL. If the allocation attempt failed immediately
+      // after a Full GC, it's unlikely we'll be able to allocate now.
+      HeapWord* result = op.result();
+      if (result != NULL && !isHumongous(word_size)) {
+        // Allocations that take place on VM operations do not do any
+        // card dirtying and we have to do it here. We only have to do
+        // this for non-humongous allocations, though.
+        dirty_young_block(result, word_size);
+      }
       return result;
+    } else {
+      assert(op.result() == NULL,
+             "the result should be NULL if the VM op did not succeed");
     }
 
     // Give a warning if we seem to be looping forever.
     if ((QueuedAllocationWarningCount > 0) &&
         (try_count % QueuedAllocationWarningCount == 0)) {
-      warning("G1CollectedHeap::mem_allocate_work retries %d times",
-              try_count);
+      warning("G1CollectedHeap::mem_allocate retries %d times", try_count);
     }
   }
+
+  ShouldNotReachHere();
 }
 
 void G1CollectedHeap::abandon_cur_alloc_region() {
@@ -841,11 +1163,11 @@
   }
 };
 
-void G1CollectedHeap::do_collection(bool explicit_gc,
+bool G1CollectedHeap::do_collection(bool explicit_gc,
                                     bool clear_all_soft_refs,
                                     size_t word_size) {
   if (GC_locker::check_active_before_gc()) {
-    return; // GC is disabled (e.g. JNI GetXXXCritical operation)
+    return false;
   }
 
   ResourceMark rm;
@@ -1047,12 +1369,19 @@
   if (PrintHeapAtGC) {
     Universe::print_heap_after_gc();
   }
+
+  return true;
 }
 
 void G1CollectedHeap::do_full_collection(bool clear_all_soft_refs) {
-  do_collection(true,                /* explicit_gc */
-                clear_all_soft_refs,
-                0                    /* word_size */);
+  // do_collection() will return whether it succeeded in performing
+  // the GC. Currently, there is no facility on the
+  // do_full_collection() API to notify the caller than the collection
+  // did not succeed (e.g., because it was locked out by the GC
+  // locker). So, right now, we'll ignore the return value.
+  bool dummy = do_collection(true,                /* explicit_gc */
+                             clear_all_soft_refs,
+                             0                    /* word_size */);
 }
 
 // This code is mostly copied from TenuredGeneration.
@@ -1175,46 +1504,74 @@
 
 
 HeapWord*
-G1CollectedHeap::satisfy_failed_allocation(size_t word_size) {
-  HeapWord* result = NULL;
+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");
+
+  *succeeded = true;
+  // Let's attempt the allocation first.
+  HeapWord* result = attempt_allocation_at_safepoint(word_size,
+                                     false /* expect_null_cur_alloc_region */);
+  if (result != NULL) {
+    assert(*succeeded, "sanity");
+    return result;
+  }
 
   // In a G1 heap, we're supposed to keep allocation from failing by
   // incremental pauses.  Therefore, at least for now, we'll favor
   // expansion over collection.  (This might change in the future if we can
   // do something smarter than full collection to satisfy a failed alloc.)
-
   result = expand_and_allocate(word_size);
   if (result != NULL) {
-    assert(is_in(result), "result not in heap");
+    assert(*succeeded, "sanity");
     return result;
   }
 
-  // OK, I guess we have to try collection.
-
-  do_collection(false, false, word_size);
-
-  result = attempt_allocation(word_size, /*permit_collection_pause*/false);
-
+  // Expansion didn't work, we'll try to do a Full GC.
+  bool gc_succeeded = do_collection(false, /* explicit_gc */
+                                    false, /* clear_all_soft_refs */
+                                    word_size);
+  if (!gc_succeeded) {
+    *succeeded = false;
+    return NULL;
+  }
+
+  // Retry the allocation
+  result = attempt_allocation_at_safepoint(word_size,
+                                      true /* expect_null_cur_alloc_region */);
   if (result != NULL) {
-    assert(is_in(result), "result not in heap");
+    assert(*succeeded, "sanity");
     return result;
   }
 
-  // Try collecting soft references.
-  do_collection(false, true, word_size);
-  result = attempt_allocation(word_size, /*permit_collection_pause*/false);
+  // Then, try a Full GC that will collect all soft references.
+  gc_succeeded = do_collection(false, /* explicit_gc */
+                               true,  /* clear_all_soft_refs */
+                               word_size);
+  if (!gc_succeeded) {
+    *succeeded = false;
+    return NULL;
+  }
+
+  // Retry the allocation once more
+  result = attempt_allocation_at_safepoint(word_size,
+                                      true /* expect_null_cur_alloc_region */);
   if (result != NULL) {
-    assert(is_in(result), "result not in heap");
+    assert(*succeeded, "sanity");
     return result;
   }
 
   assert(!collector_policy()->should_clear_all_soft_refs(),
-    "Flag should have been handled and cleared prior to this point");
+         "Flag should have been handled and cleared prior to this point");
 
   // What else?  We might try synchronous finalization later.  If the total
   // space available is large enough for the allocation, then a more
   // complete compaction phase than we've tried so far might be
   // appropriate.
+  assert(*succeeded, "sanity");
   return NULL;
 }
 
@@ -1224,14 +1581,20 @@
 // 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");
+
   size_t expand_bytes = word_size * HeapWordSize;
   if (expand_bytes < MinHeapDeltaBytes) {
     expand_bytes = MinHeapDeltaBytes;
   }
   expand(expand_bytes);
   assert(regions_accounted_for(), "Region leakage!");
-  HeapWord* result = attempt_allocation(word_size, false /* permit_collection_pause */);
-  return result;
+
+  return attempt_allocation_at_safepoint(word_size,
+                                      true /* expect_null_cur_alloc_region */);
 }
 
 size_t G1CollectedHeap::free_region_if_totally_empty(HeapRegion* hr) {
@@ -1842,21 +2205,25 @@
   unsigned int full_gc_count_before;
   {
     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();
-
-    // Don't want to do a GC until cleanup is completed.
-    wait_for_cleanup_complete();
-
-    // We give up heap lock; VMThread::execute gets it back below
   }
 
   if (should_do_concurrent_full_gc(cause)) {
     // Schedule an initial-mark evacuation pause that will start a
-    // concurrent cycle.
+    // concurrent cycle. We're setting word_size to 0 which means that
+    // we are not requesting a post-GC allocation.
     VM_G1IncCollectionPause op(gc_count_before,
-                               true, /* should_initiate_conc_mark */
+                               0,     /* word_size */
+                               true,  /* should_initiate_conc_mark */
                                g1_policy()->max_pause_time_ms(),
                                cause);
     VMThread::execute(&op);
@@ -1864,8 +2231,10 @@
     if (cause == GCCause::_gc_locker
         DEBUG_ONLY(|| cause == GCCause::_scavenge_alot)) {
 
-      // Schedule a standard evacuation pause.
+      // Schedule a standard evacuation pause. We're setting word_size
+      // to 0 which means that we are not requesting a post-GC allocation.
       VM_G1IncCollectionPause op(gc_count_before,
+                                 0,     /* word_size */
                                  false, /* should_initiate_conc_mark */
                                  g1_policy()->max_pause_time_ms(),
                                  cause);
@@ -2221,14 +2590,6 @@
   }
 }
 
-HeapWord* G1CollectedHeap::allocate_new_tlab(size_t word_size) {
-  assert(!isHumongous(word_size),
-         err_msg("a TLAB should not be of humongous size, "
-                 "word_size = "SIZE_FORMAT, word_size));
-  bool dummy;
-  return G1CollectedHeap::mem_allocate(word_size, false, true, &dummy);
-}
-
 bool G1CollectedHeap::allocs_are_zero_filled() {
   return false;
 }
@@ -2633,27 +2994,26 @@
   // always_do_update_barrier = true;
 }
 
-void G1CollectedHeap::do_collection_pause() {
-  assert(Heap_lock->owned_by_self(), "we assume we'reholding the Heap_lock");
-
-  // Read the GC count while holding the Heap_lock
-  // we need to do this _before_ wait_for_cleanup_complete(), to
-  // ensure that we do not give up the heap lock and potentially
-  // pick up the wrong count
-  unsigned int gc_count_before = SharedHeap::heap()->total_collections();
-
-  // Don't want to do a GC pause while cleanup is being completed!
-  wait_for_cleanup_complete();
-
+HeapWord* G1CollectedHeap::do_collection_pause(size_t word_size,
+                                               unsigned int gc_count_before,
+                                               bool* succeeded) {
+  assert_heap_not_locked_and_not_at_safepoint();
   g1_policy()->record_stop_world_start();
-  {
-    MutexUnlocker mu(Heap_lock);  // give up heap lock, execute gets it back
-    VM_G1IncCollectionPause op(gc_count_before,
-                               false, /* should_initiate_conc_mark */
-                               g1_policy()->max_pause_time_ms(),
-                               GCCause::_g1_inc_collection_pause);
-    VMThread::execute(&op);
-  }
+  VM_G1IncCollectionPause op(gc_count_before,
+                             word_size,
+                             false, /* should_initiate_conc_mark */
+                             g1_policy()->max_pause_time_ms(),
+                             GCCause::_g1_inc_collection_pause);
+  VMThread::execute(&op);
+
+  HeapWord* result = op.result();
+  bool ret_succeeded = op.prologue_succeeded() && op.pause_succeeded();
+  assert(result == NULL || ret_succeeded,
+         "the result should be NULL if the VM did not succeed");
+  *succeeded = ret_succeeded;
+
+  assert_heap_not_locked();
+  return result;
 }
 
 void
@@ -2797,10 +3157,10 @@
 }
 #endif // TASKQUEUE_STATS
 
-void
+bool
 G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
   if (GC_locker::check_active_before_gc()) {
-    return; // GC is disabled (e.g. JNI GetXXXCritical operation)
+    return false;
   }
 
   if (PrintHeapAtGC) {
@@ -3068,6 +3428,8 @@
       (total_collections() % G1SummarizeRSetStatsPeriod == 0)) {
     g1_rem_set()->print_summary_info();
   }
+
+  return true;
 }
 
 size_t G1CollectedHeap::desired_plab_sz(GCAllocPurpose purpose)
@@ -3361,19 +3723,6 @@
 
 // *** Sequential G1 Evacuation
 
-HeapWord* G1CollectedHeap::allocate_during_gc(GCAllocPurpose purpose, size_t word_size) {
-  HeapRegion* alloc_region = _gc_alloc_regions[purpose];
-  // let the caller handle alloc failure
-  if (alloc_region == NULL) return NULL;
-  assert(isHumongous(word_size) || !alloc_region->isHumongous(),
-         "Either the object is humongous or the region isn't");
-  HeapWord* block = alloc_region->allocate(word_size);
-  if (block == NULL) {
-    block = allocate_during_gc_slow(purpose, alloc_region, false, word_size);
-  }
-  return block;
-}
-
 class G1IsAliveClosure: public BoolObjectClosure {
   G1CollectedHeap* _g1;
 public:
@@ -4625,12 +4974,6 @@
 #endif
 }
 
-void G1CollectedHeap::do_collection_pause_if_appropriate(size_t word_size) {
-  if (g1_policy()->should_do_collection_pause(word_size)) {
-    do_collection_pause();
-  }
-}
-
 void G1CollectedHeap::free_collection_set(HeapRegion* cs_head) {
   double young_time_ms     = 0.0;
   double non_young_time_ms = 0.0;
@@ -4789,6 +5132,7 @@
 }
 
 void G1CollectedHeap::wait_for_cleanup_complete() {
+  assert_not_at_safepoint();
   MutexLockerEx x(Cleanup_mon);
   wait_for_cleanup_complete_locked();
 }
@@ -5093,13 +5437,6 @@
   return n + m;
 }
 
-bool G1CollectedHeap::should_set_young_locked() {
-  assert(heap_lock_held_for_gc(),
-              "the heap lock should already be held by or for this thread");
-  return  (g1_policy()->in_young_gc_mode() &&
-           g1_policy()->should_add_next_region_to_young_list());
-}
-
 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");