Lucene.Net  3.0.3
Lucene.Net is a .NET port of the Java Lucene Indexing Library
 All Classes Namespaces Files Functions Variables Typedefs Enumerations Properties
IndexWriter.cs
Go to the documentation of this file.
1 /*
2  * Licensed to the Apache Software Foundation (ASF) under one or more
3  * contributor license agreements. See the NOTICE file distributed with
4  * this work for additional information regarding copyright ownership.
5  * The ASF licenses this file to You under the Apache License, Version 2.0
6  * (the "License"); you may not use this file except in compliance with
7  * the License. You may obtain a copy of the License at
8  *
9  * http://www.apache.org/licenses/LICENSE-2.0
10  *
11  * Unless required by applicable law or agreed to in writing, software
12  * distributed under the License is distributed on an "AS IS" BASIS,
13  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14  * See the License for the specific language governing permissions and
15  * limitations under the License.
16  */
17 
18 using System;
19 using System.Collections.Generic;
20 using System.IO;
21 using Lucene.Net.Support;
22 using Analyzer = Lucene.Net.Analysis.Analyzer;
23 using Document = Lucene.Net.Documents.Document;
24 using IndexingChain = Lucene.Net.Index.DocumentsWriter.IndexingChain;
25 using AlreadyClosedException = Lucene.Net.Store.AlreadyClosedException;
26 using BufferedIndexInput = Lucene.Net.Store.BufferedIndexInput;
27 using Directory = Lucene.Net.Store.Directory;
28 using Lock = Lucene.Net.Store.Lock;
29 using LockObtainFailedException = Lucene.Net.Store.LockObtainFailedException;
30 using Constants = Lucene.Net.Util.Constants;
31 using Query = Lucene.Net.Search.Query;
32 using Similarity = Lucene.Net.Search.Similarity;
33 
34 namespace Lucene.Net.Index
35 {
36 
139 
140  /*
141  * Clarification: Check Points (and commits)
142  * IndexWriter writes new index files to the directory without writing a new segments_N
143  * file which references these new files. It also means that the state of
144  * the in memory SegmentInfos object is different than the most recent
145  * segments_N file written to the directory.
146  *
147  * Each time the SegmentInfos is changed, and matches the (possibly
148  * modified) directory files, we have a new "check point".
149  * If the modified/new SegmentInfos is written to disk - as a new
150  * (generation of) segments_N file - this check point is also an
151  * IndexCommit.
152  *
153  * A new checkpoint always replaces the previous checkpoint and
154  * becomes the new "front" of the index. This allows the IndexFileDeleter
155  * to delete files that are referenced only by stale checkpoints.
156  * (files that were created since the last commit, but are no longer
157  * referenced by the "front" of the index). For this, IndexFileDeleter
158  * keeps track of the last non commit checkpoint.
159  */
160  public class IndexWriter : System.IDisposable
161  {
162  private void InitBlock()
163  {
164  similarity = Search.Similarity.Default;
165  mergePolicy = new LogByteSizeMergePolicy(this);
166  readerPool = new ReaderPool(this);
167  }
168 
172  public static long WRITE_LOCK_TIMEOUT = 1000;
173 
174  private long writeLockTimeout = WRITE_LOCK_TIMEOUT;
175 
177  public const System.String WRITE_LOCK_NAME = "write.lock";
178 
180  public const int DISABLE_AUTO_FLUSH = - 1;
181 
185  public static readonly int DEFAULT_MAX_BUFFERED_DOCS = DISABLE_AUTO_FLUSH;
186 
190  public const double DEFAULT_RAM_BUFFER_SIZE_MB = 16.0;
191 
195  public static readonly int DEFAULT_MAX_BUFFERED_DELETE_TERMS = DISABLE_AUTO_FLUSH;
196 
198  public const int DEFAULT_MAX_FIELD_LENGTH = 10000;
199 
201  public const int DEFAULT_TERM_INDEX_INTERVAL = 128;
202 
208  public static readonly int MAX_TERM_LENGTH;
209 
210  // The normal read buffer size defaults to 1024, but
211  // increasing this during merging seems to yield
212  // performance gains. However we don't want to increase
213  // it too much because there are quite a few
214  // BufferedIndexInputs created during merging. See
215  // LUCENE-888 for details.
216  private const int MERGE_READ_BUFFER_SIZE = 4096;
217 
218  // Used for printing messages
219  private static System.Object MESSAGE_ID_LOCK = new System.Object();
220  private static int MESSAGE_ID = 0;
221  private int messageID = - 1;
222  private volatile bool hitOOM;
223 
224  private Directory directory; // where this index resides
225  private Analyzer analyzer; // how to analyze text
226 
227  private Similarity similarity; // how to normalize
228 
229  private volatile uint changeCount; // increments every time a change is completed
230  private long lastCommitChangeCount; // last changeCount that was committed
231 
232  private SegmentInfos rollbackSegmentInfos; // segmentInfos we will fallback to if the commit fails
233  private HashMap<SegmentInfo, int?> rollbackSegments;
234 
235  internal volatile SegmentInfos pendingCommit; // set when a commit is pending (after prepareCommit() & before commit())
236  internal volatile uint pendingCommitChangeCount;
237 
238  private SegmentInfos localRollbackSegmentInfos; // segmentInfos we will fallback to if the commit fails
239  private int localFlushedDocCount; // saved docWriter.getFlushedDocCount during local transaction
240 
241  private SegmentInfos segmentInfos = new SegmentInfos(); // the segments
242  private int optimizeMaxNumSegments;
243 
244  private DocumentsWriter docWriter;
245  private IndexFileDeleter deleter;
246 
247  private ISet<SegmentInfo> segmentsToOptimize = Lucene.Net.Support.Compatibility.SetFactory.CreateHashSet<SegmentInfo>(); // used by optimize to note those needing optimization
248 
249  private Lock writeLock;
250 
251  private int termIndexInterval = DEFAULT_TERM_INDEX_INTERVAL;
252 
253  private bool closed;
254  private bool closing;
255 
256  // Holds all SegmentInfo instances currently involved in
257  // merges
258  private HashSet<SegmentInfo> mergingSegments = new HashSet<SegmentInfo>();
259 
260  private MergePolicy mergePolicy;
261  private MergeScheduler mergeScheduler = new ConcurrentMergeScheduler();
262  private LinkedList<MergePolicy.OneMerge> pendingMerges = new LinkedList<MergePolicy.OneMerge>();
263  private ISet<MergePolicy.OneMerge> runningMerges = Lucene.Net.Support.Compatibility.SetFactory.CreateHashSet<MergePolicy.OneMerge>();
264  private IList<MergePolicy.OneMerge> mergeExceptions = new List<MergePolicy.OneMerge>();
265  private long mergeGen;
266  private bool stopMerges;
267 
268  private int flushCount;
269  private int flushDeletesCount;
270 
271  // Used to only allow one addIndexes to proceed at once
272  // TODO: use ReadWriteLock once we are on 5.0
273  private int readCount; // count of how many threads are holding read lock
274  private ThreadClass writeThread; // non-null if any thread holds write lock
275  internal ReaderPool readerPool;
276  private int upgradeCount;
277 
278  private int readerTermsIndexDivisor = IndexReader.DEFAULT_TERMS_INDEX_DIVISOR;
279 
280  // This is a "write once" variable (like the organic dye
281  // on a DVD-R that may or may not be heated by a laser and
282  // then cooled to permanently record the event): it's
283  // false, until getReader() is called for the first time,
284  // at which point it's switched to true and never changes
285  // back to false. Once this is true, we hold open and
286  // reuse SegmentReader instances internally for applying
287  // deletes, doing merges, and reopening near real-time
288  // readers.
289  private volatile bool poolReaders;
290 
351  public virtual IndexReader GetReader()
352  {
353  return GetReader(readerTermsIndexDivisor);
354  }
355 
371  public virtual IndexReader GetReader(int termInfosIndexDivisor)
372  {
373  EnsureOpen();
374 
375  if (infoStream != null)
376  {
377  Message("flush at getReader");
378  }
379 
380  // Do this up front before flushing so that the readers
381  // obtained during this flush are pooled, the first time
382  // this method is called:
383  poolReaders = true;
384 
385  // Prevent segmentInfos from changing while opening the
386  // reader; in theory we could do similar retry logic,
387  // just like we do when loading segments_N
388  IndexReader r;
389  lock (this)
390  {
391  Flush(false, true, true);
392  r = new ReadOnlyDirectoryReader(this, segmentInfos, termInfosIndexDivisor);
393  }
394  MaybeMerge();
395  return r;
396  }
397 
405 
406  internal class ReaderPool : IDisposable
407  {
408  public ReaderPool(IndexWriter enclosingInstance)
409  {
410  InitBlock(enclosingInstance);
411  }
412  private void InitBlock(IndexWriter enclosingInstance)
413  {
414  this.enclosingInstance = enclosingInstance;
415  }
416  private IndexWriter enclosingInstance;
417  public IndexWriter Enclosing_Instance
418  {
419  get
420  {
421  return enclosingInstance;
422  }
423 
424  }
425 
426  private IDictionary<SegmentInfo, SegmentReader> readerMap = new HashMap<SegmentInfo, SegmentReader>();
427 
431  internal virtual void Clear(SegmentInfos infos)
432  {
433  lock (this)
434  {
435  if (infos == null)
436  {
437  foreach(KeyValuePair<SegmentInfo, SegmentReader> ent in readerMap)
438  {
439  ent.Value.hasChanges = false;
440  }
441  }
442  else
443  {
444  foreach(SegmentInfo info in infos)
445  {
446  if (readerMap.ContainsKey(info))
447  {
448  readerMap[info].hasChanges = false;
449  }
450  }
451  }
452  }
453  }
454 
455  // used only by asserts
456  public virtual bool InfoIsLive(SegmentInfo info)
457  {
458  lock (this)
459  {
460  int idx = Enclosing_Instance.segmentInfos.IndexOf(info);
461  System.Diagnostics.Debug.Assert(idx != -1);
462  System.Diagnostics.Debug.Assert(Enclosing_Instance.segmentInfos[idx] == info);
463  return true;
464  }
465  }
466 
467  public virtual SegmentInfo MapToLive(SegmentInfo info)
468  {
469  lock (this)
470  {
471  int idx = Enclosing_Instance.segmentInfos.IndexOf(info);
472  if (idx != - 1)
473  {
474  info = Enclosing_Instance.segmentInfos[idx];
475  }
476  return info;
477  }
478  }
479 
486  public virtual void Release(SegmentReader sr)
487  {
488  lock (this)
489  {
490  Release(sr, false);
491  }
492  }
493 
501  public virtual void Release(SegmentReader sr, bool drop)
502  {
503  lock (this)
504  {
505 
506  bool pooled = readerMap.ContainsKey(sr.SegmentInfo);
507 
508  System.Diagnostics.Debug.Assert(!pooled || readerMap[sr.SegmentInfo] == sr);
509 
510  // Drop caller's ref; for an external reader (not
511  // pooled), this decRef will close it
512  sr.DecRef();
513 
514  if (pooled && (drop || (!Enclosing_Instance.poolReaders && sr.RefCount == 1)))
515  {
516 
517  // We invoke deleter.checkpoint below, so we must be
518  // sync'd on IW if there are changes:
519 
520  // TODO: Java 1.5 has this, .NET can't.
521  // System.Diagnostics.Debug.Assert(!sr.hasChanges || Thread.holdsLock(enclosingInstance));
522 
523  // Discard (don't save) changes when we are dropping
524  // the reader; this is used only on the sub-readers
525  // after a successful merge.
526  sr.hasChanges &= !drop;
527 
528  bool hasChanges = sr.hasChanges;
529 
530  // Drop our ref -- this will commit any pending
531  // changes to the dir
532  sr.Close();
533 
534  // We are the last ref to this reader; since we're
535  // not pooling readers, we release it:
536  readerMap.Remove(sr.SegmentInfo);
537 
538  if (hasChanges)
539  {
540  // Must checkpoint w/ deleter, because this
541  // segment reader will have created new _X_N.del
542  // file.
543  enclosingInstance.deleter.Checkpoint(enclosingInstance.segmentInfos, false);
544  }
545  }
546  }
547  }
548 
552  public void Dispose()
553  {
554  Dispose(true);
555  }
556 
557  protected void Dispose(bool disposing)
558  {
559  if (disposing)
560  {
561  // We invoke deleter.checkpoint below, so we must be
562  // sync'd on IW:
563  // TODO: assert Thread.holdsLock(IndexWriter.this);
564  // TODO: Should this class have bool _isDisposed?
565  lock (this)
566  {
567  //var toRemove = new List<SegmentInfo>();
568  foreach (var ent in readerMap)
569  {
570  SegmentReader sr = ent.Value;
571  if (sr.hasChanges)
572  {
573  System.Diagnostics.Debug.Assert(InfoIsLive(sr.SegmentInfo));
574  sr.DoCommit(null);
575  // Must checkpoint w/ deleter, because this
576  // segment reader will have created new _X_N.del
577  // file.
578  enclosingInstance.deleter.Checkpoint(enclosingInstance.segmentInfos, false);
579  }
580 
581  //toRemove.Add(ent.Key);
582 
583  // NOTE: it is allowed that this decRef does not
584  // actually close the SR; this can happen when a
585  // near real-time reader is kept open after the
586  // IndexWriter instance is closed
587  sr.DecRef();
588  }
589 
590  //foreach (var key in toRemove)
591  // readerMap.Remove(key);
592  readerMap.Clear();
593  }
594  }
595  }
596 
599  internal virtual void Commit()
600  {
601  // We invoke deleter.checkpoint below, so we must be
602  // sync'd on IW:
603  // TODO: assert Thread.holdsLock(IndexWriter.this);
604  lock (this)
605  {
606  foreach(KeyValuePair<SegmentInfo,SegmentReader> ent in readerMap)
607  {
608  SegmentReader sr = ent.Value;
609  if (sr.hasChanges)
610  {
611  System.Diagnostics.Debug.Assert(InfoIsLive(sr.SegmentInfo));
612  sr.DoCommit(null);
613  // Must checkpoint w/ deleter, because this
614  // segment reader will have created new _X_N.del
615  // file.
616  enclosingInstance.deleter.Checkpoint(enclosingInstance.segmentInfos, false);
617  }
618  }
619  }
620  }
621 
626  public virtual SegmentReader GetReadOnlyClone(SegmentInfo info, bool doOpenStores, int termInfosIndexDivisor)
627  {
628  lock (this)
629  {
630  SegmentReader sr = Get(info, doOpenStores, BufferedIndexInput.BUFFER_SIZE, termInfosIndexDivisor);
631  try
632  {
633  return (SegmentReader) sr.Clone(true);
634  }
635  finally
636  {
637  sr.DecRef();
638  }
639  }
640  }
641 
652  public virtual SegmentReader Get(SegmentInfo info, bool doOpenStores)
653  {
654  lock (this)
655  {
656  return Get(info, doOpenStores, BufferedIndexInput.BUFFER_SIZE, enclosingInstance.readerTermsIndexDivisor);
657  }
658  }
674  public virtual SegmentReader Get(SegmentInfo info, bool doOpenStores, int readBufferSize, int termsIndexDivisor)
675  {
676  lock (this)
677  {
678  if (Enclosing_Instance.poolReaders)
679  {
680  readBufferSize = BufferedIndexInput.BUFFER_SIZE;
681  }
682 
683  SegmentReader sr = readerMap[info];
684  if (sr == null)
685  {
686  // TODO: we may want to avoid doing this while
687  // synchronized
688  // Returns a ref, which we xfer to readerMap:
689  sr = SegmentReader.Get(false, info.dir, info, readBufferSize, doOpenStores, termsIndexDivisor);
690  if (info.dir == enclosingInstance.directory)
691  {
692  // Only pool if reader is not external
693  readerMap[info]=sr;
694  }
695  }
696  else
697  {
698  if (doOpenStores)
699  {
700  sr.OpenDocStores();
701  }
702  if (termsIndexDivisor != - 1 && !sr.TermsIndexLoaded())
703  {
704  // If this reader was originally opened because we
705  // needed to merge it, we didn't load the terms
706  // index. But now, if the caller wants the terms
707  // index (eg because it's doing deletes, or an NRT
708  // reader is being opened) we ask the reader to
709  // load its terms index.
710  sr.LoadTermsIndex(termsIndexDivisor);
711  }
712  }
713 
714  // Return a ref to our caller
715  if (info.dir == enclosingInstance.directory)
716  {
717  // Only incRef if we pooled (reader is not external)
718  sr.IncRef();
719  }
720  return sr;
721  }
722  }
723 
724  // Returns a ref
725  public virtual SegmentReader GetIfExists(SegmentInfo info)
726  {
727  lock (this)
728  {
729  SegmentReader sr = readerMap[info];
730  if (sr != null)
731  {
732  sr.IncRef();
733  }
734  return sr;
735  }
736  }
737  }
738 
743  public virtual int NumDeletedDocs(SegmentInfo info)
744  {
745  SegmentReader reader = readerPool.GetIfExists(info);
746  try
747  {
748  if (reader != null)
749  {
750  return reader.NumDeletedDocs;
751  }
752  else
753  {
754  return info.GetDelCount();
755  }
756  }
757  finally
758  {
759  if (reader != null)
760  {
761  readerPool.Release(reader);
762  }
763  }
764  }
765 
766  internal virtual void AcquireWrite()
767  {
768  lock (this)
769  {
770  System.Diagnostics.Debug.Assert(writeThread != ThreadClass.Current());
771  while (writeThread != null || readCount > 0)
772  DoWait();
773 
774  // We could have been closed while we were waiting:
775  EnsureOpen();
776 
777  writeThread = ThreadClass.Current();
778  }
779  }
780 
781  internal virtual void ReleaseWrite()
782  {
783  lock (this)
784  {
785  System.Diagnostics.Debug.Assert(ThreadClass.Current() == writeThread);
786  writeThread = null;
787  System.Threading.Monitor.PulseAll(this);
788  }
789  }
790 
791  internal virtual void AcquireRead()
792  {
793  lock (this)
794  {
795  ThreadClass current = ThreadClass.Current();
796  while (writeThread != null && writeThread != current)
797  DoWait();
798 
799  readCount++;
800  }
801  }
802 
803  // Allows one readLock to upgrade to a writeLock even if
804  // there are other readLocks as long as all other
805  // readLocks are also blocked in this method:
806  internal virtual void UpgradeReadToWrite()
807  {
808  lock (this)
809  {
810  System.Diagnostics.Debug.Assert(readCount > 0);
811  upgradeCount++;
812  while (readCount > upgradeCount || writeThread != null)
813  {
814  DoWait();
815  }
816 
817  writeThread = ThreadClass.Current();
818  readCount--;
819  upgradeCount--;
820  }
821  }
822 
823  internal virtual void ReleaseRead()
824  {
825  lock (this)
826  {
827  readCount--;
828  System.Diagnostics.Debug.Assert(readCount >= 0);
829  System.Threading.Monitor.PulseAll(this);
830  }
831  }
832 
833  internal bool IsOpen(bool includePendingClose)
834  {
835  lock (this)
836  {
837  return !(closed || (includePendingClose && closing));
838  }
839  }
840 
846  protected internal void EnsureOpen(bool includePendingClose)
847  {
848  lock (this)
849  {
850  if (!IsOpen(includePendingClose))
851  {
852  throw new AlreadyClosedException("this IndexWriter is closed");
853  }
854  }
855  }
856 
857  protected internal void EnsureOpen()
858  {
859  lock (this)
860  {
861  EnsureOpen(true);
862  }
863  }
864 
869  public virtual void Message(System.String message)
870  {
871  if (infoStream != null)
872  infoStream.WriteLine("IW " + messageID + " [" + DateTime.Now.ToString() + "; " + ThreadClass.Current().Name + "]: " + message);
873  }
874 
875  private void SetMessageID(System.IO.StreamWriter infoStream)
876  {
877  lock (this)
878  {
879  if (infoStream != null && messageID == - 1)
880  {
881  lock (MESSAGE_ID_LOCK)
882  {
883  messageID = MESSAGE_ID++;
884  }
885  }
886  this.infoStream = infoStream;
887  }
888  }
889 
893  private LogMergePolicy LogMergePolicy
894  {
895  get
896  {
897  if (mergePolicy is LogMergePolicy)
898  return (LogMergePolicy) mergePolicy;
899 
900  throw new System.ArgumentException(
901  "this method can only be called when the merge policy is the default LogMergePolicy");
902  }
903  }
904 
918  public virtual bool UseCompoundFile
919  {
920  get { return LogMergePolicy.GetUseCompoundFile(); }
921  set
922  {
923  LogMergePolicy.SetUseCompoundFile(value);
924  LogMergePolicy.SetUseCompoundDocStore(value);
925  }
926  }
927 
930  public virtual void SetSimilarity(Similarity similarity)
931  {
932  EnsureOpen();
933  this.similarity = similarity;
934  docWriter.SetSimilarity(similarity);
935  }
936 
941  public virtual Similarity Similarity
942  {
943  get
944  {
945  EnsureOpen();
946  return this.similarity;
947  }
948  }
949 
950 
973  public virtual int TermIndexInterval
974  {
975  get
976  {
977  // We pass false because this method is called by SegmentMerger while we are in the process of closing
978  EnsureOpen(false);
979  return termIndexInterval;
980  }
981  set
982  {
983  EnsureOpen();
984  this.termIndexInterval = value;
985  }
986  }
987 
1015  public IndexWriter(Directory d, Analyzer a, bool create, MaxFieldLength mfl)
1016  {
1017  InitBlock();
1018  Init(d, a, create, null, mfl.Limit, null, null);
1019  }
1020 
1043  {
1044  InitBlock();
1045  Init(d, a, null, mfl.Limit, null, null);
1046  }
1047 
1072  {
1073  InitBlock();
1074  Init(d, a, deletionPolicy, mfl.Limit, null, null);
1075  }
1076 
1107  public IndexWriter(Directory d, Analyzer a, bool create, IndexDeletionPolicy deletionPolicy, MaxFieldLength mfl)
1108  {
1109  InitBlock();
1110  Init(d, a, create, deletionPolicy, mfl.Limit, null, null);
1111  }
1112 
1149  internal IndexWriter(Directory d, Analyzer a, bool create, IndexDeletionPolicy deletionPolicy, MaxFieldLength mfl, IndexingChain indexingChain, IndexCommit commit)
1150  {
1151  InitBlock();
1152  Init(d, a, create, deletionPolicy, mfl.Limit, indexingChain, commit);
1153  }
1154 
1194  {
1195  InitBlock();
1196  Init(d, a, false, deletionPolicy, mfl.Limit, null, commit);
1197  }
1198 
1199  private void Init(Directory d, Analyzer a, IndexDeletionPolicy deletionPolicy, int maxFieldLength, IndexingChain indexingChain, IndexCommit commit)
1200  {
1201  if (IndexReader.IndexExists(d))
1202  {
1203  Init(d, a, false, deletionPolicy, maxFieldLength, indexingChain, commit);
1204  }
1205  else
1206  {
1207  Init(d, a, true, deletionPolicy, maxFieldLength, indexingChain, commit);
1208  }
1209  }
1210 
1211  private void Init(Directory d, Analyzer a, bool create, IndexDeletionPolicy deletionPolicy, int maxFieldLength, IndexingChain indexingChain, IndexCommit commit)
1212  {
1213  directory = d;
1214  analyzer = a;
1215  SetMessageID(defaultInfoStream);
1216  this.maxFieldLength = maxFieldLength;
1217 
1218  if (indexingChain == null)
1219  indexingChain = DocumentsWriter.DefaultIndexingChain;
1220 
1221  if (create)
1222  {
1223  // Clear the write lock in case it's leftover:
1224  directory.ClearLock(WRITE_LOCK_NAME);
1225  }
1226 
1227  Lock writeLock = directory.MakeLock(WRITE_LOCK_NAME);
1228  if (!writeLock.Obtain(writeLockTimeout))
1229  // obtain write lock
1230  {
1231  throw new LockObtainFailedException("Index locked for write: " + writeLock);
1232  }
1233  this.writeLock = writeLock; // save it
1234 
1235  bool success = false;
1236  try
1237  {
1238  if (create)
1239  {
1240  // Try to read first. This is to allow create
1241  // against an index that's currently open for
1242  // searching. In this case we write the next
1243  // segments_N file with no segments:
1244  bool doCommit;
1245  try
1246  {
1247  segmentInfos.Read(directory);
1248  segmentInfos.Clear();
1249  doCommit = false;
1250  }
1251  catch (System.IO.IOException)
1252  {
1253  // Likely this means it's a fresh directory
1254  doCommit = true;
1255  }
1256 
1257  if (doCommit)
1258  {
1259  // Only commit if there is no segments file
1260  // in this dir already.
1261  segmentInfos.Commit(directory);
1262  synced.UnionWith(segmentInfos.Files(directory, true));
1263  }
1264  else
1265  {
1266  // Record that we have a change (zero out all
1267  // segments) pending:
1268  changeCount++;
1269  }
1270  }
1271  else
1272  {
1273  segmentInfos.Read(directory);
1274 
1275  if (commit != null)
1276  {
1277  // Swap out all segments, but, keep metadata in
1278  // SegmentInfos, like version & generation, to
1279  // preserve write-once. This is important if
1280  // readers are open against the future commit
1281  // points.
1282  if (commit.Directory != directory)
1283  throw new System.ArgumentException("IndexCommit's directory doesn't match my directory");
1284  SegmentInfos oldInfos = new SegmentInfos();
1285  oldInfos.Read(directory, commit.SegmentsFileName);
1286  segmentInfos.Replace(oldInfos);
1287  changeCount++;
1288  if (infoStream != null)
1289  Message("init: loaded commit \"" + commit.SegmentsFileName + "\"");
1290  }
1291 
1292  // We assume that this segments_N was previously
1293  // properly sync'd:
1294  synced.UnionWith(segmentInfos.Files(directory, true));
1295  }
1296 
1297  SetRollbackSegmentInfos(segmentInfos);
1298 
1299  docWriter = new DocumentsWriter(directory, this, indexingChain);
1300  docWriter.SetInfoStream(infoStream);
1301  docWriter.SetMaxFieldLength(maxFieldLength);
1302 
1303  // Default deleter (for backwards compatibility) is
1304  // KeepOnlyLastCommitDeleter:
1305  deleter = new IndexFileDeleter(directory, deletionPolicy == null?new KeepOnlyLastCommitDeletionPolicy():deletionPolicy, segmentInfos, infoStream, docWriter, synced);
1306 
1307  if (deleter.startingCommitDeleted)
1308  // Deletion policy deleted the "head" commit point.
1309  // We have to mark ourself as changed so that if we
1310  // are closed w/o any further changes we write a new
1311  // segments_N file.
1312  changeCount++;
1313 
1314  PushMaxBufferedDocs();
1315 
1316  if (infoStream != null)
1317  {
1318  Message("init: create=" + create);
1319  MessageState();
1320  }
1321 
1322  success = true;
1323  }
1324  finally
1325  {
1326  if (!success)
1327  {
1328  if (infoStream != null)
1329  {
1330  Message("init: hit exception on init; releasing write lock");
1331  }
1332  try
1333  {
1334  writeLock.Release();
1335  }
1336  catch (Exception)
1337  {
1338  // don't mask the original exception
1339  }
1340  writeLock = null;
1341  }
1342  }
1343  }
1344 
1345  private void SetRollbackSegmentInfos(SegmentInfos infos)
1346  {
1347  lock (this)
1348  {
1349  rollbackSegmentInfos = (SegmentInfos) infos.Clone();
1350  System.Diagnostics.Debug.Assert(!rollbackSegmentInfos.HasExternalSegments(directory));
1351  rollbackSegments = new HashMap<SegmentInfo, int?>();
1352  int size = rollbackSegmentInfos.Count;
1353  for (int i = 0; i < size; i++)
1354  rollbackSegments[rollbackSegmentInfos.Info(i)] = i;
1355  }
1356  }
1357 
1359  public virtual void SetMergePolicy(MergePolicy mp)
1360  {
1361  EnsureOpen();
1362  if (mp == null)
1363  throw new System.NullReferenceException("MergePolicy must be non-null");
1364 
1365  if (mergePolicy != mp)
1366  mergePolicy.Close();
1367  mergePolicy = mp;
1368  PushMaxBufferedDocs();
1369  if (infoStream != null)
1370  {
1371  Message("setMergePolicy " + mp);
1372  }
1373  }
1374 
1378  public virtual MergePolicy MergePolicy
1379  {
1380  get
1381  {
1382  EnsureOpen();
1383  return mergePolicy;
1384  }
1385  }
1386 
1388  public virtual void SetMergeScheduler(MergeScheduler mergeScheduler)
1389  {
1390  lock (this)
1391  {
1392  EnsureOpen();
1393  if (mergeScheduler == null)
1394  throw new System.NullReferenceException("MergeScheduler must be non-null");
1395 
1396  if (this.mergeScheduler != mergeScheduler)
1397  {
1398  FinishMerges(true);
1399  this.mergeScheduler.Close();
1400  }
1401  this.mergeScheduler = mergeScheduler;
1402  if (infoStream != null)
1403  {
1404  Message("setMergeScheduler " + mergeScheduler);
1405  }
1406  }
1407  }
1408 
1414  public virtual MergeScheduler MergeScheduler
1415  {
1416  get
1417  {
1418  EnsureOpen();
1419  return mergeScheduler;
1420  }
1421  }
1422 
1446  public virtual int MaxMergeDocs
1447  {
1448  get { return LogMergePolicy.MaxMergeDocs; }
1449  set { LogMergePolicy.MaxMergeDocs = value; }
1450  }
1451 
1465  public virtual void SetMaxFieldLength(int maxFieldLength)
1466  {
1467  EnsureOpen();
1468  this.maxFieldLength = maxFieldLength;
1469  docWriter.SetMaxFieldLength(maxFieldLength);
1470  if (infoStream != null)
1471  Message("setMaxFieldLength " + maxFieldLength);
1472  }
1473 
1479  [System.Diagnostics.CodeAnalysis.SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate")]
1480  public virtual int GetMaxFieldLength()
1481  {
1482  EnsureOpen();
1483  return maxFieldLength;
1484  }
1485 
1491  public int ReaderTermsIndexDivisor
1492  {
1493  get
1494  {
1495  EnsureOpen();
1496  return readerTermsIndexDivisor;
1497  }
1498  set
1499  {
1500  EnsureOpen();
1501  if (value <= 0)
1502  {
1503  throw new ArgumentException("divisor must be >= 1 (got " + value + ")");
1504  }
1505  readerTermsIndexDivisor = value;
1506  if (infoStream != null)
1507  {
1508  Message("setReaderTermsIndexDivisor " + readerTermsIndexDivisor);
1509  }
1510  }
1511  }
1512 
1534  public virtual void SetMaxBufferedDocs(int maxBufferedDocs)
1535  {
1536  EnsureOpen();
1537  if (maxBufferedDocs != DISABLE_AUTO_FLUSH && maxBufferedDocs < 2)
1538  throw new ArgumentException("maxBufferedDocs must at least be 2 when enabled");
1539 
1540  if (maxBufferedDocs == DISABLE_AUTO_FLUSH && (int)GetRAMBufferSizeMB() == DISABLE_AUTO_FLUSH)
1541  throw new ArgumentException("at least one of ramBufferSize and maxBufferedDocs must be enabled");
1542 
1543  docWriter.MaxBufferedDocs = maxBufferedDocs;
1544  PushMaxBufferedDocs();
1545  if (infoStream != null)
1546  Message("setMaxBufferedDocs " + maxBufferedDocs);
1547  }
1548 
1553  private void PushMaxBufferedDocs()
1554  {
1555  if (docWriter.MaxBufferedDocs != DISABLE_AUTO_FLUSH)
1556  {
1557  MergePolicy mp = mergePolicy;
1558  if (mp is LogDocMergePolicy)
1559  {
1560  LogDocMergePolicy lmp = (LogDocMergePolicy) mp;
1561  int maxBufferedDocs = docWriter.MaxBufferedDocs;
1562  if (lmp.MinMergeDocs != maxBufferedDocs)
1563  {
1564  if (infoStream != null)
1565  Message("now push maxBufferedDocs " + maxBufferedDocs + " to LogDocMergePolicy");
1566  lmp.MinMergeDocs = maxBufferedDocs;
1567  }
1568  }
1569  }
1570  }
1571 
1577  [System.Diagnostics.CodeAnalysis.SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate")]
1578  public virtual int GetMaxBufferedDocs()
1579  {
1580  EnsureOpen();
1581  return docWriter.MaxBufferedDocs;
1582  }
1583 
1623  public virtual void SetRAMBufferSizeMB(double mb)
1624  {
1625  if (mb > 2048.0)
1626  {
1627  throw new System.ArgumentException("ramBufferSize " + mb + " is too large; should be comfortably less than 2048");
1628  }
1629  if (mb != DISABLE_AUTO_FLUSH && mb <= 0.0)
1630  throw new System.ArgumentException("ramBufferSize should be > 0.0 MB when enabled");
1631  if (mb == DISABLE_AUTO_FLUSH && GetMaxBufferedDocs() == DISABLE_AUTO_FLUSH)
1632  throw new System.ArgumentException("at least one of ramBufferSize and maxBufferedDocs must be enabled");
1633  docWriter.SetRAMBufferSizeMB(mb);
1634  if (infoStream != null)
1635  Message("setRAMBufferSizeMB " + mb);
1636  }
1637 
1639  [System.Diagnostics.CodeAnalysis.SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate")]
1640  public virtual double GetRAMBufferSizeMB()
1641  {
1642  return docWriter.GetRAMBufferSizeMB();
1643  }
1644 
1657  public virtual void SetMaxBufferedDeleteTerms(int maxBufferedDeleteTerms)
1658  {
1659  EnsureOpen();
1660  if (maxBufferedDeleteTerms != DISABLE_AUTO_FLUSH && maxBufferedDeleteTerms < 1)
1661  throw new System.ArgumentException("maxBufferedDeleteTerms must at least be 1 when enabled");
1662  docWriter.MaxBufferedDeleteTerms = maxBufferedDeleteTerms;
1663  if (infoStream != null)
1664  Message("setMaxBufferedDeleteTerms " + maxBufferedDeleteTerms);
1665  }
1666 
1672  [System.Diagnostics.CodeAnalysis.SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate")]
1673  public virtual int GetMaxBufferedDeleteTerms()
1674  {
1675  EnsureOpen();
1676  return docWriter.MaxBufferedDeleteTerms;
1677  }
1678 
1697  public virtual int MergeFactor
1698  {
1699  set { LogMergePolicy.MergeFactor = value; }
1700  get { return LogMergePolicy.MergeFactor; }
1701  }
1702 
1709  public static StreamWriter DefaultInfoStream
1710  {
1711  set { IndexWriter.defaultInfoStream = value; }
1712  get { return IndexWriter.defaultInfoStream; }
1713  }
1714 
1719  public virtual void SetInfoStream(System.IO.StreamWriter infoStream)
1720  {
1721  EnsureOpen();
1722  SetMessageID(infoStream);
1723  docWriter.SetInfoStream(infoStream);
1724  deleter.SetInfoStream(infoStream);
1725  if (infoStream != null)
1726  MessageState();
1727  }
1728 
1729  private void MessageState()
1730  {
1731  Message("setInfoStream: dir=" + directory +
1732  " mergePolicy=" + mergePolicy +
1733  " mergeScheduler=" + mergeScheduler +
1734  " ramBufferSizeMB=" + docWriter.GetRAMBufferSizeMB() +
1735  " maxBufferedDocs=" + docWriter.MaxBufferedDocs +
1736  " maxBuffereDeleteTerms=" + docWriter.MaxBufferedDeleteTerms +
1737  " maxFieldLength=" + maxFieldLength +
1738  " index=" + SegString());
1739  }
1740 
1744  public virtual StreamWriter InfoStream
1745  {
1746  get
1747  {
1748  EnsureOpen();
1749  return infoStream;
1750  }
1751  }
1752 
1754  public virtual bool Verbose
1755  {
1756  get { return infoStream != null; }
1757  }
1758 
1760  public virtual long WriteLockTimeout
1761  {
1762  get
1763  {
1764  EnsureOpen();
1765  return writeLockTimeout;
1766  }
1767  set
1768  {
1769  EnsureOpen();
1770  this.writeLockTimeout = value;
1771  }
1772  }
1773 
1777  public static long DefaultWriteLockTimeout
1778  {
1779  set { IndexWriter.WRITE_LOCK_TIMEOUT = value; }
1780  get { return IndexWriter.WRITE_LOCK_TIMEOUT; }
1781  }
1782 
1824  [Obsolete("Use Dispose() instead")]
1825  public void Close()
1826  {
1827  Dispose(true);
1828  }
1829 
1871  public virtual void Dispose()
1872  {
1873  Dispose(true);
1874  }
1875 
1899  public virtual void Dispose(bool waitForMerges)
1900  {
1901  Dispose(true, waitForMerges);
1902  }
1903 
1904  protected virtual void Dispose(bool disposing, bool waitForMerges)
1905  {
1906  if (disposing)
1907  {
1908  // Ensure that only one thread actually gets to do the closing:
1909  if (ShouldClose())
1910  {
1911  // If any methods have hit OutOfMemoryError, then abort
1912  // on close, in case the internal state of IndexWriter
1913  // or DocumentsWriter is corrupt
1914  if (hitOOM)
1915  RollbackInternal();
1916  else
1917  CloseInternal(waitForMerges);
1918  }
1919  }
1920  }
1921 
1945  [Obsolete("Use Dispose(bool) instead")]
1946  public virtual void Close(bool waitForMerges)
1947  {
1948  Dispose(waitForMerges);
1949  }
1950 
1951  // Returns true if this thread should attempt to close, or
1952  // false if IndexWriter is now closed; else, waits until
1953  // another thread finishes closing
1954  private bool ShouldClose()
1955  {
1956  lock (this)
1957  {
1958  while (true)
1959  {
1960  if (!closed)
1961  {
1962  if (!closing)
1963  {
1964  closing = true;
1965  return true;
1966  }
1967  else
1968  {
1969  // Another thread is presently trying to close;
1970  // wait until it finishes one way (closes
1971  // successfully) or another (fails to close)
1972  DoWait();
1973  }
1974  }
1975  else
1976  return false;
1977  }
1978  }
1979  }
1980 
1981  private void CloseInternal(bool waitForMerges)
1982  {
1983 
1984  docWriter.PauseAllThreads();
1985 
1986  try
1987  {
1988  if (infoStream != null)
1989  Message("now flush at close");
1990 
1991  docWriter.Dispose();
1992 
1993  // Only allow a new merge to be triggered if we are
1994  // going to wait for merges:
1995  if (!hitOOM)
1996  {
1997  Flush(waitForMerges, true, true);
1998  }
1999 
2000  if (waitForMerges)
2001  // Give merge scheduler last chance to run, in case
2002  // any pending merges are waiting:
2003  mergeScheduler.Merge(this);
2004 
2005  mergePolicy.Close();
2006 
2007  FinishMerges(waitForMerges);
2008  stopMerges = true;
2009 
2010  mergeScheduler.Close();
2011 
2012  if (infoStream != null)
2013  Message("now call final commit()");
2014 
2015  if (!hitOOM)
2016  {
2017  Commit(0);
2018  }
2019 
2020  if (infoStream != null)
2021  Message("at close: " + SegString());
2022 
2023  lock (this)
2024  {
2025  readerPool.Dispose();
2026  docWriter = null;
2027  deleter.Dispose();
2028  }
2029 
2030  if (writeLock != null)
2031  {
2032  writeLock.Release(); // release write lock
2033  writeLock = null;
2034  }
2035  lock (this)
2036  {
2037  closed = true;
2038  }
2039  }
2040  catch (System.OutOfMemoryException oom)
2041  {
2042  HandleOOM(oom, "closeInternal");
2043  }
2044  finally
2045  {
2046  lock (this)
2047  {
2048  closing = false;
2049  System.Threading.Monitor.PulseAll(this);
2050  if (!closed)
2051  {
2052  if (docWriter != null)
2053  docWriter.ResumeAllThreads();
2054  if (infoStream != null)
2055  Message("hit exception while closing");
2056  }
2057  }
2058  }
2059  }
2060 
2065  private bool FlushDocStores()
2066  {
2067  lock (this)
2068  {
2069  if (infoStream != null)
2070  {
2071  Message("flushDocStores segment=" + docWriter.DocStoreSegment);
2072  }
2073 
2074  bool useCompoundDocStore = false;
2075  if (infoStream != null)
2076  {
2077  Message("closeDocStores segment=" + docWriter.DocStoreSegment);
2078  }
2079 
2080  System.String docStoreSegment;
2081 
2082  bool success = false;
2083  try
2084  {
2085  docStoreSegment = docWriter.CloseDocStore();
2086  success = true;
2087  }
2088  finally
2089  {
2090  if (!success && infoStream != null)
2091  {
2092  Message("hit exception closing doc store segment");
2093  }
2094  }
2095 
2096  if (infoStream != null)
2097  {
2098  Message("flushDocStores files=" + docWriter.ClosedFiles());
2099  }
2100 
2101  useCompoundDocStore = mergePolicy.UseCompoundDocStore(segmentInfos);
2102 
2103  if (useCompoundDocStore && docStoreSegment != null && docWriter.ClosedFiles().Count != 0)
2104  {
2105  // Now build compound doc store file
2106 
2107  if (infoStream != null)
2108  {
2109  Message("create compound file " + docStoreSegment + "." + IndexFileNames.COMPOUND_FILE_STORE_EXTENSION);
2110  }
2111 
2112  success = false;
2113 
2114  int numSegments = segmentInfos.Count;
2115  System.String compoundFileName = docStoreSegment + "." + IndexFileNames.COMPOUND_FILE_STORE_EXTENSION;
2116 
2117  try
2118  {
2119  CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
2120  foreach(string file in docWriter.closedFiles)
2121  {
2122  cfsWriter.AddFile(file);
2123  }
2124 
2125  // Perform the merge
2126  cfsWriter.Close();
2127  success = true;
2128  }
2129  finally
2130  {
2131  if (!success)
2132  {
2133  if (infoStream != null)
2134  Message("hit exception building compound file doc store for segment " + docStoreSegment);
2135  deleter.DeleteFile(compoundFileName);
2136  docWriter.Abort();
2137  }
2138  }
2139 
2140  for (int i = 0; i < numSegments; i++)
2141  {
2142  SegmentInfo si = segmentInfos.Info(i);
2143  if (si.DocStoreOffset != - 1 && si.DocStoreSegment.Equals(docStoreSegment))
2144  si.DocStoreIsCompoundFile = true;
2145  }
2146 
2147  Checkpoint();
2148 
2149  // In case the files we just merged into a CFS were
2150  // not previously checkpointed:
2151  deleter.DeleteNewFiles(docWriter.ClosedFiles());
2152  }
2153 
2154  return useCompoundDocStore;
2155  }
2156  }
2157 
2159  public virtual Directory Directory
2160  {
2161  get
2162  {
2163  // Pass false because the flush during closing calls getDirectory
2164  EnsureOpen(false);
2165  return directory;
2166  }
2167  }
2168 
2170  public virtual Analyzer Analyzer
2171  {
2172  get
2173  {
2174  EnsureOpen();
2175  return analyzer;
2176  }
2177  }
2178 
2185  public virtual int MaxDoc()
2186  {
2187  lock (this)
2188  {
2189  int count;
2190  if (docWriter != null)
2191  count = docWriter.NumDocsInRAM;
2192  else
2193  count = 0;
2194 
2195  for (int i = 0; i < segmentInfos.Count; i++)
2196  count += segmentInfos.Info(i).docCount;
2197  return count;
2198  }
2199  }
2200 
2209  public virtual int NumDocs()
2210  {
2211  lock (this)
2212  {
2213  int count;
2214  if (docWriter != null)
2215  count = docWriter.NumDocsInRAM;
2216  else
2217  count = 0;
2218 
2219  for (int i = 0; i < segmentInfos.Count; i++)
2220  {
2221  SegmentInfo info = segmentInfos.Info(i);
2222  count += info.docCount - info.GetDelCount();
2223  }
2224  return count;
2225  }
2226  }
2227 
2228  public virtual bool HasDeletions()
2229  {
2230  lock (this)
2231  {
2232  EnsureOpen();
2233  if (docWriter.HasDeletes())
2234  return true;
2235  for (int i = 0; i < segmentInfos.Count; i++)
2236  if (segmentInfos.Info(i).HasDeletions())
2237  return true;
2238  return false;
2239  }
2240  }
2241 
2256  private int maxFieldLength;
2257 
2301  public virtual void AddDocument(Document doc)
2302  {
2303  AddDocument(doc, analyzer);
2304  }
2305 
2322  public virtual void AddDocument(Document doc, Analyzer analyzer)
2323  {
2324  EnsureOpen();
2325  bool doFlush = false;
2326  bool success = false;
2327  try
2328  {
2329  try
2330  {
2331  doFlush = docWriter.AddDocument(doc, analyzer);
2332  success = true;
2333  }
2334  finally
2335  {
2336  if (!success)
2337  {
2338 
2339  if (infoStream != null)
2340  Message("hit exception adding document");
2341 
2342  lock (this)
2343  {
2344  // If docWriter has some aborted files that were
2345  // never incref'd, then we clean them up here
2346  if (docWriter != null)
2347  {
2348  ICollection<string> files = docWriter.AbortedFiles();
2349  if (files != null)
2350  deleter.DeleteNewFiles(files);
2351  }
2352  }
2353  }
2354  }
2355  if (doFlush)
2356  Flush(true, false, false);
2357  }
2358  catch (System.OutOfMemoryException oom)
2359  {
2360  HandleOOM(oom, "addDocument");
2361  }
2362  }
2363 
2375  public virtual void DeleteDocuments(Term term)
2376  {
2377  EnsureOpen();
2378  try
2379  {
2380  bool doFlush = docWriter.BufferDeleteTerm(term);
2381  if (doFlush)
2382  Flush(true, false, false);
2383  }
2384  catch (System.OutOfMemoryException oom)
2385  {
2386  HandleOOM(oom, "deleteDocuments(Term)");
2387  }
2388  }
2389 
2403  public virtual void DeleteDocuments(params Term[] terms)
2404  {
2405  EnsureOpen();
2406  try
2407  {
2408  bool doFlush = docWriter.BufferDeleteTerms(terms);
2409  if (doFlush)
2410  Flush(true, false, false);
2411  }
2412  catch (System.OutOfMemoryException oom)
2413  {
2414  HandleOOM(oom, "deleteDocuments(params Term[])");
2415  }
2416  }
2417 
2429  public virtual void DeleteDocuments(Query query)
2430  {
2431  EnsureOpen();
2432  bool doFlush = docWriter.BufferDeleteQuery(query);
2433  if (doFlush)
2434  Flush(true, false, false);
2435  }
2436 
2450  public virtual void DeleteDocuments(params Query[] queries)
2451  {
2452  EnsureOpen();
2453  bool doFlush = docWriter.BufferDeleteQueries(queries);
2454  if (doFlush)
2455  Flush(true, false, false);
2456  }
2457 
2476  public virtual void UpdateDocument(Term term, Document doc)
2477  {
2478  EnsureOpen();
2479  UpdateDocument(term, doc, Analyzer);
2480  }
2481 
2502  public virtual void UpdateDocument(Term term, Document doc, Analyzer analyzer)
2503  {
2504  EnsureOpen();
2505  try
2506  {
2507  bool doFlush = false;
2508  bool success = false;
2509  try
2510  {
2511  doFlush = docWriter.UpdateDocument(term, doc, analyzer);
2512  success = true;
2513  }
2514  finally
2515  {
2516  if (!success)
2517  {
2518 
2519  if (infoStream != null)
2520  Message("hit exception updating document");
2521 
2522  lock (this)
2523  {
2524  // If docWriter has some aborted files that were
2525  // never incref'd, then we clean them up here
2526  ICollection<string> files = docWriter.AbortedFiles();
2527  if (files != null)
2528  deleter.DeleteNewFiles(files);
2529  }
2530  }
2531  }
2532  if (doFlush)
2533  Flush(true, false, false);
2534  }
2535  catch (System.OutOfMemoryException oom)
2536  {
2537  HandleOOM(oom, "updateDocument");
2538  }
2539  }
2540 
2541  // for test purpose
2542  internal int GetSegmentCount()
2543  {
2544  lock (this)
2545  {
2546  return segmentInfos.Count;
2547  }
2548  }
2549 
2550  // for test purpose
2551  internal int GetNumBufferedDocuments()
2552  {
2553  lock (this)
2554  {
2555  return docWriter.NumDocsInRAM;
2556  }
2557  }
2558 
2559  // for test purpose
2560  public /*internal*/ int GetDocCount(int i)
2561  {
2562  lock (this)
2563  {
2564  if (i >= 0 && i < segmentInfos.Count)
2565  {
2566  return segmentInfos.Info(i).docCount;
2567  }
2568  else
2569  {
2570  return - 1;
2571  }
2572  }
2573  }
2574 
2575  // for test purpose
2576  internal int GetFlushCount()
2577  {
2578  lock (this)
2579  {
2580  return flushCount;
2581  }
2582  }
2583 
2584  // for test purpose
2585  internal int GetFlushDeletesCount()
2586  {
2587  lock (this)
2588  {
2589  return flushDeletesCount;
2590  }
2591  }
2592 
2593  internal System.String NewSegmentName()
2594  {
2595  // Cannot synchronize on IndexWriter because that causes
2596  // deadlock
2597  lock (segmentInfos)
2598  {
2599  // Important to increment changeCount so that the
2600  // segmentInfos is written on close. Otherwise we
2601  // could close, re-open and re-return the same segment
2602  // name that was previously returned which can cause
2603  // problems at least with ConcurrentMergeScheduler.
2604  changeCount++;
2605  return "_" + Number.ToString(segmentInfos.counter++);
2606  }
2607  }
2608 
2610  private System.IO.StreamWriter infoStream = null;
2611  private static System.IO.StreamWriter defaultInfoStream = null;
2612 
2671  public virtual void Optimize()
2672  {
2673  Optimize(true);
2674  }
2675 
2688  public virtual void Optimize(int maxNumSegments)
2689  {
2690  Optimize(maxNumSegments, true);
2691  }
2692 
2703  public virtual void Optimize(bool doWait)
2704  {
2705  Optimize(1, doWait);
2706  }
2707 
2718  public virtual void Optimize(int maxNumSegments, bool doWait)
2719  {
2720  EnsureOpen();
2721 
2722  if (maxNumSegments < 1)
2723  throw new System.ArgumentException("maxNumSegments must be >= 1; got " + maxNumSegments);
2724 
2725  if (infoStream != null)
2726  Message("optimize: index now " + SegString());
2727 
2728  Flush(true, false, true);
2729 
2730  lock (this)
2731  {
2732  ResetMergeExceptions();
2733  segmentsToOptimize = Lucene.Net.Support.Compatibility.SetFactory.CreateHashSet<SegmentInfo>();
2734  optimizeMaxNumSegments = maxNumSegments;
2735  int numSegments = segmentInfos.Count;
2736  for (int i = 0; i < numSegments; i++)
2737  segmentsToOptimize.Add(segmentInfos.Info(i));
2738 
2739  // Now mark all pending & running merges as optimize
2740  // merge:
2741  foreach(MergePolicy.OneMerge merge in pendingMerges)
2742  {
2743  merge.optimize = true;
2744  merge.maxNumSegmentsOptimize = maxNumSegments;
2745  }
2746 
2747  foreach(MergePolicy.OneMerge merge in runningMerges)
2748  {
2749  merge.optimize = true;
2750  merge.maxNumSegmentsOptimize = maxNumSegments;
2751  }
2752  }
2753 
2754  MaybeMerge(maxNumSegments, true);
2755 
2756  if (doWait)
2757  {
2758  lock (this)
2759  {
2760  while (true)
2761  {
2762 
2763  if (hitOOM)
2764  {
2765  throw new System.SystemException("this writer hit an OutOfMemoryError; cannot complete optimize");
2766  }
2767 
2768  if (mergeExceptions.Count > 0)
2769  {
2770  // Forward any exceptions in background merge
2771  // threads to the current thread:
2772  int size = mergeExceptions.Count;
2773  for (int i = 0; i < size; i++)
2774  {
2775  MergePolicy.OneMerge merge = mergeExceptions[i];
2776  if (merge.optimize)
2777  {
2778  System.IO.IOException err;
2779  System.Exception t = merge.GetException();
2780  if (t != null)
2781  err = new System.IO.IOException("background merge hit exception: " + merge.SegString(directory), t);
2782  else
2783  err = new System.IO.IOException("background merge hit exception: " + merge.SegString(directory));
2784  throw err;
2785  }
2786  }
2787  }
2788 
2789  if (OptimizeMergesPending())
2790  DoWait();
2791  else
2792  break;
2793  }
2794  }
2795 
2796  // If close is called while we are still
2797  // running, throw an exception so the calling
2798  // thread will know the optimize did not
2799  // complete
2800  EnsureOpen();
2801  }
2802 
2803  // NOTE: in the ConcurrentMergeScheduler case, when
2804  // doWait is false, we can return immediately while
2805  // background threads accomplish the optimization
2806  }
2807 
2811  private bool OptimizeMergesPending()
2812  {
2813  lock (this)
2814  {
2815  foreach (MergePolicy.OneMerge merge in pendingMerges)
2816  {
2817  if (merge.optimize) return true;
2818  }
2819 
2820  foreach(MergePolicy.OneMerge merge in runningMerges)
2821  {
2822  if (merge.optimize) return true;
2823  }
2824 
2825  return false;
2826  }
2827  }
2828 
2839  public virtual void ExpungeDeletes(bool doWait)
2840  {
2841  EnsureOpen();
2842 
2843  if (infoStream != null)
2844  Message("expungeDeletes: index now " + SegString());
2845 
2846  MergePolicy.MergeSpecification spec;
2847 
2848  lock (this)
2849  {
2850  spec = mergePolicy.FindMergesToExpungeDeletes(segmentInfos);
2851  if (spec != null)
2852  {
2853  int numMerges = spec.merges.Count;
2854  for (int i = 0; i < numMerges; i++)
2855  RegisterMerge(spec.merges[i]);
2856  }
2857  }
2858 
2859  mergeScheduler.Merge(this);
2860 
2861  if (spec != null && doWait)
2862  {
2863  int numMerges = spec.merges.Count;
2864  lock (this)
2865  {
2866  bool running = true;
2867  while (running)
2868  {
2869 
2870  if (hitOOM)
2871  {
2872  throw new System.SystemException("this writer hit an OutOfMemoryError; cannot complete expungeDeletes");
2873  }
2874 
2875  // Check each merge that MergePolicy asked us to
2876  // do, to see if any of them are still running and
2877  // if any of them have hit an exception.
2878  running = false;
2879  for (int i = 0; i < numMerges; i++)
2880  {
2881  MergePolicy.OneMerge merge = spec.merges[i];
2882  if (pendingMerges.Contains(merge) || runningMerges.Contains(merge))
2883  running = true;
2884  System.Exception t = merge.GetException();
2885  if (t != null)
2886  {
2887  System.IO.IOException ioe = new System.IO.IOException("background merge hit exception: " + merge.SegString(directory), t);
2888  throw ioe;
2889  }
2890  }
2891 
2892  // If any of our merges are still running, wait:
2893  if (running)
2894  DoWait();
2895  }
2896  }
2897  }
2898 
2899  // NOTE: in the ConcurrentMergeScheduler case, when
2900  // doWait is false, we can return immediately while
2901  // background threads accomplish the optimization
2902  }
2903 
2904 
2925  public virtual void ExpungeDeletes()
2926  {
2927  ExpungeDeletes(true);
2928  }
2929 
2943  public void MaybeMerge()
2944  {
2945  MaybeMerge(false);
2946  }
2947 
2948  private void MaybeMerge(bool optimize)
2949  {
2950  MaybeMerge(1, optimize);
2951  }
2952 
2953  private void MaybeMerge(int maxNumSegmentsOptimize, bool optimize)
2954  {
2955  UpdatePendingMerges(maxNumSegmentsOptimize, optimize);
2956  mergeScheduler.Merge(this);
2957  }
2958 
2959  private void UpdatePendingMerges(int maxNumSegmentsOptimize, bool optimize)
2960  {
2961  lock (this)
2962  {
2963  System.Diagnostics.Debug.Assert(!optimize || maxNumSegmentsOptimize > 0);
2964 
2965  if (stopMerges)
2966  {
2967  return;
2968  }
2969 
2970  // Do not start new merges if we've hit OOME
2971  if (hitOOM)
2972  {
2973  return ;
2974  }
2975 
2976  MergePolicy.MergeSpecification spec;
2977  if (optimize)
2978  {
2979  spec = mergePolicy.FindMergesForOptimize(segmentInfos, maxNumSegmentsOptimize, segmentsToOptimize);
2980 
2981  if (spec != null)
2982  {
2983  int numMerges = spec.merges.Count;
2984  for (int i = 0; i < numMerges; i++)
2985  {
2986  MergePolicy.OneMerge merge = spec.merges[i];
2987  merge.optimize = true;
2988  merge.maxNumSegmentsOptimize = maxNumSegmentsOptimize;
2989  }
2990  }
2991  }
2992  else
2993  {
2994  spec = mergePolicy.FindMerges(segmentInfos);
2995  }
2996 
2997  if (spec != null)
2998  {
2999  int numMerges = spec.merges.Count;
3000  for (int i = 0; i < numMerges; i++)
3001  RegisterMerge(spec.merges[i]);
3002  }
3003  }
3004  }
3005 
3010  internal virtual MergePolicy.OneMerge GetNextMerge()
3011  {
3012  lock (this)
3013  {
3014  if (pendingMerges.Count == 0)
3015  return null;
3016  else
3017  {
3018  // Advance the merge from pending to running
3019  MergePolicy.OneMerge merge = pendingMerges.First.Value;
3020  pendingMerges.RemoveFirst();
3021  runningMerges.Add(merge);
3022  return merge;
3023  }
3024  }
3025  }
3026 
3030  private MergePolicy.OneMerge GetNextExternalMerge()
3031  {
3032  lock (this)
3033  {
3034  if (pendingMerges.Count == 0)
3035  return null;
3036  else
3037  {
3038  var it = pendingMerges.GetEnumerator();
3039  while (it.MoveNext())
3040  {
3041  MergePolicy.OneMerge merge = it.Current;
3042  if (merge.isExternal)
3043  {
3044  // Advance the merge from pending to running
3045  pendingMerges.Remove(merge); // {{Aroush-2.9}} From Mike Garski: this is an O(n) op... is that an issue?
3046  runningMerges.Add(merge);
3047  return merge;
3048  }
3049  }
3050 
3051  // All existing merges do not involve external segments
3052  return null;
3053  }
3054  }
3055  }
3056 
3057  /*
3058  * Begin a transaction. During a transaction, any segment
3059  * merges that happen (or ram segments flushed) will not
3060  * write a new segments file and will not remove any files
3061  * that were present at the start of the transaction. You
3062  * must make a matched (try/finally) call to
3063  * commitTransaction() or rollbackTransaction() to finish
3064  * the transaction.
3065  *
3066  * Note that buffered documents and delete terms are not handled
3067  * within the transactions, so they must be flushed before the
3068  * transaction is started.
3069  */
3070  private void StartTransaction(bool haveReadLock)
3071  {
3072  lock (this)
3073  {
3074 
3075  bool success = false;
3076  try
3077  {
3078  if (infoStream != null)
3079  Message("now start transaction");
3080 
3081  System.Diagnostics.Debug.Assert(docWriter.GetNumBufferedDeleteTerms() == 0 ,
3082  "calling startTransaction with buffered delete terms not supported: numBufferedDeleteTerms=" + docWriter.GetNumBufferedDeleteTerms());
3083  System.Diagnostics.Debug.Assert(docWriter.NumDocsInRAM == 0 ,
3084  "calling startTransaction with buffered documents not supported: numDocsInRAM=" + docWriter.NumDocsInRAM);
3085 
3086  EnsureOpen();
3087 
3088  // If a transaction is trying to roll back (because
3089  // addIndexes hit an exception) then wait here until
3090  // that's done:
3091  lock (this)
3092  {
3093  while (stopMerges)
3094  DoWait();
3095  }
3096  success = true;
3097  }
3098  finally
3099  {
3100  // Release the write lock if our caller held it, on
3101  // hitting an exception
3102  if (!success && haveReadLock)
3103  ReleaseRead();
3104  }
3105 
3106  if (haveReadLock)
3107  {
3108  UpgradeReadToWrite();
3109  }
3110  else
3111  {
3112  AcquireWrite();
3113  }
3114 
3115  success = false;
3116  try
3117  {
3118  localRollbackSegmentInfos = (SegmentInfos) segmentInfos.Clone();
3119 
3120  System.Diagnostics.Debug.Assert(!HasExternalSegments());
3121 
3122  localFlushedDocCount = docWriter.GetFlushedDocCount();
3123 
3124  // Remove the incRef we did in startTransaction:
3125  deleter.IncRef(segmentInfos, false);
3126 
3127  success = true;
3128  }
3129  finally
3130  {
3131  if (!success)
3132  FinishAddIndexes();
3133  }
3134  }
3135  }
3136 
3137  /*
3138  * Rolls back the transaction and restores state to where
3139  * we were at the start.
3140  */
3141  private void RollbackTransaction()
3142  {
3143  lock (this)
3144  {
3145 
3146  if (infoStream != null)
3147  Message("now rollback transaction");
3148 
3149  if (docWriter != null)
3150  {
3151  docWriter.SetFlushedDocCount(localFlushedDocCount);
3152  }
3153 
3154  // Must finish merges before rolling back segmentInfos
3155  // so merges don't hit exceptions on trying to commit
3156  // themselves, don't get files deleted out from under
3157  // them, etc:
3158  FinishMerges(false);
3159 
3160  // Keep the same segmentInfos instance but replace all
3161  // of its SegmentInfo instances. This is so the next
3162  // attempt to commit using this instance of IndexWriter
3163  // will always write to a new generation ("write once").
3164  segmentInfos.Clear();
3165  segmentInfos.AddRange(localRollbackSegmentInfos);
3166  localRollbackSegmentInfos = null;
3167 
3168  // This must come after we rollback segmentInfos, so
3169  // that if a commit() kicks off it does not see the
3170  // segmentInfos with external segments
3171  FinishAddIndexes();
3172 
3173  // Ask deleter to locate unreferenced files we had
3174  // created & remove them:
3175  deleter.Checkpoint(segmentInfos, false);
3176 
3177  // Remove the incRef we did in startTransaction:
3178  deleter.DecRef(segmentInfos);
3179 
3180  // Also ask deleter to remove any newly created files
3181  // that were never incref'd; this "garbage" is created
3182  // when a merge kicks off but aborts part way through
3183  // before it had a chance to incRef the files it had
3184  // partially created
3185  deleter.Refresh();
3186 
3187  System.Threading.Monitor.PulseAll(this);
3188 
3189  System.Diagnostics.Debug.Assert(!HasExternalSegments());
3190  }
3191  }
3192 
3193  /*
3194  * Commits the transaction. This will write the new
3195  * segments file and remove and pending deletions we have
3196  * accumulated during the transaction
3197  */
3198  private void CommitTransaction()
3199  {
3200  lock (this)
3201  {
3202 
3203  if (infoStream != null)
3204  Message("now commit transaction");
3205 
3206  // Give deleter a chance to remove files now:
3207  Checkpoint();
3208 
3209  // Remove the incRef we did in startTransaction.
3210  deleter.DecRef(localRollbackSegmentInfos);
3211 
3212  localRollbackSegmentInfos = null;
3213 
3214  System.Diagnostics.Debug.Assert(!HasExternalSegments());
3215 
3216  FinishAddIndexes();
3217  }
3218  }
3219 
3230  public virtual void Rollback()
3231  {
3232  EnsureOpen();
3233 
3234  // Ensure that only one thread actually gets to do the closing:
3235  if (ShouldClose())
3236  RollbackInternal();
3237  }
3238 
3239  private void RollbackInternal()
3240  {
3241 
3242  bool success = false;
3243 
3244  if (infoStream != null)
3245  {
3246  Message("rollback");
3247  }
3248 
3249  docWriter.PauseAllThreads();
3250 
3251  try
3252  {
3253  FinishMerges(false);
3254 
3255  // Must pre-close these two, in case they increment
3256  // changeCount so that we can then set it to false
3257  // before calling closeInternal
3258  mergePolicy.Close();
3259  mergeScheduler.Close();
3260 
3261  lock (this)
3262  {
3263 
3264  if (pendingCommit != null)
3265  {
3266  pendingCommit.RollbackCommit(directory);
3267  deleter.DecRef(pendingCommit);
3268  pendingCommit = null;
3269  System.Threading.Monitor.PulseAll(this);
3270  }
3271 
3272  // Keep the same segmentInfos instance but replace all
3273  // of its SegmentInfo instances. This is so the next
3274  // attempt to commit using this instance of IndexWriter
3275  // will always write to a new generation ("write
3276  // once").
3277  segmentInfos.Clear();
3278  segmentInfos.AddRange(rollbackSegmentInfos);
3279 
3280  System.Diagnostics.Debug.Assert(!HasExternalSegments());
3281 
3282  docWriter.Abort();
3283 
3284  System.Diagnostics.Debug.Assert(TestPoint("rollback before checkpoint"));
3285 
3286  // Ask deleter to locate unreferenced files & remove
3287  // them:
3288  deleter.Checkpoint(segmentInfos, false);
3289  deleter.Refresh();
3290  }
3291 
3292  // Don't bother saving any changes in our segmentInfos
3293  readerPool.Clear(null);
3294 
3295  lastCommitChangeCount = changeCount;
3296 
3297  success = true;
3298  }
3299  catch (System.OutOfMemoryException oom)
3300  {
3301  HandleOOM(oom, "rollbackInternal");
3302  }
3303  finally
3304  {
3305  lock (this)
3306  {
3307  if (!success)
3308  {
3309  docWriter.ResumeAllThreads();
3310  closing = false;
3311  System.Threading.Monitor.PulseAll(this);
3312  if (infoStream != null)
3313  Message("hit exception during rollback");
3314  }
3315  }
3316  }
3317 
3318  CloseInternal(false);
3319  }
3320 
3335  public virtual void DeleteAll()
3336  {
3337  lock (this)
3338  {
3339  docWriter.PauseAllThreads();
3340  try
3341  {
3342 
3343  // Abort any running merges
3344  FinishMerges(false);
3345 
3346  // Remove any buffered docs
3347  docWriter.Abort();
3348  docWriter.SetFlushedDocCount(0);
3349 
3350  // Remove all segments
3351  segmentInfos.Clear();
3352 
3353  // Ask deleter to locate unreferenced files & remove them:
3354  deleter.Checkpoint(segmentInfos, false);
3355  deleter.Refresh();
3356 
3357  // Don't bother saving any changes in our segmentInfos
3358  readerPool.Clear(null);
3359 
3360  // Mark that the index has changed
3361  ++changeCount;
3362  }
3363  catch (System.OutOfMemoryException oom)
3364  {
3365  HandleOOM(oom, "deleteAll");
3366  }
3367  finally
3368  {
3369  docWriter.ResumeAllThreads();
3370  if (infoStream != null)
3371  {
3372  Message("hit exception during deleteAll");
3373  }
3374  }
3375  }
3376  }
3377 
3378  private void FinishMerges(bool waitForMerges)
3379  {
3380  lock (this)
3381  {
3382  if (!waitForMerges)
3383  {
3384 
3385  stopMerges = true;
3386 
3387  // Abort all pending & running merges:
3388  foreach(MergePolicy.OneMerge merge in pendingMerges)
3389  {
3390  if (infoStream != null)
3391  Message("now abort pending merge " + merge.SegString(directory));
3392  merge.Abort();
3393  MergeFinish(merge);
3394  }
3395  pendingMerges.Clear();
3396 
3397  foreach(MergePolicy.OneMerge merge in runningMerges)
3398  {
3399  if (infoStream != null)
3400  Message("now abort running merge " + merge.SegString(directory));
3401  merge.Abort();
3402  }
3403 
3404  // Ensure any running addIndexes finishes. It's fine
3405  // if a new one attempts to start because its merges
3406  // will quickly see the stopMerges == true and abort.
3407  AcquireRead();
3408  ReleaseRead();
3409 
3410  // These merges periodically check whether they have
3411  // been aborted, and stop if so. We wait here to make
3412  // sure they all stop. It should not take very long
3413  // because the merge threads periodically check if
3414  // they are aborted.
3415  while (runningMerges.Count > 0)
3416  {
3417  if (infoStream != null)
3418  Message("now wait for " + runningMerges.Count + " running merge to abort");
3419  DoWait();
3420  }
3421 
3422  stopMerges = false;
3423  System.Threading.Monitor.PulseAll(this);
3424 
3425  System.Diagnostics.Debug.Assert(0 == mergingSegments.Count);
3426 
3427  if (infoStream != null)
3428  Message("all running merges have aborted");
3429  }
3430  else
3431  {
3432  // waitForMerges() will ensure any running addIndexes finishes.
3433  // It's fine if a new one attempts to start because from our
3434  // caller above the call will see that we are in the
3435  // process of closing, and will throw an
3436  // AlreadyClosedException.
3437  WaitForMerges();
3438  }
3439  }
3440  }
3441 
3447  public virtual void WaitForMerges()
3448  {
3449  lock (this)
3450  {
3451  // Ensure any running addIndexes finishes.
3452  AcquireRead();
3453  ReleaseRead();
3454 
3455  while (pendingMerges.Count > 0 || runningMerges.Count > 0)
3456  {
3457  DoWait();
3458  }
3459 
3460  // sanity check
3461  System.Diagnostics.Debug.Assert(0 == mergingSegments.Count);
3462  }
3463  }
3464 
3465  /*
3466  * Called whenever the SegmentInfos has been updated and
3467  * the index files referenced exist (correctly) in the
3468  * index directory.
3469  */
3470  private void Checkpoint()
3471  {
3472  lock (this)
3473  {
3474  changeCount++;
3475  deleter.Checkpoint(segmentInfos, false);
3476  }
3477  }
3478 
3479  private void FinishAddIndexes()
3480  {
3481  ReleaseWrite();
3482  }
3483 
3484  private void BlockAddIndexes(bool includePendingClose)
3485  {
3486 
3487  AcquireRead();
3488 
3489  bool success = false;
3490  try
3491  {
3492 
3493  // Make sure we are still open since we could have
3494  // waited quite a while for last addIndexes to finish
3495  EnsureOpen(includePendingClose);
3496  success = true;
3497  }
3498  finally
3499  {
3500  if (!success)
3501  ReleaseRead();
3502  }
3503  }
3504 
3505  private void ResumeAddIndexes()
3506  {
3507  ReleaseRead();
3508  }
3509 
3510  private void ResetMergeExceptions()
3511  {
3512  lock (this)
3513  {
3514  mergeExceptions = new List<MergePolicy.OneMerge>();
3515  mergeGen++;
3516  }
3517  }
3518 
3519  private void NoDupDirs(Directory[] dirs)
3520  {
3521  HashSet<Directory> dups = new HashSet<Directory>();
3522  for (int i = 0; i < dirs.Length; i++)
3523  {
3524  if (dups.Contains(dirs[i]))
3525  {
3526  throw new System.ArgumentException("Directory " + dirs[i] + " appears more than once");
3527  }
3528  if (dirs[i] == directory)
3529  throw new System.ArgumentException("Cannot add directory to itself");
3530  dups.Add(dirs[i]);
3531  }
3532  }
3533 
3583  public virtual void AddIndexesNoOptimize(params Directory[] dirs)
3584  {
3585 
3586  EnsureOpen();
3587 
3588  NoDupDirs(dirs);
3589 
3590  // Do not allow add docs or deletes while we are running:
3591  docWriter.PauseAllThreads();
3592 
3593  try
3594  {
3595  if (infoStream != null)
3596  Message("flush at addIndexesNoOptimize");
3597  Flush(true, false, true);
3598 
3599  bool success = false;
3600 
3601  StartTransaction(false);
3602 
3603  try
3604  {
3605 
3606  int docCount = 0;
3607  lock (this)
3608  {
3609  EnsureOpen();
3610 
3611  for (int i = 0; i < dirs.Length; i++)
3612  {
3613  if (directory == dirs[i])
3614  {
3615  // cannot add this index: segments may be deleted in merge before added
3616  throw new System.ArgumentException("Cannot add this index to itself");
3617  }
3618 
3619  SegmentInfos sis = new SegmentInfos(); // read infos from dir
3620  sis.Read(dirs[i]);
3621  for (int j = 0; j < sis.Count; j++)
3622  {
3623  SegmentInfo info = sis.Info(j);
3624  System.Diagnostics.Debug.Assert(!segmentInfos.Contains(info), "dup info dir=" + info.dir + " name=" + info.name);
3625  docCount += info.docCount;
3626  segmentInfos.Add(info); // add each info
3627  }
3628  }
3629  }
3630 
3631  // Notify DocumentsWriter that the flushed count just increased
3632  docWriter.UpdateFlushedDocCount(docCount);
3633 
3634  MaybeMerge();
3635 
3636  EnsureOpen();
3637 
3638  // If after merging there remain segments in the index
3639  // that are in a different directory, just copy these
3640  // over into our index. This is necessary (before
3641  // finishing the transaction) to avoid leaving the
3642  // index in an unusable (inconsistent) state.
3643  ResolveExternalSegments();
3644 
3645  EnsureOpen();
3646 
3647  success = true;
3648  }
3649  finally
3650  {
3651  if (success)
3652  {
3653  CommitTransaction();
3654  }
3655  else
3656  {
3657  RollbackTransaction();
3658  }
3659  }
3660  }
3661  catch (System.OutOfMemoryException oom)
3662  {
3663  HandleOOM(oom, "addIndexesNoOptimize");
3664  }
3665  finally
3666  {
3667  if (docWriter != null)
3668  {
3669  docWriter.ResumeAllThreads();
3670  }
3671  }
3672  }
3673 
3674  private bool HasExternalSegments()
3675  {
3676  return segmentInfos.HasExternalSegments(directory);
3677  }
3678 
3679  /* If any of our segments are using a directory != ours
3680  * then we have to either copy them over one by one, merge
3681  * them (if merge policy has chosen to) or wait until
3682  * currently running merges (in the background) complete.
3683  * We don't return until the SegmentInfos has no more
3684  * external segments. Currently this is only used by
3685  * addIndexesNoOptimize(). */
3686  private void ResolveExternalSegments()
3687  {
3688 
3689  bool any = false;
3690 
3691  bool done = false;
3692 
3693  while (!done)
3694  {
3695  SegmentInfo info = null;
3696  MergePolicy.OneMerge merge = null;
3697  lock (this)
3698  {
3699 
3700  if (stopMerges)
3701  throw new MergePolicy.MergeAbortedException("rollback() was called or addIndexes* hit an unhandled exception");
3702 
3703  int numSegments = segmentInfos.Count;
3704 
3705  done = true;
3706  for (int i = 0; i < numSegments; i++)
3707  {
3708  info = segmentInfos.Info(i);
3709  if (info.dir != directory)
3710  {
3711  done = false;
3712  MergePolicy.OneMerge newMerge = new MergePolicy.OneMerge(segmentInfos.Range(i, 1 + i), mergePolicy is LogMergePolicy && UseCompoundFile);
3713 
3714  // Returns true if no running merge conflicts
3715  // with this one (and, records this merge as
3716  // pending), ie, this segment is not currently
3717  // being merged:
3718  if (RegisterMerge(newMerge))
3719  {
3720  merge = newMerge;
3721 
3722  // If this segment is not currently being
3723  // merged, then advance it to running & run
3724  // the merge ourself (below):
3725  pendingMerges.Remove(merge); // {{Aroush-2.9}} From Mike Garski: this is an O(n) op... is that an issue?
3726  runningMerges.Add(merge);
3727  break;
3728  }
3729  }
3730  }
3731 
3732  if (!done && merge == null)
3733  // We are not yet done (external segments still
3734  // exist in segmentInfos), yet, all such segments
3735  // are currently "covered" by a pending or running
3736  // merge. We now try to grab any pending merge
3737  // that involves external segments:
3738  merge = GetNextExternalMerge();
3739 
3740  if (!done && merge == null)
3741  // We are not yet done, and, all external segments
3742  // fall under merges that the merge scheduler is
3743  // currently running. So, we now wait and check
3744  // back to see if the merge has completed.
3745  DoWait();
3746  }
3747 
3748  if (merge != null)
3749  {
3750  any = true;
3751  Merge(merge);
3752  }
3753  }
3754 
3755  if (any)
3756  // Sometimes, on copying an external segment over,
3757  // more merges may become necessary:
3758  mergeScheduler.Merge(this);
3759  }
3760 
3781  public virtual void AddIndexes(params IndexReader[] readers)
3782  {
3783 
3784  EnsureOpen();
3785 
3786  // Do not allow add docs or deletes while we are running:
3787  docWriter.PauseAllThreads();
3788 
3789  // We must pre-acquire a read lock here (and upgrade to
3790  // write lock in startTransaction below) so that no
3791  // other addIndexes is allowed to start up after we have
3792  // flushed & optimized but before we then start our
3793  // transaction. This is because the merging below
3794  // requires that only one segment is present in the
3795  // index:
3796  AcquireRead();
3797 
3798  try
3799  {
3800 
3801  SegmentInfo info = null;
3802  System.String mergedName = null;
3803  SegmentMerger merger = null;
3804 
3805  bool success = false;
3806 
3807  try
3808  {
3809  Flush(true, false, true);
3810  Optimize(); // start with zero or 1 seg
3811  success = true;
3812  }
3813  finally
3814  {
3815  // Take care to release the read lock if we hit an
3816  // exception before starting the transaction
3817  if (!success)
3818  ReleaseRead();
3819  }
3820 
3821  // true means we already have a read lock; if this
3822  // call hits an exception it will release the write
3823  // lock:
3824  StartTransaction(true);
3825 
3826  try
3827  {
3828  mergedName = NewSegmentName();
3829  merger = new SegmentMerger(this, mergedName, null);
3830 
3831  SegmentReader sReader = null;
3832  lock (this)
3833  {
3834  if (segmentInfos.Count == 1)
3835  {
3836  // add existing index, if any
3837  sReader = readerPool.Get(segmentInfos.Info(0), true, BufferedIndexInput.BUFFER_SIZE, - 1);
3838  }
3839  }
3840 
3841  success = false;
3842 
3843  try
3844  {
3845  if (sReader != null)
3846  merger.Add(sReader);
3847 
3848  for (int i = 0; i < readers.Length; i++)
3849  // add new indexes
3850  merger.Add(readers[i]);
3851 
3852  int docCount = merger.Merge(); // merge 'em
3853 
3854  lock (this)
3855  {
3856  segmentInfos.Clear(); // pop old infos & add new
3857  info = new SegmentInfo(mergedName, docCount, directory, false, true, - 1, null, false, merger.HasProx());
3858  SetDiagnostics(info, "addIndexes(params IndexReader[])");
3859  segmentInfos.Add(info);
3860  }
3861 
3862  // Notify DocumentsWriter that the flushed count just increased
3863  docWriter.UpdateFlushedDocCount(docCount);
3864 
3865  success = true;
3866  }
3867  finally
3868  {
3869  if (sReader != null)
3870  {
3871  readerPool.Release(sReader);
3872  }
3873  }
3874  }
3875  finally
3876  {
3877  if (!success)
3878  {
3879  if (infoStream != null)
3880  Message("hit exception in addIndexes during merge");
3881  RollbackTransaction();
3882  }
3883  else
3884  {
3885  CommitTransaction();
3886  }
3887  }
3888 
3889  if (mergePolicy is LogMergePolicy && UseCompoundFile)
3890  {
3891 
3892  IList<string> files = null;
3893 
3894  lock (this)
3895  {
3896  // Must incRef our files so that if another thread
3897  // is running merge/optimize, it doesn't delete our
3898  // segment's files before we have a change to
3899  // finish making the compound file.
3900  if (segmentInfos.Contains(info))
3901  {
3902  files = info.Files();
3903  deleter.IncRef(files);
3904  }
3905  }
3906 
3907  if (files != null)
3908  {
3909 
3910  success = false;
3911 
3912  StartTransaction(false);
3913 
3914  try
3915  {
3916  merger.CreateCompoundFile(mergedName + ".cfs");
3917  lock (this)
3918  {
3919  info.SetUseCompoundFile(true);
3920  }
3921 
3922  success = true;
3923  }
3924  finally
3925  {
3926  lock (this)
3927  {
3928  deleter.DecRef(files);
3929  }
3930 
3931  if (!success)
3932  {
3933  if (infoStream != null)
3934  Message("hit exception building compound file in addIndexes during merge");
3935 
3936  RollbackTransaction();
3937  }
3938  else
3939  {
3940  CommitTransaction();
3941  }
3942  }
3943  }
3944  }
3945  }
3946  catch (System.OutOfMemoryException oom)
3947  {
3948  HandleOOM(oom, "addIndexes(params IndexReader[])");
3949  }
3950  finally
3951  {
3952  if (docWriter != null)
3953  {
3954  docWriter.ResumeAllThreads();
3955  }
3956  }
3957  }
3958 
3964  protected virtual void DoAfterFlush()
3965  {
3966  }
3967 
3972  protected virtual void DoBeforeFlush()
3973  {
3974  }
3975 
3985  public void PrepareCommit()
3986  {
3987  EnsureOpen();
3988  PrepareCommit(null);
3989  }
3990 
4021  private void PrepareCommit(IDictionary<string, string> commitUserData)
4022  {
4023  if (hitOOM)
4024  {
4025  throw new System.SystemException("this writer hit an OutOfMemoryError; cannot commit");
4026  }
4027 
4028  if (pendingCommit != null)
4029  throw new System.SystemException("prepareCommit was already called with no corresponding call to commit");
4030 
4031  if (infoStream != null)
4032  Message("prepareCommit: flush");
4033 
4034  Flush(true, true, true);
4035 
4036  StartCommit(0, commitUserData);
4037  }
4038 
4039  // Used only by commit, below; lock order is commitLock -> IW
4040  private Object commitLock = new Object();
4041 
4042  private void Commit(long sizeInBytes)
4043  {
4044  lock(commitLock) {
4045  StartCommit(sizeInBytes, null);
4046  FinishCommit();
4047  }
4048  }
4049 
4081  public void Commit()
4082  {
4083  Commit(null);
4084  }
4085 
4095  public void Commit(IDictionary<string, string> commitUserData)
4096  {
4097  EnsureOpen();
4098 
4099  if (infoStream != null)
4100  {
4101  Message("commit: start");
4102  }
4103 
4104  lock (commitLock)
4105  {
4106  if (infoStream != null)
4107  {
4108  Message("commit: enter lock");
4109  }
4110  if (pendingCommit == null)
4111  {
4112  if (infoStream != null)
4113  {
4114  Message("commit: now prepare");
4115  }
4116  PrepareCommit(commitUserData);
4117  }
4118  else if (infoStream != null)
4119  {
4120  Message("commit: already prepared");
4121  }
4122 
4123  FinishCommit();
4124  }
4125  }
4126 
4127  private void FinishCommit()
4128  {
4129  lock (this)
4130  {
4131 
4132  if (pendingCommit != null)
4133  {
4134  try
4135  {
4136  if (infoStream != null)
4137  Message("commit: pendingCommit != null");
4138  pendingCommit.FinishCommit(directory);
4139  if (infoStream != null)
4140  Message("commit: wrote segments file \"" + pendingCommit.GetCurrentSegmentFileName() + "\"");
4141  lastCommitChangeCount = pendingCommitChangeCount;
4142  segmentInfos.UpdateGeneration(pendingCommit);
4143  segmentInfos.UserData = pendingCommit.UserData;
4144  SetRollbackSegmentInfos(pendingCommit);
4145  deleter.Checkpoint(pendingCommit, true);
4146  }
4147  finally
4148  {
4149  deleter.DecRef(pendingCommit);
4150  pendingCommit = null;
4151  System.Threading.Monitor.PulseAll(this);
4152  }
4153  }
4154  else if (infoStream != null)
4155  {
4156  Message("commit: pendingCommit == null; skip");
4157  }
4158 
4159  if (infoStream != null)
4160  {
4161  Message("commit: done");
4162  }
4163  }
4164  }
4165 
4178  public /*protected internal*/ void Flush(bool triggerMerge, bool flushDocStores, bool flushDeletes)
4179  {
4180  // We can be called during close, when closing==true, so we must pass false to ensureOpen:
4181  EnsureOpen(false);
4182  if (DoFlush(flushDocStores, flushDeletes) && triggerMerge)
4183  MaybeMerge();
4184  }
4185 
4186  // TODO: this method should not have to be entirely
4187  // synchronized, ie, merges should be allowed to commit
4188  // even while a flush is happening
4189  private bool DoFlush(bool flushDocStores, bool flushDeletes)
4190  {
4191  lock (this)
4192  {
4193  try
4194  {
4195  try
4196  {
4197  return DoFlushInternal(flushDocStores, flushDeletes);
4198  }
4199  finally
4200  {
4201  if (docWriter.DoBalanceRAM())
4202  {
4203  docWriter.BalanceRAM();
4204  }
4205  }
4206  }
4207  finally
4208  {
4209  docWriter.ClearFlushPending();
4210  }
4211  }
4212  }
4213 
4214  // TODO: this method should not have to be entirely
4215  // synchronized, ie, merges should be allowed to commit
4216  // even while a flush is happening
4217  private bool DoFlushInternal(bool flushDocStores, bool flushDeletes)
4218  {
4219  lock (this)
4220  {
4221  if (hitOOM)
4222  {
4223  throw new System.SystemException("this writer hit an OutOfMemoryError; cannot flush");
4224  }
4225 
4226  EnsureOpen(false);
4227 
4228  System.Diagnostics.Debug.Assert(TestPoint("startDoFlush"));
4229 
4230  DoBeforeFlush();
4231 
4232  flushCount++;
4233 
4234  // If we are flushing because too many deletes
4235  // accumulated, then we should apply the deletes to free
4236  // RAM:
4237  flushDeletes |= docWriter.DoApplyDeletes();
4238 
4239  // Make sure no threads are actively adding a document.
4240  // Returns true if docWriter is currently aborting, in
4241  // which case we skip flushing this segment
4242  if (infoStream != null)
4243  {
4244  Message("flush: now pause all indexing threads");
4245  }
4246  if (docWriter.PauseAllThreads())
4247  {
4248  docWriter.ResumeAllThreads();
4249  return false;
4250  }
4251 
4252  try
4253  {
4254 
4255  SegmentInfo newSegment = null;
4256 
4257  int numDocs = docWriter.NumDocsInRAM;
4258 
4259  // Always flush docs if there are any
4260  bool flushDocs = numDocs > 0;
4261 
4262  System.String docStoreSegment = docWriter.DocStoreSegment;
4263 
4264  System.Diagnostics.Debug.Assert(docStoreSegment != null || numDocs == 0, "dss=" + docStoreSegment + " numDocs=" + numDocs);
4265 
4266  if (docStoreSegment == null)
4267  flushDocStores = false;
4268 
4269  int docStoreOffset = docWriter.DocStoreOffset;
4270 
4271  bool docStoreIsCompoundFile = false;
4272 
4273  if (infoStream != null)
4274  {
4275  Message(" flush: segment=" + docWriter.Segment + " docStoreSegment=" + docWriter.DocStoreSegment + " docStoreOffset=" + docStoreOffset + " flushDocs=" + flushDocs + " flushDeletes=" + flushDeletes + " flushDocStores=" + flushDocStores + " numDocs=" + numDocs + " numBufDelTerms=" + docWriter.GetNumBufferedDeleteTerms());
4276  Message(" index before flush " + SegString());
4277  }
4278 
4279  // Check if the doc stores must be separately flushed
4280  // because other segments, besides the one we are about
4281  // to flush, reference it
4282  if (flushDocStores && (!flushDocs || !docWriter.Segment.Equals(docWriter.DocStoreSegment)))
4283  {
4284  // We must separately flush the doc store
4285  if (infoStream != null)
4286  Message(" flush shared docStore segment " + docStoreSegment);
4287 
4288  docStoreIsCompoundFile = FlushDocStores();
4289  flushDocStores = false;
4290  }
4291 
4292  System.String segment = docWriter.Segment;
4293 
4294  // If we are flushing docs, segment must not be null:
4295  System.Diagnostics.Debug.Assert(segment != null || !flushDocs);
4296 
4297  if (flushDocs)
4298  {
4299 
4300  bool success = false;
4301  int flushedDocCount;
4302 
4303  try
4304  {
4305  flushedDocCount = docWriter.Flush(flushDocStores);
4306  if (infoStream != null)
4307  {
4308  Message("flushedFiles=" + docWriter.GetFlushedFiles());
4309  }
4310  success = true;
4311  }
4312  finally
4313  {
4314  if (!success)
4315  {
4316  if (infoStream != null)
4317  Message("hit exception flushing segment " + segment);
4318  deleter.Refresh(segment);
4319  }
4320  }
4321 
4322  if (0 == docStoreOffset && flushDocStores)
4323  {
4324  // This means we are flushing private doc stores
4325  // with this segment, so it will not be shared
4326  // with other segments
4327  System.Diagnostics.Debug.Assert(docStoreSegment != null);
4328  System.Diagnostics.Debug.Assert(docStoreSegment.Equals(segment));
4329  docStoreOffset = - 1;
4330  docStoreIsCompoundFile = false;
4331  docStoreSegment = null;
4332  }
4333 
4334  // Create new SegmentInfo, but do not add to our
4335  // segmentInfos until deletes are flushed
4336  // successfully.
4337  newSegment = new SegmentInfo(segment, flushedDocCount, directory, false, true, docStoreOffset, docStoreSegment, docStoreIsCompoundFile, docWriter.HasProx());
4338  SetDiagnostics(newSegment, "flush");
4339  }
4340 
4341  docWriter.PushDeletes();
4342 
4343  if (flushDocs)
4344  {
4345  segmentInfos.Add(newSegment);
4346  Checkpoint();
4347  }
4348 
4349  if (flushDocs && mergePolicy.UseCompoundFile(segmentInfos, newSegment))
4350  {
4351  // Now build compound file
4352  bool success = false;
4353  try
4354  {
4355  docWriter.CreateCompoundFile(segment);
4356  success = true;
4357  }
4358  finally
4359  {
4360  if (!success)
4361  {
4362  if (infoStream != null)
4363  Message("hit exception creating compound file for newly flushed segment " + segment);
4364  deleter.DeleteFile(segment + "." + IndexFileNames.COMPOUND_FILE_EXTENSION);
4365  }
4366  }
4367 
4368  newSegment.SetUseCompoundFile(true);
4369  Checkpoint();
4370  }
4371 
4372  if (flushDeletes)
4373  {
4374  ApplyDeletes();
4375  }
4376 
4377  if (flushDocs)
4378  Checkpoint();
4379 
4380  DoAfterFlush();
4381 
4382  return flushDocs;
4383  }
4384  catch (System.OutOfMemoryException oom)
4385  {
4386  HandleOOM(oom, "doFlush");
4387  // never hit
4388  return false;
4389  }
4390  finally
4391  {
4392  docWriter.ResumeAllThreads();
4393  }
4394  }
4395  }
4396 
4400  public long RamSizeInBytes()
4401  {
4402  EnsureOpen();
4403  return docWriter.GetRAMUsed();
4404  }
4405 
4409  public int NumRamDocs()
4410  {
4411  lock (this)
4412  {
4413  EnsureOpen();
4414  return docWriter.NumDocsInRAM;
4415  }
4416  }
4417 
4418  private int EnsureContiguousMerge(MergePolicy.OneMerge merge)
4419  {
4420 
4421  int first = segmentInfos.IndexOf(merge.segments.Info(0));
4422  if (first == - 1)
4423  throw new MergePolicy.MergeException("could not find segment " + merge.segments.Info(0).name + " in current index " + SegString(), directory);
4424 
4425  int numSegments = segmentInfos.Count;
4426 
4427  int numSegmentsToMerge = merge.segments.Count;
4428  for (int i = 0; i < numSegmentsToMerge; i++)
4429  {
4430  SegmentInfo info = merge.segments.Info(i);
4431 
4432  if (first + i >= numSegments || !segmentInfos.Info(first + i).Equals(info))
4433  {
4434  if (segmentInfos.IndexOf(info) == - 1)
4435  throw new MergePolicy.MergeException("MergePolicy selected a segment (" + info.name + ") that is not in the current index " + SegString(), directory);
4436  else
4437  throw new MergePolicy.MergeException("MergePolicy selected non-contiguous segments to merge (" + merge.SegString(directory) + " vs " + SegString() + "), which IndexWriter (currently) cannot handle", directory);
4438  }
4439  }
4440 
4441  return first;
4442  }
4443 
4454  private void CommitMergedDeletes(MergePolicy.OneMerge merge, SegmentReader mergeReader)
4455  {
4456  lock (this)
4457  {
4458 
4459  System.Diagnostics.Debug.Assert(TestPoint("startCommitMergeDeletes"));
4460 
4461  SegmentInfos sourceSegments = merge.segments;
4462 
4463  if (infoStream != null)
4464  Message("commitMergeDeletes " + merge.SegString(directory));
4465 
4466  // Carefully merge deletes that occurred after we
4467  // started merging:
4468  int docUpto = 0;
4469  int delCount = 0;
4470 
4471  for (int i = 0; i < sourceSegments.Count; i++)
4472  {
4473  SegmentInfo info = sourceSegments.Info(i);
4474  int docCount = info.docCount;
4475  SegmentReader previousReader = merge.readersClone[i];
4476  SegmentReader currentReader = merge.readers[i];
4477  if (previousReader.HasDeletions)
4478  {
4479 
4480  // There were deletes on this segment when the merge
4481  // started. The merge has collapsed away those
4482  // deletes, but, if new deletes were flushed since
4483  // the merge started, we must now carefully keep any
4484  // newly flushed deletes but mapping them to the new
4485  // docIDs.
4486 
4487  if (currentReader.NumDeletedDocs > previousReader.NumDeletedDocs)
4488  {
4489  // This means this segment has had new deletes
4490  // committed since we started the merge, so we
4491  // must merge them:
4492  for (int j = 0; j < docCount; j++)
4493  {
4494  if (previousReader.IsDeleted(j))
4495  {
4496  System.Diagnostics.Debug.Assert(currentReader.IsDeleted(j));
4497  }
4498  else
4499  {
4500  if (currentReader.IsDeleted(j))
4501  {
4502  mergeReader.DoDelete(docUpto);
4503  delCount++;
4504  }
4505  docUpto++;
4506  }
4507  }
4508  }
4509  else
4510  {
4511  docUpto += docCount - previousReader.NumDeletedDocs;
4512  }
4513  }
4514  else if (currentReader.HasDeletions)
4515  {
4516  // This segment had no deletes before but now it
4517  // does:
4518  for (int j = 0; j < docCount; j++)
4519  {
4520  if (currentReader.IsDeleted(j))
4521  {
4522  mergeReader.DoDelete(docUpto);
4523  delCount++;
4524  }
4525  docUpto++;
4526  }
4527  }
4528  // No deletes before or after
4529  else
4530  docUpto += info.docCount;
4531  }
4532 
4533  System.Diagnostics.Debug.Assert(mergeReader.NumDeletedDocs == delCount);
4534 
4535  mergeReader.hasChanges = delCount > 0;
4536  }
4537  }
4538 
4539  /* FIXME if we want to support non-contiguous segment merges */
4540  private bool CommitMerge(MergePolicy.OneMerge merge, SegmentMerger merger, int mergedDocCount, SegmentReader mergedReader)
4541  {
4542  lock (this)
4543  {
4544 
4545  System.Diagnostics.Debug.Assert(TestPoint("startCommitMerge"));
4546 
4547  if (hitOOM)
4548  {
4549  throw new System.SystemException("this writer hit an OutOfMemoryError; cannot complete merge");
4550  }
4551 
4552  if (infoStream != null)
4553  Message("commitMerge: " + merge.SegString(directory) + " index=" + SegString());
4554 
4555  System.Diagnostics.Debug.Assert(merge.registerDone);
4556 
4557  // If merge was explicitly aborted, or, if rollback() or
4558  // rollbackTransaction() had been called since our merge
4559  // started (which results in an unqualified
4560  // deleter.refresh() call that will remove any index
4561  // file that current segments does not reference), we
4562  // abort this merge
4563  if (merge.IsAborted())
4564  {
4565  if (infoStream != null)
4566  Message("commitMerge: skipping merge " + merge.SegString(directory) + ": it was aborted");
4567 
4568  return false;
4569  }
4570 
4571  int start = EnsureContiguousMerge(merge);
4572 
4573  CommitMergedDeletes(merge, mergedReader);
4574  docWriter.RemapDeletes(segmentInfos, merger.GetDocMaps(), merger.GetDelCounts(), merge, mergedDocCount);
4575 
4576  // If the doc store we are using has been closed and
4577  // is in now compound format (but wasn't when we
4578  // started), then we will switch to the compound
4579  // format as well:
4580  SetMergeDocStoreIsCompoundFile(merge);
4581 
4582  merge.info.HasProx = merger.HasProx();
4583 
4584  segmentInfos.RemoveRange(start, start + merge.segments.Count - start);
4585  System.Diagnostics.Debug.Assert(!segmentInfos.Contains(merge.info));
4586  segmentInfos.Insert(start, merge.info);
4587 
4588  CloseMergeReaders(merge, false);
4589 
4590  // Must note the change to segmentInfos so any commits
4591  // in-flight don't lose it:
4592  Checkpoint();
4593 
4594  // If the merged segments had pending changes, clear
4595  // them so that they don't bother writing them to
4596  // disk, updating SegmentInfo, etc.:
4597  readerPool.Clear(merge.segments);
4598 
4599  if (merge.optimize)
4600  {
4601  // cascade the optimize:
4602  segmentsToOptimize.Add(merge.info);
4603  }
4604  return true;
4605  }
4606  }
4607 
4608  private void HandleMergeException(System.Exception t, MergePolicy.OneMerge merge)
4609  {
4610 
4611  if (infoStream != null)
4612  {
4613  Message("handleMergeException: merge=" + merge.SegString(directory) + " exc=" + t);
4614  }
4615 
4616  // Set the exception on the merge, so if
4617  // optimize() is waiting on us it sees the root
4618  // cause exception:
4619  merge.SetException(t);
4620  AddMergeException(merge);
4621 
4622  if (t is MergePolicy.MergeAbortedException)
4623  {
4624  // We can ignore this exception (it happens when
4625  // close(false) or rollback is called), unless the
4626  // merge involves segments from external directories,
4627  // in which case we must throw it so, for example, the
4628  // rollbackTransaction code in addIndexes* is
4629  // executed.
4630  if (merge.isExternal)
4631  throw t;
4632  }
4633  else if (t is System.IO.IOException || t is System.SystemException || t is System.ApplicationException)
4634  {
4635  throw t;
4636  }
4637  else
4638  {
4639  // Should not get here
4640  System.Diagnostics.Debug.Fail("Exception is not expected type!");
4641  throw new System.SystemException(null, t);
4642  }
4643  }
4644 
4645  public void Merge_ForNUnit(MergePolicy.OneMerge merge)
4646  {
4647  Merge(merge);
4648  }
4652  internal void Merge(MergePolicy.OneMerge merge)
4653  {
4654 
4655  bool success = false;
4656 
4657  try
4658  {
4659  try
4660  {
4661  try
4662  {
4663  MergeInit(merge);
4664 
4665  if (infoStream != null)
4666  {
4667  Message("now merge\n merge=" + merge.SegString(directory) + "\n merge=" + merge + "\n index=" + SegString());
4668  }
4669 
4670  MergeMiddle(merge);
4671  MergeSuccess(merge);
4672  success = true;
4673  }
4674  catch (System.Exception t)
4675  {
4676  HandleMergeException(t, merge);
4677  }
4678  }
4679  finally
4680  {
4681  lock (this)
4682  {
4683  MergeFinish(merge);
4684 
4685  if (!success)
4686  {
4687  if (infoStream != null)
4688  Message("hit exception during merge");
4689  if (merge.info != null && !segmentInfos.Contains(merge.info))
4690  deleter.Refresh(merge.info.name);
4691  }
4692 
4693  // This merge (and, generally, any change to the
4694  // segments) may now enable new merges, so we call
4695  // merge policy & update pending merges.
4696  if (success && !merge.IsAborted() && !closed && !closing)
4697  UpdatePendingMerges(merge.maxNumSegmentsOptimize, merge.optimize);
4698  }
4699  }
4700  }
4701  catch (System.OutOfMemoryException oom)
4702  {
4703  HandleOOM(oom, "merge");
4704  }
4705  }
4706 
4708  internal virtual void MergeSuccess(MergePolicy.OneMerge merge)
4709  {
4710  }
4711 
4719  internal bool RegisterMerge(MergePolicy.OneMerge merge)
4720  {
4721  lock (this)
4722  {
4723 
4724  if (merge.registerDone)
4725  return true;
4726 
4727  if (stopMerges)
4728  {
4729  merge.Abort();
4730  throw new MergePolicy.MergeAbortedException("merge is aborted: " + merge.SegString(directory));
4731  }
4732 
4733  int count = merge.segments.Count;
4734  bool isExternal = false;
4735  for (int i = 0; i < count; i++)
4736  {
4737  SegmentInfo info = merge.segments.Info(i);
4738  if (mergingSegments.Contains(info))
4739  {
4740  return false;
4741  }
4742  if (segmentInfos.IndexOf(info) == -1)
4743  {
4744  return false;
4745  }
4746  if (info.dir != directory)
4747  {
4748  isExternal = true;
4749  }
4750  if (segmentsToOptimize.Contains(info))
4751  {
4752  merge.optimize = true;
4753  merge.maxNumSegmentsOptimize = optimizeMaxNumSegments;
4754  }
4755  }
4756 
4757  EnsureContiguousMerge(merge);
4758 
4759  pendingMerges.AddLast(merge);
4760 
4761  if (infoStream != null)
4762  Message("add merge to pendingMerges: " + merge.SegString(directory) + " [total " + pendingMerges.Count + " pending]");
4763 
4764  merge.mergeGen = mergeGen;
4765  merge.isExternal = isExternal;
4766 
4767  // OK it does not conflict; now record that this merge
4768  // is running (while synchronized) to avoid race
4769  // condition where two conflicting merges from different
4770  // threads, start
4771  for (int i = 0; i < count; i++)
4772  {
4773  SegmentInfo si = merge.segments.Info(i);
4774  mergingSegments.Add(si);
4775  }
4776 
4777  // Merge is now registered
4778  merge.registerDone = true;
4779  return true;
4780  }
4781  }
4782 
4786  internal void MergeInit(MergePolicy.OneMerge merge)
4787  {
4788  lock (this)
4789  {
4790  bool success = false;
4791  try
4792  {
4793  _MergeInit(merge);
4794  success = true;
4795  }
4796  finally
4797  {
4798  if (!success)
4799  {
4800  MergeFinish(merge);
4801  }
4802  }
4803  }
4804  }
4805 
4806  private void _MergeInit(MergePolicy.OneMerge merge)
4807  {
4808  lock (this)
4809  {
4810 
4811  System.Diagnostics.Debug.Assert(TestPoint("startMergeInit"));
4812 
4813  System.Diagnostics.Debug.Assert(merge.registerDone);
4814  System.Diagnostics.Debug.Assert(!merge.optimize || merge.maxNumSegmentsOptimize > 0);
4815 
4816  if (hitOOM)
4817  {
4818  throw new System.SystemException("this writer hit an OutOfMemoryError; cannot merge");
4819  }
4820 
4821  if (merge.info != null)
4822  // mergeInit already done
4823  return ;
4824 
4825  if (merge.IsAborted())
4826  return ;
4827 
4828  ApplyDeletes();
4829 
4830  SegmentInfos sourceSegments = merge.segments;
4831  int end = sourceSegments.Count;
4832 
4833  // Check whether this merge will allow us to skip
4834  // merging the doc stores (stored field & vectors).
4835  // This is a very substantial optimization (saves tons
4836  // of IO).
4837 
4838  Directory lastDir = directory;
4839  System.String lastDocStoreSegment = null;
4840  int next = - 1;
4841 
4842  bool mergeDocStores = false;
4843  bool doFlushDocStore = false;
4844  System.String currentDocStoreSegment = docWriter.DocStoreSegment;
4845 
4846  // Test each segment to be merged: check if we need to
4847  // flush/merge doc stores
4848  for (int i = 0; i < end; i++)
4849  {
4850  SegmentInfo si = sourceSegments.Info(i);
4851 
4852  // If it has deletions we must merge the doc stores
4853  if (si.HasDeletions())
4854  mergeDocStores = true;
4855 
4856  // If it has its own (private) doc stores we must
4857  // merge the doc stores
4858  if (- 1 == si.DocStoreOffset)
4859  mergeDocStores = true;
4860 
4861  // If it has a different doc store segment than
4862  // previous segments, we must merge the doc stores
4863  System.String docStoreSegment = si.DocStoreSegment;
4864  if (docStoreSegment == null)
4865  mergeDocStores = true;
4866  else if (lastDocStoreSegment == null)
4867  lastDocStoreSegment = docStoreSegment;
4868  else if (!lastDocStoreSegment.Equals(docStoreSegment))
4869  mergeDocStores = true;
4870 
4871  // Segments' docScoreOffsets must be in-order,
4872  // contiguous. For the default merge policy now
4873  // this will always be the case but for an arbitrary
4874  // merge policy this may not be the case
4875  if (- 1 == next)
4876  next = si.DocStoreOffset + si.docCount;
4877  else if (next != si.DocStoreOffset)
4878  mergeDocStores = true;
4879  else
4880  next = si.DocStoreOffset + si.docCount;
4881 
4882  // If the segment comes from a different directory
4883  // we must merge
4884  if (lastDir != si.dir)
4885  mergeDocStores = true;
4886 
4887  // If the segment is referencing the current "live"
4888  // doc store outputs then we must merge
4889  if (si.DocStoreOffset != - 1 && currentDocStoreSegment != null && si.DocStoreSegment.Equals(currentDocStoreSegment))
4890  {
4891  doFlushDocStore = true;
4892  }
4893  }
4894 
4895  // if a mergedSegmentWarmer is installed, we must merge
4896  // the doc stores because we will open a full
4897  // SegmentReader on the merged segment:
4898  if (!mergeDocStores && mergedSegmentWarmer != null && currentDocStoreSegment != null && lastDocStoreSegment != null && lastDocStoreSegment.Equals(currentDocStoreSegment))
4899  {
4900  mergeDocStores = true;
4901  }
4902 
4903  int docStoreOffset;
4904  System.String docStoreSegment2;
4905  bool docStoreIsCompoundFile;
4906 
4907  if (mergeDocStores)
4908  {
4909  docStoreOffset = - 1;
4910  docStoreSegment2 = null;
4911  docStoreIsCompoundFile = false;
4912  }
4913  else
4914  {
4915  SegmentInfo si = sourceSegments.Info(0);
4916  docStoreOffset = si.DocStoreOffset;
4917  docStoreSegment2 = si.DocStoreSegment;
4918  docStoreIsCompoundFile = si.DocStoreIsCompoundFile;
4919  }
4920 
4921  if (mergeDocStores && doFlushDocStore)
4922  {
4923  // SegmentMerger intends to merge the doc stores
4924  // (stored fields, vectors), and at least one of the
4925  // segments to be merged refers to the currently
4926  // live doc stores.
4927 
4928  // TODO: if we know we are about to merge away these
4929  // newly flushed doc store files then we should not
4930  // make compound file out of them...
4931  if (infoStream != null)
4932  Message("now flush at merge");
4933  DoFlush(true, false);
4934  }
4935 
4936  merge.mergeDocStores = mergeDocStores;
4937 
4938  // Bind a new segment name here so even with
4939  // ConcurrentMergePolicy we keep deterministic segment
4940  // names.
4941  merge.info = new SegmentInfo(NewSegmentName(), 0, directory, false, true, docStoreOffset, docStoreSegment2, docStoreIsCompoundFile, false);
4942 
4943 
4944  IDictionary<string, string> details = new Dictionary<string, string>();
4945  details["optimize"] = merge.optimize + "";
4946  details["mergeFactor"] = end + "";
4947  details["mergeDocStores"] = mergeDocStores + "";
4948  SetDiagnostics(merge.info, "merge", details);
4949 
4950  // Also enroll the merged segment into mergingSegments;
4951  // this prevents it from getting selected for a merge
4952  // after our merge is done but while we are building the
4953  // CFS:
4954  mergingSegments.Add(merge.info);
4955  }
4956  }
4957 
4958  private void SetDiagnostics(SegmentInfo info, System.String source)
4959  {
4960  SetDiagnostics(info, source, null);
4961  }
4962 
4963  private void SetDiagnostics(SegmentInfo info, System.String source, IDictionary<string, string> details)
4964  {
4965  IDictionary<string, string> diagnostics = new Dictionary<string,string>();
4966  diagnostics["source"] = source;
4967  diagnostics["lucene.version"] = Constants.LUCENE_VERSION;
4968  diagnostics["os"] = Constants.OS_NAME + "";
4969  diagnostics["os.arch"] = Constants.OS_ARCH + "";
4970  diagnostics["os.version"] = Constants.OS_VERSION + "";
4971  diagnostics["java.version"] = Constants.JAVA_VERSION + "";
4972  diagnostics["java.vendor"] = Constants.JAVA_VENDOR + "";
4973  if (details != null)
4974  {
4975  //System.Collections.ArrayList keys = new System.Collections.ArrayList(details.Keys);
4976  //System.Collections.ArrayList values = new System.Collections.ArrayList(details.Values);
4977  foreach (string key in details.Keys)
4978  {
4979  diagnostics[key] = details[key];
4980  }
4981  }
4982  info.Diagnostics = diagnostics;
4983  }
4984 
4988  internal void MergeFinish(MergePolicy.OneMerge merge)
4989  {
4990  lock (this)
4991  {
4992 
4993  // Optimize, addIndexes or finishMerges may be waiting
4994  // on merges to finish.
4995  System.Threading.Monitor.PulseAll(this);
4996 
4997  // It's possible we are called twice, eg if there was an
4998  // exception inside mergeInit
4999  if (merge.registerDone)
5000  {
5001  SegmentInfos sourceSegments = merge.segments;
5002  int end = sourceSegments.Count;
5003  for (int i = 0; i < end; i++)
5004  mergingSegments.Remove(sourceSegments.Info(i));
5005  if(merge.info != null)
5006  mergingSegments.Remove(merge.info);
5007  merge.registerDone = false;
5008  }
5009 
5010  runningMerges.Remove(merge);
5011  }
5012  }
5013 
5014  private void SetMergeDocStoreIsCompoundFile(MergePolicy.OneMerge merge)
5015  {
5016  lock (this)
5017  {
5018  string mergeDocStoreSegment = merge.info.DocStoreSegment;
5019  if (mergeDocStoreSegment != null && !merge.info.DocStoreIsCompoundFile)
5020  {
5021  int size = segmentInfos.Count;
5022  for (int i = 0; i < size; i++)
5023  {
5024  SegmentInfo info = segmentInfos.Info(i);
5025  string docStoreSegment = info.DocStoreSegment;
5026  if (docStoreSegment != null &&
5027  docStoreSegment.Equals(mergeDocStoreSegment) &&
5028  info.DocStoreIsCompoundFile)
5029  {
5030  merge.info.DocStoreIsCompoundFile = true;
5031  break;
5032  }
5033  }
5034  }
5035  }
5036  }
5037 
5038  private void CloseMergeReaders(MergePolicy.OneMerge merge, bool suppressExceptions)
5039  {
5040  lock (this)
5041  {
5042  int numSegments = merge.segments.Count;
5043  if (suppressExceptions)
5044  {
5045  // Suppress any new exceptions so we throw the
5046  // original cause
5047  for (int i = 0; i < numSegments; i++)
5048  {
5049  if (merge.readers[i] != null)
5050  {
5051  try
5052  {
5053  readerPool.Release(merge.readers[i], false);
5054  }
5055  catch (Exception)
5056  {
5057  }
5058  merge.readers[i] = null;
5059  }
5060 
5061  if (merge.readersClone[i] != null)
5062  {
5063  try
5064  {
5065  merge.readersClone[i].Close();
5066  }
5067  catch (Exception)
5068  {
5069  }
5070  // This was a private clone and we had the
5071  // only reference
5072  System.Diagnostics.Debug.Assert(merge.readersClone[i].RefCount == 0); //: "refCount should be 0 but is " + merge.readersClone[i].getRefCount();
5073  merge.readersClone[i] = null;
5074  }
5075  }
5076  }
5077  else
5078  {
5079  for (int i = 0; i < numSegments; i++)
5080  {
5081  if (merge.readers[i] != null)
5082  {
5083  readerPool.Release(merge.readers[i], true);
5084  merge.readers[i] = null;
5085  }
5086 
5087  if (merge.readersClone[i] != null)
5088  {
5089  merge.readersClone[i].Close();
5090  // This was a private clone and we had the only reference
5091  System.Diagnostics.Debug.Assert(merge.readersClone[i].RefCount == 0);
5092  merge.readersClone[i] = null;
5093  }
5094  }
5095  }
5096  }
5097  }
5098 
5099 
5104  private int MergeMiddle(MergePolicy.OneMerge merge)
5105  {
5106 
5107  merge.CheckAborted(directory);
5108 
5109  System.String mergedName = merge.info.name;
5110 
5111  SegmentMerger merger = null;
5112 
5113  int mergedDocCount = 0;
5114 
5115  SegmentInfos sourceSegments = merge.segments;
5116  int numSegments = sourceSegments.Count;
5117 
5118  if (infoStream != null)
5119  Message("merging " + merge.SegString(directory));
5120 
5121  merger = new SegmentMerger(this, mergedName, merge);
5122 
5123  merge.readers = new SegmentReader[numSegments];
5124  merge.readersClone = new SegmentReader[numSegments];
5125 
5126  bool mergeDocStores = false;
5127 
5128  String currentDocStoreSegment;
5129  lock(this) {
5130  currentDocStoreSegment = docWriter.DocStoreSegment;
5131  }
5132  bool currentDSSMerged = false;
5133 
5134  // This is try/finally to make sure merger's readers are
5135  // closed:
5136  bool success = false;
5137  try
5138  {
5139  int totDocCount = 0;
5140 
5141  for (int i = 0; i < numSegments; i++)
5142  {
5143 
5144  SegmentInfo info = sourceSegments.Info(i);
5145 
5146  // Hold onto the "live" reader; we will use this to
5147  // commit merged deletes
5148  SegmentReader reader = merge.readers[i] = readerPool.Get(info, merge.mergeDocStores, MERGE_READ_BUFFER_SIZE, -1);
5149 
5150  // We clone the segment readers because other
5151  // deletes may come in while we're merging so we
5152  // need readers that will not change
5153  SegmentReader clone = merge.readersClone[i] = (SegmentReader)reader.Clone(true);
5154  merger.Add(clone);
5155 
5156  if (clone.HasDeletions)
5157  {
5158  mergeDocStores = true;
5159  }
5160 
5161  if (info.DocStoreOffset != -1 && currentDocStoreSegment != null)
5162  {
5163  currentDSSMerged |= currentDocStoreSegment.Equals(info.DocStoreSegment);
5164  }
5165 
5166  totDocCount += clone.NumDocs();
5167  }
5168 
5169  if (infoStream != null)
5170  {
5171  Message("merge: total " + totDocCount + " docs");
5172  }
5173 
5174  merge.CheckAborted(directory);
5175 
5176  // If deletions have arrived and it has now become
5177  // necessary to merge doc stores, go and open them:
5178  if (mergeDocStores && !merge.mergeDocStores)
5179  {
5180  merge.mergeDocStores = true;
5181  lock (this)
5182  {
5183  if (currentDSSMerged)
5184  {
5185  if (infoStream != null)
5186  {
5187  Message("now flush at mergeMiddle");
5188  }
5189  DoFlush(true, false);
5190  }
5191  }
5192 
5193  for (int i = 0; i < numSegments; i++)
5194  {
5195  merge.readersClone[i].OpenDocStores();
5196  }
5197 
5198  // Clear DSS
5199  merge.info.SetDocStore(-1, null, false);
5200 
5201  }
5202 
5203  // This is where all the work happens:
5204  mergedDocCount = merge.info.docCount = merger.Merge(merge.mergeDocStores);
5205 
5206  System.Diagnostics.Debug.Assert(mergedDocCount == totDocCount);
5207 
5208  if (merge.useCompoundFile)
5209  {
5210 
5211  success = false;
5212  string compoundFileName = IndexFileNames.SegmentFileName(mergedName, IndexFileNames.COMPOUND_FILE_EXTENSION);
5213 
5214  try
5215  {
5216  if (infoStream != null)
5217  {
5218  Message("create compound file " + compoundFileName);
5219  }
5220  merger.CreateCompoundFile(compoundFileName);
5221  success = true;
5222  }
5223  catch (System.IO.IOException ioe)
5224  {
5225  lock (this)
5226  {
5227  if (merge.IsAborted())
5228  {
5229  // This can happen if rollback or close(false)
5230  // is called -- fall through to logic below to
5231  // remove the partially created CFS:
5232  }
5233  else
5234  {
5235  HandleMergeException(ioe, merge);
5236  }
5237  }
5238  }
5239  catch (Exception t)
5240  {
5241  HandleMergeException(t, merge);
5242  }
5243  finally
5244  {
5245  if (!success)
5246  {
5247  if (infoStream != null)
5248  {
5249  Message("hit exception creating compound file during merge");
5250  }
5251 
5252  lock (this)
5253  {
5254  deleter.DeleteFile(compoundFileName);
5255  deleter.DeleteNewFiles(merger.GetMergedFiles());
5256  }
5257  }
5258  }
5259 
5260  success = false;
5261 
5262  lock (this)
5263  {
5264 
5265  // delete new non cfs files directly: they were never
5266  // registered with IFD
5267  deleter.DeleteNewFiles(merger.GetMergedFiles());
5268 
5269  if (merge.IsAborted())
5270  {
5271  if (infoStream != null)
5272  {
5273  Message("abort merge after building CFS");
5274  }
5275  deleter.DeleteFile(compoundFileName);
5276  return 0;
5277  }
5278  }
5279 
5280  merge.info.SetUseCompoundFile(true);
5281  }
5282 
5283  int termsIndexDivisor;
5284  bool loadDocStores;
5285 
5286  // if the merged segment warmer was not installed when
5287  // this merge was started, causing us to not force
5288  // the docStores to close, we can't warm it now
5289  bool canWarm = merge.info.DocStoreSegment == null || currentDocStoreSegment == null || !merge.info.DocStoreSegment.Equals(currentDocStoreSegment);
5290 
5291  if (poolReaders && mergedSegmentWarmer != null && canWarm)
5292  {
5293  // Load terms index & doc stores so the segment
5294  // warmer can run searches, load documents/term
5295  // vectors
5296  termsIndexDivisor = readerTermsIndexDivisor;
5297  loadDocStores = true;
5298  }
5299  else
5300  {
5301  termsIndexDivisor = -1;
5302  loadDocStores = false;
5303  }
5304 
5305  // TODO: in the non-realtime case, we may want to only
5306  // keep deletes (it's costly to open entire reader
5307  // when we just need deletes)
5308 
5309  SegmentReader mergedReader = readerPool.Get(merge.info, loadDocStores, BufferedIndexInput.BUFFER_SIZE, termsIndexDivisor);
5310  try
5311  {
5312  if (poolReaders && mergedSegmentWarmer != null)
5313  {
5314  mergedSegmentWarmer.Warm(mergedReader);
5315  }
5316  if (!CommitMerge(merge, merger, mergedDocCount, mergedReader))
5317  {
5318  // commitMerge will return false if this merge was aborted
5319  return 0;
5320  }
5321  }
5322  finally
5323  {
5324  lock (this)
5325  {
5326  readerPool.Release(mergedReader);
5327  }
5328  }
5329 
5330  success = true;
5331  }
5332  finally
5333  {
5334  // Readers are already closed in commitMerge if we didn't hit
5335  // an exc:
5336  if (!success)
5337  {
5338  CloseMergeReaders(merge, true);
5339  }
5340  }
5341 
5342  return mergedDocCount;
5343  }
5344 
5345  internal virtual void AddMergeException(MergePolicy.OneMerge merge)
5346  {
5347  lock (this)
5348  {
5349  System.Diagnostics.Debug.Assert(merge.GetException() != null);
5350  if (!mergeExceptions.Contains(merge) && mergeGen == merge.mergeGen)
5351  mergeExceptions.Add(merge);
5352  }
5353  }
5354 
5355  // Apply buffered deletes to all segments.
5356  private bool ApplyDeletes()
5357  {
5358  lock (this)
5359  {
5360  System.Diagnostics.Debug.Assert(TestPoint("startApplyDeletes"));
5361  flushDeletesCount++;
5362 
5363  bool success = false;
5364  bool changed;
5365  try
5366  {
5367  changed = docWriter.ApplyDeletes(segmentInfos);
5368  success = true;
5369  }
5370  finally
5371  {
5372  if (!success && infoStream != null)
5373  {
5374  Message("hit exception flushing deletes");
5375  }
5376  }
5377 
5378  if (changed)
5379  Checkpoint();
5380  return changed;
5381  }
5382  }
5383 
5384  // For test purposes.
5385  internal int GetBufferedDeleteTermsSize()
5386  {
5387  lock (this)
5388  {
5389  return docWriter.GetBufferedDeleteTerms().Count;
5390  }
5391  }
5392 
5393  // For test purposes.
5394  internal int GetNumBufferedDeleteTerms()
5395  {
5396  lock (this)
5397  {
5398  return docWriter.GetNumBufferedDeleteTerms();
5399  }
5400  }
5401 
5402  // utility routines for tests
5403  public /*internal*/ virtual SegmentInfo NewestSegment()
5404  {
5405  return segmentInfos.Count > 0 ? segmentInfos.Info(segmentInfos.Count - 1) : null;
5406  }
5407 
5408  public virtual System.String SegString()
5409  {
5410  lock (this)
5411  {
5412  return SegString(segmentInfos);
5413  }
5414  }
5415 
5416  private System.String SegString(SegmentInfos infos)
5417  {
5418  lock (this)
5419  {
5420  System.Text.StringBuilder buffer = new System.Text.StringBuilder();
5421  int count = infos.Count;
5422  for (int i = 0; i < count; i++)
5423  {
5424  if (i > 0)
5425  {
5426  buffer.Append(' ');
5427  }
5428  SegmentInfo info = infos.Info(i);
5429  buffer.Append(info.SegString(directory));
5430  if (info.dir != directory)
5431  buffer.Append("**");
5432  }
5433  return buffer.ToString();
5434  }
5435  }
5436 
5437  // Files that have been sync'd already
5438  private HashSet<string> synced = new HashSet<string>();
5439 
5440  // Files that are now being sync'd
5441  private HashSet<string> syncing = new HashSet<string>();
5442 
5443  private bool StartSync(System.String fileName, ICollection<string> pending)
5444  {
5445  lock (synced)
5446  {
5447  if (!synced.Contains(fileName))
5448  {
5449  if (!syncing.Contains(fileName))
5450  {
5451  syncing.Add(fileName);
5452  return true;
5453  }
5454  else
5455  {
5456  pending.Add(fileName);
5457  return false;
5458  }
5459  }
5460  else
5461  return false;
5462  }
5463  }
5464 
5465  private void FinishSync(System.String fileName, bool success)
5466  {
5467  lock (synced)
5468  {
5469  System.Diagnostics.Debug.Assert(syncing.Contains(fileName));
5470  syncing.Remove(fileName);
5471  if (success)
5472  synced.Add(fileName);
5473  System.Threading.Monitor.PulseAll(synced);
5474  }
5475  }
5476 
5478  private bool WaitForAllSynced(ICollection<System.String> syncing)
5479  {
5480  lock (synced)
5481  {
5482  IEnumerator<string> it = syncing.GetEnumerator();
5483  while (it.MoveNext())
5484  {
5485  System.String fileName = it.Current;
5486  while (!synced.Contains(fileName))
5487  {
5488  if (!syncing.Contains(fileName))
5489  // There was an error because a file that was
5490  // previously syncing failed to appear in synced
5491  return false;
5492  else
5493  System.Threading.Monitor.Wait(synced);
5494 
5495  }
5496  }
5497  return true;
5498  }
5499  }
5500 
5501  private void DoWait()
5502  {
5503  lock (this)
5504  {
5505  // NOTE: the callers of this method should in theory
5506  // be able to do simply wait(), but, as a defense
5507  // against thread timing hazards where notifyAll()
5508  // falls to be called, we wait for at most 1 second
5509  // and then return so caller can check if wait
5510  // conditions are satisified:
5511  System.Threading.Monitor.Wait(this, TimeSpan.FromMilliseconds(1000));
5512 
5513  }
5514  }
5515 
5522  private void StartCommit(long sizeInBytes, IDictionary<string, string> commitUserData)
5523  {
5524 
5525  System.Diagnostics.Debug.Assert(TestPoint("startStartCommit"));
5526 
5527  // TODO: as of LUCENE-2095, we can simplify this method,
5528  // since only 1 thread can be in here at once
5529 
5530  if (hitOOM)
5531  {
5532  throw new System.SystemException("this writer hit an OutOfMemoryError; cannot commit");
5533  }
5534 
5535  try
5536  {
5537 
5538  if (infoStream != null)
5539  Message("startCommit(): start sizeInBytes=" + sizeInBytes);
5540 
5541  SegmentInfos toSync = null;
5542  long myChangeCount;
5543 
5544  lock (this)
5545  {
5546  // Wait for any running addIndexes to complete
5547  // first, then block any from running until we've
5548  // copied the segmentInfos we intend to sync:
5549  BlockAddIndexes(false);
5550 
5551  // On commit the segmentInfos must never
5552  // reference a segment in another directory:
5553  System.Diagnostics.Debug.Assert(!HasExternalSegments());
5554 
5555  try
5556  {
5557 
5558  System.Diagnostics.Debug.Assert(lastCommitChangeCount <= changeCount);
5559  myChangeCount = changeCount;
5560 
5561  if (changeCount == lastCommitChangeCount)
5562  {
5563  if (infoStream != null)
5564  Message(" skip startCommit(): no changes pending");
5565  return ;
5566  }
5567 
5568  // First, we clone & incref the segmentInfos we intend
5569  // to sync, then, without locking, we sync() each file
5570  // referenced by toSync, in the background. Multiple
5571  // threads can be doing this at once, if say a large
5572  // merge and a small merge finish at the same time:
5573 
5574  if (infoStream != null)
5575  Message("startCommit index=" + SegString(segmentInfos) + " changeCount=" + changeCount);
5576 
5577  readerPool.Commit();
5578 
5579  // It's possible another flush (that did not close
5580  // the open do stores) snuck in after the flush we
5581  // just did, so we remove any tail segments
5582  // referencing the open doc store from the
5583  // SegmentInfos we are about to sync (the main
5584  // SegmentInfos will keep them):
5585  toSync = (SegmentInfos) segmentInfos.Clone();
5586  string dss = docWriter.DocStoreSegment;
5587  if (dss != null)
5588  {
5589  while (true)
5590  {
5591  String dss2 = toSync.Info(toSync.Count - 1).DocStoreSegment;
5592  if (dss2 == null || !dss2.Equals(dss))
5593  {
5594  break;
5595  }
5596  toSync.RemoveAt(toSync.Count - 1);
5597  changeCount++;
5598  }
5599  }
5600 
5601  if (commitUserData != null)
5602  toSync.UserData = commitUserData;
5603 
5604  deleter.IncRef(toSync, false);
5605 
5606  ICollection<string> files = toSync.Files(directory, false);
5607  foreach(string fileName in files)
5608  {
5609  System.Diagnostics.Debug.Assert(directory.FileExists(fileName), "file " + fileName + " does not exist");
5610  // If this trips it means we are missing a call to
5611  // .checkpoint somewhere, because by the time we
5612  // are called, deleter should know about every
5613  // file referenced by the current head
5614  // segmentInfos:
5615  System.Diagnostics.Debug.Assert(deleter.Exists(fileName));
5616  }
5617  }
5618  finally
5619  {
5620  ResumeAddIndexes();
5621  }
5622  }
5623 
5624  System.Diagnostics.Debug.Assert(TestPoint("midStartCommit"));
5625 
5626  bool setPending = false;
5627 
5628  try
5629  {
5630  // Loop until all files toSync references are sync'd:
5631  while (true)
5632  {
5633  ICollection<string> pending = new List<string>();
5634 
5635  IEnumerator<string> it = toSync.Files(directory, false).GetEnumerator();
5636  while (it.MoveNext())
5637  {
5638  string fileName = it.Current;
5639  if (StartSync(fileName, pending))
5640  {
5641  bool success = false;
5642  try
5643  {
5644  // Because we incRef'd this commit point, above,
5645  // the file had better exist:
5646  System.Diagnostics.Debug.Assert(directory.FileExists(fileName), "file '" + fileName + "' does not exist dir=" + directory);
5647  if (infoStream != null)
5648  Message("now sync " + fileName);
5649  directory.Sync(fileName);
5650  success = true;
5651  }
5652  finally
5653  {
5654  FinishSync(fileName, success);
5655  }
5656  }
5657  }
5658 
5659  // All files that I require are either synced or being
5660  // synced by other threads. If they are being synced,
5661  // we must at this point block until they are done.
5662  // If this returns false, that means an error in
5663  // another thread resulted in failing to actually
5664  // sync one of our files, so we repeat:
5665  if (WaitForAllSynced(pending))
5666  break;
5667  }
5668 
5669  System.Diagnostics.Debug.Assert(TestPoint("midStartCommit2"));
5670 
5671  lock (this)
5672  {
5673  // If someone saved a newer version of segments file
5674  // since I first started syncing my version, I can
5675  // safely skip saving myself since I've been
5676  // superseded:
5677 
5678  while (true)
5679  {
5680  if (myChangeCount <= lastCommitChangeCount)
5681  {
5682  if (infoStream != null)
5683  {
5684  Message("sync superseded by newer infos");
5685  }
5686  break;
5687  }
5688  else if (pendingCommit == null)
5689  {
5690  // My turn to commit
5691 
5692  if (segmentInfos.Generation > toSync.Generation)
5693  toSync.UpdateGeneration(segmentInfos);
5694 
5695  bool success = false;
5696  try
5697  {
5698 
5699  // Exception here means nothing is prepared
5700  // (this method unwinds everything it did on
5701  // an exception)
5702  try
5703  {
5704  toSync.PrepareCommit(directory);
5705  }
5706  finally
5707  {
5708  // Have our master segmentInfos record the
5709  // generations we just prepared. We do this
5710  // on error or success so we don't
5711  // double-write a segments_N file.
5712  segmentInfos.UpdateGeneration(toSync);
5713  }
5714 
5715  System.Diagnostics.Debug.Assert(pendingCommit == null);
5716  setPending = true;
5717  pendingCommit = toSync;
5718  pendingCommitChangeCount = (uint) myChangeCount;
5719  success = true;
5720  }
5721  finally
5722  {
5723  if (!success && infoStream != null)
5724  Message("hit exception committing segments file");
5725  }
5726  break;
5727  }
5728  else
5729  {
5730  // Must wait for other commit to complete
5731  DoWait();
5732  }
5733  }
5734  }
5735 
5736  if (infoStream != null)
5737  Message("done all syncs");
5738 
5739  System.Diagnostics.Debug.Assert(TestPoint("midStartCommitSuccess"));
5740  }
5741  finally
5742  {
5743  lock (this)
5744  {
5745  if (!setPending)
5746  deleter.DecRef(toSync);
5747  }
5748  }
5749  }
5750  catch (System.OutOfMemoryException oom)
5751  {
5752  HandleOOM(oom, "startCommit");
5753  }
5754  System.Diagnostics.Debug.Assert(TestPoint("finishStartCommit"));
5755  }
5756 
5763  public static bool IsLocked(Directory directory)
5764  {
5765  return directory.MakeLock(WRITE_LOCK_NAME).IsLocked();
5766  }
5767 
5774  public static void Unlock(Directory directory)
5775  {
5776  directory.MakeLock(IndexWriter.WRITE_LOCK_NAME).Release();
5777  }
5778 
5783  public sealed class MaxFieldLength
5784  {
5785 
5786  private int limit;
5787  private System.String name;
5788 
5796  internal MaxFieldLength(System.String name, int limit)
5797  {
5798  this.name = name;
5799  this.limit = limit;
5800  }
5801 
5807  public MaxFieldLength(int limit):this("User-specified", limit)
5808  {
5809  }
5810 
5811  public int Limit
5812  {
5813  get { return limit; }
5814  }
5815 
5816  public override System.String ToString()
5817  {
5818  return name + ":" + limit;
5819  }
5820 
5822  public static readonly MaxFieldLength UNLIMITED = new MaxFieldLength("UNLIMITED", System.Int32.MaxValue);
5823 
5828  public static readonly MaxFieldLength LIMITED;
5829  static MaxFieldLength()
5830  {
5831  LIMITED = new MaxFieldLength("LIMITED", Lucene.Net.Index.IndexWriter.DEFAULT_MAX_FIELD_LENGTH);
5832  }
5833  }
5834 
5849  public abstract class IndexReaderWarmer
5850  {
5851  public abstract void Warm(IndexReader reader);
5852  }
5853 
5854  private IndexReaderWarmer mergedSegmentWarmer;
5855 
5859  public virtual IndexReaderWarmer MergedSegmentWarmer
5860  {
5861  set { mergedSegmentWarmer = value; }
5862  get { return mergedSegmentWarmer; }
5863  }
5864 
5865  private void HandleOOM(System.OutOfMemoryException oom, System.String location)
5866  {
5867  if (infoStream != null)
5868  {
5869  Message("hit OutOfMemoryError inside " + location);
5870  }
5871  hitOOM = true;
5872  throw oom;
5873  }
5874 
5875  // Used only by assert for testing. Current points:
5876  // startDoFlush
5877  // startCommitMerge
5878  // startStartCommit
5879  // midStartCommit
5880  // midStartCommit2
5881  // midStartCommitSuccess
5882  // finishStartCommit
5883  // startCommitMergeDeletes
5884  // startMergeInit
5885  // startApplyDeletes
5886  // DocumentsWriter.ThreadState.init start
5887  public /*internal*/ virtual bool TestPoint(System.String name)
5888  {
5889  return true;
5890  }
5891 
5892  internal virtual bool NrtIsCurrent(SegmentInfos infos)
5893  {
5894  lock (this)
5895  {
5896  if (!infos.Equals(segmentInfos))
5897  {
5898  // if any structural changes (new segments), we are
5899  // stale
5900  return false;
5901  }
5902  else if (infos.Generation != segmentInfos.Generation)
5903  {
5904  // if any commit took place since we were opened, we
5905  // are stale
5906  return false;
5907  }
5908  else
5909  {
5910  return !docWriter.AnyChanges;
5911  }
5912  }
5913  }
5914 
5915  internal virtual bool IsClosed()
5916  {
5917  lock (this)
5918  {
5919  return closed;
5920  }
5921  }
5922 
5923  static IndexWriter()
5924  {
5925  MAX_TERM_LENGTH = DocumentsWriter.MAX_TERM_LENGTH;
5926  }
5927  }
5928 }