You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2013/07/02 09:12:03 UTC

svn commit: r1498804 [3/8] - in /lucene/dev/branches/lucene4258: lucene/ lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/ lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/ lucene/analysis/common/src/java/org/apache/l...

Modified: lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Tue Jul  2 07:12:00 2013
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -57,133 +58,132 @@ import org.apache.lucene.util.InfoStream
 import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
- * An <code>IndexWriter</code> creates and maintains an index.
- * 
- * <p>
- * The {@link OpenMode} option on
- * {@link IndexWriterConfig#setOpenMode(OpenMode)} determines whether a new
- * index is created, or whether an existing index is opened. Note that you can
- * open an index with {@link OpenMode#CREATE} even while readers are using the
- * index. The old readers will continue to search the "point in time" snapshot
- * they had opened, and won't see the newly created index until they re-open. If
- * {@link OpenMode#CREATE_OR_APPEND} is used IndexWriter will create a new index
- * if there is not already an index at the provided path and otherwise open the
- * existing index.
- * </p>
- * 
- * <p>
- * In either case, documents are added with {@link #addDocument(IndexDocument)
- * addDocument} and removed with {@link #deleteDocuments(Term)} or
- * {@link #deleteDocuments(Query)}. A document can be updated with
- * {@link #updateDocument(Term, IndexDocument) updateDocument} (which just
- * deletes and then adds the entire document). When finished adding, deleting
- * and updating documents, {@link #close() close} should be called.
- * </p>
- * 
- * <a name="flush"></a>
- * <p>
- * These changes are buffered in memory and periodically flushed to the
- * {@link Directory} (during the above method calls). A flush is triggered when
- * there are enough added documents since the last flush. Flushing is triggered
- * either by RAM usage of the documents (see
- * {@link IndexWriterConfig#setRAMBufferSizeMB}) or the number of added
- * documents (see {@link IndexWriterConfig#setMaxBufferedDocs(int)}). The
- * default is to flush when RAM usage hits
- * {@link IndexWriterConfig#DEFAULT_RAM_BUFFER_SIZE_MB} MB. For best indexing
- * speed you should flush by RAM usage with a large RAM buffer. Additionally, if
- * IndexWriter reaches the configured number of buffered deletes (see
- * {@link IndexWriterConfig#setMaxBufferedDeleteTerms}) the deleted terms and
- * queries are flushed and applied to existing segments. In contrast to the
- * other flush options {@link IndexWriterConfig#setRAMBufferSizeMB} and
- * {@link IndexWriterConfig#setMaxBufferedDocs(int)}, deleted terms won't
- * trigger a segment flush. Note that flushing just moves the internal buffered
- * state in IndexWriter into the index, but these changes are not visible to
- * IndexReader until either {@link #commit()} or {@link #close} is called. A
- * flush may also trigger one or more segment merges which by default run with a
- * background thread so as not to block the addDocument calls (see <a
- * href="#mergePolicy">below</a> for changing the {@link MergeScheduler}).
- * </p>
- * 
- * <p>
- * Opening an <code>IndexWriter</code> creates a lock file for the directory in
- * use. Trying to open another <code>IndexWriter</code> on the same directory
- * will lead to a {@link LockObtainFailedException}. The
- * {@link LockObtainFailedException} is also thrown if an IndexReader on the
- * same directory is used to delete documents from the index.
- * </p>
- * 
- * <a name="deletionPolicy"></a>
- * <p>
- * Expert: <code>IndexWriter</code> allows an optional
- * {@link IndexDeletionPolicy} implementation to be specified. You can use this
- * to control when prior commits are deleted from the index. The default policy
- * is {@link KeepOnlyLastCommitDeletionPolicy} which removes all prior commits
- * as soon as a new commit is done (this matches behavior before 2.2). Creating
- * your own policy can allow you to explicitly keep previous "point in time"
- * commits alive in the index for some time, to allow readers to refresh to the
- * new commit without having the old commit deleted out from under them. This is
- * necessary on filesystems like NFS that do not support "delete on last
- * close" semantics, which Lucene's "point in time" search normally relies on.
- * </p>
- * 
- * <a name="mergePolicy"></a>
- * <p>
- * Expert: <code>IndexWriter</code> allows you to separately change the
- * {@link MergePolicy} and the {@link MergeScheduler}. The {@link MergePolicy}
- * is invoked whenever there are changes to the segments in the index. Its role
- * is to select which merges to do, if any, and return a
- * {@link MergePolicy.MergeSpecification} describing the merges. The default is
- * {@link LogByteSizeMergePolicy}. Then, the {@link MergeScheduler} is invoked
- * with the requested merges and it decides when and how to run the merges. The
- * default is {@link ConcurrentMergeScheduler}.
- * </p>
- * 
- * <a name="OOME"></a>
- * <p>
- * <b>NOTE</b>: if you hit an OutOfMemoryError then IndexWriter will quietly
- * record this fact and block all future segment commits. This is a defensive
- * measure in case any internal state (buffered documents and deletions) were
- * corrupted. Any subsequent calls to {@link #commit()} will throw an
- * IllegalStateException. The only course of action is to call {@link #close()},
- * which internally will call {@link #rollback()}, to undo any changes to the
- * index since the last commit. You can also just call {@link #rollback()}
- * directly.
- * </p>
- * 
- * <a name="thread-safety"></a>
- * <p>
- * <b>NOTE</b>: {@link IndexWriter} instances are completely thread safe,
- * meaning multiple threads can call any of its methods, concurrently. If your
- * application requires external synchronization, you should <b>not</b>
- * synchronize on the <code>IndexWriter</code> instance as this may cause
- * deadlock; use your own (non-Lucene) objects instead.
- * </p>
- * 
- * <p>
- * <b>NOTE</b>: If you call <code>Thread.interrupt()</code> on a thread that's
- * within IndexWriter, IndexWriter will try to catch this (eg, if it's in a
- * wait() or Thread.sleep()), and will then throw the unchecked exception
- * {@link ThreadInterruptedException} and <b>clear</b> the interrupt status on
- * the thread.
- * </p>
- */
+  An <code>IndexWriter</code> creates and maintains an index.
+
+  <p>The {@link OpenMode} option on 
+  {@link IndexWriterConfig#setOpenMode(OpenMode)} determines 
+  whether a new index is created, or whether an existing index is
+  opened. Note that you can open an index with {@link OpenMode#CREATE}
+  even while readers are using the index. The old readers will 
+  continue to search the "point in time" snapshot they had opened, 
+  and won't see the newly created index until they re-open. If 
+  {@link OpenMode#CREATE_OR_APPEND} is used IndexWriter will create a 
+  new index if there is not already an index at the provided path
+  and otherwise open the existing index.</p>
+
+  <p>In either case, documents are added with {@link #addDocument(IndexDocument)
+  addDocument} and removed with {@link #deleteDocuments(Term)} or {@link
+  #deleteDocuments(Query)}. A document can be updated with {@link
+  #updateDocument(Term, IndexDocument) updateDocument} (which just deletes
+  and then adds the entire document). When finished adding, deleting 
+  and updating documents, {@link #close() close} should be called.</p>
+
+  <a name="flush"></a>
+  <p>These changes are buffered in memory and periodically
+  flushed to the {@link Directory} (during the above method
+  calls). A flush is triggered when there are enough added documents
+  since the last flush. Flushing is triggered either by RAM usage of the
+  documents (see {@link IndexWriterConfig#setRAMBufferSizeMB}) or the
+  number of added documents (see {@link IndexWriterConfig#setMaxBufferedDocs(int)}).
+  The default is to flush when RAM usage hits
+  {@link IndexWriterConfig#DEFAULT_RAM_BUFFER_SIZE_MB} MB. For
+  best indexing speed you should flush by RAM usage with a
+  large RAM buffer. Additionally, if IndexWriter reaches the configured number of
+  buffered deletes (see {@link IndexWriterConfig#setMaxBufferedDeleteTerms})
+  the deleted terms and queries are flushed and applied to existing segments.
+  In contrast to the other flush options {@link IndexWriterConfig#setRAMBufferSizeMB} and 
+  {@link IndexWriterConfig#setMaxBufferedDocs(int)}, deleted terms
+  won't trigger a segment flush. Note that flushing just moves the
+  internal buffered state in IndexWriter into the index, but
+  these changes are not visible to IndexReader until either
+  {@link #commit()} or {@link #close} is called.  A flush may
+  also trigger one or more segment merges which by default
+  run with a background thread so as not to block the
+  addDocument calls (see <a href="#mergePolicy">below</a>
+  for changing the {@link MergeScheduler}).</p>
+
+  <p>Opening an <code>IndexWriter</code> creates a lock file for the directory in use. Trying to open
+  another <code>IndexWriter</code> on the same directory will lead to a
+  {@link LockObtainFailedException}. The {@link LockObtainFailedException}
+  is also thrown if an IndexReader on the same directory is used to delete documents
+  from the index.</p>
+  
+  <a name="deletionPolicy"></a>
+  <p>Expert: <code>IndexWriter</code> allows an optional
+  {@link IndexDeletionPolicy} implementation to be
+  specified.  You can use this to control when prior commits
+  are deleted from the index.  The default policy is {@link
+  KeepOnlyLastCommitDeletionPolicy} which removes all prior
+  commits as soon as a new commit is done (this matches
+  behavior before 2.2).  Creating your own policy can allow
+  you to explicitly keep previous "point in time" commits
+  alive in the index for some time, to allow readers to
+  refresh to the new commit without having the old commit
+  deleted out from under them.  This is necessary on
+  filesystems like NFS that do not support "delete on last
+  close" semantics, which Lucene's "point in time" search
+  normally relies on. </p>
+
+  <a name="mergePolicy"></a> <p>Expert:
+  <code>IndexWriter</code> allows you to separately change
+  the {@link MergePolicy} and the {@link MergeScheduler}.
+  The {@link MergePolicy} is invoked whenever there are
+  changes to the segments in the index.  Its role is to
+  select which merges to do, if any, and return a {@link
+  MergePolicy.MergeSpecification} describing the merges.
+  The default is {@link LogByteSizeMergePolicy}.  Then, the {@link
+  MergeScheduler} is invoked with the requested merges and
+  it decides when and how to run the merges.  The default is
+  {@link ConcurrentMergeScheduler}. </p>
+
+  <a name="OOME"></a><p><b>NOTE</b>: if you hit an
+  OutOfMemoryError then IndexWriter will quietly record this
+  fact and block all future segment commits.  This is a
+  defensive measure in case any internal state (buffered
+  documents and deletions) were corrupted.  Any subsequent
+  calls to {@link #commit()} will throw an
+  IllegalStateException.  The only course of action is to
+  call {@link #close()}, which internally will call {@link
+  #rollback()}, to undo any changes to the index since the
+  last commit.  You can also just call {@link #rollback()}
+  directly.</p>
+
+  <a name="thread-safety"></a><p><b>NOTE</b>: {@link
+  IndexWriter} instances are completely thread
+  safe, meaning multiple threads can call any of its
+  methods, concurrently.  If your application requires
+  external synchronization, you should <b>not</b>
+  synchronize on the <code>IndexWriter</code> instance as
+  this may cause deadlock; use your own (non-Lucene) objects
+  instead. </p>
+  
+  <p><b>NOTE</b>: If you call
+  <code>Thread.interrupt()</code> on a thread that's within
+  IndexWriter, IndexWriter will try to catch this (eg, if
+  it's in a wait() or Thread.sleep()), and will then throw
+  the unchecked exception {@link ThreadInterruptedException}
+  and <b>clear</b> the interrupt status on the thread.</p>
+*/
 
 /*
- * Clarification: Check Points (and commits) IndexWriter writes new index files
- * to the directory without writing a new segments_N file which references these
- * new files. It also means that the state of the in memory SegmentInfos object
- * is different than the most recent segments_N file written to the directory.
- * 
- * Each time the SegmentInfos is changed, and matches the (possibly modified)
- * directory files, we have a new "check point". If the modified/new
- * SegmentInfos is written to disk - as a new (generation of) segments_N file -
- * this check point is also an IndexCommit.
- * 
- * A new checkpoint always replaces the previous checkpoint and becomes the new
- * "front" of the index. This allows the IndexFileDeleter to delete files that
- * are referenced only by stale checkpoints. (files that were created since the
- * last commit, but are no longer referenced by the "front" of the index). For
- * this, IndexFileDeleter keeps track of the last non commit checkpoint.
+ * Clarification: Check Points (and commits)
+ * IndexWriter writes new index files to the directory without writing a new segments_N
+ * file which references these new files. It also means that the state of
+ * the in memory SegmentInfos object is different than the most recent
+ * segments_N file written to the directory.
+ *
+ * Each time the SegmentInfos is changed, and matches the (possibly
+ * modified) directory files, we have a new "check point".
+ * If the modified/new SegmentInfos is written to disk - as a new
+ * (generation of) segments_N file - this check point is also an
+ * IndexCommit.
+ *
+ * A new checkpoint always replaces the previous checkpoint and
+ * becomes the new "front" of the index. This allows the IndexFileDeleter
+ * to delete files that are referenced only by stale checkpoints.
+ * (files that were created since the last commit, but are no longer
+ * referenced by the "front" of the index). For this, IndexFileDeleter
+ * keeps track of the last non commit checkpoint.
  */
 public class IndexWriter implements Closeable, TwoPhaseCommit {
   
@@ -193,7 +193,7 @@ public class IndexWriter implements Clos
    * Name of the write lock in the index.
    */
   public static final String WRITE_LOCK_NAME = "write.lock";
-  
+
   /** Key for the source of a segment in the {@link SegmentInfo#getDiagnostics() diagnostics}. */
   public static final String SOURCE = "source";
   /** Source of a segment which results from a merge of other segments. */
@@ -204,50 +204,50 @@ public class IndexWriter implements Clos
   public static final String SOURCE_ADDINDEXES_READERS = "addIndexes(IndexReader...)";
 
   /**
-   * Absolute hard maximum length for a term, in bytes once encoded as UTF8. If
-   * a term arrives from the analyzer longer than this length, it is skipped and
-   * a message is printed to infoStream, if set (see
-   * {@link IndexWriterConfig#setInfoStream(InfoStream)}).
+   * Absolute hard maximum length for a term, in bytes once
+   * encoded as UTF8.  If a term arrives from the analyzer
+   * longer than this length, it is skipped and a message is
+   * printed to infoStream, if set (see {@link
+   * IndexWriterConfig#setInfoStream(InfoStream)}).
    */
   public final static int MAX_TERM_LENGTH = DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8;
   volatile private boolean hitOOM;
-  
-  private final Directory directory; // where this index resides
-  private final Analyzer analyzer; // how to analyze text
-  
-  private volatile long changeCount; // increments every time a change is
-                                     // completed
+
+  private final Directory directory;  // where this index resides
+  private final Analyzer analyzer;    // how to analyze text
+
+  private volatile long changeCount; // increments every time a change is completed
   private long lastCommitChangeCount; // last changeCount that was committed
-  
-  private List<SegmentInfoPerCommit> rollbackSegments; // list of segmentInfo we
-                                                       // will fallback to if
-                                                       // the commit fails
-  
-  volatile SegmentInfos pendingCommit; // set when a commit is pending (after
-                                       // prepareCommit() & before commit())
+
+  private List<SegmentInfoPerCommit> rollbackSegments;      // list of segmentInfo we will fallback to if the commit fails
+
+  volatile SegmentInfos pendingCommit;            // set when a commit is pending (after prepareCommit() & before commit())
   volatile long pendingCommitChangeCount;
+
+  volatile AtomicBoolean deletesPending; // set when there are pending deletes
+                                         // to be flushed before adding updates
   
   private Collection<String> filesToCommit;
-  
-  final SegmentInfos segmentInfos; // the segments
+
+  final SegmentInfos segmentInfos;       // the segments
   final FieldNumbers globalFieldNumberMap;
-  
+
   private DocumentsWriter docWriter;
   final IndexFileDeleter deleter;
-  
+
   // used by forceMerge to note those needing merging
   private Map<SegmentInfoPerCommit,Boolean> segmentsToMerge = new HashMap<SegmentInfoPerCommit,Boolean>();
   private int mergeMaxNumSegments;
-  
+
   private Lock writeLock;
-  
+
   private volatile boolean closed;
   private volatile boolean closing;
-  
+
   // Holds all SegmentInfo instances currently involved in
   // merges
   private HashSet<SegmentInfoPerCommit> mergingSegments = new HashSet<SegmentInfoPerCommit>();
-  
+
   private MergePolicy mergePolicy;
   private final MergeScheduler mergeScheduler;
   private LinkedList<MergePolicy.OneMerge> pendingMerges = new LinkedList<MergePolicy.OneMerge>();
@@ -255,106 +255,95 @@ public class IndexWriter implements Clos
   private List<MergePolicy.OneMerge> mergeExceptions = new ArrayList<MergePolicy.OneMerge>();
   private long mergeGen;
   private boolean stopMerges;
-  
+
   final AtomicInteger flushCount = new AtomicInteger();
   final AtomicInteger flushDeletesCount = new AtomicInteger();
-  
+
   final ReaderPool readerPool = new ReaderPool();
   final BufferedDeletesStream bufferedDeletesStream;
-  
-  private boolean updatesPending;
-  
+
   // This is a "write once" variable (like the organic dye
   // on a DVD-R that may or may not be heated by a laser and
   // then cooled to permanently record the event): it's
   // false, until getReader() is called for the first time,
   // at which point it's switched to true and never changes
-  // back to false. Once this is true, we hold open and
+  // back to false.  Once this is true, we hold open and
   // reuse SegmentReader instances internally for applying
   // deletes, doing merges, and reopening near real-time
   // readers.
   private volatile boolean poolReaders;
-  
+
   // The instance that was passed to the constructor. It is saved only in order
   // to allow users to query an IndexWriter settings.
   private final LiveIndexWriterConfig config;
-  
+
   DirectoryReader getReader() throws IOException {
     return getReader(true);
   }
-  
+
   /**
-   * Expert: returns a readonly reader, covering all committed as well as
-   * un-committed changes to the index. This provides "near real-time"
-   * searching, in that changes made during an IndexWriter session can be
-   * quickly made available for searching without closing the writer nor calling
-   * {@link #commit}.
-   * 
-   * <p>
-   * Note that this is functionally equivalent to calling {#flush} and then
-   * opening a new reader. But the turnaround time of this method should be
-   * faster since it avoids the potentially costly {@link #commit}.
-   * </p>
-   * 
-   * <p>
-   * You must close the {@link IndexReader} returned by this method once you are
-   * done using it.
-   * </p>
-   * 
-   * <p>
-   * It's <i>near</i> real-time because there is no hard guarantee on how
-   * quickly you can get a new reader after making changes with IndexWriter.
-   * You'll have to experiment in your situation to determine if it's fast
-   * enough. As this is a new and experimental feature, please report back on
-   * your findings so we can learn, improve and iterate.
-   * </p>
-   * 
-   * <p>
-   * The resulting reader supports {@link DirectoryReader#openIfChanged}, but
-   * that call will simply forward back to this method (though this may change
-   * in the future).
-   * </p>
-   * 
-   * <p>
-   * The very first time this method is called, this writer instance will make
-   * every effort to pool the readers that it opens for doing merges, applying
-   * deletes, etc. This means additional resources (RAM, file descriptors, CPU
-   * time) will be consumed.
-   * </p>
-   * 
-   * <p>
-   * For lower latency on reopening a reader, you should call
-   * {@link IndexWriterConfig#setMergedSegmentWarmer} to pre-warm a newly merged
-   * segment before it's committed to the index. This is important for
-   * minimizing index-to-search delay after a large merge.
-   * </p>
-   * 
-   * <p>
-   * If an addIndexes* call is running in another thread, then this reader will
-   * only search those segments from the foreign index that have been
-   * successfully copied over, so far
-   * </p>
-   * .
-   * 
-   * <p>
-   * <b>NOTE</b>: Once the writer is closed, any outstanding readers may
-   * continue to be used. However, if you attempt to reopen any of those
-   * readers, you'll hit an {@link AlreadyClosedException}.
-   * </p>
-   * 
+   * Expert: returns a readonly reader, covering all
+   * committed as well as un-committed changes to the index.
+   * This provides "near real-time" searching, in that
+   * changes made during an IndexWriter session can be
+   * quickly made available for searching without closing
+   * the writer nor calling {@link #commit}.
+   *
+   * <p>Note that this is functionally equivalent to calling
+   * {#flush} and then opening a new reader.  But the turnaround time of this
+   * method should be faster since it avoids the potentially
+   * costly {@link #commit}.</p>
+   *
+   * <p>You must close the {@link IndexReader} returned by
+   * this method once you are done using it.</p>
+   *
+   * <p>It's <i>near</i> real-time because there is no hard
+   * guarantee on how quickly you can get a new reader after
+   * making changes with IndexWriter.  You'll have to
+   * experiment in your situation to determine if it's
+   * fast enough.  As this is a new and experimental
+   * feature, please report back on your findings so we can
+   * learn, improve and iterate.</p>
+   *
+   * <p>The resulting reader supports {@link
+   * DirectoryReader#openIfChanged}, but that call will simply forward
+   * back to this method (though this may change in the
+   * future).</p>
+   *
+   * <p>The very first time this method is called, this
+   * writer instance will make every effort to pool the
+   * readers that it opens for doing merges, applying
+   * deletes, etc.  This means additional resources (RAM,
+   * file descriptors, CPU time) will be consumed.</p>
+   *
+   * <p>For lower latency on reopening a reader, you should
+   * call {@link IndexWriterConfig#setMergedSegmentWarmer} to
+   * pre-warm a newly merged segment before it's committed
+   * to the index.  This is important for minimizing
+   * index-to-search delay after a large merge.  </p>
+   *
+   * <p>If an addIndexes* call is running in another thread,
+   * then this reader will only search those segments from
+   * the foreign index that have been successfully copied
+   * over, so far</p>.
+   *
+   * <p><b>NOTE</b>: Once the writer is closed, any
+   * outstanding readers may continue to be used.  However,
+   * if you attempt to reopen any of those readers, you'll
+   * hit an {@link AlreadyClosedException}.</p>
+   *
    * @lucene.experimental
-   * 
-   * @return IndexReader that covers entire index plus all changes made so far
-   *         by this IndexWriter instance
-   * 
-   * @throws IOException
-   *           If there is a low-level I/O error
+   *
+   * @return IndexReader that covers entire index plus all
+   * changes made so far by this IndexWriter instance
+   *
+   * @throws IOException If there is a low-level I/O error
    */
   DirectoryReader getReader(boolean applyAllDeletes) throws IOException {
     ensureOpen();
-    
+
     final long tStart = System.currentTimeMillis();
-    
+
     if (infoStream.isEnabled("IW")) {
       infoStream.message("IW", "flush at getReader");
     }
@@ -366,10 +355,11 @@ public class IndexWriter implements Clos
     doBeforeFlush();
     boolean anySegmentFlushed = false;
     /*
-     * for releasing a NRT reader we must ensure that DW doesn't add any
-     * segments or deletes until we are done with creating the NRT
-     * DirectoryReader. We release the two stage full flush after we are done
-     * opening the directory reader!
+     * for releasing a NRT reader we must ensure that 
+     * DW doesn't add any segments or deletes until we are
+     * done with creating the NRT DirectoryReader. 
+     * We release the two stage full flush after we are done opening the
+     * directory reader!
      */
     boolean success2 = false;
     try {
@@ -422,27 +412,26 @@ public class IndexWriter implements Clos
     }
     return r;
   }
-  
-  /**
-   * Holds shared SegmentReader instances. IndexWriter uses SegmentReaders for
-   * 1) applying deletes, 2) doing merges, 3) handing out a real-time reader.
-   * This pool reuses instances of the SegmentReaders in all these places if it
-   * is in "near real-time mode" (getReader() has been called on this instance).
-   */
-  
+
+  /** Holds shared SegmentReader instances. IndexWriter uses
+   *  SegmentReaders for 1) applying deletes, 2) doing
+   *  merges, 3) handing out a real-time reader.  This pool
+   *  reuses instances of the SegmentReaders in all these
+   *  places if it is in "near real-time mode" (getReader()
+   *  has been called on this instance). */
+
   class ReaderPool {
     
     private final Map<SegmentInfoPerCommit,ReadersAndLiveDocs> readerMap = new HashMap<SegmentInfoPerCommit,ReadersAndLiveDocs>();
-    
+
     // used only by asserts
     public synchronized boolean infoIsLive(SegmentInfoPerCommit info) {
       int idx = segmentInfos.indexOf(info);
-      assert idx != -1 : "info=" + info + " isn't live";
-      assert segmentInfos.info(idx) == info : "info=" + info
-          + " doesn't match live info in segmentInfos";
+      assert idx != -1: "info=" + info + " isn't live";
+      assert segmentInfos.info(idx) == info: "info=" + info + " doesn't match live info in segmentInfos";
       return true;
     }
-    
+
     public synchronized void drop(SegmentInfoPerCommit info) throws IOException {
       final ReadersAndLiveDocs rld = readerMap.get(info);
       if (rld != null) {
@@ -451,7 +440,7 @@ public class IndexWriter implements Clos
         rld.dropReaders();
       }
     }
-    
+
     public synchronized boolean anyPendingDeletes() {
       for(ReadersAndLiveDocs rld : readerMap.values()) {
         if (rld.getPendingDeleteCount() != 0) {
@@ -463,10 +452,10 @@ public class IndexWriter implements Clos
     }
 
     public synchronized void release(ReadersAndLiveDocs rld) throws IOException {
-      
+
       // Matches incRef in get:
       rld.decRef();
-      
+
       // Pool still holds a ref:
       assert rld.refCount() >= 1;
       
@@ -482,15 +471,14 @@ public class IndexWriter implements Clos
           // created created new _X_N.del file.
           deleter.checkpoint(segmentInfos, false);
         }
-        
+
         rld.dropReaders();
         readerMap.remove(rld.info);
       }
     }
-    
-    /**
-     * Remove all our references to readers, and commits any pending changes.
-     */
+
+    /** Remove all our references to readers, and commits
+     *  any pending changes. */
     synchronized void dropAll(boolean doSave) throws IOException {
       Throwable priorE = null;
       final Iterator<Map.Entry<SegmentInfoPerCommit,ReadersAndLiveDocs>> it = readerMap.entrySet().iterator();
@@ -510,13 +498,13 @@ public class IndexWriter implements Clos
             priorE = t;
           }
         }
-        
+
         // Important to remove as-we-go, not with .clear()
         // in the end, in case we hit an exception;
         // otherwise we could over-decref if close() is
         // called again:
         it.remove();
-        
+
         // NOTE: it is allowed that these decRefs do not
         // actually close the SRs; this happens when a
         // near real-time reader is kept open after the
@@ -534,12 +522,12 @@ public class IndexWriter implements Clos
         throw new RuntimeException(priorE);
       }
     }
-    
+
     /**
-     * Commit live docs changes for the segment readers for the provided infos.
-     * 
-     * @throws IOException
-     *           If there is a low-level I/O error
+     * Commit live docs changes for the segment readers for
+     * the provided infos.
+     *
+     * @throws IOException If there is a low-level I/O error
      */
     public synchronized void commit(SegmentInfos infos) throws IOException {
       for (SegmentInfoPerCommit info : infos) {
@@ -556,17 +544,16 @@ public class IndexWriter implements Clos
         }
       }
     }
-    
+
     /**
-     * Obtain a ReadersAndLiveDocs instance from the readerPool. If create is
-     * true, you must later call {@link #release(ReadersAndLiveDocs)}.
+     * Obtain a ReadersAndLiveDocs instance from the
+     * readerPool.  If create is true, you must later call
+     * {@link #release(ReadersAndLiveDocs)}.
      */
-    public synchronized ReadersAndLiveDocs get(SegmentInfoPerCommit info,
-        boolean create) {
-      
-      assert info.info.dir == directory : "info.dir=" + info.info.dir + " vs "
-          + directory;
-      
+    public synchronized ReadersAndLiveDocs get(SegmentInfoPerCommit info, boolean create) {
+
+      assert info.info.dir == directory: "info.dir=" + info.info.dir + " vs " + directory;
+
       ReadersAndLiveDocs rld = readerMap.get(info);
       if (rld == null) {
         if (!create) {
@@ -576,15 +563,14 @@ public class IndexWriter implements Clos
         // Steal initial reference:
         readerMap.put(info, rld);
       } else {
-        assert rld.info == info : "rld.info=" + rld.info + " info=" + info
-            + " isLive?=" + infoIsLive(rld.info) + " vs " + infoIsLive(info);
+        assert rld.info == info: "rld.info=" + rld.info + " info=" + info + " isLive?=" + infoIsLive(rld.info) + " vs " + infoIsLive(info);
       }
-      
+
       if (create) {
         // Return ref to caller:
         rld.incRef();
       }
-      
+
       assert noDups();
 
       return rld;
@@ -601,22 +587,23 @@ public class IndexWriter implements Clos
       return true;
     }
   }
-  
+
   /**
-   * Obtain the number of deleted docs for a pooled reader. If the reader isn't
-   * being pooled, the segmentInfo's delCount is returned.
+   * Obtain the number of deleted docs for a pooled reader.
+   * If the reader isn't being pooled, the segmentInfo's 
+   * delCount is returned.
    */
   public int numDeletedDocs(SegmentInfoPerCommit info) {
     ensureOpen(false);
     int delCount = info.getDelCount();
-    
+
     final ReadersAndLiveDocs rld = readerPool.get(info, false);
     if (rld != null) {
       delCount += rld.getPendingDeleteCount();
     }
     return delCount;
   }
-  
+
   /**
    * Used internally to throw an {@link AlreadyClosedException} if this
    * IndexWriter has been closed or is in the process of closing.
@@ -628,34 +615,32 @@ public class IndexWriter implements Clos
    * @throws AlreadyClosedException
    *           if this IndexWriter is closed or in the process of closing
    */
-  protected final void ensureOpen(boolean failIfClosing)
-      throws AlreadyClosedException {
+  protected final void ensureOpen(boolean failIfClosing) throws AlreadyClosedException {
     if (closed || (failIfClosing && closing)) {
       throw new AlreadyClosedException("this IndexWriter is closed");
     }
   }
-  
+
   /**
-   * Used internally to throw an {@link AlreadyClosedException} if this
-   * IndexWriter has been closed ({@code closed=true}) or is in the process of
+   * Used internally to throw an {@link
+   * AlreadyClosedException} if this IndexWriter has been
+   * closed ({@code closed=true}) or is in the process of
    * closing ({@code closing=true}).
    * <p>
    * Calls {@link #ensureOpen(boolean) ensureOpen(true)}.
-   * 
-   * @throws AlreadyClosedException
-   *           if this IndexWriter is closed
+   * @throws AlreadyClosedException if this IndexWriter is closed
    */
   protected final void ensureOpen() throws AlreadyClosedException {
     ensureOpen(true);
   }
-  
+
   final Codec codec; // for writing new segments
-  
+
   /**
    * Constructs a new IndexWriter per the settings given in <code>conf</code>.
-   * Note that the passed in {@link IndexWriterConfig} is privately cloned; if
-   * you need to make subsequent "live" changes to the configuration use
-   * {@link #getConfig}.
+   * Note that the passed in {@link IndexWriterConfig} is
+   * privately cloned; if you need to make subsequent "live"
+   * changes to the configuration use {@link #getConfig}.
    * <p>
    * 
    * @param d
@@ -679,15 +664,16 @@ public class IndexWriter implements Clos
     mergePolicy.setIndexWriter(this);
     mergeScheduler = config.getMergeScheduler();
     codec = config.getCodec();
-    
+
     bufferedDeletesStream = new BufferedDeletesStream(infoStream);
     poolReaders = config.getReaderPooling();
+    deletesPending = new AtomicBoolean(false);
     
     writeLock = directory.makeLock(WRITE_LOCK_NAME);
-    
+
     if (!writeLock.obtain(config.getWriteLockTimeout())) // obtain write lock
-    throw new LockObtainFailedException("Index locked for write: " + writeLock);
-    
+      throw new LockObtainFailedException("Index locked for write: " + writeLock);
+
     boolean success = false;
     try {
       OpenMode mode = config.getOpenMode();
@@ -700,17 +686,17 @@ public class IndexWriter implements Clos
         // CREATE_OR_APPEND - create only if an index does not exist
         create = !DirectoryReader.indexExists(directory);
       }
-      
+
       // If index is too old, reading the segments will throw
       // IndexFormatTooOldException.
       segmentInfos = new SegmentInfos();
-      
+
       boolean initialIndexExists = true;
 
       if (create) {
-        // Try to read first. This is to allow create
+        // Try to read first.  This is to allow create
         // against an index that's currently open for
-        // searching. In this case we write the next
+        // searching.  In this case we write the next
         // segments_N file with no segments:
         try {
           segmentInfos.read(directory);
@@ -719,49 +705,47 @@ public class IndexWriter implements Clos
           // Likely this means it's a fresh directory
           initialIndexExists = false;
         }
-        
+
         // Record that we have a change (zero out all
         // segments) pending:
         changed();
       } else {
         segmentInfos.read(directory);
-        
+
         IndexCommit commit = config.getIndexCommit();
         if (commit != null) {
           // Swap out all segments, but, keep metadata in
           // SegmentInfos, like version & generation, to
-          // preserve write-once. This is important if
+          // preserve write-once.  This is important if
           // readers are open against the future commit
           // points.
-          if (commit.getDirectory() != directory) throw new IllegalArgumentException(
-              "IndexCommit's directory doesn't match my directory");
+          if (commit.getDirectory() != directory)
+            throw new IllegalArgumentException("IndexCommit's directory doesn't match my directory");
           SegmentInfos oldInfos = new SegmentInfos();
           oldInfos.read(directory, commit.getSegmentsFileName());
           segmentInfos.replace(oldInfos);
           changed();
           if (infoStream.isEnabled("IW")) {
-            infoStream.message("IW",
-                "init: loaded commit \"" + commit.getSegmentsFileName() + "\"");
+            infoStream.message("IW", "init: loaded commit \"" + commit.getSegmentsFileName() + "\"");
           }
         }
       }
-      
+
       rollbackSegments = segmentInfos.createBackupSegmentInfos();
-      
+
       // start with previous field numbers, but new FieldInfos
       globalFieldNumberMap = getFieldNumberMap();
-      docWriter = new DocumentsWriter(codec, config, directory, this,
-          globalFieldNumberMap, bufferedDeletesStream);
-      
+      docWriter = new DocumentsWriter(codec, config, directory, this, globalFieldNumberMap, bufferedDeletesStream);
+
       // Default deleter (for backwards compatibility) is
       // KeepOnlyLastCommitDeleter:
-      synchronized (this) {
+      synchronized(this) {
         deleter = new IndexFileDeleter(directory,
                                        config.getIndexDeletionPolicy(),
                                        segmentInfos, infoStream, this,
                                        initialIndexExists);
       }
-      
+
       if (deleter.startingCommitDeleted) {
         // Deletion policy deleted the "head" commit point.
         // We have to mark ourself as changed so that if we
@@ -769,19 +753,18 @@ public class IndexWriter implements Clos
         // segments_N file.
         changed();
       }
-      
+
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "init: create=" + create);
         messageState();
       }
-      
+
       success = true;
-      
+
     } finally {
       if (!success) {
         if (infoStream.isEnabled("IW")) {
-          infoStream.message("IW",
-              "init: hit exception on init; releasing write lock");
+          infoStream.message("IW", "init: hit exception on init; releasing write lock");
         }
         try {
           writeLock.release();
@@ -792,31 +775,31 @@ public class IndexWriter implements Clos
       }
     }
   }
-  
+
   private FieldInfos getFieldInfos(SegmentInfo info) throws IOException {
     Directory cfsDir = null;
     try {
       if (info.getUseCompoundFile()) {
         cfsDir = new CompoundFileDirectory(info.dir,
-            IndexFileNames.segmentFileName(info.name, "",
-                IndexFileNames.COMPOUND_FILE_EXTENSION), IOContext.READONCE,
-            false);
+                                           IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION),
+                                           IOContext.READONCE,
+                                           false);
       } else {
         cfsDir = info.dir;
       }
-      return info.getCodec().fieldInfosFormat().getFieldInfosReader()
-          .read(cfsDir, info.name, IOContext.READONCE);
+      return info.getCodec().fieldInfosFormat().getFieldInfosReader().read(cfsDir,
+                                                                                info.name,
+                                                                                IOContext.READONCE);
     } finally {
       if (info.getUseCompoundFile() && cfsDir != null) {
         cfsDir.close();
       }
     }
   }
-  
+
   /**
-   * Loads or returns the already loaded the global field number map for this
-   * {@link SegmentInfos}. If this {@link SegmentInfos} has no global field
-   * number map the returned instance is empty
+   * Loads or returns the already loaded the global field number map for this {@link SegmentInfos}.
+   * If this {@link SegmentInfos} has no global field number map the returned instance is empty
    */
   private FieldNumbers getFieldNumberMap() throws IOException {
     final FieldNumbers map = new FieldNumbers();
@@ -826,55 +809,57 @@ public class IndexWriter implements Clos
         map.addOrGet(fi.name, fi.number, fi.getDocValuesType());
       }
     }
-    
+
     return map;
   }
   
   /**
-   * Returns a {@link LiveIndexWriterConfig}, which can be used to query the
-   * IndexWriter current settings, as well as modify "live" ones.
+   * Returns a {@link LiveIndexWriterConfig}, which can be used to query the IndexWriter
+   * current settings, as well as modify "live" ones.
    */
   public LiveIndexWriterConfig getConfig() {
     ensureOpen(false);
     return config;
   }
-  
+
   private void messageState() {
     if (infoStream.isEnabled("IW")) {
-      infoStream.message("IW", "\ndir=" + directory + "\n" + "index="
-          + segString() + "\n" + "version=" + Constants.LUCENE_VERSION + "\n"
-          + config.toString());
+      infoStream.message("IW", "\ndir=" + directory + "\n" +
+            "index=" + segString() + "\n" +
+            "version=" + Constants.LUCENE_VERSION + "\n" +
+            config.toString());
     }
   }
-  
+
   /**
-   * Commits all changes to an index, waits for pending merges to complete, and
-   * closes all associated files.
-   * <p>
-   * This is a "slow graceful shutdown" which may take a long time especially if
-   * a big merge is pending: If you only want to close resources use
-   * {@link #rollback()}. If you only want to commit pending changes and close
-   * resources see {@link #close(boolean)}.
-   * <p>
-   * Note that this may be a costly operation, so, try to re-use a single writer
-   * instead of closing and opening a new one. See {@link #commit()} for caveats
-   * about write caching done by some IO devices.
-   * 
-   * <p>
-   * If an Exception is hit during close, eg due to disk full or some other
-   * reason, then both the on-disk index and the internal state of the
-   * IndexWriter instance will be consistent. However, the close will not be
-   * complete even though part of it (flushing buffered documents) may have
-   * succeeded, so the write lock will still be held.
-   * </p>
-   * 
+   * Commits all changes to an index, waits for pending merges
+   * to complete, and closes all associated files.  
    * <p>
-   * If you can correct the underlying cause (eg free up some disk space) then
-   * you can call close() again. Failing that, if you want to force the write
-   * lock to be released (dangerous, because you may then lose buffered docs in
-   * the IndexWriter instance) then you can do something like this:
-   * </p>
-   * 
+   * This is a "slow graceful shutdown" which may take a long time
+   * especially if a big merge is pending: If you only want to close
+   * resources use {@link #rollback()}. If you only want to commit
+   * pending changes and close resources see {@link #close(boolean)}.
+   * <p>
+   * Note that this may be a costly
+   * operation, so, try to re-use a single writer instead of
+   * closing and opening a new one.  See {@link #commit()} for
+   * caveats about write caching done by some IO devices.
+   *
+   * <p> If an Exception is hit during close, eg due to disk
+   * full or some other reason, then both the on-disk index
+   * and the internal state of the IndexWriter instance will
+   * be consistent.  However, the close will not be complete
+   * even though part of it (flushing buffered documents)
+   * may have succeeded, so the write lock will still be
+   * held.</p>
+   *
+   * <p> If you can correct the underlying cause (eg free up
+   * some disk space) then you can call close() again.
+   * Failing that, if you want to force the write lock to be
+   * released (dangerous, because you may then lose buffered
+   * docs in the IndexWriter instance) then you can do
+   * something like this:</p>
+   *
    * <pre class="prettyprint">
    * try {
    *   writer.close();
@@ -884,50 +869,49 @@ public class IndexWriter implements Clos
    *   }
    * }
    * </pre>
-   * 
-   * after which, you must be certain not to use the writer instance
-   * anymore.</p>
-   * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer, again. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
-   * @throws IOException
-   *           if there is a low-level IO error
+   *
+   * after which, you must be certain not to use the writer
+   * instance anymore.</p>
+   *
+   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
+   * you should immediately close the writer, again.  See <a
+   * href="#OOME">above</a> for details.</p>
+   *
+   * @throws IOException if there is a low-level IO error
    */
   @Override
   public void close() throws IOException {
     close(true);
   }
-  
+
   /**
-   * Closes the index with or without waiting for currently running merges to
-   * finish. This is only meaningful when using a MergeScheduler that runs
-   * merges in background threads.
-   * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer, again. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
-   * <p>
-   * <b>NOTE</b>: it is dangerous to always call close(false), especially when
-   * IndexWriter is not open for very long, because this can result in "merge
-   * starvation" whereby long merges will never have a chance to finish. This
-   * will cause too many segments in your index over time.
-   * </p>
-   * 
-   * @param waitForMerges
-   *          if true, this call will block until all merges complete; else, it
-   *          will ask all running merges to abort, wait until those merges have
-   *          finished (which should be at most a few seconds), and then return.
+   * Closes the index with or without waiting for currently
+   * running merges to finish.  This is only meaningful when
+   * using a MergeScheduler that runs merges in background
+   * threads.
+   *
+   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
+   * you should immediately close the writer, again.  See <a
+   * href="#OOME">above</a> for details.</p>
+   *
+   * <p><b>NOTE</b>: it is dangerous to always call
+   * close(false), especially when IndexWriter is not open
+   * for very long, because this can result in "merge
+   * starvation" whereby long merges will never have a
+   * chance to finish.  This will cause too many segments in
+   * your index over time.</p>
+   *
+   * @param waitForMerges if true, this call will block
+   * until all merges complete; else, it will ask all
+   * running merges to abort, wait until those merges have
+   * finished (which should be at most a few seconds), and
+   * then return.
    */
   public void close(boolean waitForMerges) throws IOException {
-    
+
     // Ensure that only one thread actually gets to do the
     // closing, and make sure no commit is also in progress:
-    synchronized (commitLock) {
+    synchronized(commitLock) {
       if (shouldClose()) {
         // If any methods have hit OutOfMemoryError, then abort
         // on close, in case the internal state of IndexWriter
@@ -940,12 +924,12 @@ public class IndexWriter implements Clos
       }
     }
   }
-  
+
   // Returns true if this thread should attempt to close, or
   // false if IndexWriter is now closed; else, waits until
   // another thread finishes closing
   synchronized private boolean shouldClose() {
-    while (true) {
+    while(true) {
       if (!closed) {
         if (!closing) {
           closing = true;
@@ -961,39 +945,34 @@ public class IndexWriter implements Clos
       }
     }
   }
-  
-  private void closeInternal(boolean waitForMerges, boolean doFlush)
-      throws IOException {
+
+  private void closeInternal(boolean waitForMerges, boolean doFlush) throws IOException {
     boolean interrupted = false;
     try {
-      
+
       if (pendingCommit != null) {
-        throw new IllegalStateException(
-            "cannot close: prepareCommit was already called with no corresponding call to commit");
+        throw new IllegalStateException("cannot close: prepareCommit was already called with no corresponding call to commit");
       }
-      
+
       if (infoStream.isEnabled("IW")) {
-        infoStream.message("IW", "now flush at close waitForMerges="
-            + waitForMerges);
+        infoStream.message("IW", "now flush at close waitForMerges=" + waitForMerges);
       }
-      
+
       try {
         // Only allow a new merge to be triggered if we are
         // going to wait for merges:
         if (doFlush) {
           flush(waitForMerges, true);
+          docWriter.close();
         } else {
-          docWriter.abort(); // already closed
+          docWriter.abort(); // already closed -- never sync on IW 
         }
         
-        docWriter.close();
-        
       } finally {
         try {
-          // clean up merge scheduler in all cases, although flushing may have
-          // failed:
+          // clean up merge scheduler in all cases, although flushing may have failed:
           interrupted = Thread.interrupted();
-          
+        
           if (waitForMerges) {
             try {
               // Give merge scheduler last chance to run, in case
@@ -1003,13 +982,12 @@ public class IndexWriter implements Clos
               // ignore any interruption, does not matter
               interrupted = true;
               if (infoStream.isEnabled("IW")) {
-                infoStream.message("IW",
-                    "interrupted while waiting for final merges");
+                infoStream.message("IW", "interrupted while waiting for final merges");
               }
             }
           }
           
-          synchronized (this) {
+          synchronized(this) {
             for (;;) {
               try {
                 finishMerges(waitForMerges && !interrupted);
@@ -1020,8 +998,7 @@ public class IndexWriter implements Clos
                 // so it will not wait
                 interrupted = true;
                 if (infoStream.isEnabled("IW")) {
-                  infoStream.message("IW",
-                      "interrupted while waiting for merges to finish");
+                  infoStream.message("IW", "interrupted while waiting for merges to finish");
                 }
               }
             }
@@ -1029,44 +1006,42 @@ public class IndexWriter implements Clos
           }
           
         } finally {
-          // shutdown policy, scheduler and all threads (this call is not
-          // interruptible):
+          // shutdown policy, scheduler and all threads (this call is not interruptible):
           IOUtils.closeWhileHandlingException(mergePolicy, mergeScheduler);
         }
       }
-      
+
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "now call final commit()");
       }
-      
+
       if (doFlush) {
         commitInternal();
       }
-      
+
       if (infoStream.isEnabled("IW")) {
         infoStream.message("IW", "at close: " + segString());
       }
       // used by assert below
       final DocumentsWriter oldWriter = docWriter;
-      synchronized (this) {
+      synchronized(this) {
         readerPool.dropAll(true);
         docWriter = null;
         deleter.close();
       }
-      
+
       if (writeLock != null) {
-        writeLock.release(); // release write lock
+        writeLock.release();                          // release write lock
         writeLock = null;
       }
-      synchronized (this) {
+      synchronized(this) {
         closed = true;
       }
-      assert oldWriter.perThreadPool.numDeactivatedThreadStates() == oldWriter.perThreadPool
-          .getMaxThreadStates();
+      assert oldWriter.perThreadPool.numDeactivatedThreadStates() == oldWriter.perThreadPool.getMaxThreadStates();
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "closeInternal");
     } finally {
-      synchronized (this) {
+      synchronized(this) {
         closing = false;
         notifyAll();
         if (!closed) {
@@ -1079,54 +1054,54 @@ public class IndexWriter implements Clos
       if (interrupted) Thread.currentThread().interrupt();
     }
   }
-  
+
   /** Returns the Directory used by this index. */
   public Directory getDirectory() {
     return directory;
   }
-  
+
   /** Returns the analyzer used by this index. */
   public Analyzer getAnalyzer() {
     ensureOpen();
     return analyzer;
   }
-  
-  /**
-   * Returns total number of docs in this index, including docs not yet flushed
-   * (still in the RAM buffer), not counting deletions.
-   * 
-   * @see #numDocs
-   */
+
+  /** Returns total number of docs in this index, including
+   *  docs not yet flushed (still in the RAM buffer),
+   *  not counting deletions.
+   *  @see #numDocs */
   public synchronized int maxDoc() {
     ensureOpen();
     int count;
-    if (docWriter != null) count = docWriter.getNumDocs();
-    else count = 0;
-    
+    if (docWriter != null)
+      count = docWriter.getNumDocs();
+    else
+      count = 0;
+
     count += segmentInfos.totalDocCount();
     return count;
   }
-  
-  /**
-   * Returns total number of docs in this index, including docs not yet flushed
-   * (still in the RAM buffer), and including deletions. <b>NOTE:</b> buffered
-   * deletions are not counted. If you really need these to be counted you
-   * should call {@link #commit()} first.
-   * 
-   * @see #numDocs
-   */
+
+  /** Returns total number of docs in this index, including
+   *  docs not yet flushed (still in the RAM buffer), and
+   *  including deletions.  <b>NOTE:</b> buffered deletions
+   *  are not counted.  If you really need these to be
+   *  counted you should call {@link #commit()} first.
+   *  @see #numDocs */
   public synchronized int numDocs() {
     ensureOpen();
     int count;
-    if (docWriter != null) count = docWriter.getNumDocs();
-    else count = 0;
-    
+    if (docWriter != null)
+      count = docWriter.getNumDocs();
+    else
+      count = 0;
+
     for (final SegmentInfoPerCommit info : segmentInfos) {
       count += info.info.getDocCount() - numDeletedDocs(info);
     }
     return count;
   }
-  
+
   /**
    * Returns true if this index has deletions (including buffered deletions).
    */
@@ -1148,159 +1123,143 @@ public class IndexWriter implements Clos
     }
     return false;
   }
-  
+
   /**
    * Adds a document to this index.
-   * 
-   * <p>
-   * Note that if an Exception is hit (for example disk full) then the index
-   * will be consistent, but this document may not have been added. Furthermore,
-   * it's possible the index will have one segment in non-compound format even
-   * when using compound files (when a merge has partially succeeded).
-   * </p>
-   * 
-   * <p>
-   * This method periodically flushes pending documents to the Directory (see <a
-   * href="#flush">above</a>), and also periodically triggers segment merges in
-   * the index according to the {@link MergePolicy} in use.
-   * </p>
-   * 
-   * <p>
-   * Merges temporarily consume space in the directory. The amount of space
-   * required is up to 1X the size of all segments being merged, when no
-   * readers/searchers are open against the index, and up to 2X the size of all
-   * segments being merged when readers/searchers are open against the index
-   * (see {@link #forceMerge(int)} for details). The sequence of primitive merge
-   * operations performed is governed by the merge policy.
-   * 
-   * <p>
-   * Note that each term in the document can be no longer than 16383 characters,
-   * otherwise an IllegalArgumentException will be thrown.
-   * </p>
-   * 
-   * <p>
-   * Note that it's possible to create an invalid Unicode string in java if a
-   * UTF16 surrogate pair is malformed. In this case, the invalid characters are
-   * silently replaced with the Unicode replacement character U+FFFD.
-   * </p>
-   * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
-   * @throws CorruptIndexException
-   *           if the index is corrupt
-   * @throws IOException
-   *           if there is a low-level IO error
+   *
+   * <p> Note that if an Exception is hit (for example disk full)
+   * then the index will be consistent, but this document
+   * may not have been added.  Furthermore, it's possible
+   * the index will have one segment in non-compound format
+   * even when using compound files (when a merge has
+   * partially succeeded).</p>
+   *
+   * <p> This method periodically flushes pending documents
+   * to the Directory (see <a href="#flush">above</a>), and
+   * also periodically triggers segment merges in the index
+   * according to the {@link MergePolicy} in use.</p>
+   *
+   * <p>Merges temporarily consume space in the
+   * directory. The amount of space required is up to 1X the
+   * size of all segments being merged, when no
+   * readers/searchers are open against the index, and up to
+   * 2X the size of all segments being merged when
+   * readers/searchers are open against the index (see
+   * {@link #forceMerge(int)} for details). The sequence of
+   * primitive merge operations performed is governed by the
+   * merge policy.
+   *
+   * <p>Note that each term in the document can be no longer
+   * than 16383 characters, otherwise an
+   * IllegalArgumentException will be thrown.</p>
+   *
+   * <p>Note that it's possible to create an invalid Unicode
+   * string in java if a UTF16 surrogate pair is malformed.
+   * In this case, the invalid characters are silently
+   * replaced with the Unicode replacement character
+   * U+FFFD.</p>
+   *
+   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
+   * you should immediately close the writer.  See <a
+   * href="#OOME">above</a> for details.</p>
+   *
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
    */
   public void addDocument(IndexDocument doc) throws IOException {
     addDocument(doc, analyzer);
   }
-  
+
   /**
    * Adds a document to this index, using the provided analyzer instead of the
    * value of {@link #getAnalyzer()}.
-   * 
-   * <p>
-   * See {@link #addDocument(IndexDocument)} for details on index and
-   * IndexWriter state after an Exception, and flushing/merging temporary free
-   * space requirements.
-   * </p>
-   * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
-   * @throws CorruptIndexException
-   *           if the index is corrupt
-   * @throws IOException
-   *           if there is a low-level IO error
+   *
+   * <p>See {@link #addDocument(IndexDocument)} for details on
+   * index and IndexWriter state after an Exception, and
+   * flushing/merging temporary free space requirements.</p>
+   *
+   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
+   * you should immediately close the writer.  See <a
+   * href="#OOME">above</a> for details.</p>
+   *
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
    */
-  public void addDocument(IndexDocument doc, Analyzer analyzer)
-      throws IOException {
-    replaceDocument(null, doc, analyzer);
+  public void addDocument(IndexDocument doc, Analyzer analyzer) throws IOException {
+    updateDocument(null, doc, analyzer);
   }
-  
+
   /**
-   * Atomically adds a block of documents with sequentially assigned document
-   * IDs, such that an external reader will see all or none of the documents.
-   * 
-   * <p>
-   * <b>WARNING</b>: the index does not currently record which documents were
-   * added as a block. Today this is fine, because merging will preserve a
-   * block. The order of documents within a segment will be preserved, even when
-   * child documents within a block are deleted. Most search features (like
-   * result grouping and block joining) require you to mark documents; when
-   * these documents are deleted these search features will not work as
-   * expected. Obviously adding documents to an existing block will require you
-   * the reindex the entire block.
-   * 
-   * <p>
-   * However it's possible that in the future Lucene may merge more aggressively
-   * re-order documents (for example, perhaps to obtain better index
-   * compression), in which case you may need to fully re-index your documents
-   * at that time.
-   * 
-   * <p>
-   * See {@link #addDocument(IndexDocument)} for details on index and
-   * IndexWriter state after an Exception, and flushing/merging temporary free
-   * space requirements.
-   * </p>
-   * 
-   * <p>
-   * <b>NOTE</b>: tools that do offline splitting of an index (for example,
-   * IndexSplitter in contrib) or re-sorting of documents (for example,
-   * IndexSorter in contrib) are not aware of these atomically added documents
-   * and will likely break them up. Use such tools at your own risk!
-   * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
-   * @throws CorruptIndexException
-   *           if the index is corrupt
-   * @throws IOException
-   *           if there is a low-level IO error
-   * 
+   * Atomically adds a block of documents with sequentially
+   * assigned document IDs, such that an external reader
+   * will see all or none of the documents.
+   *
+   * <p><b>WARNING</b>: the index does not currently record
+   * which documents were added as a block.  Today this is
+   * fine, because merging will preserve a block. The order of
+   * documents within a segment will be preserved, even when child
+   * documents within a block are deleted. Most search features
+   * (like result grouping and block joining) require you to
+   * mark documents; when these documents are deleted these
+   * search features will not work as expected. Obviously adding
+   * documents to an existing block will require you the reindex
+   * the entire block.
+   *
+   * <p>However it's possible that in the future Lucene may
+   * merge more aggressively re-order documents (for example,
+   * perhaps to obtain better index compression), in which case
+   * you may need to fully re-index your documents at that time.
+   *
+   * <p>See {@link #addDocument(IndexDocument)} for details on
+   * index and IndexWriter state after an Exception, and
+   * flushing/merging temporary free space requirements.</p>
+   *
+   * <p><b>NOTE</b>: tools that do offline splitting of an index
+   * (for example, IndexSplitter in contrib) or
+   * re-sorting of documents (for example, IndexSorter in
+   * contrib) are not aware of these atomically added documents
+   * and will likely break them up.  Use such tools at your
+   * own risk!
+   *
+   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
+   * you should immediately close the writer.  See <a
+   * href="#OOME">above</a> for details.</p>
+   *
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   *
    * @lucene.experimental
    */
-  public void addDocuments(Iterable<? extends IndexDocument> docs)
-      throws IOException {
+  public void addDocuments(Iterable<? extends IndexDocument> docs) throws IOException {
     addDocuments(docs, analyzer);
   }
-  
+
   /**
-   * Atomically adds a block of documents, analyzed using the provided analyzer,
-   * with sequentially assigned document IDs, such that an external reader will
-   * see all or none of the documents.
-   * 
-   * @throws CorruptIndexException
-   *           if the index is corrupt
-   * @throws IOException
-   *           if there is a low-level IO error
-   * 
+   * Atomically adds a block of documents, analyzed using the
+   * provided analyzer, with sequentially assigned document
+   * IDs, such that an external reader will see all or none
+   * of the documents. 
+   *
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   *
    * @lucene.experimental
    */
-  public void addDocuments(Iterable<? extends IndexDocument> docs,
-      Analyzer analyzer) throws IOException {
+  public void addDocuments(Iterable<? extends IndexDocument> docs, Analyzer analyzer) throws IOException {
     updateDocuments(null, docs, analyzer);
   }
-  
+
   /**
-   * Atomically deletes documents matching the provided delTerm and adds a block
-   * of documents with sequentially assigned document IDs, such that an external
-   * reader will see all or none of the documents.
-   * 
+   * Atomically deletes documents matching the provided
+   * delTerm and adds a block of documents with sequentially
+   * assigned document IDs, such that an external reader
+   * will see all or none of the documents. 
+   *
    * See {@link #addDocuments(Iterable)}.
-   * 
-   * @throws CorruptIndexException
-   *           if the index is corrupt
-   * @throws IOException
-   *           if there is a low-level IO error
-   * 
+   *
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   *
    * @lucene.experimental
    */
   public void replaceDocuments(Term delTerm,
@@ -1318,20 +1277,19 @@ public class IndexWriter implements Clos
       Iterable<? extends IndexDocument> docs) throws IOException {
     replaceDocuments(delTerm, docs, analyzer);
   }
-  
+
   /**
-   * Atomically deletes documents matching the provided delTerm and adds a block
-   * of documents, analyzed using the provided analyzer, with sequentially
-   * assigned document IDs, such that an external reader will see all or none of
-   * the documents.
-   * 
+   * Atomically deletes documents matching the provided
+   * delTerm and adds a block of documents, analyzed  using
+   * the provided analyzer, with sequentially
+   * assigned document IDs, such that an external reader
+   * will see all or none of the documents. 
+   *
    * See {@link #addDocuments(Iterable)}.
-   * 
-   * @throws CorruptIndexException
-   *           if the index is corrupt
-   * @throws IOException
-   *           if there is a low-level IO error
-   * 
+   *
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   *
    * @lucene.experimental
    */
   public void replaceDocuments(Term delTerm,
@@ -1344,6 +1302,9 @@ public class IndexWriter implements Clos
       try {
         anySegmentFlushed = docWriter.updateDocuments(docs, analyzer, delTerm);
         success = true;
+        if (delTerm != null) {
+          deletesPending.set(true);
+        }
       } finally {
         if (!success) {
           if (infoStream.isEnabled("IW")) {
@@ -1435,6 +1396,12 @@ public class IndexWriter implements Clos
   public void updateFields(FieldsUpdate.Operation operation, Term term,
       IndexDocument fields, Analyzer analyzer) throws IOException {
     ensureOpen();
+
+    if (deletesPending.get()) {
+      commit();
+      deletesPending.set(false);
+    }
+    
     try {
       boolean success = false;
       boolean anySegmentFlushed = false;
@@ -1442,7 +1409,6 @@ public class IndexWriter implements Clos
         anySegmentFlushed = docWriter.updateFields(term, operation, fields,
             analyzer, globalFieldNumberMap);
         success = true;
-        updatesPending = true;
       } finally {
         if (!success) {
           if (infoStream.isEnabled("IW")) {
@@ -1461,44 +1427,43 @@ public class IndexWriter implements Clos
   
   /**
    * Deletes the document(s) containing <code>term</code>.
-   * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
-   * @param term
-   *          the term to identify the documents to be deleted
-   * @throws CorruptIndexException
-   *           if the index is corrupt
-   * @throws IOException
-   *           if there is a low-level IO error
+   *
+   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
+   * you should immediately close the writer.  See <a
+   * href="#OOME">above</a> for details.</p>
+   *
+   * @param term the term to identify the documents to be deleted
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
    */
   public void deleteDocuments(Term term) throws IOException {
     ensureOpen();
     try {
       docWriter.deleteTerms(term);
+      if (term != null) {
+        deletesPending.set(true);
+      }
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteDocuments(Term)");
     }
   }
-  
-  /**
-   * Expert: attempts to delete by document ID, as long as the provided reader
-   * is a near-real-time reader (from
-   * {@link DirectoryReader#open(IndexWriter,boolean)}). If the provided reader
-   * is an NRT reader obtained from this writer, and its segment has not been
-   * merged away, then the delete succeeds and this method returns true; else,
-   * it returns false the caller must then separately delete by Term or Query.
-   * 
-   * <b>NOTE</b>: this method can only delete documents visible to the currently
-   * open NRT reader. If you need to delete documents indexed after opening the
-   * NRT reader you must use the other deleteDocument methods (e.g.,
-   * {@link #deleteDocuments(Term)}).
-   */
-  public synchronized boolean tryDeleteDocument(IndexReader readerIn, int docID)
-      throws IOException {
-    
+
+  /** Expert: attempts to delete by document ID, as long as
+   *  the provided reader is a near-real-time reader (from {@link
+   *  DirectoryReader#open(IndexWriter,boolean)}).  If the
+   *  provided reader is an NRT reader obtained from this
+   *  writer, and its segment has not been merged away, then
+   *  the delete succeeds and this method returns true; else, it
+   *  returns false the caller must then separately delete by
+   *  Term or Query.
+   *
+   *  <b>NOTE</b>: this method can only delete documents
+   *  visible to the currently open NRT reader.  If you need
+   *  to delete documents indexed after opening the NRT
+   *  reader you must use the other deleteDocument methods
+   *  (e.g., {@link #deleteDocuments(Term)}). */
+  public synchronized boolean tryDeleteDocument(IndexReader readerIn, int docID) throws IOException {
+
     final AtomicReader reader;
     if (readerIn instanceof AtomicReader) {
       // Reader is already atomic: use the incoming docID:
@@ -1512,27 +1477,25 @@ public class IndexWriter implements Clos
       assert docID >= 0;
       assert docID < reader.maxDoc();
     }
-    
+
     if (!(reader instanceof SegmentReader)) {
-      throw new IllegalArgumentException(
-          "the reader must be a SegmentReader or composite reader containing only SegmentReaders");
+      throw new IllegalArgumentException("the reader must be a SegmentReader or composite reader containing only SegmentReaders");
     }
-    
+      
     final SegmentInfoPerCommit info = ((SegmentReader) reader).getSegmentInfo();
-    
+
     // TODO: this is a slow linear search, but, number of
     // segments should be contained unless something is
     // seriously wrong w/ the index, so it should be a minor
     // cost:
-    
+
     if (segmentInfos.indexOf(info) != -1) {
       ReadersAndLiveDocs rld = readerPool.get(info, false);
       if (rld != null) {
-        synchronized (bufferedDeletesStream) {
+        synchronized(bufferedDeletesStream) {
           rld.initWritableLiveDocs();
           if (rld.delete(docID)) {
-            final int fullDelCount = rld.info.getDelCount()
-                + rld.getPendingDeleteCount();
+            final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount();
             if (fullDelCount == rld.info.info.getDocCount()) {
               // If a merge has already registered for this
               // segment, we leave it in the readerPool; the
@@ -1544,92 +1507,92 @@ public class IndexWriter implements Clos
                 checkpoint();
               }
             }
-            
+
             // Must bump changeCount so if no other changes
             // happened, we still commit this change:
             changed();
           }
-          // System.out.println("  yes " + info.info.name + " " + docID);
+          //System.out.println("  yes " + info.info.name + " " + docID);
           return true;
         }
       } else {
-        // System.out.println("  no rld " + info.info.name + " " + docID);
+        //System.out.println("  no rld " + info.info.name + " " + docID);
       }
     } else {
-      // System.out.println("  no seg " + info.info.name + " " + docID);
+      //System.out.println("  no seg " + info.info.name + " " + docID);
     }
     return false;
   }
-  
+
   /**
-   * Deletes the document(s) containing any of the terms. All given deletes are
-   * applied and flushed atomically at the same time.
-   * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
-   * @param terms
-   *          array of terms to identify the documents to be deleted
-   * @throws CorruptIndexException
-   *           if the index is corrupt
-   * @throws IOException
-   *           if there is a low-level IO error
+   * Deletes the document(s) containing any of the
+   * terms. All given deletes are applied and flushed atomically
+   * at the same time.
+   *
+   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
+   * you should immediately close the writer.  See <a
+   * href="#OOME">above</a> for details.</p>
+   *
+   * @param terms array of terms to identify the documents
+   * to be deleted
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
    */
   public void deleteDocuments(Term... terms) throws IOException {
     ensureOpen();
     try {
       docWriter.deleteTerms(terms);
+      if (terms != null && terms.length > 0) {
+        deletesPending.set(true);
+      }
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteDocuments(Term..)");
     }
   }
-  
+
   /**
    * Deletes the document(s) matching the provided query.
-   * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
-   * @param query
-   *          the query to identify the documents to be deleted
-   * @throws CorruptIndexException
-   *           if the index is corrupt
-   * @throws IOException
-   *           if there is a low-level IO error
+   *
+   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
+   * you should immediately close the writer.  See <a
+   * href="#OOME">above</a> for details.</p>
+   *
+   * @param query the query to identify the documents to be deleted
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
    */
   public void deleteDocuments(Query query) throws IOException {
     ensureOpen();
     try {
       docWriter.deleteQueries(query);
+      if (query != null) {
+        deletesPending.set(true);
+      }
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteDocuments(Query)");
     }
   }
-  
+
   /**
-   * Deletes the document(s) matching any of the provided queries. All given
-   * deletes are applied and flushed atomically at the same time.
-   * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
-   * @param queries
-   *          array of queries to identify the documents to be deleted
-   * @throws CorruptIndexException
-   *           if the index is corrupt
-   * @throws IOException
-   *           if there is a low-level IO error
+   * Deletes the document(s) matching any of the provided queries.
+   * All given deletes are applied and flushed atomically at the same time.
+   *
+   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
+   * you should immediately close the writer.  See <a
+   * href="#OOME">above</a> for details.</p>
+   *
+   * @param queries array of queries to identify the documents
+   * to be deleted
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
    */
   public void deleteDocuments(Query... queries) throws IOException {
     ensureOpen();
     try {
       docWriter.deleteQueries(queries);
+      if (queries != null && queries.length > 0) {
+        deletesPending.set(true);
+      }
     } catch (OutOfMemoryError oom) {
       handleOOM(oom, "deleteDocuments(Query..)");
     }
@@ -1673,28 +1636,24 @@ public class IndexWriter implements Clos
     ensureOpen();
     replaceDocument(term, doc, analyzer);
   }
-  
+
   /**
-   * Updates a document by first deleting the document(s) containing
-   * <code>term</code> and then adding the new document. The delete and then add
-   * are atomic as seen by a reader on the same index (flush may happen only
-   * after the add).
-   * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
-   * @param term
-   *          the term to identify the document(s) to be deleted
-   * @param doc
-   *          the document to be added
-   * @param analyzer
-   *          the analyzer to use when analyzing the document
-   * @throws CorruptIndexException
-   *           if the index is corrupt
-   * @throws IOException
-   *           if there is a low-level IO error
+   * Updates a document by first deleting the document(s)
+   * containing <code>term</code> and then adding the new
+   * document.  The delete and then add are atomic as seen
+   * by a reader on the same index (flush may happen only after
+   * the add).
+   *
+   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
+   * you should immediately close the writer.  See <a
+   * href="#OOME">above</a> for details.</p>
+   *
+   * @param term the term to identify the document(s) to be
+   * deleted
+   * @param doc the document to be added
+   * @param analyzer the analyzer to use when analyzing the document
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
    */
   public void replaceDocument(Term term, IndexDocument doc, Analyzer analyzer)
       throws IOException {
@@ -1705,6 +1664,9 @@ public class IndexWriter implements Clos
       try {
         anySegmentFlushed = docWriter.updateDocument(doc, analyzer, term);
         success = true;
+        if (term != null) {
+          deletesPending.set(true);
+        }
       } finally {
         if (!success) {
           if (infoStream.isEnabled("IW")) {
@@ -1712,7 +1674,7 @@ public class IndexWriter implements Clos
           }
         }
       }
-      
+
       if (anySegmentFlushed) {
         maybeMerge(MergeTrigger.SEGMENT_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
       }
@@ -1738,20 +1700,20 @@ public class IndexWriter implements Clos
   }
   
   // for test purpose
-  final synchronized int getSegmentCount() {
+  final synchronized int getSegmentCount(){
     return segmentInfos.size();
   }
-  
+
   // for test purpose
-  final synchronized int getNumBufferedDocuments() {
+  final synchronized int getNumBufferedDocuments(){
     return docWriter.getNumDocs();
   }
-  
+
   // for test purpose
   final synchronized Collection<String> getIndexFileNames() throws IOException {
     return segmentInfos.files(directory, true);
   }
-  
+
   // for test purpose
   final synchronized int getDocCount(int i) {
     if (i >= 0 && i < segmentInfos.size()) {
@@ -1760,407 +1722,392 @@ public class IndexWriter implements Clos
       return -1;
     }
   }
-  
+
   // for test purpose
   final int getFlushCount() {
     return flushCount.get();
   }
-  
+
   // for test purpose
   final int getFlushDeletesCount() {
     return flushDeletesCount.get();
   }
-  
+
   final String newSegmentName() {
     // Cannot synchronize on IndexWriter because that causes
     // deadlock
-    synchronized (segmentInfos) {
+    synchronized(segmentInfos) {
       // Important to increment changeCount so that the
-      // segmentInfos is written on close. Otherwise we
+      // segmentInfos is written on close.  Otherwise we
       // could close, re-open and re-return the same segment
       // name that was previously returned which can cause
       // problems at least with ConcurrentMergeScheduler.
       changeCount++;
       segmentInfos.changed();
-      return "_"
-          + Integer.toString(segmentInfos.counter++, Character.MAX_RADIX);
+      return "_" + Integer.toString(segmentInfos.counter++, Character.MAX_RADIX);
     }
   }
-  
-  /**
-   * If non-null, information about merges will be printed to this.
+
+  /** If non-null, information about merges will be printed to this.
    */
   final InfoStream infoStream;
-  
+
   /**
-   * Forces merge policy to merge segments until there are <= maxNumSegments.
-   * The actual merges to be executed are determined by the {@link MergePolicy}.
-   * 
-   * <p>
-   * This is a horribly costly operation, especially when you pass a small
-   * {@code maxNumSegments}; usually you should only call this if the index is
-   * static (will no longer be changed).
-   * </p>
-   * 
-   * <p>
-   * Note that this requires up to 2X the index size free space in your
-   * Directory (3X if you're using compound file format). For example, if your
-   * index size is 10 MB then you need up to 20 MB free for this to complete (30
-   * MB if you're using compound file format). Also, it's best to call
-   * {@link #commit()} afterwards, to allow IndexWriter to free up disk space.
-   * </p>
-   * 
-   * <p>
-   * If some but not all readers re-open while merging is underway, this will
-   * cause > 2X temporary space to be consumed as those new readers will then
-   * hold open the temporary segments at that time. It is best not to re-open
-   * readers while merging is running.
-   * </p>
-   * 
-   * <p>
-   * The actual temporary usage could be much less than these figures (it
-   * depends on many factors).
-   * </p>
-   * 
-   * <p>
-   * In general, once this completes, the total size of the index will be less
-   * than the size of the starting index. It could be quite a bit smaller (if
-   * there were many pending deletes) or just slightly smaller.
-   * </p>
-   * 
-   * <p>
-   * If an Exception is hit, for example due to disk full, the index will not be
-   * corrupted and no documents will be lost. However, it may have been
-   * partially merged (some segments were merged but not all), and it's possible
-   * that one of the segments in the index will be in non-compound format even
-   * when using compound file format. This will occur when the Exception is hit
-   * during conversion of the segment into compound format.
-   * </p>
-   * 
-   * <p>
-   * This call will merge those segments present in the index when the call
-   * started. If other threads are still adding documents and flushing segments,
-   * those newly created segments will not be merged unless you call forceMerge
-   * again.
-   * </p>
-   * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
+   * Forces merge policy to merge segments until there are <=
+   * maxNumSegments.  The actual merges to be
+   * executed are determined by the {@link MergePolicy}.
+   *
+   * <p>This is a horribly costly operation, especially when
+   * you pass a small {@code maxNumSegments}; usually you
+   * should only call this if the index is static (will no
+   * longer be changed).</p>
+   *
+   * <p>Note that this requires up to 2X the index size free
+   * space in your Directory (3X if you're using compound
+   * file format).  For example, if your index size is 10 MB
+   * then you need up to 20 MB free for this to complete (30
+   * MB if you're using compound file format).  Also,
+   * it's best to call {@link #commit()} afterwards,
+   * to allow IndexWriter to free up disk space.</p>
+   *
+   * <p>If some but not all readers re-open while merging
+   * is underway, this will cause > 2X temporary
+   * space to be consumed as those new readers will then
+   * hold open the temporary segments at that time.  It is
+   * best not to re-open readers while merging is running.</p>
+   *
+   * <p>The actual temporary usage could be much less than
+   * these figures (it depends on many factors).</p>
+   *
+   * <p>In general, once this completes, the total size of the
+   * index will be less than the size of the starting index.
+   * It could be quite a bit smaller (if there were many
+   * pending deletes) or just slightly smaller.</p>
+   *
+   * <p>If an Exception is hit, for example
+   * due to disk full, the index will not be corrupted and no
+   * documents will be lost.  However, it may have
+   * been partially merged (some segments were merged but
+   * not all), and it's possible that one of the segments in
+   * the index will be in non-compound format even when
+   * using compound file format.  This will occur when the
+   * Exception is hit during conversion of the segment into
+   * compound format.</p>
+   *
+   * <p>This call will merge those segments present in
+   * the index when the call started.  If other threads are
+   * still adding documents and flushing segments, those
+   * newly created segments will not be merged unless you
+   * call forceMerge again.</p>
+   *
+   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
+   * you should immediately close the writer.  See <a
+   * href="#OOME">above</a> for details.</p>
+   *
+   * <p><b>NOTE</b>: if you call {@link #close(boolean)}
+   * with <tt>false</tt>, which aborts all running merges,
+   * then any thread still running this method might hit a
+   * {@link MergePolicy.MergeAbortedException}.
+   *
+   * @param maxNumSegments maximum number of segments left
+   * in the index after merging finishes
    * 
-   * <p>
-   * <b>NOTE</b>: if you call {@link #close(boolean)} with <tt>false</tt>, which
-   * aborts all running merges, then any thread still running this method might
-   * hit a {@link MergePolicy.MergeAbortedException}.
-   * 
-   * @param maxNumSegments
-   *          maximum number of segments left in the index after merging
-   *          finishes
-   * 
-   * @throws CorruptIndexException
-   *           if the index is corrupt
-   * @throws IOException
-   *           if there is a low-level IO error
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
    * @see MergePolicy#findMerges
-   * 
-   */
+   *
+  */
   public void forceMerge(int maxNumSegments) throws IOException {
     forceMerge(maxNumSegments, true);
   }
-  
-  /**
-   * Just like {@link #forceMerge(int)}, except you can specify whether the call
-   * should block until all merging completes. This is only meaningful with a
-   * {@link MergeScheduler} that is able to run merges in background threads.
-   * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
+
+  /** Just like {@link #forceMerge(int)}, except you can
+   *  specify whether the call should block until
+   *  all merging completes.  This is only meaningful with a
+   *  {@link MergeScheduler} that is able to run merges in
+   *  background threads.
+   *
+   *  <p><b>NOTE</b>: if this method hits an OutOfMemoryError
+   *  you should immediately close the writer.  See <a
+   *  href="#OOME">above</a> for details.</p>
    */
   public void forceMerge(int maxNumSegments, boolean doWait) throws IOException {
     ensureOpen();
-    
-    if (maxNumSegments < 1) throw new IllegalArgumentException(
-        "maxNumSegments must be >= 1; got " + maxNumSegments);
-    
+
+    if (maxNumSegments < 1)
+      throw new IllegalArgumentException("maxNumSegments must be >= 1; got " + maxNumSegments);
+
     if (infoStream.isEnabled("IW")) {
       infoStream.message("IW", "forceMerge: index now " + segString());
       infoStream.message("IW", "now flush at forceMerge");
     }
-    
+
     flush(true, true);
-    
-    synchronized (this) {
+
+    synchronized(this) {
       resetMergeExceptions();
       segmentsToMerge.clear();
-      for (SegmentInfoPerCommit info : segmentInfos) {
+      for(SegmentInfoPerCommit info : segmentInfos) {
         segmentsToMerge.put(info, Boolean.TRUE);
       }
       mergeMaxNumSegments = maxNumSegments;

[... 3257 lines stripped ...]