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 [10/10] - /incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermPositions.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/TermPositions.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermPositions.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermPositions.cs Tue Jun 24 19:52:22 2008
@@ -38,5 +38,44 @@
 		/// the first time.
 		/// </summary>
 		int NextPosition();
+		
+		/// <summary> Returns the length of the payload at the current term position.
+		/// This is invalid until {@link #NextPosition()} is called for
+		/// the first time.<br>
+		/// </summary>
+		/// <returns> length of the current payload in number of bytes
+		/// </returns>
+		int GetPayloadLength();
+		
+		/// <summary> Returns the payload data at the current term position.
+		/// This is invalid until {@link #NextPosition()} is called for
+		/// the first time.
+		/// This method must not be called more than once after each call
+		/// of {@link #NextPosition()}. However, payloads are loaded lazily,
+		/// so if the payload data for the current position is not needed,
+		/// this method may not be called at all for performance reasons.<br>
+		/// 
+		/// </summary>
+		/// <param name="data">the array into which the data of this payload is to be
+		/// stored, if it is big enough; otherwise, a new byte[] array
+		/// is allocated for this purpose. 
+		/// </param>
+		/// <param name="offset">the offset in the array into which the data of this payload
+		/// is to be stored.
+		/// </param>
+		/// <returns> a byte[] array containing the data of this payload
+		/// </returns>
+		/// <throws>  IOException </throws>
+		byte[] GetPayload(byte[] data, int offset);
+		
+		/// <summary> Checks if a payload can be loaded at this position.
+		/// <p>
+		/// Payloads can only be loaded once per call to 
+		/// {@link #NextPosition()}.
+		/// 
+		/// </summary>
+		/// <returns> true if there is a payload available at this position that can be loaded
+		/// </returns>
+		bool IsPayloadAvailable();
 	}
 }
\ No newline at end of file

