You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by ar...@apache.org on 2008/06/25 04:52:24 UTC

svn commit: r671404 [3/10] - /incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldInfo.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/FieldInfo.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldInfo.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldInfo.cs Tue Jun 24 19:52:22 2008
@@ -20,7 +20,7 @@
 namespace Lucene.Net.Index
 {
 	
-	public sealed class FieldInfo
+	public sealed class FieldInfo : System.ICloneable
 	{
 		internal System.String name;
 		internal bool isIndexed;
@@ -37,8 +37,10 @@
 		{
 			return isIndexed;
 		}
-
-		internal FieldInfo(System.String na, bool tk, int nu, bool storeTermVector, bool storePositionWithTermVector, bool storeOffsetWithTermVector, bool omitNorms)
+		
+		internal bool storePayloads; // whether this field stores payloads together with term positions
+		
+		internal FieldInfo(System.String na, bool tk, int nu, bool storeTermVector, bool storePositionWithTermVector, bool storeOffsetWithTermVector, bool omitNorms, bool storePayloads)
 		{
 			name = na;
 			isIndexed = tk;
@@ -47,6 +49,12 @@
 			this.storeOffsetWithTermVector = storeOffsetWithTermVector;
 			this.storePositionWithTermVector = storePositionWithTermVector;
 			this.omitNorms = omitNorms;
+			this.storePayloads = storePayloads;
+		}
+		
+		public System.Object Clone()
+		{
+			return new FieldInfo(name, isIndexed, number, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads);
 		}
 
         // For testing only

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldInfos.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/FieldInfos.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldInfos.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldInfos.cs Tue Jun 24 19:52:22 2008
@@ -16,6 +16,7 @@
  */
 
 using System;
+
 using Document = Lucene.Net.Documents.Document;
 using Fieldable = Lucene.Net.Documents.Fieldable;
 using Directory = Lucene.Net.Store.Directory;
@@ -31,7 +32,7 @@
 	/// be adding documents at a time, with no other reader or writer threads
 	/// accessing this object.
 	/// </summary>
-	public sealed class FieldInfos
+	public sealed class FieldInfos : System.ICloneable
 	{
 		
 		internal const byte IS_INDEXED = (byte) (0x1);
@@ -39,6 +40,7 @@
 		internal const byte STORE_POSITIONS_WITH_TERMVECTOR = (byte) (0x4);
 		internal const byte STORE_OFFSET_WITH_TERMVECTOR = (byte) (0x8);
 		internal const byte OMIT_NORMS = (byte) (0x10);
+		internal const byte STORE_PAYLOADS = (byte) (0x20);
 		
 		private System.Collections.ArrayList byNumber = new System.Collections.ArrayList();
 		private System.Collections.Hashtable byName = new System.Collections.Hashtable();
@@ -68,6 +70,20 @@
 			}
 		}
 		
+		/// <summary> Returns a deep clone of this FieldInfos instance.</summary>
+		public System.Object Clone()
+		{
+			FieldInfos fis = new FieldInfos();
+			int numField = byNumber.Count;
+			for (int i = 0; i < numField; i++)
+			{
+				FieldInfo fi = (FieldInfo) ((FieldInfo) byNumber[i]).Clone();
+				fis.byNumber.Add(fi);
+				fis.byName[fi.name] = fi;
+			}
+			return fis;
+		}
+		
 		/// <summary>Adds field info for a Document. </summary>
 		public void  Add(Document doc)
 		{
@@ -109,7 +125,7 @@
 		/// <param name="isIndexed">Whether the fields are indexed or not
 		/// 
 		/// </param>
-		/// <seealso cref="boolean)">
+		/// <seealso cref="Add(String, boolean)">
 		/// </seealso>
 		public void  Add(System.Collections.ICollection names, bool isIndexed)
 		{
@@ -128,7 +144,7 @@
 		/// </param>
 		/// <param name="isIndexed">true if the field is indexed
 		/// </param>
-		/// <seealso cref="boolean, boolean, boolean, boolean)">
+		/// <seealso cref="Add(String, boolean, boolean, boolean, boolean)">
 		/// </seealso>
 		public void  Add(System.String name, bool isIndexed)
 		{
@@ -191,10 +207,35 @@
 		/// </param>
 		public void  Add(System.String name, bool isIndexed, bool storeTermVector, bool storePositionWithTermVector, bool storeOffsetWithTermVector, bool omitNorms)
 		{
+			Add(name, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, false);
+		}
+		
+		/// <summary>If the field is not yet known, adds it. If it is known, checks to make
+		/// sure that the isIndexed flag is the same as was given previously for this
+		/// field. If not - marks it as being indexed.  Same goes for the TermVector
+		/// parameters.
+		/// 
+		/// </summary>
+		/// <param name="name">The name of the field
+		/// </param>
+		/// <param name="isIndexed">true if the field is indexed
+		/// </param>
+		/// <param name="storeTermVector">true if the term vector should be stored
+		/// </param>
+		/// <param name="storePositionWithTermVector">true if the term vector with positions should be stored
+		/// </param>
+		/// <param name="storeOffsetWithTermVector">true if the term vector with offsets should be stored
+		/// </param>
+		/// <param name="omitNorms">true if the norms for the indexed field should be omitted
+		/// </param>
+		/// <param name="storePayloads">true if payloads should be stored for this field
+		/// </param>
+		public FieldInfo Add(System.String name, bool isIndexed, bool storeTermVector, bool storePositionWithTermVector, bool storeOffsetWithTermVector, bool omitNorms, bool storePayloads)
+		{
 			FieldInfo fi = FieldInfo(name);
 			if (fi == null)
 			{
-				AddInternal(name, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms);
+				return AddInternal(name, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads);
 			}
 			else
 			{
@@ -218,30 +259,26 @@
 				{
 					fi.omitNorms = false; // once norms are stored, always store
 				}
+				if (fi.storePayloads != storePayloads)
+				{
+					fi.storePayloads = true;
+				}
 			}
+			return fi;
 		}
 		
-		
-		private void  AddInternal(System.String name, bool isIndexed, bool storeTermVector, bool storePositionWithTermVector, bool storeOffsetWithTermVector, bool omitNorms)
+		private FieldInfo AddInternal(System.String name, bool isIndexed, bool storeTermVector, bool storePositionWithTermVector, bool storeOffsetWithTermVector, bool omitNorms, bool storePayloads)
 		{
-			FieldInfo fi = new FieldInfo(name, isIndexed, byNumber.Count, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms);
+			FieldInfo fi = new FieldInfo(name, isIndexed, byNumber.Count, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads);
 			byNumber.Add(fi);
 			byName[name] = fi;
+			return fi;
 		}
 		
 		public int FieldNumber(System.String fieldName)
 		{
-			try
-			{
-				FieldInfo fi = FieldInfo(fieldName);
-				if (fi != null)
-					return fi.number;
-			}
-			catch (System.IndexOutOfRangeException ioobe)
-			{
-				return - 1;
-			}
-			return - 1;
+			FieldInfo fi = FieldInfo(fieldName);
+			return (fi != null) ? fi.number : -1;
 		}
 		
 		public FieldInfo FieldInfo(System.String fieldName)
@@ -260,22 +297,18 @@
 		public System.String FieldName(int fieldNumber)
 		{
 			FieldInfo fi = FieldInfo(fieldNumber);
-			if (fi != null)
-				return fi.name;
-			return "";
+			return (fi != null) ? fi.name : "";
 		}
 		
 		/// <summary> Return the fieldinfo object referenced by the fieldNumber.</summary>
-		/// <param name="">fieldNumber
+		/// <param name="fieldNumber">
 		/// </param>
 		/// <returns> the FieldInfo object or null when the given fieldNumber
 		/// doesn't exist.
 		/// </returns>
 		public FieldInfo FieldInfo(int fieldNumber)
 		{
-			if (fieldNumber > -1 && fieldNumber < byNumber.Count)
-				return (FieldInfo) byNumber[fieldNumber];
-			return null;
+			return (fieldNumber >= 0) ? (FieldInfo) byNumber[fieldNumber] : null;
 		}
 		
 		public int Size()
@@ -327,6 +360,8 @@
 					bits |= STORE_OFFSET_WITH_TERMVECTOR;
 				if (fi.omitNorms)
 					bits |= OMIT_NORMS;
+				if (fi.storePayloads)
+					bits |= STORE_PAYLOADS;
 				output.WriteString(fi.name);
 				output.WriteByte(bits);
 			}
@@ -344,8 +379,9 @@
 				bool storePositionsWithTermVector = (bits & STORE_POSITIONS_WITH_TERMVECTOR) != 0;
 				bool storeOffsetWithTermVector = (bits & STORE_OFFSET_WITH_TERMVECTOR) != 0;
 				bool omitNorms = (bits & OMIT_NORMS) != 0;
+				bool storePayloads = (bits & STORE_PAYLOADS) != 0;
 				
-				AddInternal(name, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms);
+				AddInternal(name, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads);
 			}
 		}
 	}

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldReaderException.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/FieldReaderException.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldReaderException.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldReaderException.cs Tue Jun 24 19:52:22 2008
@@ -35,9 +35,11 @@
 		}
 		
 		/// <summary> Constructs a new runtime exception with the specified cause and a
-		/// detail message of <tt>(cause==null ? null : cause.toString())</tt>
+		/// detail message of <tt>(cause==null &#63; null : cause.toString())</tt>
 		/// (which typically contains the class and detail message of
-		/// <tt>cause</tt>).  This constructor is useful for runtime exceptions
+		/// <tt>cause</tt>).  
+		/// <p>
+		/// This constructor is useful for runtime exceptions
 		/// that are little more than wrappers for other throwables.
 		/// 
 		/// </summary>
@@ -48,7 +50,7 @@
 		/// </param>
 		/// <since> 1.4
 		/// </since>
-		public FieldReaderException(System.Exception cause)
+		public FieldReaderException(System.Exception cause) : base((cause == null)? null : cause.ToString(), cause)
 		{
 		}
 		
@@ -60,7 +62,7 @@
 		/// <param name="message">the detail message. The detail message is saved for
 		/// later retrieval by the {@link #getMessage()} method.
 		/// </param>
-		public FieldReaderException(System.String message)
+		public FieldReaderException(System.String message) : base(message)
 		{
 		}
 		
