add --shared
[pylucene.git] / lucene-java-3.4.0 / lucene / src / java / org / apache / lucene / index / IndexWriter.java
1 package org.apache.lucene.index;
2
3 /**
4  * Licensed to the Apache Software Foundation (ASF) under one or more
5  * contributor license agreements.  See the NOTICE file distributed with
6  * this work for additional information regarding copyright ownership.
7  * The ASF licenses this file to You under the Apache License, Version 2.0
8  * (the "License"); you may not use this file except in compliance with
9  * the License.  You may obtain a copy of the License at
10  *
11  *     http://www.apache.org/licenses/LICENSE-2.0
12  *
13  * Unless required by applicable law or agreed to in writing, software
14  * distributed under the License is distributed on an "AS IS" BASIS,
15  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16  * See the License for the specific language governing permissions and
17  * limitations under the License.
18  */
19
20 import java.io.Closeable;
21 import java.io.IOException;
22 import java.io.PrintStream;
23 import java.util.ArrayList;
24 import java.util.Collection;
25 import java.util.Collections;
26 import java.util.Comparator;
27 import java.util.Date;
28 import java.util.HashMap;
29 import java.util.HashSet;
30 import java.util.LinkedList;
31 import java.util.List;
32 import java.util.Map;
33 import java.util.Set;
34 import java.util.concurrent.atomic.AtomicInteger;
35 import java.util.concurrent.ConcurrentHashMap;
36
37 import org.apache.lucene.analysis.Analyzer;
38 import org.apache.lucene.analysis.LimitTokenCountAnalyzer;
39 import org.apache.lucene.document.Document;
40 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
41 import org.apache.lucene.index.PayloadProcessorProvider.DirPayloadProcessor;
42 import org.apache.lucene.search.Similarity;
43 import org.apache.lucene.search.Query;
44 import org.apache.lucene.store.AlreadyClosedException;
45 import org.apache.lucene.store.BufferedIndexInput;
46 import org.apache.lucene.store.Directory;
47 import org.apache.lucene.store.Lock;
48 import org.apache.lucene.store.LockObtainFailedException;
49 import org.apache.lucene.util.Constants;
50 import org.apache.lucene.util.StringHelper;
51 import org.apache.lucene.util.ThreadInterruptedException;
52 import org.apache.lucene.util.Version;
53 import org.apache.lucene.util.MapBackedSet;
54 import org.apache.lucene.util.TwoPhaseCommit;
55
56 /**
57   An <code>IndexWriter</code> creates and maintains an index.
58
59   <p>The <code>create</code> argument to the {@link
60   #IndexWriter(Directory, Analyzer, boolean, MaxFieldLength) constructor} determines 
61   whether a new index is created, or whether an existing index is
62   opened.  Note that you can open an index with <code>create=true</code>
63   even while readers are using the index.  The old readers will 
64   continue to search the "point in time" snapshot they had opened, 
65   and won't see the newly created index until they re-open.  There are
66   also {@link #IndexWriter(Directory, Analyzer, MaxFieldLength) constructors}
67   with no <code>create</code> argument which will create a new index
68   if there is not already an index at the provided path and otherwise 
69   open the existing index.</p>
70
71   <p>In either case, documents are added with {@link #addDocument(Document)
72   addDocument} and removed with {@link #deleteDocuments(Term)} or {@link
73   #deleteDocuments(Query)}. A document can be updated with {@link
74   #updateDocument(Term, Document) updateDocument} (which just deletes
75   and then adds the entire document). When finished adding, deleting 
76   and updating documents, {@link #close() close} should be called.</p>
77
78   <a name="flush"></a>
79   <p>These changes are buffered in memory and periodically
80   flushed to the {@link Directory} (during the above method
81   calls).  A flush is triggered when there are enough
82   buffered deletes (see {@link #setMaxBufferedDeleteTerms})
83   or enough added documents since the last flush, whichever
84   is sooner.  For the added documents, flushing is triggered
85   either by RAM usage of the documents (see {@link
86   #setRAMBufferSizeMB}) or the number of added documents.
87   The default is to flush when RAM usage hits 16 MB.  For
88   best indexing speed you should flush by RAM usage with a
89   large RAM buffer.  Note that flushing just moves the
90   internal buffered state in IndexWriter into the index, but
91   these changes are not visible to IndexReader until either
92   {@link #commit()} or {@link #close} is called.  A flush may
93   also trigger one or more segment merges which by default
94   run with a background thread so as not to block the
95   addDocument calls (see <a href="#mergePolicy">below</a>
96   for changing the {@link MergeScheduler}).</p>
97
98   <p>If an index will not have more documents added for a while and optimal search
99   performance is desired, then either the full {@link #optimize() optimize}
100   method or partial {@link #optimize(int)} method should be
101   called before the index is closed.</p>
102
103   <p>Opening an <code>IndexWriter</code> creates a lock file for the directory in use. Trying to open
104   another <code>IndexWriter</code> on the same directory will lead to a
105   {@link LockObtainFailedException}. The {@link LockObtainFailedException}
106   is also thrown if an IndexReader on the same directory is used to delete documents
107   from the index.</p>
108   
109   <a name="deletionPolicy"></a>
110   <p>Expert: <code>IndexWriter</code> allows an optional
111   {@link IndexDeletionPolicy} implementation to be
112   specified.  You can use this to control when prior commits
113   are deleted from the index.  The default policy is {@link
114   KeepOnlyLastCommitDeletionPolicy} which removes all prior
115   commits as soon as a new commit is done (this matches
116   behavior before 2.2).  Creating your own policy can allow
117   you to explicitly keep previous "point in time" commits
118   alive in the index for some time, to allow readers to
119   refresh to the new commit without having the old commit
120   deleted out from under them.  This is necessary on
121   filesystems like NFS that do not support "delete on last
122   close" semantics, which Lucene's "point in time" search
123   normally relies on. </p>
124
125   <a name="mergePolicy"></a> <p>Expert:
126   <code>IndexWriter</code> allows you to separately change
127   the {@link MergePolicy} and the {@link MergeScheduler}.
128   The {@link MergePolicy} is invoked whenever there are
129   changes to the segments in the index.  Its role is to
130   select which merges to do, if any, and return a {@link
131   MergePolicy.MergeSpecification} describing the merges.  It
132   also selects merges to do for optimize().  (The default is
133   {@link LogByteSizeMergePolicy}.  Then, the {@link
134   MergeScheduler} is invoked with the requested merges and
135   it decides when and how to run the merges.  The default is
136   {@link ConcurrentMergeScheduler}. </p>
137
138   <a name="OOME"></a><p><b>NOTE</b>: if you hit an
139   OutOfMemoryError then IndexWriter will quietly record this
140   fact and block all future segment commits.  This is a
141   defensive measure in case any internal state (buffered
142   documents and deletions) were corrupted.  Any subsequent
143   calls to {@link #commit()} will throw an
144   IllegalStateException.  The only course of action is to
145   call {@link #close()}, which internally will call {@link
146   #rollback()}, to undo any changes to the index since the
147   last commit.  You can also just call {@link #rollback()}
148   directly.</p>
149
150   <a name="thread-safety"></a><p><b>NOTE</b>: {@link
151   IndexWriter} instances are completely thread
152   safe, meaning multiple threads can call any of its
153   methods, concurrently.  If your application requires
154   external synchronization, you should <b>not</b>
155   synchronize on the <code>IndexWriter</code> instance as
156   this may cause deadlock; use your own (non-Lucene) objects
157   instead. </p>
158   
159   <p><b>NOTE</b>: If you call
160   <code>Thread.interrupt()</code> on a thread that's within
161   IndexWriter, IndexWriter will try to catch this (eg, if
162   it's in a wait() or Thread.sleep()), and will then throw
163   the unchecked exception {@link ThreadInterruptedException}
164   and <b>clear</b> the interrupt status on the thread.</p>
165 */
166
167 /*
168  * Clarification: Check Points (and commits)
169  * IndexWriter writes new index files to the directory without writing a new segments_N
170  * file which references these new files. It also means that the state of 
171  * the in memory SegmentInfos object is different than the most recent
172  * segments_N file written to the directory.
173  * 
174  * Each time the SegmentInfos is changed, and matches the (possibly 
175  * modified) directory files, we have a new "check point". 
176  * If the modified/new SegmentInfos is written to disk - as a new 
177  * (generation of) segments_N file - this check point is also an 
178  * IndexCommit.
179  * 
180  * A new checkpoint always replaces the previous checkpoint and 
181  * becomes the new "front" of the index. This allows the IndexFileDeleter 
182  * to delete files that are referenced only by stale checkpoints.
183  * (files that were created since the last commit, but are no longer
184  * referenced by the "front" of the index). For this, IndexFileDeleter 
185  * keeps track of the last non commit checkpoint.
186  */
187 public class IndexWriter implements Closeable, TwoPhaseCommit {
188
189   /**
190    * Default value for the write lock timeout (1,000).
191    * @see #setDefaultWriteLockTimeout
192    * @deprecated use {@link IndexWriterConfig#WRITE_LOCK_TIMEOUT} instead
193    */
194   @Deprecated
195   public static long WRITE_LOCK_TIMEOUT = IndexWriterConfig.WRITE_LOCK_TIMEOUT;
196
197   private long writeLockTimeout;
198
199   /**
200    * Name of the write lock in the index.
201    */
202   public static final String WRITE_LOCK_NAME = "write.lock";
203
204   /**
205    * Value to denote a flush trigger is disabled
206    * @deprecated use {@link IndexWriterConfig#DISABLE_AUTO_FLUSH} instead
207    */
208   @Deprecated
209   public final static int DISABLE_AUTO_FLUSH = IndexWriterConfig.DISABLE_AUTO_FLUSH;
210
211   /**
212    * Disabled by default (because IndexWriter flushes by RAM usage
213    * by default). Change using {@link #setMaxBufferedDocs(int)}.
214    * @deprecated use {@link IndexWriterConfig#DEFAULT_MAX_BUFFERED_DOCS} instead.
215    */
216   @Deprecated
217   public final static int DEFAULT_MAX_BUFFERED_DOCS = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS;
218
219   /**
220    * Default value is 16 MB (which means flush when buffered
221    * docs consume 16 MB RAM).  Change using {@link #setRAMBufferSizeMB}.
222    * @deprecated use {@link IndexWriterConfig#DEFAULT_RAM_BUFFER_SIZE_MB} instead.
223    */
224   @Deprecated
225   public final static double DEFAULT_RAM_BUFFER_SIZE_MB = IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB;
226
227   /**
228    * Disabled by default (because IndexWriter flushes by RAM usage
229    * by default). Change using {@link #setMaxBufferedDeleteTerms(int)}.
230    * @deprecated use {@link IndexWriterConfig#DEFAULT_MAX_BUFFERED_DELETE_TERMS} instead
231    */
232   @Deprecated
233   public final static int DEFAULT_MAX_BUFFERED_DELETE_TERMS = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DELETE_TERMS;
234
235   /**
236    * Default value is 10,000. Change using {@link #setMaxFieldLength(int)}.
237    * 
238    * @deprecated see {@link IndexWriterConfig}
239    */
240   @Deprecated
241   public final static int DEFAULT_MAX_FIELD_LENGTH = MaxFieldLength.UNLIMITED.getLimit();
242
243   /**
244    * Default value is 128. Change using {@link #setTermIndexInterval(int)}.
245    * @deprecated use {@link IndexWriterConfig#DEFAULT_TERM_INDEX_INTERVAL} instead.
246    */
247   @Deprecated
248   public final static int DEFAULT_TERM_INDEX_INTERVAL = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL;
249
250   /**
251    * Absolute hard maximum length for a term.  If a term
252    * arrives from the analyzer longer than this length, it
253    * is skipped and a message is printed to infoStream, if
254    * set (see {@link #setInfoStream}).
255    */
256   public final static int MAX_TERM_LENGTH = DocumentsWriter.MAX_TERM_LENGTH;
257
258   // The normal read buffer size defaults to 1024, but
259   // increasing this during merging seems to yield
260   // performance gains.  However we don't want to increase
261   // it too much because there are quite a few
262   // BufferedIndexInputs created during merging.  See
263   // LUCENE-888 for details.
264   private final static int MERGE_READ_BUFFER_SIZE = 4096;
265
266   // Used for printing messages
267   private static final AtomicInteger MESSAGE_ID = new AtomicInteger();
268   private int messageID = MESSAGE_ID.getAndIncrement();
269   volatile private boolean hitOOM;
270
271   private final Directory directory;  // where this index resides
272   private final Analyzer analyzer;    // how to analyze text
273
274   // TODO 4.0: this should be made final once the setter is out
275   private /*final*/Similarity similarity = Similarity.getDefault(); // how to normalize
276
277   private volatile long changeCount; // increments every time a change is completed
278   private long lastCommitChangeCount; // last changeCount that was committed
279
280   private List<SegmentInfo> rollbackSegments;      // list of segmentInfo we will fallback to if the commit fails
281
282   volatile SegmentInfos pendingCommit;            // set when a commit is pending (after prepareCommit() & before commit())
283   volatile long pendingCommitChangeCount;
284
285   final SegmentInfos segmentInfos = new SegmentInfos();       // the segments
286
287   private DocumentsWriter docWriter;
288   private IndexFileDeleter deleter;
289
290   private Map<SegmentInfo,Boolean> segmentsToOptimize = new HashMap<SegmentInfo,Boolean>();           // used by optimize to note those needing optimization
291   private int optimizeMaxNumSegments;
292
293   private Lock writeLock;
294
295   private boolean closed;
296   private boolean closing;
297
298   // Holds all SegmentInfo instances currently involved in
299   // merges
300   private HashSet<SegmentInfo> mergingSegments = new HashSet<SegmentInfo>();
301
302   private MergePolicy mergePolicy;
303   // TODO 4.0: this should be made final once the setter is removed
304   private /*final*/MergeScheduler mergeScheduler;
305   private LinkedList<MergePolicy.OneMerge> pendingMerges = new LinkedList<MergePolicy.OneMerge>();
306   private Set<MergePolicy.OneMerge> runningMerges = new HashSet<MergePolicy.OneMerge>();
307   private List<MergePolicy.OneMerge> mergeExceptions = new ArrayList<MergePolicy.OneMerge>();
308   private long mergeGen;
309   private boolean stopMerges;
310
311   private final AtomicInteger flushCount = new AtomicInteger();
312   private final AtomicInteger flushDeletesCount = new AtomicInteger();
313
314   final ReaderPool readerPool = new ReaderPool();
315   final BufferedDeletesStream bufferedDeletesStream;
316   
317   // This is a "write once" variable (like the organic dye
318   // on a DVD-R that may or may not be heated by a laser and
319   // then cooled to permanently record the event): it's
320   // false, until getReader() is called for the first time,
321   // at which point it's switched to true and never changes
322   // back to false.  Once this is true, we hold open and
323   // reuse SegmentReader instances internally for applying
324   // deletes, doing merges, and reopening near real-time
325   // readers.
326   private volatile boolean poolReaders;
327
328   // The instance that was passed to the constructor. It is saved only in order
329   // to allow users to query an IndexWriter settings.
330   private final IndexWriterConfig config;
331
332   // The PayloadProcessorProvider to use when segments are merged
333   private PayloadProcessorProvider payloadProcessorProvider;
334
335   // for testing
336   boolean anyNonBulkMerges;
337
338   /**
339    * Expert: returns a readonly reader, covering all
340    * committed as well as un-committed changes to the index.
341    * This provides "near real-time" searching, in that
342    * changes made during an IndexWriter session can be
343    * quickly made available for searching without closing
344    * the writer nor calling {@link #commit}.
345    *
346    * <p>Note that this is functionally equivalent to calling
347    * {#flush} and then using {@link IndexReader#open} to
348    * open a new reader.  But the turarnound time of this
349    * method should be faster since it avoids the potentially
350    * costly {@link #commit}.</p>
351    *
352    * <p>You must close the {@link IndexReader} returned by
353    * this method once you are done using it.</p>
354    *
355    * <p>It's <i>near</i> real-time because there is no hard
356    * guarantee on how quickly you can get a new reader after
357    * making changes with IndexWriter.  You'll have to
358    * experiment in your situation to determine if it's
359    * fast enough.  As this is a new and experimental
360    * feature, please report back on your findings so we can
361    * learn, improve and iterate.</p>
362    *
363    * <p>The resulting reader supports {@link
364    * IndexReader#reopen}, but that call will simply forward
365    * back to this method (though this may change in the
366    * future).</p>
367    *
368    * <p>The very first time this method is called, this
369    * writer instance will make every effort to pool the
370    * readers that it opens for doing merges, applying
371    * deletes, etc.  This means additional resources (RAM,
372    * file descriptors, CPU time) will be consumed.</p>
373    *
374    * <p>For lower latency on reopening a reader, you should
375    * call {@link #setMergedSegmentWarmer} to
376    * pre-warm a newly merged segment before it's committed
377    * to the index.  This is important for minimizing
378    * index-to-search delay after a large merge.  </p>
379    *
380    * <p>If an addIndexes* call is running in another thread,
381    * then this reader will only search those segments from
382    * the foreign index that have been successfully copied
383    * over, so far</p>.
384    *
385    * <p><b>NOTE</b>: Once the writer is closed, any
386    * outstanding readers may continue to be used.  However,
387    * if you attempt to reopen any of those readers, you'll
388    * hit an {@link AlreadyClosedException}.</p>
389    *
390    * @lucene.experimental
391    *
392    * @return IndexReader that covers entire index plus all
393    * changes made so far by this IndexWriter instance
394    *
395    * @deprecated Please use {@link
396    * IndexReader#open(IndexWriter,boolean)} instead.
397    *
398    * @throws IOException
399    */
400   @Deprecated
401   public IndexReader getReader() throws IOException {
402     return getReader(config.getReaderTermsIndexDivisor(), true);
403   }
404
405   IndexReader getReader(boolean applyAllDeletes) throws IOException {
406     return getReader(config.getReaderTermsIndexDivisor(), applyAllDeletes);
407   }
408
409   /** Expert: like {@link #getReader}, except you can
410    *  specify which termInfosIndexDivisor should be used for
411    *  any newly opened readers.
412    * @param termInfosIndexDivisor Subsamples which indexed
413    *  terms are loaded into RAM. This has the same effect as {@link
414    *  IndexWriter#setTermIndexInterval} except that setting
415    *  must be done at indexing time while this setting can be
416    *  set per reader.  When set to N, then one in every
417    *  N*termIndexInterval terms in the index is loaded into
418    *  memory.  By setting this to a value > 1 you can reduce
419    *  memory usage, at the expense of higher latency when
420    *  loading a TermInfo.  The default value is 1.  Set this
421    *  to -1 to skip loading the terms index entirely.
422    *  
423    *  @deprecated Please use {@link
424    *  IndexReader#open(IndexWriter,boolean)} instead.  Furthermore,
425    *  this method cannot guarantee the reader (and its
426    *  sub-readers) will be opened with the
427    *  termInfosIndexDivisor setting because some of them may
428    *  have already been opened according to {@link
429    *  IndexWriterConfig#setReaderTermsIndexDivisor}. You
430    *  should set the requested termInfosIndexDivisor through 
431    *  {@link IndexWriterConfig#setReaderTermsIndexDivisor} and use 
432    *  {@link #getReader()}. */
433   @Deprecated
434   public IndexReader getReader(int termInfosIndexDivisor) throws IOException {
435     return getReader(termInfosIndexDivisor, true);
436   }
437
438   IndexReader getReader(int termInfosIndexDivisor, boolean applyAllDeletes) throws IOException {
439     ensureOpen();
440     
441     final long tStart = System.currentTimeMillis();
442
443     if (infoStream != null) {
444       message("flush at getReader");
445     }
446
447     // Do this up front before flushing so that the readers
448     // obtained during this flush are pooled, the first time
449     // this method is called:
450     poolReaders = true;
451
452     // Prevent segmentInfos from changing while opening the
453     // reader; in theory we could do similar retry logic,
454     // just like we do when loading segments_N
455     IndexReader r;
456     synchronized(this) {
457       flush(false, applyAllDeletes);
458       r = new ReadOnlyDirectoryReader(this, segmentInfos, termInfosIndexDivisor, applyAllDeletes);
459       if (infoStream != null) {
460         message("return reader version=" + r.getVersion() + " reader=" + r);
461       }
462     }
463
464     maybeMerge();
465
466     if (infoStream != null) {
467       message("getReader took " + (System.currentTimeMillis() - tStart) + " msec");
468     }
469     return r;
470   }
471
472   // Used for all SegmentReaders we open
473   private final Collection<IndexReader.ReaderFinishedListener> readerFinishedListeners = new MapBackedSet<IndexReader.ReaderFinishedListener>(new ConcurrentHashMap<IndexReader.ReaderFinishedListener,Boolean>());
474
475   Collection<IndexReader.ReaderFinishedListener> getReaderFinishedListeners() throws IOException {
476     return readerFinishedListeners;
477   }
478
479   /** Holds shared SegmentReader instances. IndexWriter uses
480    *  SegmentReaders for 1) applying deletes, 2) doing
481    *  merges, 3) handing out a real-time reader.  This pool
482    *  reuses instances of the SegmentReaders in all these
483    *  places if it is in "near real-time mode" (getReader()
484    *  has been called on this instance). */
485
486   class ReaderPool {
487
488     private final Map<SegmentInfo,SegmentReader> readerMap = new HashMap<SegmentInfo,SegmentReader>();
489
490     /** Forcefully clear changes for the specified segments.  This is called on successful merge. */
491     synchronized void clear(List<SegmentInfo> infos) throws IOException {
492       if (infos == null) {
493         for (Map.Entry<SegmentInfo,SegmentReader> ent: readerMap.entrySet()) {
494           ent.getValue().hasChanges = false;
495         }
496       } else {
497         for (final SegmentInfo info: infos) {
498           final SegmentReader r = readerMap.get(info);
499           if (r != null) {
500             r.hasChanges = false;
501           }
502         }     
503       }
504     }
505     
506     // used only by asserts
507     public synchronized boolean infoIsLive(SegmentInfo info) {
508       int idx = segmentInfos.indexOf(info);
509       assert idx != -1: "info=" + info + " isn't in pool";
510       assert segmentInfos.info(idx) == info: "info=" + info + " doesn't match live info in segmentInfos";
511       return true;
512     }
513
514     public synchronized SegmentInfo mapToLive(SegmentInfo info) {
515       int idx = segmentInfos.indexOf(info);
516       if (idx != -1) {
517         info = segmentInfos.info(idx);
518       }
519       return info;
520     }
521     
522     /**
523      * Release the segment reader (i.e. decRef it and close if there
524      * are no more references.
525      * @return true if this release altered the index (eg
526      * the SegmentReader had pending changes to del docs and
527      * was closed).  Caller must call checkpoint() if so.
528      * @param sr
529      * @throws IOException
530      */
531     public synchronized boolean release(SegmentReader sr) throws IOException {
532       return release(sr, false);
533     }
534     
535     /**
536      * Release the segment reader (i.e. decRef it and close if there
537      * are no more references.
538      * @return true if this release altered the index (eg
539      * the SegmentReader had pending changes to del docs and
540      * was closed).  Caller must call checkpoint() if so.
541      * @param sr
542      * @throws IOException
543      */
544     public synchronized boolean release(SegmentReader sr, boolean drop) throws IOException {
545
546       final boolean pooled = readerMap.containsKey(sr.getSegmentInfo());
547
548       assert !pooled || readerMap.get(sr.getSegmentInfo()) == sr;
549
550       // Drop caller's ref; for an external reader (not
551       // pooled), this decRef will close it
552       sr.decRef();
553
554       if (pooled && (drop || (!poolReaders && sr.getRefCount() == 1))) {
555
556         // We invoke deleter.checkpoint below, so we must be
557         // sync'd on IW if there are changes:
558         assert !sr.hasChanges || Thread.holdsLock(IndexWriter.this);
559
560         // Discard (don't save) changes when we are dropping
561         // the reader; this is used only on the sub-readers
562         // after a successful merge.
563         sr.hasChanges &= !drop;
564
565         final boolean hasChanges = sr.hasChanges;
566
567         // Drop our ref -- this will commit any pending
568         // changes to the dir
569         sr.close();
570
571         // We are the last ref to this reader; since we're
572         // not pooling readers, we release it:
573         readerMap.remove(sr.getSegmentInfo());
574
575         return hasChanges;
576       }
577
578       return false;
579     }
580
581     public synchronized void drop(List<SegmentInfo> infos) throws IOException {
582       for(SegmentInfo info : infos) {
583         drop(info);
584       }
585     }
586
587     public synchronized void drop(SegmentInfo info) throws IOException {
588       final SegmentReader sr = readerMap.get(info);
589       if (sr != null) {
590         sr.hasChanges = false;
591         readerMap.remove(info);
592         sr.close();
593       }
594     }
595     
596     public synchronized void dropAll() throws IOException {
597       for(SegmentReader reader : readerMap.values()) {
598         reader.hasChanges = false;
599
600         // NOTE: it is allowed that this decRef does not
601         // actually close the SR; this can happen when a
602         // near real-time reader using this SR is still open
603         reader.decRef();
604       }
605       readerMap.clear();
606     }
607
608     /** Remove all our references to readers, and commits
609      *  any pending changes. */
610     synchronized void close() throws IOException {
611       // We invoke deleter.checkpoint below, so we must be
612       // sync'd on IW:
613       assert Thread.holdsLock(IndexWriter.this);
614
615       for(Map.Entry<SegmentInfo,SegmentReader> ent : readerMap.entrySet()) {
616         
617         SegmentReader sr = ent.getValue();
618         if (sr.hasChanges) {
619           assert infoIsLive(sr.getSegmentInfo());
620           sr.doCommit(null);
621
622           // Must checkpoint w/ deleter, because this
623           // segment reader will have created new _X_N.del
624           // file.
625           deleter.checkpoint(segmentInfos, false);
626         }
627
628         // NOTE: it is allowed that this decRef does not
629         // actually close the SR; this can happen when a
630         // near real-time reader is kept open after the
631         // IndexWriter instance is closed
632         sr.decRef();
633       }
634
635       readerMap.clear();
636     }
637     
638     /**
639      * Commit all segment reader in the pool.
640      * @throws IOException
641      */
642     synchronized void commit(SegmentInfos infos) throws IOException {
643
644       // We invoke deleter.checkpoint below, so we must be
645       // sync'd on IW:
646       assert Thread.holdsLock(IndexWriter.this);
647
648       for (SegmentInfo info : infos) {
649
650         final SegmentReader sr = readerMap.get(info);
651         if (sr != null && sr.hasChanges) {
652           assert infoIsLive(info);
653           sr.doCommit(null);
654           // Must checkpoint w/ deleter, because this
655           // segment reader will have created new _X_N.del
656           // file.
657           deleter.checkpoint(segmentInfos, false);
658         }
659       }
660     }
661     
662     /**
663      * Returns a ref to a clone.  NOTE: this clone is not
664      * enrolled in the pool, so you should simply close()
665      * it when you're done (ie, do not call release()).
666      */
667     public synchronized SegmentReader getReadOnlyClone(SegmentInfo info, boolean doOpenStores, int termInfosIndexDivisor) throws IOException {
668       SegmentReader sr = get(info, doOpenStores, BufferedIndexInput.BUFFER_SIZE, termInfosIndexDivisor);
669       try {
670         return (SegmentReader) sr.clone(true);
671       } finally {
672         sr.decRef();
673       }
674     }
675    
676     /**
677      * Obtain a SegmentReader from the readerPool.  The reader
678      * must be returned by calling {@link #release(SegmentReader)}
679      * @see #release(SegmentReader)
680      * @param info
681      * @param doOpenStores
682      * @throws IOException
683      */
684     public synchronized SegmentReader get(SegmentInfo info, boolean doOpenStores) throws IOException {
685       return get(info, doOpenStores, BufferedIndexInput.BUFFER_SIZE, config.getReaderTermsIndexDivisor());
686     }
687
688     /**
689      * Obtain a SegmentReader from the readerPool.  The reader
690      * must be returned by calling {@link #release(SegmentReader)}
691      * 
692      * @see #release(SegmentReader)
693      * @param info
694      * @param doOpenStores
695      * @param readBufferSize
696      * @param termsIndexDivisor
697      * @throws IOException
698      */
699     public synchronized SegmentReader get(SegmentInfo info, boolean doOpenStores, int readBufferSize, int termsIndexDivisor) throws IOException {
700
701       if (poolReaders) {
702         readBufferSize = BufferedIndexInput.BUFFER_SIZE;
703       }
704
705       SegmentReader sr = readerMap.get(info);
706       if (sr == null) {
707         // TODO: we may want to avoid doing this while
708         // synchronized
709         // Returns a ref, which we xfer to readerMap:
710         sr = SegmentReader.get(false, info.dir, info, readBufferSize, doOpenStores, termsIndexDivisor);
711         sr.readerFinishedListeners = readerFinishedListeners;
712
713         if (info.dir == directory) {
714           // Only pool if reader is not external
715           readerMap.put(info, sr);
716         }
717       } else {
718         if (doOpenStores) {
719           sr.openDocStores();
720         }
721         if (termsIndexDivisor != -1 && !sr.termsIndexLoaded()) {
722           // If this reader was originally opened because we
723           // needed to merge it, we didn't load the terms
724           // index.  But now, if the caller wants the terms
725           // index (eg because it's doing deletes, or an NRT
726           // reader is being opened) we ask the reader to
727           // load its terms index.
728           sr.loadTermsIndex(termsIndexDivisor);
729         }
730       }
731
732       // Return a ref to our caller
733       if (info.dir == directory) {
734         // Only incRef if we pooled (reader is not external)
735         sr.incRef();
736       }
737       return sr;
738     }
739
740     // Returns a ref
741     public synchronized SegmentReader getIfExists(SegmentInfo info) throws IOException {
742       SegmentReader sr = readerMap.get(info);
743       if (sr != null) {
744         sr.incRef();
745       }
746       return sr;
747     }
748   }
749   
750   
751   
752   /**
753    * Obtain the number of deleted docs for a pooled reader.
754    * If the reader isn't being pooled, the segmentInfo's 
755    * delCount is returned.
756    */
757   public int numDeletedDocs(SegmentInfo info) throws IOException {
758     SegmentReader reader = readerPool.getIfExists(info);
759     try {
760       if (reader != null) {
761         return reader.numDeletedDocs();
762       } else {
763         return info.getDelCount();
764       }
765     } finally {
766       if (reader != null) {
767         readerPool.release(reader);
768       }
769     }
770   }
771   
772   /**
773    * Used internally to throw an {@link
774    * AlreadyClosedException} if this IndexWriter has been
775    * closed.
776    * @throws AlreadyClosedException if this IndexWriter is closed
777    */
778   protected final void ensureOpen(boolean includePendingClose) throws AlreadyClosedException {
779     if (closed || (includePendingClose && closing)) {
780       throw new AlreadyClosedException("this IndexWriter is closed");
781     }
782   }
783
784   protected final void ensureOpen() throws AlreadyClosedException {
785     ensureOpen(true);
786   }
787
788   /**
789    * Prints a message to the infoStream (if non-null),
790    * prefixed with the identifying information for this
791    * writer and the thread that's calling it.
792    */
793   public void message(String message) {
794     if (infoStream != null)
795       infoStream.println("IW " + messageID + " [" + new Date() + "; " + Thread.currentThread().getName() + "]: " + message);
796   }
797
798   /**
799    * Casts current mergePolicy to LogMergePolicy, and throws
800    * an exception if the mergePolicy is not a LogMergePolicy.
801    */
802   private LogMergePolicy getLogMergePolicy() {
803     if (mergePolicy instanceof LogMergePolicy)
804       return (LogMergePolicy) mergePolicy;
805     else
806       throw new IllegalArgumentException("this method can only be called when the merge policy is the default LogMergePolicy");
807   }
808
809   /** <p>Get the current setting of whether newly flushed
810    *  segments will use the compound file format.  Note that
811    *  this just returns the value previously set with
812    *  setUseCompoundFile(boolean), or the default value
813    *  (true).  You cannot use this to query the status of
814    *  previously flushed segments.</p>
815    *
816    *  <p>Note that this method is a convenience method: it
817    *  just calls mergePolicy.getUseCompoundFile as long as
818    *  mergePolicy is an instance of {@link LogMergePolicy}.
819    *  Otherwise an IllegalArgumentException is thrown.</p>
820    *
821    *  @see #setUseCompoundFile(boolean)
822    *  @deprecated use {@link LogMergePolicy#getUseCompoundFile()}
823    */
824   @Deprecated
825   public boolean getUseCompoundFile() {
826     return getLogMergePolicy().getUseCompoundFile();
827   }
828
829   /**
830    * <p>
831    * Setting to turn on usage of a compound file. When on, multiple files for
832    * each segment are merged into a single file when a new segment is flushed.
833    * </p>
834    * 
835    * <p>
836    * Note that this method is a convenience method: it just calls
837    * mergePolicy.setUseCompoundFile as long as mergePolicy is an instance of
838    * {@link LogMergePolicy}. Otherwise an IllegalArgumentException is thrown.
839    * </p>
840    * 
841    * @deprecated use {@link LogMergePolicy#setUseCompoundFile(boolean)}.
842    */
843   @Deprecated
844   public void setUseCompoundFile(boolean value) {
845     getLogMergePolicy().setUseCompoundFile(value);
846   }
847
848   /** Expert: Set the Similarity implementation used by this IndexWriter.
849    *
850    * @see Similarity#setDefault(Similarity)
851    * @deprecated use {@link IndexWriterConfig#setSimilarity(Similarity)} instead
852    */
853   @Deprecated
854   public void setSimilarity(Similarity similarity) {
855     ensureOpen();
856     this.similarity = similarity;
857     docWriter.setSimilarity(similarity);
858     // Required so config.getSimilarity returns the right value. But this will
859     // go away together with the method in 4.0.
860     config.setSimilarity(similarity);
861   }
862
863   /** Expert: Return the Similarity implementation used by this IndexWriter.
864    *
865    * <p>This defaults to the current value of {@link Similarity#getDefault()}.
866    * @deprecated use {@link IndexWriterConfig#getSimilarity()} instead
867    */
868   @Deprecated
869   public Similarity getSimilarity() {
870     ensureOpen();
871     return similarity;
872   }
873
874   /** Expert: Set the interval between indexed terms.  Large values cause less
875    * memory to be used by IndexReader, but slow random-access to terms.  Small
876    * values cause more memory to be used by an IndexReader, and speed
877    * random-access to terms.
878    *
879    * This parameter determines the amount of computation required per query
880    * term, regardless of the number of documents that contain that term.  In
881    * particular, it is the maximum number of other terms that must be
882    * scanned before a term is located and its frequency and position information
883    * may be processed.  In a large index with user-entered query terms, query
884    * processing time is likely to be dominated not by term lookup but rather
885    * by the processing of frequency and positional data.  In a small index
886    * or when many uncommon query terms are generated (e.g., by wildcard
887    * queries) term lookup may become a dominant cost.
888    *
889    * In particular, <code>numUniqueTerms/interval</code> terms are read into
890    * memory by an IndexReader, and, on average, <code>interval/2</code> terms
891    * must be scanned for each random term access.
892    *
893    * @see #DEFAULT_TERM_INDEX_INTERVAL
894    * @deprecated use {@link IndexWriterConfig#setTermIndexInterval(int)}
895    */
896   @Deprecated
897   public void setTermIndexInterval(int interval) {
898     ensureOpen();
899     config.setTermIndexInterval(interval);
900   }
901
902   /** Expert: Return the interval between indexed terms.
903    *
904    * @see #setTermIndexInterval(int)
905    * @deprecated use {@link IndexWriterConfig#getTermIndexInterval()}
906    */
907   @Deprecated
908   public int getTermIndexInterval() {
909     // We pass false because this method is called by SegmentMerger while we are in the process of closing
910     ensureOpen(false);
911     return config.getTermIndexInterval();
912   }
913
914   /**
915    * Constructs an IndexWriter for the index in <code>d</code>.
916    * Text will be analyzed with <code>a</code>.  If <code>create</code>
917    * is true, then a new, empty index will be created in
918    * <code>d</code>, replacing the index already there, if any.
919    *
920    * @param d the index directory
921    * @param a the analyzer to use
922    * @param create <code>true</code> to create the index or overwrite
923    *  the existing one; <code>false</code> to append to the existing
924    *  index
925    * @param mfl Maximum field length in number of terms/tokens: LIMITED, UNLIMITED, or user-specified
926    *   via the MaxFieldLength constructor.
927    * @throws CorruptIndexException if the index is corrupt
928    * @throws LockObtainFailedException if another writer
929    *  has this index open (<code>write.lock</code> could not
930    *  be obtained)
931    * @throws IOException if the directory cannot be read/written to, or
932    *  if it does not exist and <code>create</code> is
933    *  <code>false</code> or if there is any other low-level
934    *  IO error
935    *  @deprecated use {@link #IndexWriter(Directory, IndexWriterConfig)} instead
936    */
937   @Deprecated
938   public IndexWriter(Directory d, Analyzer a, boolean create, MaxFieldLength mfl)
939        throws CorruptIndexException, LockObtainFailedException, IOException {
940     this(d, new IndexWriterConfig(Version.LUCENE_31, a).setOpenMode(
941         create ? OpenMode.CREATE : OpenMode.APPEND));
942     setMaxFieldLength(mfl.getLimit());
943   }
944
945   /**
946    * Constructs an IndexWriter for the index in
947    * <code>d</code>, first creating it if it does not
948    * already exist.  Text will be analyzed with
949    * <code>a</code>.
950    *
951    * @param d the index directory
952    * @param a the analyzer to use
953    * @param mfl Maximum field length in number of terms/tokens: LIMITED, UNLIMITED, or user-specified
954    *   via the MaxFieldLength constructor.
955    * @throws CorruptIndexException if the index is corrupt
956    * @throws LockObtainFailedException if another writer
957    *  has this index open (<code>write.lock</code> could not
958    *  be obtained)
959    * @throws IOException if the directory cannot be
960    *  read/written to or if there is any other low-level
961    *  IO error
962    *  @deprecated use {@link #IndexWriter(Directory, IndexWriterConfig)} instead
963    */
964   @Deprecated
965   public IndexWriter(Directory d, Analyzer a, MaxFieldLength mfl)
966     throws CorruptIndexException, LockObtainFailedException, IOException {
967     this(d, new IndexWriterConfig(Version.LUCENE_31, a));
968     setMaxFieldLength(mfl.getLimit());
969   }
970
971   /**
972    * Expert: constructs an IndexWriter with a custom {@link
973    * IndexDeletionPolicy}, for the index in <code>d</code>,
974    * first creating it if it does not already exist.  Text
975    * will be analyzed with <code>a</code>.
976    *
977    * @param d the index directory
978    * @param a the analyzer to use
979    * @param deletionPolicy see <a href="#deletionPolicy">above</a>
980    * @param mfl whether or not to limit field lengths
981    * @throws CorruptIndexException if the index is corrupt
982    * @throws LockObtainFailedException if another writer
983    *  has this index open (<code>write.lock</code> could not
984    *  be obtained)
985    * @throws IOException if the directory cannot be
986    *  read/written to or if there is any other low-level
987    *  IO error
988    *  @deprecated use {@link #IndexWriter(Directory, IndexWriterConfig)} instead
989    */
990   @Deprecated
991   public IndexWriter(Directory d, Analyzer a, IndexDeletionPolicy deletionPolicy, MaxFieldLength mfl)
992     throws CorruptIndexException, LockObtainFailedException, IOException {
993     this(d, new IndexWriterConfig(Version.LUCENE_31, a).setIndexDeletionPolicy(deletionPolicy));
994     setMaxFieldLength(mfl.getLimit());
995   }
996
997   /**
998    * Expert: constructs an IndexWriter with a custom {@link
999    * IndexDeletionPolicy}, for the index in <code>d</code>.
1000    * Text will be analyzed with <code>a</code>.  If
1001    * <code>create</code> is true, then a new, empty index
1002    * will be created in <code>d</code>, replacing the index
1003    * already there, if any.
1004    *
1005    * @param d the index directory
1006    * @param a the analyzer to use
1007    * @param create <code>true</code> to create the index or overwrite
1008    *  the existing one; <code>false</code> to append to the existing
1009    *  index
1010    * @param deletionPolicy see <a href="#deletionPolicy">above</a>
1011    * @param mfl {@link org.apache.lucene.index.IndexWriter.MaxFieldLength}, whether or not to limit field lengths.  Value is in number of terms/tokens
1012    * @throws CorruptIndexException if the index is corrupt
1013    * @throws LockObtainFailedException if another writer
1014    *  has this index open (<code>write.lock</code> could not
1015    *  be obtained)
1016    * @throws IOException if the directory cannot be read/written to, or
1017    *  if it does not exist and <code>create</code> is
1018    *  <code>false</code> or if there is any other low-level
1019    *  IO error
1020    *  @deprecated use {@link #IndexWriter(Directory, IndexWriterConfig)} instead
1021    */
1022   @Deprecated
1023   public IndexWriter(Directory d, Analyzer a, boolean create, IndexDeletionPolicy deletionPolicy, MaxFieldLength mfl)
1024        throws CorruptIndexException, LockObtainFailedException, IOException {
1025     this(d, new IndexWriterConfig(Version.LUCENE_31, a).setOpenMode(
1026         create ? OpenMode.CREATE : OpenMode.APPEND).setIndexDeletionPolicy(deletionPolicy));
1027     setMaxFieldLength(mfl.getLimit());
1028   }
1029   
1030   /**
1031    * Expert: constructs an IndexWriter on specific commit
1032    * point, with a custom {@link IndexDeletionPolicy}, for
1033    * the index in <code>d</code>.  Text will be analyzed
1034    * with <code>a</code>.
1035    *
1036    * <p> This is only meaningful if you've used a {@link
1037    * IndexDeletionPolicy} in that past that keeps more than
1038    * just the last commit.
1039    * 
1040    * <p>This operation is similar to {@link #rollback()},
1041    * except that method can only rollback what's been done
1042    * with the current instance of IndexWriter since its last
1043    * commit, whereas this method can rollback to an
1044    * arbitrary commit point from the past, assuming the
1045    * {@link IndexDeletionPolicy} has preserved past
1046    * commits.
1047    *
1048    * @param d the index directory
1049    * @param a the analyzer to use
1050    * @param deletionPolicy see <a href="#deletionPolicy">above</a>
1051    * @param mfl whether or not to limit field lengths, value is in number of terms/tokens.  See {@link org.apache.lucene.index.IndexWriter.MaxFieldLength}.
1052    * @param commit which commit to open
1053    * @throws CorruptIndexException if the index is corrupt
1054    * @throws LockObtainFailedException if another writer
1055    *  has this index open (<code>write.lock</code> could not
1056    *  be obtained)
1057    * @throws IOException if the directory cannot be read/written to, or
1058    *  if it does not exist and <code>create</code> is
1059    *  <code>false</code> or if there is any other low-level
1060    *  IO error
1061    *  @deprecated use {@link #IndexWriter(Directory, IndexWriterConfig)} instead
1062    */
1063   @Deprecated
1064   public IndexWriter(Directory d, Analyzer a, IndexDeletionPolicy deletionPolicy, MaxFieldLength mfl, IndexCommit commit)
1065        throws CorruptIndexException, LockObtainFailedException, IOException {
1066     this(d, new IndexWriterConfig(Version.LUCENE_31, a)
1067         .setOpenMode(OpenMode.APPEND).setIndexDeletionPolicy(deletionPolicy).setIndexCommit(commit));
1068     setMaxFieldLength(mfl.getLimit());
1069   }
1070
1071   /**
1072    * Constructs a new IndexWriter per the settings given in <code>conf</code>.
1073    * Note that the passed in {@link IndexWriterConfig} is
1074    * privately cloned; if you need to make subsequent "live"
1075    * changes to the configuration use {@link #getConfig}.
1076    * <p>
1077    * 
1078    * @param d
1079    *          the index directory. The index is either created or appended
1080    *          according <code>conf.getOpenMode()</code>.
1081    * @param conf
1082    *          the configuration settings according to which IndexWriter should
1083    *          be initialized.
1084    * @throws CorruptIndexException
1085    *           if the index is corrupt
1086    * @throws LockObtainFailedException
1087    *           if another writer has this index open (<code>write.lock</code>
1088    *           could not be obtained)
1089    * @throws IOException
1090    *           if the directory cannot be read/written to, or if it does not
1091    *           exist and <code>conf.getOpenMode()</code> is
1092    *           <code>OpenMode.APPEND</code> or if there is any other low-level
1093    *           IO error
1094    */
1095   public IndexWriter(Directory d, IndexWriterConfig conf)
1096       throws CorruptIndexException, LockObtainFailedException, IOException {
1097     config = (IndexWriterConfig) conf.clone();
1098     directory = d;
1099     analyzer = conf.getAnalyzer();
1100     infoStream = defaultInfoStream;
1101     writeLockTimeout = conf.getWriteLockTimeout();
1102     similarity = conf.getSimilarity();
1103     mergePolicy = conf.getMergePolicy();
1104     mergePolicy.setIndexWriter(this);
1105     mergeScheduler = conf.getMergeScheduler();
1106     bufferedDeletesStream = new BufferedDeletesStream(messageID);
1107     bufferedDeletesStream.setInfoStream(infoStream);
1108     poolReaders = conf.getReaderPooling();
1109
1110     writeLock = directory.makeLock(WRITE_LOCK_NAME);
1111
1112     if (!writeLock.obtain(writeLockTimeout)) // obtain write lock
1113       throw new LockObtainFailedException("Index locked for write: " + writeLock);
1114
1115     OpenMode mode = conf.getOpenMode();
1116     boolean create;
1117     if (mode == OpenMode.CREATE) {
1118       create = true;
1119     } else if (mode == OpenMode.APPEND) {
1120       create = false;
1121     } else {
1122       // CREATE_OR_APPEND - create only if an index does not exist
1123       create = !IndexReader.indexExists(directory);
1124     }
1125     
1126     boolean success = false;
1127
1128     // TODO: we should check whether this index is too old,
1129     // and throw an IndexFormatTooOldExc up front, here,
1130     // instead of later when merge, applyDeletes, getReader
1131     // is attempted.  I think to do this we should store the
1132     // oldest segment's version in segments_N.
1133
1134     try {
1135       if (create) {
1136         // Try to read first.  This is to allow create
1137         // against an index that's currently open for
1138         // searching.  In this case we write the next
1139         // segments_N file with no segments:
1140         try {
1141           segmentInfos.read(directory);
1142           segmentInfos.clear();
1143         } catch (IOException e) {
1144           // Likely this means it's a fresh directory
1145         }
1146
1147         // Record that we have a change (zero out all
1148         // segments) pending:
1149         changeCount++;
1150         segmentInfos.changed();
1151       } else {
1152         segmentInfos.read(directory);
1153
1154         IndexCommit commit = conf.getIndexCommit();
1155         if (commit != null) {
1156           // Swap out all segments, but, keep metadata in
1157           // SegmentInfos, like version & generation, to
1158           // preserve write-once.  This is important if
1159           // readers are open against the future commit
1160           // points.
1161           if (commit.getDirectory() != directory)
1162             throw new IllegalArgumentException("IndexCommit's directory doesn't match my directory");
1163           SegmentInfos oldInfos = new SegmentInfos();
1164           oldInfos.read(directory, commit.getSegmentsFileName());
1165           segmentInfos.replace(oldInfos);
1166           changeCount++;
1167           segmentInfos.changed();
1168           if (infoStream != null)
1169             message("init: loaded commit \"" + commit.getSegmentsFileName() + "\"");
1170         }
1171       }
1172
1173       rollbackSegments = segmentInfos.createBackupSegmentInfos(true);
1174
1175       docWriter = new DocumentsWriter(config, directory, this, getCurrentFieldInfos(), bufferedDeletesStream);
1176       docWriter.setInfoStream(infoStream);
1177       docWriter.setMaxFieldLength(maxFieldLength);
1178
1179       // Default deleter (for backwards compatibility) is
1180       // KeepOnlyLastCommitDeleter:
1181       synchronized(this) {
1182         deleter = new IndexFileDeleter(directory,
1183                                        conf.getIndexDeletionPolicy(),
1184                                        segmentInfos, infoStream,
1185                                        this);
1186       }
1187
1188       if (deleter.startingCommitDeleted) {
1189         // Deletion policy deleted the "head" commit point.
1190         // We have to mark ourself as changed so that if we
1191         // are closed w/o any further changes we write a new
1192         // segments_N file.
1193         changeCount++;
1194         segmentInfos.changed();
1195       }
1196
1197       if (infoStream != null) {
1198         messageState();
1199       }
1200
1201       success = true;
1202
1203     } finally {
1204       if (!success) {
1205         if (infoStream != null) {
1206           message("init: hit exception on init; releasing write lock");
1207         }
1208         try {
1209           writeLock.release();
1210         } catch (Throwable t) {
1211           // don't mask the original exception
1212         }
1213         writeLock = null;
1214       }
1215     }
1216   }
1217
1218   private FieldInfos getFieldInfos(SegmentInfo info) throws IOException {
1219     Directory cfsDir = null;
1220     try {
1221       if (info.getUseCompoundFile()) {
1222         cfsDir = new CompoundFileReader(directory, IndexFileNames.segmentFileName(info.name, IndexFileNames.COMPOUND_FILE_EXTENSION));
1223       } else {
1224         cfsDir = directory;
1225       }
1226       return new FieldInfos(cfsDir, IndexFileNames.segmentFileName(info.name, IndexFileNames.FIELD_INFOS_EXTENSION));
1227     } finally {
1228       if (info.getUseCompoundFile() && cfsDir != null) {
1229         cfsDir.close();
1230       }
1231     }
1232   }
1233
1234   private FieldInfos getCurrentFieldInfos() throws IOException {
1235     final FieldInfos fieldInfos;
1236     if (segmentInfos.size() > 0) {
1237       if (segmentInfos.getFormat() > SegmentInfos.FORMAT_DIAGNOSTICS) {
1238         // Pre-3.1 index.  In this case we sweep all
1239         // segments, merging their FieldInfos:
1240         fieldInfos = new FieldInfos();
1241         for(SegmentInfo info : segmentInfos) {
1242           final FieldInfos segFieldInfos = getFieldInfos(info);
1243           final int fieldCount = segFieldInfos.size();
1244           for(int fieldNumber=0;fieldNumber<fieldCount;fieldNumber++) {
1245             fieldInfos.add(segFieldInfos.fieldInfo(fieldNumber));
1246           }
1247         }
1248       } else {
1249         // Already a 3.1 index; just seed the FieldInfos
1250         // from the last segment
1251         fieldInfos = getFieldInfos(segmentInfos.info(segmentInfos.size()-1));
1252       }
1253     } else {
1254       fieldInfos = new FieldInfos();
1255     }
1256     return fieldInfos;
1257   }
1258
1259   /**
1260    * Returns the private {@link IndexWriterConfig}, cloned
1261    * from the {@link IndexWriterConfig} passed to
1262    * {@link #IndexWriter(Directory, IndexWriterConfig)}.
1263    * <p>
1264    * <b>NOTE:</b> some settings may be changed on the
1265    * returned {@link IndexWriterConfig}, and will take
1266    * effect in the current IndexWriter instance.  See the
1267    * javadocs for the specific setters in {@link
1268    * IndexWriterConfig} for details.
1269    */
1270   public IndexWriterConfig getConfig() {
1271     return config;
1272   }
1273   
1274   /**
1275    * Expert: set the merge policy used by this writer.
1276    * 
1277    * @deprecated use {@link IndexWriterConfig#setMergePolicy(MergePolicy)} instead.
1278    */
1279   @Deprecated
1280   public void setMergePolicy(MergePolicy mp) {
1281     ensureOpen();
1282     if (mp == null)
1283       throw new NullPointerException("MergePolicy must be non-null");
1284
1285     if (mergePolicy != mp)
1286       mergePolicy.close();
1287     mergePolicy = mp;
1288     mergePolicy.setIndexWriter(this);
1289     pushMaxBufferedDocs();
1290     if (infoStream != null)
1291       message("setMergePolicy " + mp);
1292     // Required so config.getMergePolicy returns the right value. But this will
1293     // go away together with the method in 4.0.
1294     config.setMergePolicy(mp);
1295   }
1296
1297   /**
1298    * Expert: returns the current MergePolicy in use by this writer.
1299    * @see #setMergePolicy
1300    * 
1301    * @deprecated use {@link IndexWriterConfig#getMergePolicy()} instead
1302    */
1303   @Deprecated
1304   public MergePolicy getMergePolicy() {
1305     ensureOpen();
1306     return mergePolicy;
1307   }
1308
1309   /**
1310    * Expert: set the merge scheduler used by this writer.
1311    * @deprecated use {@link IndexWriterConfig#setMergeScheduler(MergeScheduler)} instead
1312    */
1313   @Deprecated
1314   synchronized public void setMergeScheduler(MergeScheduler mergeScheduler) throws CorruptIndexException, IOException {
1315     ensureOpen();
1316     if (mergeScheduler == null)
1317       throw new NullPointerException("MergeScheduler must be non-null");
1318
1319     if (this.mergeScheduler != mergeScheduler) {
1320       finishMerges(true);
1321       this.mergeScheduler.close();
1322     }
1323     this.mergeScheduler = mergeScheduler;
1324     if (infoStream != null)
1325       message("setMergeScheduler " + mergeScheduler);
1326     // Required so config.getMergeScheduler returns the right value. But this will
1327     // go away together with the method in 4.0.
1328     config.setMergeScheduler(mergeScheduler);
1329   }
1330
1331   /**
1332    * Expert: returns the current MergeScheduler in use by this
1333    * writer.
1334    * @see #setMergeScheduler(MergeScheduler)
1335    * @deprecated use {@link IndexWriterConfig#getMergeScheduler()} instead
1336    */
1337   @Deprecated
1338   public MergeScheduler getMergeScheduler() {
1339     ensureOpen();
1340     return mergeScheduler;
1341   }
1342
1343   /** <p>Determines the largest segment (measured by
1344    * document count) that may be merged with other segments.
1345    * Small values (e.g., less than 10,000) are best for
1346    * interactive indexing, as this limits the length of
1347    * pauses while indexing to a few seconds.  Larger values
1348    * are best for batched indexing and speedier
1349    * searches.</p>
1350    *
1351    * <p>The default value is {@link Integer#MAX_VALUE}.</p>
1352    *
1353    * <p>Note that this method is a convenience method: it
1354    * just calls mergePolicy.setMaxMergeDocs as long as
1355    * mergePolicy is an instance of {@link LogMergePolicy}.
1356    * Otherwise an IllegalArgumentException is thrown.</p>
1357    *
1358    * <p>The default merge policy ({@link
1359    * LogByteSizeMergePolicy}) also allows you to set this
1360    * limit by net size (in MB) of the segment, using {@link
1361    * LogByteSizeMergePolicy#setMaxMergeMB}.</p>
1362    * @deprecated use {@link LogMergePolicy#setMaxMergeDocs(int)} directly.
1363    */
1364   @Deprecated
1365   public void setMaxMergeDocs(int maxMergeDocs) {
1366     getLogMergePolicy().setMaxMergeDocs(maxMergeDocs);
1367   }
1368
1369   /**
1370    * <p>Returns the largest segment (measured by document
1371    * count) that may be merged with other segments.</p>
1372    *
1373    * <p>Note that this method is a convenience method: it
1374    * just calls mergePolicy.getMaxMergeDocs as long as
1375    * mergePolicy is an instance of {@link LogMergePolicy}.
1376    * Otherwise an IllegalArgumentException is thrown.</p>
1377    *
1378    * @see #setMaxMergeDocs
1379    * @deprecated use {@link LogMergePolicy#getMaxMergeDocs()} directly.
1380    */
1381   @Deprecated
1382   public int getMaxMergeDocs() {
1383     return getLogMergePolicy().getMaxMergeDocs();
1384   }
1385
1386   /**
1387    * The maximum number of terms that will be indexed for a single field in a
1388    * document. This limits the amount of memory required for indexing, so that
1389    * collections with very large files will not crash the indexing process by
1390    * running out of memory. This setting refers to the number of running terms,
1391    * not to the number of different terms.
1392    * <p/>
1393    * <strong>Note:</strong> this silently truncates large documents, excluding
1394    * from the index all terms that occur further in the document. If you know
1395    * your source documents are large, be sure to set this value high enough to
1396    * accomodate the expected size. If you set it to Integer.MAX_VALUE, then the
1397    * only limit is your memory, but you should anticipate an OutOfMemoryError.
1398    * <p/>
1399    * By default, no more than {@link #DEFAULT_MAX_FIELD_LENGTH} terms will be
1400    * indexed for a field.
1401    * 
1402    * @deprecated use {@link LimitTokenCountAnalyzer} instead. Note that the
1403    *             behvaior slightly changed - the analyzer limits the number of
1404    *             tokens per token stream created, while this setting limits the
1405    *             total number of tokens to index. This only matters if you index
1406    *             many multi-valued fields though.
1407    */
1408   @Deprecated
1409   public void setMaxFieldLength(int maxFieldLength) {
1410     ensureOpen();
1411     this.maxFieldLength = maxFieldLength;
1412     docWriter.setMaxFieldLength(maxFieldLength);
1413     if (infoStream != null)
1414       message("setMaxFieldLength " + maxFieldLength);
1415   }
1416
1417   /**
1418    * Returns the maximum number of terms that will be
1419    * indexed for a single field in a document.
1420    * @see #setMaxFieldLength
1421    * @deprecated use {@link LimitTokenCountAnalyzer} to limit number of tokens.
1422    */
1423   @Deprecated
1424   public int getMaxFieldLength() {
1425     ensureOpen();
1426     return maxFieldLength;
1427   }
1428
1429   /**
1430    * @deprecated use {@link
1431    *  IndexWriterConfig#setReaderTermsIndexDivisor} instead.
1432    */
1433   @Deprecated
1434   public void setReaderTermsIndexDivisor(int divisor) {
1435     ensureOpen();
1436     config.setReaderTermsIndexDivisor(divisor);
1437     if (infoStream != null) {
1438       message("setReaderTermsIndexDivisor " + divisor);
1439     }
1440   }
1441
1442   /**
1443    * @deprecated use {@link
1444    *  IndexWriterConfig#getReaderTermsIndexDivisor} instead.
1445    */
1446   @Deprecated
1447   public int getReaderTermsIndexDivisor() {
1448     ensureOpen();
1449     return config.getReaderTermsIndexDivisor();
1450   }
1451
1452   /** Determines the minimal number of documents required
1453    * before the buffered in-memory documents are flushed as
1454    * a new Segment.  Large values generally gives faster
1455    * indexing.
1456    *
1457    * <p>When this is set, the writer will flush every
1458    * maxBufferedDocs added documents.  Pass in {@link
1459    * #DISABLE_AUTO_FLUSH} to prevent triggering a flush due
1460    * to number of buffered documents.  Note that if flushing
1461    * by RAM usage is also enabled, then the flush will be
1462    * triggered by whichever comes first.</p>
1463    *
1464    * <p>Disabled by default (writer flushes by RAM usage).</p>
1465    *
1466    * @throws IllegalArgumentException if maxBufferedDocs is
1467    * enabled but smaller than 2, or it disables maxBufferedDocs
1468    * when ramBufferSize is already disabled
1469    * @see #setRAMBufferSizeMB
1470    * @deprecated use {@link IndexWriterConfig#setMaxBufferedDocs(int)} instead.
1471    */
1472   @Deprecated
1473   public void setMaxBufferedDocs(int maxBufferedDocs) {
1474     ensureOpen();
1475     pushMaxBufferedDocs();
1476     if (infoStream != null) {
1477       message("setMaxBufferedDocs " + maxBufferedDocs);
1478     }
1479     // Required so config.getMaxBufferedDocs returns the right value. But this
1480     // will go away together with the method in 4.0.
1481     config.setMaxBufferedDocs(maxBufferedDocs);
1482   }
1483
1484   /**
1485    * If we are flushing by doc count (not by RAM usage), and
1486    * using LogDocMergePolicy then push maxBufferedDocs down
1487    * as its minMergeDocs, to keep backwards compatibility.
1488    */
1489   private void pushMaxBufferedDocs() {
1490     if (config.getMaxBufferedDocs() != DISABLE_AUTO_FLUSH) {
1491       final MergePolicy mp = mergePolicy;
1492       if (mp instanceof LogDocMergePolicy) {
1493         LogDocMergePolicy lmp = (LogDocMergePolicy) mp;
1494         final int maxBufferedDocs = config.getMaxBufferedDocs();
1495         if (lmp.getMinMergeDocs() != maxBufferedDocs) {
1496           if (infoStream != null)
1497             message("now push maxBufferedDocs " + maxBufferedDocs + " to LogDocMergePolicy");
1498           lmp.setMinMergeDocs(maxBufferedDocs);
1499         }
1500       }
1501     }
1502   }
1503
1504   /**
1505    * Returns the number of buffered added documents that will
1506    * trigger a flush if enabled.
1507    * @see #setMaxBufferedDocs
1508    * @deprecated use {@link IndexWriterConfig#getMaxBufferedDocs()} instead.
1509    */
1510   @Deprecated
1511   public int getMaxBufferedDocs() {
1512     ensureOpen();
1513     return config.getMaxBufferedDocs();
1514   }
1515
1516   /** Determines the amount of RAM that may be used for
1517    * buffering added documents and deletions before they are
1518    * flushed to the Directory.  Generally for faster
1519    * indexing performance it's best to flush by RAM usage
1520    * instead of document count and use as large a RAM buffer
1521    * as you can.
1522    *
1523    * <p>When this is set, the writer will flush whenever
1524    * buffered documents and deletions use this much RAM.
1525    * Pass in {@link #DISABLE_AUTO_FLUSH} to prevent
1526    * triggering a flush due to RAM usage.  Note that if
1527    * flushing by document count is also enabled, then the
1528    * flush will be triggered by whichever comes first.</p>
1529    *
1530    * <p> <b>NOTE</b>: the account of RAM usage for pending
1531    * deletions is only approximate.  Specifically, if you
1532    * delete by Query, Lucene currently has no way to measure
1533    * the RAM usage if individual Queries so the accounting
1534    * will under-estimate and you should compensate by either
1535    * calling commit() periodically yourself, or by using
1536    * {@link #setMaxBufferedDeleteTerms} to flush by count
1537    * instead of RAM usage (each buffered delete Query counts
1538    * as one).
1539    *
1540    * <p> <b>NOTE</b>: because IndexWriter uses
1541    * <code>int</code>s when managing its internal storage,
1542    * the absolute maximum value for this setting is somewhat
1543    * less than 2048 MB.  The precise limit depends on
1544    * various factors, such as how large your documents are,
1545    * how many fields have norms, etc., so it's best to set
1546    * this value comfortably under 2048.</p>
1547    *
1548    * <p> The default value is {@link #DEFAULT_RAM_BUFFER_SIZE_MB}.</p>
1549    * 
1550    * @throws IllegalArgumentException if ramBufferSize is
1551    * enabled but non-positive, or it disables ramBufferSize
1552    * when maxBufferedDocs is already disabled
1553    * @deprecated use {@link IndexWriterConfig#setRAMBufferSizeMB(double)} instead.
1554    */
1555   @Deprecated
1556   public void setRAMBufferSizeMB(double mb) {
1557     if (infoStream != null) {
1558       message("setRAMBufferSizeMB " + mb);
1559     }
1560     // Required so config.getRAMBufferSizeMB returns the right value. But this
1561     // will go away together with the method in 4.0.
1562     config.setRAMBufferSizeMB(mb);
1563   }
1564
1565   /**
1566    * Returns the value set by {@link #setRAMBufferSizeMB} if enabled.
1567    * @deprecated use {@link IndexWriterConfig#getRAMBufferSizeMB()} instead.
1568    */
1569   @Deprecated
1570   public double getRAMBufferSizeMB() {
1571     return config.getRAMBufferSizeMB();
1572   }
1573
1574   /**
1575    * <p>Determines the minimal number of delete terms required before the buffered
1576    * in-memory delete terms are applied and flushed. If there are documents
1577    * buffered in memory at the time, they are merged and a new segment is
1578    * created.</p>
1579
1580    * <p>Disabled by default (writer flushes by RAM usage).</p>
1581    * 
1582    * @throws IllegalArgumentException if maxBufferedDeleteTerms
1583    * is enabled but smaller than 1
1584    * @see #setRAMBufferSizeMB
1585    * @deprecated use {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)} instead.
1586    */
1587   @Deprecated
1588   public void setMaxBufferedDeleteTerms(int maxBufferedDeleteTerms) {
1589     ensureOpen();
1590     if (infoStream != null)
1591       message("setMaxBufferedDeleteTerms " + maxBufferedDeleteTerms);
1592     // Required so config.getMaxBufferedDeleteTerms returns the right value. But
1593     // this will go away together with the method in 4.0.
1594     config.setMaxBufferedDeleteTerms(maxBufferedDeleteTerms);
1595   }
1596
1597   /**
1598    * Returns the number of buffered deleted terms that will
1599    * trigger a flush if enabled.
1600    * @see #setMaxBufferedDeleteTerms
1601    * @deprecated use {@link IndexWriterConfig#getMaxBufferedDeleteTerms()} instead
1602    */
1603   @Deprecated
1604   public int getMaxBufferedDeleteTerms() {
1605     ensureOpen();
1606     return config.getMaxBufferedDeleteTerms();
1607   }
1608
1609   /** Determines how often segment indices are merged by addDocument().  With
1610    * smaller values, less RAM is used while indexing, and searches on
1611    * unoptimized indices are faster, but indexing speed is slower.  With larger
1612    * values, more RAM is used during indexing, and while searches on unoptimized
1613    * indices are slower, indexing is faster.  Thus larger values (> 10) are best
1614    * for batch index creation, and smaller values (< 10) for indices that are
1615    * interactively maintained.
1616    *
1617    * <p>Note that this method is a convenience method: it
1618    * just calls mergePolicy.setMergeFactor as long as
1619    * mergePolicy is an instance of {@link LogMergePolicy}.
1620    * Otherwise an IllegalArgumentException is thrown.</p>
1621    *
1622    * <p>This must never be less than 2.  The default value is 10.
1623    * @deprecated use {@link LogMergePolicy#setMergeFactor(int)} directly.
1624    */
1625   @Deprecated
1626   public void setMergeFactor(int mergeFactor) {
1627     getLogMergePolicy().setMergeFactor(mergeFactor);
1628   }
1629
1630   /**
1631    * <p>Returns the number of segments that are merged at
1632    * once and also controls the total number of segments
1633    * allowed to accumulate in the index.</p>
1634    *
1635    * <p>Note that this method is a convenience method: it
1636    * just calls mergePolicy.getMergeFactor as long as
1637    * mergePolicy is an instance of {@link LogMergePolicy}.
1638    * Otherwise an IllegalArgumentException is thrown.</p>
1639    *
1640    * @see #setMergeFactor
1641    * @deprecated use {@link LogMergePolicy#getMergeFactor()} directly.
1642    */
1643   @Deprecated
1644   public int getMergeFactor() {
1645     return getLogMergePolicy().getMergeFactor();
1646   }
1647
1648   /** If non-null, this will be the default infoStream used
1649    * by a newly instantiated IndexWriter.
1650    * @see #setInfoStream
1651    */
1652   public static void setDefaultInfoStream(PrintStream infoStream) {
1653     IndexWriter.defaultInfoStream = infoStream;
1654   }
1655
1656   /**
1657    * Returns the current default infoStream for newly
1658    * instantiated IndexWriters.
1659    * @see #setDefaultInfoStream
1660    */
1661   public static PrintStream getDefaultInfoStream() {
1662     return IndexWriter.defaultInfoStream;
1663   }
1664
1665   /** If non-null, information about merges, deletes and a
1666    * message when maxFieldLength is reached will be printed
1667    * to this.
1668    */
1669   public void setInfoStream(PrintStream infoStream) throws IOException {
1670     ensureOpen();
1671     this.infoStream = infoStream;
1672     docWriter.setInfoStream(infoStream);
1673     deleter.setInfoStream(infoStream);
1674     bufferedDeletesStream.setInfoStream(infoStream);
1675     if (infoStream != null)
1676       messageState();
1677   }
1678
1679   private void messageState() throws IOException {
1680     message("\ndir=" + directory + "\n" +
1681             "index=" + segString() + "\n" +
1682             "version=" + Constants.LUCENE_VERSION + "\n" +
1683             config.toString());
1684   }
1685
1686   /**
1687    * Returns the current infoStream in use by this writer.
1688    * @see #setInfoStream
1689    */
1690   public PrintStream getInfoStream() {
1691     ensureOpen();
1692     return infoStream;
1693   }
1694
1695   /** Returns true if verbosing is enabled (i.e., infoStream != null). */
1696   public boolean verbose() {
1697     return infoStream != null;
1698   }
1699   
1700   /**
1701    * Sets the maximum time to wait for a write lock (in milliseconds) for this instance of IndexWriter.  @see
1702    * @see #setDefaultWriteLockTimeout to change the default value for all instances of IndexWriter.
1703    * @deprecated use {@link IndexWriterConfig#setWriteLockTimeout(long)} instead
1704    */
1705   @Deprecated
1706   public void setWriteLockTimeout(long writeLockTimeout) {
1707     ensureOpen();
1708     this.writeLockTimeout = writeLockTimeout;
1709     // Required so config.getWriteLockTimeout returns the right value. But this
1710     // will go away together with the method in 4.0.
1711     config.setWriteLockTimeout(writeLockTimeout);
1712   }
1713
1714   /**
1715    * Returns allowed timeout when acquiring the write lock.
1716    * @see #setWriteLockTimeout
1717    * @deprecated use {@link IndexWriterConfig#getWriteLockTimeout()}
1718    */
1719   @Deprecated
1720   public long getWriteLockTimeout() {
1721     ensureOpen();
1722     return writeLockTimeout;
1723   }
1724
1725   /**
1726    * Sets the default (for any instance of IndexWriter) maximum time to wait for a write lock (in
1727    * milliseconds).
1728    * @deprecated use {@link IndexWriterConfig#setDefaultWriteLockTimeout(long)} instead
1729    */
1730   @Deprecated
1731   public static void setDefaultWriteLockTimeout(long writeLockTimeout) {
1732     IndexWriterConfig.setDefaultWriteLockTimeout(writeLockTimeout);
1733   }
1734
1735   /**
1736    * Returns default write lock timeout for newly
1737    * instantiated IndexWriters.
1738    * @see #setDefaultWriteLockTimeout
1739    * @deprecated use {@link IndexWriterConfig#getDefaultWriteLockTimeout()} instead
1740    */
1741   @Deprecated
1742   public static long getDefaultWriteLockTimeout() {
1743     return IndexWriterConfig.getDefaultWriteLockTimeout();
1744   }
1745
1746   /**
1747    * Commits all changes to an index and closes all
1748    * associated files.  Note that this may be a costly
1749    * operation, so, try to re-use a single writer instead of
1750    * closing and opening a new one.  See {@link #commit()} for
1751    * caveats about write caching done by some IO devices.
1752    *
1753    * <p> If an Exception is hit during close, eg due to disk
1754    * full or some other reason, then both the on-disk index
1755    * and the internal state of the IndexWriter instance will
1756    * be consistent.  However, the close will not be complete
1757    * even though part of it (flushing buffered documents)
1758    * may have succeeded, so the write lock will still be
1759    * held.</p>
1760    * 
1761    * <p> If you can correct the underlying cause (eg free up
1762    * some disk space) then you can call close() again.
1763    * Failing that, if you want to force the write lock to be
1764    * released (dangerous, because you may then lose buffered
1765    * docs in the IndexWriter instance) then you can do
1766    * something like this:</p>
1767    *
1768    * <pre>
1769    * try {
1770    *   writer.close();
1771    * } finally {
1772    *   if (IndexWriter.isLocked(directory)) {
1773    *     IndexWriter.unlock(directory);
1774    *   }
1775    * }
1776    * </pre>
1777    *
1778    * after which, you must be certain not to use the writer
1779    * instance anymore.</p>
1780    *
1781    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
1782    * you should immediately close the writer, again.  See <a
1783    * href="#OOME">above</a> for details.</p>
1784    *
1785    * @throws CorruptIndexException if the index is corrupt
1786    * @throws IOException if there is a low-level IO error
1787    */
1788   public void close() throws CorruptIndexException, IOException {
1789     close(true);
1790   }
1791
1792   /**
1793    * Closes the index with or without waiting for currently
1794    * running merges to finish.  This is only meaningful when
1795    * using a MergeScheduler that runs merges in background
1796    * threads.
1797    *
1798    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
1799    * you should immediately close the writer, again.  See <a
1800    * href="#OOME">above</a> for details.</p>
1801    *
1802    * <p><b>NOTE</b>: it is dangerous to always call
1803    * close(false), especially when IndexWriter is not open
1804    * for very long, because this can result in "merge
1805    * starvation" whereby long merges will never have a
1806    * chance to finish.  This will cause too many segments in
1807    * your index over time.</p>
1808    *
1809    * @param waitForMerges if true, this call will block
1810    * until all merges complete; else, it will ask all
1811    * running merges to abort, wait until those merges have
1812    * finished (which should be at most a few seconds), and
1813    * then return.
1814    */
1815   public void close(boolean waitForMerges) throws CorruptIndexException, IOException {
1816
1817     // Ensure that only one thread actually gets to do the closing:
1818     if (shouldClose()) {
1819       // If any methods have hit OutOfMemoryError, then abort
1820       // on close, in case the internal state of IndexWriter
1821       // or DocumentsWriter is corrupt
1822       if (hitOOM)
1823         rollbackInternal();
1824       else
1825         closeInternal(waitForMerges);
1826     }
1827   }
1828
1829   // Returns true if this thread should attempt to close, or
1830   // false if IndexWriter is now closed; else, waits until
1831   // another thread finishes closing
1832   synchronized private boolean shouldClose() {
1833     while(true) {
1834       if (!closed) {
1835         if (!closing) {
1836           closing = true;
1837           return true;
1838         } else {
1839           // Another thread is presently trying to close;
1840           // wait until it finishes one way (closes
1841           // successfully) or another (fails to close)
1842           doWait();
1843         }
1844       } else
1845         return false;
1846     }
1847   }
1848
1849   private void closeInternal(boolean waitForMerges) throws CorruptIndexException, IOException {
1850
1851     try {
1852       if (infoStream != null) {
1853         message("now flush at close waitForMerges=" + waitForMerges);
1854       }
1855
1856       docWriter.close();
1857
1858       // Only allow a new merge to be triggered if we are
1859       // going to wait for merges:
1860       if (!hitOOM) {
1861         flush(waitForMerges, true);
1862       }
1863
1864       if (waitForMerges)
1865         // Give merge scheduler last chance to run, in case
1866         // any pending merges are waiting:
1867         mergeScheduler.merge(this);
1868
1869       mergePolicy.close();
1870
1871       synchronized(this) {
1872         finishMerges(waitForMerges);
1873         stopMerges = true;
1874       }
1875
1876       mergeScheduler.close();
1877
1878       if (infoStream != null)
1879         message("now call final commit()");
1880       
1881       if (!hitOOM) {
1882         commitInternal(null);
1883       }
1884
1885       if (infoStream != null)
1886         message("at close: " + segString());
1887
1888       synchronized(this) {
1889         readerPool.close();
1890         docWriter = null;
1891         deleter.close();
1892       }
1893       
1894       if (writeLock != null) {
1895         writeLock.release();                          // release write lock
1896         writeLock = null;
1897       }
1898       synchronized(this) {
1899         closed = true;
1900       }
1901     } catch (OutOfMemoryError oom) {
1902       handleOOM(oom, "closeInternal");
1903     } finally {
1904       synchronized(this) {
1905         closing = false;
1906         notifyAll();
1907         if (!closed) {
1908           if (infoStream != null)
1909             message("hit exception while closing");
1910         }
1911       }
1912     }
1913   }
1914
1915   /** Returns the Directory used by this index. */
1916   public Directory getDirectory() {     
1917     // Pass false because the flush during closing calls getDirectory
1918     ensureOpen(false);
1919     return directory;
1920   }
1921
1922   /** Returns the analyzer used by this index. */
1923   public Analyzer getAnalyzer() {
1924     ensureOpen();
1925     return analyzer;
1926   }
1927
1928   /** Returns total number of docs in this index, including
1929    *  docs not yet flushed (still in the RAM buffer),
1930    *  not counting deletions.
1931    *  @see #numDocs */
1932   public synchronized int maxDoc() {
1933     int count;
1934     if (docWriter != null)
1935       count = docWriter.getNumDocs();
1936     else
1937       count = 0;
1938
1939     count += segmentInfos.totalDocCount();
1940     return count;
1941   }
1942
1943   /** Returns total number of docs in this index, including
1944    *  docs not yet flushed (still in the RAM buffer), and
1945    *  including deletions.  <b>NOTE:</b> buffered deletions
1946    *  are not counted.  If you really need these to be
1947    *  counted you should call {@link #commit()} first.
1948    *  @see #numDocs */
1949   public synchronized int numDocs() throws IOException {
1950     int count;
1951     if (docWriter != null)
1952       count = docWriter.getNumDocs();
1953     else
1954       count = 0;
1955
1956     for (final SegmentInfo info : segmentInfos) {
1957       count += info.docCount - numDeletedDocs(info);
1958     }
1959     return count;
1960   }
1961
1962   public synchronized boolean hasDeletions() throws IOException {
1963     ensureOpen();
1964     if (bufferedDeletesStream.any()) {
1965       return true;
1966     }
1967     if (docWriter.anyDeletions()) {
1968       return true;
1969     }
1970     for (final SegmentInfo info : segmentInfos) {
1971       if (info.hasDeletions()) {
1972         return true;
1973       }
1974     }
1975     return false;
1976   }
1977
1978   /**
1979    * The maximum number of terms that will be indexed for a single field in a
1980    * document.  This limits the amount of memory required for indexing, so that
1981    * collections with very large files will not crash the indexing process by
1982    * running out of memory.<p/>
1983    * Note that this effectively truncates large documents, excluding from the
1984    * index terms that occur further in the document.  If you know your source
1985    * documents are large, be sure to set this value high enough to accommodate
1986    * the expected size.  If you set it to Integer.MAX_VALUE, then the only limit
1987    * is your memory, but you should anticipate an OutOfMemoryError.<p/>
1988    * By default, no more than 10,000 terms will be indexed for a field.
1989    *
1990    * @see MaxFieldLength
1991    * @deprecated remove in 4.0
1992    */
1993   @Deprecated
1994   private int maxFieldLength = DEFAULT_MAX_FIELD_LENGTH;
1995
1996   /**
1997    * Adds a document to this index.  If the document contains more than
1998    * {@link #setMaxFieldLength(int)} terms for a given field, the remainder are
1999    * discarded.
2000    *
2001    * <p> Note that if an Exception is hit (for example disk full)
2002    * then the index will be consistent, but this document
2003    * may not have been added.  Furthermore, it's possible
2004    * the index will have one segment in non-compound format
2005    * even when using compound files (when a merge has
2006    * partially succeeded).</p>
2007    *
2008    * <p> This method periodically flushes pending documents
2009    * to the Directory (see <a href="#flush">above</a>), and
2010    * also periodically triggers segment merges in the index
2011    * according to the {@link MergePolicy} in use.</p>
2012    *
2013    * <p>Merges temporarily consume space in the
2014    * directory. The amount of space required is up to 1X the
2015    * size of all segments being merged, when no
2016    * readers/searchers are open against the index, and up to
2017    * 2X the size of all segments being merged when
2018    * readers/searchers are open against the index (see
2019    * {@link #optimize()} for details). The sequence of
2020    * primitive merge operations performed is governed by the
2021    * merge policy.
2022    *
2023    * <p>Note that each term in the document can be no longer
2024    * than 16383 characters, otherwise an
2025    * IllegalArgumentException will be thrown.</p>
2026    *
2027    * <p>Note that it's possible to create an invalid Unicode
2028    * string in java if a UTF16 surrogate pair is malformed.
2029    * In this case, the invalid characters are silently
2030    * replaced with the Unicode replacement character
2031    * U+FFFD.</p>
2032    *
2033    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2034    * you should immediately close the writer.  See <a
2035    * href="#OOME">above</a> for details.</p>
2036    *
2037    * @throws CorruptIndexException if the index is corrupt
2038    * @throws IOException if there is a low-level IO error
2039    */
2040   public void addDocument(Document doc) throws CorruptIndexException, IOException {
2041     addDocument(doc, analyzer);
2042   }
2043
2044   /**
2045    * Adds a document to this index, using the provided analyzer instead of the
2046    * value of {@link #getAnalyzer()}.  If the document contains more than
2047    * {@link #setMaxFieldLength(int)} terms for a given field, the remainder are
2048    * discarded.
2049    *
2050    * <p>See {@link #addDocument(Document)} for details on
2051    * index and IndexWriter state after an Exception, and
2052    * flushing/merging temporary free space requirements.</p>
2053    *
2054    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2055    * you should immediately close the writer.  See <a
2056    * href="#OOME">above</a> for details.</p>
2057    *
2058    * @throws CorruptIndexException if the index is corrupt
2059    * @throws IOException if there is a low-level IO error
2060    */
2061   public void addDocument(Document doc, Analyzer analyzer) throws CorruptIndexException, IOException {
2062     ensureOpen();
2063     boolean doFlush = false;
2064     boolean success = false;
2065     try {
2066       try {
2067         doFlush = docWriter.updateDocument(doc, analyzer, null);
2068         success = true;
2069       } finally {
2070         if (!success && infoStream != null)
2071           message("hit exception adding document");
2072       }
2073       if (doFlush)
2074         flush(true, false);
2075     } catch (OutOfMemoryError oom) {
2076       handleOOM(oom, "addDocument");
2077     }
2078   }
2079
2080   /**
2081    * Atomically adds a block of documents with sequentially
2082    * assigned document IDs, such that an external reader
2083    * will see all or none of the documents.
2084    *
2085    * <p><b>WARNING</b>: the index does not currently record
2086    * which documents were added as a block.  Today this is
2087    * fine, because merging will preserve the block (as long
2088    * as none them were deleted).  But it's possible in the
2089    * future that Lucene may more aggressively re-order
2090    * documents (for example, perhaps to obtain better index
2091    * compression), in which case you may need to fully
2092    * re-index your documents at that time.
2093    *
2094    * <p>See {@link #addDocument(Document)} for details on
2095    * index and IndexWriter state after an Exception, and
2096    * flushing/merging temporary free space requirements.</p>
2097    *
2098    * <p><b>NOTE</b>: tools that do offline splitting of an index
2099    * (for example, IndexSplitter in contrib) or
2100    * re-sorting of documents (for example, IndexSorter in
2101    * contrib) are not aware of these atomically added documents
2102    * and will likely break them up.  Use such tools at your
2103    * own risk!
2104    *
2105    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2106    * you should immediately close the writer.  See <a
2107    * href="#OOME">above</a> for details.</p>
2108    *
2109    * @throws CorruptIndexException if the index is corrupt
2110    * @throws IOException if there is a low-level IO error
2111    *
2112    * @lucene.experimental
2113    */
2114   public void addDocuments(Collection<Document> docs) throws CorruptIndexException, IOException {
2115     // TODO: if we backport DWPT we should change arg to Iterable<Document>
2116     addDocuments(docs, analyzer);
2117   }
2118
2119   /**
2120    * Atomically adds a block of documents, analyzed using the
2121    * provided analyzer, with sequentially assigned document
2122    * IDs, such that an external reader will see all or none
2123    * of the documents. 
2124    *
2125    * @throws CorruptIndexException if the index is corrupt
2126    * @throws IOException if there is a low-level IO error
2127    *
2128    * @lucene.experimental
2129    */
2130   public void addDocuments(Collection<Document> docs, Analyzer analyzer) throws CorruptIndexException, IOException {
2131     // TODO: if we backport DWPT we should change arg to Iterable<Document>
2132     updateDocuments(null, docs, analyzer);
2133   }
2134
2135   /**
2136    * Atomically deletes documents matching the provided
2137    * delTerm and adds a block of documents with sequentially
2138    * assigned document IDs, such that an external reader
2139    * will see all or none of the documents. 
2140    *
2141    * See {@link #addDocuments(Collection)}.
2142    *
2143    * @throws CorruptIndexException if the index is corrupt
2144    * @throws IOException if there is a low-level IO error
2145    *
2146    * @lucene.experimental
2147    */
2148   public void updateDocuments(Term delTerm, Collection<Document> docs) throws CorruptIndexException, IOException {
2149     // TODO: if we backport DWPT we should change arg to Iterable<Document>
2150     updateDocuments(delTerm, docs, analyzer);
2151   }
2152
2153   /**
2154    * Atomically deletes documents matching the provided
2155    * delTerm and adds a block of documents, analyzed  using
2156    * the provided analyzer, with sequentially
2157    * assigned document IDs, such that an external reader
2158    * will see all or none of the documents. 
2159    *
2160    * See {@link #addDocuments(Collection)}.
2161    *
2162    * @throws CorruptIndexException if the index is corrupt
2163    * @throws IOException if there is a low-level IO error
2164    *
2165    * @lucene.experimental
2166    */
2167   public void updateDocuments(Term delTerm, Collection<Document> docs, Analyzer analyzer) throws CorruptIndexException, IOException {
2168     // TODO: if we backport DWPT we should change arg to Iterable<Document>
2169     ensureOpen();
2170     try {
2171       boolean success = false;
2172       boolean doFlush = false;
2173       try {
2174         doFlush = docWriter.updateDocuments(docs, analyzer, delTerm);
2175         success = true;
2176       } finally {
2177         if (!success && infoStream != null) {
2178           message("hit exception updating document");
2179         }
2180       }
2181       if (doFlush) {
2182         flush(true, false);
2183       }
2184     } catch (OutOfMemoryError oom) {
2185       handleOOM(oom, "updateDocuments");
2186     }
2187   }
2188
2189   /**
2190    * Deletes the document(s) containing <code>term</code>.
2191    *
2192    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2193    * you should immediately close the writer.  See <a
2194    * href="#OOME">above</a> for details.</p>
2195    *
2196    * @param term the term to identify the documents to be deleted
2197    * @throws CorruptIndexException if the index is corrupt
2198    * @throws IOException if there is a low-level IO error
2199    */
2200   public void deleteDocuments(Term term) throws CorruptIndexException, IOException {
2201     ensureOpen();
2202     try {
2203       if (docWriter.deleteTerm(term, false)) {
2204         flush(true, false);
2205       }
2206     } catch (OutOfMemoryError oom) {
2207       handleOOM(oom, "deleteDocuments(Term)");
2208     }
2209   }
2210
2211   /**
2212    * Deletes the document(s) containing any of the
2213    * terms. All deletes are flushed at the same time.
2214    *
2215    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2216    * you should immediately close the writer.  See <a
2217    * href="#OOME">above</a> for details.</p>
2218    *
2219    * @param terms array of terms to identify the documents
2220    * to be deleted
2221    * @throws CorruptIndexException if the index is corrupt
2222    * @throws IOException if there is a low-level IO error
2223    */
2224   public void deleteDocuments(Term... terms) throws CorruptIndexException, IOException {
2225     ensureOpen();
2226     try {
2227       if (docWriter.deleteTerms(terms)) {
2228         flush(true, false);
2229       }
2230     } catch (OutOfMemoryError oom) {
2231       handleOOM(oom, "deleteDocuments(Term..)");
2232     }
2233   }
2234
2235   /**
2236    * Deletes the document(s) matching the provided query.
2237    *
2238    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2239    * you should immediately close the writer.  See <a
2240    * href="#OOME">above</a> for details.</p>
2241    *
2242    * @param query the query to identify the documents to be deleted
2243    * @throws CorruptIndexException if the index is corrupt
2244    * @throws IOException if there is a low-level IO error
2245    */
2246   public void deleteDocuments(Query query) throws CorruptIndexException, IOException {
2247     ensureOpen();
2248     try {
2249       if (docWriter.deleteQuery(query)) {
2250         flush(true, false);
2251       }
2252     } catch (OutOfMemoryError oom) {
2253       handleOOM(oom, "deleteDocuments(Query)");
2254     }
2255   }
2256
2257   /**
2258    * Deletes the document(s) matching any of the provided queries.
2259    * All deletes are flushed at the same time.
2260    *
2261    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2262    * you should immediately close the writer.  See <a
2263    * href="#OOME">above</a> for details.</p>
2264    *
2265    * @param queries array of queries to identify the documents
2266    * to be deleted
2267    * @throws CorruptIndexException if the index is corrupt
2268    * @throws IOException if there is a low-level IO error
2269    */
2270   public void deleteDocuments(Query... queries) throws CorruptIndexException, IOException {
2271     ensureOpen();
2272     try {
2273       if (docWriter.deleteQueries(queries)) {
2274         flush(true, false);
2275       }
2276     } catch (OutOfMemoryError oom) {
2277       handleOOM(oom, "deleteDocuments(Query..)");
2278     }
2279   }
2280
2281   /**
2282    * Updates a document by first deleting the document(s)
2283    * containing <code>term</code> and then adding the new
2284    * document.  The delete and then add are atomic as seen
2285    * by a reader on the same index (flush may happen only after
2286    * the add).
2287    *
2288    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2289    * you should immediately close the writer.  See <a
2290    * href="#OOME">above</a> for details.</p>
2291    *
2292    * @param term the term to identify the document(s) to be
2293    * deleted
2294    * @param doc the document to be added
2295    * @throws CorruptIndexException if the index is corrupt
2296    * @throws IOException if there is a low-level IO error
2297    */
2298   public void updateDocument(Term term, Document doc) throws CorruptIndexException, IOException {
2299     ensureOpen();
2300     updateDocument(term, doc, getAnalyzer());
2301   }
2302
2303   /**
2304    * Updates a document by first deleting the document(s)
2305    * containing <code>term</code> and then adding the new
2306    * document.  The delete and then add are atomic as seen
2307    * by a reader on the same index (flush may happen only after
2308    * the add).
2309    *
2310    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2311    * you should immediately close the writer.  See <a
2312    * href="#OOME">above</a> for details.</p>
2313    *
2314    * @param term the term to identify the document(s) to be
2315    * deleted
2316    * @param doc the document to be added
2317    * @param analyzer the analyzer to use when analyzing the document
2318    * @throws CorruptIndexException if the index is corrupt
2319    * @throws IOException if there is a low-level IO error
2320    */
2321   public void updateDocument(Term term, Document doc, Analyzer analyzer)
2322       throws CorruptIndexException, IOException {
2323     ensureOpen();
2324     try {
2325       boolean doFlush = false;
2326       boolean success = false;
2327       try {
2328         doFlush = docWriter.updateDocument(doc, analyzer, term);
2329         success = true;
2330       } finally {
2331         if (!success && infoStream != null)
2332           message("hit exception updating document");
2333       }
2334       if (doFlush) {
2335         flush(true, false);
2336       }
2337     } catch (OutOfMemoryError oom) {
2338       handleOOM(oom, "updateDocument");
2339     }
2340   }
2341
2342   // for test purpose
2343   final synchronized int getSegmentCount(){
2344     return segmentInfos.size();
2345   }
2346
2347   // for test purpose
2348   final synchronized int getNumBufferedDocuments(){
2349     return docWriter.getNumDocs();
2350   }
2351
2352   // for test purpose
2353   final synchronized int getDocCount(int i) {
2354     if (i >= 0 && i < segmentInfos.size()) {
2355       return segmentInfos.info(i).docCount;
2356     } else {
2357       return -1;
2358     }
2359   }
2360
2361   // for test purpose
2362   final int getFlushCount() {
2363     return flushCount.get();
2364   }
2365
2366   // for test purpose
2367   final int getFlushDeletesCount() {
2368     return flushDeletesCount.get();
2369   }
2370
2371   final String newSegmentName() {
2372     // Cannot synchronize on IndexWriter because that causes
2373     // deadlock
2374     synchronized(segmentInfos) {
2375       // Important to increment changeCount so that the
2376       // segmentInfos is written on close.  Otherwise we
2377       // could close, re-open and re-return the same segment
2378       // name that was previously returned which can cause
2379       // problems at least with ConcurrentMergeScheduler.
2380       changeCount++;
2381       segmentInfos.changed();
2382       return "_" + Integer.toString(segmentInfos.counter++, Character.MAX_RADIX);
2383     }
2384   }
2385
2386   /** If non-null, information about merges will be printed to this.
2387    */
2388   private PrintStream infoStream;
2389   private static PrintStream defaultInfoStream;
2390
2391   /**
2392    * Requests an "optimize" operation on an index, priming the index
2393    * for the fastest available search. Traditionally this has meant
2394    * merging all segments into a single segment as is done in the
2395    * default merge policy, but individual merge policies may implement
2396    * optimize in different ways.
2397    *
2398    * <p> Optimize is a fairly costly operation, so you
2399    * should only do it if your search performance really
2400    * requires it.  Many search applications do fine never
2401    * calling optimize. </p>
2402    *
2403    * <p>Note that optimize requires 2X the index size free
2404    * space in your Directory (3X if you're using compound
2405    * file format).  For example, if your index size is 10 MB
2406    * then you need 20 MB free for optimize to complete (30
2407    * MB if you're using compound file format).  Also,
2408    * it's best to call {@link #commit()} after the optimize
2409    * completes to allow IndexWriter to free up disk space.</p>
2410    *
2411    * <p>If some but not all readers re-open while an
2412    * optimize is underway, this will cause > 2X temporary
2413    * space to be consumed as those new readers will then
2414    * hold open the partially optimized segments at that
2415    * time.  It is best not to re-open readers while optimize
2416    * is running.</p>
2417    *
2418    * <p>The actual temporary usage could be much less than
2419    * these figures (it depends on many factors).</p>
2420    *
2421    * <p>In general, once the optimize completes, the total size of the
2422    * index will be less than the size of the starting index.
2423    * It could be quite a bit smaller (if there were many
2424    * pending deletes) or just slightly smaller.</p>
2425    *
2426    * <p>If an Exception is hit during optimize(), for example
2427    * due to disk full, the index will not be corrupt and no
2428    * documents will have been lost.  However, it may have
2429    * been partially optimized (some segments were merged but
2430    * not all), and it's possible that one of the segments in
2431    * the index will be in non-compound format even when
2432    * using compound file format.  This will occur when the
2433    * Exception is hit during conversion of the segment into
2434    * compound format.</p>
2435    *
2436    * <p>This call will optimize those segments present in
2437    * the index when the call started.  If other threads are
2438    * still adding documents and flushing segments, those
2439    * newly created segments will not be optimized unless you
2440    * call optimize again.</p>
2441    *
2442    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2443    * you should immediately close the writer.  See <a
2444    * href="#OOME">above</a> for details.</p>
2445    *
2446    * <p><b>NOTE</b>: if you call {@link #close(boolean)}
2447    * with <tt>false</tt>, which aborts all running merges,
2448    * then any thread still running this method might hit a
2449    * {@link MergePolicy.MergeAbortedException}.
2450    *
2451    * @throws CorruptIndexException if the index is corrupt
2452    * @throws IOException if there is a low-level IO error
2453    * @see MergePolicy#findMergesForOptimize
2454   */
2455   public void optimize() throws CorruptIndexException, IOException {
2456     optimize(true);
2457   }
2458
2459   /**
2460    * Optimize the index down to <= maxNumSegments.  If
2461    * maxNumSegments==1 then this is the same as {@link
2462    * #optimize()}.
2463    *
2464    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2465    * you should immediately close the writer.  See <a
2466    * href="#OOME">above</a> for details.</p>
2467    *
2468    * @param maxNumSegments maximum number of segments left
2469    * in the index after optimization finishes
2470    */
2471   public void optimize(int maxNumSegments) throws CorruptIndexException, IOException {
2472     optimize(maxNumSegments, true);
2473   }
2474
2475   /** Just like {@link #optimize()}, except you can specify
2476    *  whether the call should block until the optimize
2477    *  completes.  This is only meaningful with a
2478    *  {@link MergeScheduler} that is able to run merges in
2479    *  background threads.
2480    *
2481    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2482    * you should immediately close the writer.  See <a
2483    * href="#OOME">above</a> for details.</p>
2484    */
2485   public void optimize(boolean doWait) throws CorruptIndexException, IOException {
2486     optimize(1, doWait);
2487   }
2488
2489   /** Just like {@link #optimize(int)}, except you can
2490    *  specify whether the call should block until the
2491    *  optimize completes.  This is only meaningful with a
2492    *  {@link MergeScheduler} that is able to run merges in
2493    *  background threads.
2494    *
2495    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2496    * you should immediately close the writer.  See <a
2497    * href="#OOME">above</a> for details.</p>
2498    */
2499   public void optimize(int maxNumSegments, boolean doWait) throws CorruptIndexException, IOException {
2500     ensureOpen();
2501
2502     if (maxNumSegments < 1)
2503       throw new IllegalArgumentException("maxNumSegments must be >= 1; got " + maxNumSegments);
2504
2505     if (infoStream != null) {
2506       message("optimize: index now " + segString());
2507       message("now flush at optimize");
2508     }
2509
2510     flush(true, true);
2511
2512     synchronized(this) {
2513       resetMergeExceptions();
2514       segmentsToOptimize.clear();
2515       for(SegmentInfo info : segmentInfos) {
2516         segmentsToOptimize.put(info, Boolean.TRUE);
2517       }
2518       optimizeMaxNumSegments = maxNumSegments;
2519       
2520       // Now mark all pending & running merges as optimize
2521       // merge:
2522       for(final MergePolicy.OneMerge merge  : pendingMerges) {
2523         merge.optimize = true;
2524         merge.maxNumSegmentsOptimize = maxNumSegments;
2525         segmentsToOptimize.put(merge.info, Boolean.TRUE);
2526       }
2527
2528       for ( final MergePolicy.OneMerge merge: runningMerges ) {
2529         merge.optimize = true;
2530         merge.maxNumSegmentsOptimize = maxNumSegments;
2531         segmentsToOptimize.put(merge.info, Boolean.TRUE);
2532       }
2533     }
2534
2535     maybeMerge(maxNumSegments, true);
2536
2537     if (doWait) {
2538       synchronized(this) {
2539         while(true) {
2540
2541           if (hitOOM) {
2542             throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete optimize");
2543           }
2544
2545           if (mergeExceptions.size() > 0) {
2546             // Forward any exceptions in background merge
2547             // threads to the current thread:
2548             final int size = mergeExceptions.size();
2549             for(int i=0;i<size;i++) {
2550               final MergePolicy.OneMerge merge = mergeExceptions.get(i);
2551               if (merge.optimize) {
2552                 IOException err = new IOException("background merge hit exception: " + merge.segString(directory));
2553                 final Throwable t = merge.getException();
2554                 if (t != null)
2555                   err.initCause(t);
2556                 throw err;
2557               }
2558             }
2559           }
2560
2561           if (optimizeMergesPending())
2562             doWait();
2563           else
2564             break;
2565         }
2566       }
2567
2568       // If close is called while we are still
2569       // running, throw an exception so the calling
2570       // thread will know the optimize did not
2571       // complete
2572       ensureOpen();
2573     }
2574
2575     // NOTE: in the ConcurrentMergeScheduler case, when
2576     // doWait is false, we can return immediately while
2577     // background threads accomplish the optimization
2578   }
2579
2580   /** Returns true if any merges in pendingMerges or
2581    *  runningMerges are optimization merges. */
2582   private synchronized boolean optimizeMergesPending() {
2583     for (final MergePolicy.OneMerge merge : pendingMerges) {
2584       if (merge.optimize)
2585         return true;
2586     }
2587     
2588     for (final MergePolicy.OneMerge merge : runningMerges) {
2589       if (merge.optimize)
2590         return true;
2591     }
2592     
2593     return false;
2594   }
2595
2596   /** Just like {@link #expungeDeletes()}, except you can
2597    *  specify whether the call should block until the
2598    *  operation completes.  This is only meaningful with a
2599    *  {@link MergeScheduler} that is able to run merges in
2600    *  background threads.
2601    *
2602    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2603    * you should immediately close the writer.  See <a
2604    * href="#OOME">above</a> for details.</p>
2605    *
2606    * <p><b>NOTE</b>: if you call {@link #close(boolean)}
2607    * with <tt>false</tt>, which aborts all running merges,
2608    * then any thread still running this method might hit a
2609    * {@link MergePolicy.MergeAbortedException}.
2610    */
2611   public void expungeDeletes(boolean doWait)
2612     throws CorruptIndexException, IOException {
2613     ensureOpen();
2614
2615     flush(true, true);
2616
2617     if (infoStream != null)
2618       message("expungeDeletes: index now " + segString());
2619
2620     MergePolicy.MergeSpecification spec;
2621
2622     synchronized(this) {
2623       spec = mergePolicy.findMergesToExpungeDeletes(segmentInfos);
2624       if (spec != null) {
2625         final int numMerges = spec.merges.size();
2626         for(int i=0;i<numMerges;i++)
2627           registerMerge(spec.merges.get(i));
2628       }
2629     }
2630
2631     mergeScheduler.merge(this);
2632
2633     if (spec != null && doWait) {
2634       final int numMerges = spec.merges.size();
2635       synchronized(this) {
2636         boolean running = true;
2637         while(running) {
2638
2639           if (hitOOM) {
2640             throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete expungeDeletes");
2641           }
2642
2643           // Check each merge that MergePolicy asked us to
2644           // do, to see if any of them are still running and
2645           // if any of them have hit an exception.
2646           running = false;
2647           for(int i=0;i<numMerges;i++) {
2648             final MergePolicy.OneMerge merge = spec.merges.get(i);
2649             if (pendingMerges.contains(merge) || runningMerges.contains(merge))
2650               running = true;
2651             Throwable t = merge.getException();
2652             if (t != null) {
2653               IOException ioe = new IOException("background merge hit exception: " + merge.segString(directory));
2654               ioe.initCause(t);
2655               throw ioe;
2656             }
2657           }
2658
2659           // If any of our merges are still running, wait:
2660           if (running)
2661             doWait();
2662         }
2663       }
2664     }
2665
2666     // NOTE: in the ConcurrentMergeScheduler case, when
2667     // doWait is false, we can return immediately while
2668     // background threads accomplish the optimization
2669   }
2670
2671
2672   /** Expunges all deletes from the index.  When an index
2673    *  has many document deletions (or updates to existing
2674    *  documents), it's best to either call optimize or
2675    *  expungeDeletes to remove all unused data in the index
2676    *  associated with the deleted documents.  To see how
2677    *  many deletions you have pending in your index, call
2678    *  {@link IndexReader#numDeletedDocs}
2679    *  This saves disk space and memory usage while
2680    *  searching.  expungeDeletes should be somewhat faster
2681    *  than optimize since it does not insist on reducing the
2682    *  index to a single segment (though, this depends on the
2683    *  {@link MergePolicy}; see {@link
2684    *  MergePolicy#findMergesToExpungeDeletes}.). Note that
2685    *  this call does not first commit any buffered
2686    *  documents, so you must do so yourself if necessary.
2687    *  See also {@link #expungeDeletes(boolean)}
2688    *
2689    *  <p><b>NOTE</b>: this method first flushes a new
2690    *  segment (if there are indexed documents), and applies
2691    *  all buffered deletes.
2692    *
2693    *  <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2694    *  you should immediately close the writer.  See <a
2695    *  href="#OOME">above</a> for details.</p>
2696    */
2697   public void expungeDeletes() throws CorruptIndexException, IOException {
2698     expungeDeletes(true);
2699   }
2700
2701   /**
2702    * Expert: asks the mergePolicy whether any merges are
2703    * necessary now and if so, runs the requested merges and
2704    * then iterate (test again if merges are needed) until no
2705    * more merges are returned by the mergePolicy.
2706    *
2707    * Explicit calls to maybeMerge() are usually not
2708    * necessary. The most common case is when merge policy
2709    * parameters have changed.
2710    *
2711    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
2712    * you should immediately close the writer.  See <a
2713    * href="#OOME">above</a> for details.</p>
2714    */
2715   public final void maybeMerge() throws CorruptIndexException, IOException {
2716     maybeMerge(false);
2717   }
2718
2719   private final void maybeMerge(boolean optimize) throws CorruptIndexException, IOException {
2720     maybeMerge(1, optimize);
2721   }
2722
2723   private final void maybeMerge(int maxNumSegmentsOptimize, boolean optimize) throws CorruptIndexException, IOException {
2724     updatePendingMerges(maxNumSegmentsOptimize, optimize);
2725     mergeScheduler.merge(this);
2726   }
2727
2728   private synchronized void updatePendingMerges(int maxNumSegmentsOptimize, boolean optimize)
2729     throws CorruptIndexException, IOException {
2730     assert !optimize || maxNumSegmentsOptimize > 0;
2731
2732     if (stopMerges) {
2733       return;
2734     }
2735
2736     // Do not start new merges if we've hit OOME
2737     if (hitOOM) {
2738       return;
2739     }
2740
2741     final MergePolicy.MergeSpecification spec;
2742     if (optimize) {
2743       spec = mergePolicy.findMergesForOptimize(segmentInfos, maxNumSegmentsOptimize, Collections.unmodifiableMap(segmentsToOptimize));
2744       if (spec != null) {
2745         final int numMerges = spec.merges.size();
2746         for(int i=0;i<numMerges;i++) {
2747           final MergePolicy.OneMerge merge = spec.merges.get(i);
2748           merge.optimize = true;
2749           merge.maxNumSegmentsOptimize = maxNumSegmentsOptimize;
2750         }
2751       }
2752
2753     } else {
2754       spec = mergePolicy.findMerges(segmentInfos);
2755     }
2756
2757     if (spec != null) {
2758       final int numMerges = spec.merges.size();
2759       for(int i=0;i<numMerges;i++) {
2760         registerMerge(spec.merges.get(i));
2761       }
2762     }
2763   }
2764
2765   /** Expert: to be used by a {@link MergePolicy} to avoid
2766    *  selecting merges for segments already being merged.
2767    *  The returned collection is not cloned, and thus is
2768    *  only safe to access if you hold IndexWriter's lock
2769    *  (which you do when IndexWriter invokes the
2770    *  MergePolicy).
2771    *
2772    *  <p>Do not alter the returned collection! */
2773   public synchronized Collection<SegmentInfo> getMergingSegments() {
2774     return mergingSegments;
2775   }
2776
2777   /** Expert: the {@link MergeScheduler} calls this method
2778    *  to retrieve the next merge requested by the
2779    *  MergePolicy
2780    *
2781    * @lucene.experimental
2782    */
2783   public synchronized MergePolicy.OneMerge getNextMerge() {
2784     if (pendingMerges.size() == 0)
2785       return null;
2786     else {
2787       // Advance the merge from pending to running
2788       MergePolicy.OneMerge merge = pendingMerges.removeFirst();
2789       runningMerges.add(merge);
2790       return merge;
2791     }
2792   }
2793
2794   /**
2795    * Close the <code>IndexWriter</code> without committing
2796    * any changes that have occurred since the last commit
2797    * (or since it was opened, if commit hasn't been called).
2798    * This removes any temporary files that had been created,
2799    * after which the state of the index will be the same as
2800    * it was when commit() was last called or when this
2801    * writer was first opened.  This also clears a previous
2802    * call to {@link #prepareCommit}.
2803    * @throws IOException if there is a low-level IO error
2804    */
2805   public void rollback() throws IOException {
2806     ensureOpen();
2807
2808     // Ensure that only one thread actually gets to do the closing:
2809     if (shouldClose())
2810       rollbackInternal();
2811   }
2812
2813   private void rollbackInternal() throws IOException {
2814
2815     boolean success = false;
2816
2817     if (infoStream != null ) {
2818       message("rollback");
2819     }
2820
2821     try {
2822       synchronized(this) {
2823         finishMerges(false);
2824         stopMerges = true;
2825       }
2826
2827       if (infoStream != null ) {
2828         message("rollback: done finish merges");
2829       }
2830
2831       // Must pre-close these two, in case they increment
2832       // changeCount so that we can then set it to false
2833       // before calling closeInternal
2834       mergePolicy.close();
2835       mergeScheduler.close();
2836
2837       bufferedDeletesStream.clear();
2838
2839       synchronized(this) {
2840
2841         if (pendingCommit != null) {
2842           pendingCommit.rollbackCommit(directory);
2843           deleter.decRef(pendingCommit);
2844           pendingCommit = null;
2845           notifyAll();
2846         }
2847
2848         // Keep the same segmentInfos instance but replace all
2849         // of its SegmentInfo instances.  This is so the next
2850         // attempt to commit using this instance of IndexWriter
2851         // will always write to a new generation ("write
2852         // once").
2853         segmentInfos.rollbackSegmentInfos(rollbackSegments);
2854         if (infoStream != null ) {
2855           message("rollback: infos=" + segString(segmentInfos));
2856         }
2857
2858         docWriter.abort();
2859
2860         assert testPoint("rollback before checkpoint");
2861
2862         // Ask deleter to locate unreferenced files & remove
2863         // them:
2864         deleter.checkpoint(segmentInfos, false);
2865         deleter.refresh();
2866       }
2867
2868       // Don't bother saving any changes in our segmentInfos
2869       readerPool.clear(null);
2870
2871       lastCommitChangeCount = changeCount;
2872
2873       success = true;
2874     } catch (OutOfMemoryError oom) {
2875       handleOOM(oom, "rollbackInternal");
2876     } finally {
2877       synchronized(this) {
2878         if (!success) {
2879           closing = false;
2880           notifyAll();
2881           if (infoStream != null)
2882             message("hit exception during rollback");
2883         }
2884       }
2885     }
2886
2887     closeInternal(false);
2888   }
2889
2890   /**
2891    * Delete all documents in the index.
2892    *
2893    * <p>This method will drop all buffered documents and will 
2894    *    remove all segments from the index. This change will not be
2895    *    visible until a {@link #commit()} has been called. This method
2896    *    can be rolled back using {@link #rollback()}.</p>
2897    *
2898    * <p>NOTE: this method is much faster than using deleteDocuments( new MatchAllDocsQuery() ).</p>
2899    *
2900    * <p>NOTE: this method will forcefully abort all merges
2901    *    in progress.  If other threads are running {@link
2902    *    #optimize()}, {@link #addIndexes(IndexReader[])} or
2903    *    {@link #expungeDeletes} methods, they may receive
2904    *    {@link MergePolicy.MergeAbortedException}s.
2905    */
2906   public synchronized void deleteAll() throws IOException {
2907     try {
2908
2909       // Abort any running merges
2910       finishMerges(false);
2911
2912       // Remove any buffered docs
2913       docWriter.abort();
2914
2915       // Remove all segments
2916       segmentInfos.clear();
2917
2918       // Ask deleter to locate unreferenced files & remove them:
2919       deleter.checkpoint(segmentInfos, false);
2920       deleter.refresh();
2921
2922       // Don't bother saving any changes in our segmentInfos
2923       readerPool.dropAll();
2924
2925       // Mark that the index has changed
2926       ++changeCount;
2927       segmentInfos.changed();
2928     } catch (OutOfMemoryError oom) {
2929       handleOOM(oom, "deleteAll");
2930     } finally {
2931       if (infoStream != null) {
2932         message("hit exception during deleteAll");
2933       }
2934     }
2935   }
2936
2937   private synchronized void finishMerges(boolean waitForMerges) throws IOException {
2938     if (!waitForMerges) {
2939
2940       stopMerges = true;
2941
2942       // Abort all pending & running merges:
2943       for (final MergePolicy.OneMerge merge : pendingMerges) {
2944         if (infoStream != null)
2945           message("now abort pending merge " + merge.segString(directory));
2946         merge.abort();
2947         mergeFinish(merge);
2948       }
2949       pendingMerges.clear();
2950       
2951       for (final MergePolicy.OneMerge merge : runningMerges) {
2952         if (infoStream != null)
2953           message("now abort running merge " + merge.segString(directory));
2954         merge.abort();
2955       }
2956
2957       // These merges periodically check whether they have
2958       // been aborted, and stop if so.  We wait here to make
2959       // sure they all stop.  It should not take very long
2960       // because the merge threads periodically check if
2961       // they are aborted.
2962       while(runningMerges.size() > 0) {
2963         if (infoStream != null)
2964           message("now wait for " + runningMerges.size() + " running merge to abort");
2965         doWait();
2966       }
2967
2968       stopMerges = false;
2969       notifyAll();
2970
2971       assert 0 == mergingSegments.size();
2972
2973       if (infoStream != null)
2974         message("all running merges have aborted");
2975
2976     } else {
2977       // waitForMerges() will ensure any running addIndexes finishes.  
2978       // It's fine if a new one attempts to start because from our
2979       // caller above the call will see that we are in the
2980       // process of closing, and will throw an
2981       // AlreadyClosedException.
2982       waitForMerges();
2983     }
2984   }
2985
2986   /**
2987    * Wait for any currently outstanding merges to finish.
2988    *
2989    * <p>It is guaranteed that any merges started prior to calling this method 
2990    *    will have completed once this method completes.</p>
2991    */
2992   public synchronized void waitForMerges() {
2993     if (infoStream != null) {
2994       message("waitForMerges");
2995     }
2996     while(pendingMerges.size() > 0 || runningMerges.size() > 0) {
2997       doWait();
2998     }
2999
3000     // sanity check
3001     assert 0 == mergingSegments.size();
3002
3003     if (infoStream != null) {
3004       message("waitForMerges done");
3005     }
3006   }
3007
3008   /**
3009    * Called whenever the SegmentInfos has been updated and
3010    * the index files referenced exist (correctly) in the
3011    * index directory.
3012    */
3013   synchronized void checkpoint() throws IOException {
3014     changeCount++;
3015     segmentInfos.changed();
3016     deleter.checkpoint(segmentInfos, false);
3017   }
3018
3019   private synchronized void resetMergeExceptions() {
3020     mergeExceptions = new ArrayList<MergePolicy.OneMerge>();
3021     mergeGen++;
3022   }
3023
3024   private void noDupDirs(Directory... dirs) {
3025     HashSet<Directory> dups = new HashSet<Directory>();
3026     for (Directory dir : dirs) {
3027       if (dups.contains(dir))
3028         throw new IllegalArgumentException("Directory " + dir + " appears more than once");
3029       if (dir == directory)
3030         throw new IllegalArgumentException("Cannot add directory to itself");
3031       dups.add(dir);
3032     }
3033   }
3034
3035   /**
3036    * @deprecated use {@link #addIndexes(Directory...)} instead
3037    */
3038   @Deprecated
3039   public void addIndexesNoOptimize(Directory... dirs)
3040       throws CorruptIndexException, IOException {
3041     addIndexes(dirs);
3042   }
3043
3044   /** 
3045    * Merges the provided indexes into this index. This method is useful 
3046    * if you use extensions of {@link IndexReader}. Otherwise, using 
3047    * {@link #addIndexes(Directory...)} is highly recommended for performance 
3048    * reasons. It uses the {@link MergeScheduler} and {@link MergePolicy} set 
3049    * on this writer, which may perform merges in parallel.
3050    * 
3051    * <p>The provided IndexReaders are not closed.
3052    *
3053    * <p><b>NOTE:</b> this method does not merge the current segments, 
3054    * only the incoming ones.
3055    * 
3056    * <p>See {@link #addIndexes(Directory...)} for details on transactional 
3057    * semantics, temporary free space required in the Directory, 
3058    * and non-CFS segments on an Exception.
3059    *
3060    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
3061    * you should immediately close the writer.  See <a
3062    * href="#OOME">above</a> for details.
3063    *
3064    * <p><b>NOTE</b>: if you call {@link #close(boolean)}
3065    * with <tt>false</tt>, which aborts all running merges,
3066    * then any thread still running this method might hit a
3067    * {@link MergePolicy.MergeAbortedException}.
3068    *
3069    * @throws CorruptIndexException if the index is corrupt
3070    * @throws IOException if there is a low-level IO error
3071    */
3072   public void addIndexes(IndexReader... readers) throws CorruptIndexException, IOException {
3073
3074     ensureOpen();
3075
3076     try {
3077       if (infoStream != null)
3078         message("flush at addIndexes(IndexReader...)");
3079       flush(false, true);
3080
3081       String mergedName = newSegmentName();
3082       // TODO: somehow we should fix this merge so it's
3083       // abortable so that IW.close(false) is able to stop it
3084       SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(),
3085                                                mergedName, null, payloadProcessorProvider,
3086                                                ((FieldInfos) docWriter.getFieldInfos().clone()));
3087       
3088       for (IndexReader reader : readers)      // add new indexes
3089         merger.add(reader);
3090       
3091       int docCount = merger.merge();                // merge 'em
3092       
3093       SegmentInfo info = new SegmentInfo(mergedName, docCount, directory,
3094                                          false, true,
3095                                          merger.fieldInfos().hasProx(),
3096                                          merger.fieldInfos().hasVectors());
3097       setDiagnostics(info, "addIndexes(IndexReader...)");
3098
3099       boolean useCompoundFile;
3100       synchronized(this) { // Guard segmentInfos
3101         if (stopMerges) {
3102           deleter.deleteNewFiles(info.files());
3103           return;
3104         }
3105         ensureOpen();
3106         useCompoundFile = mergePolicy.useCompoundFile(segmentInfos, info);
3107       }
3108
3109       // Now create the compound file if needed
3110       if (useCompoundFile) {
3111         merger.createCompoundFile(mergedName + ".cfs", info);
3112
3113         // delete new non cfs files directly: they were never
3114         // registered with IFD
3115         synchronized(this) {
3116           deleter.deleteNewFiles(info.files());
3117         }
3118         info.setUseCompoundFile(true);
3119       }
3120
3121       // Register the new segment
3122       synchronized(this) {
3123         if (stopMerges) {
3124           deleter.deleteNewFiles(info.files());
3125           return;
3126         }
3127         ensureOpen();
3128         segmentInfos.add(info);
3129         checkpoint();
3130       }
3131       
3132     } catch (OutOfMemoryError oom) {
3133       handleOOM(oom, "addIndexes(IndexReader...)");
3134     }
3135   }
3136
3137   /**
3138    * Adds all segments from an array of indexes into this index.
3139    *
3140    * <p>This may be used to parallelize batch indexing. A large document
3141    * collection can be broken into sub-collections. Each sub-collection can be
3142    * indexed in parallel, on a different thread, process or machine. The
3143    * complete index can then be created by merging sub-collection indexes
3144    * with this method.
3145    *
3146    * <p>
3147    * <b>NOTE:</b> the index in each {@link Directory} must not be
3148    * changed (opened by a writer) while this method is
3149    * running.  This method does not acquire a write lock in
3150    * each input Directory, so it is up to the caller to
3151    * enforce this.
3152    *
3153    * <p>This method is transactional in how Exceptions are
3154    * handled: it does not commit a new segments_N file until
3155    * all indexes are added.  This means if an Exception
3156    * occurs (for example disk full), then either no indexes
3157    * will have been added or they all will have been.
3158    *
3159    * <p>Note that this requires temporary free space in the
3160    * {@link Directory} up to 2X the sum of all input indexes
3161    * (including the starting index). If readers/searchers
3162    * are open against the starting index, then temporary
3163    * free space required will be higher by the size of the
3164    * starting index (see {@link #optimize()} for details).
3165    *
3166    * <p>
3167    * <b>NOTE:</b> this method only copies the segments of the incomning indexes
3168    * and does not merge them. Therefore deleted documents are not removed and
3169    * the new segments are not merged with the existing ones. Also, if the merge 
3170    * policy allows compound files, then any segment that is not compound is 
3171    * converted to such. However, if the segment is compound, it is copied as-is
3172    * even if the merge policy does not allow compound files.
3173    * 
3174    * <p>
3175    * <p>This requires this index not be among those to be added.
3176    *
3177    * <p>
3178    * <b>NOTE</b>: if this method hits an OutOfMemoryError
3179    * you should immediately close the writer. See <a
3180    * href="#OOME">above</a> for details.
3181    *
3182    * @throws CorruptIndexException if the index is corrupt
3183    * @throws IOException if there is a low-level IO error
3184    */
3185   public void addIndexes(Directory... dirs) throws CorruptIndexException, IOException {
3186     ensureOpen();
3187     
3188     noDupDirs(dirs);
3189
3190     try {
3191       if (infoStream != null)
3192         message("flush at addIndexes(Directory...)");
3193       flush(false, true);
3194       
3195       int docCount = 0;
3196       List<SegmentInfo> infos = new ArrayList<SegmentInfo>();
3197       Comparator<String> versionComparator = StringHelper.getVersionComparator();
3198       for (Directory dir : dirs) {
3199         if (infoStream != null) {
3200           message("addIndexes: process directory " + dir);
3201         }
3202         SegmentInfos sis = new SegmentInfos(); // read infos from dir
3203         sis.read(dir);
3204         final Set<String> dsFilesCopied = new HashSet<String>();
3205         final Map<String, String> dsNames = new HashMap<String, String>();
3206         for (SegmentInfo info : sis) {
3207           assert !infos.contains(info): "dup info dir=" + info.dir + " name=" + info.name;
3208           
3209           docCount += info.docCount;
3210           String newSegName = newSegmentName();
3211           String dsName = info.getDocStoreSegment();
3212           
3213           if (infoStream != null) {
3214             message("addIndexes: process segment origName=" + info.name + " newName=" + newSegName + " dsName=" + dsName + " info=" + info);
3215           }
3216           
3217           // create CFS only if the source segment is not CFS, and MP agrees it
3218           // should be CFS.
3219           boolean createCFS;
3220           synchronized (this) { // Guard segmentInfos
3221             createCFS = !info.getUseCompoundFile()
3222                 && mergePolicy.useCompoundFile(segmentInfos, info)
3223                 // optimize case only for segments that don't share doc stores
3224                 && versionComparator.compare(info.getVersion(), "3.1") >= 0;
3225           }
3226
3227           if (createCFS) {
3228             copySegmentIntoCFS(info, newSegName);
3229           } else {
3230             copySegmentAsIs(info, newSegName, dsNames, dsFilesCopied);
3231           }
3232           infos.add(info);
3233         }
3234       }      
3235
3236       synchronized (this) {
3237         ensureOpen();
3238         segmentInfos.addAll(infos);
3239         checkpoint();
3240       }
3241       
3242     } catch (OutOfMemoryError oom) {
3243       handleOOM(oom, "addIndexes(Directory...)");
3244     }
3245   }
3246
3247   /** Copies the segment into the IndexWriter's directory, as a compound segment. */
3248   private void copySegmentIntoCFS(SegmentInfo info, String segName) throws IOException {
3249     String segFileName = IndexFileNames.segmentFileName(segName, IndexFileNames.COMPOUND_FILE_EXTENSION);
3250     Collection<String> files = info.files();
3251     CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, segFileName);
3252     for (String file : files) {
3253       String newFileName = segName + IndexFileNames.stripSegmentName(file);
3254       if (!IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
3255           && !IndexFileNames.isSeparateNormsFile(file)) {
3256         cfsWriter.addFile(file, info.dir);
3257       } else {
3258         assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists";
3259         info.dir.copy(directory, file, newFileName);
3260       }
3261     }
3262     
3263     // Create the .cfs
3264     cfsWriter.close();
3265     
3266     info.dir = directory;
3267     info.name = segName;
3268     info.setUseCompoundFile(true);
3269   }
3270   
3271   /** Copies the segment files as-is into the IndexWriter's directory. */
3272   private void copySegmentAsIs(SegmentInfo info, String segName,
3273       Map<String, String> dsNames, Set<String> dsFilesCopied)
3274       throws IOException {
3275     // Determine if the doc store of this segment needs to be copied. It's
3276     // only relevant for segments that share doc store with others,
3277     // because the DS might have been copied already, in which case we
3278     // just want to update the DS name of this SegmentInfo.
3279     // NOTE: pre-3x segments include a null DSName if they don't share doc
3280     // store. The following code ensures we don't accidentally insert
3281     // 'null' to the map.
3282     String dsName = info.getDocStoreSegment();
3283     final String newDsName;
3284     if (dsName != null) {
3285       if (dsNames.containsKey(dsName)) {
3286         newDsName = dsNames.get(dsName);
3287       } else {
3288         dsNames.put(dsName, segName);
3289         newDsName = segName;
3290       }
3291     } else {
3292       newDsName = segName;
3293     }
3294     
3295     // Copy the segment files
3296     for (String file: info.files()) {
3297       final String newFileName;
3298       if (IndexFileNames.isDocStoreFile(file)) {
3299         newFileName = newDsName + IndexFileNames.stripSegmentName(file);
3300         if (dsFilesCopied.contains(newFileName)) {
3301           continue;
3302         }
3303         dsFilesCopied.add(newFileName);
3304       } else {
3305         newFileName = segName + IndexFileNames.stripSegmentName(file);
3306       }
3307       
3308       assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists";
3309       info.dir.copy(directory, file, newFileName);
3310     }
3311     
3312     info.setDocStore(info.getDocStoreOffset(), newDsName, info.getDocStoreIsCompoundFile());
3313     info.dir = directory;
3314     info.name = segName;
3315   }
3316   
3317   /**
3318    * A hook for extending classes to execute operations after pending added and
3319    * deleted documents have been flushed to the Directory but before the change
3320    * is committed (new segments_N file written).
3321    */
3322   protected void doAfterFlush() throws IOException {}
3323
3324   /**
3325    * A hook for extending classes to execute operations before pending added and
3326    * deleted documents are flushed to the Directory.
3327    */
3328   protected void doBeforeFlush() throws IOException {}
3329
3330   /** Expert: prepare for commit.
3331    *
3332    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
3333    * you should immediately close the writer.  See <a
3334    * href="#OOME">above</a> for details.</p>
3335    *
3336    * @see #prepareCommit(Map) */
3337   public final void prepareCommit() throws CorruptIndexException, IOException {
3338     ensureOpen();
3339     prepareCommit(null);
3340   }
3341
3342   /** <p>Expert: prepare for commit, specifying
3343    *  commitUserData Map (String -> String).  This does the
3344    *  first phase of 2-phase commit. This method does all
3345    *  steps necessary to commit changes since this writer
3346    *  was opened: flushes pending added and deleted docs,
3347    *  syncs the index files, writes most of next segments_N
3348    *  file.  After calling this you must call either {@link
3349    *  #commit()} to finish the commit, or {@link
3350    *  #rollback()} to revert the commit and undo all changes
3351    *  done since the writer was opened.</p>
3352    * 
3353    *  You can also just call {@link #commit(Map)} directly
3354    *  without prepareCommit first in which case that method
3355    *  will internally call prepareCommit.
3356    *
3357    *  <p><b>NOTE</b>: if this method hits an OutOfMemoryError
3358    *  you should immediately close the writer.  See <a
3359    *  href="#OOME">above</a> for details.</p>
3360    *
3361    *  @param commitUserData Opaque Map (String->String)
3362    *  that's recorded into the segments file in the index,
3363    *  and retrievable by {@link
3364    *  IndexReader#getCommitUserData}.  Note that when
3365    *  IndexWriter commits itself during {@link #close}, the
3366    *  commitUserData is unchanged (just carried over from
3367    *  the prior commit).  If this is null then the previous
3368    *  commitUserData is kept.  Also, the commitUserData will
3369    *  only "stick" if there are actually changes in the
3370    *  index to commit.
3371    */
3372   public final void prepareCommit(Map<String, String> commitUserData)
3373       throws CorruptIndexException, IOException {
3374
3375     if (hitOOM) {
3376       throw new IllegalStateException(
3377           "this writer hit an OutOfMemoryError; cannot commit");
3378     }
3379
3380     if (pendingCommit != null)
3381       throw new IllegalStateException(
3382           "prepareCommit was already called with no corresponding call to commit");
3383
3384     if (infoStream != null)
3385       message("prepareCommit: flush");
3386
3387     ensureOpen(false);
3388     boolean anySegmentsFlushed = false;
3389     SegmentInfos toCommit = null;
3390     boolean success = false;
3391     try {
3392       try {
3393         synchronized (this) {
3394           anySegmentsFlushed = doFlush(true);
3395           readerPool.commit(segmentInfos);
3396           toCommit = (SegmentInfos) segmentInfos.clone();
3397           pendingCommitChangeCount = changeCount;
3398           // This protects the segmentInfos we are now going
3399           // to commit. This is important in case, eg, while
3400           // we are trying to sync all referenced files, a
3401           // merge completes which would otherwise have
3402           // removed the files we are now syncing.
3403           deleter.incRef(toCommit, false);
3404         }
3405         success = true;
3406       } finally {
3407         if (!success && infoStream != null) {
3408           message("hit exception during prepareCommit");
3409         }
3410         doAfterFlush();
3411       }
3412     } catch (OutOfMemoryError oom) {
3413       handleOOM(oom, "prepareCommit");
3414     }
3415
3416     success = false;
3417     try {
3418       if (anySegmentsFlushed) {
3419         maybeMerge();
3420       } 
3421       success = true;
3422     } finally {
3423       if (!success) {
3424         synchronized (this) {
3425           deleter.decRef(toCommit);
3426         }
3427       }
3428     }
3429
3430     startCommit(toCommit, commitUserData);
3431   }
3432
3433   // Used only by commit, below; lock order is commitLock -> IW
3434   private final Object commitLock = new Object();
3435
3436   /**
3437    * <p>Commits all pending changes (added & deleted
3438    * documents, optimizations, segment merges, added
3439    * indexes, etc.) to the index, and syncs all referenced
3440    * index files, such that a reader will see the changes
3441    * and the index updates will survive an OS or machine
3442    * crash or power loss.  Note that this does not wait for
3443    * any running background merges to finish.  This may be a
3444    * costly operation, so you should test the cost in your
3445    * application and do it only when really necessary.</p>
3446    *
3447    * <p> Note that this operation calls Directory.sync on
3448    * the index files.  That call should not return until the
3449    * file contents & metadata are on stable storage.  For
3450    * FSDirectory, this calls the OS's fsync.  But, beware:
3451    * some hardware devices may in fact cache writes even
3452    * during fsync, and return before the bits are actually
3453    * on stable storage, to give the appearance of faster
3454    * performance.  If you have such a device, and it does
3455    * not have a battery backup (for example) then on power
3456    * loss it may still lose data.  Lucene cannot guarantee
3457    * consistency on such devices.  </p>
3458    *
3459    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
3460    * you should immediately close the writer.  See <a
3461    * href="#OOME">above</a> for details.</p>
3462    *
3463    * @see #prepareCommit
3464    * @see #commit(Map)
3465    */
3466   public final void commit() throws CorruptIndexException, IOException {
3467     commit(null);
3468   }
3469
3470   /** Commits all changes to the index, specifying a
3471    *  commitUserData Map (String -> String).  This just
3472    *  calls {@link #prepareCommit(Map)} (if you didn't
3473    *  already call it) and then {@link #finishCommit}.
3474    *
3475    * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
3476    * you should immediately close the writer.  See <a
3477    * href="#OOME">above</a> for details.</p>
3478    */
3479   public final void commit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
3480
3481     ensureOpen();
3482
3483     commitInternal(commitUserData);
3484   }
3485
3486   private final void commitInternal(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
3487
3488     if (infoStream != null) {
3489       message("commit: start");
3490     }
3491
3492     synchronized(commitLock) {
3493       if (infoStream != null) {
3494         message("commit: enter lock");
3495       }
3496
3497       if (pendingCommit == null) {
3498         if (infoStream != null) {
3499           message("commit: now prepare");
3500         }
3501         prepareCommit(commitUserData);
3502       } else if (infoStream != null) {
3503         message("commit: already prepared");
3504       }
3505
3506       finishCommit();
3507     }
3508   }
3509
3510   private synchronized final void finishCommit() throws CorruptIndexException, IOException {
3511
3512     if (pendingCommit != null) {
3513       try {
3514         if (infoStream != null)
3515           message("commit: pendingCommit != null");
3516         pendingCommit.finishCommit(directory);
3517         if (infoStream != null)
3518           message("commit: wrote segments file \"" + pendingCommit.getCurrentSegmentFileName() + "\"");
3519         lastCommitChangeCount = pendingCommitChangeCount;
3520         segmentInfos.updateGeneration(pendingCommit);
3521         segmentInfos.setUserData(pendingCommit.getUserData());
3522         rollbackSegments = pendingCommit.createBackupSegmentInfos(true);
3523         deleter.checkpoint(pendingCommit, true);
3524       } finally {
3525         // Matches the incRef done in startCommit:
3526         deleter.decRef(pendingCommit);
3527         pendingCommit = null;
3528         notifyAll();
3529       }
3530
3531     } else if (infoStream != null) {
3532       message("commit: pendingCommit == null; skip");
3533     }
3534
3535     if (infoStream != null) {
3536       message("commit: done");
3537     }
3538   }
3539
3540   /** NOTE: flushDocStores is ignored now (hardwired to
3541    *  true); this method is only here for backwards
3542    *  compatibility */
3543   protected final void flush(boolean triggerMerge, boolean flushDocStores, boolean flushDeletes) throws CorruptIndexException, IOException {
3544     flush(triggerMerge, flushDeletes);
3545   }
3546
3547   /**
3548    * Flush all in-memory buffered updates (adds and deletes)
3549    * to the Directory.
3550    * @param triggerMerge if true, we may merge segments (if
3551    *  deletes or docs were flushed) if necessary
3552    * @param applyAllDeletes whether pending deletes should also
3553    */
3554   protected final void flush(boolean triggerMerge, boolean applyAllDeletes) throws CorruptIndexException, IOException {
3555
3556     // NOTE: this method cannot be sync'd because
3557     // maybeMerge() in turn calls mergeScheduler.merge which
3558     // in turn can take a long time to run and we don't want
3559     // to hold the lock for that.  In the case of
3560     // ConcurrentMergeScheduler this can lead to deadlock
3561     // when it stalls due to too many running merges.
3562
3563     // We can be called during close, when closing==true, so we must pass false to ensureOpen:
3564     ensureOpen(false);
3565     if (doFlush(applyAllDeletes) && triggerMerge) {
3566       maybeMerge();
3567     }
3568   }
3569
3570   // TODO: this method should not have to be entirely
3571   // synchronized, ie, merges should be allowed to commit
3572   // even while a flush is happening
3573   private synchronized boolean doFlush(boolean applyAllDeletes) throws CorruptIndexException, IOException {
3574
3575     if (hitOOM) {
3576       throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot flush");
3577     }
3578
3579     doBeforeFlush();
3580
3581     assert testPoint("startDoFlush");
3582
3583     // We may be flushing because it was triggered by doc
3584     // count, del count, ram usage (in which case flush
3585     // pending is already set), or we may be flushing
3586     // due to external event eg getReader or commit is
3587     // called (in which case we now set it, and this will
3588     // pause all threads):
3589     flushControl.setFlushPendingNoWait("explicit flush");
3590
3591     boolean success = false;
3592
3593     try {
3594
3595       if (infoStream != null) {
3596         message("  start flush: applyAllDeletes=" + applyAllDeletes);
3597         message("  index before flush " + segString());
3598       }
3599     
3600       final SegmentInfo newSegment = docWriter.flush(this, deleter, mergePolicy, segmentInfos);
3601       if (newSegment != null) {
3602         setDiagnostics(newSegment, "flush");
3603         segmentInfos.add(newSegment);
3604         checkpoint();
3605       }
3606
3607       if (!applyAllDeletes) {
3608         // If deletes alone are consuming > 1/2 our RAM
3609         // buffer, force them all to apply now. This is to
3610         // prevent too-frequent flushing of a long tail of
3611         // tiny segments:
3612         if (flushControl.getFlushDeletes() ||
3613             (config.getRAMBufferSizeMB() != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
3614              bufferedDeletesStream.bytesUsed() > (1024*1024*config.getRAMBufferSizeMB()/2))) {
3615           applyAllDeletes = true;
3616           if (infoStream != null) {
3617             message("force apply deletes bytesUsed=" + bufferedDeletesStream.bytesUsed() + " vs ramBuffer=" + (1024*1024*config.getRAMBufferSizeMB()));
3618           }
3619         }
3620       }
3621
3622       if (applyAllDeletes) {
3623         if (infoStream != null) {
3624           message("apply all deletes during flush");
3625         }
3626         
3627         flushDeletesCount.incrementAndGet();
3628         final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream
3629           .applyDeletes(readerPool, segmentInfos.asList());
3630         if (result.anyDeletes) {
3631           checkpoint();
3632         }
3633         if (!keepFullyDeletedSegments && result.allDeleted != null) {
3634           if (infoStream != null) {
3635             message("drop 100% deleted segments: " + result.allDeleted);
3636           }
3637           for (SegmentInfo info : result.allDeleted) {
3638             // If a merge has already registered for this
3639             // segment, we leave it in the readerPool; the
3640             // merge will skip merging it and will then drop
3641             // it once it's done:
3642             if (!mergingSegments.contains(info)) {
3643               segmentInfos.remove(info);
3644               if (readerPool != null) {
3645                 readerPool.drop(info);
3646               }
3647             }
3648           }
3649           checkpoint();
3650         }
3651         bufferedDeletesStream.prune(segmentInfos);
3652
3653         assert !bufferedDeletesStream.any();
3654         flushControl.clearDeletes();
3655       } else if (infoStream != null) {
3656         message("don't apply deletes now delTermCount=" + bufferedDeletesStream.numTerms() + " bytesUsed=" + bufferedDeletesStream.bytesUsed());
3657       }
3658       
3659
3660       doAfterFlush();
3661       flushCount.incrementAndGet();
3662
3663       success = true;
3664
3665       return newSegment != null;
3666
3667     } catch (OutOfMemoryError oom) {
3668       handleOOM(oom, "doFlush");
3669       // never hit
3670       return false;
3671     } finally {
3672       flushControl.clearFlushPending();
3673       if (!success && infoStream != null)
3674         message("hit exception during flush");
3675     }
3676   }
3677
3678   /** Expert:  Return the total size of all index files currently cached in memory.
3679    * Useful for size management with flushRamDocs()
3680    */
3681   public final long ramSizeInBytes() {
3682     ensureOpen();
3683     return docWriter.bytesUsed() + bufferedDeletesStream.bytesUsed();
3684   }
3685
3686   /** Expert:  Return the number of documents currently
3687    *  buffered in RAM. */
3688   public final synchronized int numRamDocs() {
3689     ensureOpen();
3690     return docWriter.getNumDocs();
3691   }
3692
3693   private void ensureValidMerge(MergePolicy.OneMerge merge) throws IOException {
3694     for(SegmentInfo info : merge.segments) {
3695       if (!segmentInfos.contains(info)) {
3696         throw new MergePolicy.MergeException("MergePolicy selected a segment (" + info.name + ") that is not in the current index " + segString(), directory);
3697       }
3698     }
3699   }
3700
3701   /** Carefully merges deletes for the segments we just
3702    *  merged.  This is tricky because, although merging will
3703    *  clear all deletes (compacts the documents), new
3704    *  deletes may have been flushed to the segments since
3705    *  the merge was started.  This method "carries over"
3706    *  such new deletes onto the newly merged segment, and
3707    *  saves the resulting deletes file (incrementing the
3708    *  delete generation for merge.info).  If no deletes were
3709    *  flushed, no new deletes file is saved. */
3710   synchronized private void commitMergedDeletes(MergePolicy.OneMerge merge, SegmentReader mergedReader) throws IOException {
3711
3712     assert testPoint("startCommitMergeDeletes");
3713
3714     final List<SegmentInfo> sourceSegments = merge.segments;
3715
3716     if (infoStream != null)
3717       message("commitMergeDeletes " + merge.segString(directory));
3718
3719     // Carefully merge deletes that occurred after we
3720     // started merging:
3721     int docUpto = 0;
3722     int delCount = 0;
3723     long minGen = Long.MAX_VALUE;
3724
3725     for(int i=0; i < sourceSegments.size(); i++) {
3726       SegmentInfo info = sourceSegments.get(i);
3727       minGen = Math.min(info.getBufferedDeletesGen(), minGen);
3728       int docCount = info.docCount;
3729       final SegmentReader previousReader = merge.readerClones.get(i);
3730       if (previousReader == null) {
3731         // Reader was skipped because it was 100% deletions
3732         continue;
3733       }
3734       final SegmentReader currentReader = merge.readers.get(i);
3735       if (previousReader.hasDeletions()) {
3736
3737         // There were deletes on this segment when the merge
3738         // started.  The merge has collapsed away those
3739         // deletes, but, if new deletes were flushed since
3740         // the merge started, we must now carefully keep any
3741         // newly flushed deletes but mapping them to the new
3742         // docIDs.
3743
3744         if (currentReader.numDeletedDocs() > previousReader.numDeletedDocs()) {
3745           // This means this segment has had new deletes
3746           // committed since we started the merge, so we
3747           // must merge them:
3748           for(int j=0;j<docCount;j++) {
3749             if (previousReader.isDeleted(j))
3750               assert currentReader.isDeleted(j);
3751             else {
3752               if (currentReader.isDeleted(j)) {
3753                 mergedReader.doDelete(docUpto);
3754                 delCount++;
3755               }
3756               docUpto++;
3757             }
3758           }
3759         } else {
3760           docUpto += docCount - previousReader.numDeletedDocs();
3761         }
3762       } else if (currentReader.hasDeletions()) {
3763         // This segment had no deletes before but now it
3764         // does:
3765         for(int j=0; j<docCount; j++) {
3766           if (currentReader.isDeleted(j)) {
3767             mergedReader.doDelete(docUpto);
3768             delCount++;
3769           }
3770           docUpto++;
3771         }
3772       } else
3773         // No deletes before or after
3774         docUpto += info.docCount;
3775     }
3776
3777     assert mergedReader.numDeletedDocs() == delCount;
3778
3779     mergedReader.hasChanges = delCount > 0;
3780
3781     // If new deletes were applied while we were merging
3782     // (which happens if eg commit() or getReader() is
3783     // called during our merge), then it better be the case
3784     // that the delGen has increased for all our merged
3785     // segments:
3786     assert !mergedReader.hasChanges || minGen > mergedReader.getSegmentInfo().getBufferedDeletesGen();
3787
3788     mergedReader.getSegmentInfo().setBufferedDeletesGen(minGen);
3789   }
3790
3791   synchronized private boolean commitMerge(MergePolicy.OneMerge merge, SegmentReader mergedReader) throws IOException {
3792
3793     assert testPoint("startCommitMerge");
3794
3795     if (hitOOM) {
3796       throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete merge");
3797     }
3798
3799     if (infoStream != null)
3800       message("commitMerge: " + merge.segString(directory) + " index=" + segString());
3801
3802     assert merge.registerDone;
3803
3804     // If merge was explicitly aborted, or, if rollback() or
3805     // rollbackTransaction() had been called since our merge
3806     // started (which results in an unqualified
3807     // deleter.refresh() call that will remove any index
3808     // file that current segments does not reference), we
3809     // abort this merge
3810     if (merge.isAborted()) {
3811       if (infoStream != null)
3812         message("commitMerge: skipping merge " + merge.segString(directory) + ": it was aborted");
3813       return false;
3814     }
3815
3816     commitMergedDeletes(merge, mergedReader);
3817       
3818     // If the doc store we are using has been closed and
3819     // is in now compound format (but wasn't when we
3820     // started), then we will switch to the compound
3821     // format as well:
3822
3823     assert !segmentInfos.contains(merge.info);
3824
3825     final boolean allDeleted = mergedReader.numDocs() == 0;
3826
3827     if (infoStream != null && allDeleted) {
3828       message("merged segment " + merge.info + " is 100% deleted" +  (keepFullyDeletedSegments ? "" : "; skipping insert"));
3829     }
3830
3831     final boolean dropSegment = allDeleted && !keepFullyDeletedSegments;
3832     segmentInfos.applyMergeChanges(merge, dropSegment);
3833     
3834     if (dropSegment) {
3835       readerPool.drop(merge.info);
3836     }
3837     
3838     if (infoStream != null) {
3839       message("after commit: " + segString());
3840     }
3841
3842     closeMergeReaders(merge, false);
3843
3844     // Must note the change to segmentInfos so any commits
3845     // in-flight don't lose it:
3846     checkpoint();
3847
3848     // If the merged segments had pending changes, clear
3849     // them so that they don't bother writing them to
3850     // disk, updating SegmentInfo, etc.:
3851     readerPool.clear(merge.segments);
3852     
3853     if (merge.optimize) {
3854       // cascade the optimize:
3855       if (!segmentsToOptimize.containsKey(merge.info)) {
3856         segmentsToOptimize.put(merge.info, Boolean.FALSE);
3857       }
3858     }
3859     
3860     return true;
3861   }
3862   
3863   final private void handleMergeException(Throwable t, MergePolicy.OneMerge merge) throws IOException {
3864
3865     if (infoStream != null) {
3866       message("handleMergeException: merge=" + merge.segString(directory) + " exc=" + t);
3867     }
3868
3869     // Set the exception on the merge, so if
3870     // optimize() is waiting on us it sees the root
3871     // cause exception:
3872     merge.setException(t);
3873     addMergeException(merge);
3874
3875     if (t instanceof MergePolicy.MergeAbortedException) {
3876       // We can ignore this exception (it happens when
3877       // close(false) or rollback is called), unless the
3878       // merge involves segments from external directories,
3879       // in which case we must throw it so, for example, the
3880       // rollbackTransaction code in addIndexes* is
3881       // executed.
3882       if (merge.isExternal)
3883         throw (MergePolicy.MergeAbortedException) t;
3884     } else if (t instanceof IOException)
3885       throw (IOException) t;
3886     else if (t instanceof RuntimeException)
3887       throw (RuntimeException) t;
3888     else if (t instanceof Error)
3889       throw (Error) t;
3890     else
3891       // Should not get here
3892       throw new RuntimeException(t);
3893   }
3894
3895   /**
3896    * Merges the indicated segments, replacing them in the stack with a
3897    * single segment.
3898    * 
3899    * @lucene.experimental
3900    */
3901   public void merge(MergePolicy.OneMerge merge)
3902     throws CorruptIndexException, IOException {
3903
3904     boolean success = false;
3905
3906     final long t0 = System.currentTimeMillis();
3907     //System.out.println(Thread.currentThread().getName() + ": merge start: size=" + (merge.estimatedMergeBytes/1024./1024.) + " MB\n  merge=" + merge.segString(directory) + "\n  idx=" + segString());
3908
3909     try {
3910       try {
3911         try {
3912           mergeInit(merge);
3913
3914           if (infoStream != null)
3915             message("now merge\n  merge=" + merge.segString(directory) + "\n  merge=" + merge + "\n  index=" + segString());
3916
3917           mergeMiddle(merge);
3918           mergeSuccess(merge);
3919           success = true;
3920         } catch (Throwable t) {
3921           handleMergeException(t, merge);
3922         }
3923       } finally {
3924         synchronized(this) {
3925           mergeFinish(merge);
3926
3927           if (!success) {
3928             if (infoStream != null)
3929               message("hit exception during merge");
3930             if (merge.info != null && !segmentInfos.contains(merge.info))
3931               deleter.refresh(merge.info.name);
3932           }
3933
3934           // This merge (and, generally, any change to the
3935           // segments) may now enable new merges, so we call
3936           // merge policy & update pending merges.
3937           if (success && !merge.isAborted() && (merge.optimize || (!closed && !closing))) {
3938             updatePendingMerges(merge.maxNumSegmentsOptimize, merge.optimize);
3939           }
3940         }
3941       }
3942     } catch (OutOfMemoryError oom) {
3943       handleOOM(oom, "merge");
3944     }
3945     if (infoStream != null && merge.info != null) {
3946       message("merge time " + (System.currentTimeMillis()-t0) + " msec for " + merge.info.docCount + " docs");
3947     }
3948     //System.out.println(Thread.currentThread().getName() + ": merge end");
3949   }
3950
3951   /** Hook that's called when the specified merge is complete. */
3952   void mergeSuccess(MergePolicy.OneMerge merge) {
3953   }
3954   
3955   /** Checks whether this merge involves any segments
3956    *  already participating in a merge.  If not, this merge
3957    *  is "registered", meaning we record that its segments
3958    *  are now participating in a merge, and true is
3959    *  returned.  Else (the merge conflicts) false is
3960    *  returned. */
3961   final synchronized boolean registerMerge(MergePolicy.OneMerge merge) throws MergePolicy.MergeAbortedException, IOException {
3962
3963     if (merge.registerDone)
3964       return true;
3965
3966     if (stopMerges) {
3967       merge.abort();
3968       throw new MergePolicy.MergeAbortedException("merge is aborted: " + merge.segString(directory));
3969     }
3970
3971     boolean isExternal = false;
3972     for(SegmentInfo info : merge.segments) {
3973       if (mergingSegments.contains(info)) {
3974         return false;
3975       }
3976       if (!segmentInfos.contains(info)) {
3977         return false;
3978       }
3979       if (info.dir != directory) {
3980         isExternal = true;
3981       }
3982       if (segmentsToOptimize.containsKey(info)) {
3983         merge.optimize = true;
3984         merge.maxNumSegmentsOptimize = optimizeMaxNumSegments;
3985       }
3986     }
3987
3988     ensureValidMerge(merge);
3989
3990     pendingMerges.add(merge);
3991
3992     if (infoStream != null)
3993       message("add merge to pendingMerges: " + merge.segString(directory) + " [total " + pendingMerges.size() + " pending]");
3994
3995     merge.mergeGen = mergeGen;
3996     merge.isExternal = isExternal;
3997
3998     // OK it does not conflict; now record that this merge
3999     // is running (while synchronized) to avoid race
4000     // condition where two conflicting merges from different
4001     // threads, start
4002     message("registerMerge merging=" + mergingSegments);
4003     for(SegmentInfo info : merge.segments) {
4004       message("registerMerge info=" + info);
4005       mergingSegments.add(info);
4006     }
4007
4008     // Merge is now registered
4009     merge.registerDone = true;
4010     return true;
4011   }
4012
4013   /** Does initial setup for a merge, which is fast but holds
4014    *  the synchronized lock on IndexWriter instance.  */
4015   final synchronized void mergeInit(MergePolicy.OneMerge merge) throws IOException {
4016     boolean success = false;
4017     try {
4018       _mergeInit(merge);
4019       success = true;
4020     } finally {
4021       if (!success) {
4022         if (infoStream != null) {
4023           message("hit exception in mergeInit");
4024         }
4025         mergeFinish(merge);
4026       }
4027     }
4028   }
4029
4030   synchronized private void _mergeInit(MergePolicy.OneMerge merge) throws IOException {
4031
4032     assert testPoint("startMergeInit");
4033
4034     assert merge.registerDone;
4035     assert !merge.optimize || merge.maxNumSegmentsOptimize > 0;
4036
4037     if (hitOOM) {
4038       throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot merge");
4039     }
4040
4041     // TODO: is there any perf benefit to sorting
4042     // merged segments?  eg biggest to smallest?
4043
4044     if (merge.info != null)
4045       // mergeInit already done
4046       return;
4047
4048     if (merge.isAborted())
4049       return;
4050
4051     boolean hasVectors = false;
4052     for (SegmentInfo sourceSegment : merge.segments) {
4053       if (sourceSegment.getHasVectors()) {
4054         hasVectors = true;
4055       }
4056     }
4057
4058     // Bind a new segment name here so even with
4059     // ConcurrentMergePolicy we keep deterministic segment
4060     // names.
4061     merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, true, false, hasVectors);
4062
4063     // Lock order: IW -> BD
4064     final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream.applyDeletes(readerPool, merge.segments);
4065
4066     if (result.anyDeletes) {
4067       checkpoint();
4068     }
4069
4070     if (!keepFullyDeletedSegments && result.allDeleted != null) {
4071       if (infoStream != null) {
4072         message("drop 100% deleted segments: " + result.allDeleted);
4073       }
4074       for(SegmentInfo info : result.allDeleted) {
4075         segmentInfos.remove(info);
4076         if (merge.segments.contains(info)) {
4077           mergingSegments.remove(info);
4078           merge.segments.remove(info);
4079         }
4080       }
4081       if (readerPool != null) {
4082         readerPool.drop(result.allDeleted);
4083       }
4084       checkpoint();
4085     }
4086
4087     merge.info.setBufferedDeletesGen(result.gen);
4088
4089     // Lock order: IW -> BD
4090     bufferedDeletesStream.prune(segmentInfos);
4091
4092     Map<String,String> details = new HashMap<String,String>();
4093     details.put("optimize", Boolean.toString(merge.optimize));
4094     details.put("mergeFactor", Integer.toString(merge.segments.size()));
4095     setDiagnostics(merge.info, "merge", details);
4096
4097     if (infoStream != null) {
4098       message("merge seg=" + merge.info.name);
4099     }
4100
4101     assert merge.estimatedMergeBytes == 0;
4102     for(SegmentInfo info : merge.segments) {
4103       if (info.docCount > 0) {
4104         final int delCount = numDeletedDocs(info);
4105         assert delCount <= info.docCount;
4106         final double delRatio = ((double) delCount)/info.docCount;
4107         merge.estimatedMergeBytes += info.sizeInBytes(true) * (1.0 - delRatio);
4108       }
4109     }
4110
4111     // TODO: I think this should no longer be needed (we
4112     // now build CFS before adding segment to the infos);
4113     // however, on removing it, tests fail for some reason!
4114
4115     // Also enroll the merged segment into mergingSegments;
4116     // this prevents it from getting selected for a merge
4117     // after our merge is done but while we are building the
4118     // CFS:
4119     mergingSegments.add(merge.info);
4120   }
4121
4122   private void setDiagnostics(SegmentInfo info, String source) {
4123     setDiagnostics(info, source, null);
4124   }
4125
4126   private void setDiagnostics(SegmentInfo info, String source, Map<String,String> details) {
4127     Map<String,String> diagnostics = new HashMap<String,String>();
4128     diagnostics.put("source", source);
4129     diagnostics.put("lucene.version", Constants.LUCENE_VERSION);
4130     diagnostics.put("os", Constants.OS_NAME);
4131     diagnostics.put("os.arch", Constants.OS_ARCH);
4132     diagnostics.put("os.version", Constants.OS_VERSION);
4133     diagnostics.put("java.version", Constants.JAVA_VERSION);
4134     diagnostics.put("java.vendor", Constants.JAVA_VENDOR);
4135     if (details != null) {
4136       diagnostics.putAll(details);
4137     }
4138     info.setDiagnostics(diagnostics);
4139   }
4140
4141   /** Does fininishing for a merge, which is fast but holds
4142    *  the synchronized lock on IndexWriter instance. */
4143   final synchronized void mergeFinish(MergePolicy.OneMerge merge) throws IOException {
4144     
4145     // Optimize, addIndexes or finishMerges may be waiting
4146     // on merges to finish.
4147     notifyAll();
4148
4149     // It's possible we are called twice, eg if there was an
4150     // exception inside mergeInit
4151     if (merge.registerDone) {
4152       final List<SegmentInfo> sourceSegments = merge.segments;
4153       for(SegmentInfo info : sourceSegments) {
4154         mergingSegments.remove(info);
4155       }
4156       // TODO: if we remove the add in _mergeInit, we should
4157       // also remove this:
4158       mergingSegments.remove(merge.info);
4159       merge.registerDone = false;
4160     }
4161
4162     runningMerges.remove(merge);
4163   }
4164
4165   private final synchronized void closeMergeReaders(MergePolicy.OneMerge merge, boolean suppressExceptions) throws IOException {
4166     final int numSegments = merge.readers.size();
4167     Throwable th = null;
4168     
4169     boolean anyChanges = false;
4170     boolean drop = !suppressExceptions;
4171     for (int i = 0; i < numSegments; i++) {
4172       if (merge.readers.get(i) != null) {
4173         try {
4174           anyChanges |= readerPool.release(merge.readers.get(i), drop);
4175         } catch (Throwable t) {
4176           if (th == null) {
4177             th = t;
4178           }
4179         }
4180         merge.readers.set(i, null);
4181       }
4182       
4183       if (i < merge.readerClones.size() && merge.readerClones.get(i) != null) {
4184         try {
4185           merge.readerClones.get(i).close();
4186         } catch (Throwable t) {
4187           if (th == null) {
4188             th = t;
4189           }
4190         }
4191         // This was a private clone and we had the
4192         // only reference
4193         assert merge.readerClones.get(i).getRefCount() == 0: "refCount should be 0 but is " + merge.readerClones.get(i).getRefCount();
4194         merge.readerClones.set(i, null);
4195       }
4196     }
4197     
4198     if (suppressExceptions && anyChanges) {
4199       checkpoint();
4200     }
4201     
4202     // If any error occured, throw it.
4203     if (!suppressExceptions && th != null) {
4204       if (th instanceof IOException) throw (IOException) th;
4205       if (th instanceof RuntimeException) throw (RuntimeException) th;
4206       if (th instanceof Error) throw (Error) th;
4207       throw new RuntimeException(th);
4208     }
4209   }
4210
4211   /** Does the actual (time-consuming) work of the merge,
4212    *  but without holding synchronized lock on IndexWriter
4213    *  instance */
4214   final private int mergeMiddle(MergePolicy.OneMerge merge) 
4215     throws CorruptIndexException, IOException {
4216     
4217     merge.checkAborted(directory);
4218
4219     final String mergedName = merge.info.name;
4220     
4221     int mergedDocCount = 0;
4222
4223     List<SegmentInfo> sourceSegments = merge.segments;
4224
4225     SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(), mergedName, merge,
4226                                              payloadProcessorProvider,
4227                                              ((FieldInfos) docWriter.getFieldInfos().clone()));
4228
4229     if (infoStream != null) {
4230       message("merging " + merge.segString(directory) + " mergeVectors=" + merge.info.getHasVectors());
4231     }
4232
4233     merge.readers = new ArrayList<SegmentReader>();
4234     merge.readerClones = new ArrayList<SegmentReader>();
4235
4236     // This is try/finally to make sure merger's readers are
4237     // closed:
4238     boolean success = false;
4239     try {
4240       int totDocCount = 0;
4241       int segUpto = 0;
4242       while(segUpto < sourceSegments.size()) {
4243
4244         final SegmentInfo info = sourceSegments.get(segUpto);
4245
4246         // Hold onto the "live" reader; we will use this to
4247         // commit merged deletes
4248         final SegmentReader reader = readerPool.get(info, true,
4249                                                     MERGE_READ_BUFFER_SIZE,
4250                                                     -1);
4251         merge.readers.add(reader);
4252
4253         // We clone the segment readers because other
4254         // deletes may come in while we're merging so we
4255         // need readers that will not change
4256         final SegmentReader clone = (SegmentReader) reader.clone(true);
4257         merge.readerClones.add(clone);
4258
4259         if (clone.numDocs() > 0) {
4260           merger.add(clone);
4261           totDocCount += clone.numDocs();
4262         }
4263         segUpto++;
4264       }
4265
4266       if (infoStream != null) {
4267         message("merge: total " + totDocCount + " docs");
4268       }
4269
4270       merge.checkAborted(directory);
4271
4272       // This is where all the work happens:
4273       mergedDocCount = merge.info.docCount = merger.merge();
4274
4275       // LUCENE-3403: set hasVectors after merge(), so that it is properly set.
4276       merge.info.setHasVectors(merger.fieldInfos().hasVectors());
4277
4278       assert mergedDocCount == totDocCount;
4279
4280       if (infoStream != null) {
4281         message("merge store matchedCount=" + merger.getMatchedSubReaderCount() + " vs " + merge.readers.size());
4282       }
4283
4284       anyNonBulkMerges |= merger.getAnyNonBulkMerges();
4285       
4286       assert mergedDocCount == totDocCount: "mergedDocCount=" + mergedDocCount + " vs " + totDocCount;
4287
4288       // Very important to do this before opening the reader
4289       // because SegmentReader must know if prox was written for
4290       // this segment:
4291       merge.info.setHasProx(merger.fieldInfos().hasProx());
4292
4293       boolean useCompoundFile;
4294       synchronized (this) { // Guard segmentInfos
4295         useCompoundFile = mergePolicy.useCompoundFile(segmentInfos, merge.info);
4296       }
4297
4298       if (useCompoundFile) {
4299
4300         success = false;
4301         final String compoundFileName = IndexFileNames.segmentFileName(mergedName, IndexFileNames.COMPOUND_FILE_EXTENSION);
4302
4303         try {
4304           if (infoStream != null) {
4305             message("create compound file " + compoundFileName);
4306           }
4307           merger.createCompoundFile(compoundFileName, merge.info);
4308           success = true;
4309         } catch (IOException ioe) {
4310           synchronized(this) {
4311             if (merge.isAborted()) {
4312               // This can happen if rollback or close(false)
4313               // is called -- fall through to logic below to
4314               // remove the partially created CFS:
4315             } else {
4316               handleMergeException(ioe, merge);
4317             }
4318           }
4319         } catch (Throwable t) {
4320           handleMergeException(t, merge);
4321         } finally {
4322           if (!success) {
4323             if (infoStream != null) {
4324               message("hit exception creating compound file during merge");
4325             }
4326
4327             synchronized(this) {
4328               deleter.deleteFile(compoundFileName);
4329               deleter.deleteNewFiles(merge.info.files());
4330             }
4331           }
4332         }
4333
4334         success = false;
4335
4336         synchronized(this) {
4337
4338           // delete new non cfs files directly: they were never
4339           // registered with IFD
4340           deleter.deleteNewFiles(merge.info.files());
4341
4342           if (merge.isAborted()) {
4343             if (infoStream != null) {
4344               message("abort merge after building CFS");
4345             }
4346             deleter.deleteFile(compoundFileName);
4347             return 0;
4348           }
4349         }
4350
4351         merge.info.setUseCompoundFile(true);
4352       }
4353
4354       if (infoStream != null) {
4355         message(String.format("merged segment size=%.3f MB vs estimate=%.3f MB", merge.info.sizeInBytes(true)/1024./1024., merge.estimatedMergeBytes/1024/1024.));
4356       }
4357
4358       final IndexReaderWarmer mergedSegmentWarmer = config.getMergedSegmentWarmer();
4359
4360       final int termsIndexDivisor;
4361       final boolean loadDocStores;
4362
4363       if (mergedSegmentWarmer != null) {
4364         // Load terms index & doc stores so the segment
4365         // warmer can run searches, load documents/term
4366         // vectors
4367         termsIndexDivisor = config.getReaderTermsIndexDivisor();
4368         loadDocStores = true;
4369       } else {
4370         termsIndexDivisor = -1;
4371         loadDocStores = false;
4372       }
4373
4374       // TODO: in the non-realtime case, we may want to only
4375       // keep deletes (it's costly to open entire reader
4376       // when we just need deletes)
4377
4378       final SegmentReader mergedReader = readerPool.get(merge.info, loadDocStores, BufferedIndexInput.BUFFER_SIZE, termsIndexDivisor);
4379       try {
4380         if (poolReaders && mergedSegmentWarmer != null) {
4381           mergedSegmentWarmer.warm(mergedReader);
4382         }
4383
4384         if (!commitMerge(merge, mergedReader)) {
4385           // commitMerge will return false if this merge was aborted
4386           return 0;
4387         }
4388       } finally {
4389         synchronized(this) {
4390           if (readerPool.release(mergedReader)) {
4391             // Must checkpoint after releasing the
4392             // mergedReader since it may have written a new
4393             // deletes file:
4394             checkpoint();
4395           }
4396         }
4397       }
4398
4399       success = true;
4400
4401     } finally {
4402       // Readers are already closed in commitMerge if we didn't hit
4403       // an exc:
4404       if (!success) {
4405         closeMergeReaders(merge, true);
4406       }
4407     }
4408
4409     return mergedDocCount;
4410   }
4411
4412   synchronized void addMergeException(MergePolicy.OneMerge merge) {
4413     assert merge.getException() != null;
4414     if (!mergeExceptions.contains(merge) && mergeGen == merge.mergeGen)
4415       mergeExceptions.add(merge);
4416   }
4417
4418   // For test purposes.
4419   final int getBufferedDeleteTermsSize() {
4420     return docWriter.getPendingDeletes().terms.size();
4421   }
4422
4423   // For test purposes.
4424   final int getNumBufferedDeleteTerms() {
4425     return docWriter.getPendingDeletes().numTermDeletes.get();
4426   }
4427
4428   // utility routines for tests
4429   synchronized SegmentInfo newestSegment() {
4430     return segmentInfos.size() > 0 ? segmentInfos.info(segmentInfos.size()-1) : null;
4431   }
4432
4433   /** @lucene.internal */
4434   public synchronized String segString() throws IOException {
4435     return segString(segmentInfos);
4436   }
4437
4438   /** @lucene.internal */
4439   public synchronized String segString(Iterable<SegmentInfo> infos) throws IOException {
4440     final StringBuilder buffer = new StringBuilder();
4441     for(final SegmentInfo s : infos) {
4442       if (buffer.length() > 0) {
4443         buffer.append(' ');
4444       }
4445       buffer.append(segString(s));
4446     }
4447     return buffer.toString();
4448   }
4449
4450   /** @lucene.internal */
4451   public synchronized String segString(SegmentInfo info) throws IOException {
4452     StringBuilder buffer = new StringBuilder();
4453     SegmentReader reader = readerPool.getIfExists(info);
4454     try {
4455       if (reader != null) {
4456         buffer.append(reader.toString());
4457       } else {
4458         buffer.append(info.toString(directory, 0));
4459         if (info.dir != directory) {
4460           buffer.append("**");
4461         }
4462       }
4463     } finally {
4464       if (reader != null) {
4465         readerPool.release(reader);
4466       }
4467     }
4468     return buffer.toString();
4469   }
4470
4471   private synchronized void doWait() {
4472     // NOTE: the callers of this method should in theory
4473     // be able to do simply wait(), but, as a defense
4474     // against thread timing hazards where notifyAll()
4475     // fails to be called, we wait for at most 1 second
4476     // and then return so caller can check if wait
4477     // conditions are satisfied:
4478     try {
4479       wait(1000);
4480     } catch (InterruptedException ie) {
4481       throw new ThreadInterruptedException(ie);
4482     }
4483   }
4484
4485   private boolean keepFullyDeletedSegments;
4486
4487   /** Only for testing.
4488    *
4489    * @lucene.internal */
4490   void keepFullyDeletedSegments() {
4491     keepFullyDeletedSegments = true;
4492   }
4493
4494   boolean getKeepFullyDeletedSegments() {
4495     return keepFullyDeletedSegments;
4496   }
4497
4498   // called only from assert
4499   private boolean filesExist(SegmentInfos toSync) throws IOException {
4500     Collection<String> files = toSync.files(directory, false);
4501     for(final String fileName: files) {
4502       assert directory.fileExists(fileName): "file " + fileName + " does not exist";
4503       // If this trips it means we are missing a call to
4504       // .checkpoint somewhere, because by the time we
4505       // are called, deleter should know about every
4506       // file referenced by the current head
4507       // segmentInfos:
4508       assert deleter.exists(fileName): "IndexFileDeleter doesn't know about file " + fileName;
4509     }
4510     return true;
4511   }
4512
4513   /** Walk through all files referenced by the current
4514    *  segmentInfos and ask the Directory to sync each file,
4515    *  if it wasn't already.  If that succeeds, then we
4516    *  prepare a new segments_N file but do not fully commit
4517    *  it. */
4518   private void startCommit(SegmentInfos toSync, Map<String,String> commitUserData) throws IOException {
4519
4520     assert testPoint("startStartCommit");
4521     assert pendingCommit == null;
4522
4523     if (hitOOM) {
4524       throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot commit");
4525     }
4526
4527     try {
4528
4529       if (infoStream != null)
4530         message("startCommit(): start");
4531
4532
4533       synchronized(this) {
4534
4535         assert lastCommitChangeCount <= changeCount;
4536         
4537         if (pendingCommitChangeCount == lastCommitChangeCount) {
4538           if (infoStream != null) {
4539             message("  skip startCommit(): no changes pending");
4540           }
4541           deleter.decRef(toSync);
4542           return;
4543         }
4544         
4545         // First, we clone & incref the segmentInfos we intend
4546         // to sync, then, without locking, we sync() all files
4547         // referenced by toSync, in the background.
4548         
4549         if (infoStream != null)
4550           message("startCommit index=" + segString(toSync) + " changeCount=" + changeCount);
4551
4552         assert filesExist(toSync);
4553         
4554         if (commitUserData != null) {
4555           toSync.setUserData(commitUserData);
4556         }
4557       }
4558
4559       assert testPoint("midStartCommit");
4560
4561       boolean pendingCommitSet = false;
4562
4563       try {
4564         // This call can take a long time -- 10s of seconds
4565         // or more.  We do it without sync:
4566         directory.sync(toSync.files(directory, false));
4567
4568         assert testPoint("midStartCommit2");
4569
4570         synchronized(this) {
4571
4572           assert pendingCommit == null;
4573
4574           assert segmentInfos.getGeneration() == toSync.getGeneration();
4575
4576           // Exception here means nothing is prepared
4577           // (this method unwinds everything it did on
4578           // an exception)
4579           toSync.prepareCommit(directory);
4580           pendingCommitSet = true;
4581           pendingCommit = toSync;
4582         }
4583
4584         if (infoStream != null) {
4585           message("done all syncs");
4586         }
4587
4588         assert testPoint("midStartCommitSuccess");
4589
4590       } finally {
4591         synchronized(this) {
4592
4593           // Have our master segmentInfos record the
4594           // generations we just prepared.  We do this
4595           // on error or success so we don't
4596           // double-write a segments_N file.
4597           segmentInfos.updateGeneration(toSync);
4598
4599           if (!pendingCommitSet) {
4600             if (infoStream != null) {
4601               message("hit exception committing segments file");
4602             }
4603
4604             deleter.decRef(toSync);
4605           }
4606         }
4607       }
4608     } catch (OutOfMemoryError oom) {
4609       handleOOM(oom, "startCommit");
4610     }
4611     assert testPoint("finishStartCommit");
4612   }
4613
4614   /**
4615    * Returns <code>true</code> iff the index in the named directory is
4616    * currently locked.
4617    * @param directory the directory to check for a lock
4618    * @throws IOException if there is a low-level IO error
4619    */
4620   public static boolean isLocked(Directory directory) throws IOException {
4621     return directory.makeLock(WRITE_LOCK_NAME).isLocked();
4622   }
4623
4624   /**
4625    * Forcibly unlocks the index in the named directory.
4626    * <P>
4627    * Caution: this should only be used by failure recovery code,
4628    * when it is known that no other process nor thread is in fact
4629    * currently accessing this index.
4630    */
4631   public static void unlock(Directory directory) throws IOException {
4632     directory.makeLock(IndexWriter.WRITE_LOCK_NAME).release();
4633   }
4634
4635   /**
4636    * Specifies maximum field length (in number of tokens/terms) in
4637    * {@link IndexWriter} constructors. {@link #setMaxFieldLength(int)} overrides
4638    * the value set by the constructor.
4639    * 
4640    * @deprecated use {@link LimitTokenCountAnalyzer} instead.
4641    */
4642   @Deprecated
4643   public static final class MaxFieldLength {
4644
4645     private int limit;
4646     private String name;
4647
4648     /**
4649      * Private type-safe-enum-pattern constructor.
4650      * 
4651      * @param name instance name
4652      * @param limit maximum field length
4653      */
4654     private MaxFieldLength(String name, int limit) {
4655       this.name = name;
4656       this.limit = limit;
4657     }
4658
4659     /**
4660      * Public constructor to allow users to specify the maximum field size limit.
4661      * 
4662      * @param limit The maximum field length
4663      */
4664     public MaxFieldLength(int limit) {
4665       this("User-specified", limit);
4666     }
4667     
4668     public int getLimit() {
4669       return limit;
4670     }
4671     
4672     @Override
4673     public String toString()
4674     {
4675       return name + ":" + limit;
4676     }
4677
4678     /** Sets the maximum field length to {@link Integer#MAX_VALUE}. */
4679     public static final MaxFieldLength UNLIMITED
4680         = new MaxFieldLength("UNLIMITED", Integer.MAX_VALUE);
4681
4682     /**
4683      *  Sets the maximum field length to 
4684      * {@link #DEFAULT_MAX_FIELD_LENGTH} 
4685      * */
4686     public static final MaxFieldLength LIMITED
4687         = new MaxFieldLength("LIMITED", 10000);
4688   }
4689
4690   /** If {@link #getReader} has been called (ie, this writer
4691    *  is in near real-time mode), then after a merge
4692    *  completes, this class can be invoked to warm the
4693    *  reader on the newly merged segment, before the merge
4694    *  commits.  This is not required for near real-time
4695    *  search, but will reduce search latency on opening a
4696    *  new near real-time reader after a merge completes.
4697    *
4698    * @lucene.experimental
4699    *
4700    * <p><b>NOTE</b>: warm is called before any deletes have
4701    * been carried over to the merged segment. */
4702   public static abstract class IndexReaderWarmer {
4703     public abstract void warm(IndexReader reader) throws IOException;
4704   }
4705
4706   /**
4707    * Set the merged segment warmer. See {@link IndexReaderWarmer}.
4708    * 
4709    * @deprecated use
4710    *             {@link IndexWriterConfig#setMergedSegmentWarmer}
4711    *             instead.
4712    */
4713   @Deprecated
4714   public void setMergedSegmentWarmer(IndexReaderWarmer warmer) {
4715     config.setMergedSegmentWarmer(warmer);
4716   }
4717
4718   /**
4719    * Returns the current merged segment warmer. See {@link IndexReaderWarmer}.
4720    * 
4721    * @deprecated use {@link IndexWriterConfig#getMergedSegmentWarmer()} instead.
4722    */
4723   @Deprecated
4724   public IndexReaderWarmer getMergedSegmentWarmer() {
4725     return config.getMergedSegmentWarmer();
4726   }
4727
4728   private void handleOOM(OutOfMemoryError oom, String location) {
4729     if (infoStream != null) {
4730       message("hit OutOfMemoryError inside " + location);
4731     }
4732     hitOOM = true;
4733     throw oom;
4734   }
4735
4736   // Used only by assert for testing.  Current points:
4737   //   startDoFlush
4738   //   startCommitMerge
4739   //   startStartCommit
4740   //   midStartCommit
4741   //   midStartCommit2
4742   //   midStartCommitSuccess
4743   //   finishStartCommit
4744   //   startCommitMergeDeletes
4745   //   startMergeInit
4746   //   DocumentsWriter.ThreadState.init start
4747   boolean testPoint(String name) {
4748     return true;
4749   }
4750
4751   synchronized boolean nrtIsCurrent(SegmentInfos infos) {
4752     //System.out.println("IW.nrtIsCurrent " + (infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any()));
4753     return infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any();
4754   }
4755
4756   synchronized boolean isClosed() {
4757     return closed;
4758   }
4759
4760   /** Expert: remove any index files that are no longer
4761    *  used.
4762    * 
4763    *  <p> IndexWriter normally deletes unused files itself,
4764    *  during indexing.  However, on Windows, which disallows
4765    *  deletion of open files, if there is a reader open on
4766    *  the index then those files cannot be deleted.  This is
4767    *  fine, because IndexWriter will periodically retry
4768    *  the deletion.</p>
4769    *
4770    *  <p> However, IndexWriter doesn't try that often: only
4771    *  on open, close, flushing a new segment, and finishing
4772    *  a merge.  If you don't do any of these actions with your
4773    *  IndexWriter, you'll see the unused files linger.  If
4774    *  that's a problem, call this method to delete them
4775    *  (once you've closed the open readers that were
4776    *  preventing their deletion). 
4777    *  
4778    *  <p> In addition, you can call this method to delete 
4779    *  unreferenced index commits. This might be useful if you 
4780    *  are using an {@link IndexDeletionPolicy} which holds
4781    *  onto index commits until some criteria are met, but those
4782    *  commits are no longer needed. Otherwise, those commits will
4783    *  be deleted the next time commit() is called.
4784    */
4785   public synchronized void deleteUnusedFiles() throws IOException {
4786     deleter.deletePendingFiles();
4787     deleter.revisitPolicy();
4788   }
4789
4790   /**
4791    * Sets the {@link PayloadProcessorProvider} to use when merging payloads.
4792    * Note that the given <code>pcp</code> will be invoked for every segment that
4793    * is merged, not only external ones that are given through
4794    * {@link #addIndexes}. If you want only the payloads of the external segments
4795    * to be processed, you can return <code>null</code> whenever a
4796    * {@link DirPayloadProcessor} is requested for the {@link Directory} of the
4797    * {@link IndexWriter}.
4798    * <p>
4799    * The default is <code>null</code> which means payloads are processed
4800    * normally (copied) during segment merges. You can also unset it by passing
4801    * <code>null</code>.
4802    * <p>
4803    * <b>NOTE:</b> the set {@link PayloadProcessorProvider} will be in effect
4804    * immediately, potentially for already running merges too. If you want to be
4805    * sure it is used for further operations only, such as {@link #addIndexes} or
4806    * {@link #optimize}, you can call {@link #waitForMerges()} before.
4807    */
4808   public void setPayloadProcessorProvider(PayloadProcessorProvider pcp) {
4809     payloadProcessorProvider = pcp;
4810   }
4811   
4812   /**
4813    * Returns the {@link PayloadProcessorProvider} that is used during segment
4814    * merges to process payloads.
4815    */
4816   public PayloadProcessorProvider getPayloadProcessorProvider() {
4817     return payloadProcessorProvider;
4818   }
4819
4820   // decides when flushes happen
4821   final class FlushControl {
4822
4823     private boolean flushPending;
4824     private boolean flushDeletes;
4825     private int delCount;
4826     private int docCount;
4827     private boolean flushing;
4828
4829     private synchronized boolean setFlushPending(String reason, boolean doWait) {
4830       if (flushPending || flushing) {
4831         if (doWait) {
4832           while(flushPending || flushing) {
4833             try {
4834               wait();
4835             } catch (InterruptedException ie) {
4836               throw new ThreadInterruptedException(ie);
4837             }
4838           }
4839         }
4840         return false;
4841       } else {
4842         if (infoStream != null) {
4843           message("now trigger flush reason=" + reason);
4844         }
4845         flushPending = true;
4846         return flushPending;
4847       }
4848     }
4849
4850     public synchronized void setFlushPendingNoWait(String reason) {
4851       setFlushPending(reason, false);
4852     }
4853
4854     public synchronized boolean getFlushPending() {
4855       return flushPending;
4856     }
4857
4858     public synchronized boolean getFlushDeletes() {
4859       return flushDeletes;
4860     }
4861
4862     public synchronized void clearFlushPending() {
4863       if (infoStream != null) {
4864         message("clearFlushPending");
4865       }
4866       flushPending = false;
4867       flushDeletes = false;
4868       docCount = 0;
4869       notifyAll();
4870     }
4871
4872     public synchronized void clearDeletes() {
4873       delCount = 0;
4874     }
4875
4876     public synchronized boolean waitUpdate(int docInc, int delInc) {
4877       return waitUpdate(docInc, delInc, false);
4878     }
4879
4880     public synchronized boolean waitUpdate(int docInc, int delInc, boolean skipWait) {
4881       while(flushPending) {
4882         try {
4883           wait();
4884         } catch (InterruptedException ie) {
4885           throw new ThreadInterruptedException(ie);
4886         }
4887       }
4888
4889       docCount += docInc;
4890       delCount += delInc;
4891
4892       // skipWait is only used when a thread is BOTH adding
4893       // a doc and buffering a del term, and, the adding of
4894       // the doc already triggered a flush
4895       if (skipWait) {
4896         return false;
4897       }
4898
4899       final int maxBufferedDocs = config.getMaxBufferedDocs();
4900       if (maxBufferedDocs != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
4901           docCount >= maxBufferedDocs) {
4902         return setFlushPending("maxBufferedDocs", true);
4903       }
4904
4905       final int maxBufferedDeleteTerms = config.getMaxBufferedDeleteTerms();
4906       if (maxBufferedDeleteTerms != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
4907           delCount >= maxBufferedDeleteTerms) {
4908         flushDeletes = true;
4909         return setFlushPending("maxBufferedDeleteTerms", true);
4910       }
4911
4912       return flushByRAMUsage("add delete/doc");
4913     }
4914
4915     public synchronized boolean flushByRAMUsage(String reason) {
4916       final double ramBufferSizeMB = config.getRAMBufferSizeMB();
4917       if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH) {
4918         final long limit = (long) (ramBufferSizeMB*1024*1024);
4919         long used = bufferedDeletesStream.bytesUsed() + docWriter.bytesUsed();
4920         if (used >= limit) {
4921           
4922           // DocumentsWriter may be able to free up some
4923           // RAM:
4924           // Lock order: FC -> DW
4925           docWriter.balanceRAM();
4926
4927           used = bufferedDeletesStream.bytesUsed() + docWriter.bytesUsed();
4928           if (used >= limit) {
4929             return setFlushPending("ram full: " + reason, false);
4930           }
4931         }
4932       }
4933       return false;
4934     }
4935   }
4936
4937   final FlushControl flushControl = new FlushControl();
4938 }