Added: 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=671404&view=auto
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorEntry.cs (added)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorEntry.cs Tue Jun 24 19:52:22 2008
@@ -0,0 +1,114 @@
+/*
+ * 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> Convenience class for holding TermVector information.</summary>
+	public class TermVectorEntry
+	{
+		private System.String field;
+		private System.String term;
+		private int frequency;
+		private TermVectorOffsetInfo[] offsets;
+		internal int[] positions;
+		
+		
+		public TermVectorEntry()
+		{
+		}
+		
+		public TermVectorEntry(System.String field, System.String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions)
+		{
+			this.field = field;
+			this.term = term;
+			this.frequency = frequency;
+			this.offsets = offsets;
+			this.positions = positions;
+		}
+		
+		
+		public virtual System.String GetField()
+		{
+			return field;
+		}
+		
+		public virtual int GetFrequency()
+		{
+			return frequency;
+		}
+		
+		public virtual TermVectorOffsetInfo[] GetOffsets()
+		{
+			return offsets;
+		}
+		
+		public virtual int[] GetPositions()
+		{
+			return positions;
+		}
+		
+		public virtual System.String GetTerm()
+		{
+			return term;
+		}
+		
+		//Keep package local
+		internal virtual void  SetFrequency(int frequency)
+		{
+			this.frequency = frequency;
+		}
+		
+		internal virtual void  SetOffsets(TermVectorOffsetInfo[] offsets)
+		{
+			this.offsets = offsets;
+		}
+		
+		internal virtual void  SetPositions(int[] positions)
+		{
+			this.positions = positions;
+		}
+		
+		
+		public  override bool Equals(System.Object o)
+		{
+			if (this == o)
+				return true;
+			if (o == null || GetType() != o.GetType())
+				return false;
+			
+			TermVectorEntry that = (TermVectorEntry) o;
+			
+			if (term != null ? !term.Equals(that.term) : that.term != null)
+				return false;
+			
+			return true;
+		}
+		
+		public override int GetHashCode()
+		{
+			return (term != null ? term.GetHashCode() : 0);
+		}
+		
+		public override System.String ToString()
+		{
+			return "TermVectorEntry{" + "field='" + field + '\'' + ", term='" + term + '\'' + ", frequency=" + frequency + '}';
+		}
+	}
+}
\ No newline at end of file

Added: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorEntryFreqSortedComparator.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/TermVectorEntryFreqSortedComparator.cs?rev=671404&view=auto
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorEntryFreqSortedComparator.cs (added)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorEntryFreqSortedComparator.cs Tue Jun 24 19:52:22 2008
@@ -0,0 +1,47 @@
+/*
+ * 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> Compares {@link Lucene.Net.Index.TermVectorEntry}s first by frequency and then by
+	/// the term (case-sensitive)
+	/// 
+	/// 
+	/// </summary>
+	public class TermVectorEntryFreqSortedComparator : System.Collections.IComparer
+	{
+		public virtual int Compare(System.Object object_Renamed, System.Object object1)
+		{
+			int result = 0;
+			TermVectorEntry entry = (TermVectorEntry) object_Renamed;
+			TermVectorEntry entry1 = (TermVectorEntry) object1;
+			result = entry1.GetFrequency() - entry.GetFrequency();
+			if (result == 0)
+			{
+				result = String.CompareOrdinal(entry.GetTerm(), entry1.GetTerm());
+				if (result == 0)
+				{
+					result = String.CompareOrdinal(entry.GetField(), entry1.GetField());
+				}
+			}
+			return result;
+		}
+	}
+}
\ No newline at end of file

Added: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorMapper.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/TermVectorMapper.cs?rev=671404&view=auto
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorMapper.cs (added)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorMapper.cs Tue Jun 24 19:52:22 2008
@@ -0,0 +1,115 @@
+/*
+ * 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> The TermVectorMapper can be used to map Term Vectors into your own
+	/// structure instead of the parallel array structure used by
+	/// {@link Lucene.Net.Index.IndexReader#GetTermFreqVector(int,String)}.
+	/// <p/>
+	/// It is up to the implementation to make sure it is thread-safe.
+	/// 
+	/// 
+	/// 
+	/// </summary>
+	public abstract class TermVectorMapper
+	{
+		
+		private bool ignoringPositions;
+		private bool ignoringOffsets;
+		
+		
+		protected internal TermVectorMapper()
+		{
+		}
+		
+		/// <summary> </summary>
+		/// <param name="ignoringPositions">true if this mapper should tell Lucene to ignore positions even if they are stored
+		/// </param>
+		/// <param name="ignoringOffsets">similar to ignoringPositions
+		/// </param>
+		protected internal TermVectorMapper(bool ignoringPositions, bool ignoringOffsets)
+		{
+			this.ignoringPositions = ignoringPositions;
+			this.ignoringOffsets = ignoringOffsets;
+		}
+		
+		/// <summary> Tell the mapper what to expect in regards to field, number of terms, offset and position storage.
+		/// This method will be called once before retrieving the vector for a field.
+		/// 
+		/// This method will be called before {@link #Map(String,int,TermVectorOffsetInfo[],int[])}.
+		/// </summary>
+		/// <param name="field">The field the vector is for
+		/// </param>
+		/// <param name="numTerms">The number of terms that need to be mapped
+		/// </param>
+		/// <param name="storeOffsets">true if the mapper should expect offset information
+		/// </param>
+		/// <param name="storePositions">true if the mapper should expect positions info
+		/// </param>
+		public abstract void  SetExpectations(System.String field, int numTerms, bool storeOffsets, bool storePositions);
+		/// <summary> Map the Term Vector information into your own structure</summary>
+		/// <param name="term">The term to add to the vector
+		/// </param>
+		/// <param name="frequency">The frequency of the term in the document
+		/// </param>
+		/// <param name="offsets">null if the offset is not specified, otherwise the offset into the field of the term
+		/// </param>
+		/// <param name="positions">null if the position is not specified, otherwise the position in the field of the term
+		/// </param>
+		public abstract void  Map(System.String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions);
+		
+		/// <summary> Indicate to Lucene that even if there are positions stored, this mapper is not interested in them and they
+		/// can be skipped over.  Derived classes should set this to true if they want to ignore positions.  The default
+		/// is false, meaning positions will be loaded if they are stored.
+		/// </summary>
+		/// <returns> false
+		/// </returns>
+		public virtual bool IsIgnoringPositions()
+		{
+			return ignoringPositions;
+		}
+		
+		/// <summary> </summary>
+		/// <seealso cref="IsIgnoringPositions() Same principal as {@link #IsIgnoringPositions()}, but applied to offsets.  false by default.">
+		/// </seealso>
+		/// <returns> false
+		/// </returns>
+		public virtual bool IsIgnoringOffsets()
+		{
+			return ignoringOffsets;
+		}
+		
+		/// <summary> Passes down the index of the document whose term vector is currently being mapped,
+		/// once for each top level call to a term vector reader.
+		/// <p/>
+		/// Default implementation IGNORES the document number.  Override if your implementation needs the document number.
+		/// <p/> 
+		/// NOTE: Document numbers are internal to Lucene and subject to change depending on indexing operations.
+		/// 
+		/// </summary>
+		/// <param name="documentNumber">index of document currently being mapped
+		/// </param>
+		public virtual void  SetDocumentNumber(int documentNumber)
+		{
+		}
+	}
+}
\ No newline at end of file

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorOffsetInfo.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/TermVectorOffsetInfo.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorOffsetInfo.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorOffsetInfo.cs Tue Jun 24 19:52:22 2008
@@ -20,8 +20,13 @@
 namespace Lucene.Net.Index
 {
 	
+	/// <summary> The TermVectorOffsetInfo class holds information pertaining to a Term in a {@link Lucene.Net.Index.TermPositionVector}'s
+	/// offset information.  This offset information is the character offset as set during the Analysis phase (and thus may not be the actual offset in the
+	/// original content).
+	/// </summary>
 	public class TermVectorOffsetInfo
 	{
+		/// <summary> Convenience declaration when creating a {@link Lucene.Net.Index.TermPositionVector} that stores only position information.</summary>
 		public static readonly TermVectorOffsetInfo[] EMPTY_OFFSET_INFO = new TermVectorOffsetInfo[0];
 		private int startOffset;
 		private int endOffset;
@@ -36,6 +41,9 @@
 			this.startOffset = startOffset;
 		}
 		
+		/// <summary> The accessor for the ending offset for the term</summary>
+		/// <returns> The offset
+		/// </returns>
 		public virtual int GetEndOffset()
 		{
 			return endOffset;
@@ -46,6 +54,11 @@
 			this.endOffset = endOffset;
 		}
 		
+		/// <summary> The accessor for the starting offset of the term.
+		/// 
+		/// </summary>
+		/// <returns> The offset
+		/// </returns>
 		public virtual int GetStartOffset()
 		{
 			return startOffset;
@@ -56,6 +69,11 @@
 			this.startOffset = startOffset;
 		}
 		
+		/// <summary> Two TermVectorOffsetInfos are equals if both the start and end offsets are the same</summary>
+		/// <param name="o">The comparison Object
+		/// </param>
+		/// <returns> true if both {@link #GetStartOffset()} and {@link #GetEndOffset()} are the same for both objects.
+		/// </returns>
 		public  override bool Equals(System.Object o)
 		{
 			if (this == o)

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorsReader.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/TermVectorsReader.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorsReader.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorsReader.cs Tue Jun 24 19:52:22 2008
@@ -16,16 +16,26 @@
  */
 
 using System;