@@ -80,7 +82,7 @@
 		/// </param>
 		/// <since> 1.4
 		/// </since>
-		public FieldReaderException(System.String message, System.Exception cause)
+		public FieldReaderException(System.String message, System.Exception cause) : base(message, cause)
 		{
 		}
 	}

Added: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldSortedTermVectorMapper.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/FieldSortedTermVectorMapper.cs?rev=671404&view=auto
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldSortedTermVectorMapper.cs (added)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldSortedTermVectorMapper.cs Tue Jun 24 19:52:22 2008
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+using System;
+
+namespace Lucene.Net.Index
+{
+	
+	/// <summary> For each Field, store a sorted collection of {@link TermVectorEntry}s
+	/// <p/>
+	/// This is not thread-safe.
+	/// </summary>
+	public class FieldSortedTermVectorMapper : TermVectorMapper
+	{
+		private System.Collections.IDictionary fieldToTerms = new System.Collections.Hashtable();
+		private System.Collections.Generic.SortedDictionary<Object, Object> currentSet;
+		private System.String currentField;
+		private System.Collections.Generic.IComparer<Object> comparator;
+
+		/// <summary> </summary>
+		/// <param name="comparator">A Comparator for sorting {@link TermVectorEntry}s
+		/// </param>
+		public FieldSortedTermVectorMapper(System.Collections.Generic.IComparer<Object> comparator) : this(false, false, comparator)
+		{
+		}
+		
+		
+		public FieldSortedTermVectorMapper(bool ignoringPositions, bool ignoringOffsets, System.Collections.Generic.IComparer<Object> comparator) : base(ignoringPositions, ignoringOffsets)
+		{
+			this.comparator = comparator;
+		}
+		
+		public override void  Map(System.String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions)
+		{
+			TermVectorEntry entry = new TermVectorEntry(currentField, term, frequency, offsets, positions);
+			currentSet.Add(entry, entry);
+		}
+		
+		public override void  SetExpectations(System.String field, int numTerms, bool storeOffsets, bool storePositions)
+		{
+			currentSet = new System.Collections.Generic.SortedDictionary<Object, Object>(comparator);
+			currentField = field;
+			fieldToTerms[field] = currentSet;
+		}
+		
+		/// <summary> Get the mapping between fields and terms, sorted by the comparator
+		/// 
+		/// </summary>
+		/// <returns> A map between field names and {@link java.util.SortedSet}s per field.  SortedSet entries are {@link TermVectorEntry}
+		/// </returns>
+		public virtual System.Collections.IDictionary GetFieldToTerms()
+		{
+			return fieldToTerms;
+		}
+		
+		
+		public virtual System.Collections.Generic.IComparer<Object> GetComparator()
+		{
+			return comparator;
+		}
+	}
+}
\ No newline at end of file

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldsReader.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/FieldsReader.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldsReader.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldsReader.cs Tue Jun 24 19:52:22 2008
@@ -16,9 +16,13 @@
  */
 
 using System;
+
 using Lucene.Net.Documents;
+using AlreadyClosedException = Lucene.Net.Store.AlreadyClosedException;
+using BufferedIndexInput = Lucene.Net.Store.BufferedIndexInput;
 using Directory = Lucene.Net.Store.Directory;
 using IndexInput = Lucene.Net.Store.IndexInput;
+using TokenStream = Lucene.Net.Analysis.TokenStream;
 
 namespace Lucene.Net.Index
 {
@@ -28,7 +32,7 @@
 	/// It uses &lt;segment&gt;.fdt and &lt;segment&gt;.fdx; files.
 	/// 
 	/// </summary>
-	/// <version>  $Id: FieldsReader.java 507009 2007-02-13 14:06:52Z gsingers $
+	/// <version>  $Id: FieldsReader.java 620759 2008-02-12 11:10:21Z mikemccand $
 	/// </version>
 	public sealed class FieldsReader
 	{
@@ -42,18 +46,76 @@
 		private IndexInput fieldsStream;
 		
 		private IndexInput indexStream;
+		private int numTotalDocs;
 		private int size;
+		private bool closed;
+		
+		// The docID offset where our docs begin in the index
+		// file.  This will be 0 if we have our own private file.
+		private int docStoreOffset;
 		
 		private System.LocalDataStoreSlot fieldsStreamTL = System.Threading.Thread.AllocateDataSlot();
 		
-		public FieldsReader(Directory d, System.String segment, FieldInfos fn)
+		internal FieldsReader(Directory d, System.String segment, FieldInfos fn) : this(d, segment, fn, BufferedIndexInput.BUFFER_SIZE, - 1, 0)
+		{
+		}
+		
+		internal FieldsReader(Directory d, System.String segment, FieldInfos fn, int readBufferSize) : this(d, segment, fn, readBufferSize, - 1, 0)
+		{
+		}
+		
+		internal FieldsReader(Directory d, System.String segment, FieldInfos fn, int readBufferSize, int docStoreOffset, int size)
 		{
-			fieldInfos = fn;
+			bool success = false;
 			
-			cloneableFieldsStream = d.OpenInput(segment + ".fdt");
-			fieldsStream = (IndexInput) cloneableFieldsStream.Clone();
-			indexStream = d.OpenInput(segment + ".fdx");
-			size = (int) (indexStream.Length() / 8);
+			try
+			{
+				fieldInfos = fn;
+				
+				cloneableFieldsStream = d.OpenInput(segment + ".fdt", readBufferSize);
+				fieldsStream = (IndexInput) cloneableFieldsStream.Clone();
+				indexStream = d.OpenInput(segment + ".fdx", readBufferSize);
+				
+				if (docStoreOffset != - 1)
+				{
+					// We read only a slice out of this shared fields file
+					this.docStoreOffset = docStoreOffset;
+					this.size = size;
+					
+					// Verify the file is long enough to hold all of our
+					// docs
+					System.Diagnostics.Debug.Assert(((int)(indexStream.Length() / 8)) >= size + this.docStoreOffset);
+				}
+				else
+				{
+					this.docStoreOffset = 0;
+					this.size = (int) (indexStream.Length() >> 3);
+				}
+				
+				numTotalDocs = (int) (indexStream.Length() >> 3);
+				success = true;
+			}
+			finally
+			{
+				// With lock-less commits, it's entirely possible (and
+				// fine) to hit a FileNotFound exception above. In
+				// this case, we want to explicitly close any subset
+				// of things that were opened so that we don't have to
+				// wait for a GC to do so.
+				if (!success)
+				{
+					Close();
+				}
+			}
+		}
+		
+		/// <throws>  AlreadyClosedException if this FieldsReader is closed </throws>
+		internal void  EnsureOpen()
+		{
+			if (closed)
+			{
+				throw new AlreadyClosedException("this FieldsReader is closed");
+			}
 		}
 		
 		/// <summary> Closes the underlying {@link Lucene.Net.Store.IndexInput} streams, including any ones associated with a
@@ -63,14 +125,27 @@
 		/// <throws>  IOException </throws>
 		public void  Close()
 		{
-			fieldsStream.Close();
-			cloneableFieldsStream.Close();
-			indexStream.Close();
-			IndexInput localFieldsStream = (IndexInput) System.Threading.Thread.GetData(fieldsStreamTL);
-			if (localFieldsStream != null)
+			if (!closed)
 			{
-				localFieldsStream.Close();
-				System.Threading.Thread.SetData(fieldsStreamTL, null);
+				if (fieldsStream != null)
+				{
+					fieldsStream.Close();
+				}
+				if (cloneableFieldsStream != null)
+				{
+					cloneableFieldsStream.Close();
+				}
+				if (indexStream != null)
+				{
+					indexStream.Close();
+				}
+				IndexInput localFieldsStream = (IndexInput) System.Threading.Thread.GetData(fieldsStreamTL);
+				if (localFieldsStream != null)
+				{
+					localFieldsStream.Close();
+					System.Threading.Thread.SetData(fieldsStreamTL, null);
+				}
+				closed = true;
 			}
 		}
 		
@@ -94,6 +169,8 @@
 				FieldSelectorResult acceptField = fieldSelector == null ? FieldSelectorResult.LOAD : fieldSelector.Accept(fi.name);
 				
 				byte bits = fieldsStream.ReadByte();
+				System.Diagnostics.Debug.Assert(bits <= FieldsWriter.FIELD_IS_COMPRESSED + FieldsWriter.FIELD_IS_TOKENIZED + FieldsWriter.FIELD_IS_BINARY);
+
 				bool compressed = (bits & FieldsWriter.FIELD_IS_COMPRESSED) != 0;
 				bool tokenize = (bits & FieldsWriter.FIELD_IS_TOKENIZED) != 0;
 				bool binary = (bits & FieldsWriter.FIELD_IS_BINARY) != 0;
@@ -134,6 +211,35 @@
 			return doc;
 		}
 		
+		/// <summary>Returns the length in bytes of each raw document in a
+		/// contiguous range of length numDocs starting with
+		/// startDocID.  Returns the IndexInput (the fieldStream),
+		/// already seeked to the starting point for startDocID.
+		/// </summary>
+		internal IndexInput RawDocs(int[] lengths, int startDocID, int numDocs)
+		{
+			indexStream.Seek((docStoreOffset + startDocID) * 8L);
+			long startOffset = indexStream.ReadLong();
+			long lastOffset = startOffset;
+			int count = 0;
+			while (count < numDocs)
+			{
+				long offset;
+				int docID = docStoreOffset + startDocID + count + 1;
+				System.Diagnostics.Debug.Assert(docID <= numTotalDocs);
+				if (docID < numTotalDocs)
+					offset = indexStream.ReadLong();
+				else
+					offset = fieldsStream.Length();
+				lengths[count++] = (int) (offset - lastOffset);
+				lastOffset = offset;
+			}
+			
+			fieldsStream.Seek(startOffset);
+			
+			return fieldsStream;
+		}
+		
 		/// <summary> Skip the field.  We still have to read some of the information about the field, but can skip past the actual content.
 		/// This will have the most payoff on large fields.
 		/// </summary>
@@ -373,12 +479,13 @@
 				return localFieldsStream;
 			}
 			
