/* * Copyright 2004 The Apache Software Foundation * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ using System; using Analyzer = Lucene.Net.Analysis.Analyzer; using Document = Lucene.Net.Documents.Document; using Similarity = Lucene.Net.Search.Similarity; using Directory = Lucene.Net.Store.Directory; using FSDirectory = Lucene.Net.Store.FSDirectory; using IndexInput = Lucene.Net.Store.IndexInput; using IndexOutput = Lucene.Net.Store.IndexOutput; using Lock = Lucene.Net.Store.Lock; using RAMDirectory = Lucene.Net.Store.RAMDirectory; namespace Lucene.Net.Index { /// An IndexWriter creates and maintains an index. /// The third argument to the /// constructor /// determines whether a new index is created, or whether an existing index is /// opened for the addition of new documents. /// In either case, documents are added with the addDocument method. /// When finished adding documents, close should be called. /// If an index will not have more documents added for a while and optimal search /// performance is desired, then the optimize /// method should be called before the index is closed. /// public class IndexWriter { private class AnonymousClassWith : Lock.With { private void InitBlock(bool create, IndexWriter enclosingInstance) { this.create = create; this.enclosingInstance = enclosingInstance; } private bool create; private IndexWriter enclosingInstance; public IndexWriter Enclosing_Instance { get { return enclosingInstance; } } internal AnonymousClassWith(bool create, IndexWriter enclosingInstance, Lucene.Net.Store.Lock Param1, long Param2) : base(Param1, Param2) { InitBlock(create, enclosingInstance); } public override System.Object DoBody() { if (create) Enclosing_Instance.segmentInfos.Write(Enclosing_Instance.directory); else Enclosing_Instance.segmentInfos.Read(Enclosing_Instance.directory); return null; } } private class AnonymousClassWith1 : Lock.With { private void InitBlock(System.Collections.ArrayList segmentsToDelete, IndexWriter enclosingInstance) { this.segmentsToDelete = segmentsToDelete; this.enclosingInstance = enclosingInstance; } private System.Collections.ArrayList segmentsToDelete; private IndexWriter enclosingInstance; public IndexWriter Enclosing_Instance { get { return enclosingInstance; } } internal AnonymousClassWith1(System.Collections.ArrayList segmentsToDelete, IndexWriter enclosingInstance, Lucene.Net.Store.Lock Param1, long Param2) : base(Param1, Param2) { InitBlock(segmentsToDelete, enclosingInstance); } public override System.Object DoBody() { Enclosing_Instance.segmentInfos.Write(Enclosing_Instance.directory); // commit changes Enclosing_Instance.DeleteSegments(segmentsToDelete); // delete now-unused segments return null; } } private class AnonymousClassWith2 : Lock.With { private void InitBlock(System.String mergedName, System.Collections.ArrayList filesToDelete, IndexWriter enclosingInstance) { this.mergedName = mergedName; this.filesToDelete = filesToDelete; this.enclosingInstance = enclosingInstance; } private System.String mergedName; private System.Collections.ArrayList filesToDelete; private IndexWriter enclosingInstance; public IndexWriter Enclosing_Instance { get { return enclosingInstance; } } internal AnonymousClassWith2(System.String mergedName, System.Collections.ArrayList filesToDelete, IndexWriter enclosingInstance, Lucene.Net.Store.Lock Param1, long Param2):base(Param1, Param2) { InitBlock(mergedName, filesToDelete, enclosingInstance); } public override System.Object DoBody() { // make compound file visible for SegmentReaders Enclosing_Instance.directory.RenameFile(mergedName + ".tmp", mergedName + ".cfs"); // delete now unused files of segment Enclosing_Instance.DeleteFiles(filesToDelete); return null; } } private class AnonymousClassWith3 : Lock.With { private void InitBlock(System.Collections.ArrayList segmentsToDelete, IndexWriter enclosingInstance) { this.segmentsToDelete = segmentsToDelete; this.enclosingInstance = enclosingInstance; } private System.Collections.ArrayList segmentsToDelete; private IndexWriter enclosingInstance; public IndexWriter Enclosing_Instance { get { return enclosingInstance; } } internal AnonymousClassWith3(System.Collections.ArrayList segmentsToDelete, IndexWriter enclosingInstance, Lucene.Net.Store.Lock Param1, long Param2):base(Param1, Param2) { InitBlock(segmentsToDelete, enclosingInstance); } public override System.Object DoBody() { Enclosing_Instance.segmentInfos.Write(Enclosing_Instance.directory); // commit before deleting Enclosing_Instance.DeleteSegments(segmentsToDelete); // delete now-unused segments return null; } } private class AnonymousClassWith4 : Lock.With { private void InitBlock(System.String mergedName, System.Collections.ArrayList filesToDelete, IndexWriter enclosingInstance) { this.mergedName = mergedName; this.filesToDelete = filesToDelete; this.enclosingInstance = enclosingInstance; } private System.String mergedName; private System.Collections.ArrayList filesToDelete; private IndexWriter enclosingInstance; public IndexWriter Enclosing_Instance { get { return enclosingInstance; } } internal AnonymousClassWith4(System.String mergedName, System.Collections.ArrayList filesToDelete, IndexWriter enclosingInstance, Lucene.Net.Store.Lock Param1, long Param2):base(Param1, Param2) { InitBlock(mergedName, filesToDelete, enclosingInstance); } public override System.Object DoBody() { // make compound file visible for SegmentReaders Enclosing_Instance.directory.RenameFile(mergedName + ".tmp", mergedName + ".cfs"); // delete now unused files of segment Enclosing_Instance.DeleteFiles(filesToDelete); return null; } } private void InitBlock() { similarity = Similarity.GetDefault(); } /// Default value is 1000. Use Lucene.Net.writeLockTimeout /// system property to override. /// public static long WRITE_LOCK_TIMEOUT = SupportClass.AppSettings.Get("Lucene.Net.writeLockTimeout", 1000L); /// Default value is 10000. Use Lucene.Net.commitLockTimeout /// system property to override. /// public static long COMMIT_LOCK_TIMEOUT = System.Int32.Parse(SupportClass.AppSettings.Get("Lucene.Net.commitLockTimeout", "10000")); public const System.String WRITE_LOCK_NAME = "write.lock"; public const System.String COMMIT_LOCK_NAME = "commit.lock"; /// Default value is 10. Use Lucene.Net.mergeFactor /// system property to override. /// public static readonly int DEFAULT_MERGE_FACTOR = System.Int32.Parse(SupportClass.AppSettings.Get("Lucene.Net.mergeFactor", "10")); /// Default value is 10. Use Lucene.Net.minMergeDocs /// system property to override. /// public static readonly int DEFAULT_MIN_MERGE_DOCS = System.Int32.Parse(SupportClass.AppSettings.Get("Lucene.Net.minMergeDocs", "10")); /// Default value is {@link Integer#MAX_VALUE}. /// Use Lucene.Net.maxMergeDocs system property to override. /// public static readonly int DEFAULT_MAX_MERGE_DOCS = System.Int32.Parse(SupportClass.AppSettings.Get("Lucene.Net.maxMergeDocs", System.Convert.ToString(System.Int32.MaxValue))); /// Default value is 10000. Use Lucene.Net.maxFieldLength /// system property to override. /// public static readonly int DEFAULT_MAX_FIELD_LENGTH = System.Int32.Parse(SupportClass.AppSettings.Get("Lucene.Net.maxFieldLength", "10000")); //// "5000000")); // "2147483647")); /// The default value for {@link #GetTermIndexInterval()}. This is /// determined by the Lucene.Net.termIndexInterval system /// property. The default is 128. /// public static readonly int DEFAULT_TERM_INDEX_INTERVAL = System.Int32.Parse(SupportClass.AppSettings.Get("Lucene.Net.termIndexInterval", "128")); private Directory directory; // where this index resides private Analyzer analyzer; // how to analyze text private Similarity similarity; // how to normalize private SegmentInfos segmentInfos = new SegmentInfos(); // the segments private Directory ramDirectory = new RAMDirectory(); // for temp segs private Lock writeLock; private int termIndexInterval = DEFAULT_TERM_INDEX_INTERVAL; /// Use compound file setting. Defaults to true, minimizing the number of /// files used. Setting this to false may improve indexing performance, but /// may also cause file handle problems. /// private bool useCompoundFile = true; private bool closeDir; /// Get the current setting of whether to use the compound file format. /// Note that this just returns the value you set with setUseCompoundFile(boolean) /// or the default. You cannot use this to query the status of an existing index. /// /// /// public virtual bool GetUseCompoundFile() { return useCompoundFile; } /// Setting to turn on usage of a compound file. When on, multiple files /// for each segment are merged into a single file once the segment creation /// is finished. This is done regardless of what directory is in use. /// public virtual void SetUseCompoundFile(bool value_Renamed) { useCompoundFile = value_Renamed; } /// Expert: Set the Similarity implementation used by this IndexWriter. /// /// /// /// public virtual void SetSimilarity(Similarity similarity) { this.similarity = similarity; } /// Expert: Return the Similarity implementation used by this IndexWriter. /// ///

This defaults to the current value of {@link Similarity#GetDefault()}. ///

public virtual Similarity GetSimilarity() { return this.similarity; } /// Expert: Set the interval between indexed terms. Large values cause less /// memory to be used by IndexReader, but slow random-access to terms. Small /// values cause more memory to be used by an IndexReader, and speed /// random-access to terms. /// /// This parameter determines the amount of computation required per query /// term, regardless of the number of documents that contain that term. In /// particular, it is the maximum number of other terms that must be /// scanned before a term is located and its frequency and position information /// may be processed. In a large index with user-entered query terms, query /// processing time is likely to be dominated not by term lookup but rather /// by the processing of frequency and positional data. In a small index /// or when many uncommon query terms are generated (e.g., by wildcard /// queries) term lookup may become a dominant cost. /// /// In particular, numUniqueTerms/interval terms are read into /// memory by an IndexReader, and, on average, interval/2 terms /// must be scanned for each random term access. /// /// /// /// public virtual void SetTermIndexInterval(int interval) { this.termIndexInterval = interval; } /// Expert: Return the interval between indexed terms. /// /// /// /// public virtual int GetTermIndexInterval() { return termIndexInterval; } /// Constructs an IndexWriter for the index in path. /// Text will be analyzed with a. If create /// is true, then a new, empty index will be created in /// path, replacing the index already there, if any. /// /// /// the path to the index directory /// /// the analyzer to use /// /// true to create the index or overwrite /// the existing one; false to append to the existing /// index /// /// IOException if the directory cannot be read/written to, or /// if it does not exist, and create is /// false /// public IndexWriter(System.String path, Analyzer a, bool create) : this(FSDirectory.GetDirectory(path, create), a, create, true) { } /// Constructs an IndexWriter for the index in path. /// Text will be analyzed with a. If create /// is true, then a new, empty index will be created in /// path, replacing the index already there, if any. /// /// /// the path to the index directory /// /// the analyzer to use /// /// true to create the index or overwrite /// the existing one; false to append to the existing /// index /// /// IOException if the directory cannot be read/written to, or /// if it does not exist, and create is /// false /// public IndexWriter(System.IO.FileInfo path, Analyzer a, bool create) : this(FSDirectory.GetDirectory(path, create), a, create, true) { } /// Constructs an IndexWriter for the index in d. /// Text will be analyzed with a. If create /// is true, then a new, empty index will be created in /// d, replacing the index already there, if any. /// /// /// the index directory /// /// the analyzer to use /// /// true to create the index or overwrite /// the existing one; false to append to the existing /// index /// /// IOException if the directory cannot be read/written to, or /// if it does not exist, and create is /// false /// public IndexWriter(Directory d, Analyzer a, bool create):this(d, a, create, false) { } private IndexWriter(Directory d, Analyzer a, bool create, bool closeDir) { InitBlock(); this.closeDir = closeDir; directory = d; analyzer = a; Lock writeLock = directory.MakeLock(IndexWriter.WRITE_LOCK_NAME); if (!writeLock.Obtain(WRITE_LOCK_TIMEOUT)) // obtain write lock { throw new System.IO.IOException("Index locked for write: " + writeLock); } this.writeLock = writeLock; // save it lock (directory) { // in- & inter-process sync new AnonymousClassWith(create, this, directory.MakeLock(IndexWriter.COMMIT_LOCK_NAME), COMMIT_LOCK_TIMEOUT).Run(); } } /// Determines the largest number of documents ever merged by addDocument(). /// Small values (e.g., less than 10,000) are best for interactive indexing, /// as this limits the length of pauses while indexing to a few seconds. /// Larger values are best for batched indexing and speedier searches. /// ///

The default value is {@link Integer#MAX_VALUE}. ///

public virtual void SetMaxMergeDocs(int maxMergeDocs) { this.maxMergeDocs = maxMergeDocs; } /// /// public virtual int GetMaxMergeDocs() { return maxMergeDocs; } /// The maximum number of terms that will be indexed for a single field in a /// document. This limits the amount of memory required for indexing, so that /// collections with very large files will not crash the indexing process by /// running out of memory.

/// Note that this effectively truncates large documents, excluding from the /// index terms that occur further in the document. If you know your source /// documents are large, be sure to set this value high enough to accomodate /// the expected size. If you set it to Integer.MAX_VALUE, then the only limit /// is your memory, but you should anticipate an OutOfMemoryError.

/// By default, no more than 10,000 terms will be indexed for a field. ///

public virtual void SetMaxFieldLength(int maxFieldLength) { this.maxFieldLength = maxFieldLength; } /// /// public virtual int GetMaxFieldLength() { return maxFieldLength; } /// Determines the minimal number of documents required before the buffered /// in-memory documents are merging and a new Segment is created. /// Since Documents are merged in a {@link Lucene.Net.store.RAMDirectory}, /// large value gives faster indexing. At the same time, mergeFactor limits /// the number of files open in a FSDirectory. /// ///

The default value is 10. /// ///

/// IllegalArgumentException if maxBufferedDocs is smaller than 1 public virtual void SetMaxBufferedDocs(int maxBufferedDocs) { if (maxBufferedDocs < 1) throw new System.ArgumentException("maxBufferedDocs must at least be 1"); this.minMergeDocs = maxBufferedDocs; } /// /// public virtual int GetMaxBufferedDocs() { return minMergeDocs; } /// Determines how often segment indices are merged by addDocument(). With /// smaller values, less RAM is used while indexing, and searches on /// unoptimized indices are faster, but indexing speed is slower. With larger /// values, more RAM is used during indexing, and while searches on unoptimized /// indices are slower, indexing is faster. Thus larger values (> 10) are best /// for batch index creation, and smaller values (< 10) for indices that are /// interactively maintained. /// ///

This must never be less than 2. The default value is 10. ///

public virtual void SetMergeFactor(int mergeFactor) { if (mergeFactor < 2) throw new System.ArgumentException("mergeFactor cannot be less than 2"); this.mergeFactor = mergeFactor; } /// /// public virtual int GetMergeFactor() { return mergeFactor; } /// If non-null, information about merges and a message when /// maxFieldLength is reached will be printed to this. /// public virtual void SetInfoStream(System.IO.TextWriter infoStream) { this.infoStream = infoStream; } /// /// public virtual System.IO.TextWriter GetInfoStream() { return infoStream; } /// Flushes all changes to an index and closes all associated files. public virtual void Close() { lock (this) { FlushRamSegments(); ramDirectory.Close(); if (writeLock != null) { writeLock.Release(); // release write lock writeLock = null; } if (closeDir) directory.Close(); System.GC.SuppressFinalize(this); } } /// Release the write lock, if needed. ~IndexWriter() { if (writeLock != null) { writeLock.Release(); // release write lock writeLock = null; } } /// Returns the Directory used by this index. public virtual Directory GetDirectory() { return directory; } /// Returns the analyzer used by this index. public virtual Analyzer GetAnalyzer() { return analyzer; } /// Returns the number of documents currently in this index. public virtual int DocCount() { lock (this) { int count = 0; for (int i = 0; i < segmentInfos.Count; i++) { SegmentInfo si = segmentInfos.Info(i); count += si.docCount; } return count; } } /// The maximum number of terms that will be indexed for a single Field in a /// document. This limits the amount of memory required for indexing, so that /// collections with very large files will not crash the indexing process by /// running out of memory.

/// Note that this effectively truncates large documents, excluding from the /// index terms that occur further in the document. If you know your source /// documents are large, be sure to set this value high enough to accomodate /// the expected size. If you set it to Integer.MAX_VALUE, then the only limit /// is your memory, but you should anticipate an OutOfMemoryError.

/// By default, no more than 10,000 terms will be indexed for a Field. /// ///

/// use {@link #setMaxFieldLength} instead /// public int maxFieldLength = DEFAULT_MAX_FIELD_LENGTH; /// Adds a document to this index. If the document contains more than /// {@link #maxFieldLength} terms for a given Field, the remainder are /// discarded. /// public virtual void AddDocument(Document doc) { AddDocument(doc, analyzer); } /// Adds a document to this index, using the provided analyzer instead of the /// value of {@link #GetAnalyzer()}. If the document contains more than /// {@link #maxFieldLength} terms for a given Field, the remainder are /// discarded. /// public virtual void AddDocument(Document doc, Analyzer analyzer) { DocumentWriter dw = new DocumentWriter(ramDirectory, analyzer, this); dw.SetInfoStream(infoStream); System.String segmentName = NewSegmentName(); dw.AddDocument(segmentName, doc); lock (this) { segmentInfos.Add(new SegmentInfo(segmentName, 1, ramDirectory)); MaybeMergeSegments(); } } internal int GetSegmentsCounter() { return segmentInfos.counter; } private System.String NewSegmentName() { lock (this) { return "_" + SupportClass.Number.ToString(segmentInfos.counter++, SupportClass.Number.MAX_RADIX); } } /// Determines how often segment indices are merged by addDocument(). With /// smaller values, less RAM is used while indexing, and searches on /// unoptimized indices are faster, but indexing speed is slower. With larger /// values, more RAM is used during indexing, and while searches on unoptimized /// indices are slower, indexing is faster. Thus larger values (> 10) are best /// for batch index creation, and smaller values (< 10) for indices that are /// interactively maintained. /// ///

This must never be less than 2. The default value is 10. ///

/// use {@link #setMergeFactor} instead /// public int mergeFactor = DEFAULT_MERGE_FACTOR; /// Determines the minimal number of documents required before the buffered /// in-memory documents are merging and a new Segment is created. /// Since Documents are merged in a {@link Lucene.Net.Store.RAMDirectory}, /// large value gives faster indexing. At the same time, mergeFactor limits /// the number of files open in a FSDirectory. /// ///

The default value is 10. ///

/// use {@link #setMaxBufferedDocs} instead /// public int minMergeDocs = DEFAULT_MIN_MERGE_DOCS; /// Determines the largest number of documents ever merged by addDocument(). /// Small values (e.g., less than 10,000) are best for interactive indexing, /// as this limits the length of pauses while indexing to a few seconds. /// Larger values are best for batched indexing and speedier searches. /// ///

The default value is {@link Integer#MAX_VALUE}. ///

/// use {@link #setMaxMergeDocs} instead /// public int maxMergeDocs = DEFAULT_MAX_MERGE_DOCS; /// If non-null, information about merges will be printed to this. /// use {@link #setInfoStream} instead /// public System.IO.TextWriter infoStream = null; /// Merges all segments together into a single segment, optimizing an index /// for search. /// public virtual void Optimize() { lock (this) { FlushRamSegments(); while (segmentInfos.Count > 1 || (segmentInfos.Count == 1 && (SegmentReader.HasDeletions(segmentInfos.Info(0)) || segmentInfos.Info(0).dir != directory || (useCompoundFile && (!SegmentReader.UsesCompoundFile(segmentInfos.Info(0)) || SegmentReader.HasSeparateNorms(segmentInfos.Info(0))))))) { int minSegment = segmentInfos.Count - mergeFactor; MergeSegments(minSegment < 0?0:minSegment); } } } /// Merges all segments from an array of indexes into this index. /// ///

This may be used to parallelize batch indexing. A large document /// collection can be broken into sub-collections. Each sub-collection can be /// indexed in parallel, on a different thread, process or machine. The /// complete index can then be created by merging sub-collection indexes /// with this method. /// ///

After this completes, the index is optimized. ///

public virtual void AddIndexes(Directory[] dirs) { lock (this) { Optimize(); // start with zero or 1 seg for (int i = 0; i < dirs.Length; i++) { SegmentInfos sis = new SegmentInfos(); // read infos from dir sis.Read(dirs[i]); for (int j = 0; j < sis.Count; j++) { segmentInfos.Add(sis.Info(j)); // add each info } } Optimize(); // final cleanup } } /// Merges the provided indexes into this index. ///

After this completes, the index is optimized.

///

The provided IndexReaders are not closed.

///
public virtual void AddIndexes(IndexReader[] readers) { lock (this) { Optimize(); // start with zero or 1 seg System.String mergedName = NewSegmentName(); SegmentMerger merger = new SegmentMerger(this, mergedName); System.Collections.ArrayList segmentsToDelete = System.Collections.ArrayList.Synchronized(new System.Collections.ArrayList(10)); IndexReader sReader = null; if (segmentInfos.Count == 1) { // add existing index, if any sReader = SegmentReader.Get(segmentInfos.Info(0)); merger.Add(sReader); segmentsToDelete.Add(sReader); // queue segment for deletion } for (int i = 0; i < readers.Length; i++) // add new indexes merger.Add(readers[i]); int docCount = merger.Merge(); // merge 'em segmentInfos.RemoveRange(0, segmentInfos.Count - 0); // pop old infos & add new segmentInfos.Add(new SegmentInfo(mergedName, docCount, directory)); if (sReader != null) sReader.Close(); lock (directory) { // in- & inter-process sync new AnonymousClassWith1(segmentsToDelete, this, directory.MakeLock(COMMIT_LOCK_NAME), COMMIT_LOCK_TIMEOUT).Run(); } if (useCompoundFile) { System.Collections.ArrayList filesToDelete = merger.CreateCompoundFile(mergedName + ".tmp"); lock (directory) { // in- & inter-process sync new AnonymousClassWith2(mergedName, filesToDelete, this, directory.MakeLock(COMMIT_LOCK_NAME), COMMIT_LOCK_TIMEOUT).Run(); } } } } /// Merges all RAM-resident segments. private void FlushRamSegments() { int minSegment = segmentInfos.Count - 1; int docCount = 0; while (minSegment >= 0 && (segmentInfos.Info(minSegment)).dir == ramDirectory) { docCount += segmentInfos.Info(minSegment).docCount; minSegment--; } if (minSegment < 0 || (docCount + segmentInfos.Info(minSegment).docCount) > mergeFactor || !(segmentInfos.Info(segmentInfos.Count - 1).dir == ramDirectory)) minSegment++; if (minSegment >= segmentInfos.Count) return ; // none to merge MergeSegments(minSegment); } /// Incremental segment merger. private void MaybeMergeSegments() { long targetMergeDocs = minMergeDocs; while (targetMergeDocs <= maxMergeDocs) { // find segments smaller than current target size int minSegment = segmentInfos.Count; int mergeDocs = 0; while (--minSegment >= 0) { SegmentInfo si = segmentInfos.Info(minSegment); if (si.docCount >= targetMergeDocs) break; mergeDocs += si.docCount; } if (mergeDocs >= targetMergeDocs) // found a merge to do MergeSegments(minSegment + 1); else break; targetMergeDocs *= mergeFactor; // increase target size } } /// Pops segments off of segmentInfos stack down to minSegment, merges them, /// and pushes the merged index onto the top of the segmentInfos stack. /// private void MergeSegments(int minSegment) { System.String mergedName = NewSegmentName(); if (infoStream != null) infoStream.Write("merging segments"); SegmentMerger merger = new SegmentMerger(this, mergedName); System.Collections.ArrayList segmentsToDelete = System.Collections.ArrayList.Synchronized(new System.Collections.ArrayList(10)); for (int i = minSegment; i < segmentInfos.Count; i++) { SegmentInfo si = segmentInfos.Info(i); if (infoStream != null) infoStream.Write(" " + si.name + " (" + si.docCount + " docs)"); IndexReader reader = SegmentReader.Get(si); merger.Add(reader); if ((reader.Directory() == this.directory) || (reader.Directory() == this.ramDirectory)) segmentsToDelete.Add(reader); // queue segment for deletion } int mergedDocCount = merger.Merge(); if (infoStream != null) { infoStream.WriteLine(" into " + mergedName + " (" + mergedDocCount + " docs)"); } segmentInfos.RemoveRange(minSegment, segmentInfos.Count - minSegment); // pop old infos & add new segmentInfos.Add(new SegmentInfo(mergedName, mergedDocCount, directory)); // close readers before we attempt to delete now-obsolete segments merger.CloseReaders(); lock (directory) { // in- & inter-process sync new AnonymousClassWith3(segmentsToDelete, this, directory.MakeLock(COMMIT_LOCK_NAME), COMMIT_LOCK_TIMEOUT).Run(); } if (useCompoundFile) { //UPGRADE_NOTE: Final was removed from the declaration of 'filesToDelete '. 'ms-help://MS.VSCC.2003/commoner/redir/redirect.htm?keyword="jlca1003_3"' System.Collections.ArrayList filesToDelete = merger.CreateCompoundFile(mergedName + ".tmp"); lock (directory) { // in- & inter-process sync new AnonymousClassWith4(mergedName, filesToDelete, this, directory.MakeLock(COMMIT_LOCK_NAME), COMMIT_LOCK_TIMEOUT).Run(); } } } /* * Some operating systems (e.g. Windows) don't permit a file to be deleted * while it is opened for read (e.g. by another process or thread). So we * assume that when a delete fails it is because the file is open in another * process, and queue the file for subsequent deletion. */ private void DeleteSegments(System.Collections.ArrayList segments) { System.Collections.ArrayList deletable = System.Collections.ArrayList.Synchronized(new System.Collections.ArrayList(10)); DeleteFiles(ReadDeleteableFiles(), deletable); // try to delete deleteable for (int i = 0; i < segments.Count; i++) { SegmentReader reader = (SegmentReader) segments[i]; if (reader.Directory() == this.directory) DeleteFiles(reader.Files(), deletable); // try to delete our files else DeleteFiles(reader.Files(), reader.Directory()); // delete other files } WriteDeleteableFiles(deletable); // note files we can't delete } private void DeleteFiles(System.Collections.ArrayList files) { System.Collections.ArrayList deletable = System.Collections.ArrayList.Synchronized(new System.Collections.ArrayList(10)); DeleteFiles(ReadDeleteableFiles(), deletable); // try to delete deleteable DeleteFiles(files, deletable); // try to delete our files WriteDeleteableFiles(deletable); // note files we can't delete } private void DeleteFiles(System.Collections.ArrayList files, Directory directory) { for (int i = 0; i < files.Count; i++) directory.DeleteFile((System.String) files[i]); } private void DeleteFiles(System.Collections.ArrayList files, System.Collections.ArrayList deletable) { for (int i = 0; i < files.Count; i++) { System.String file = (System.String) files[i]; try { directory.DeleteFile(file); // try to delete each file } catch (System.IO.IOException e) { // if delete fails if (directory.FileExists(file)) { if (infoStream != null) { infoStream.WriteLine(e.Message + "; Will re-try later."); } deletable.Add(file); // add to deletable } } } } private System.Collections.ArrayList ReadDeleteableFiles() { System.Collections.ArrayList result = System.Collections.ArrayList.Synchronized(new System.Collections.ArrayList(10)); if (!directory.FileExists("deletable")) return result; IndexInput input = directory.OpenInput("deletable"); try { for (int i = input.ReadInt(); i > 0; i--) // read file names result.Add(input.ReadString()); } finally { input.Close(); } return result; } private void WriteDeleteableFiles(System.Collections.ArrayList files) { IndexOutput output = directory.CreateOutput("deleteable.new"); try { output.WriteInt(files.Count); for (int i = 0; i < files.Count; i++) output.WriteString((System.String) files[i]); } finally { output.Close(); } directory.RenameFile("deleteable.new", "deletable"); } } }