You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by do...@apache.org on 2009/07/29 20:04:24 UTC

svn commit: r798995 [13/35] - in /incubator/lucene.net/trunk/C#/src: Lucene.Net/ Lucene.Net/Analysis/ Lucene.Net/Analysis/Standard/ Lucene.Net/Document/ Lucene.Net/Index/ Lucene.Net/QueryParser/ Lucene.Net/Search/ Lucene.Net/Search/Function/ Lucene.Net...

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentReader.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/SegmentReader.cs?rev=798995&r1=798994&r2=798995&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentReader.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentReader.cs Wed Jul 29 18:04:12 2009
@@ -15,22 +15,20 @@
  * limitations under the License.
  */
 
-using System;
+using System.Collections.Generic;
 
-using Document = Lucene.Net.Documents.Document;
-using FieldSelector = Lucene.Net.Documents.FieldSelector;
+using BitVector = Lucene.Net.Util.BitVector;
 using BufferedIndexInput = Lucene.Net.Store.BufferedIndexInput;
+using CloseableThreadLocal = Lucene.Net.Util.CloseableThreadLocal;
+using DefaultSimilarity = Lucene.Net.Search.DefaultSimilarity;
 using Directory = Lucene.Net.Store.Directory;
+using Document = Lucene.Net.Documents.Document;
+using FieldSelector = Lucene.Net.Documents.FieldSelector;
 using IndexInput = Lucene.Net.Store.IndexInput;
 using IndexOutput = Lucene.Net.Store.IndexOutput;
-using BitVector = Lucene.Net.Util.BitVector;
-using DefaultSimilarity = Lucene.Net.Search.DefaultSimilarity;
 
 namespace Lucene.Net.Index
 {
-	
-	/// <version>  $Id: SegmentReader.java 603061 2007-12-10 21:49:41Z gsingers $
-	/// </version>
 	public class SegmentReader : DirectoryIndexReader
 	{
 		private System.String segment;
@@ -42,16 +40,19 @@
 		
 		internal TermInfosReader tis;
 		internal TermVectorsReader termVectorsReaderOrig = null;
-		internal System.LocalDataStoreSlot termVectorsLocal = System.Threading.Thread.AllocateDataSlot();
+        internal CloseableThreadLocal termVectorsLocal = new CloseableThreadLocal();
 		
 		internal BitVector deletedDocs = null;
 		private bool deletedDocsDirty = false;
 		private bool normsDirty = false;
 		private bool undeleteAll = false;
+        private int pendingDeleteCount;
 		
 		private bool rollbackDeletedDocsDirty = false;
 		private bool rollbackNormsDirty = false;
 		private bool rollbackUndeleteAll = false;
+        private int rollbackPendingDeleteCount;
+        new private bool readOnly;
 		
 		internal IndexInput freqStream;
 		internal IndexInput proxStream;
@@ -167,7 +168,7 @@
 		/// <summary> Increments the RC of this reader, as well as
 		/// of all norms this reader is using
 		/// </summary>
-		protected internal override void  IncRef()
+		public override void IncRef()
 		{
 			lock (this)
 			{
@@ -194,7 +195,7 @@
 			}
 		}
 		
-		protected internal override void  DecRef()
+		public override void DecRef()
 		{
 			lock (this)
 			{
@@ -203,7 +204,7 @@
 				while (it.MoveNext())
 				{
 					Norm norm = (Norm) it.Current;
-					norm.DecRef();
+				  	norm.DecRef();
 				}
 			}
 		}
@@ -221,62 +222,79 @@
 		/// <summary>The class which implements SegmentReader. </summary>
 		private static System.Type IMPL;
 		
+        private static System.Type READONLY_IMPL;
+
 		/// <throws>  CorruptIndexException if the index is corrupt </throws>
 		/// <throws>  IOException if there is a low-level IO error </throws>
 		public static SegmentReader Get(SegmentInfo si)
 		{
-			return Get(si.dir, si, null, false, false, BufferedIndexInput.BUFFER_SIZE, true);
+			return Get(READ_ONLY_DEFAULT, si.dir, si, null, false, false, BufferedIndexInput.BUFFER_SIZE, true);
 		}
 		
+        public static SegmentReader Get(bool readOnly, SegmentInfo si)
+        {
+            return Get(readOnly, si.dir, si, null, false, false, BufferedIndexInput.BUFFER_SIZE, true);
+        }
+
 		/// <throws>  CorruptIndexException if the index is corrupt </throws>
 		/// <throws>  IOException if there is a low-level IO error </throws>
 		internal static SegmentReader Get(SegmentInfo si, bool doOpenStores)
 		{
-			return Get(si.dir, si, null, false, false, BufferedIndexInput.BUFFER_SIZE, doOpenStores);
+            return Get(READ_ONLY_DEFAULT, si.dir, si, null, false, false, BufferedIndexInput.BUFFER_SIZE, doOpenStores);
 		}
 		
 		/// <throws>  CorruptIndexException if the index is corrupt </throws>
 		/// <throws>  IOException if there is a low-level IO error </throws>
 		public static SegmentReader Get(SegmentInfo si, int readBufferSize)
 		{
-			return Get(si.dir, si, null, false, false, readBufferSize, true);
+            return Get(READ_ONLY_DEFAULT, si.dir, si, null, false, false, readBufferSize, true);
 		}
 		
 		/// <throws>  CorruptIndexException if the index is corrupt </throws>
 		/// <throws>  IOException if there is a low-level IO error </throws>
 		internal static SegmentReader Get(SegmentInfo si, int readBufferSize, bool doOpenStores)
 		{
-			return Get(si.dir, si, null, false, false, readBufferSize, doOpenStores);
+            return Get(READ_ONLY_DEFAULT, si.dir, si, null, false, false, readBufferSize, doOpenStores);
 		}
-		
-		/// <throws>  CorruptIndexException if the index is corrupt </throws>
+
+        /// <throws>  CorruptIndexException if the index is corrupt </throws>
+        /// <throws>  IOException if there is a low-level IO error </throws>
+        internal static SegmentReader Get(bool readOnly, SegmentInfo si, int readBufferSize, bool doOpenStores)
+        {
+            return Get(readOnly, si.dir, si, null, false, false, readBufferSize, doOpenStores);
+        }
+
+        /// <throws>  CorruptIndexException if the index is corrupt </throws>
 		/// <throws>  IOException if there is a low-level IO error </throws>
-		public static SegmentReader Get(SegmentInfos sis, SegmentInfo si, bool closeDir)
+		public static SegmentReader Get(bool readOnly, SegmentInfos sis, SegmentInfo si, bool closeDir)
 		{
-			return Get(si.dir, si, sis, closeDir, true, BufferedIndexInput.BUFFER_SIZE, true);
+			return Get(readOnly, si.dir, si, sis, closeDir, true, BufferedIndexInput.BUFFER_SIZE, true);
 		}
 		
 		/// <throws>  CorruptIndexException if the index is corrupt </throws>
 		/// <throws>  IOException if there is a low-level IO error </throws>
 		public static SegmentReader Get(Directory dir, SegmentInfo si, SegmentInfos sis, bool closeDir, bool ownDir, int readBufferSize)
 		{
-			return Get(dir, si, sis, closeDir, ownDir, readBufferSize, true);
+            return Get(READ_ONLY_DEFAULT, dir, si, sis, closeDir, ownDir, readBufferSize, true);
 		}
 		
 		/// <throws>  CorruptIndexException if the index is corrupt </throws>
 		/// <throws>  IOException if there is a low-level IO error </throws>
-		public static SegmentReader Get(Directory dir, SegmentInfo si, SegmentInfos sis, bool closeDir, bool ownDir, int readBufferSize, bool doOpenStores)
+		public static SegmentReader Get(bool readOnly, Directory dir, SegmentInfo si, SegmentInfos sis, bool closeDir, bool ownDir, int readBufferSize, bool doOpenStores)
 		{
 			SegmentReader instance;
 			try
 			{
-				instance = (SegmentReader) System.Activator.CreateInstance(IMPL);
-			}
+                if (readOnly)
+                    instance = (SegmentReader)System.Activator.CreateInstance(READONLY_IMPL);
+                else
+                    instance = (SegmentReader)System.Activator.CreateInstance(IMPL);
+            }
 			catch (System.Exception e)
 			{
 				throw new System.Exception("cannot load SegmentReader class: " + e, e);
 			}
-			instance.Init(dir, sis, closeDir);
+			instance.Init(dir, sis, closeDir, readOnly);
 			instance.Initialize(si, readBufferSize, doOpenStores);
 			return instance;
 		}
@@ -323,9 +341,14 @@
 				else
 					storeDir = null;
 				
-				// No compound file exists - use the multi-file format
 				fieldInfos = new FieldInfos(cfsDir, segment + ".fnm");
-				
+
+                bool anyProx = false;
+                int numFields = fieldInfos.Size();
+                for (int i = 0; !anyProx && i < numFields; i++)
+                    if (!fieldInfos.FieldInfo(i).omitTf)
+                        anyProx = true;
+
 				System.String fieldsSegment;
 				
 				if (si.GetDocStoreOffset() != - 1)
@@ -351,8 +374,9 @@
 				// make sure that all index files have been read or are kept open
 				// so that if an index update removes them we'll still have them
 				freqStream = cfsDir.OpenInput(segment + ".frq", readBufferSize);
-				proxStream = cfsDir.OpenInput(segment + ".prx", readBufferSize);
-				OpenNorms(cfsDir, readBufferSize);
+                if (anyProx)
+                    proxStream = cfsDir.OpenInput(segment + ".prx", readBufferSize);
+                OpenNorms(cfsDir, readBufferSize);
 				
 				if (doOpenStores && fieldInfos.HasVectors())
 				{
@@ -380,21 +404,24 @@
 				}
 			}
 		}