+
+using BufferedIndexInput = Lucene.Net.Store.BufferedIndexInput;
 using Directory = Lucene.Net.Store.Directory;
 using IndexInput = Lucene.Net.Store.IndexInput;
 
 namespace Lucene.Net.Index
 {
 	
-	/// <version>  $Id: TermVectorsReader.java 472959 2006-11-09 16:21:50Z yonik $
+	/// <version>  $Id: TermVectorsReader.java 601337 2007-12-05 13:59:37Z mikemccand $
 	/// </version>
-	public class TermVectorsReader : System.ICloneable
+	class TermVectorsReader : System.ICloneable
 	{
+		
+		internal const int FORMAT_VERSION = 2;
+		//The size in bytes that the FORMAT_VERSION will take up at the beginning of each file 
+		internal const int FORMAT_SIZE = 4;
+		
+		internal const byte STORE_POSITIONS_WITH_TERMVECTOR = (byte) (0x1);
+		internal const byte STORE_OFFSET_WITH_TERMVECTOR = (byte) (0x2);
+		
 		private FieldInfos fieldInfos;
 		
 		private IndexInput tvx;
@@ -33,31 +43,73 @@
 		private IndexInput tvf;
 		private int size;
 		
+		// 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 int tvdFormat;
 		private int tvfFormat;
 		
-		public TermVectorsReader(Directory d, System.String segment, FieldInfos fieldInfos)
+		internal TermVectorsReader(Directory d, System.String segment, FieldInfos fieldInfos) : this(d, segment, fieldInfos, BufferedIndexInput.BUFFER_SIZE)
 		{
-			if (d.FileExists(segment + TermVectorsWriter.TVX_EXTENSION))
+		}
+		
+		internal TermVectorsReader(Directory d, System.String segment, FieldInfos fieldInfos, int readBufferSize) : this(d, segment, fieldInfos, BufferedIndexInput.BUFFER_SIZE, - 1, 0)
+		{
+		}
+		
+		internal TermVectorsReader(Directory d, System.String segment, FieldInfos fieldInfos, int readBufferSize, int docStoreOffset, int size)
+		{
+			bool success = false;
+			
+			try
+			{
+				if (d.FileExists(segment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION))
+				{
+					tvx = d.OpenInput(segment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION, readBufferSize);
+					CheckValidFormat(tvx);
+					tvd = d.OpenInput(segment + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION, readBufferSize);
+					tvdFormat = CheckValidFormat(tvd);
+					tvf = d.OpenInput(segment + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION, readBufferSize);
+					tvfFormat = CheckValidFormat(tvf);
+					if (- 1 == docStoreOffset)
+					{
+						this.docStoreOffset = 0;
+						this.size = (int) (tvx.Length() >> 3);
+					}
+					else
+					{
+						this.docStoreOffset = docStoreOffset;
+						this.size = size;
+						// Verify the file is long enough to hold all of our
+						// docs
+						System.Diagnostics.Debug.Assert(((int) (tvx.Length() / 8)) >= size + docStoreOffset);
+					}
+				}
+				
+				this.fieldInfos = fieldInfos;
+				success = true;
+			}
+			finally
 			{
-				tvx = d.OpenInput(segment + TermVectorsWriter.TVX_EXTENSION);
-				CheckValidFormat(tvx);
-				tvd = d.OpenInput(segment + TermVectorsWriter.TVD_EXTENSION);
-				tvdFormat = CheckValidFormat(tvd);
-				tvf = d.OpenInput(segment + TermVectorsWriter.TVF_EXTENSION);
-				tvfFormat = CheckValidFormat(tvf);
-				size = (int) tvx.Length() / 8;
+				// 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();
+				}
 			}
-			
-			this.fieldInfos = fieldInfos;
 		}
 		
 		private int CheckValidFormat(IndexInput in_Renamed)
 		{
 			int format = in_Renamed.ReadInt();
-			if (format > TermVectorsWriter.FORMAT_VERSION)
+			if (format > FORMAT_VERSION)
 			{
-				throw new System.IO.IOException("Incompatible format version: " + format + " expected " + TermVectorsWriter.FORMAT_VERSION + " or less");
+				throw new CorruptIndexException("Incompatible format version: " + format + " expected " + FORMAT_VERSION + " or less");
 			}
 			return format;
 		}
@@ -111,26 +163,16 @@
 			return size;
 		}
 		