-			/// <summary> The value of the field in Binary, or null.  If null, the Reader or
-			/// String value is used.  Exactly one of stringValue(), readerValue() and
-			/// binaryValue() must be set.
+			/// <summary>The value of the field in Binary, or null.  If null, the Reader value,
+			/// String value, or TokenStream value is used. Exactly one of stringValue(), 
+			/// readerValue(), binaryValue(), and tokenStreamValue() must be set. 
 			/// </summary>
 			public override byte[] BinaryValue()
 			{
+				Enclosing_Instance.EnsureOpen();
 				if (fieldsData == null)
 				{
 					byte[] b = new byte[toRead];
@@ -406,21 +513,34 @@
 				return fieldsData is byte[] ? (byte[]) fieldsData : null;
 			}
 			
-			/// <summary> The value of the field as a Reader, or null.  If null, the String value
-			/// or binary value is  used.  Exactly one of stringValue(), readerValue(),
-			/// and binaryValue() must be set.
+			/// <summary>The value of the field as a Reader, or null.  If null, the String value,
+			/// binary value, or TokenStream value is used.  Exactly one of stringValue(), 
+			/// readerValue(), binaryValue(), and tokenStreamValue() must be set. 
 			/// </summary>
 			public override System.IO.TextReader ReaderValue()
 			{
+				Enclosing_Instance.EnsureOpen();
 				return fieldsData is System.IO.TextReader ? (System.IO.TextReader) fieldsData : null;
 			}
 			
-			/// <summary> The value of the field as a String, or null.  If null, the Reader value
-			/// or binary value is used.  Exactly one of stringValue(), readerValue(), and
-			/// binaryValue() must be set.
+			/// <summary>The value of the field as a TokesStream, or null.  If null, the Reader value,
+			/// String value, or binary value is used. Exactly one of stringValue(), 
+			/// readerValue(), binaryValue(), and tokenStreamValue() must be set. 
+			/// </summary>
+			public override TokenStream TokenStreamValue()
+			{
+				Enclosing_Instance.EnsureOpen();
+				return fieldsData is TokenStream ? (TokenStream) fieldsData : null;
+			}
+			
+			
+			/// <summary>The value of the field as a String, or null.  If null, the Reader value,
+			/// binary value, or TokenStream value is used.  Exactly one of stringValue(), 
+			/// readerValue(), binaryValue(), and tokenStreamValue() must be set. 
 			/// </summary>
 			public override System.String StringValue()
 			{
+				Enclosing_Instance.EnsureOpen();
 				if (fieldsData == null)
 				{
 					IndexInput localFieldsStream = GetFieldStream();
@@ -451,21 +571,25 @@
 			
 			public long GetPointer()
 			{
+				Enclosing_Instance.EnsureOpen();
 				return pointer;
 			}
 			
 			public void  SetPointer(long pointer)
 			{
+				Enclosing_Instance.EnsureOpen();
 				this.pointer = pointer;
 			}
 			
 			public int GetToRead()
 			{
+				Enclosing_Instance.EnsureOpen();
 				return toRead;
 			}
 			
 			public void  SetToRead(int toRead)
 			{
+				Enclosing_Instance.EnsureOpen();
 				this.toRead = toRead;
 			}
 		}
@@ -496,6 +620,12 @@
 				return (byte[]) this.fieldsData;
 			}
 			
+			public override TokenStream TokenStreamValue()
+			{
+				// not needed for merge
+				return null;
+			}
+			
 			public FieldForMerge(System.Object value_Renamed, FieldInfo fi, bool binary, bool compressed, bool tokenize)
 			{
 				this.isStored = true;

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldsWriter.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/FieldsWriter.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldsWriter.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FieldsWriter.cs Tue Jun 24 19:52:22 2008
@@ -16,10 +16,13 @@
  */
 
 using System;
+
 using Document = Lucene.Net.Documents.Document;
 using Fieldable = Lucene.Net.Documents.Fieldable;
 using Directory = Lucene.Net.Store.Directory;
+using IndexInput = Lucene.Net.Store.IndexInput;
 using IndexOutput = Lucene.Net.Store.IndexOutput;
+using RAMOutputStream = Lucene.Net.Store.RAMOutputStream;
 
 namespace Lucene.Net.Index
 {
@@ -36,17 +39,128 @@
 		
 		private IndexOutput indexStream;
 		
+		private bool doClose;
+		
 		internal FieldsWriter(Directory d, System.String segment, FieldInfos fn)
 		{
 			fieldInfos = fn;
 			fieldsStream = d.CreateOutput(segment + ".fdt");
 			indexStream = d.CreateOutput(segment + ".fdx");
+			doClose = true;
+		}
+		
+		internal FieldsWriter(IndexOutput fdx, IndexOutput fdt, FieldInfos fn)
+		{
+			fieldInfos = fn;
+			fieldsStream = fdt;
+			indexStream = fdx;
+			doClose = false;
+		}
+		
+		// Writes the contents of buffer into the fields stream
+		// and adds a new entry for this document into the index
+		// stream.  This assumes the buffer was already written
+		// in the correct fields format.
+		internal void  FlushDocument(int numStoredFields, RAMOutputStream buffer)
+		{
+			indexStream.WriteLong(fieldsStream.GetFilePointer());
+			fieldsStream.WriteVInt(numStoredFields);
+			buffer.WriteTo(fieldsStream);
+		}
+		
+		internal void  Flush()
+		{
+			indexStream.Flush();
+			fieldsStream.Flush();
 		}
 		
 		internal void  Close()
 		{
-			fieldsStream.Close();
-			indexStream.Close();
+			if (doClose)
+			{
+				fieldsStream.Close();
+				indexStream.Close();
+			}
+		}
+		
+		internal void  WriteField(FieldInfo fi, Fieldable field)
+		{
+			// if the field as an instanceof FieldsReader.FieldForMerge, we're in merge mode
+			// and field.binaryValue() already returns the compressed value for a field
+			// with isCompressed()==true, so we disable compression in that case
+			bool disableCompression = (field is FieldsReader.FieldForMerge);
+			fieldsStream.WriteVInt(fi.number);
+			byte bits = 0;
+			if (field.IsTokenized())
+				bits |= FieldsWriter.FIELD_IS_TOKENIZED;
+			if (field.IsBinary())
+				bits |= FieldsWriter.FIELD_IS_BINARY;
+			if (field.IsCompressed())
+				bits |= FieldsWriter.FIELD_IS_COMPRESSED;
+			
+			fieldsStream.WriteByte(bits);
+			
+			if (field.IsCompressed())
+			{
+				// compression is enabled for the current field
+				byte[] data = null;
+				
+				if (disableCompression)
+				{
+					// optimized case for merging, the data
+					// is already compressed
+					data = field.BinaryValue();
+				}
+				else
+				{
+					// check if it is a binary field
+					if (field.IsBinary())
+					{
+						data = Compress(field.BinaryValue());
+					}
+					else
+					{
+						data = Compress(System.Text.Encoding.GetEncoding("UTF-8").GetBytes(field.StringValue()));
+					}
+				}
+				int len = data.Length;
+				fieldsStream.WriteVInt(len);
+				fieldsStream.WriteBytes(data, len);
+			}
+			else
+			{
+				// compression is disabled for the current field
+				if (field.IsBinary())
+				{
+					byte[] data = field.BinaryValue();
+					int len = data.Length;
+					fieldsStream.WriteVInt(len);
+					fieldsStream.WriteBytes(data, len);
+				}
+				else
+				{
+					fieldsStream.WriteString(field.StringValue());
+				}
+			}
+		}
+		
+		/// <summary>Bulk write a contiguous series of documents.  The
+		/// lengths array is the length (in bytes) of each raw
+		/// document.  The stream IndexInput is the
+		/// fieldsStream from which we should bulk-copy all
+		/// bytes. 
+		/// </summary>
+		internal void  AddRawDocuments(IndexInput stream, int[] lengths, int numDocs)
+		{
+			long position = fieldsStream.GetFilePointer();
+			long start = position;
+			for (int i = 0; i < numDocs; i++)
+			{
+				indexStream.WriteLong(position);
+				position += lengths[i];
+			}
+			fieldsStream.CopyBytes(stream, position - start);
+			System.Diagnostics.Debug.Assert(fieldsStream.GetFilePointer() == position);
 		}
 		
 		internal void  AddDocument(Document doc)
@@ -67,67 +181,8 @@
 			while (fieldIterator.MoveNext())
 			{
 				Fieldable field = (Fieldable) fieldIterator.Current;
-				// if the field as an instanceof FieldsReader.FieldForMerge, we're in merge mode
-				// and field.binaryValue() already returns the compressed value for a field
-				// with isCompressed()==true, so we disable compression in that case
-				bool disableCompression = (field is FieldsReader.FieldForMerge);
 				if (field.IsStored())
-				{
-					fieldsStream.WriteVInt(fieldInfos.FieldNumber(field.Name()));
-					
-					byte bits = 0;
-					if (field.IsTokenized())
-						bits |= FieldsWriter.FIELD_IS_TOKENIZED;
-					if (field.IsBinary())
-						bits |= FieldsWriter.FIELD_IS_BINARY;
-					if (field.IsCompressed())
-						bits |= FieldsWriter.FIELD_IS_COMPRESSED;
-					
-					fieldsStream.WriteByte(bits);
-					
-					if (field.IsCompressed())
-					{
-						// compression is enabled for the current field
-						byte[] data = null;
-						
-						if (disableCompression)
-						{
-							// optimized case for merging, the data
-							// is already compressed
-							data = field.BinaryValue();
-						}
-						else
-						{
-							// check if it is a binary field
-							if (field.IsBinary())
-							{
-								data = Compress(field.BinaryValue());
-							}
-							else
-							{
-								data = Compress(System.Text.Encoding.GetEncoding("UTF-8").GetBytes(field.StringValue()));
-							}
-						}
-						int len = data.Length;
-						fieldsStream.WriteVInt(len);
-						fieldsStream.WriteBytes(data, len);
-					}
-					else
-					{
-						// compression is disabled for the current field
-						if (field.IsBinary())
-						{
-							byte[] data = field.BinaryValue();
-							int len = data.Length;
-							fieldsStream.WriteVInt(len);
-							fieldsStream.WriteBytes(data, len);
-						}
-						else
-						{
-							fieldsStream.WriteString(field.StringValue());
-						}
-					}
-				}
+					WriteField(fieldInfos.FieldInfo(field.Name()), field);
 			}
 		}
 		

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FilterIndexReader.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/FilterIndexReader.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FilterIndexReader.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/FilterIndexReader.cs Tue Jun 24 19:52:22 2008
@@ -16,8 +16,10 @@
  */
 
 using System;
+
 using Document = Lucene.Net.Documents.Document;
 using FieldSelector = Lucene.Net.Documents.FieldSelector;
+using Directory = Lucene.Net.Store.Directory;
 
 namespace Lucene.Net.Index
 {
@@ -90,6 +92,22 @@
 			{
 				return ((TermPositions) this.in_Renamed).NextPosition();
 			}
+			
+			public virtual int GetPayloadLength()
+			{
+				return ((TermPositions) this.in_Renamed).GetPayloadLength();
+			}
+			
+			public virtual byte[] GetPayload(byte[] data, int offset)
+			{
+				return ((TermPositions) this.in_Renamed).GetPayload(data, offset);
+			}
+			
+			
+			public virtual bool IsPayloadAvailable()
+			{
+				return ((TermPositions) this.in_Renamed).IsPayloadAvailable();
+			}
 		}
 		
 		/// <summary>Base class for filtering {@link TermEnum} implementations. </summary>
@@ -129,43 +147,71 @@
 		/// </summary>
 		/// <param name="in">specified base reader.
 		/// </param>
-		public FilterIndexReader(IndexReader in_Renamed):base(in_Renamed.Directory())
+		public FilterIndexReader(IndexReader in_Renamed) : base()
 		{
 			this.in_Renamed = in_Renamed;
 		}
 		
+		public override Directory Directory()
+		{
+			return in_Renamed.Directory();
+		}
+		
 		public override TermFreqVector[] GetTermFreqVectors(int docNumber)
 		{
+			EnsureOpen();
 			return in_Renamed.GetTermFreqVectors(docNumber);
 		}
 		
 		public override TermFreqVector GetTermFreqVector(int docNumber, System.String field)
 		{
+			EnsureOpen();
 			return in_Renamed.GetTermFreqVector(docNumber, field);
 		}
 		
+		
+		public override void  GetTermFreqVector(int docNumber, System.String field, TermVectorMapper mapper)
+		{
+			EnsureOpen();
+			in_Renamed.GetTermFreqVector(docNumber, field, mapper);
+		}
+		
+		public override void  GetTermFreqVector(int docNumber, TermVectorMapper mapper)
+		{
+			EnsureOpen();
+			in_Renamed.GetTermFreqVector(docNumber, mapper);
+		}
+		
 		public override int NumDocs()
 		{
+			// Don't call ensureOpen() here (it could affect performance)
 			return in_Renamed.NumDocs();
 		}
+
 		public override int MaxDoc()
 		{
+			// Don't call ensureOpen() here (it could affect performance)
 			return in_Renamed.MaxDoc();
 		}
 		
 		public override Document Document(int n, FieldSelector fieldSelector)
 		{
+			EnsureOpen();
 			return in_Renamed.Document(n, fieldSelector);
 		}
 		
 		public override bool IsDeleted(int n)
 		{
+			// Don't call ensureOpen() here (it could affect performance)
 			return in_Renamed.IsDeleted(n);
 		}
+
 		public override bool HasDeletions()
 		{
+			// Don't call ensureOpen() here (it could affect performance)
 			return in_Renamed.HasDeletions();
 		}
+
 		protected internal override void  DoUndeleteAll()
 		{
 			in_Renamed.UndeleteAll();
@@ -173,17 +219,22 @@
 		
 		public override bool HasNorms(System.String field)
 		{
+			EnsureOpen();
 			return in_Renamed.HasNorms(field);
 		}
 		
 		public override byte[] Norms(System.String f)
 		{
+			EnsureOpen();
 			return in_Renamed.Norms(f);
 		}
+
 		public override void  Norms(System.String f, byte[] bytes, int offset)
 		{
+			EnsureOpen();
 			in_Renamed.Norms(f, bytes, offset);
 		}
+
 		protected internal override void  DoSetNorm(int d, System.String f, byte b)
 		{
 			in_Renamed.SetNorm(d, f, b);
@@ -191,25 +242,31 @@
 		
 		public override TermEnum Terms()
 		{
+			EnsureOpen();
 			return in_Renamed.Terms();
 		}
+
 		public override TermEnum Terms(Term t)
 		{
+			EnsureOpen();
 			return in_Renamed.Terms(t);
 		}
 		
 		public override int DocFreq(Term t)
 		{
+			EnsureOpen();
 			return in_Renamed.DocFreq(t);
 		}
 		
 		public override TermDocs TermDocs()
 		{
+			EnsureOpen();
 			return in_Renamed.TermDocs();
 		}
 		
 		public override TermPositions TermPositions()
 		{
+			EnsureOpen();
 			return in_Renamed.TermPositions();
 		}
 		
@@ -229,17 +286,26 @@
 		
 		public override System.Collections.ICollection GetFieldNames(IndexReader.FieldOption fieldNames)
 		{
+			EnsureOpen();
 			return in_Renamed.GetFieldNames(fieldNames);
 		}
 		
 		public override long GetVersion()
 		{
+			EnsureOpen();
 			return in_Renamed.GetVersion();
 		}
 
 		public override bool IsCurrent()
 		{
+			EnsureOpen();
 			return in_Renamed.IsCurrent();
 		}
+		
+		public override bool IsOptimized()
+		{
+			EnsureOpen();
+			return in_Renamed.IsOptimized();
+		}
 	}
 }
\ No newline at end of file

Added: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexCommitPoint.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/IndexCommitPoint.cs?rev=671404&view=auto
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexCommitPoint.cs (added)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexCommitPoint.cs Tue Jun 24 19:52:22 2008
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/// <summary> <p>Expert: represents a single commit into an index as seen by the
+/// {@link IndexDeletionPolicy}. 
+/// <p>
+/// Changes to the content of an index are made visible only
+/// after the writer who made that change had written to the
+/// directory a new segments file (<code>segments_N</code>). This point in 
+/// time, when the action of writing of a new segments file to the
+/// directory is completed, is therefore an index commit point.
+/// <p>
+/// Each index commit point has a unique segments file associated
+/// with it. The segments file associated with a later 
+/// index commit point would have a larger N.
+/// </summary>
+
+using System;
+
+namespace Lucene.Net.Index
+{
+	
+	public interface IndexCommitPoint
+	{
+		
+		/// <summary> Get the segments file (<code>segments_N</code>) associated 
+		/// with this commit point.
+		/// </summary>
+		System.String GetSegmentsFileName();
+		
+		/// <summary> Returns all index files referenced by this commit point.</summary>
+		System.Collections.ICollection GetFileNames();
+		
+		/// <summary> Delete this commit point.
+		/// <p>
+		/// Upon calling this, the writer is notified that this commit 
+		/// point should be deleted. 
+		/// <p>
+		/// Decision that a commit-point should be deleted is taken by the {@link IndexDeletionPolicy} in effect
+		/// and therefore this should only be called by its {@link IndexDeletionPolicy#onInit onInit()} or 
+		/// {@link IndexDeletionPolicy#onCommit onCommit()} methods.
+		/// </summary>
+		void  Delete();
+	}
+}
\ No newline at end of file

Added: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexDeletionPolicy.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/IndexDeletionPolicy.cs?rev=671404&view=auto
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexDeletionPolicy.cs (added)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexDeletionPolicy.cs Tue Jun 24 19:52:22 2008
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+using System;
+
+namespace Lucene.Net.Index
+{
+	
+	/// <summary> <p>Expert: policy for deletion of stale {@link IndexCommitPoint index commits}. 
+	/// 
+	/// <p>Implement this interface, and pass it to one
+	/// of the {@link IndexWriter} or {@link IndexReader}
+	/// constructors, to customize when older
+	/// {@link IndexCommitPoint point-in-time commits}
+	/// are deleted from the index directory.  The default deletion policy
+	/// is {@link KeepOnlyLastCommitDeletionPolicy}, which always
+	/// removes old commits as soon as a new commit is done (this
+	/// matches the behavior before 2.2).</p>
+	/// 
+	/// <p>One expected use case for this (and the reason why it
+	/// was first created) is to work around problems with an
+	/// index directory accessed via filesystems like NFS because
+	/// NFS does not provide the "delete on last close" semantics
+	/// that Lucene's "point in time" search normally relies on.
+	/// By implementing a custom deletion policy, such as "a
+	/// commit is only removed once it has been stale for more
+	/// than X minutes", you can give your readers time to
+	/// refresh to the new commit before {@link IndexWriter}
+	/// removes the old commits.  Note that doing so will
+	/// increase the storage requirements of the index.  See <a
+	/// target="top"
+	/// href="http://issues.apache.org/jira/browse/LUCENE-710">LUCENE-710</a>
+	/// for details.</p>
+	/// </summary>
+	
+	public interface IndexDeletionPolicy
+	{
+		
+		/// <summary> <p>This is called once when a writer is first
+		/// instantiated to give the policy a chance to remove old
+		/// commit points.</p>
+		/// 
+		/// <p>The writer locates all index commits present in the 
+		/// index directory and calls this method.  The policy may 
+		/// choose to delete some of the commit points, doing so by
+		/// calling method {@link IndexCommitPoint#delete delete()} 
+		/// of {@link IndexCommitPoint}.</p>
+		/// 
+		/// <p><u>Note:</u> the last CommitPoint is the most recent one,
+		/// i.e. the "front index state". Be careful not to delete it,
+		/// unless you know for sure what you are doing, and unless 
+		/// you can afford to lose the index content while doing that. 
+		/// 
+		/// </summary>
+		/// <param name="commits">List of current 
+		/// {@link IndexCommitPoint point-in-time commits},
+		/// sorted by age (the 0th one is the oldest commit).
+		/// </param>
+		void  OnInit(System.Collections.IList commits);
+		
+		/// <summary> <p>This is called each time the writer completed a commit.
+		/// This gives the policy a chance to remove old commit points
+		/// with each commit.</p>
+		/// 
+		/// <p>The policy may now choose to delete old commit points 
+		/// by calling method {@link IndexCommitPoint#delete delete()} 
+		/// of {@link IndexCommitPoint}.</p>
+		/// 
+		/// <p>If writer has <code>autoCommit = true</code> then
+		/// this method will in general be called many times during
+		/// one instance of {@link IndexWriter}.  If
+		/// <code>autoCommit = false</code> then this method is
+		/// only called once when {@link IndexWriter#close} is
+		/// called, or not at all if the {@link IndexWriter#abort}
+		/// is called. 
+		/// 
+		/// <p><u>Note:</u> the last CommitPoint is the most recent one,
+		/// i.e. the "front index state". Be careful not to delete it,
+		/// unless you know for sure what you are doing, and unless 
+		/// you can afford to lose the index content while doing that.
+		/// 
+		/// </summary>
+		/// <param name="commits">List of {@link IndexCommitPoint},
+		/// sorted by age (the 0th one is the oldest commit).
+		/// </param>
+		void  OnCommit(System.Collections.IList commits);
+	}
+}
\ No newline at end of file

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexFileDeleter.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/IndexFileDeleter.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexFileDeleter.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexFileDeleter.cs Tue Jun 24 19:52:22 2008
@@ -16,343 +16,708 @@
  */
 
 using System;
+
 using Directory = Lucene.Net.Store.Directory;
-using Pattern = System.Text.RegularExpressions.Regex;
 
 namespace Lucene.Net.Index
 {
 	
-	/// <summary> A utility class (used by both IndexReader and
-	/// IndexWriter) to keep track of files that need to be
-	/// deleted because they are no longer referenced by the
-	/// index.
-	/// </summary>
+	/*
+	* This class keeps track of each SegmentInfos instance that
+	* is still "live", either because it corresponds to a 
+	* segments_N file in the Directory (a "commit", i.e. a 
+	* committed SegmentInfos) or because it's the in-memory SegmentInfos 
+	* that a writer is actively updating but has not yet committed 
+	* (currently this only applies when autoCommit=false in IndexWriter).
+	* This class uses simple reference counting to map the live
+	* SegmentInfos instances to individual files in the Directory. 
+	* 
+	* The same directory file may be referenced by more than
+	* one IndexCommitPoints, i.e. more than one SegmentInfos.
+	* Therefore we count how many commits reference each file.
+	* When all the commits referencing a certain file have been
+	* deleted, the refcount for that file becomes zero, and the
+	* file is deleted.
+	*
+	* A separate deletion policy interface
+	* (IndexDeletionPolicy) is consulted on creation (onInit)
+	* and once per commit (onCommit), to decide when a commit
+	* should be removed.
+	* 
+	* It is the business of the IndexDeletionPolicy to choose
+	* when to delete commit points.  The actual mechanics of
+	* file deletion, retrying, etc, derived from the deletion
+	* of commit points is the business of the IndexFileDeleter.
+	* 
+	* The current default deletion policy is {@link
+	* KeepOnlyLastCommitDeletionPolicy}, which removes all
+	* prior commits when a new commit has completed.  This
+	* matches the behavior before 2.2.
+	*
+	* Note that you must hold the write.lock before
+	* instantiating this class.  It opens segments_N file(s)
+	* directly with no retry logic.
+	*/
+	
 	sealed public class IndexFileDeleter
 	{
-		private System.Collections.ArrayList deletable;
-		private System.Collections.Hashtable pending;
+		
+		/* Files that we tried to delete but failed (likely
+		* because they are open and we are running on Windows),
+		* so we will retry them again later: */
+		private System.Collections.IList deletable;
+		
+		/* Reference count for all files in the index.  
+		* Counts how many existing commits reference a file.
+		* Maps String to RefCount (class below) instances: */
+		private System.Collections.IDictionary refCounts = new System.Collections.Hashtable();
+		
+		/* Holds all commits (segments_N) currently in the index.
+		* This will have just 1 commit if you are using the
+		* default delete policy (KeepOnlyLastCommitDeletionPolicy).
+		* Other policies may leave commit points live for longer
+		* in which case this list would be longer than 1: */
+		private System.Collections.ArrayList commits = new System.Collections.ArrayList();
+		
+		/* Holds files we had incref'd from the previous
+		* non-commit checkpoint: */
+		private System.Collections.IList lastFiles = new System.Collections.ArrayList();
+		
+		/* Commits that the IndexDeletionPolicy have decided to delete: */
+		private System.Collections.IList commitsToDelete = new System.Collections.ArrayList();
+
+        private System.IO.TextWriter infoStream;
 		private Directory directory;
-		private SegmentInfos segmentInfos;
-		private System.IO.TextWriter infoStream;
+		private IndexDeletionPolicy policy;
+		private DocumentsWriter docWriter;
 		
-		public IndexFileDeleter(SegmentInfos segmentInfos, Directory directory)
-		{
-			this.segmentInfos = segmentInfos;
-			this.directory = directory;
-		}
-		internal void  SetSegmentInfos(SegmentInfos segmentInfos)
-		{
-			this.segmentInfos = segmentInfos;
-		}
-		internal SegmentInfos GetSegmentInfos()
-		{
-			return segmentInfos;
-		}
+		/// <summary>Change to true to see details of reference counts when
+		/// infoStream != null 
+		/// </summary>
+		public static bool VERBOSE_REF_COUNTS = false;
 		
 		internal void  SetInfoStream(System.IO.TextWriter infoStream)
 		{
 			this.infoStream = infoStream;
+			if (infoStream != null)
+			{
+				Message("setInfoStream deletionPolicy=" + policy);
+			}
 		}
 		
-		/// <summary>Determine index files that are no longer referenced
-		/// and therefore should be deleted.  This is called once
-		/// (by the writer), and then subsequently we add onto
-		/// deletable any files that are no longer needed at the
-		/// point that we create the unused file (eg when merging
-		/// segments), and we only remove from deletable when a
-		/// file is successfully deleted.
-		/// </summary>
+		private void  Message(System.String message)
+		{
+			infoStream.WriteLine("IFD [" + SupportClass.ThreadClass.Current().Name + "]: " + message);
+		}
 		
-		public void  FindDeletableFiles()
+		/// <summary> Initialize the deleter: find all previous commits in
+		/// the Directory, incref the files they reference, call
+		/// the policy to let it delete commits.  The incoming
+		/// segmentInfos must have been loaded from a commit point
+		/// and not yet modified.  This will remove any files not
+		/// referenced by any of the commits.
+		/// </summary>
+		/// <throws>  CorruptIndexException if the index is corrupt </throws>
+		/// <throws>  IOException if there is a low-level IO error </throws>
+		public IndexFileDeleter(Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos, System.IO.TextWriter infoStream, DocumentsWriter docWriter)
 		{
 			
-			// Gather all "current" segments:
-			System.Collections.Hashtable current = new System.Collections.Hashtable();
-			for (int j = 0; j < segmentInfos.Count; j++)
+			this.docWriter = docWriter;
+			this.infoStream = infoStream;
+			
+			if (infoStream != null)
 			{
-				SegmentInfo segmentInfo = (SegmentInfo) segmentInfos[j];
-				current[segmentInfo.name] = segmentInfo;
+				Message("init: current segments file is \"" + segmentInfos.GetCurrentSegmentFileName() + "\"; deletionPolicy=" + policy);
 			}
 			
-			// Then go through all files in the Directory that are
-			// Lucene index files, and add to deletable if they are
-			// not referenced by the current segments info:
+			this.policy = policy;
+			this.directory = directory;
 			
-			System.String segmentsInfosFileName = segmentInfos.GetCurrentSegmentFileName();
+			// First pass: walk the files and initialize our ref
+			// counts:
+			long currentGen = segmentInfos.GetGeneration();
 			IndexFileNameFilter filter = IndexFileNameFilter.GetFilter();
 			
 			System.String[] files = directory.List();
+			if (files == null)
+			{
+				throw new System.IO.IOException("cannot read directory " + directory + ": list() returned null");
+			}
+			
+			CommitPoint currentCommitPoint = null;
 			
 			for (int i = 0; i < files.Length; i++)
 			{
 				
-				if (filter.Accept(null, files[i]) && !files[i].Equals(segmentsInfosFileName) && !files[i].Equals(IndexFileNames.SEGMENTS_GEN))
+				System.String fileName = files[i];
+				
+				if (filter.Accept(null, fileName) && !fileName.Equals(IndexFileNames.SEGMENTS_GEN))
 				{
 					
-					System.String segmentName;
-					System.String extension;
-					
-					// First remove any extension:
-					int loc = files[i].IndexOf((System.Char) '.');
-					if (loc != - 1)
-					{
-						extension = files[i].Substring(1 + loc);
-						segmentName = files[i].Substring(0, (loc) - (0));
-					}
-					else
-					{
-						extension = null;
-						segmentName = files[i];
-					}
-					
-					// Then, remove any generation count:
-					loc = segmentName.IndexOf((System.Char) '_', 1);
-					if (loc != - 1)
-					{
-						segmentName = segmentName.Substring(0, (loc) - (0));
-					}
+					// Add this file to refCounts with initial count 0:
+					GetRefCount(fileName);
 					
-					// Delete this file if it's not a "current" segment,
-					// or, it is a single index file but there is now a
-					// corresponding compound file:
-					bool doDelete = false;
-					
-					if (!current.ContainsKey(segmentName))
-					{
-						// Delete if segment is not referenced:
-						doDelete = true;
-					}
-					else
+					if (fileName.StartsWith(IndexFileNames.SEGMENTS))
 					{
-						// OK, segment is referenced, but file may still
-						// be orphan'd:
-						SegmentInfo info = (SegmentInfo) current[segmentName];
 						
-						if (filter.IsCFSFile(files[i]) && info.GetUseCompoundFile())
+						// This is a commit (segments or segments_N), and
+						// it's valid (<= the max gen).  Load it, then
+						// incref all files it refers to:
+						if (SegmentInfos.GenerationFromSegmentsFileName(fileName) <= currentGen)
 						{
-							// This file is in fact stored in a CFS file for
-							// this segment:
-							doDelete = true;
-						}
-						else
-						{
-                            Pattern p = new System.Text.RegularExpressions.Regex("s\\d+");
-							
-							if ("del".Equals(extension))
+							if (infoStream != null)
 							{
-								// This is a _segmentName_N.del file:
-								if (!files[i].Equals(info.GetDelFileName()))
-								{
-									// If this is a seperate .del file, but it
-									// doesn't match the current del filename for
-									// this segment, then delete it:
-									doDelete = true;
-								}
+								Message("init: load commit \"" + fileName + "\"");
 							}
-							else if (extension != null && extension.StartsWith("s") && p.Match(extension).Success)
+							SegmentInfos sis = new SegmentInfos();
+							try
 							{
-
-								int field = System.Int32.Parse(extension.Substring(1));
-								// This is a _segmentName_N.sX file:
-								if (!files[i].Equals(info.GetNormFileName(field)))
+								sis.Read(directory, fileName);
+							}
+							catch (System.IO.FileNotFoundException e)
+							{
+								// LUCENE-948: on NFS (and maybe others), if
+								// you have writers switching back and forth
+								// between machines, it's very likely that the
+								// dir listing will be stale and will claim a
+								// file segments_X exists when in fact it
+								// doesn't.  So, we catch this and handle it
+								// as if the file does not exist
+								if (infoStream != null)
 								{
-									// This is an orphan'd separate norms file:
-									doDelete = true;
+									Message("init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
 								}
+								sis = null;
 							}
-							else if ("cfs".Equals(extension) && !info.GetUseCompoundFile())
+							if (sis != null)
 							{
-								// This is a partially written
-								// _segmentName.cfs:
-								doDelete = true;
+								CommitPoint commitPoint = new CommitPoint(this, sis);
+								if (sis.GetGeneration() == segmentInfos.GetGeneration())
+								{
+									currentCommitPoint = commitPoint;
+								}
+								commits.Add(commitPoint);
+								IncRef(sis, true);
 							}
 						}
 					}
-					
-					if (doDelete)
+				}
+			}
+			
+			if (currentCommitPoint == null)
+			{
+				// We did not in fact see the segments_N file
+				// corresponding to the segmentInfos that was passed
+				// in.  Yet, it must exist, because our caller holds
+				// the write lock.  This can happen when the directory
+				// listing was stale (eg when index accessed via NFS
+				// client with stale directory listing cache).  So we
+				// try now to explicitly open this commit point:
+				SegmentInfos sis = new SegmentInfos();
+				try
+				{
+					sis.Read(directory, segmentInfos.GetCurrentSegmentFileName());
+				}
+				catch (System.IO.IOException e)
+				{
+					throw new CorruptIndexException("failed to locate current segments_N file");
+				}
+				if (infoStream != null)
+					Message("forced open of current segments file " + segmentInfos.GetCurrentSegmentFileName());
+				currentCommitPoint = new CommitPoint(this, sis);
+				commits.Add(currentCommitPoint);
+				IncRef(sis, true);
+			}
+			
+			// We keep commits list in sorted order (oldest to newest):
+			commits.Sort();
+			
+			// Now delete anything with ref count at 0.  These are
+			// presumably abandoned files eg due to crash of
+			// IndexWriter.
+			System.Collections.IEnumerator it = refCounts.Keys.GetEnumerator();
+			while (it.MoveNext())
+			{
+				System.String fileName = (System.String) it.Current;
+				RefCount rc = (RefCount) refCounts[fileName];
+				if (0 == rc.count)
+				{
+					if (infoStream != null)
 					{
-						AddDeletableFile(files[i]);
-						if (infoStream != null)
-						{
-							infoStream.WriteLine("IndexFileDeleter: file \"" + files[i] + "\" is unreferenced in index and will be deleted on next commit");
-						}
+						Message("init: removing unreferenced file \"" + fileName + "\"");
 					}
+					DeleteFile(fileName);
 				}
 			}
-		}
-		
-		/*
-		* 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.
-		*/
-		
-		internal void  DeleteSegments(System.Collections.ArrayList segments)
-		{
 			
-			DeleteFiles(); // try to delete files that we couldn't before
+			// Finally, give policy a chance to remove things on
+			// startup:
+			policy.OnInit(commits);
 			
-			for (int i = 0; i < segments.Count; i++)
+			// It's OK for the onInit to remove the current commit
+			// point; we just have to checkpoint our in-memory
+			// SegmentInfos to protect those files that it uses:
+			if (currentCommitPoint.deleted)
 			{
-				SegmentReader reader = (SegmentReader) segments[i];
-				if (reader.Directory() == this.directory)
-					DeleteFiles(reader.Files());
-				// try to delete our files
-				else
-					DeleteFiles(reader.Files(), reader.Directory()); // delete other files
+				Checkpoint(segmentInfos, false);
 			}
+			
+			DeleteCommits();
 		}
 		
-		/// <summary> Delete these segments, as long as they are not listed
-		/// in protectedSegments.  If they are, then, instead, add
-		/// them to the pending set.
+		/// <summary> Remove the CommitPoints in the commitsToDelete List by
+		/// DecRef'ing all files from each SegmentInfos.
 		/// </summary>
-		
-		internal void  DeleteSegments(System.Collections.ArrayList segments, System.Collections.Hashtable protectedSegments)
+		private void  DeleteCommits()
 		{
 			
-			DeleteFiles(); // try to delete files that we couldn't before
+			int size = commitsToDelete.Count;
 			
-			for (int i = 0; i < segments.Count; i++)
+			if (size > 0)
 			{
-				SegmentReader reader = (SegmentReader) segments[i];
-				if (reader.Directory() == this.directory)
+				
+				// First decref all files that had been referred to by
+				// the now-deleted commits:
+				for (int i = 0; i < size; i++)
 				{
-					if (protectedSegments.Contains(reader.GetSegmentName()))
+					CommitPoint commit = (CommitPoint) commitsToDelete[i];
+					if (infoStream != null)
 					{
-						AddPendingFiles(reader.Files()); // record these for deletion on commit
+						Message("deleteCommits: now remove commit \"" + commit.GetSegmentsFileName() + "\"");
 					}
-					else
+					int size2 = commit.files.Count;
+					for (int j = 0; j < size2; j++)
 					{
-						DeleteFiles(reader.Files()); // try to delete our files
+						DecRef((System.String) commit.files[j]);
 					}
 				}
-				else
+				commitsToDelete.Clear();
+				
+				// Now compact commits to remove deleted ones (preserving the sort):
+				size = commits.Count;
+				int readFrom = 0;
+				int writeTo = 0;
+				while (readFrom < size)
+				{
+					CommitPoint commit = (CommitPoint) commits[readFrom];
+					if (!commit.deleted)
+					{
+						if (writeTo != readFrom)
+						{
+							commits[writeTo] = commits[readFrom];
+						}
+						writeTo++;
+					}
+					readFrom++;
+				}
+				
+				while (size > writeTo)
+				{
+					commits.RemoveAt(size - 1);
+					size--;
+				}
+			}
+		}
+		
+		/// <summary> Writer calls this when it has hit an error and had to
+		/// roll back, to tell us that there may now be
+		/// unreferenced files in the filesystem.  So we re-list
+		/// the filesystem and delete such files.  If segmentName
+		/// is non-null, we will only delete files corresponding to
+		/// that segment.
+		/// </summary>
+		public void  Refresh(System.String segmentName)
+		{
+			System.String[] files = directory.List();
+			if (files == null)
+			{
+				throw new System.IO.IOException("cannot read directory " + directory + ": list() returned null");
+			}
+			IndexFileNameFilter filter = IndexFileNameFilter.GetFilter();
+			System.String segmentPrefix1;
+			System.String segmentPrefix2;
+			if (segmentName != null)
+			{
+				segmentPrefix1 = segmentName + ".";
+				segmentPrefix2 = segmentName + "_";
+			}
+			else
+			{
+				segmentPrefix1 = null;
+				segmentPrefix2 = null;
+			}
+			
+			for (int i = 0; i < files.Length; i++)
+			{
+				System.String fileName = files[i];
+				if (filter.Accept(null, fileName) && (segmentName == null || fileName.StartsWith(segmentPrefix1) || fileName.StartsWith(segmentPrefix2)) && !refCounts.Contains(fileName) && !fileName.Equals(IndexFileNames.SEGMENTS_GEN))
 				{
-					DeleteFiles(reader.Files(), reader.Directory()); // delete other files
+					// Unreferenced file, so remove it
+					if (infoStream != null)
+					{
+						Message("refresh [prefix=" + segmentName + "]: removing newly created unreferenced file \"" + fileName + "\"");
+					}
+					DeleteFile(fileName);
 				}
 			}
 		}
 		
-		internal void  DeleteFiles(System.Collections.ArrayList files, Directory directory)
+		public void  Refresh()
 		{
-			for (int i = 0; i < files.Count; i++)
-				directory.DeleteFile((System.String) files[i]);
+			Refresh(null);
 		}
 		
-		internal void  DeleteFiles(System.Collections.ArrayList files)
+		public void  Close()
 		{
-			DeleteFiles(); // try to delete files that we couldn't before
-			for (int i = 0; i < files.Count; i++)
+			DeletePendingFiles();
+		}
+		
+		private void  DeletePendingFiles()
+		{
+			if (deletable != null)
 			{
-				DeleteFile((System.String) files[i]);
+				System.Collections.IList oldDeletable = deletable;
+				deletable = null;
+				int size = oldDeletable.Count;
+				for (int i = 0; i < size; i++)
+				{
+					if (infoStream != null)
+					{
+						Message("delete pending file " + oldDeletable[i]);
+					}
+					DeleteFile((System.String) oldDeletable[i]);
+				}
 			}
 		}
 		
-		internal void  DeleteFile(System.String file)
+		/// <summary> For definition of "check point" see IndexWriter comments:
+		/// "Clarification: Check Points (and commits)".
+		/// 
+		/// Writer calls this when it has made a "consistent
+		/// change" to the index, meaning new files are written to
+		/// the index and the in-memory SegmentInfos have been
+		/// modified to point to those files.
+		/// 
+		/// This may or may not be a commit (segments_N may or may
+		/// not have been written).
+		/// 
+		/// We simply incref the files referenced by the new
+		/// SegmentInfos and decref the files we had previously
+		/// seen (if any).
+		/// 
+		/// If this is a commit, we also call the policy to give it
+		/// a chance to remove other commits.  If any commits are
+		/// removed, we decref their files as well.
+		/// </summary>
+		public void  Checkpoint(SegmentInfos segmentInfos, bool isCommit)
 		{
-			try
+			
+			if (infoStream != null)
 			{
-				directory.DeleteFile(file); // try to delete each file
+				Message("now checkpoint \"" + segmentInfos.GetCurrentSegmentFileName() + "\" [" + segmentInfos.Count + " segments " + "; isCommit = " + isCommit + "]");
 			}
-			catch (System.IO.IOException e)
+			
+			// Try again now to delete any previously un-deletable
+			// files (because they were in use, on Windows):
+			DeletePendingFiles();
+			
+			// Incref the files:
+			IncRef(segmentInfos, isCommit);
+			System.Collections.IList docWriterFiles;
+			if (docWriter != null)
+			{
+				docWriterFiles = docWriter.Files();
+				if (docWriterFiles != null)
+					IncRef(docWriterFiles);
+			}
+			else
+				docWriterFiles = null;
+			
+			if (isCommit)
 			{
-				// if delete fails
-				if (directory.FileExists(file))
+				// Append to our commits list:
+				commits.Add(new CommitPoint(this, segmentInfos));
+				
+				// Tell policy so it can remove commits:
+				policy.OnCommit(commits);
+				
+				// Decref files for commits that were deleted by the policy:
+				DeleteCommits();
+			}
+			
+			// DecRef old files from the last checkpoint, if any:
+			int size = lastFiles.Count;
+			if (size > 0)
+			{
+				for (int i = 0; i < size; i++)
+					DecRef((System.Collections.IList) lastFiles[i]);
+				lastFiles.Clear();
+			}
+			
+			if (!isCommit)
+			{
+				// Save files so we can decr on next checkpoint/commit:
+				size = segmentInfos.Count;
+				for (int i = 0; i < size; i++)
 				{
-					if (infoStream != null)
+					SegmentInfo segmentInfo = segmentInfos.Info(i);
+					if (segmentInfo.dir == directory)
 					{
-						infoStream.WriteLine("IndexFileDeleter: unable to remove file \"" + file + "\": " + e.ToString() + "; Will re-try later.");
+						lastFiles.Add(segmentInfo.Files());
 					}
-					AddDeletableFile(file); // add to deletable
 				}
 			}
+			if (docWriterFiles != null)
+				lastFiles.Add(docWriterFiles);
 		}
 		
-		internal void  ClearPendingFiles()
+		internal void  IncRef(SegmentInfos segmentInfos, bool isCommit)
 		{
-			pending = null;
+			int size = segmentInfos.Count;
+			for (int i = 0; i < size; i++)
+			{
+				SegmentInfo segmentInfo = segmentInfos.Info(i);
+				if (segmentInfo.dir == directory)
+				{
+					IncRef(segmentInfo.Files());
+				}
+			}
+			
+			if (isCommit)
+			{
+				// Since this is a commit point, also incref its
+				// segments_N file:
+				GetRefCount(segmentInfos.GetCurrentSegmentFileName()).IncRef();
+			}
 		}
 		
-		/*
-		Record that the files for these segments should be
-		deleted, once the pending deletes are committed.
-		*/
-		internal void  AddPendingSegments(System.Collections.ArrayList segments)
+		internal void  IncRef(System.Collections.IList files)
 		{
-			for (int i = 0; i < segments.Count; i++)
+			int size = files.Count;
+			for (int i = 0; i < size; i++)
 			{
-				SegmentReader reader = (SegmentReader) segments[i];
-				if (reader.Directory() == this.directory)
+				System.String fileName = (System.String) files[i];
+				RefCount rc = GetRefCount(fileName);
+				if (infoStream != null && VERBOSE_REF_COUNTS)
 				{
-					AddPendingFiles(reader.Files());
+					Message("  IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
 				}
+				rc.IncRef();
 			}
 		}
 		
-		/*
-		Record list of files for deletion, but do not delete
-		them until commitPendingFiles is called.
-		*/
-		internal void  AddPendingFiles(System.Collections.ArrayList files)
+		internal void  DecRef(System.Collections.IList files)
 		{
-			for (int i = 0; i < files.Count; i++)
+			int size = files.Count;
+			for (int i = 0; i < size; i++)
 			{
-				AddPendingFile((System.String) files[i]);
+				DecRef((System.String) files[i]);
 			}
 		}
 		
-		/*
-		Record a file for deletion, but do not delete it until
-		commitPendingFiles is called.
-		*/
-		internal void  AddPendingFile(System.String fileName)
+		private void  DecRef(System.String fileName)
 		{
-			if (pending == null)
+			RefCount rc = GetRefCount(fileName);
+			if (infoStream != null && VERBOSE_REF_COUNTS)
+			{
+				Message("  DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
+			}
+			if (0 == rc.DecRef())
 			{
-				pending = new System.Collections.Hashtable();
+				// This file is no longer referenced by any past
+				// commit points nor by the in-memory SegmentInfos:
+				DeleteFile(fileName);
+				refCounts.Remove(fileName);
 			}
-            if (pending.ContainsKey(fileName) == false)
-            {
-                pending.Add(fileName, fileName);
-            }
 		}
 		
-		internal void  CommitPendingFiles()
+		internal void  DecRef(SegmentInfos segmentInfos)
 		{
-			if (pending != null)
+			int size = segmentInfos.Count;
+			for (int i = 0; i < size; i++)
 			{
-				if (deletable == null)
+				SegmentInfo segmentInfo = segmentInfos.Info(i);
+				if (segmentInfo.dir == directory)
 				{
-					deletable = System.Collections.ArrayList.Synchronized(new System.Collections.ArrayList(10));
+					DecRef(segmentInfo.Files());
 				}
-				System.Collections.IEnumerator it = pending.GetEnumerator();
-				while (it.MoveNext())
+			}
+		}
+		
+		private RefCount GetRefCount(System.String fileName)
+		{
+			RefCount rc;
+			if (!refCounts.Contains(fileName))
+			{
+				rc = new RefCount();
+				refCounts[fileName] = rc;
+			}
+			else
+			{
+				rc = (RefCount) refCounts[fileName];
+			}
+			return rc;
+		}
+		
+		internal void  DeleteFiles(System.Collections.IList files)
+		{
+			int size = files.Count;
+			for (int i = 0; i < size; i++)
+				DeleteFile((System.String) files[i]);
+		}
+		
+		/// <summary>Delets the specified files, but only if they are new
+		/// (have not yet been incref'd). 
+		/// </summary>
+		internal void  DeleteNewFiles(System.Collections.IList files)
+		{
+			int size = files.Count;
+			for (int i = 0; i < size; i++)
+				if (!refCounts.Contains(files[i]))
+					DeleteFile((System.String) files[i]);
+		}
+		
+		internal void  DeleteFile(System.String fileName)
+		{
+			try
+			{
+				if (infoStream != null)
 				{
-					deletable.Add(((System.Collections.DictionaryEntry)(it.Current)).Value);
+					Message("delete \"" + fileName + "\"");
+				}
+				directory.DeleteFile(fileName);
+			}
+			catch (System.IO.IOException e)
+			{
+				// if delete fails
+				if (directory.FileExists(fileName))
+				{
+					
+					// 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.
+					
+					if (infoStream != null)
+					{
+						Message("IndexFileDeleter: unable to remove file \"" + fileName + "\": " + e.ToString() + "; Will re-try later.");
+					}
+					if (deletable == null)
+					{
+						deletable = new System.Collections.ArrayList();
+					}
+					deletable.Add(fileName); // add to deletable
 				}
-				pending = null;
-				DeleteFiles();
 			}
 		}
 		
-		internal void  AddDeletableFile(System.String fileName)
+		/// <summary> Tracks the reference count for a single index file:</summary>
+		sealed private class RefCount
 		{
-			if (deletable == null)
+			
+			internal int count;
+			
+			public int IncRef()
 			{
-				deletable = System.Collections.ArrayList.Synchronized(new System.Collections.ArrayList(10));
+				return ++count;
+			}
+			
+			public int DecRef()
+			{
+				System.Diagnostics.Debug.Assert(count > 0);
+				return --count;
 			}
-			deletable.Add(fileName);
 		}
 		
-		public void  DeleteFiles()
+		/// <summary> Holds details for each commit point.  This class is
+		/// also passed to the deletion policy.  Note: this class
+		/// has a natural ordering that is inconsistent with
+		/// equals.
+		/// </summary>
+		
+		sealed private class CommitPoint : System.IComparable, IndexCommitPoint
 		{
-			if (deletable != null)
+			private void  InitBlock(IndexFileDeleter enclosingInstance)
 			{
-				System.Collections.ArrayList oldDeletable = deletable;
-				deletable = null;
-				DeleteFiles(oldDeletable); // try to delete deletable
+				this.enclosingInstance = enclosingInstance;
+			}
+			private IndexFileDeleter enclosingInstance;
+			public IndexFileDeleter Enclosing_Instance
+			{
+				get
+				{
+					return enclosingInstance;
+				}
+				
+			}
+			
+			internal long gen;
+			internal System.Collections.IList files;
+			internal System.String segmentsFileName;
+			internal bool deleted;
+			
+			public CommitPoint(IndexFileDeleter enclosingInstance, SegmentInfos segmentInfos)
+			{
+				InitBlock(enclosingInstance);
+				segmentsFileName = segmentInfos.GetCurrentSegmentFileName();
+				int size = segmentInfos.Count;
+				files = new System.Collections.ArrayList(size);
+				files.Add(segmentsFileName);
+				gen = segmentInfos.GetGeneration();
+				for (int i = 0; i < size; i++)
+				{
+					SegmentInfo segmentInfo = segmentInfos.Info(i);
+					if (segmentInfo.dir == Enclosing_Instance.directory)
+					{
+                        System.Collections.IEnumerator filesEnum = segmentInfo.Files().GetEnumerator();
+                        while (filesEnum.MoveNext())
+                        {
+                            files.Add(filesEnum.Current);
+                        }
+					}
+				}
+			}
+			
+			/// <summary> Get the segments_N file for this commit point.</summary>
+			public System.String GetSegmentsFileName()
+			{
+				return segmentsFileName;
+			}
+			
+			public System.Collections.ICollection GetFileNames()
+			{
+				return System.Collections.ArrayList.ReadOnly(new System.Collections.ArrayList(files));
+			}
+			
+			/// <summary> Called only be the deletion policy, to remove this
+			/// commit point from the index.
+			/// </summary>
+			public void  Delete()
+			{
+				if (!deleted)
+				{
+					deleted = true;
+					Enclosing_Instance.commitsToDelete.Add(this);
+				}
+			}
+			
+			public int CompareTo(System.Object obj)
+			{
+				CommitPoint commit = (CommitPoint) obj;
+				if (gen < commit.gen)
+				{
+					return - 1;
+				}
+				else if (gen > commit.gen)
+				{
+					return 1;
+				}
+				else
+				{
+					return 0;
+				}
 			}
 		}
 	}

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexFileNameFilter.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/IndexFileNameFilter.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexFileNameFilter.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexFileNameFilter.cs Tue Jun 24 19:52:22 2008
@@ -34,6 +34,7 @@
 		
 		internal static IndexFileNameFilter singleton = new IndexFileNameFilter();
 		private System.Collections.Hashtable extensions;
+		private System.Collections.Hashtable extensionsInCFS;
 		
 		public IndexFileNameFilter()
 		{
@@ -42,6 +43,11 @@
 			{
 				extensions.Add(IndexFileNames.INDEX_EXTENSIONS[i], IndexFileNames.INDEX_EXTENSIONS[i]);
 			}
+			extensionsInCFS = new System.Collections.Hashtable();
+			for (int i = 0; i < IndexFileNames.INDEX_EXTENSIONS_IN_COMPOUND_FILE.Length; i++)
+			{
+                extensionsInCFS.Add(IndexFileNames.INDEX_EXTENSIONS_IN_COMPOUND_FILE[i], IndexFileNames.INDEX_EXTENSIONS_IN_COMPOUND_FILE[i]);
+			}
 		}
 		
 		/* (non-Javadoc)
@@ -87,7 +93,7 @@
 			if (i != - 1)
 			{
 				System.String extension = name.Substring(1 + i);
-				if (extensions.Contains(extension) && !extension.Equals("del") && !extension.Equals("gen") && !extension.Equals("cfs"))
+				if (extensions.Contains(extension))
 				{
 					return true;
 				}

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexFileNames.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/IndexFileNames.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexFileNames.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/IndexFileNames.cs Tue Jun 24 19:52:22 2008
@@ -27,22 +27,70 @@
 	/// </author>
 	/// <version>  $rcs = ' $Id: Exp $ ' ;
 	/// </version>
-	sealed public class IndexFileNames
+	sealed class IndexFileNames
 	{
 		
 		/// <summary>Name of the index segment file </summary>
-		public const System.String SEGMENTS = "segments";
+		internal const System.String SEGMENTS = "segments";
 		
 		/// <summary>Name of the generation reference file name </summary>
-		public const System.String SEGMENTS_GEN = "segments.gen";
+		internal const System.String SEGMENTS_GEN = "segments.gen";
 		
 		/// <summary>Name of the index deletable file (only used in
 		/// pre-lockless indices) 
 		/// </summary>
-		public const System.String DELETABLE = "deletable";
+		internal const System.String DELETABLE = "deletable";
 		
 		/// <summary>Extension of norms file </summary>
-		public const System.String NORMS_EXTENSION = "nrm";
+		internal const System.String NORMS_EXTENSION = "nrm";
+		
+		/// <summary>Extension of freq postings file </summary>
+		internal const System.String FREQ_EXTENSION = "frq";
+		
+		/// <summary>Extension of prox postings file </summary>
+		internal const System.String PROX_EXTENSION = "prx";
+		
+		/// <summary>Extension of terms file </summary>
+		internal const System.String TERMS_EXTENSION = "tis";
+		
+		/// <summary>Extension of terms index file </summary>
+		internal const System.String TERMS_INDEX_EXTENSION = "tii";
+		
+		/// <summary>Extension of stored fields index file </summary>
+		internal const System.String FIELDS_INDEX_EXTENSION = "fdx";
+		
+		/// <summary>Extension of stored fields file </summary>
+		internal const System.String FIELDS_EXTENSION = "fdt";
+		
+		/// <summary>Extension of vectors fields file </summary>
+		internal const System.String VECTORS_FIELDS_EXTENSION = "tvf";
+		
+		/// <summary>Extension of vectors documents file </summary>
+		internal const System.String VECTORS_DOCUMENTS_EXTENSION = "tvd";
+		
+		/// <summary>Extension of vectors index file </summary>
+		internal const System.String VECTORS_INDEX_EXTENSION = "tvx";
+		
+		/// <summary>Extension of compound file </summary>
+		internal const System.String COMPOUND_FILE_EXTENSION = "cfs";
+		
+		/// <summary>Extension of compound file for doc store files</summary>
+		internal const System.String COMPOUND_FILE_STORE_EXTENSION = "cfx";
+		
+		/// <summary>Extension of deletes </summary>
+		internal const System.String DELETES_EXTENSION = "del";
+		
+		/// <summary>Extension of field infos </summary>
+		internal const System.String FIELD_INFOS_EXTENSION = "fnm";
+		
+		/// <summary>Extension of plain norms </summary>
+		internal const System.String PLAIN_NORMS_EXTENSION = "f";
+		
+		/// <summary>Extension of separate norms </summary>
+		internal const System.String SEPARATE_NORMS_EXTENSION = "s";
+		
+		/// <summary>Extension of gen file </summary>
+		internal const System.String GEN_EXTENSION = "gen";
 		
 		/// <summary> This array contains all filename extensions used by
 		/// Lucene's index files, with two exceptions, namely the
@@ -51,13 +99,22 @@
 		/// Lucene's <code>segments_N</code> files do not have any
 		/// filename extension.
 		/// </summary>
-		public static readonly System.String[] INDEX_EXTENSIONS = new System.String[]{"cfs", "fnm", "fdx", "fdt", "tii", "tis", "frq", "prx", "del", "tvx", "tvd", "tvf", "gen", "nrm"};
+		internal static readonly System.String[] INDEX_EXTENSIONS = new System.String[]{COMPOUND_FILE_EXTENSION, FIELD_INFOS_EXTENSION, FIELDS_INDEX_EXTENSION, FIELDS_EXTENSION, TERMS_INDEX_EXTENSION, TERMS_EXTENSION, FREQ_EXTENSION, PROX_EXTENSION, DELETES_EXTENSION, VECTORS_INDEX_EXTENSION, VECTORS_DOCUMENTS_EXTENSION, VECTORS_FIELDS_EXTENSION, GEN_EXTENSION, NORMS_EXTENSION, COMPOUND_FILE_STORE_EXTENSION};
+		
+		/// <summary>File extensions that are added to a compound file
+		/// (same as above, minus "del", "gen", "cfs"). 
+		/// </summary>
+		internal static readonly System.String[] INDEX_EXTENSIONS_IN_COMPOUND_FILE = new System.String[]{FIELD_INFOS_EXTENSION, FIELDS_INDEX_EXTENSION, FIELDS_EXTENSION, TERMS_INDEX_EXTENSION, TERMS_EXTENSION, FREQ_EXTENSION, PROX_EXTENSION, VECTORS_INDEX_EXTENSION, VECTORS_DOCUMENTS_EXTENSION, VECTORS_FIELDS_EXTENSION, NORMS_EXTENSION};
+		
+		internal static readonly System.String[] STORE_INDEX_EXTENSIONS = new System.String[]{VECTORS_INDEX_EXTENSION, VECTORS_FIELDS_EXTENSION, VECTORS_DOCUMENTS_EXTENSION, FIELDS_INDEX_EXTENSION, FIELDS_EXTENSION};
+		
+		internal static readonly System.String[] NON_STORE_INDEX_EXTENSIONS = new System.String[]{FIELD_INFOS_EXTENSION, FREQ_EXTENSION, PROX_EXTENSION, TERMS_EXTENSION, TERMS_INDEX_EXTENSION, NORMS_EXTENSION};
 		
 		/// <summary>File extensions of old-style index files </summary>
-		public static readonly System.String[] COMPOUND_EXTENSIONS = new System.String[]{"fnm", "frq", "prx", "fdx", "fdt", "tii", "tis"};
+		internal static readonly System.String[] COMPOUND_EXTENSIONS = new System.String[]{FIELD_INFOS_EXTENSION, FREQ_EXTENSION, PROX_EXTENSION, FIELDS_INDEX_EXTENSION, FIELDS_EXTENSION, TERMS_INDEX_EXTENSION, TERMS_EXTENSION};
 		
 		/// <summary>File extensions for term vector support </summary>
-		public static readonly System.String[] VECTOR_EXTENSIONS = new System.String[]{"tvx", "tvd", "tvf"};
+		internal static readonly System.String[] VECTOR_EXTENSIONS = new System.String[]{VECTORS_INDEX_EXTENSION, VECTORS_DOCUMENTS_EXTENSION, VECTORS_FIELDS_EXTENSION};
 		
 		/// <summary> Computes the full file name from base, extension and
 		/// generation.  If the generation is -1, the file name is
@@ -71,24 +128,38 @@
 		/// </param>
 		/// <param name="gen">-- generation
 		/// </param>
-		public static System.String FileNameFromGeneration(System.String base_Renamed, System.String extension, long gen)
+		internal static System.String FileNameFromGeneration(System.String base_Renamed, System.String extension, long gen)
 		{
-			if (gen == - 1)
+			if (gen == SegmentInfo.NO)
 			{
 				return null;
 			}
-			else if (gen == 0)
+			else if (gen == SegmentInfo.WITHOUT_GEN)
 			{
 				return base_Renamed + extension;
 			}
 			else
 			{
 #if !PRE_LUCENE_NET_2_0_0_COMPATIBLE
-                return base_Renamed + "_" + Lucene.Net.Documents.NumberTools.ToString(gen) + extension;
+				return base_Renamed + "_" + SupportClass.Number.ToString(gen) + extension;
 #else
 				return base_Renamed + "_" + System.Convert.ToString(gen, 16) + extension;
 #endif
 			}
 		}
+		
+		/// <summary> Returns true if the provided filename is one of the doc
+		/// store files (ends with an extension in
+		/// STORE_INDEX_EXTENSIONS).
+		/// </summary>
+		internal static bool IsDocStoreFile(System.String fileName)
+		{
+			if (fileName.EndsWith(COMPOUND_FILE_STORE_EXTENSION))
+				return true;
+			for (int i = 0; i < STORE_INDEX_EXTENSIONS.Length; i++)
+				if (fileName.EndsWith(STORE_INDEX_EXTENSIONS[i]))
+					return true;
+			return false;
+		}
 	}
 }
\ No newline at end of file