-		
-		private void  LoadDeletedDocs()
-		{
-			// NOTE: the bitvector is stored using the regular directory, not cfs
-			if (HasDeletions(si))
-			{
-				deletedDocs = new BitVector(Directory(), si.GetDelFileName());
-				
-				// Verify # deletes does not exceed maxDoc for this segment:
-				if (deletedDocs.Count() > MaxDoc())
-				{
-					throw new CorruptIndexException("number of deletes (" + deletedDocs.Count() + ") exceeds max doc (" + MaxDoc() + ") for segment " + si.name);
-				}
-			}
-		}
+
+        private void LoadDeletedDocs()
+        {
+            // NOTE: the bitvector is stored using the regular directory, not cfs
+            if (HasDeletions(si))
+            {
+                deletedDocs = new BitVector(Directory(), si.GetDelFileName());
+
+                System.Diagnostics.Debug.Assert(si.GetDelCount() == deletedDocs.Count(),
+                    "delete count mismatch: info=" + si.GetDelCount() + " vs BitVector=" + deletedDocs.Count());
+
+                // Verify # deletes does not exceed maxDoc for this segment:
+                System.Diagnostics.Debug.Assert(si.GetDelCount() <= MaxDoc(),
+                  "delete count mismatch: " + deletedDocs.Count() + ") exceeds max doc (" + MaxDoc() + ") for segment " + si.name);
+            }
+            else
+                System.Diagnostics.Debug.Assert(si.GetDelCount() == 0);
+        }
 		
 		protected internal override DirectoryIndexReader DoReopen(SegmentInfos infos)
 		{
@@ -413,13 +440,16 @@
 					{
 						// segment not referenced anymore, reopen not possible
 						// or segment format changed
-						newReader = SegmentReader.Get(infos, infos.Info(0), false);
+						newReader = SegmentReader.Get(readOnly, infos, infos.Info(0), false);
 					}
 				}
 				else
 				{
-					return new MultiSegmentReader(directory, infos, closeDirectory, new SegmentReader[]{this}, null, null);
-				}
+                    if (readOnly)
+                        return new ReadOnlyMultiSegmentReader(directory, infos, closeDirectory, new SegmentReader[] { this }, null, null);
+                    else
+                        return new MultiSegmentReader(directory, infos, closeDirectory, new SegmentReader[] { this }, null, null, false);
+                }
 				
 				return newReader;
 			}
@@ -453,10 +483,16 @@
 				
 				
 				// clone reader
-				SegmentReader clone = new SegmentReader();
-				bool success = false;
+				SegmentReader clone;
+                if (readOnly)
+                    clone = new ReadOnlySegmentReader();
+                else
+                    clone = new SegmentReader();
+
+                bool success = false;
 				try
 				{
+                    clone.readOnly = readOnly;
 					clone.directory = directory;
 					clone.si = si;
 					clone.segment = segment;
@@ -609,11 +645,14 @@
 				// .tmp & renaming it) because the file is not live
 				// until segments file is written:
 				deletedDocs.Write(Directory(), si.GetDelFileName());
+
+                si.SetDelCount(si.GetDelCount() + pendingDeleteCount);
 			}
 			if (undeleteAll && si.HasDeletions())
 			{
 				si.ClearDelGen();
-			}
+                si.SetDelCount(0);
+            }
 			if (normsDirty)
 			{
 				// re-write norms
@@ -637,60 +676,69 @@
 		{
 			return fieldsReader;
 		}
-		
+
+        public void foo()
+        { 
+            //termVectorsLocal.Clear();
+            termVectorsLocal.Close(); 
+        }
+
 		protected internal override void  DoClose()
 		{
-			bool hasReferencedReader = (referencedSegmentReader != null);
-			
-			if (hasReferencedReader)
-			{
-				referencedSegmentReader.DecRefReaderNotNorms();
-				referencedSegmentReader = null;
-			}
-			
-			deletedDocs = null;
-			
-			// close the single norms stream
-			if (singleNormStream != null)
-			{
-				// we can close this stream, even if the norms
-				// are shared, because every reader has it's own 
-				// singleNormStream
-				singleNormStream.Close();
-				singleNormStream = null;
-			}
-			
-			// re-opened SegmentReaders have their own instance of FieldsReader
-			if (fieldsReader != null)
-			{
-				fieldsReader.Close();
-			}
-			
-			if (!hasReferencedReader)
-			{
-				// close everything, nothing is shared anymore with other readers
-				if (tis != null)
-				{
-					tis.Close();
-				}
-				
-				if (freqStream != null)
-					freqStream.Close();
-				if (proxStream != null)
-					proxStream.Close();
-				
-				if (termVectorsReaderOrig != null)
-					termVectorsReaderOrig.Close();
-				
-				if (cfsReader != null)
-					cfsReader.Close();
-				
-				if (storeCFSReader != null)
-					storeCFSReader.Close();
-				
-				// maybe close directory
-				base.DoClose();
-			}
+            bool hasReferencedReader = (referencedSegmentReader != null);
+
+            termVectorsLocal.Close();
+            //termVectorsLocal.Clear();
+
+            if (hasReferencedReader)
+            {
+                referencedSegmentReader.DecRefReaderNotNorms();
+                referencedSegmentReader = null;
+            }
+
+            deletedDocs = null;
+
+            // close the single norms stream
+            if (singleNormStream != null)
+            {
+                // we can close this stream, even if the norms
+                // are shared, because every reader has it's own 
+                // singleNormStream
+                singleNormStream.Close();
+                singleNormStream = null;
+            }
+
+            // re-opened SegmentReaders have their own instance of FieldsReader
+            if (fieldsReader != null)
+            {
+                fieldsReader.Close();
+            }
+
+            if (!hasReferencedReader)
+            {
+                // close everything, nothing is shared anymore with other readers
+                if (tis != null)
+                {
+                    tis.Close();
+                }
+
+                if (freqStream != null)
+                    freqStream.Close();
+                if (proxStream != null)
+                    proxStream.Close();
+
+                if (termVectorsReaderOrig != null)
+                    termVectorsReaderOrig.Close();
+
+                if (cfsReader != null)
+                    cfsReader.Close();
+
+                if (storeCFSReader != null)
+                    storeCFSReader.Close();
+
+                // maybe close directory
+                base.DoClose();
+            }
 		}
 		
 		internal static bool HasDeletions(SegmentInfo si)
@@ -721,7 +769,8 @@
 				deletedDocs = new BitVector(MaxDoc());
 			deletedDocsDirty = true;
 			undeleteAll = false;
-			deletedDocs.Set(docNum);
+			if (!deletedDocs.GetAndSet(docNum))
+                pendingDeleteCount++;
 		}
 		
 		protected internal override void  DoUndeleteAll()
@@ -731,12 +780,14 @@
 			undeleteAll = true;
 		}
 		
-		internal virtual System.Collections.ArrayList Files()
+		internal virtual List<string> Files()
 		{
-			return System.Collections.ArrayList.Synchronized(new System.Collections.ArrayList(si.Files()));
-		}
-		
-		public override TermEnum Terms()
+            List<string> copy = new List<string>(si.Files().Count);
+            copy.AddRange(si.Files());
+            return copy;
+        }
+
+        public override TermEnum Terms()
 		{
 			EnsureOpen();
 			return tis.Terms();
@@ -823,54 +874,58 @@
 		
 		/// <seealso cref="IndexReader.GetFieldNames(IndexReader.FieldOption fldOption)">
 		/// </seealso>
-		public override System.Collections.ICollection GetFieldNames(IndexReader.FieldOption fieldOption)
+		public override System.Collections.Generic.ICollection<string> GetFieldNames(IndexReader.FieldOption fieldOption)
 		{
 			EnsureOpen();
 
-            System.Collections.Hashtable fieldSet = new System.Collections.Hashtable();
+            System.Collections.Generic.Dictionary<string, string> fieldSet = new System.Collections.Generic.Dictionary<string, string>();
 			for (int i = 0; i < fieldInfos.Size(); i++)
 			{
 				FieldInfo fi = fieldInfos.FieldInfo(i);
 				if (fieldOption == IndexReader.FieldOption.ALL)
 				{
-                    fieldSet.Add(fi.name, fi.name);
+                    fieldSet[fi.name] = fi.name;
 				}
 				else if (!fi.isIndexed && fieldOption == IndexReader.FieldOption.UNINDEXED)
 				{
-                    fieldSet.Add(fi.name, fi.name);
-				}
-				else if (fi.storePayloads && fieldOption == IndexReader.FieldOption.STORES_PAYLOADS)
-				{
-                    fieldSet.Add(fi.name, fi.name);
-				}
-				else if (fi.isIndexed && fieldOption == IndexReader.FieldOption.INDEXED)
-				{
-                    fieldSet.Add(fi.name, fi.name);
-				}
-				else if (fi.isIndexed && fi.storeTermVector == false && fieldOption == IndexReader.FieldOption.INDEXED_NO_TERMVECTOR)
-				{
-                    fieldSet.Add(fi.name, fi.name);
-				}
-				else if (fi.storeTermVector == true && fi.storePositionWithTermVector == false && fi.storeOffsetWithTermVector == false && fieldOption == IndexReader.FieldOption.TERMVECTOR)
-				{
-                    fieldSet.Add(fi.name, fi.name);
-				}
-				else if (fi.isIndexed && fi.storeTermVector && fieldOption == IndexReader.FieldOption.INDEXED_WITH_TERMVECTOR)
-				{
-                    fieldSet.Add(fi.name, fi.name);
-				}
-				else if (fi.storePositionWithTermVector && fi.storeOffsetWithTermVector == false && fieldOption == IndexReader.FieldOption.TERMVECTOR_WITH_POSITION)
-				{
-                    fieldSet.Add(fi.name, fi.name);
-				}
-				else if (fi.storeOffsetWithTermVector && fi.storePositionWithTermVector == false && fieldOption == IndexReader.FieldOption.TERMVECTOR_WITH_OFFSET)
-				{
-                    fieldSet.Add(fi.name, fi.name);
-				}
-				else if ((fi.storeOffsetWithTermVector && fi.storePositionWithTermVector) && fieldOption == IndexReader.FieldOption.TERMVECTOR_WITH_POSITION_OFFSET)
-				{
-                    fieldSet.Add(fi.name, fi.name);
+                    fieldSet[fi.name] = fi.name;
 				}
+                else if (fi.omitTf && fieldOption == IndexReader.FieldOption.OMIT_TF)
+                {
+                    fieldSet[fi.name] = fi.name;
+                }
+                else if (fi.storePayloads && fieldOption == IndexReader.FieldOption.STORES_PAYLOADS)
+                {
+                    fieldSet[fi.name] = fi.name;
+                }
+                else if (fi.isIndexed && fieldOption == IndexReader.FieldOption.INDEXED)
+                {
+                    fieldSet[fi.name] = fi.name;
+                }
+                else if (fi.isIndexed && fi.storeTermVector == false && fieldOption == IndexReader.FieldOption.INDEXED_NO_TERMVECTOR)
+                {
+                    fieldSet[fi.name] = fi.name;
+                }
+                else if (fi.storeTermVector == true && fi.storePositionWithTermVector == false && fi.storeOffsetWithTermVector == false && fieldOption == IndexReader.FieldOption.TERMVECTOR)
+                {
+                    fieldSet[fi.name] = fi.name;
+                }
+                else if (fi.isIndexed && fi.storeTermVector && fieldOption == IndexReader.FieldOption.INDEXED_WITH_TERMVECTOR)
+                {
+                    fieldSet[fi.name] = fi.name;
+                }
+                else if (fi.storePositionWithTermVector && fi.storeOffsetWithTermVector == false && fieldOption == IndexReader.FieldOption.TERMVECTOR_WITH_POSITION)
+                {
+                    fieldSet[fi.name] = fi.name;
+                }
+                else if (fi.storeOffsetWithTermVector && fi.storePositionWithTermVector == false && fieldOption == IndexReader.FieldOption.TERMVECTOR_WITH_OFFSET)
+                {
+                    fieldSet[fi.name] = fi.name;
+                }
+                else if ((fi.storeOffsetWithTermVector && fi.storePositionWithTermVector) && fieldOption == IndexReader.FieldOption.TERMVECTOR_WITH_POSITION_OFFSET)
+                {
+                    fieldSet[fi.name] = fi.name;
+                }
 			}
 			return fieldSet.Keys;
 		}
@@ -934,7 +989,13 @@
 			{
 				EnsureOpen();
 				byte[] bytes = GetNorms(field);
-				if (bytes == null)
+
+//for (int i = 0; i < bytes.Length; i++)
+//    if (bytes[i] == 0)
+//        System.Console.Error.WriteLine("in directory " + directory + " norm for " + field + " = 0 for doc " + i);
+//    System.Diagnostics.Debug.Assert(bytes[i] > 0, "norm for " + field + " = 0 for doc " + i);
+
+                if (bytes == null)
 					bytes = FakeNorms();
 				return bytes;
 			}
@@ -963,7 +1024,7 @@
 				Norm norm = (Norm) norms[field];
 				if (norm == null)
 				{
-					Array.Copy(FakeNorms(), 0, bytes, offset, MaxDoc());
+					System.Array.Copy(FakeNorms(), 0, bytes, offset, MaxDoc());
 					return ;
 				}
 				
@@ -972,7 +1033,7 @@
 					if (norm.bytes != null)
 					{
 						// can copy from cache
-						Array.Copy(norm.bytes, 0, bytes, offset, MaxDoc());
+						System.Array.Copy(norm.bytes, 0, bytes, offset, MaxDoc());
 						return ;
 					}
 					
@@ -1076,12 +1137,20 @@
 		/// </returns>
 		private TermVectorsReader GetTermVectorsReader()
 		{
-			TermVectorsReader tvReader = (TermVectorsReader) System.Threading.Thread.GetData(termVectorsLocal);
-			if (tvReader == null)
-			{
-				tvReader = (TermVectorsReader) termVectorsReaderOrig.Clone();
-				System.Threading.Thread.SetData(termVectorsLocal, tvReader);
-			}
+            System.Diagnostics.Debug.Assert(termVectorsReaderOrig != null);
+            TermVectorsReader tvReader = (TermVectorsReader)termVectorsLocal.Get();
+            if (tvReader == null)
+            {
+                try
+                {
+                    tvReader = (TermVectorsReader)termVectorsReaderOrig.Clone();
+                }
+                catch (System.Exception)
+                {
+                    return null;
+                }
+                termVectorsLocal.Set(tvReader);
+            }
 			return tvReader;
 		}
 		
@@ -1106,7 +1175,6 @@
 			return termVectorsReader.Get(docNumber, field);
 		}
 		