-		/// <summary> Retrieve the term vector for the given document and field</summary>
-		/// <param name="docNum">The document number to retrieve the vector for
-		/// </param>
-		/// <param name="field">The field within the document to retrieve
-		/// </param>
-		/// <returns> The TermFreqVector for the document and field or null if there is no termVector for this field.
-		/// </returns>
-		/// <throws>  IOException if there is an error reading the term vector files </throws>
-		public virtual TermFreqVector Get(int docNum, System.String field)
+		public virtual void  Get(int docNum, System.String field, TermVectorMapper mapper)
 		{
-			// Check if no term vectors are available for this segment at all
-			int fieldNumber = fieldInfos.FieldNumber(field);
-			TermFreqVector result = null;
 			if (tvx != null)
 			{
+				int fieldNumber = fieldInfos.FieldNumber(field);
 				//We need to account for the FORMAT_SIZE at when seeking in the tvx
 				//We don't need to do this in other seeks because we already have the
 				// file pointer
 				//that was written in another file
-				tvx.Seek((docNum * 8L) + TermVectorsWriter.FORMAT_SIZE);
+				tvx.Seek(((docNum + docStoreOffset) * 8L) + FORMAT_SIZE);
 				//System.out.println("TVX Pointer: " + tvx.getFilePointer());
 				long position = tvx.ReadLong();
 				
@@ -144,7 +186,7 @@
 				int found = - 1;
 				for (int i = 0; i < fieldCount; i++)
 				{
-					if (tvdFormat == TermVectorsWriter.FORMAT_VERSION)
+					if (tvdFormat == FORMAT_VERSION)
 						number = tvd.ReadVInt();
 					else
 						number += tvd.ReadVInt();
@@ -162,7 +204,8 @@
 					for (int i = 0; i <= found; i++)
 						position += tvd.ReadVLong();
 					
-					result = ReadTermVector(field, position);
+					mapper.SetDocumentNumber(docNum);
+					ReadTermVector(field, position, mapper);
 				}
 				else
 				{
@@ -173,7 +216,25 @@
 			{
 				//System.out.println("No tvx file");
 			}
-			return result;
+		}
+		
+		
+		
+		/// <summary> Retrieve the term vector for the given document and field</summary>
+		/// <param name="docNum">The document number to retrieve the vector for
+		/// </param>
+		/// <param name="field">The field within the document to retrieve
+		/// </param>
+		/// <returns> The TermFreqVector for the document and field or null if there is no termVector for this field.
+		/// </returns>
+		/// <throws>  IOException if there is an error reading the term vector files </throws>
+		internal virtual TermFreqVector Get(int docNum, System.String field)
+		{
+			// Check if no term vectors are available for this segment at all
+			ParallelArrayTermVectorMapper mapper = new ParallelArrayTermVectorMapper();
+			Get(docNum, field, mapper);
+			
+			return mapper.MaterializeVector();
 		}
 		
 		/// <summary> Return all term vectors stored for this document or null if the could not be read in.
@@ -184,14 +245,13 @@
 		/// <returns> All term frequency vectors
 		/// </returns>
 		/// <throws>  IOException if there is an error reading the term vector files  </throws>
-		public virtual TermFreqVector[] Get(int docNum)
+		internal virtual TermFreqVector[] Get(int docNum)
 		{
 			TermFreqVector[] result = null;
-			// Check if no term vectors are available for this segment at all
 			if (tvx != null)
 			{
 				//We need to offset by
-				tvx.Seek((docNum * 8L) + TermVectorsWriter.FORMAT_SIZE);
+				tvx.Seek(((docNum + docStoreOffset) * 8L) + FORMAT_SIZE);
 				long position = tvx.ReadLong();
 				
 				tvd.Seek(position);
@@ -205,7 +265,7 @@
 					
 					for (int i = 0; i < fieldCount; i++)
 					{
-						if (tvdFormat == TermVectorsWriter.FORMAT_VERSION)
+						if (tvdFormat == FORMAT_VERSION)
 							number = tvd.ReadVInt();
 						else
 							number += tvd.ReadVInt();
@@ -222,7 +282,7 @@
 						tvfPointers[i] = position;
 					}
 					
-					result = ReadTermVectors(fields, tvfPointers);
+					result = ReadTermVectors(docNum, fields, tvfPointers);
 				}
 			}
 			else
@@ -232,26 +292,87 @@
 			return result;
 		}
 		
+		public virtual void  Get(int docNumber, TermVectorMapper mapper)
+		{
+			// Check if no term vectors are available for this segment at all
+			if (tvx != null)
+			{
+				//We need to offset by
+				tvx.Seek((docNumber * 8L) + FORMAT_SIZE);
+				long position = tvx.ReadLong();
+				
+				tvd.Seek(position);
+				int fieldCount = tvd.ReadVInt();
+				
+				// No fields are vectorized for this document
+				if (fieldCount != 0)
+				{
+					int number = 0;
+					System.String[] fields = new System.String[fieldCount];
+					
+					for (int i = 0; i < fieldCount; i++)
+					{
+						if (tvdFormat == FORMAT_VERSION)
+							number = tvd.ReadVInt();
+						else
+							number += tvd.ReadVInt();
+						
+						fields[i] = fieldInfos.FieldName(number);
+					}
+					
+					// Compute position in the tvf file
+					position = 0;
+					long[] tvfPointers = new long[fieldCount];
+					for (int i = 0; i < fieldCount; i++)
+					{
+						position += tvd.ReadVLong();
+						tvfPointers[i] = position;
+					}
+					
+					mapper.SetDocumentNumber(docNumber);
+					ReadTermVectors(fields, tvfPointers, mapper);
+				}
+			}
+			else
+			{
+				//System.out.println("No tvx file");
+			}
+		}
+		
 		
-		private SegmentTermVector[] ReadTermVectors(System.String[] fields, long[] tvfPointers)
+		private SegmentTermVector[] ReadTermVectors(int docNum, System.String[] fields, long[] tvfPointers)
 		{
 			SegmentTermVector[] res = new SegmentTermVector[fields.Length];
 			for (int i = 0; i < fields.Length; i++)
 			{
-				res[i] = ReadTermVector(fields[i], tvfPointers[i]);
+				ParallelArrayTermVectorMapper mapper = new ParallelArrayTermVectorMapper();
+				mapper.SetDocumentNumber(docNum);
+				ReadTermVector(fields[i], tvfPointers[i], mapper);
+				res[i] = (SegmentTermVector) mapper.MaterializeVector();
 			}
 			return res;
 		}
 		
+		private void  ReadTermVectors(System.String[] fields, long[] tvfPointers, TermVectorMapper mapper)
+		{
+			for (int i = 0; i < fields.Length; i++)
+			{
+				ReadTermVector(fields[i], tvfPointers[i], mapper);
+			}
+		}
+		
+		
 		/// <summary> </summary>
 		/// <param name="field">The field to read in
 		/// </param>
 		/// <param name="tvfPointer">The pointer within the tvf file where we should start reading
 		/// </param>
+		/// <param name="mapper">The mapper used to map the TermVector
+		/// </param>
 		/// <returns> The TermVector located at that position
 		/// </returns>
 		/// <throws>  IOException </throws>
-		private SegmentTermVector ReadTermVector(System.String field, long tvfPointer)
+		private void  ReadTermVector(System.String field, long tvfPointer, TermVectorMapper mapper)
 		{
 			
 			// Now read the data from specified position
@@ -262,16 +383,16 @@
 			//System.out.println("Num Terms: " + numTerms);
 			// If no terms - return a constant empty termvector. However, this should never occur!
 			if (numTerms == 0)
-				return new SegmentTermVector(field, null, null);
+				return ;
 			
 			bool storePositions;
 			bool storeOffsets;
 			
-			if (tvfFormat == TermVectorsWriter.FORMAT_VERSION)
+			if (tvfFormat == FORMAT_VERSION)
 			{
 				byte bits = tvf.ReadByte();
-				storePositions = (bits & TermVectorsWriter.STORE_POSITIONS_WITH_TERMVECTOR) != 0;
-				storeOffsets = (bits & TermVectorsWriter.STORE_OFFSET_WITH_TERMVECTOR) != 0;
+				storePositions = (bits & STORE_POSITIONS_WITH_TERMVECTOR) != 0;
+				storeOffsets = (bits & STORE_OFFSET_WITH_TERMVECTOR) != 0;
 			}
 			else
 			{
@@ -279,18 +400,7 @@
 				storePositions = false;
 				storeOffsets = false;
 			}
-			
-			System.String[] terms = new System.String[numTerms];
-			int[] termFreqs = new int[numTerms];
-			
-			//  we may not need these, but declare them
-			int[][] positions = null;
-			TermVectorOffsetInfo[][] offsets = null;
-			if (storePositions)
-				positions = new int[numTerms][];
-			if (storeOffsets)
-				offsets = new TermVectorOffsetInfo[numTerms][];
-			
+			mapper.SetExpectations(field, numTerms, storeOffsets, storePositions);
 			int start = 0;
 			int deltaLength = 0;
 			int totalLength = 0;
@@ -309,56 +419,70 @@
 					buffer = new char[totalLength];
 					
 					if (start > 0)
-						// just copy if necessary
+					// just copy if necessary
 						Array.Copy(previousBuffer, 0, buffer, 0, start);
 				}
 				
 				tvf.ReadChars(buffer, start, deltaLength);
-				terms[i] = new System.String(buffer, 0, totalLength);
+				System.String term = new System.String(buffer, 0, totalLength);
 				previousBuffer = buffer;
 				int freq = tvf.ReadVInt();
-				termFreqs[i] = freq;
-				
+				int[] positions = null;
 				if (storePositions)
 				{
 					//read in the positions
-					int[] pos = new int[freq];
-					positions[i] = pos;
-					int prevPosition = 0;
-					for (int j = 0; j < freq; j++)
+					//does the mapper even care about positions?
+					if (mapper.IsIgnoringPositions() == false)
 					{
-						pos[j] = prevPosition + tvf.ReadVInt();
-						prevPosition = pos[j];
+						positions = new int[freq];
+						int prevPosition = 0;
+						for (int j = 0; j < freq; j++)
+						{
+							positions[j] = prevPosition + tvf.ReadVInt();
+							prevPosition = positions[j];
+						}
+					}
+					else
+					{
+						//we need to skip over the positions.  Since these are VInts, I don't believe there is anyway to know for sure how far to skip
+						//
+						for (int j = 0; j < freq; j++)
+						{
+							tvf.ReadVInt();
+						}
 					}
 				}
-				
+				TermVectorOffsetInfo[] offsets = null;
 				if (storeOffsets)
 				{
-					TermVectorOffsetInfo[] offs = new TermVectorOffsetInfo[freq];
-					offsets[i] = offs;
-					int prevOffset = 0;
-					for (int j = 0; j < freq; j++)
+					//does the mapper even care about offsets?
+					if (mapper.IsIgnoringOffsets() == false)
+					{
+						offsets = new TermVectorOffsetInfo[freq];
+						int prevOffset = 0;
+						for (int j = 0; j < freq; j++)
+						{
+							int startOffset = prevOffset + tvf.ReadVInt();
+							int endOffset = startOffset + tvf.ReadVInt();
+							offsets[j] = new TermVectorOffsetInfo(startOffset, endOffset);
+							prevOffset = endOffset;
+						}
+					}
+					else
 					{
-						int startOffset = prevOffset + tvf.ReadVInt();
-						int endOffset = startOffset + tvf.ReadVInt();
-						offs[j] = new TermVectorOffsetInfo(startOffset, endOffset);
-						prevOffset = endOffset;
+						for (int j = 0; j < freq; j++)
+						{
+							tvf.ReadVInt();
+							tvf.ReadVInt();
+						}
 					}
 				}
+				mapper.Map(term, freq, offsets, positions);
 			}
