Index: lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/DocumentsWriter.java	(revision 1102572)
+++ lucene/src/java/org/apache/lucene/index/DocumentsWriter.java	(working copy)
@@ -126,7 +126,6 @@
   final DocumentsWriterPerThreadPool perThreadPool;
   final FlushPolicy flushPolicy;
   final DocumentsWriterFlushControl flushControl;
-  final Healthiness healthiness;
   DocumentsWriter(IndexWriterConfig config, Directory directory, IndexWriter writer, FieldNumberBiMap globalFieldNumbers,
       BufferedDeletesStream bufferedDeletesStream) throws IOException {
     this.directory = directory;
@@ -143,9 +142,8 @@
     }
     flushPolicy.init(this);
     
-    healthiness = new Healthiness();
     final long maxRamPerDWPT = config.getRAMPerThreadHardLimitMB() * 1024 * 1024;
-    flushControl = new DocumentsWriterFlushControl(this, healthiness, maxRamPerDWPT);
+    flushControl = new DocumentsWriterFlushControl(this, maxRamPerDWPT);
   }
 
   synchronized void deleteQueries(final Query... queries) throws IOException {
@@ -283,32 +281,29 @@
     ensureOpen();
     boolean maybeMerge = false;
     final boolean isUpdate = delTerm != null;
-    if (healthiness.anyStalledThreads()) {
-
-      // Help out flushing any pending DWPTs so we can un-stall:
+    if (flushControl.anyStalledThreads() || flushControl.numQueuedFlushes() > 0) {
+      // Help out flushing any queued DWPTs so we can un-stall:
       if (infoStream != null) {
-        message("WARNING DocumentsWriter has stalled threads; will hijack this thread to flush pending segment(s)");
+        message("DocumentsWriter has queued dwpt; will hijack this thread to flush pending segment(s)");
       }
-
-      // Try pick up pending threads here if possible
-      DocumentsWriterPerThread flushingDWPT;
-      while ((flushingDWPT = flushControl.nextPendingFlush()) != null) {
-        // Don't push the delete here since the update could fail!
-        maybeMerge = doFlush(flushingDWPT);
-        if (!healthiness.anyStalledThreads()) {
-          break;
+      do {
+        // Try pick up pending threads here if possible
+        DocumentsWriterPerThread flushingDWPT;
+        while ((flushingDWPT = flushControl.nextPendingFlush()) != null) {
+          // Don't push the delete here since the update could fail!
+          maybeMerge |= doFlush(flushingDWPT);
         }
-      }
+  
+        if (infoStream != null && flushControl.anyStalledThreads()) {
+          message("WARNING DocumentsWriter has stalled threads; waiting");
+        }
+        
+        flushControl.waitIfStalled(); // block if stalled
+      } while (flushControl.numQueuedFlushes() != 0); // still queued DWPTs try help flushing
 
-      if (infoStream != null && healthiness.anyStalledThreads()) {
-        message("WARNING DocumentsWriter still has stalled threads; waiting");
+      if (infoStream != null) {
+        message("continue indexing after helpling out flushing DocumentsWriter is healthy");
       }
-
-      healthiness.waitIfStalled(); // block if stalled
-
-      if (infoStream != null && healthiness.anyStalledThreads()) {
-        message("WARNING DocumentsWriter done waiting");
-      }
     }
 
     final ThreadState perThread = perThreadPool.getAndLock(Thread.currentThread(),
@@ -353,7 +348,6 @@
       maybeMerge = true;
       boolean success = false;
       FlushTicket ticket = null;
-      
       try {
         assert currentFullFlushDelQueue == null
             || flushingDWPT.deleteQueue == currentFullFlushDelQueue : "expected: "
@@ -511,9 +505,7 @@
         anythingFlushed |= doFlush(flushingDWPT);
       }
       // If a concurrent flush is still in flight wait for it
-      while (flushControl.anyFlushing()) {
-        flushControl.waitForFlush();  
-      }
+      flushControl.waitForFlush();  
       if (!anythingFlushed) { // apply deletes if we did not flush any document
         synchronized (ticketQueue) {
           ticketQueue.add(new FlushTicket(flushingDeleteQueue.freezeGlobalBuffer(null), false));
Index: lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java	(revision 1102572)
+++ lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java	(working copy)
@@ -44,18 +44,17 @@
   private long activeBytes = 0;
   private long flushBytes = 0;
   private volatile int numPending = 0;
-  private volatile int numFlushing = 0;
   final AtomicBoolean flushDeletes = new AtomicBoolean(false);
   private boolean fullFlush = false;
-  private Queue<DocumentsWriterPerThread> flushQueue = new LinkedList<DocumentsWriterPerThread>();
+  private final Queue<DocumentsWriterPerThread> flushQueue = new LinkedList<DocumentsWriterPerThread>();
   // only for safety reasons if a DWPT is close to the RAM limit
-  private Queue<DocumentsWriterPerThread> blockedFlushes = new LinkedList<DocumentsWriterPerThread>();
+  private final Queue<BlockedFlush> blockedFlushes = new LinkedList<BlockedFlush>();
 
 
   long peakActiveBytes = 0;// only with assert
   long peakFlushBytes = 0;// only with assert
   long peakNetBytes = 0;// only with assert
-  private final Healthiness healthiness;
+  final DocumentsWriterStallControl stallControl;
   private final DocumentsWriterPerThreadPool perThreadPool;
   private final FlushPolicy flushPolicy;
   private boolean closed = false;
@@ -63,8 +62,8 @@
   private final DocumentsWriter documentsWriter;
 
   DocumentsWriterFlushControl(DocumentsWriter documentsWriter,
-      Healthiness healthiness, long hardMaxBytesPerDWPT) {
-    this.healthiness = healthiness;
+      long hardMaxBytesPerDWPT) {
+    this.stallControl = new DocumentsWriterStallControl();
     this.perThreadPool = documentsWriter.perThreadPool;
     this.flushPolicy = documentsWriter.flushPolicy;
     this.hardMaxBytesPerDWPT = hardMaxBytesPerDWPT;
@@ -110,48 +109,53 @@
 
   synchronized DocumentsWriterPerThread doAfterDocument(ThreadState perThread,
       boolean isUpdate) {
-    commitPerThreadBytes(perThread);
-    if (!perThread.flushPending) {
-      if (isUpdate) {
-        flushPolicy.onUpdate(this, perThread);
-      } else {
-        flushPolicy.onInsert(this, perThread);
+    try {
+      commitPerThreadBytes(perThread);
+      if (!perThread.flushPending) {
+        if (isUpdate) {
+          flushPolicy.onUpdate(this, perThread);
+        } else {
+          flushPolicy.onInsert(this, perThread);
+        }
+        if (!perThread.flushPending && perThread.bytesUsed > hardMaxBytesPerDWPT) {
+          // Safety check to prevent a single DWPT exceeding its RAM limit. This
+          // is super important since we can not address more than 2048 MB per DWPT
+          setFlushPending(perThread);
+        }
       }
-      if (!perThread.flushPending && perThread.bytesUsed > hardMaxBytesPerDWPT) {
-        // Safety check to prevent a single DWPT exceeding its RAM limit. This
-        // is super important since we can not address more than 2048 MB per DWPT
-        setFlushPending(perThread);
-        if (fullFlush) {
-          DocumentsWriterPerThread toBlock = internalTryCheckOutForFlush(perThread);
-          assert toBlock != null;
-          blockedFlushes.add(toBlock);
+      final DocumentsWriterPerThread flushingDWPT;
+      if (fullFlush) {
+        if (perThread.flushPending) {
+          checkoutAndBlock(perThread);
+          flushingDWPT = nextPendingFlush();
+        } else {
+          flushingDWPT = null;
         }
+      } else {
+       flushingDWPT = tryCheckoutForFlush(perThread);
       }
+      return flushingDWPT;
+    } finally {
+      stallControl.updateStalled(this);  
     }
-    final DocumentsWriterPerThread flushingDWPT = tryCheckoutForFlush(perThread);
-    healthiness.updateStalled(this);
-    return flushingDWPT;
+    
+    
   }
 
   synchronized void doAfterFlush(DocumentsWriterPerThread dwpt) {
     assert flushingWriters.containsKey(dwpt);
     try {
-      numFlushing--;
       Long bytes = flushingWriters.remove(dwpt);
       flushBytes -= bytes.longValue();
       perThreadPool.recycle(dwpt);
-      healthiness.updateStalled(this);
+      stallControl.updateStalled(this);
     } finally {
       notifyAll();
     }
   }
   
-  public synchronized boolean anyFlushing() {
-    return numFlushing != 0;
-  }
-  
   public synchronized void waitForFlush() {
-    if (numFlushing != 0) {
+    while (flushingWriters.size() != 0) {
       try {
         this.wait();
       } catch (InterruptedException e) {
@@ -178,27 +182,44 @@
   }
 
   synchronized void doOnAbort(ThreadState state) {
-    if (state.flushPending) {
-      flushBytes -= state.bytesUsed;
-    } else {
-      activeBytes -= state.bytesUsed;
+    try {
+      if (state.flushPending) {
+        flushBytes -= state.bytesUsed;
+      } else {
+        activeBytes -= state.bytesUsed;
+      }
+      // Take it out of the loop this DWPT is stale
+      perThreadPool.replaceForFlush(state, closed);
+    }finally {
+      stallControl.updateStalled(this);
     }
-    // Take it out of the loop this DWPT is stale
-    perThreadPool.replaceForFlush(state, closed);
-    healthiness.updateStalled(this);
   }
 
   synchronized DocumentsWriterPerThread tryCheckoutForFlush(
       ThreadState perThread) {
-    if (fullFlush) {
-      return null;
+   return perThread.flushPending ? internalTryCheckOutForFlush(perThread) : null;
+  }
+  
+  private void checkoutAndBlock(ThreadState perThread) {
+    perThread.lock();
+    try {
+      assert perThread.flushPending : "can not block non-pending threadstate";
+      assert fullFlush : "can not block if fullFlush == false";
+      final DocumentsWriterPerThread dwpt;
+      final long bytes = perThread.bytesUsed;
+      dwpt = perThreadPool.replaceForFlush(perThread, closed);
+      numPending--;
+      blockedFlushes.add(new BlockedFlush(dwpt, bytes));
+    }finally {
+      perThread.unlock();
     }
-    return internalTryCheckOutForFlush(perThread);
   }
 
   private DocumentsWriterPerThread internalTryCheckOutForFlush(
       ThreadState perThread) {
-    if (perThread.flushPending) {
+    assert Thread.holdsLock(this);
+    assert perThread.flushPending;
+    try {
       // We are pending so all memory is already moved to flushBytes
       if (perThread.tryLock()) {
         try {
@@ -212,15 +233,16 @@
             // Record the flushing DWPT to reduce flushBytes in doAfterFlush
             flushingWriters.put(dwpt, Long.valueOf(bytes));
             numPending--; // write access synced
-            numFlushing++;
             return dwpt;
           }
         } finally {
           perThread.unlock();
         }
       }
+      return null;
+    } finally {
+      stallControl.updateStalled(this);
     }
-    return null;
   }
 
   @Override
@@ -231,12 +253,13 @@
 
   DocumentsWriterPerThread nextPendingFlush() {
     synchronized (this) {
-      DocumentsWriterPerThread poll = flushQueue.poll();
-      if (poll != null) {
+      final DocumentsWriterPerThread poll;
+      if ((poll = flushQueue.poll()) != null) {
+        stallControl.updateStalled(this);
         return poll;
-      }  
+      }
     }
-    if (numPending > 0) {
+    if (numPending > 0 && !fullFlush) { // don't check if we are doing a full flush
       final Iterator<ThreadState> allActiveThreads = perThreadPool
           .getActivePerThreadsIterator();
       while (allActiveThreads.hasNext() && numPending > 0) {
@@ -276,8 +299,8 @@
     return documentsWriter.deleteQueue.numGlobalTermDeletes();
   }
 
-  int numFlushingDWPT() {
-    return numFlushing;
+  synchronized int numFlushingDWPT() {
+    return flushingWriters.size();
   }
   
   public boolean doApplyAllDeletes() {	
@@ -289,7 +312,7 @@
   }
   
   int numActiveDWPT() {
-    return this.perThreadPool.getMaxThreadStates();
+    return this.perThreadPool.getActiveThreadState();
   }
   
   void markForFullFlush() {
@@ -331,11 +354,11 @@
             if (!next.flushPending) {
               setFlushPending(next);
             }
+            final DocumentsWriterPerThread flushingDWPT = internalTryCheckOutForFlush(next);
+            assert flushingDWPT != null : "DWPT must never be null here since we hold the lock and it holds documents";
+            assert dwpt == flushingDWPT : "flushControl returned different DWPT";
+            toFlush.add(flushingDWPT);
           }
-          final DocumentsWriterPerThread flushingDWPT = internalTryCheckOutForFlush(next);
-          assert flushingDWPT != null : "DWPT must never be null here since we hold the lock and it holds documents";
-          assert dwpt == flushingDWPT : "flushControl returned different DWPT";
-          toFlush.add(flushingDWPT);
         } else {
           // get the new delete queue from DW
           next.perThread.initialize();
@@ -345,31 +368,54 @@
       }
     }
     synchronized (this) {
-      assert assertBlockedFlushes(flushingQueue);
-      flushQueue.addAll(blockedFlushes);
-      blockedFlushes.clear();
+      /* make sure we move all DWPT that are where concurrently marked as
+       * pending and moved to blocked are moved over to the flushQueue. There is
+       * a chance that this happens since we marking DWPT for full flush without
+       * blocking indexing.*/
+      pruneBlockedQueue(flushingQueue);   
+      assert assertBlockedFlushes(documentsWriter.deleteQueue);
       flushQueue.addAll(toFlush);
+      stallControl.updateStalled(this);
     }
   }
   
+  /**
+   * Prunes the blockedQueue by removing all DWPT that are associated with the given flush queue. 
+   */
+  private void pruneBlockedQueue(final DocumentsWriterDeleteQueue flushingQueue) {
+    Iterator<BlockedFlush> iterator = blockedFlushes.iterator();
+    while (iterator.hasNext()) {
+      BlockedFlush blockedFlush = iterator.next();
+      if (blockedFlush.dwpt.deleteQueue == flushingQueue) {
+        iterator.remove();
+        assert !flushingWriters.containsKey(blockedFlush.dwpt) : "DWPT is already flushing";
+        // Record the flushing DWPT to reduce flushBytes in doAfterFlush
+        flushingWriters.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
+        // don't decr pending here - its already done when DWPT is blocked
+        flushQueue.add(blockedFlush.dwpt);
+      }
+    }
+  }
+  
   synchronized void finishFullFlush() {
     assert fullFlush;
     assert flushQueue.isEmpty();
+    assert flushingWriters.isEmpty();
     try {
       if (!blockedFlushes.isEmpty()) {
         assert assertBlockedFlushes(documentsWriter.deleteQueue);
-        flushQueue.addAll(blockedFlushes);
-        blockedFlushes.clear();
+        pruneBlockedQueue(documentsWriter.deleteQueue);
+        assert blockedFlushes.isEmpty();
       }
     } finally {
       fullFlush = false;
+      stallControl.updateStalled(this);
     }
   }
   
   boolean assertBlockedFlushes(DocumentsWriterDeleteQueue flushingQueue) {
-    Queue<DocumentsWriterPerThread> flushes = this.blockedFlushes;
-    for (DocumentsWriterPerThread documentsWriterPerThread : flushes) {
-      assert documentsWriterPerThread.deleteQueue == flushingQueue;
+    for (BlockedFlush blockedFlush : blockedFlushes) {
+      assert blockedFlush.dwpt.deleteQueue == flushingQueue;
     }
     return true;
   }
@@ -379,18 +425,65 @@
       for (DocumentsWriterPerThread dwpt : flushQueue) {
         doAfterFlush(dwpt);
       }
-      for (DocumentsWriterPerThread dwpt : blockedFlushes) {
-        doAfterFlush(dwpt);
+      for (BlockedFlush blockedFlush : blockedFlushes) {
+        flushingWriters.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
+        doAfterFlush(blockedFlush.dwpt);
       }
-      
     } finally {
       fullFlush = false;
       flushQueue.clear();
       blockedFlushes.clear();
+      stallControl.updateStalled(this);
     }
   }
   
-  synchronized boolean isFullFlush() {
+  /**
+   * Returns <code>true</code> if a full flush is currently running
+   */
+  synchronized boolean isFullFlush() { // used by assert
     return fullFlush;
   }
+
+  /**
+   * Returns the number of flushes that are already checked out but not yet
+   * actively flushing
+   */
+  synchronized int numQueuedFlushes() {
+    return flushQueue.size();
+  }
+
+  /**
+   * Returns the number of flushes that are checked out but not yet available
+   * for flushing. This only applies during a full flush if a DWPT needs
+   * flushing but must not be flushed until the full flush has finished.
+   */
+  synchronized int numBlockedFlushes() {
+    return blockedFlushes.size();
+  }
+  
+  private static class BlockedFlush {
+    final DocumentsWriterPerThread dwpt;
+    final long bytes;
+    BlockedFlush(DocumentsWriterPerThread dwpt, long bytes) {
+      super();
+      this.dwpt = dwpt;
+      this.bytes = bytes;
+    }
+  }
+
+  /**
+   * This method will block if too many DWPT are currently flushing and no
+   * checked out DWPT are available
+   */
+  void waitIfStalled() {
+      stallControl.waitIfStalled();
+  }
+
+  /**
+   * Returns <code>true</code> iff stalled
+   */
+  boolean anyStalledThreads() {
+    return stallControl.anyStalledThreads();
+  }
+ 
 }
\ No newline at end of file
Index: lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java	(revision 1102572)
+++ lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java	(working copy)
@@ -165,6 +165,13 @@
   public int getMaxThreadStates() {
     return perThreads.length;
   }
+  
+  /**
+   * Returns the active number of {@link ThreadState} instances.
+   */
+  public int getActiveThreadState() {
+    return numThreadStatesActive;
+  }
 
   /**
    * Returns a new {@link ThreadState} iff any new state is available otherwise
Index: lucene/src/java/org/apache/lucene/index/DocumentsWriterStallControl.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/DocumentsWriterStallControl.java	(revision 1102572)
+++ lucene/src/java/org/apache/lucene/index/DocumentsWriterStallControl.java	(working copy)
@@ -36,8 +36,7 @@
  * continue indexing.
  */
 //TODO: rename this to DocumentsWriterStallControl (or something like that)?
-final class Healthiness {
-
+final class DocumentsWriterStallControl {
   @SuppressWarnings("serial")
   private static final class Sync extends AbstractQueuedSynchronizer {
     volatile boolean hasBlockedThreads = false; // only with assert
@@ -96,13 +95,14 @@
    * <code>true</code> iff the number of flushing
    * {@link DocumentsWriterPerThread} is greater than the number of active
    * {@link DocumentsWriterPerThread}. Otherwise it will reset the
-   * {@link Healthiness} to healthy and release all threads waiting on
+   * {@link DocumentsWriterStallControl} to healthy and release all threads waiting on
    * {@link #waitIfStalled()}
    */
   void updateStalled(DocumentsWriterFlushControl flushControl) {
     do {
-      // if we have more flushing DWPT than numActiveDWPT we stall!
-      while (flushControl.numActiveDWPT() < flushControl.numFlushingDWPT()) {
+      // if we have more flushing / blocked DWPT than numActiveDWPT we stall!
+      // don't stall if we have queued flushes - threads should be hijacked instead
+      while (flushControl.numQueuedFlushes() == 0 && flushControl.numActiveDWPT()+1 < (flushControl.numFlushingDWPT() + flushControl.numBlockedFlushes())) {
         if (sync.trySetStalled()) {
           assert wasStalled = true;
           return;
@@ -114,8 +114,8 @@
   void waitIfStalled() {
     sync.acquireShared(0);
   }
-
-  boolean hasBlocked() {
+  
+  boolean hasBlocked() { // for tests
     return sync.hasBlockedThreads;
   }
 }
\ No newline at end of file
Index: lucene/src/java/org/apache/lucene/index/Healthiness.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/Healthiness.java	(revision 1102572)
+++ lucene/src/java/org/apache/lucene/index/Healthiness.java	(working copy)
@@ -1,121 +0,0 @@
-package org.apache.lucene.index;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-import java.util.concurrent.locks.AbstractQueuedSynchronizer;
-
-import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
-
-/**
- * Controls the health status of a {@link DocumentsWriter} sessions. This class
- * used to block incoming indexing threads if flushing significantly slower than
- * indexing to ensure the {@link DocumentsWriter}s healthiness. If flushing is
- * significantly slower than indexing the net memory used within an
- * {@link IndexWriter} session can increase very quickly and easily exceed the
- * JVM's available memory.
- * <p>
- * To prevent OOM Errors and ensure IndexWriter's stability this class blocks
- * incoming threads from indexing once 2 x number of available
- * {@link ThreadState}s in {@link DocumentsWriterPerThreadPool} is exceeded.
- * Once flushing catches up and the number of flushing DWPT is equal or lower
- * than the number of active {@link ThreadState}s threads are released and can
- * continue indexing.
- */
-//TODO: rename this to DocumentsWriterStallControl (or something like that)?
-final class Healthiness {
-
-  @SuppressWarnings("serial")
-  private static final class Sync extends AbstractQueuedSynchronizer {
-    volatile boolean hasBlockedThreads = false; // only with assert
-
-    Sync() {
-      setState(0);
-    }
-
-    boolean isHealthy() {
-      return getState() == 0;
-    }
-
-    boolean trySetStalled() {
-      int state = getState();
-      return compareAndSetState(state, state + 1);
-    }
-
-    boolean tryReset() {
-      final int oldState = getState();
-      if (oldState == 0)
-        return true;
-      if (compareAndSetState(oldState, 0)) {
-        releaseShared(0);
-        return true;
-      }
-      return false;
-    }
-
-    @Override
-    public int tryAcquireShared(int acquires) {
-      assert maybeSetHasBlocked(getState());
-      return getState() == 0 ? 1 : -1;
-    }
-
-    // only used for testing
-    private boolean maybeSetHasBlocked(int state) {
-      hasBlockedThreads |= getState() != 0;
-      return true;
-    }
-
-    @Override
-    public boolean tryReleaseShared(int newState) {
-      return (getState() == 0);
-    }
-  }
-
-  private final Sync sync = new Sync();
-  volatile boolean wasStalled = false; // only with asserts
-
-  boolean anyStalledThreads() {
-    return !sync.isHealthy();
-  }
-
-  /**
-   * Update the stalled flag status. This method will set the stalled flag to
-   * <code>true</code> iff the number of flushing
-   * {@link DocumentsWriterPerThread} is greater than the number of active
-   * {@link DocumentsWriterPerThread}. Otherwise it will reset the
-   * {@link Healthiness} to healthy and release all threads waiting on
-   * {@link #waitIfStalled()}
-   */
-  void updateStalled(DocumentsWriterFlushControl flushControl) {
-    do {
-      // if we have more flushing DWPT than numActiveDWPT we stall!
-      while (flushControl.numActiveDWPT() < flushControl.numFlushingDWPT()) {
-        if (sync.trySetStalled()) {
-          assert wasStalled = true;
-          return;
-        }
-      }
-    } while (!sync.tryReset());
-  }
-
-  void waitIfStalled() {
-    sync.acquireShared(0);
-  }
-
-  boolean hasBlocked() {
-    return sync.hasBlockedThreads;
-  }
-}
\ No newline at end of file
Index: lucene/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java	(revision 1102572)
+++ lucene/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java	(working copy)
@@ -105,7 +105,7 @@
       assertTrue(maxRAMBytes < flushControl.peakActiveBytes);
     }
     if (ensureNotStalled) {
-      assertFalse(docsWriter.healthiness.wasStalled);
+      assertFalse(docsWriter.flushControl.stallControl.wasStalled);
     }
     writer.close();
     assertEquals(0, flushControl.activeBytes());
@@ -216,8 +216,8 @@
     assertEquals(numDocumentsToIndex, r.numDocs());
     assertEquals(numDocumentsToIndex, r.maxDoc());
     if (!flushPolicy.flushOnRAM()) {
-      assertFalse("never stall if we don't flush on RAM", docsWriter.healthiness.wasStalled);
-      assertFalse("never block if we don't flush on RAM", docsWriter.healthiness.hasBlocked());
+      assertFalse("never stall if we don't flush on RAM", docsWriter.flushControl.stallControl.wasStalled);
+      assertFalse("never block if we don't flush on RAM", docsWriter.flushControl.stallControl.hasBlocked());
     }
     r.close();
     writer.close();
@@ -266,10 +266,14 @@
       if (numThreads[i] == 1) {
         assertFalse(
             "single thread must not stall",
-            docsWriter.healthiness.wasStalled);
+            docsWriter.flushControl.stallControl.wasStalled);
         assertFalse(
             "single thread must not block numThreads: " + numThreads[i],
-            docsWriter.healthiness.hasBlocked());
+            docsWriter.flushControl.stallControl.hasBlocked());
+      } else {
+        assertTrue(
+            "multiple threads should stall",
+            docsWriter.flushControl.stallControl.wasStalled);
       }
       assertActiveBytesAfter(flushControl);
       writer.close(true);