-		
 		public override void  GetTermFreqVector(int docNumber, System.String field, TermVectorMapper mapper)
 		{
 			EnsureOpen();
@@ -1187,6 +1255,7 @@
 			rollbackDeletedDocsDirty = deletedDocsDirty;
 			rollbackNormsDirty = normsDirty;
 			rollbackUndeleteAll = undeleteAll;
+            rollbackPendingDeleteCount = pendingDeleteCount;
 			System.Collections.IEnumerator it = norms.Values.GetEnumerator();
 			while (it.MoveNext())
 			{
@@ -1201,6 +1270,7 @@
 			deletedDocsDirty = rollbackDeletedDocsDirty;
 			normsDirty = rollbackNormsDirty;
 			undeleteAll = rollbackUndeleteAll;
+            pendingDeleteCount = rollbackPendingDeleteCount;
 			System.Collections.IEnumerator it = norms.Values.GetEnumerator();
 			while (it.MoveNext())
 			{
@@ -1210,28 +1280,42 @@
 		}
 		static SegmentReader()
 		{
-			{
-				try
-				{
-					System.String name = SupportClass.AppSettings.Get("Lucene.Net.SegmentReader.class", typeof(SegmentReader).FullName);
-					IMPL = System.Type.GetType(name);
-				}
-				catch (System.Security.SecurityException se)
-				{
-					try
-					{
-						IMPL = System.Type.GetType(typeof(SegmentReader).FullName);
-					}
-					catch (System.Exception e)
-					{
-						throw new System.Exception("cannot load default SegmentReader class: " + e, e);
-					}
-				}
-				catch (System.Exception e)
-				{
-					throw new System.Exception("cannot load SegmentReader class: " + e, e);
-				}
-			}
-		}
-	}
-}
+            // IMPL type
+            try
+            {
+                string name = SupportClass.AppSettings.Get("Lucene.Net.SegmentReader.class", typeof(SegmentReader).FullName);
+                IMPL = System.Type.GetType(name);
+            }
+            catch (System.Security.SecurityException)
+            {
+                try
+                {
+                    IMPL = System.Type.GetType(typeof(SegmentReader).FullName);
+                }
+                catch (System.Exception e)
+                {
+                    throw new System.Exception("cannot load default SegmentReader class: " + e, e);
+                }
+            }
+            catch (System.Exception e) { throw new System.Exception("cannot load SegmentReader class: " + e, e); }
+            // READONLY_IMPL type
+            try
+            {
+                string name = SupportClass.AppSettings.Get("Lucene.Net.ReadOnlySegmentReader.class", typeof(ReadOnlySegmentReader).FullName);
+                READONLY_IMPL = System.Type.GetType(name);
+            }
+            catch (System.Security.SecurityException)
+            {
+                try
+                {
+                    READONLY_IMPL = System.Type.GetType(typeof(ReadOnlySegmentReader).FullName);
+                }
+                catch (System.Exception e)
+                {
+                    throw new System.Exception("cannot load default ReadOnlySegmentReader class: " + e, e);
+                }
+            }
+            catch (System.Exception e) { throw new System.Exception("cannot load ReadOnlSegmentReader class: " + e, e); }
+        }
+    }
+}
\ No newline at end of file

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentTermDocs.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/SegmentTermDocs.cs?rev=798995&r1=798994&r2=798995&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentTermDocs.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentTermDocs.cs Wed Jul 29 18:04:12 2009
@@ -42,8 +42,9 @@
 		
 		private long skipPointer;
 		private bool haveSkipped;
-		
-		protected internal bool currentFieldStoresPayloads;
+
+        protected internal bool currentFieldStoresPayloads;
+        protected internal bool currentFieldOmitTf;
 
         // for testing
         public IndexInput FreqStream_ForNUnitTest
@@ -94,7 +95,8 @@
 		{
 			count = 0;
 			FieldInfo fi = parent.fieldInfos.FieldInfo(term.field);
-			currentFieldStoresPayloads = (fi != null) ? fi.storePayloads : false;
+            currentFieldOmitTf = (fi != null) ? fi.omitTf : false;
+            currentFieldStoresPayloads = (fi != null) ? fi.storePayloads : false;
 			if (ti == null)
 			{
 				df = 0;
@@ -139,13 +141,20 @@
 					return false;
 				
 				int docCode = freqStream.ReadVInt();
-				doc += (int) (((uint) docCode) >> 1); // shift off low bit
-				if ((docCode & 1) != 0)
-				// if low bit is set
-					freq = 1;
-				// freq is one
-				else
-					freq = freqStream.ReadVInt(); // else read freq
+
+                if (currentFieldOmitTf)
+                {
+                    doc += docCode;
+                    freq = 1;
+                }
+                else
+                {
+                    doc += (int)(((uint)docCode) >> 1); // shift off low bit
+                    if ((docCode & 1) != 0) // if low bit is set
+                        freq = 1; // freq is one
+                    else
+                        freq = freqStream.ReadVInt(); // else read freq
+                }
 				
 				count++;
 				
@@ -157,34 +166,62 @@
 		}
 		
 		/// <summary>Optimized implementation. </summary>
-		public virtual int Read(int[] docs, int[] freqs)
-		{
-			int length = docs.Length;
-			int i = 0;
-			while (i < length && count < df)
-			{
-				
-				// manually inlined call to next() for speed
-				int docCode = freqStream.ReadVInt();
-				doc += (int) (((uint) docCode) >> 1); // shift off low bit
-				if ((docCode & 1) != 0)
-				// if low bit is set
-					freq = 1;
-				// freq is one
-				else
-					freq = freqStream.ReadVInt(); // else read freq
-				count++;
-				
-				if (deletedDocs == null || !deletedDocs.Get(doc))
-				{
-					docs[i] = doc;
-					freqs[i] = freq;
-					++i;
-				}
-			}
-			return i;
-		}
-		
+        public virtual int Read(int[] docs, int[] freqs)
+        {
+            int length = docs.Length;
+            if (currentFieldOmitTf)
+            {
+                return ReadNoTf(docs, freqs, length);
+            }
+            else
+            {
+                int i = 0;
+                while (i < length && count < df)
+                {
+
+                    // manually inlined call to next() for speed
+                    int docCode = freqStream.ReadVInt();
+                    doc += (int)(((uint)docCode) >> 1); // shift off low bit
+                    if ((docCode & 1) != 0)
+                        // if low bit is set
+                        freq = 1;
+                    // freq is one
+                    else
+                        freq = freqStream.ReadVInt(); // else read freq
+                    count++;
+
+                    if (deletedDocs == null || !deletedDocs.Get(doc))
+                    {
+                        docs[i] = doc;
+                        freqs[i] = freq;
+                        ++i;
+                    }
+                }
+                return i;
+            }
+        }
+
+        private int ReadNoTf(int[] docs, int[] freqs, int length)
+        {
+            int i = 0;
+            while (i < length && count < df)
+            {
+                // manually inlined call to next() for speed
+                doc += freqStream.ReadVInt();
+                count++;
+
+                if (deletedDocs == null || !deletedDocs.Get(doc))
+                {
+                    docs[i] = doc;
+                    // hardware freq to 1 when term freqs were not
+                    // stored in the index
+                    freqs[i] = 1;
+                    ++i;
+                }
+            }
+            return i;
+        }
+
 		/// <summary>Overridden by SegmentTermPositions to skip in prox stream. </summary>
 		protected internal virtual void  SkipProx(long proxPointer)
 		{

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentTermEnum.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/SegmentTermEnum.cs?rev=798995&r1=798994&r2=798995&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentTermEnum.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentTermEnum.cs Wed Jul 29 18:04:12 2009
@@ -31,7 +31,7 @@
 		
 		private TermBuffer termBuffer = new TermBuffer();
 		private TermBuffer prevBuffer = new TermBuffer();
-		private TermBuffer scratch; // used for scanning
+        private TermBuffer scanBuffer = new TermBuffer();
 		
 		private TermInfo termInfo = new TermInfo();
 		
@@ -67,8 +67,8 @@
 				format = firstInt;
 				
 				// check that it is a format we can understand
-				if (format < TermInfosWriter.FORMAT)
-					throw new CorruptIndexException("Unknown format version:" + format);
+				if (format < TermInfosWriter.FORMAT_CURRENT)
+					throw new CorruptIndexException("Unknown format version:" + format + " expected " + TermInfosWriter.FORMAT_CURRENT + " or higher");
 				
 				size = input.ReadLong(); // read the size
 				
@@ -87,16 +87,22 @@
 				{
 					indexInterval = input.ReadInt();
 					skipInterval = input.ReadInt();
-					if (format == - 3)
+					if (format <= TermInfosWriter.FORMAT)
 					{
 						// this new format introduces multi-level skipping
 						maxSkipLevels = input.ReadInt();
 					}
 				}
 			}
+            if (format > TermInfosWriter.FORMAT_VERSION_UTF8_LENGTH_IN_BYTES)
+            {
+                termBuffer.SetPreUTF8Strings();
+                scanBuffer.SetPreUTF8Strings();
+                prevBuffer.SetPreUTF8Strings();
+            }
 		}
 		