-			
-			SegmentTermVector tv;
-			if (storePositions || storeOffsets)
-			{
-				tv = new SegmentTermPositionVector(field, terms, termFreqs, positions, offsets);
-			}
-			else
-			{
-				tv = new SegmentTermVector(field, terms, termFreqs);
-			}
-			return tv;
 		}
 		
+		
+		
 		public virtual System.Object Clone()
 		{
 			
@@ -370,7 +494,7 @@
 			{
 				clone = (TermVectorsReader) base.MemberwiseClone();
 			}
-			catch (System.Exception)
+			catch (System.Exception e)
 			{
 			}
 			
@@ -381,4 +505,66 @@
 			return clone;
 		}
 	}
+	
+	/// <summary> Models the existing parallel array structure</summary>
+	class ParallelArrayTermVectorMapper:TermVectorMapper
+	{
+		
+		private System.String[] terms;
+		private int[] termFreqs;
+		private int[][] positions;
+		private TermVectorOffsetInfo[][] offsets;
+		private int currentPosition;
+		private bool storingOffsets;
+		private bool storingPositions;
+		private System.String field;
+		
+		public override void  SetExpectations(System.String field, int numTerms, bool storeOffsets, bool storePositions)
+		{
+			this.field = field;
+			terms = new System.String[numTerms];
+			termFreqs = new int[numTerms];
+			this.storingOffsets = storeOffsets;
+			this.storingPositions = storePositions;
+			if (storePositions)
+				this.positions = new int[numTerms][];
+			if (storeOffsets)
+				this.offsets = new TermVectorOffsetInfo[numTerms][];
+		}
+		
+		public override void  Map(System.String term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions)
+		{
+			terms[currentPosition] = term;
+			termFreqs[currentPosition] = frequency;
+			if (storingOffsets)
+			{
+				this.offsets[currentPosition] = offsets;
+			}
+			if (storingPositions)
+			{
+				this.positions[currentPosition] = positions;
+			}
+			currentPosition++;
+		}
+		
+		/// <summary> Construct the vector</summary>
+		/// <returns> The {@link TermFreqVector} based on the mappings.
+		/// </returns>
+		public virtual TermFreqVector MaterializeVector()
+		{
+			SegmentTermVector tv = null;
+			if (field != null && terms != null)
+			{
+				if (storingPositions || storingOffsets)
+				{
+					tv = new SegmentTermPositionVector(field, terms, termFreqs, positions, offsets);
+				}
+				else
+				{
+					tv = new SegmentTermVector(field, terms, termFreqs);
+				}
+			}
+			return tv;
+		}
+	}
 }
\ No newline at end of file

Modified: incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorsWriter.cs
URL: http://svn.apache.org/viewvc/incubator/lucene.net/trunk/C%23/src/Lucene.Net/Index/TermVectorsWriter.cs?rev=671404&r1=671403&r2=671404&view=diff
==============================================================================
--- incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorsWriter.cs (original)
+++ incubator/lucene.net/trunk/C#/src/Lucene.Net/Index/TermVectorsWriter.cs Tue Jun 24 19:52:22 2008
@@ -16,6 +16,7 @@
  */
 
 using System;