-		public System.Object Clone()
+		public object Clone()
 		{
 			SegmentTermEnum clone = null;
 			try
@@ -111,8 +117,8 @@
 			clone.termInfo = new TermInfo(termInfo);
 			
 			clone.termBuffer = (TermBuffer) termBuffer.Clone();
-			clone.prevBuffer = (TermBuffer) prevBuffer.Clone();
-			clone.scratch = null;
+            clone.prevBuffer = (TermBuffer)prevBuffer.Clone();
+            clone.scanBuffer = new TermBuffer();
 			
 			return clone;
 		}
@@ -167,16 +173,17 @@
 			return true;
 		}
 		
-		/// <summary>Optimized scan, without allocating new terms. </summary>
-		internal void  ScanTo(Term term)
+		/// <summary>Optimized scan, without allocating new terms.  Return numver of invocations to Next(). </summary>
+		internal int ScanTo(Term term)
 		{
-			if (scratch == null)
-				scratch = new TermBuffer();
-			scratch.Set(term);
-			while (scratch.CompareTo(termBuffer) > 0 && Next())
-			{
-			}
-		}
+            scanBuffer.Set(term);
+            int count = 0;
+            while (scanBuffer.CompareTo(termBuffer) > 0 && Next())
+            {
+                count++;
+            }
+            return count;
+        }
 		
 		/// <summary>Returns the current Term in the enumeration.
 		/// Initially invalid, valid after next() called for the first time.

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentTermPositions.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/SegmentTermPositions.cs?rev=798995&r1=798994&r2=798995&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentTermPositions.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentTermPositions.cs Wed Jul 29 18:04:12 2009
@@ -65,6 +65,9 @@
 		
 		public int NextPosition()
 		{
+            if (currentFieldOmitTf)
+                return 0; // this field does not store term freq, positions, payloads
+
 			// perform lazy skips if neccessary
 			LazySkip();
 			proxCount--;
@@ -131,6 +134,7 @@
 		
 		private void  SkipPositions(int n)
 		{
+            System.Diagnostics.Debug.Assert(!currentFieldOmitTf);
 			for (int f = n; f > 0; f--)
 			{
 				// skip unread positions

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentTermVector.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/SegmentTermVector.cs?rev=798995&r1=798994&r2=798995&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentTermVector.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SegmentTermVector.cs Wed Jul 29 18:04:12 2009
@@ -19,8 +19,6 @@
 
 namespace Lucene.Net.Index
 {
-	
-	
 	public class SegmentTermVector : TermFreqVector
 	{
 		private System.String field;
@@ -80,7 +78,7 @@
 		{
 			if (terms == null)
 				return - 1;
-			int res = System.Array.BinarySearch(terms, termText,StringComparer.Ordinal);
+			int res = System.Array.BinarySearch(terms, termText);
 			return res >= 0 ? res : - 1;
 		}
 		

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SnapshotDeletionPolicy.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/SnapshotDeletionPolicy.cs?rev=798995&r1=798994&r2=798995&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SnapshotDeletionPolicy.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SnapshotDeletionPolicy.cs Wed Jul 29 18:04:12 2009
@@ -15,7 +15,9 @@
  * limitations under the License.
  */
 
-using System;
+using System.Collections.Generic;
+
+using Directory = Lucene.Net.Store.Directory;
 
 namespace Lucene.Net.Index
 {
@@ -29,36 +31,43 @@
 	/// we wrap another arbitrary {@link IndexDeletionPolicy}, this
 	/// gives you the freedom to continue using whatever {@link
 	/// IndexDeletionPolicy} you would normally want to use with your
-	/// index. 
+    /// index.  Note that you can re-use a single instance of
+    /// SnapshotDeletionPolicy across multiple writers as long
+    /// as they are against the same index Directory.  Any 
+    /// snapshot held when a writer is closed will "survive"
+    /// when the next writer is opened.
+    /// <para>
+    /// WARNING: This API is new and experimental and may suddnely changendex. 
+    /// </para>
 	/// </summary>
 	
 	public class SnapshotDeletionPolicy : IndexDeletionPolicy
 	{
 		
-		private IndexCommitPoint lastCommit;
+		private IndexCommit lastCommit;
 		private IndexDeletionPolicy primary;
-		private IndexCommitPoint snapshot;
+		private string snapshot;
 		
 		public SnapshotDeletionPolicy(IndexDeletionPolicy primary)
 		{
 			this.primary = primary;
 		}
-		
-		public virtual void  OnInit(System.Collections.IList commits)
+
+        public virtual void OnInit(List<IndexCommitPoint> commits)
 		{
 			lock (this)
 			{
 				primary.OnInit(WrapCommits(commits));
-				lastCommit = (IndexCommitPoint) commits[commits.Count - 1];
+				lastCommit = (IndexCommit)(commits[commits.Count - 1]);
 			}
 		}
-		
-		public virtual void  OnCommit(System.Collections.IList commits)
+
+        public virtual void OnCommit(List<IndexCommitPoint> commits)
 		{
 			lock (this)
 			{
 				primary.OnCommit(WrapCommits(commits));
-				lastCommit = (IndexCommitPoint) commits[commits.Count - 1];
+                lastCommit = (IndexCommit)(commits[commits.Count - 1]);
 			}
 		}
 		
@@ -72,15 +81,16 @@
 		/// consume an extra 1X of your total index size, until
 		/// you release the snapshot. 
 		/// </summary>
+        /// // TODO: 3.9: change this to return IndexCommit instead
 		public virtual IndexCommitPoint Snapshot()
 		{
 			lock (this)
 			{
 				if (snapshot == null)
-					snapshot = lastCommit;
+					snapshot = lastCommit.GetSegmentsFileName();
 				else
 					throw new System.SystemException("snapshot is already set; please call release() first");
-				return snapshot;
+				return lastCommit;
 			}
 		}
 		
@@ -111,8 +121,8 @@
 				}
 				
 			}
-			internal IndexCommitPoint cp;
-			internal MyCommitPoint(SnapshotDeletionPolicy enclosingInstance, IndexCommitPoint cp)
+			internal IndexCommit cp;
+			internal MyCommitPoint(SnapshotDeletionPolicy enclosingInstance, IndexCommit cp)
 			{
 				InitBlock(enclosingInstance);
 				this.cp = cp;
@@ -121,28 +131,36 @@
 			{
 				return cp.GetSegmentsFileName();
 			}
-			public virtual System.Collections.ICollection GetFileNames()
+			public virtual System.Collections.Generic.ICollection<string> GetFileNames()
 			{
 				return cp.GetFileNames();
 			}
+            public Directory GetDirectory()
+            {
+                return cp.GetDirectory();
+            }
 			public virtual void  Delete()
 			{
 				lock (Enclosing_Instance)
 				{
 					// Suppress the delete request if this commit point is
 					// our current snapshot.
-					if (Enclosing_Instance.snapshot != cp)
+					if (Enclosing_Instance.snapshot == null || !Enclosing_Instance.snapshot.Equals(GetSegmentsFileName()))
 						cp.Delete();
 				}
 			}
+            public bool IsDeleted() { return cp.IsDeleted(); }
+            public int GetVersion() { return (int) cp.GetVersion(); }
+            public int GetGeneration() { return (int) cp.GetGeneration(); }
+
 		}
-		
-		private System.Collections.IList WrapCommits(System.Collections.IList commits)
+
+        private List<IndexCommitPoint> WrapCommits(List<IndexCommitPoint> commits)
 		{
 			int count = commits.Count;
-			System.Collections.IList myCommits = new System.Collections.ArrayList(count);
+            List<IndexCommitPoint> myCommits = new List<IndexCommitPoint>(count);
 			for (int i = 0; i < count; i++)
-				myCommits.Add(new MyCommitPoint(this, (IndexCommitPoint) commits[i]));
+				myCommits.Add(new MyCommitPoint(this, (IndexCommit) commits[i]));
 			return myCommits;
 		}
 	}

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SortedTermVectorMapper.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/SortedTermVectorMapper.cs?rev=798995&r1=798994&r2=798995&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SortedTermVectorMapper.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/SortedTermVectorMapper.cs Wed Jul 29 18:04:12 2009
@@ -32,7 +32,7 @@
 	{
 		
 		
-		private System.Collections.Generic.SortedDictionary<Object, Object> currentSet;
+		private System.Collections.Generic.SortedDictionary<object, object> currentSet;
 		private System.Collections.IDictionary termToTVE = new System.Collections.Hashtable();
 		private bool storeOffsets;
 		private bool storePositions;
@@ -42,14 +42,14 @@
 		/// <summary> </summary>
 		/// <param name="comparator">A Comparator for sorting {@link TermVectorEntry}s
 		/// </param>
-		public SortedTermVectorMapper(System.Collections.Generic.IComparer<Object> comparator) : this(false, false, comparator)
+		public SortedTermVectorMapper(System.Collections.Generic.IComparer<object> comparator) : this(false, false, comparator)
 		{
 		}
 		
 		
-		public SortedTermVectorMapper(bool ignoringPositions, bool ignoringOffsets, System.Collections.Generic.IComparer<Object> comparator) : base(ignoringPositions, ignoringOffsets)
+		public SortedTermVectorMapper(bool ignoringPositions, bool ignoringOffsets, System.Collections.Generic.IComparer<object> comparator) : base(ignoringPositions, ignoringOffsets)
 		{
-			currentSet = new System.Collections.Generic.SortedDictionary<Object,Object>(comparator);
+			currentSet = new System.Collections.Generic.SortedDictionary<object,object>(comparator);
 		}
 		
 		/// <summary> </summary>
@@ -124,7 +124,7 @@
 		/// </summary>
 		/// <returns> The SortedSet of {@link TermVectorEntry}.
 		/// </returns>
-		public virtual System.Collections.Generic.SortedDictionary<Object, Object> GetTermVectorEntrySet()
+		public virtual System.Collections.Generic.SortedDictionary<object, object> GetTermVectorEntrySet()
 		{
 			return currentSet;
 		}

Added: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/StoredFieldWriterPerThread.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/StoredFieldWriterPerThread.cs?rev=798995&view=auto
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/StoredFieldWriterPerThread.cs (added)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/StoredFieldWriterPerThread.cs Wed Jul 29 18:04:12 2009
@@ -0,0 +1,78 @@
+/**
+ * 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 IndexOutput = Lucene.Net.Store.IndexOutput;
+
+namespace Lucene.Net.Index
+{
+    internal sealed class StoredFieldsWriterPerThread : DocFieldConsumerPerThread
+    {
+
+        internal readonly FieldsWriter localFieldsWriter;
+        internal readonly StoredFieldsWriter storedFieldsWriter;
+        internal readonly DocumentsWriter.DocState docState;
+
+        internal StoredFieldsWriter.PerDoc doc;
+
+        public StoredFieldsWriterPerThread(DocFieldProcessorPerThread docFieldProcessorPerThread, StoredFieldsWriter storedFieldsWriter)
+        {
+            this.storedFieldsWriter = storedFieldsWriter;
+            this.docState = docFieldProcessorPerThread.docState;
+            localFieldsWriter = new FieldsWriter((IndexOutput)null, (IndexOutput)null, storedFieldsWriter.fieldInfos);
+        }
+
+        internal override void startDocument()
+        {
+            if (doc != null)
+            {
+                // Only happens if previous document hit non-aborting
+                // exception while writing stored fields into
+                // localFieldsWriter:
+                doc.reset();
+                doc.docID = docState.docID;
+            }
+        }
+
+        internal override DocumentsWriter.DocWriter finishDocument()
+        {
+            // If there were any stored fields in this doc, doc will
+            // be non-null; else it's null.
+            try
+            {
+                return doc;
+            }
+            finally
+            {
+                doc = null;
+            }
+        }
+
+        internal override void abort()
+        {
+            if (doc != null)
+            {
+                doc.Abort();
+                doc = null;
+            }
+        }
+
+        internal override DocFieldConsumerPerField addField(FieldInfo fieldInfo)
+        {
+            return new StoredFieldsWriterPerField(this, fieldInfo);
+        }
+    }
+}

Added: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/StoredFieldsWriter.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/StoredFieldsWriter.cs?rev=798995&view=auto
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/StoredFieldsWriter.cs (added)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/StoredFieldsWriter.cs Wed Jul 29 18:04:12 2009
@@ -0,0 +1,248 @@
+/**
+ * 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.Collections.Generic;
+
+using ArrayUtil = Lucene.Net.Util.ArrayUtil;
+using RAMOutputStream = Lucene.Net.Store.RAMOutputStream;
+
+namespace Lucene.Net.Index
+{
+    /** This is a DocFieldConsumer that writes stored fields. */
+    internal sealed class StoredFieldsWriter : DocFieldConsumer
+    {
+
+        internal FieldsWriter fieldsWriter;
+        internal readonly DocumentsWriter docWriter;
+        internal int lastDocID;
+
+        internal PerDoc[] docFreeList = new PerDoc[1];
+        internal int freeCount;
+
+        public StoredFieldsWriter(DocumentsWriter docWriter)
+        {
+            this.docWriter = docWriter;
+        }
+
+        internal override DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread)
+        {
+            return new StoredFieldsWriterPerThread(docFieldProcessorPerThread, this);
+        }
+
+        internal override void flush(IDictionary<object, ICollection<object>> threadsAndFields, DocumentsWriter.FlushState state)
+        {
+            lock (this)
+            {
+
+                if (state.numDocsInStore > 0)
+                {
+                    // It's possible that all documents seen in this segment
+                    // hit non-aborting exceptions, in which case we will
+                    // not have yet init'd the FieldsWriter:
+                    initFieldsWriter();
+
+                    // Fill fdx file to include any final docs that we
+                    // skipped because they hit non-aborting exceptions
+                    fill(state.numDocsInStore - docWriter.GetDocStoreOffset());
+                }
+
+                if (fieldsWriter != null)
+                    fieldsWriter.Flush();
+            }
+        }
+
+        private void initFieldsWriter()
+        {
+            if (fieldsWriter == null)
+            {
+                string docStoreSegment = docWriter.GetDocStoreSegment();
+                if (docStoreSegment != null)
+                {
+                    System.Diagnostics.Debug.Assert(docStoreSegment != null);
+                    fieldsWriter = new FieldsWriter(docWriter.directory,
+                                                    docStoreSegment,
+                                                    fieldInfos);
+                    docWriter.AddOpenFile(docStoreSegment + "." + IndexFileNames.FIELDS_EXTENSION);
+                    docWriter.AddOpenFile(docStoreSegment + "." + IndexFileNames.FIELDS_INDEX_EXTENSION);
+                    lastDocID = 0;
+                }
+            }
+        }
+
+        internal override void closeDocStore(DocumentsWriter.FlushState state)
+        {
+            lock (this)
+            {
+                int inc = state.numDocsInStore - lastDocID;
+                if (inc > 0)
+                {
+                    initFieldsWriter();
+                    fill(state.numDocsInStore - docWriter.GetDocStoreOffset());
+                }
+
+                if (fieldsWriter != null)
+                {
+                    fieldsWriter.Close();
+                    fieldsWriter = null;
+                    lastDocID = 0;
+                    System.Diagnostics.Debug.Assert(state.docStoreSegmentName != null);
+
+                    string fdtFile = state.docStoreSegmentName + "." + IndexFileNames.FIELDS_EXTENSION;
+                    string fdxFile = state.docStoreSegmentName + "." + IndexFileNames.FIELDS_INDEX_EXTENSION;
+
+                    state.flushedFiles[fdtFile] = fdtFile;
+                    state.flushedFiles[fdxFile] = fdxFile;
+
+                    state.docWriter.RemoveOpenFile(fdtFile);
+                    state.docWriter.RemoveOpenFile(fdxFile);
+
+                    if (4 + state.numDocsInStore * 8 != state.directory.FileLength(fdxFile))
+                        throw new System.SystemException("after flush: fdx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.FileLength(fdxFile) + " length in bytes of " + fdxFile);
+                }
+            }
+        }
+
+        internal int allocCount;
+
+        internal PerDoc getPerDoc() {
+            lock (this)
+            {
+                if (freeCount == 0)
+                {
+                    allocCount++;
+                    if (allocCount > docFreeList.Length)
+                    {
+                        // Grow our free list up front to make sure we have
+                        // enough space to recycle all outstanding PerDoc
+                        // instances
+                        System.Diagnostics.Debug.Assert(allocCount == 1 + docFreeList.Length);
+                        docFreeList = new PerDoc[ArrayUtil.GetNextSize(allocCount)];
+                    }
+                    return new PerDoc(this);
+                }
+                else
+                    return docFreeList[--freeCount];
+            }
+        }
+
+        internal override void Abort()
+        {
+            lock (this)
+            {
+                if (fieldsWriter != null)
+                {
+                    try
+                    {
+                        fieldsWriter.Close();
+                    }
+                    catch (System.Exception)
+                    {
+                    }
+                    fieldsWriter = null;
+                    lastDocID = 0;
+                }
+            }
+        }
+
+        /** Fills in any hole in the docIDs */
+        internal void fill(int docID)
+        {
+            int docStoreOffset = docWriter.GetDocStoreOffset();
+
+            // We must "catch up" for all docs before us
+            // that had no stored fields:
+            int end = docID + docStoreOffset;
+            while (lastDocID < end)
+            {
+                fieldsWriter.SkipDocument();
+                lastDocID++;
+            }
+        }
+
+        internal void finishDocument(PerDoc perDoc)
+        {
+            lock (this)
+            {
+                System.Diagnostics.Debug.Assert(docWriter.writer.TestPoint("StoredFieldsWriter.finishDocument start"));
+                initFieldsWriter();
+
+                fill(perDoc.docID);
+
+                // Append stored fields to the real FieldsWriter:
+                fieldsWriter.FlushDocument(perDoc.numStoredFields, perDoc.fdt);
+                lastDocID++;
+                perDoc.reset();
+                free(perDoc);
+                System.Diagnostics.Debug.Assert(docWriter.writer.TestPoint("StoredFieldsWriter.finishDocument end"));
+            }
+        }
+
+        internal override bool freeRAM()
+        {
+            return false;
+        }
+
+        internal void free(PerDoc perDoc)
+        {
+            lock (this)
+            {
+                System.Diagnostics.Debug.Assert(freeCount < docFreeList.Length);
+                System.Diagnostics.Debug.Assert(0 == perDoc.numStoredFields);
+                System.Diagnostics.Debug.Assert(0 == perDoc.fdt.Length());
+                System.Diagnostics.Debug.Assert(0 == perDoc.fdt.GetFilePointer());
+                docFreeList[freeCount++] = perDoc;
+            }
+        }
+
+        internal class PerDoc : DocumentsWriter.DocWriter
+        {
+            // TODO: use something more memory efficient; for small
+            // docs the 1024 buffer size of RAMOutputStream wastes alot
+            internal RAMOutputStream fdt = new RAMOutputStream();
+            internal int numStoredFields;
+
+            private StoredFieldsWriter enclosing_instance;
+            
+            internal PerDoc(StoredFieldsWriter enclosing_instance)
+            {
+                this.enclosing_instance = enclosing_instance;
+            }
+
+            internal void reset()
+            {
+                fdt.Reset();
+                numStoredFields = 0;
+            }
+
+            internal override void Abort()
+            {
+                reset();
+                enclosing_instance.free(this);
+            }
+
+            internal override long SizeInBytes()
+            {
+                return fdt.SizeInBytes();
+            }
+
+            internal override void Finish()
+            {
+                enclosing_instance.finishDocument(this);
+            }
+        }
+    }
+}

Added: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/StoredFieldsWriterPerField.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/StoredFieldsWriterPerField.cs?rev=798995&view=auto
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/StoredFieldsWriterPerField.cs (added)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/StoredFieldsWriterPerField.cs Wed Jul 29 18:04:12 2009
@@ -0,0 +1,73 @@
+/**
+ * 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 Fieldable = Lucene.Net.Documents.Fieldable;
+
+namespace Lucene.Net.Index
+{
+    internal sealed class StoredFieldsWriterPerField : DocFieldConsumerPerField
+    {
+
+        internal readonly StoredFieldsWriterPerThread perThread;
+        internal readonly FieldInfo fieldInfo;
+        internal readonly DocumentsWriter.DocState docState;
+
+        public StoredFieldsWriterPerField(StoredFieldsWriterPerThread perThread, FieldInfo fieldInfo)
+        {
+            this.perThread = perThread;
+            this.fieldInfo = fieldInfo;
+            docState = perThread.docState;
+        }
+
+        // Process all occurrences of a single field in one doc;
+        // count is 1 if a given field occurs only once in the
+        // Document, which is the "typical" case
+        internal override void processFields(Fieldable[] fields, int count)
+        {
+            StoredFieldsWriter.PerDoc doc;
+            if (perThread.doc == null)
+            {
+                doc = perThread.doc = perThread.storedFieldsWriter.getPerDoc();
+                doc.docID = docState.docID;
+                perThread.localFieldsWriter.SetFieldsStream(doc.fdt);
+                System.Diagnostics.Debug.Assert(doc.numStoredFields == 0, "doc.numStoredFields=" + doc.numStoredFields);
+                System.Diagnostics.Debug.Assert(0 == doc.fdt.Length());
+                System.Diagnostics.Debug.Assert(0 == doc.fdt.GetFilePointer());
+            }
+            else
+            {
+                doc = perThread.doc;
+                System.Diagnostics.Debug.Assert(doc.docID == docState.docID, "doc.docID=" + doc.docID + " docState.docID=" + docState.docID);
+            }
+
+            for (int i = 0; i < count; i++)
+            {
+                Fieldable field = fields[i];
+                if (field.IsStored())
+                {
+                    perThread.localFieldsWriter.WriteField(fieldInfo, field);
+                    System.Diagnostics.Debug.Assert(docState.TestPoint("StoredFieldsWriterPerField.processFields.writeField"));
+                    doc.numStoredFields++;
+                }
+            }
+        }
+
+        internal override void abort()
+        {
+        }
+    }
+}

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/Term.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/Term.cs?rev=798995&r1=798994&r2=798995&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/Term.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/Term.cs Wed Jul 29 18:04:12 2009
@@ -46,6 +46,18 @@
 		public Term(System.String fld, System.String txt) : this(fld, txt, true)
 		{
 		}
+
+        /// <summary>Constructs a Term with the given field and empty text.
+        /// This serves 2 purposes:
+        /// 1) reuse of a Term with the same field
+        /// 2) pattern for a query
+        /// </summary>
+        /// <param name="fld"/>
+        public Term(System.String fld)
+            : this(fld, "", true)
+        {
+        }
+
 		internal Term(System.String fld, System.String txt, bool intern)
 		{
 			field = intern ? String.Intern(fld) : fld; // field names are interned
@@ -84,7 +96,7 @@
 		/// <summary>Compares two terms, returning true iff they have the same
 		/// field and text. 
 		/// </summary>
-		public  override bool Equals(System.Object o)
+		public  override bool Equals(object o)
 		{
 			if (o == this)
 				return true;
@@ -102,7 +114,7 @@
 			return field.GetHashCode() + text.GetHashCode();
 		}
 		
-		public int CompareTo(System.Object other)
+		public int CompareTo(object other)
 		{
 			return CompareTo((Term) other);
 		}
@@ -133,7 +145,7 @@
 			return field + ":" + text;
 		}
 		
-		public void  GetObjectData(System.Runtime.Serialization.SerializationInfo info, System.Runtime.Serialization.StreamingContext context)
+		public void  GetobjectData(System.Runtime.Serialization.SerializationInfo info, System.Runtime.Serialization.StreamingContext context)
 		{
 			System.Type thisType = this.GetType();
 			System.Reflection.MemberInfo[] mi = System.Runtime.Serialization.FormatterServices.GetSerializableMembers(thisType, context);

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermBuffer.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/TermBuffer.cs?rev=798995&r1=798994&r2=798995&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermBuffer.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermBuffer.cs Wed Jul 29 18:04:12 2009
@@ -18,35 +18,37 @@
 using System;
 
 using IndexInput = Lucene.Net.Store.IndexInput;
+using UnicodeUtil = Lucene.Net.Util.UnicodeUtil;
 
 namespace Lucene.Net.Index
 {
 	
 	sealed class TermBuffer : System.ICloneable
 	{
-		private static readonly char[] NO_CHARS = new char[0];
-		
 		private System.String field;
-		private char[] text = NO_CHARS;
-		private int textLength;
 		private Term term; // cached
+        private bool preUTF8Strings; // true if strings are stored in "modified UTF-8" encoding
+        private bool dirty; // true if text was set externally (i.e., not read via UTF-8 bytes)
+
+        private UnicodeUtil.UTF16Result text = new UnicodeUtil.UTF16Result();
+        private UnicodeUtil.UTF8Result bytes = new UnicodeUtil.UTF8Result();
 		
 		public int CompareTo(TermBuffer other)
 		{
 			if (field == other.field)
 				// fields are interned
-				return CompareChars(text, textLength, other.text, other.textLength);
+				return CompareChars(text.result, text.length, other.text.result, other.text.length);
 			else
 				return String.CompareOrdinal(field, other.field);
 		}
 		
-		private static int CompareChars(char[] v1, int len1, char[] v2, int len2)
+		private static int CompareChars(char[] chars1, int len1, char[] chars2, int len2)
 		{
-			int end = System.Math.Min(len1, len2);
+			int end = len1 < len2 ? len1 : len2;
 			for (int k = 0; k < end; k++)
 			{
-				char c1 = v1[k];
-				char c2 = v2[k];
+				char c1 = chars1[k];
+				char c2 = chars2[k];
 				if (c1 != c2)
 				{
 					return c1 - c2;
@@ -54,17 +56,15 @@
 			}
 			return len1 - len2;
 		}
-		
-		private void  SetTextLength(int newLength)
-		{
-			if (text.Length < newLength)
-			{
-				char[] newText = new char[newLength];
-				Array.Copy(text, 0, newText, 0, textLength);
-				text = newText;
-			}
-			textLength = newLength;
-		}
+
+        /// <summary>
+        /// Call this if the IndexInput passed to Read() stores terms
+        /// in the modified UTF-8 (pre-LUCENE-510) format.
+        /// </summary>
+        internal void SetPreUTF8Strings()
+        {
+            preUTF8Strings = true;
+        }
 		
 		public void  Read(IndexInput input, FieldInfos fieldInfos)
 		{
@@ -72,9 +72,31 @@
 			int start = input.ReadVInt();
 			int length = input.ReadVInt();
 			int totalLength = start + length;
-			SetTextLength(totalLength);
-			input.ReadChars(this.text, start, length);
-			this.field = fieldInfos.FieldName(input.ReadVInt());
+            if (preUTF8Strings)
+            {
+                text.setLength(totalLength);
+                input.ReadChars(text.result, start, length);
+            }
+            else
+            {
+                if (dirty)
+                {
+                    // fully convert all bytes since bytes is dirty
+                    UnicodeUtil.UTF16toUTF8(text.result, 0, text.length, bytes);
+                    bytes.setLength(totalLength);
+                    input.ReadBytes(bytes.result, start, length);
+                    UnicodeUtil.UTF8toUTF16(bytes.result, 0, totalLength, text);
+                    dirty = false;
+                }
+                else
+                {
+                    // incrementally convert only the UTF-8 bytes that are new
+                    bytes.setLength(totalLength);
+                    input.ReadBytes(bytes.result, start, length);
+                    UnicodeUtil.UTF8toUTF16(bytes.result, start, length, text);
+                }
+            }
+            this.field = fieldInfos.FieldName(input.ReadVInt());
 		}
 		
 		public void  Set(Term term)
@@ -84,35 +106,32 @@
 				Reset();
 				return ;
 			}
-			
-			// copy text into the buffer
-			SetTextLength(term.Text().Length);
-
-			System.String sourceString = term.Text();
-			int sourceEnd = term.Text().Length;
-			for (int i = 0; i < sourceEnd; i++)
+            string termText = term.Text();
+            int termLen = termText.Length;
+            text.setLength(termLen);
+			for (int i = 0; i < termLen; i++)
 			{
-				text[i] = (char) sourceString[i];
+				text.result[i] = (char) termText[i];
 			}
-			
-			this.field = term.Field();
+			dirty = true;
+			field = term.Field();
 			this.term = term;
 		}
 		
 		public void  Set(TermBuffer other)
 		{
-			SetTextLength(other.textLength);
-			Array.Copy(other.text, 0, text, 0, textLength);
-			
-			this.field = other.field;
-			this.term = other.term;
+            text.copyText(other.text);
+            dirty = true;
+			field = other.field;
+			term = other.term;
 		}
 		
 		public void  Reset()
 		{
-			this.field = null;
-			this.textLength = 0;
-			this.term = null;
+			field = null;
+			text.setLength(0);
+			term = null;
+            dirty = true;
 		}
 		
 		public Term ToTerm()
@@ -122,12 +141,12 @@
 				return null;
 			
 			if (term == null)
-				term = new Term(field, new System.String(text, 0, textLength), false);
+				term = new Term(field, new System.String(text.result, 0, text.length), false);
 			
 			return term;
 		}
 		
-		public System.Object Clone()
+		public object Clone()
 		{
 			TermBuffer clone = null;
 			try
@@ -137,9 +156,11 @@
 			catch (System.Exception)
 			{
 			}
-			
-			clone.text = new char[text.Length];
-			Array.Copy(text, 0, clone.text, 0, textLength);
+
+            clone.dirty = true;
+            clone.bytes = new UnicodeUtil.UTF8Result();
+            clone.text = new UnicodeUtil.UTF16Result();
+            clone.text.copyText(text);
 			
 			return clone;
 		}

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermInfosReader.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/TermInfosReader.cs?rev=798995&r1=798994&r2=798995&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermInfosReader.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermInfosReader.cs Wed Jul 29 18:04:12 2009
@@ -18,6 +18,9 @@
 using System;
 
 using BufferedIndexInput = Lucene.Net.Store.BufferedIndexInput;
+using Cache = Lucene.Net.Util.Cache.Cache;
+using SimpleLRUCache = Lucene.Net.Util.Cache.SimpleLRUCache;
+using CloseableThreadLocal = Lucene.Net.Util.CloseableThreadLocal;
 using Directory = Lucene.Net.Store.Directory;
 
 namespace Lucene.Net.Index
@@ -33,8 +36,8 @@
 		private Directory directory;
 		private System.String segment;
 		private FieldInfos fieldInfos;
-		
-		private System.LocalDataStoreSlot enumerators = System.Threading.Thread.AllocateDataSlot();
+
+        private CloseableThreadLocal threadResources = new CloseableThreadLocal();
 		private SegmentTermEnum origEnum;
 		private long size;
 		
@@ -46,9 +49,22 @@
 		
 		private int indexDivisor = 1;
 		private int totalIndexInterval;
-		
-		internal TermInfosReader(Directory dir, System.String seg, FieldInfos fis) : this(dir, seg, fis, BufferedIndexInput.BUFFER_SIZE)
-		{
+	
+        private const int DEFAULT_CACHE_SIZE = 1024;
+
+        /// <summary>
+        /// Per-thread resources managed by ThreadLocal.
+        /// </summary>
+        internal sealed class ThreadResources
+        {
+            internal SegmentTermEnum termEnum;
+            // used for caching the least recently looked-up Terms
+            internal Cache termInfoCache;
+        }
+
+        internal TermInfosReader(Directory dir, System.String seg, FieldInfos fis)
+            : this(dir, seg, fis, BufferedIndexInput.BUFFER_SIZE)
+        {
 		}
 		
 		internal TermInfosReader(Directory dir, System.String seg, FieldInfos fis, int readBufferSize)
@@ -61,11 +77,11 @@
 				segment = seg;
 				fieldInfos = fis;
 				
-				origEnum = new SegmentTermEnum(directory.OpenInput(segment + ".tis", readBufferSize), fieldInfos, false);
+				origEnum = new SegmentTermEnum(directory.OpenInput(segment + "." + IndexFileNames.TERMS_EXTENSION, readBufferSize), fieldInfos, false);
 				size = origEnum.size;
 				totalIndexInterval = origEnum.indexInterval;
 				
-				indexEnum = new SegmentTermEnum(directory.OpenInput(segment + ".tii", readBufferSize), fieldInfos, true);
+				indexEnum = new SegmentTermEnum(directory.OpenInput(segment + "." + IndexFileNames.TERMS_INDEX_EXTENSION, readBufferSize), fieldInfos, true);
 				
 				success = true;
 			}
@@ -137,6 +153,7 @@
 				origEnum.Close();
 			if (indexEnum != null)
 				indexEnum.Close();
+            threadResources.Close();
 		}
 		
 		/// <summary>Returns the number of term/value pairs in the set. </summary>
@@ -144,18 +161,21 @@
 		{
 			return size;
 		}
-		
-		private SegmentTermEnum GetEnum()
-		{
-			SegmentTermEnum termEnum = (SegmentTermEnum) System.Threading.Thread.GetData(enumerators);
-			if (termEnum == null)
-			{
-				termEnum = Terms();
-				System.Threading.Thread.SetData(enumerators, termEnum);
-			}
-			return termEnum;
-		}
-		
+
+        internal ThreadResources GetThreadResources()
+        {
+            ThreadResources resources = (ThreadResources)threadResources.Get();
+            if (resources == null)
+            {
+                resources = new ThreadResources();
+                resources.termEnum = Terms();
+                // cache does not have to be thread-safe, it is only used by one thread at the same time
+                resources.termInfoCache = new SimpleLRUCache(DEFAULT_CACHE_SIZE);
+                threadResources.Set(resources);
+            }
+            return resources;
+        }
+
 		private void  EnsureIndexIsRead()
 		{
 			lock (this)
@@ -210,61 +230,104 @@
 			return hi;
 		}
 		
-		private void  SeekEnum(int indexOffset)
+		private void  SeekEnum(SegmentTermEnum enumerator, int indexOffset)
 		{
-			GetEnum().Seek(indexPointers[indexOffset], (indexOffset * totalIndexInterval) - 1, indexTerms[indexOffset], indexInfos[indexOffset]);
+			enumerator.Seek(indexPointers[indexOffset], (indexOffset * totalIndexInterval) - 1, indexTerms[indexOffset], indexInfos[indexOffset]);
 		}
-		
+
+        /// <summary>Returns the TermInfo for a Term in the set, or null. </summary>
+        internal TermInfo Get(Term term)
+        {
+            return Get(term, true);
+        }
+
 		/// <summary>Returns the TermInfo for a Term in the set, or null. </summary>
-		public TermInfo Get(Term term)
+		public TermInfo Get(Term term, bool useCache)
 		{
 			if (size == 0)
 				return null;
 			
 			EnsureIndexIsRead();
-			
-			// optimize sequential access: first try scanning cached enum w/o seeking
-			SegmentTermEnum enumerator = GetEnum();
+
+            TermInfo ti;
+            ThreadResources resources = GetThreadResources();
+            Cache cache = null;
+
+            if (useCache)
+            {
+                cache = resources.termInfoCache;
+                // check the cache first if the term was recently looked up
+                ti = (TermInfo)cache.Get(term);
+                if (ti != null)
+                {
+                    return ti;
+                }
+            }
+
+            // optimize sequential access: first try scanning cached enum w/o seeking
+			SegmentTermEnum enumerator = resources.termEnum;
 			if (enumerator.Term() != null && ((enumerator.Prev() != null && term.CompareTo(enumerator.Prev()) > 0) || term.CompareTo(enumerator.Term()) >= 0))
 			{
 				int enumOffset = (int) (enumerator.position / totalIndexInterval) + 1;
-				if (indexTerms.Length == enumOffset || term.CompareTo(indexTerms[enumOffset]) < 0)
-					return ScanEnum(term); // no need to seek
+                if (indexTerms.Length == enumOffset || term.CompareTo(indexTerms[enumOffset]) < 0)
+                {
+                    int numScans = enumerator.ScanTo(term);
+                    if (enumerator.Term() != null && term.CompareTo(enumerator.Term()) == 0)
+                    {
+                        ti = enumerator.TermInfo();
+                        if (cache != null && numScans > 1)
+                        {
+                            // we only want to put this TermInfo into the cache if
+                            // scanEnum skipped more than one dictionary entry.
+                            // this prevents RangeQueries or WildcardQueries from
+                            // wiping out the cache when they iterate over a large number
+                            // of terms in order
+                            cache.Put(term, ti);
+                        }
+                    }
+                    else
+                    {
+                        ti = null;
+                    }
+                    return ti;
+                }
 			}
 			
 			// random-access: must seek
-			SeekEnum(GetIndexOffset(term));
-			return ScanEnum(term);
-		}
-		
-		/// <summary>Scans within block for matching term. </summary>
-		private TermInfo ScanEnum(Term term)
-		{
-			SegmentTermEnum enumerator = GetEnum();
-			enumerator.ScanTo(term);
-			if (enumerator.Term() != null && term.CompareTo(enumerator.Term()) == 0)
-				return enumerator.TermInfo();
-			else
-				return null;
+			SeekEnum(enumerator, GetIndexOffset(term));
+            enumerator.ScanTo(term);
+            if (enumerator.Term() != null && term.CompareTo(enumerator.Term()) == 0)
+            {
+                ti = enumerator.TermInfo();
+                if (cache != null)
+                {
+                    cache.Put(term, ti);
+                }
+            }
+            else
+            {
+                ti = null;
+            }
+
+            return ti;
 		}
-		
+				
 		/// <summary>Returns the nth term in the set. </summary>
 		internal Term Get(int position)
 		{
 			if (size == 0)
 				return null;
 			
-			SegmentTermEnum enumerator = GetEnum();
+			SegmentTermEnum enumerator = GetThreadResources().termEnum;
 			if (enumerator != null && enumerator.Term() != null && position >= enumerator.position && position < (enumerator.position + totalIndexInterval))
-				return ScanEnum(position); // can avoid seek
+				return ScanEnum(enumerator, position); // can avoid seek
 			
-			SeekEnum(position / totalIndexInterval); // must seek
-			return ScanEnum(position);
+			SeekEnum(enumerator, position / totalIndexInterval); // must seek
+			return ScanEnum(enumerator, position);
 		}
 		
-		private Term ScanEnum(int position)
+		private Term ScanEnum(SegmentTermEnum enumerator, int position)
 		{
-			SegmentTermEnum enumerator = GetEnum();
 			while (enumerator.position < position)
 				if (!enumerator.Next())
 					return null;
@@ -280,9 +343,10 @@
 			
 			EnsureIndexIsRead();
 			int indexOffset = GetIndexOffset(term);
-			SeekEnum(indexOffset);
+
+            SegmentTermEnum enumerator = GetThreadResources().termEnum;
+			SeekEnum(enumerator, indexOffset);
 			
-			SegmentTermEnum enumerator = GetEnum();
 			while (term.CompareTo(enumerator.Term()) > 0 && enumerator.Next())
 			{
 			}
@@ -302,8 +366,9 @@
 		/// <summary>Returns an enumeration of terms starting at or after the named term. </summary>
 		public SegmentTermEnum Terms(Term term)
 		{
-			Get(term);
-			return (SegmentTermEnum) GetEnum().Clone();
+            // don't use the cache in this call because we want to reposition the enumeration
+			Get(term, false);
+			return (SegmentTermEnum) GetThreadResources().termEnum.Clone();
 		}
 	}
 }
\ No newline at end of file

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermInfosWriter.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/TermInfosWriter.cs?rev=798995&r1=798994&r2=798995&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermInfosWriter.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermInfosWriter.cs Wed Jul 29 18:04:12 2009
@@ -19,6 +19,7 @@
 
 using Directory = Lucene.Net.Store.Directory;
 using IndexOutput = Lucene.Net.Store.IndexOutput;
+using UnicodeUtil = Lucene.Net.Util.UnicodeUtil;
 
 namespace Lucene.Net.Index
 {
@@ -32,6 +33,12 @@
 		/// <summary>The file format version, a negative number. </summary>
 		public const int FORMAT = - 3;
 		
+        // changed strings to true utf8 with length in bytes not length in chars
+        public const int FORMAT_VERSION_UTF8_LENGTH_IN_BYTES = -4;
+
+        // NOTE: always chage this if you switch to a new format!
+        public static readonly int FORMAT_CURRENT = FORMAT_VERSION_UTF8_LENGTH_IN_BYTES;
+
 		private FieldInfos fieldInfos;
 		private IndexOutput output;
 		private TermInfo lastTi = new TermInfo();
@@ -67,14 +74,13 @@
 		
 		private long lastIndexPointer;
 		private bool isIndex;
-		private char[] lastTermText = new char[10];
-		private int lastTermTextLength;
-		private int lastFieldNumber = - 1;
-		
-		private char[] termTextBuffer = new char[10];
+		private byte[] lastTermBytes = new byte[10];
+		private int lastTermBytesLength = 0;
+		private int lastFieldNumber = -1;
 		
 		private TermInfosWriter other;
-		
+        private UnicodeUtil.UTF8Result utf8Result = new UnicodeUtil.UTF8Result();
+
 		public TermInfosWriter(Directory directory, System.String segment, FieldInfos fis, int interval)
 		{
 			Initialize(directory, segment, fis, interval, false);
@@ -86,44 +92,43 @@
 		{
 			Initialize(directory, segment, fis, interval, isIndex);
 		}
+
+        private void Initialize(Directory directory, System.String segment, FieldInfos fis, int interval, bool isi)
+        {
+            indexInterval = interval;
+            fieldInfos = fis;
+            isIndex = isi;
+            output = directory.CreateOutput(segment + (isIndex ? ".tii" : ".tis"));
+            output.WriteInt(FORMAT_CURRENT); // write format
+            output.WriteLong(0); // leave space for size
+            output.WriteInt(indexInterval); // write indexInterval
+            output.WriteInt(skipInterval); // write skipInterval
+            output.WriteInt(maxSkipLevels); // write maxSkipLevels
+            System.Diagnostics.Debug.Assert(InitUTF16Results());
+        }
 		
-		private void  Initialize(Directory directory, System.String segment, FieldInfos fis, int interval, bool isi)
-		{
-			indexInterval = interval;
-			fieldInfos = fis;
-			isIndex = isi;
-			output = directory.CreateOutput(segment + (isIndex ? ".tii" : ".tis"));
-			output.WriteInt(FORMAT); // write format
-			output.WriteLong(0); // leave space for size
-			output.WriteInt(indexInterval); // write indexInterval
-			output.WriteInt(skipInterval); // write skipInterval
-			output.WriteInt(maxSkipLevels); // write maxSkipLevels
-		}
 		
 		internal void  Add(Term term, TermInfo ti)
 		{
-			
-			int length = term.text.Length;
-			if (termTextBuffer.Length < length)
-			{
-				termTextBuffer = new char[(int) (length * 1.25)];
-			}
+            UnicodeUtil.UTF16toUTF8(term.text, 0, term.text.Length, utf8Result);
+            Add(fieldInfos.FieldNumber(term.field), utf8Result.result, utf8Result.length, ti);
+        }
+
+        // currently used only by assert statements
+        UnicodeUtil.UTF16Result utf16Result1;
+        UnicodeUtil.UTF16Result utf16Result2;
+
+        // currently used only by assert statements
+        private bool InitUTF16Results()
+        {
+            utf16Result1 = new UnicodeUtil.UTF16Result();
+            utf16Result2 = new UnicodeUtil.UTF16Result();
+			return true;
+        }
 
-            int i = 0;
-            System.Collections.Generic.IEnumerator<char> chars = term.text.GetEnumerator();
-            while (chars.MoveNext())
-            {
-                termTextBuffer[i++] = (char)chars.Current;
-            }
-			
-			Add(fieldInfos.FieldNumber(term.field), termTextBuffer, 0, length, ti);
-		}
-		
 		// Currently used only by assert statement
-		private int CompareToLastTerm(int fieldNumber, char[] termText, int start, int length)
+		private int CompareToLastTerm(int fieldNumber, byte[] termBytes, int termBytesLength)
 		{
-			int pos = 0;
-			
 			if (lastFieldNumber != fieldNumber)
 			{
 				int cmp = String.CompareOrdinal(fieldInfos.FieldName(lastFieldNumber), fieldInfos.FieldName(fieldNumber));
@@ -135,47 +140,44 @@
 					return cmp;
 			}
 			
-			while (pos < length && pos < lastTermTextLength)
-			{
-				char c1 = lastTermText[pos];
-				char c2 = termText[pos + start];
-				if (c1 < c2)
-					return - 1;
-				else if (c1 > c2)
-					return 1;
-				pos++;
-			}
-			
-			if (pos < lastTermTextLength)
-			// Last term was longer
-				return 1;
-			else if (pos < length)
-			// Last term was shorter
-				return - 1;
-			else
-				return 0;
+            UnicodeUtil.UTF8toUTF16(lastTermBytes, 0, lastTermBytesLength, utf16Result1);
+            UnicodeUtil.UTF8toUTF16(termBytes, 0, termBytesLength, utf16Result2);
+
+            int len;
+            if (utf16Result1.length < utf16Result2.length)
+                len = utf16Result1.length;
+            else
+                len = utf16Result2.length;
+
+            for (int i = 0; i < len; i++)
+            {
+                char ch1 = utf16Result1.result[i];
+                char ch2 = utf16Result2.result[i];
+                if (ch1 != ch2)
+                    return ch1 - ch2;
+            }
+            return utf16Result1.length - utf16Result2.length;
 		}
 		
-		/// <summary>Adds a new <<fieldNumber, termText>, TermInfo> pair to the set.
+		/// <summary>Adds a new <<fieldNumber, termBytes>, TermInfo> pair to the set.
 		/// Term must be lexicographically greater than all previous Terms added.
 		/// TermInfo pointers must be positive and greater than all previous.
 		/// </summary>
-		internal void  Add(int fieldNumber, char[] termText, int termTextStart, int termTextLength, TermInfo ti)
+		internal void  Add(int fieldNumber, byte[] termBytes, int termBytesLength, TermInfo ti)
 		{
-			
-			System.Diagnostics.Debug.Assert(CompareToLastTerm(fieldNumber, termText, termTextStart, termTextLength) < 0 ||
-				(isIndex && termTextLength == 0 && lastTermTextLength == 0),
+			System.Diagnostics.Debug.Assert(CompareToLastTerm(fieldNumber, termBytes, termBytesLength) < 0 ||
+				(isIndex && termBytesLength == 0 && lastTermBytesLength == 0),
 				"Terms are out of order: field=" + fieldInfos.FieldName(fieldNumber) +  "(number " + fieldNumber + ")" + 
 				" lastField=" + fieldInfos.FieldName(lastFieldNumber) + " (number " + lastFieldNumber + ")" + 
-				" text=" + new String(termText, termTextStart, termTextLength) + " lastText=" + new String(lastTermText, 0, lastTermTextLength));
+				" text=" + System.Text.Encoding.UTF8.GetString(termBytes, 0, termBytesLength) + " lastText=" + System.Text.Encoding.UTF8.GetString(lastTermBytes, 0, lastTermBytesLength));
 			
 			System.Diagnostics.Debug.Assert(ti.freqPointer >= lastTi.freqPointer, "freqPointer out of order (" + ti.freqPointer + " < " + lastTi.freqPointer + ")");
 			System.Diagnostics.Debug.Assert(ti.proxPointer >= lastTi.proxPointer, "proxPointer out of order (" + ti.proxPointer + " < " + lastTi.proxPointer + ")");
 			
 			if (!isIndex && size % indexInterval == 0)
-				other.Add(lastFieldNumber, lastTermText, 0, lastTermTextLength, lastTi); // add an index term
+				other.Add(lastFieldNumber, lastTermBytes, lastTermBytesLength, lastTi); // add an index term
 			
-			WriteTerm(fieldNumber, termText, termTextStart, termTextLength); // write term
+			WriteTerm(fieldNumber, termBytes, termBytesLength); // write term
 			
 			output.WriteVInt(ti.docFreq); // write doc freq
 			output.WriteVLong(ti.freqPointer - lastTi.freqPointer); // write pointers
@@ -192,37 +194,38 @@
 				lastIndexPointer = other.output.GetFilePointer(); // write pointer
 			}
 			
-			if (lastTermText.Length < termTextLength)
-			{
-				lastTermText = new char[(int) (termTextLength * 1.25)];
-			}
-			Array.Copy(termText, termTextStart, lastTermText, 0, termTextLength);
-			lastTermTextLength = termTextLength;
 			lastFieldNumber = fieldNumber;
-			
 			lastTi.Set(ti);
 			size++;
 		}
 		
-		private void  WriteTerm(int fieldNumber, char[] termText, int termTextStart, int termTextLength)
+		private void  WriteTerm(int fieldNumber, byte[] termBytes, int termBytesLength)
 		{
-			
+            // TODO: UTF16toUTF8 could tell us this prefix
 			// Compute prefix in common with last term:
 			int start = 0;
-			int limit = termTextLength < lastTermTextLength ? termTextLength : lastTermTextLength;
+			int limit = termBytesLength < lastTermBytesLength ? termBytesLength : lastTermBytesLength;
 			while (start < limit)
 			{
-				if (termText[termTextStart + start] != lastTermText[start])
+				if (termBytes[start] != lastTermBytes[start])
 					break;
 				start++;
 			}
 			
-			int length = termTextLength - start;
+			int length = termBytesLength - start;
 			
 			output.WriteVInt(start); // write shared prefix length
 			output.WriteVInt(length); // write delta length
-			output.WriteChars(termText, start + termTextStart, length); // write delta chars
+			output.WriteBytes(termBytes, start, length); // write delta chars
 			output.WriteVInt(fieldNumber); // write field num
+            if (lastTermBytes.Length < termBytesLength)
+            {
+                byte[] newArray = new byte[(int)(termBytesLength*1.5)];
+                Array.Copy(lastTermBytes, 0, newArray, 0, start);
+                lastTermBytes = newArray;
+            }
+            Array.Copy(termBytes, start, lastTermBytes, start, length);
+            lastTermBytesLength = termBytesLength;
 		}
 		
 		/// <summary>Called to complete TermInfos creation. </summary>

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorEntry.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/TermVectorEntry.cs?rev=798995&r1=798994&r2=798995&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorEntry.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorEntry.cs Wed Jul 29 18:04:12 2009
@@ -86,7 +86,7 @@
 		}
 		
 		
-		public  override bool Equals(System.Object o)
+		public  override bool Equals(object o)
 		{
 			if (this == o)
 				return true;