+
 using Directory = Lucene.Net.Store.Directory;
 using IndexOutput = Lucene.Net.Store.IndexOutput;
 using StringHelper = Lucene.Net.Util.StringHelper;
@@ -23,404 +24,189 @@
 namespace Lucene.Net.Index
 {
 	
-	/// <summary> Writer works by opening a document and then opening the fields within the document and then
-	/// writing out the vectors for each field.
-	/// 
-	/// Rough usage:
-	/// 
-	/// <CODE>
-	/// for each document
-	/// {
-	/// writer.openDocument();
-	/// for each field on the document
-	/// {
-	/// writer.openField(field);
-	/// for all of the terms
-	/// {
-	/// writer.addTerm(...)
-	/// }
-	/// writer.closeField
-	/// }
-	/// writer.closeDocument()    
-	/// }
-	/// </CODE>
-	/// 
-	/// </summary>
-	/// <version>  $Id: TermVectorsWriter.java 472959 2006-11-09 16:21:50Z yonik $
-	/// 
-	/// </version>
 	public sealed class TermVectorsWriter
 	{
-		internal const byte STORE_POSITIONS_WITH_TERMVECTOR = (byte) (0x1);
-		internal const byte STORE_OFFSET_WITH_TERMVECTOR = (byte) (0x2);
-		
-		internal const int FORMAT_VERSION = 2;
-		//The size in bytes that the FORMAT_VERSION will take up at the beginning of each file 
-		internal const int FORMAT_SIZE = 4;
-		
-		internal const System.String TVX_EXTENSION = ".tvx";
-		internal const System.String TVD_EXTENSION = ".tvd";
-		internal const System.String TVF_EXTENSION = ".tvf";
 		
 		private IndexOutput tvx = null, tvd = null, tvf = null;
-		private System.Collections.ArrayList fields = null;
-		private System.Collections.ArrayList terms = null;
 		private FieldInfos fieldInfos;
 		
-		private TVField currentField = null;
-		private long currentDocPointer = - 1;
-
-		// Those three get'ers are helper for Lucene.Net only
-		public static System.String TvxExtension
-		{
-			get {   return TVX_EXTENSION;   }
-		}
-		public static System.String TvdExtension
-		{
-			get {   return TVD_EXTENSION;   }
-		}
-		public static System.String TvfExtension
-		{
-			get {   return TVF_EXTENSION;   }
-		}
-
 		public TermVectorsWriter(Directory directory, System.String segment, FieldInfos fieldInfos)
 		{
 			// Open files for TermVector storage
-			tvx = directory.CreateOutput(segment + TVX_EXTENSION);
-			tvx.WriteInt(FORMAT_VERSION);
-			tvd = directory.CreateOutput(segment + TVD_EXTENSION);
-			tvd.WriteInt(FORMAT_VERSION);
-			tvf = directory.CreateOutput(segment + TVF_EXTENSION);
-			tvf.WriteInt(FORMAT_VERSION);
+			tvx = directory.CreateOutput(segment + "." + IndexFileNames.VECTORS_INDEX_EXTENSION);
+			tvx.WriteInt(TermVectorsReader.FORMAT_VERSION);
+			tvd = directory.CreateOutput(segment + "." + IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
+			tvd.WriteInt(TermVectorsReader.FORMAT_VERSION);
+			tvf = directory.CreateOutput(segment + "." + IndexFileNames.VECTORS_FIELDS_EXTENSION);
+			tvf.WriteInt(TermVectorsReader.FORMAT_VERSION);
 			
 			this.fieldInfos = fieldInfos;
-			fields = System.Collections.ArrayList.Synchronized(new System.Collections.ArrayList(fieldInfos.Size()));
-			terms = System.Collections.ArrayList.Synchronized(new System.Collections.ArrayList(10));
-		}
-		
-		
-		public void  OpenDocument()
-		{
-			CloseDocument();
-			currentDocPointer = tvd.GetFilePointer();
-		}
-		
-		
-		public void  CloseDocument()
-		{
-			if (IsDocumentOpen())
-			{
-				CloseField();
-				WriteDoc();
-				fields.Clear();
-				currentDocPointer = - 1;
-			}
-		}
-		
-		
-		public bool IsDocumentOpen()
-		{
-			return currentDocPointer != - 1;
-		}
-		
-		
-		/// <summary>Start processing a field. This can be followed by a number of calls to
-		/// addTerm, and a final call to closeField to indicate the end of
-		/// processing of this field. If a field was previously open, it is
-		/// closed automatically.
-		/// </summary>
-		public void  OpenField(System.String field)
-		{
-			FieldInfo fieldInfo = fieldInfos.FieldInfo(field);
-			OpenField(fieldInfo.number, fieldInfo.storePositionWithTermVector, fieldInfo.storeOffsetWithTermVector);
-		}
-		
-		private void  OpenField(int fieldNumber, bool storePositionWithTermVector, bool storeOffsetWithTermVector)
-		{
-			if (!IsDocumentOpen())
-				throw new System.SystemException("Cannot open field when no document is open.");
-			CloseField();
-			currentField = new TVField(fieldNumber, storePositionWithTermVector, storeOffsetWithTermVector);
-		}
-		
-		/// <summary>Finished processing current field. This should be followed by a call to
-		/// openField before future calls to addTerm.
-		/// </summary>
-		public void  CloseField()
-		{
-			if (IsFieldOpen())
-			{
-				/* DEBUG */
-				//System.out.println("closeField()");
-				/* DEBUG */
-				
-				// save field and terms
-				WriteField();
-				fields.Add(currentField);
-				terms.Clear();
-				currentField = null;
-			}
-		}
-		
-		/// <summary>Return true if a field is currently open. </summary>
-		public bool IsFieldOpen()
-		{
-			return currentField != null;
-		}
-		
-		/// <summary>Add term to the field's term vector. Fieldable must already be open.
-		/// Terms should be added in
-		/// increasing order of terms, one call per unique termNum. ProxPointer
-		/// is a pointer into the TermPosition file (prx). Freq is the number of
-		/// times this term appears in this field, in this document.
-		/// </summary>
-		/// <throws>  IllegalStateException if document or field is not open </throws>
-		public void  AddTerm(System.String termText, int freq)
-		{
-			AddTerm(termText, freq, null, null);
-		}
-		
-		public void  AddTerm(System.String termText, int freq, int[] positions, TermVectorOffsetInfo[] offsets)
-		{
-			if (!IsDocumentOpen())
-				throw new System.SystemException("Cannot add terms when document is not open");
-			if (!IsFieldOpen())
-				throw new System.SystemException("Cannot add terms when field is not open");
-			
-			AddTermInternal(termText, freq, positions, offsets);
-		}
-		
-		private void  AddTermInternal(System.String termText, int freq, int[] positions, TermVectorOffsetInfo[] offsets)
-		{
-			TVTerm term = new TVTerm();
-			term.termText = termText;
-			term.freq = freq;
-			term.positions = positions;
-			term.offsets = offsets;
-			terms.Add(term);
 		}
 		
 		/// <summary> Add a complete document specified by all its term vectors. If document has no
 		/// term vectors, add value for tvx.
 		/// 
 		/// </summary>
-		/// <param name="">vectors
+		/// <param name="vectors">
 		/// </param>
 		/// <throws>  IOException </throws>
 		public void  AddAllDocVectors(TermFreqVector[] vectors)
 		{
-			OpenDocument();
+			
+			tvx.WriteLong(tvd.GetFilePointer());
 			
 			if (vectors != null)
 			{
-				for (int i = 0; i < vectors.Length; i++)
+				int numFields = vectors.Length;
+				tvd.WriteVInt(numFields);
+				
+				long[] fieldPointers = new long[numFields];
+				
+				for (int i = 0; i < numFields; i++)
 				{
-					bool storePositionWithTermVector = false;
-					bool storeOffsetWithTermVector = false;
+					fieldPointers[i] = tvf.GetFilePointer();
+					
+					int fieldNumber = fieldInfos.FieldNumber(vectors[i].GetField());
+					
+					// 1st pass: write field numbers to tvd
+					tvd.WriteVInt(fieldNumber);
+					
+					int numTerms = vectors[i].Size();
+					tvf.WriteVInt(numTerms);
+					
+					TermPositionVector tpVector;
+					
+					byte bits;
+					bool storePositions;
+					bool storeOffsets;
 					
 					if (vectors[i] is TermPositionVector)
 					{
-						
-						TermPositionVector tpVector = (TermPositionVector) vectors[i];
-						
-						if (tpVector.Size() > 0 && tpVector.GetTermPositions(0) != null)
-							storePositionWithTermVector = true;
-						if (tpVector.Size() > 0 && tpVector.GetOffsets(0) != null)
-							storeOffsetWithTermVector = true;
-						
-						FieldInfo fieldInfo = fieldInfos.FieldInfo(tpVector.GetField());
-						OpenField(fieldInfo.number, storePositionWithTermVector, storeOffsetWithTermVector);
-						
-						for (int j = 0; j < tpVector.Size(); j++)
-							AddTermInternal(tpVector.GetTerms()[j], tpVector.GetTermFrequencies()[j], tpVector.GetTermPositions(j), tpVector.GetOffsets(j));
-						
-						CloseField();
+						// May have positions & offsets
+						tpVector = (TermPositionVector) vectors[i];
+						storePositions = tpVector.Size() > 0 && tpVector.GetTermPositions(0) != null;
+						storeOffsets = tpVector.Size() > 0 && tpVector.GetOffsets(0) != null;
+						bits = (byte) ((storePositions ? TermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR : (byte) 0) + (storeOffsets ? TermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR : (byte) 0));
 					}
 					else
 					{
-						
-						TermFreqVector tfVector = vectors[i];
-						
-						FieldInfo fieldInfo = fieldInfos.FieldInfo(tfVector.GetField());
-						OpenField(fieldInfo.number, storePositionWithTermVector, storeOffsetWithTermVector);
-						
-						for (int j = 0; j < tfVector.Size(); j++)
-							AddTermInternal(tfVector.GetTerms()[j], tfVector.GetTermFrequencies()[j], null, null);
-						
-						CloseField();
-					}
-				}
-			}
-			
-			CloseDocument();
-		}
-		
-		/// <summary>Close all streams. </summary>
-		public void  Close()
-		{
-			try
-			{
-				CloseDocument();
-			}
-			finally
-			{
-				// make an effort to close all streams we can but remember and re-throw
-				// the first exception encountered in this process
-				System.IO.IOException keep = null;
-				if (tvx != null)
-					try
-					{
-						tvx.Close();
-					}
-					catch (System.IO.IOException e)
-					{
-						if (keep == null)
-							keep = e;
-					}
-				if (tvd != null)
-					try
-					{
-						tvd.Close();
+						tpVector = null;
+						bits = 0;
+						storePositions = false;
+						storeOffsets = false;
 					}
-					catch (System.IO.IOException e)
-					{
-						if (keep == null)
-							keep = e;
-					}
-				if (tvf != null)
-					try
-					{
-						tvf.Close();
-					}
-					catch (System.IO.IOException e)
+					
+					tvf.WriteVInt(bits);
+					
+					System.String[] terms = vectors[i].GetTerms();
+					int[] freqs = vectors[i].GetTermFrequencies();
+					
+					System.String lastTermText = "";
+					for (int j = 0; j < numTerms; j++)
 					{
-						if (keep == null)
-							keep = e;
+						System.String termText = terms[j];
+						int start = StringHelper.StringDifference(lastTermText, termText);
+						int length = termText.Length - start;
+						tvf.WriteVInt(start); // write shared prefix length
+						tvf.WriteVInt(length); // write delta length
+						tvf.WriteChars(termText, start, length); // write delta chars
+						lastTermText = termText;
+						
+						int termFreq = freqs[j];
+						
+						tvf.WriteVInt(termFreq);
+						
+						if (storePositions)
+						{
+							int[] positions = tpVector.GetTermPositions(j);
+							if (positions == null)
+								throw new System.SystemException("Trying to write positions that are null!");
+							System.Diagnostics.Debug.Assert(positions.Length == termFreq);
+							
+							// use delta encoding for positions
+							int lastPosition = 0;
+							for (int k = 0; k < positions.Length; k++)
+							{
+								int position = positions[k];
+								tvf.WriteVInt(position - lastPosition);
+								lastPosition = position;
+							}
+						}
+						
+						if (storeOffsets)
+						{
+							TermVectorOffsetInfo[] offsets = tpVector.GetOffsets(j);
+							if (offsets == null)
+								throw new System.SystemException("Trying to write offsets that are null!");
+							System.Diagnostics.Debug.Assert(offsets.Length == termFreq);
+							
+							// use delta encoding for offsets
+							int lastEndOffset = 0;
+							for (int k = 0; k < offsets.Length; k++)
+							{
+								int startOffset = offsets[k].GetStartOffset();
+								int endOffset = offsets[k].GetEndOffset();
+								tvf.WriteVInt(startOffset - lastEndOffset);
+								tvf.WriteVInt(endOffset - startOffset);
+								lastEndOffset = endOffset;
+							}
+						}
 					}
-				if (keep != null)
+				}
+				
+				// 2nd pass: write field pointers to tvd
+				long lastFieldPointer = 0;
+				for (int i = 0; i < numFields; i++)
 				{
-					throw new System.IO.IOException(keep.StackTrace);
+					long fieldPointer = fieldPointers[i];
+					tvd.WriteVLong(fieldPointer - lastFieldPointer);
+					lastFieldPointer = fieldPointer;
 				}
 			}
+			else
+				tvd.WriteVInt(0);
 		}
 		
-		
-		
-		private void  WriteField()
+		/// <summary>Close all streams. </summary>
+		internal void  Close()
 		{
-			// remember where this field is written
-			currentField.tvfPointer = tvf.GetFilePointer();
-			//System.out.println("Fieldable Pointer: " + currentField.tvfPointer);
-			
-			int size = terms.Count;
-			tvf.WriteVInt(size);
-			
-			bool storePositions = currentField.storePositions;
-			bool storeOffsets = currentField.storeOffsets;
-			byte bits = (byte) (0x0);
-			if (storePositions)
-				bits |= STORE_POSITIONS_WITH_TERMVECTOR;
-			if (storeOffsets)
-				bits |= STORE_OFFSET_WITH_TERMVECTOR;
-			tvf.WriteByte(bits);
-			
-			System.String lastTermText = "";
-			for (int i = 0; i < size; i++)
-			{
-				TVTerm term = (TVTerm) terms[i];
-				int start = StringHelper.StringDifference(lastTermText, term.termText);
-				int length = term.termText.Length - start;
-				tvf.WriteVInt(start); // write shared prefix length
-				tvf.WriteVInt(length); // write delta length
-				tvf.WriteChars(term.termText, start, length); // write delta chars
-				tvf.WriteVInt(term.freq);
-				lastTermText = term.termText;
-				
-				if (storePositions)
+			// make an effort to close all streams we can but remember and re-throw
+			// the first exception encountered in this process
+			System.IO.IOException keep = null;
+			if (tvx != null)
+				try
 				{
-					if (term.positions == null)
-						throw new System.SystemException("Trying to write positions that are null!");
-					
-					// use delta encoding for positions
-					int position = 0;
-					for (int j = 0; j < term.freq; j++)
-					{
-						tvf.WriteVInt(term.positions[j] - position);
-						position = term.positions[j];
-					}
+					tvx.Close();
 				}
-				
-				if (storeOffsets)
+				catch (System.IO.IOException e)
 				{
-					if (term.offsets == null)
-						throw new System.SystemException("Trying to write offsets that are null!");
-					
-					// use delta encoding for offsets
-					int position = 0;
-					for (int j = 0; j < term.freq; j++)
-					{
-						tvf.WriteVInt(term.offsets[j].GetStartOffset() - position);
-						tvf.WriteVInt(term.offsets[j].GetEndOffset() - term.offsets[j].GetStartOffset()); //Save the diff between the two.
-						position = term.offsets[j].GetEndOffset();
-					}
+					if (keep == null)
+						keep = e;
 				}
-			}
-		}
-		
-		private void  WriteDoc()
-		{
-			if (IsFieldOpen())
-				throw new System.SystemException("Field is still open while writing document");
-			//System.out.println("Writing doc pointer: " + currentDocPointer);
-			// write document index record
-			tvx.WriteLong(currentDocPointer);
-			
-			// write document data record
-			int size = fields.Count;
-			
-			// write the number of fields
-			tvd.WriteVInt(size);
-			
-			// write field numbers
-			for (int i = 0; i < size; i++)
-			{
-				TVField field = (TVField) fields[i];
-				tvd.WriteVInt(field.number);
-			}
-			
-			// write field pointers
-			long lastFieldPointer = 0;
-			for (int i = 0; i < size; i++)
-			{
-				TVField field = (TVField) fields[i];
-				tvd.WriteVLong(field.tvfPointer - lastFieldPointer);
-				lastFieldPointer = field.tvfPointer;
-			}
-			//System.out.println("After writing doc pointer: " + tvx.getFilePointer());
-		}
-		
-		
-		private class TVField
-		{
-			internal int number;
-			internal long tvfPointer = 0;
-			internal bool storePositions = false;
-			internal bool storeOffsets = false;
-			internal TVField(int number, bool storePos, bool storeOff)
+			if (tvd != null)
+				try
+				{
+					tvd.Close();
+				}
+				catch (System.IO.IOException e)
+				{
+					if (keep == null)
+						keep = e;
+				}
+			if (tvf != null)
+				try
+				{
+					tvf.Close();
+				}
+				catch (System.IO.IOException e)
+				{
+					if (keep == null)
+						keep = e;
+				}
+			if (keep != null)
 			{
-				this.number = number;
-				storePositions = storePos;
-				storeOffsets = storeOff;
+				throw new System.IO.IOException(keep.StackTrace);
 			}
 		}
-		
-		private class TVTerm
-		{
-			internal System.String termText;
-			internal int freq = 0;
-			internal int[] positions = null;
-			internal TermVectorOffsetInfo[] offsets = null;
-		}
 	}
 }
\ No newline at end of file