You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by pn...@apache.org on 2014/09/29 09:23:56 UTC

[2/3] Lucene.Net.Codecs/Memory porting part 1

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/af4d125b/src/Lucene.Net.Codecs/Memory/FSTPulsing41PostingsFormat.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Memory/FSTPulsing41PostingsFormat.cs b/src/Lucene.Net.Codecs/Memory/FSTPulsing41PostingsFormat.cs
index 7ff2a8c..84ca223 100644
--- a/src/Lucene.Net.Codecs/Memory/FSTPulsing41PostingsFormat.cs
+++ b/src/Lucene.Net.Codecs/Memory/FSTPulsing41PostingsFormat.cs
@@ -1,102 +1,94 @@
-namespace org.apache.lucene.codecs.memory
-{
-
-	/*
-	 * 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.
-	 */
+/*
+ * 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 Lucene41PostingsWriter = org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
-	using Lucene41PostingsReader = org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
-	using Lucene41PostingsBaseFormat = org.apache.lucene.codecs.lucene41.Lucene41PostingsBaseFormat;
-	using Lucene41PostingsFormat = org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
-	using PulsingPostingsWriter = org.apache.lucene.codecs.pulsing.PulsingPostingsWriter;
-	using PulsingPostingsReader = org.apache.lucene.codecs.pulsing.PulsingPostingsReader;
-	using SegmentReadState = org.apache.lucene.index.SegmentReadState;
-	using SegmentWriteState = org.apache.lucene.index.SegmentWriteState;
-	using IOUtils = org.apache.lucene.util.IOUtils;
-
-	/// <summary>
-	/// FST + Pulsing41, test only, since
-	///  FST does no delta encoding here!
-	///  @lucene.experimental 
-	/// </summary>
+namespace Lucene.Net.Codecs.Memory
+{
+    
+    using Lucene41PostingsBaseFormat = Lucene41.Lucene41PostingsBaseFormat;
+    using PulsingPostingsWriter = Pulsing.PulsingPostingsWriter;
+    using PulsingPostingsReader = Pulsing.PulsingPostingsReader;
+    using SegmentReadState = Index.SegmentReadState;
+    using SegmentWriteState = Index.SegmentWriteState;
+    using IOUtils = Util.IOUtils;
 
-	public class FSTPulsing41PostingsFormat : PostingsFormat
-	{
-	  private readonly PostingsBaseFormat wrappedPostingsBaseFormat;
-	  private readonly int freqCutoff;
+    /// <summary>
+    /// FST + Pulsing41, test only, since
+    ///  FST does no delta encoding here!
+    ///  @lucene.experimental 
+    /// </summary>
 
-	  public FSTPulsing41PostingsFormat() : this(1)
-	  {
-	  }
+    public class FSTPulsing41PostingsFormat : PostingsFormat
+    {
+        private readonly PostingsBaseFormat _wrappedPostingsBaseFormat;
+        private readonly int _freqCutoff;
 
-	  public FSTPulsing41PostingsFormat(int freqCutoff) : base("FSTPulsing41")
-	  {
-		this.wrappedPostingsBaseFormat = new Lucene41PostingsBaseFormat();
-		this.freqCutoff = freqCutoff;
-	  }
+        public FSTPulsing41PostingsFormat() : this(1)
+        {
+        }
 
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public org.apache.lucene.codecs.FieldsConsumer fieldsConsumer(org.apache.lucene.index.SegmentWriteState state) throws java.io.IOException
-	  public override FieldsConsumer fieldsConsumer(SegmentWriteState state)
-	  {
-		PostingsWriterBase docsWriter = null;
-		PostingsWriterBase pulsingWriter = null;
+        public FSTPulsing41PostingsFormat(int freqCutoff) : base("FSTPulsing41")
+        {
+            _wrappedPostingsBaseFormat = new Lucene41PostingsBaseFormat();
+            _freqCutoff = freqCutoff;
+        }
 
-		bool success = false;
-		try
-		{
-		  docsWriter = wrappedPostingsBaseFormat.postingsWriterBase(state);
-		  pulsingWriter = new PulsingPostingsWriter(state, freqCutoff, docsWriter);
-		  FieldsConsumer ret = new FSTTermsWriter(state, pulsingWriter);
-		  success = true;
-		  return ret;
-		}
-		finally
-		{
-		  if (!success)
-		  {
-			IOUtils.closeWhileHandlingException(docsWriter, pulsingWriter);
-		  }
-		}
-	  }
+        public override FieldsConsumer FieldsConsumer(SegmentWriteState state)
+        {
+            PostingsWriterBase docsWriter = null;
+            PostingsWriterBase pulsingWriter = null;
 
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public org.apache.lucene.codecs.FieldsProducer fieldsProducer(org.apache.lucene.index.SegmentReadState state) throws java.io.IOException
-	  public override FieldsProducer fieldsProducer(SegmentReadState state)
-	  {
-		PostingsReaderBase docsReader = null;
-		PostingsReaderBase pulsingReader = null;
-		bool success = false;
-		try
-		{
-		  docsReader = wrappedPostingsBaseFormat.postingsReaderBase(state);
-		  pulsingReader = new PulsingPostingsReader(state, docsReader);
-		  FieldsProducer ret = new FSTTermsReader(state, pulsingReader);
-		  success = true;
-		  return ret;
-		}
-		finally
-		{
-		  if (!success)
-		  {
-			IOUtils.closeWhileHandlingException(docsReader, pulsingReader);
-		  }
-		}
-	  }
-	}
+            bool success = false;
+            try
+            {
+                docsWriter = _wrappedPostingsBaseFormat.PostingsWriterBase(state);
+                pulsingWriter = new PulsingPostingsWriter(state, _freqCutoff, docsWriter);
+                FieldsConsumer ret = new FSTTermsWriter(state, pulsingWriter);
+                success = true;
+                return ret;
+            }
+            finally
+            {
+                if (!success)
+                {
+                    IOUtils.CloseWhileHandlingException(docsWriter, pulsingWriter);
+                }
+            }
+        }
 
+        public override FieldsProducer FieldsProducer(SegmentReadState state)
+        {
+            PostingsReaderBase docsReader = null;
+            PostingsReaderBase pulsingReader = null;
+            bool success = false;
+            try
+            {
+                docsReader = _wrappedPostingsBaseFormat.PostingsReaderBase(state);
+                pulsingReader = new PulsingPostingsReader(state, docsReader);
+                FieldsProducer ret = new FSTTermsReader(state, pulsingReader);
+                success = true;
+                return ret;
+            }
+            finally
+            {
+                if (!success)
+                {
+                    IOUtils.CloseWhileHandlingException(docsReader, pulsingReader);
+                }
+            }
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/af4d125b/src/Lucene.Net.Codecs/Memory/FSTTermOutputs.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Memory/FSTTermOutputs.cs b/src/Lucene.Net.Codecs/Memory/FSTTermOutputs.cs
index 104ff61..1906ce2 100644
--- a/src/Lucene.Net.Codecs/Memory/FSTTermOutputs.cs
+++ b/src/Lucene.Net.Codecs/Memory/FSTTermOutputs.cs
@@ -1,423 +1,390 @@
-using System.Diagnostics;
+/*
+ * 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.
+ */
 
-namespace org.apache.lucene.codecs.memory
+using System.Linq;
+
+namespace Lucene.Net.Codecs.Memory
 {
 
-	/*
-	 * 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 FieldInfo = org.apache.lucene.index.FieldInfo;
-	using IndexOptions = org.apache.lucene.index.FieldInfo.IndexOptions;
-	using DataInput = org.apache.lucene.store.DataInput;
-	using DataOutput = org.apache.lucene.store.DataOutput;
-	using Outputs = org.apache.lucene.util.fst.Outputs;
-	using LongsRef = org.apache.lucene.util.LongsRef;
-
-	/// <summary>
-	/// An FST <seealso cref="Outputs"/> implementation for 
-	/// <seealso cref="FSTTermsWriter"/>.
-	/// 
-	/// @lucene.experimental
-	/// </summary>
-
-	// NOTE: outputs should be per-field, since
-	// longsSize is fixed for each field
-	internal class FSTTermOutputs : Outputs<FSTTermOutputs.TermData>
-	{
-	  private static readonly TermData NO_OUTPUT = new TermData();
-	  //private static boolean TEST = false;
-	  private readonly bool hasPos;
-	  private readonly int longsSize;
-
-	  /// <summary>
-	  /// Represents the metadata for one term.
-	  /// On an FST, only long[] part is 'shared' and pushed towards root.
-	  /// byte[] and term stats will be kept on deeper arcs.
-	  /// </summary>
-	  internal class TermData
-	  {
-		internal long[] longs;
-		internal sbyte[] bytes;
-		internal int docFreq;
-		internal long totalTermFreq;
-		internal TermData()
-		{
-		  this.longs = null;
-		  this.bytes = null;
-		  this.docFreq = 0;
-		  this.totalTermFreq = -1;
-		}
-		internal TermData(long[] longs, sbyte[] bytes, int docFreq, long totalTermFreq)
-		{
-		  this.longs = longs;
-		  this.bytes = bytes;
-		  this.docFreq = docFreq;
-		  this.totalTermFreq = totalTermFreq;
-		}
-
-		// NOTE: actually, FST nodes are seldom 
-		// identical when outputs on their arcs 
-		// aren't NO_OUTPUTs.
-		public override int GetHashCode()
-		{
-		  int hash = 0;
-		  if (longs != null)
-		  {
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final int end = longs.length;
-			int end = longs.Length;
-			for (int i = 0; i < end; i++)
-			{
-			  hash -= (int)longs[i];
-			}
-		  }
-		  if (bytes != null)
-		  {
-			hash = -hash;
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final int end = bytes.length;
-			int end = bytes.Length;
-			for (int i = 0; i < end; i++)
-			{
-			  hash += bytes[i];
-			}
-		  }
-		  hash += (int)(docFreq + totalTermFreq);
-		  return hash;
-		}
-
-		public override bool Equals(object other_)
-		{
-		  if (other_ == this)
-		  {
-			return true;
-		  }
-		  else if (!(other_ is FSTTermOutputs.TermData))
-		  {
-			return false;
-		  }
-		  TermData other = (TermData) other_;
-		  return statsEqual(this, other) && longsEqual(this, other) && bytesEqual(this, other);
-
-		}
-	  }
-
-	  protected internal FSTTermOutputs(FieldInfo fieldInfo, int longsSize)
-	  {
-		this.hasPos = (fieldInfo.IndexOptions != FieldInfo.IndexOptions.DOCS_ONLY);
-		this.longsSize = longsSize;
-	  }
-
-	  public override TermData common(TermData t1, TermData t2)
-	  //
-	  // The return value will be the smaller one, when these two are 
-	  // 'comparable', i.e. 
-	  // 1. every value in t1 is not larger than in t2, or
-	  // 2. every value in t1 is not smaller than t2.
-	  //
-	  {
-		//if (TEST) System.out.print("common("+t1+", "+t2+") = ");
-		if (t1 == NO_OUTPUT || t2 == NO_OUTPUT)
-		{
-		  //if (TEST) System.out.println("ret:"+NO_OUTPUT);
-		  return NO_OUTPUT;
-		}
-		Debug.Assert(t1.longs.Length == t2.longs.Length);
-
-		long[] min = t1.longs, max = t2.longs;
-		int pos = 0;
-		TermData ret;
-
-		while (pos < longsSize && min[pos] == max[pos])
-		{
-		  pos++;
-		}
-		if (pos < longsSize) // unequal long[]
-		{
-		  if (min[pos] > max[pos])
-		  {
-			min = t2.longs;
-			max = t1.longs;
-		  }
-		  // check whether strictly smaller
-		  while (pos < longsSize && min[pos] <= max[pos])
-		  {
-			pos++;
-		  }
-		  if (pos < longsSize || allZero(min)) // not comparable or all-zero
-		  {
-			ret = NO_OUTPUT;
-		  }
-		  else
-		  {
-			ret = new TermData(min, null, 0, -1);
-		  }
-		} // equal long[]
-		else
-		{
-		  if (statsEqual(t1, t2) && bytesEqual(t1, t2))
-		  {
-			ret = t1;
-		  }
-		  else if (allZero(min))
-		  {
-			ret = NO_OUTPUT;
-		  }
-		  else
-		  {
-			ret = new TermData(min, null, 0, -1);
-		  }
-		}
-		//if (TEST) System.out.println("ret:"+ret);
-		return ret;
-	  }
-
-	  public override TermData subtract(TermData t1, TermData t2)
-	  {
-		//if (TEST) System.out.print("subtract("+t1+", "+t2+") = ");
-		if (t2 == NO_OUTPUT)
-		{
-		  //if (TEST) System.out.println("ret:"+t1);
-		  return t1;
-		}
-		Debug.Assert(t1.longs.Length == t2.longs.Length);
-
-		int pos = 0;
-		long diff = 0;
-		long[] share = new long[longsSize];
-
-		while (pos < longsSize)
-		{
-		  share[pos] = t1.longs[pos] - t2.longs[pos];
-		  diff += share[pos];
-		  pos++;
-		}
-
-		TermData ret;
-		if (diff == 0 && statsEqual(t1, t2) && bytesEqual(t1, t2))
-		{
-		  ret = NO_OUTPUT;
-		}
-		else
-		{
-		  ret = new TermData(share, t1.bytes, t1.docFreq, t1.totalTermFreq);
-		}
-		//if (TEST) System.out.println("ret:"+ret);
-		return ret;
-	  }
-
-	  // TODO: if we refactor a 'addSelf(TermData other)',
-	  // we can gain about 5~7% for fuzzy queries, however this also 
-	  // means we are putting too much stress on FST Outputs decoding?
-	  public override TermData add(TermData t1, TermData t2)
-	  {
-		//if (TEST) System.out.print("add("+t1+", "+t2+") = ");
-		if (t1 == NO_OUTPUT)
-		{
-		  //if (TEST) System.out.println("ret:"+t2);
-		  return t2;
-		}
-		else if (t2 == NO_OUTPUT)
-		{
-		  //if (TEST) System.out.println("ret:"+t1);
-		  return t1;
-		}
-		Debug.Assert(t1.longs.Length == t2.longs.Length);
-
-		int pos = 0;
-		long[] accum = new long[longsSize];
-
-		while (pos < longsSize)
-		{
-		  accum[pos] = t1.longs[pos] + t2.longs[pos];
-		  pos++;
-		}
-
-		TermData ret;
-		if (t2.bytes != null || t2.docFreq > 0)
-		{
-		  ret = new TermData(accum, t2.bytes, t2.docFreq, t2.totalTermFreq);
-		}
-		else
-		{
-		  ret = new TermData(accum, t1.bytes, t1.docFreq, t1.totalTermFreq);
-		}
-		//if (TEST) System.out.println("ret:"+ret);
-		return ret;
-	  }
-
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public void write(TermData data, org.apache.lucene.store.DataOutput out) throws java.io.IOException
-	  public override void write(TermData data, DataOutput @out)
-	  {
-		int bit0 = allZero(data.longs) ? 0 : 1;
-		int bit1 = ((data.bytes == null || data.bytes.Length == 0) ? 0 : 1) << 1;
-		int bit2 = ((data.docFreq == 0) ? 0 : 1) << 2;
-		int bits = bit0 | bit1 | bit2;
-		if (bit1 > 0) // determine extra length
-		{
-		  if (data.bytes.Length < 32)
-		  {
-			bits |= (data.bytes.Length << 3);
-			@out.writeByte((sbyte)bits);
-		  }
-		  else
-		  {
-			@out.writeByte((sbyte)bits);
-			@out.writeVInt(data.bytes.Length);
-		  }
-		}
-		else
-		{
-		  @out.writeByte((sbyte)bits);
-		}
-		if (bit0 > 0) // not all-zero case
-		{
-		  for (int pos = 0; pos < longsSize; pos++)
-		  {
-			@out.writeVLong(data.longs[pos]);
-		  }
-		}
-		if (bit1 > 0) // bytes exists
-		{
-		  @out.writeBytes(data.bytes, 0, data.bytes.Length);
-		}
-		if (bit2 > 0) // stats exist
-		{
-		  if (hasPos)
-		  {
-			if (data.docFreq == data.totalTermFreq)
-			{
-			  @out.writeVInt((data.docFreq << 1) | 1);
-			}
-			else
-			{
-			  @out.writeVInt((data.docFreq << 1));
-			  @out.writeVLong(data.totalTermFreq - data.docFreq);
-			}
-		  }
-		  else
-		  {
-			@out.writeVInt(data.docFreq);
-		  }
-		}
-	  }
-
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public TermData read(org.apache.lucene.store.DataInput in) throws java.io.IOException
-	  public override TermData read(DataInput @in)
-	  {
-		long[] longs = new long[longsSize];
-		sbyte[] bytes = null;
-		int docFreq = 0;
-		long totalTermFreq = -1;
-		int bits = @in.readByte() & 0xff;
-		int bit0 = bits & 1;
-		int bit1 = bits & 2;
-		int bit2 = bits & 4;
-		int bytesSize = ((int)((uint)bits >> 3));
-		if (bit1 > 0 && bytesSize == 0) // determine extra length
-		{
-		  bytesSize = @in.readVInt();
-		}
-		if (bit0 > 0) // not all-zero case
-		{
-		  for (int pos = 0; pos < longsSize; pos++)
-		  {
-			longs[pos] = @in.readVLong();
-		  }
-		}
-		if (bit1 > 0) // bytes exists
-		{
-		  bytes = new sbyte[bytesSize];
-		  @in.readBytes(bytes, 0, bytesSize);
-		}
-		if (bit2 > 0) // stats exist
-		{
-		  int code = @in.readVInt();
-		  if (hasPos)
-		  {
-			totalTermFreq = docFreq = (int)((uint)code >> 1);
-			if ((code & 1) == 0)
-			{
-			  totalTermFreq += @in.readVLong();
-			}
-		  }
-		  else
-		  {
-			docFreq = code;
-		  }
-		}
-		return new TermData(longs, bytes, docFreq, totalTermFreq);
-	  }
-
-	  public override TermData NoOutput
-	  {
-		  get
-		  {
-			return NO_OUTPUT;
-		  }
-	  }
-
-	  public override string outputToString(TermData data)
-	  {
-		return data.ToString();
-	  }
-
-//JAVA TO C# CONVERTER WARNING: 'final' parameters are not available in .NET:
-//ORIGINAL LINE: static boolean statsEqual(final TermData t1, final TermData t2)
-	  internal static bool statsEqual(TermData t1, TermData t2)
-	  {
-		return t1.docFreq == t2.docFreq && t1.totalTermFreq == t2.totalTermFreq;
-	  }
-//JAVA TO C# CONVERTER WARNING: 'final' parameters are not available in .NET:
-//ORIGINAL LINE: static boolean bytesEqual(final TermData t1, final TermData t2)
-	  internal static bool bytesEqual(TermData t1, TermData t2)
-	  {
-		if (t1.bytes == null && t2.bytes == null)
-		{
-		  return true;
-		}
-		return t1.bytes != null && t2.bytes != null && Arrays.Equals(t1.bytes, t2.bytes);
-	  }
-//JAVA TO C# CONVERTER WARNING: 'final' parameters are not available in .NET:
-//ORIGINAL LINE: static boolean longsEqual(final TermData t1, final TermData t2)
-	  internal static bool longsEqual(TermData t1, TermData t2)
-	  {
-		if (t1.longs == null && t2.longs == null)
-		{
-		  return true;
-		}
-		return t1.longs != null && t2.longs != null && Arrays.Equals(t1.longs, t2.longs);
-	  }
-//JAVA TO C# CONVERTER WARNING: 'final' parameters are not available in .NET:
-//ORIGINAL LINE: static boolean allZero(final long[] l)
-	  internal static bool allZero(long[] l)
-	  {
-		for (int i = 0; i < l.Length; i++)
-		{
-		  if (l[i] != 0)
-		  {
-			return false;
-		  }
-		}
-		return true;
-	  }
-	}
+    using System.Diagnostics;
+    using Support;
+
+    using FieldInfo = Index.FieldInfo;
+    using IndexOptions = Index.FieldInfo.IndexOptions;
+    using DataInput = Store.DataInput;
+    using DataOutput = Store.DataOutput;
+   
+    /// <summary>
+    /// An FST implementation for 
+    /// <seealso cref="FSTTermsWriter"/>.
+    /// 
+    /// @lucene.experimental
+    /// </summary>
+
+    // NOTE: outputs should be per-field, since
+    // longsSize is fixed for each field
+    internal class FSTTermOutputs : Util.Fst.Outputs<FSTTermOutputs.TermData>
+    {
+        private static readonly TermData NO_OUTPUT = new TermData();
+        private readonly bool _hasPos;
+        private readonly int _longsSize;
+
+        /// <summary>
+        /// Represents the metadata for one term.
+        /// On an FST, only long[] part is 'shared' and pushed towards root.
+        /// byte[] and term stats will be kept on deeper arcs.
+        /// </summary>
+        internal class TermData
+        {
+            internal readonly long[] LONGS;
+            internal readonly sbyte[] BYTES;
+            internal readonly int DOC_FREQ;
+            internal readonly long TOTAL_TERM_FREQ;
+
+            internal TermData()
+            {
+                LONGS = null;
+                BYTES = null;
+                DOC_FREQ = 0;
+                TOTAL_TERM_FREQ = -1;
+            }
+
+            internal TermData(long[] longs, sbyte[] bytes, int docFreq, long totalTermFreq)
+            {
+                LONGS = longs;
+                BYTES = bytes;
+                DOC_FREQ = docFreq;
+                TOTAL_TERM_FREQ = totalTermFreq;
+            }
+
+            // NOTE: actually, FST nodes are seldom 
+            // identical when outputs on their arcs 
+            // aren't NO_OUTPUTs.
+            public override int GetHashCode()
+            {
+                var hash = 0;
+                if (LONGS != null)
+                {
+                    var end = LONGS.Length;
+                    for (var i = 0; i < end; i++)
+                    {
+                        hash -= (int) LONGS[i];
+                    }
+                }
+                if (BYTES != null)
+                {
+                    hash = -hash;
+                    var end = BYTES.Length;
+                    for (var i = 0; i < end; i++)
+                    {
+                        hash += BYTES[i];
+                    }
+                }
+                hash += (int) (DOC_FREQ + TOTAL_TERM_FREQ);
+                return hash;
+            }
+
+            public override bool Equals(object other)
+            {
+                if (other == this)
+                    return true;
+                
+                if (!(other is TermData))
+                    return false;
+                
+                var _other = (TermData) other;
+                return StatsEqual(this, _other) && LongsEqual(this, _other) && BytesEqual(this, _other);
+            }
+
+        }
+
+        protected internal FSTTermOutputs(FieldInfo fieldInfo, int longsSize)
+        {
+            _hasPos = (fieldInfo.FieldIndexOptions != IndexOptions.DOCS_ONLY);
+            _longsSize = longsSize;
+        }
+
+        /// <summary>
+        /// The return value will be the smaller one, when these two are 
+        /// 'comparable', i.e. 
+        /// 1. every value in t1 is not larger than in t2, or
+        /// 2. every value in t1 is not smaller than t2.
+        /// </summary>
+        /// <param name="t1"></param>
+        /// <param name="t2"></param>
+        /// <returns></returns>
+        public override TermData Common(TermData t1, TermData t2)
+        {
+            if (Equals(t1, NO_OUTPUT) || Equals(t2, NO_OUTPUT))
+                return NO_OUTPUT;
+            
+            Debug.Assert(t1.LONGS.Length == t2.LONGS.Length);
+
+            long[] min = t1.LONGS, max = t2.LONGS;
+            int pos = 0;
+            TermData ret;
+
+            while (pos < _longsSize && min[pos] == max[pos])
+            {
+                pos++;
+            }
+            if (pos < _longsSize) // unequal long[]
+            {
+                if (min[pos] > max[pos])
+                {
+                    min = t2.LONGS;
+                    max = t1.LONGS;
+                }
+                // check whether strictly smaller
+                while (pos < _longsSize && min[pos] <= max[pos])
+                {
+                    pos++;
+                }
+                if (pos < _longsSize || AllZero(min)) // not comparable or all-zero
+                {
+                    ret = NO_OUTPUT;
+                }
+                else
+                {
+                    ret = new TermData(min, null, 0, -1);
+                }
+            } // equal long[]
+            else
+            {
+                if (StatsEqual(t1, t2) && BytesEqual(t1, t2))
+                {
+                    ret = t1;
+                }
+                else if (AllZero(min))
+                {
+                    ret = NO_OUTPUT;
+                }
+                else
+                {
+                    ret = new TermData(min, null, 0, -1);
+                }
+            }
+            //if (TEST) System.out.println("ret:"+ret);
+            return ret;
+        }
+
+        public override TermData Subtract(TermData t1, TermData t2)
+        {
+            if (Equals(t2, NO_OUTPUT))
+                return t1;
+            
+            Debug.Assert(t1.LONGS.Length == t2.LONGS.Length);
+
+            int pos = 0;
+            long diff = 0;
+            var share = new long[_longsSize];
+
+            while (pos < _longsSize)
+            {
+                share[pos] = t1.LONGS[pos] - t2.LONGS[pos];
+                diff += share[pos];
+                pos++;
+            }
+
+            TermData ret;
+            if (diff == 0 && StatsEqual(t1, t2) && BytesEqual(t1, t2))
+            {
+                ret = NO_OUTPUT;
+            }
+            else
+            {
+                ret = new TermData(share, t1.BYTES, t1.DOC_FREQ, t1.TOTAL_TERM_FREQ);
+            }
+            //if (TEST) System.out.println("ret:"+ret);
+            return ret;
+        }
+
+        // TODO: if we refactor a 'addSelf(TermData other)',
+        // we can gain about 5~7% for fuzzy queries, however this also 
+        // means we are putting too much stress on FST Outputs decoding?
+        public override TermData Add(TermData t1, TermData t2)
+        {
+            if (Equals(t1, NO_OUTPUT))
+                return t2;
+            
+            if (Equals(t2, NO_OUTPUT))
+                return t1;
+            
+            Debug.Assert(t1.LONGS.Length == t2.LONGS.Length);
+
+            var pos = 0;
+            var accum = new long[_longsSize];
+
+            while (pos < _longsSize)
+            {
+                accum[pos] = t1.LONGS[pos] + t2.LONGS[pos];
+                pos++;
+            }
+
+            TermData ret;
+            if (t2.BYTES != null || t2.DOC_FREQ > 0)
+            {
+                ret = new TermData(accum, t2.BYTES, t2.DOC_FREQ, t2.TOTAL_TERM_FREQ);
+            }
+            else
+            {
+                ret = new TermData(accum, t1.BYTES, t1.DOC_FREQ, t1.TOTAL_TERM_FREQ);
+            }
+
+            return ret;
+        }
+
+        public override void Write(TermData data, DataOutput output)
+        {
+            int bit0 = AllZero(data.LONGS) ? 0 : 1;
+            int bit1 = ((data.BYTES == null || data.BYTES.Length == 0) ? 0 : 1) << 1;
+            int bit2 = ((data.DOC_FREQ == 0) ? 0 : 1) << 2;
+            int bits = bit0 | bit1 | bit2;
+            if (bit1 > 0) // determine extra length
+            {
+                if (data.BYTES.Length < 32)
+                {
+                    bits |= (data.BYTES.Length << 3);
+                    output.WriteByte((sbyte) bits);
+                }
+                else
+                {
+                    output.WriteByte((sbyte) bits);
+                    output.WriteVInt(data.BYTES.Length);
+                }
+            }
+            else
+            {
+                output.WriteByte((sbyte) bits);
+            }
+            if (bit0 > 0) // not all-zero case
+            {
+                for (int pos = 0; pos < _longsSize; pos++)
+                {
+                    output.WriteVLong(data.LONGS[pos]);
+                }
+            }
+            if (bit1 > 0) // bytes exists
+            {
+                output.WriteBytes(data.BYTES, 0, data.BYTES.Length);
+            }
+            if (bit2 > 0) // stats exist
+            {
+                if (_hasPos)
+                {
+                    if (data.DOC_FREQ == data.TOTAL_TERM_FREQ)
+                    {
+                        output.WriteVInt((data.DOC_FREQ << 1) | 1);
+                    }
+                    else
+                    {
+                        output.WriteVInt((data.DOC_FREQ << 1));
+                        output.WriteVLong(data.TOTAL_TERM_FREQ - data.DOC_FREQ);
+                    }
+                }
+                else
+                {
+                    output.WriteVInt(data.DOC_FREQ);
+                }
+            }
+        }
+
+        public override TermData Read(DataInput input)
+        {
+            var longs = new long[_longsSize];
+            sbyte[] bytes = null;
+            int docFreq = 0;
+            long totalTermFreq = -1;
+            int bits = input.ReadByte() & 0xff;
+            int bit0 = bits & 1;
+            int bit1 = bits & 2;
+            int bit2 = bits & 4;
+            var bytesSize = ((int) ((uint) bits >> 3));
+            if (bit1 > 0 && bytesSize == 0) // determine extra length
+            {
+                bytesSize = input.ReadVInt();
+            }
+            if (bit0 > 0) // not all-zero case
+            {
+                for (int pos = 0; pos < _longsSize; pos++)
+                {
+                    longs[pos] = input.ReadVLong();
+                }
+            }
+            if (bit1 > 0) // bytes exists
+            {
+                bytes = new sbyte[bytesSize];
+                input.ReadBytes(bytes, 0, bytesSize);
+            }
+            if (bit2 > 0) // stats exist
+            {
+                int code = input.ReadVInt();
+                if (_hasPos)
+                {
+                    totalTermFreq = docFreq = (int) ((uint) code >> 1);
+                    if ((code & 1) == 0)
+                    {
+                        totalTermFreq += input.ReadVLong();
+                    }
+                }
+                else
+                {
+                    docFreq = code;
+                }
+            }
+            return new TermData(longs, bytes, docFreq, totalTermFreq);
+        }
+
+        public override TermData NoOutput
+        {
+            get { return NO_OUTPUT; }
+        }
+
+        public override string OutputToString(TermData data)
+        {
+            return data.ToString();
+        }
+
+        internal static bool StatsEqual(TermData t1, TermData t2)
+        {
+            return t1.DOC_FREQ == t2.DOC_FREQ && t1.TOTAL_TERM_FREQ == t2.TOTAL_TERM_FREQ;
+        }
+
+        internal static bool BytesEqual(TermData t1, TermData t2)
+        {
+            if (t1.BYTES == null && t2.BYTES == null)
+            {
+                return true;
+            }
+            return t1.BYTES != null && t2.BYTES != null && Arrays.Equals(t1.BYTES, t2.BYTES);
+        }
+
+        internal static bool LongsEqual(TermData t1, TermData t2)
+        {
+            if (t1.LONGS == null && t2.LONGS == null)
+            {
+                return true;
+            }
+            return t1.LONGS != null && t2.LONGS != null && Arrays.Equals(t1.LONGS, t2.LONGS);
+        }
 
+        internal static bool AllZero(long[] l)
+        {
+            return l.All(t => t == 0);
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/af4d125b/src/Lucene.Net.Codecs/Memory/FSTTermsReader.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Memory/FSTTermsReader.cs b/src/Lucene.Net.Codecs/Memory/FSTTermsReader.cs
index fb81d4d..2fcc6c2 100644
--- a/src/Lucene.Net.Codecs/Memory/FSTTermsReader.cs
+++ b/src/Lucene.Net.Codecs/Memory/FSTTermsReader.cs
@@ -2,8 +2,9 @@
 using System.Diagnostics;
 using System.Collections;
 using System.Collections.Generic;
+using Lucene.Net.Index;
 
-namespace org.apache.lucene.codecs.memory
+namespace Lucene.Net.Codecs.Memory
 {
 
 	/*
@@ -24,32 +25,32 @@ namespace org.apache.lucene.codecs.memory
 	 */
 
 
-	using CorruptIndexException = org.apache.lucene.index.CorruptIndexException;
-	using DocsAndPositionsEnum = org.apache.lucene.index.DocsAndPositionsEnum;
-	using DocsEnum = org.apache.lucene.index.DocsEnum;
-	using IndexOptions = org.apache.lucene.index.FieldInfo.IndexOptions;
-	using FieldInfo = org.apache.lucene.index.FieldInfo;
-	using FieldInfos = org.apache.lucene.index.FieldInfos;
-	using IndexFileNames = org.apache.lucene.index.IndexFileNames;
-	using SegmentInfo = org.apache.lucene.index.SegmentInfo;
-	using SegmentReadState = org.apache.lucene.index.SegmentReadState;
-	using TermState = org.apache.lucene.index.TermState;
-	using Terms = org.apache.lucene.index.Terms;
-	using TermsEnum = org.apache.lucene.index.TermsEnum;
-	using ByteArrayDataInput = org.apache.lucene.store.ByteArrayDataInput;
-	using IndexInput = org.apache.lucene.store.IndexInput;
-	using ArrayUtil = org.apache.lucene.util.ArrayUtil;
-	using Bits = org.apache.lucene.util.Bits;
-	using BytesRef = org.apache.lucene.util.BytesRef;
-	using IOUtils = org.apache.lucene.util.IOUtils;
-	using RamUsageEstimator = org.apache.lucene.util.RamUsageEstimator;
-	using ByteRunAutomaton = org.apache.lucene.util.automaton.ByteRunAutomaton;
-	using CompiledAutomaton = org.apache.lucene.util.automaton.CompiledAutomaton;
-	using InputOutput = org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
-	using BytesRefFSTEnum = org.apache.lucene.util.fst.BytesRefFSTEnum;
-	using FST = org.apache.lucene.util.fst.FST;
-	using Outputs = org.apache.lucene.util.fst.Outputs;
-	using Util = org.apache.lucene.util.fst.Util;
+	using CorruptIndexException = index.CorruptIndexException;
+	using DocsAndPositionsEnum = index.DocsAndPositionsEnum;
+	using DocsEnum = index.DocsEnum;
+	using IndexOptions = index.FieldInfo.IndexOptions;
+	using FieldInfo = index.FieldInfo;
+	using FieldInfos = index.FieldInfos;
+	using IndexFileNames = index.IndexFileNames;
+	using SegmentInfo = index.SegmentInfo;
+	using SegmentReadState = index.SegmentReadState;
+	using TermState = index.TermState;
+	using Terms = index.Terms;
+	using TermsEnum = index.TermsEnum;
+	using ByteArrayDataInput = store.ByteArrayDataInput;
+	using IndexInput = store.IndexInput;
+	using ArrayUtil = util.ArrayUtil;
+	using Bits = util.Bits;
+	using BytesRef = util.BytesRef;
+	using IOUtils = util.IOUtils;
+	using RamUsageEstimator = util.RamUsageEstimator;
+	using ByteRunAutomaton = util.automaton.ByteRunAutomaton;
+	using CompiledAutomaton = util.automaton.CompiledAutomaton;
+	using InputOutput = util.fst.BytesRefFSTEnum.InputOutput;
+	using BytesRefFSTEnum = util.fst.BytesRefFSTEnum;
+	using FST = util.fst.FST;
+	using Outputs = util.fst.Outputs;
+	using Util = util.fst.Util;
 
 	/// <summary>
 	/// FST-based terms dictionary reader.
@@ -68,16 +69,16 @@ namespace org.apache.lucene.codecs.memory
 	  internal readonly int version;
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: public FSTTermsReader(org.apache.lucene.index.SegmentReadState state, org.apache.lucene.codecs.PostingsReaderBase postingsReader) throws java.io.IOException
+//ORIGINAL LINE: public FSTTermsReader(index.SegmentReadState state, codecs.PostingsReaderBase postingsReader) throws java.io.IOException
 	  public FSTTermsReader(SegmentReadState state, PostingsReaderBase postingsReader)
 	  {
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final String termsFileName = org.apache.lucene.index.IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, FSTTermsWriter.TERMS_EXTENSION);
+//ORIGINAL LINE: final String termsFileName = index.IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, FSTTermsWriter.TERMS_EXTENSION);
 		string termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, FSTTermsWriter.TERMS_EXTENSION);
 
 		this.postingsReader = postingsReader;
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final org.apache.lucene.store.IndexInput in = state.directory.openInput(termsFileName, state.context);
+//ORIGINAL LINE: final store.IndexInput in = state.directory.openInput(termsFileName, state.context);
 		IndexInput @in = state.directory.openInput(termsFileName, state.context);
 
 		bool success = false;
@@ -92,7 +93,7 @@ namespace org.apache.lucene.codecs.memory
 		  seekDir(@in);
 
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final org.apache.lucene.index.FieldInfos fieldInfos = state.fieldInfos;
+//ORIGINAL LINE: final index.FieldInfos fieldInfos = state.fieldInfos;
 		  FieldInfos fieldInfos = state.fieldInfos;
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
 //ORIGINAL LINE: final int numFields = in.readVInt();
@@ -126,13 +127,13 @@ namespace org.apache.lucene.codecs.memory
 	  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: private int readHeader(org.apache.lucene.store.IndexInput in) throws java.io.IOException
+//ORIGINAL LINE: private int readHeader(store.IndexInput in) throws java.io.IOException
 	  private int readHeader(IndexInput @in)
 	  {
 		return CodecUtil.checkHeader(@in, FSTTermsWriter.TERMS_CODEC_NAME, FSTTermsWriter.TERMS_VERSION_START, FSTTermsWriter.TERMS_VERSION_CURRENT);
 	  }
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: private void seekDir(org.apache.lucene.store.IndexInput in) throws java.io.IOException
+//ORIGINAL LINE: private void seekDir(store.IndexInput in) throws java.io.IOException
 	  private void seekDir(IndexInput @in)
 	  {
 		if (version >= FSTTermsWriter.TERMS_VERSION_CHECKSUM)
@@ -146,7 +147,7 @@ namespace org.apache.lucene.codecs.memory
 		@in.seek(@in.readLong());
 	  }
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: private void checkFieldSummary(org.apache.lucene.index.SegmentInfo info, org.apache.lucene.store.IndexInput in, TermsReader field, TermsReader previous) throws java.io.IOException
+//ORIGINAL LINE: private void checkFieldSummary(index.SegmentInfo info, store.IndexInput in, TermsReader field, TermsReader previous) throws java.io.IOException
 	  private void checkFieldSummary(SegmentInfo info, IndexInput @in, TermsReader field, TermsReader previous)
 	  {
 		// #docs with field must be <= #docs
@@ -176,7 +177,7 @@ namespace org.apache.lucene.codecs.memory
 	  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public org.apache.lucene.index.Terms terms(String field) throws java.io.IOException
+//ORIGINAL LINE: @Override public index.Terms terms(String field) throws java.io.IOException
 	  public override Terms terms(string field)
 	  {
 		Debug.Assert(field != null);
@@ -215,7 +216,7 @@ namespace org.apache.lucene.codecs.memory
 		internal readonly FST<FSTTermOutputs.TermData> dict;
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: TermsReader(org.apache.lucene.index.FieldInfo fieldInfo, org.apache.lucene.store.IndexInput in, long numTerms, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize) throws java.io.IOException
+//ORIGINAL LINE: TermsReader(index.FieldInfo fieldInfo, store.IndexInput in, long numTerms, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize) throws java.io.IOException
 		internal TermsReader(FSTTermsReader outerInstance, FieldInfo fieldInfo, IndexInput @in, long numTerms, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize)
 		{
 			this.outerInstance = outerInstance;
@@ -290,14 +291,14 @@ namespace org.apache.lucene.codecs.memory
 		}
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public org.apache.lucene.index.TermsEnum iterator(org.apache.lucene.index.TermsEnum reuse) throws java.io.IOException
+//ORIGINAL LINE: @Override public index.TermsEnum iterator(index.TermsEnum reuse) throws java.io.IOException
 		public override TermsEnum iterator(TermsEnum reuse)
 		{
 		  return new SegmentTermsEnum(this);
 		}
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public org.apache.lucene.index.TermsEnum intersect(org.apache.lucene.util.automaton.CompiledAutomaton compiled, org.apache.lucene.util.BytesRef startTerm) throws java.io.IOException
+//ORIGINAL LINE: @Override public index.TermsEnum intersect(util.automaton.CompiledAutomaton compiled, util.BytesRef startTerm) throws java.io.IOException
 		public override TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm)
 		{
 		  return new IntersectTermsEnum(this, compiled, startTerm);
@@ -336,7 +337,7 @@ namespace org.apache.lucene.codecs.memory
 		  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public org.apache.lucene.index.TermState termState() throws java.io.IOException
+//ORIGINAL LINE: @Override public index.TermState termState() throws java.io.IOException
 		  public override TermState termState()
 		  {
 			decodeMetaData();
@@ -363,7 +364,7 @@ namespace org.apache.lucene.codecs.memory
 		  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public org.apache.lucene.index.DocsEnum docs(org.apache.lucene.util.Bits liveDocs, org.apache.lucene.index.DocsEnum reuse, int flags) throws java.io.IOException
+//ORIGINAL LINE: @Override public index.DocsEnum docs(util.Bits liveDocs, index.DocsEnum reuse, int flags) throws java.io.IOException
 		  public override DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags)
 		  {
 			decodeMetaData();
@@ -371,7 +372,7 @@ namespace org.apache.lucene.codecs.memory
 		  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public org.apache.lucene.index.DocsAndPositionsEnum docsAndPositions(org.apache.lucene.util.Bits liveDocs, org.apache.lucene.index.DocsAndPositionsEnum reuse, int flags) throws java.io.IOException
+//ORIGINAL LINE: @Override public index.DocsAndPositionsEnum docsAndPositions(util.Bits liveDocs, index.DocsAndPositionsEnum reuse, int flags) throws java.io.IOException
 		  public override DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags)
 		  {
 			if (!outerInstance.hasPositions())
@@ -435,18 +436,18 @@ namespace org.apache.lucene.codecs.memory
 		  {
 			if (!decoded && !seekPending)
 			{
-			  if (meta.bytes != null)
+			  if (meta.BYTES != null)
 			  {
-				bytesReader.reset(meta.bytes, 0, meta.bytes.Length);
+				bytesReader.reset(meta.BYTES, 0, meta.BYTES.Length);
 			  }
-			  outerInstance.outerInstance.postingsReader.decodeTerm(meta.longs, bytesReader, outerInstance.fieldInfo, state, true);
+			  outerInstance.outerInstance.postingsReader.decodeTerm(meta.LONGS, bytesReader, outerInstance.fieldInfo, state, true);
 			  decoded = true;
 			}
 		  }
 
 		  // Update current enum according to FSTEnum
 //JAVA TO C# CONVERTER WARNING: 'final' parameters are not available in .NET:
-//ORIGINAL LINE: void updateEnum(final org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput<FSTTermOutputs.TermData> pair)
+//ORIGINAL LINE: void updateEnum(final util.fst.BytesRefFSTEnum.InputOutput<FSTTermOutputs.TermData> pair)
 		  internal void updateEnum(InputOutput<FSTTermOutputs.TermData> pair)
 		  {
 			if (pair == null)
@@ -457,15 +458,15 @@ namespace org.apache.lucene.codecs.memory
 			{
 			  term_Renamed = pair.input;
 			  meta = pair.output;
-			  state.docFreq = meta.docFreq;
-			  state.totalTermFreq = meta.totalTermFreq;
+			  state.docFreq = meta.DOC_FREQ;
+			  state.totalTermFreq = meta.TOTAL_TERM_FREQ;
 			}
 			decoded = false;
 			seekPending = false;
 		  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public org.apache.lucene.util.BytesRef next() throws java.io.IOException
+//ORIGINAL LINE: @Override public util.BytesRef next() throws java.io.IOException
 		  public override BytesRef next()
 		  {
 			if (seekPending) // previously positioned, but termOutputs not fetched
@@ -479,7 +480,7 @@ namespace org.apache.lucene.codecs.memory
 		  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public boolean seekExact(org.apache.lucene.util.BytesRef target) throws java.io.IOException
+//ORIGINAL LINE: @Override public boolean seekExact(util.BytesRef target) throws java.io.IOException
 		  public override bool seekExact(BytesRef target)
 		  {
 			updateEnum(fstEnum.seekExact(target));
@@ -487,7 +488,7 @@ namespace org.apache.lucene.codecs.memory
 		  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public SeekStatus seekCeil(org.apache.lucene.util.BytesRef target) throws java.io.IOException
+//ORIGINAL LINE: @Override public SeekStatus seekCeil(util.BytesRef target) throws java.io.IOException
 		  public override SeekStatus seekCeil(BytesRef target)
 		  {
 			updateEnum(fstEnum.seekCeil(target));
@@ -566,7 +567,7 @@ namespace org.apache.lucene.codecs.memory
 		  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: IntersectTermsEnum(org.apache.lucene.util.automaton.CompiledAutomaton compiled, org.apache.lucene.util.BytesRef startTerm) throws java.io.IOException
+//ORIGINAL LINE: IntersectTermsEnum(util.automaton.CompiledAutomaton compiled, util.BytesRef startTerm) throws java.io.IOException
 		  internal IntersectTermsEnum(FSTTermsReader.TermsReader outerInstance, CompiledAutomaton compiled, BytesRef startTerm) : base(outerInstance)
 		  {
 			  this.outerInstance = outerInstance;
@@ -619,11 +620,11 @@ namespace org.apache.lucene.codecs.memory
 			Debug.Assert(term_Renamed != null);
 			if (!decoded)
 			{
-			  if (meta.bytes != null)
+			  if (meta.BYTES != null)
 			  {
-				bytesReader.reset(meta.bytes, 0, meta.bytes.Length);
+				bytesReader.reset(meta.BYTES, 0, meta.BYTES.Length);
 			  }
-			  outerInstance.outerInstance.postingsReader.decodeTerm(meta.longs, bytesReader, outerInstance.fieldInfo, state, true);
+			  outerInstance.outerInstance.postingsReader.decodeTerm(meta.LONGS, bytesReader, outerInstance.fieldInfo, state, true);
 			  decoded = true;
 			}
 		  }
@@ -651,12 +652,12 @@ namespace org.apache.lucene.codecs.memory
 			{
 			  meta = last.output;
 			}
-			state.docFreq = meta.docFreq;
-			state.totalTermFreq = meta.totalTermFreq;
+			state.docFreq = meta.DOC_FREQ;
+			state.totalTermFreq = meta.TOTAL_TERM_FREQ;
 		  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public SeekStatus seekCeil(org.apache.lucene.util.BytesRef target) throws java.io.IOException
+//ORIGINAL LINE: @Override public SeekStatus seekCeil(util.BytesRef target) throws java.io.IOException
 		  public override SeekStatus seekCeil(BytesRef target)
 		  {
 			decoded = false;
@@ -673,7 +674,7 @@ namespace org.apache.lucene.codecs.memory
 		  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public org.apache.lucene.util.BytesRef next() throws java.io.IOException
+//ORIGINAL LINE: @Override public util.BytesRef next() throws java.io.IOException
 		  public override BytesRef next()
 		  {
 			//if (TEST) System.out.println("Enum next()");
@@ -719,7 +720,7 @@ namespace org.apache.lucene.codecs.memory
 		  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: private org.apache.lucene.util.BytesRef doSeekCeil(org.apache.lucene.util.BytesRef target) throws java.io.IOException
+//ORIGINAL LINE: private util.BytesRef doSeekCeil(util.BytesRef target) throws java.io.IOException
 		  internal BytesRef doSeekCeil(BytesRef target)
 		  {
 			//if (TEST) System.out.println("Enum doSeekCeil()");
@@ -894,7 +895,7 @@ namespace org.apache.lucene.codecs.memory
 			if (level + 1 == stack.Length)
 			{
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final Frame[] temp = new Frame[org.apache.lucene.util.ArrayUtil.oversize(level+2, org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+//ORIGINAL LINE: final Frame[] temp = new Frame[util.ArrayUtil.oversize(level+2, util.RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
 			  Frame[] temp = new Frame[ArrayUtil.oversize(level + 2, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
 			  Array.Copy(stack, 0, temp, 0, stack.Length);
 			  for (int i = stack.Length; i < temp.Length; i++)
@@ -944,26 +945,26 @@ namespace org.apache.lucene.codecs.memory
 	  }
 
 //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: static<T> void walk(org.apache.lucene.util.fst.FST<T> fst) throws java.io.IOException
+//ORIGINAL LINE: static<T> void walk(util.fst.FST<T> fst) throws java.io.IOException
 	  internal static void walk<T>(FST<T> fst)
 	  {
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final java.util.ArrayList<org.apache.lucene.util.fst.FST.Arc<T>> queue = new java.util.ArrayList<>();
+//ORIGINAL LINE: final java.util.ArrayList<util.fst.FST.Arc<T>> queue = new java.util.ArrayList<>();
 		List<FST.Arc<T>> queue = new List<FST.Arc<T>>();
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
 //ORIGINAL LINE: final java.util.BitSet seen = new java.util.BitSet();
 		BitArray seen = new BitArray();
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final org.apache.lucene.util.fst.FST.BytesReader reader = fst.getBytesReader();
+//ORIGINAL LINE: final util.fst.FST.BytesReader reader = fst.getBytesReader();
 		FST.BytesReader reader = fst.BytesReader;
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final org.apache.lucene.util.fst.FST.Arc<T> startArc = fst.getFirstArc(new org.apache.lucene.util.fst.FST.Arc<T>());
+//ORIGINAL LINE: final util.fst.FST.Arc<T> startArc = fst.getFirstArc(new util.fst.FST.Arc<T>());
 		FST.Arc<T> startArc = fst.getFirstArc(new FST.Arc<T>());
 		queue.Add(startArc);
 		while (queue.Count > 0)
 		{
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final org.apache.lucene.util.fst.FST.Arc<T> arc = queue.remove(0);
+//ORIGINAL LINE: final util.fst.FST.Arc<T> arc = queue.remove(0);
 		  FST.Arc<T> arc = queue.Remove(0);
 //JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
 //ORIGINAL LINE: final long node = arc.target;

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/af4d125b/src/Lucene.Net.Codecs/Memory/FSTTermsWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Memory/FSTTermsWriter.cs b/src/Lucene.Net.Codecs/Memory/FSTTermsWriter.cs
index 785b137..55b0631 100644
--- a/src/Lucene.Net.Codecs/Memory/FSTTermsWriter.cs
+++ b/src/Lucene.Net.Codecs/Memory/FSTTermsWriter.cs
@@ -1,329 +1,301 @@
-using System.Collections.Generic;
+/*
+ * 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.
+ */
 
-namespace org.apache.lucene.codecs.memory
+namespace Lucene.Net.Codecs.Memory
 {
+    using System.Collections.Generic;
+    using System.IO;
 
-	/*
-	 * 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 IndexOptions = Index.FieldInfo.IndexOptions;
+    using FieldInfo = Index.FieldInfo;
+    using FieldInfos = Index.FieldInfos;
+    using IndexFileNames = Index.IndexFileNames;
+    using SegmentWriteState = Index.SegmentWriteState;
+    using DataOutput = Store.DataOutput;
+    using IndexOutput = Store.IndexOutput;
+    using RAMOutputStream = Store.RAMOutputStream;
+    using BytesRef = Util.BytesRef;
+    using IOUtils = Util.IOUtils;
+    using IntsRef = Util.IntsRef;
+    using FST = Util.Fst.FST;
+    using Util.Fst;
+    using Util = Util.Fst.Util;
 
+    /// <summary>
+    /// FST-based term dict, using metadata as FST output.
+    /// 
+    /// The FST directly holds the mapping between &lt;term, metadata&gt;.
+    /// 
+    /// Term metadata consists of three parts:
+    /// 1. term statistics: docFreq, totalTermFreq;
+    /// 2. monotonic long[], e.g. the pointer to the postings list for that term;
+    /// 3. generic byte[], e.g. other information need by postings reader.
+    /// 
+    /// <para>
+    /// File:
+    /// <ul>
+    ///   <li><tt>.tst</tt>: <a href="#Termdictionary">Term Dictionary</a></li>
+    /// </ul>
+    /// </para>
+    /// <para>
+    /// 
+    /// <a name="Termdictionary" id="Termdictionary"></a>
+    /// <h3>Term Dictionary</h3>
+    /// </para>
+    /// <para>
+    ///  The .tst contains a list of FSTs, one for each field.
+    ///  The FST maps a term to its corresponding statistics (e.g. docfreq) 
+    ///  and metadata (e.g. information for postings list reader like file pointer
+    ///  to postings list).
+    /// </para>
+    /// <para>
+    ///  Typically the metadata is separated into two parts:
+    ///  <ul>
+    ///   <li>
+    ///    Monotonical long array: Some metadata will always be ascending in order
+    ///    with the corresponding term. This part is used by FST to share outputs between arcs.
+    ///   </li>
+    ///   <li>
+    ///    Generic byte array: Used to store non-monotonic metadata.
+    ///   </li>
+    ///  </ul>
+    /// </para>
+    /// 
+    /// File format:
+    /// <ul>
+    ///  <li>TermsDict(.tst) --&gt; Header, <i>PostingsHeader</i>, FieldSummary, DirOffset</li>
+    ///  <li>FieldSummary --&gt; NumFields, &lt;FieldNumber, NumTerms, SumTotalTermFreq?, 
+    ///                                      SumDocFreq, DocCount, LongsSize, TermFST &gt;<sup>NumFields</sup></li>
+    ///  <li>TermFST TermData
+    ///  <li>TermData --&gt; Flag, BytesSize?, LongDelta<sup>LongsSize</sup>?, Byte<sup>BytesSize</sup>?, 
+    ///                      &lt; DocFreq[Same?], (TotalTermFreq-DocFreq) &gt; ? </li>
+    ///  <li>Header --&gt; <seealso cref="CodecUtil#writeHeader CodecHeader"/></li>
+    ///  <li>DirOffset --&gt; <seealso cref="DataOutput#writeLong Uint64"/></li>
+    ///  <li>DocFreq, LongsSize, BytesSize, NumFields,
+    ///        FieldNumber, DocCount --&gt; <seealso cref="DataOutput#writeVInt VInt"/></li>
+    ///  <li>TotalTermFreq, NumTerms, SumTotalTermFreq, SumDocFreq, LongDelta --&gt; 
+    ///        <seealso cref="DataOutput#writeVLong VLong"/></li>
+    /// </ul>
+    /// <para>Notes:</para>
+    /// <ul>
+    ///  <li>
+    ///   The format of PostingsHeader and generic meta bytes are customized by the specific postings implementation:
+    ///   they contain arbitrary per-file data (such as parameters or versioning information), and per-term data
+    ///   (non-monotonic ones like pulsed postings data).
+    ///  </li>
+    ///  <li>
+    ///   The format of TermData is determined by FST, typically monotonic metadata will be dense around shallow arcs,
+    ///   while in deeper arcs only generic bytes and term statistics exist.
+    ///  </li>
+    ///  <li>
+    ///   The byte Flag is used to indicate which part of metadata exists on current arc. Specially the monotonic part
+    ///   is omitted when it is an array of 0s.
+    ///  </li>
+    ///  <li>
+    ///   Since LongsSize is per-field fixed, it is only written once in field summary.
+    ///  </li>
+    /// </ul>
+    /// 
+    /// @lucene.experimental
+    /// </summary>
+    public class FSTTermsWriter : FieldsConsumer
+    {
+        internal const string TERMS_EXTENSION = "tmp";
+        internal const string TERMS_CODEC_NAME = "FST_TERMS_DICT";
+        public const int TERMS_VERSION_START = 0;
+        public const int TERMS_VERSION_CHECKSUM = 1;
+        public const int TERMS_VERSION_CURRENT = TERMS_VERSION_CHECKSUM;
 
-	using IndexOptions = org.apache.lucene.index.FieldInfo.IndexOptions;
-	using FieldInfo = org.apache.lucene.index.FieldInfo;
-	using FieldInfos = org.apache.lucene.index.FieldInfos;
-	using IndexFileNames = org.apache.lucene.index.IndexFileNames;
-	using SegmentWriteState = org.apache.lucene.index.SegmentWriteState;
-	using DataOutput = org.apache.lucene.store.DataOutput;
-	using IndexOutput = org.apache.lucene.store.IndexOutput;
-	using RAMOutputStream = org.apache.lucene.store.RAMOutputStream;
-	using ArrayUtil = org.apache.lucene.util.ArrayUtil;
-	using BytesRef = org.apache.lucene.util.BytesRef;
-	using IOUtils = org.apache.lucene.util.IOUtils;
-	using IntsRef = org.apache.lucene.util.IntsRef;
-	using Builder = org.apache.lucene.util.fst.Builder;
-	using FST = org.apache.lucene.util.fst.FST;
-	using Util = org.apache.lucene.util.fst.Util;
+        private readonly PostingsWriterBase _postingsWriter;
+        private readonly FieldInfos _fieldInfos;
+        private IndexOutput _output;
+        private readonly IList<FieldMetaData> _fields = new List<FieldMetaData>();
 
-	/// <summary>
-	/// FST-based term dict, using metadata as FST output.
-	/// 
-	/// The FST directly holds the mapping between &lt;term, metadata&gt;.
-	/// 
-	/// Term metadata consists of three parts:
-	/// 1. term statistics: docFreq, totalTermFreq;
-	/// 2. monotonic long[], e.g. the pointer to the postings list for that term;
-	/// 3. generic byte[], e.g. other information need by postings reader.
-	/// 
-	/// <para>
-	/// File:
-	/// <ul>
-	///   <li><tt>.tst</tt>: <a href="#Termdictionary">Term Dictionary</a></li>
-	/// </ul>
-	/// </para>
-	/// <para>
-	/// 
-	/// <a name="Termdictionary" id="Termdictionary"></a>
-	/// <h3>Term Dictionary</h3>
-	/// </para>
-	/// <para>
-	///  The .tst contains a list of FSTs, one for each field.
-	///  The FST maps a term to its corresponding statistics (e.g. docfreq) 
-	///  and metadata (e.g. information for postings list reader like file pointer
-	///  to postings list).
-	/// </para>
-	/// <para>
-	///  Typically the metadata is separated into two parts:
-	///  <ul>
-	///   <li>
-	///    Monotonical long array: Some metadata will always be ascending in order
-	///    with the corresponding term. This part is used by FST to share outputs between arcs.
-	///   </li>
-	///   <li>
-	///    Generic byte array: Used to store non-monotonic metadata.
-	///   </li>
-	///  </ul>
-	/// </para>
-	/// 
-	/// File format:
-	/// <ul>
-	///  <li>TermsDict(.tst) --&gt; Header, <i>PostingsHeader</i>, FieldSummary, DirOffset</li>
-	///  <li>FieldSummary --&gt; NumFields, &lt;FieldNumber, NumTerms, SumTotalTermFreq?, 
-	///                                      SumDocFreq, DocCount, LongsSize, TermFST &gt;<sup>NumFields</sup></li>
-	///  <li>TermFST --&gt; <seealso cref="FST FST&lt;TermData&gt;"/></li>
-	///  <li>TermData --&gt; Flag, BytesSize?, LongDelta<sup>LongsSize</sup>?, Byte<sup>BytesSize</sup>?, 
-	///                      &lt; DocFreq[Same?], (TotalTermFreq-DocFreq) &gt; ? </li>
-	///  <li>Header --&gt; <seealso cref="CodecUtil#writeHeader CodecHeader"/></li>
-	///  <li>DirOffset --&gt; <seealso cref="DataOutput#writeLong Uint64"/></li>
-	///  <li>DocFreq, LongsSize, BytesSize, NumFields,
-	///        FieldNumber, DocCount --&gt; <seealso cref="DataOutput#writeVInt VInt"/></li>
-	///  <li>TotalTermFreq, NumTerms, SumTotalTermFreq, SumDocFreq, LongDelta --&gt; 
-	///        <seealso cref="DataOutput#writeVLong VLong"/></li>
-	/// </ul>
-	/// <para>Notes:</para>
-	/// <ul>
-	///  <li>
-	///   The format of PostingsHeader and generic meta bytes are customized by the specific postings implementation:
-	///   they contain arbitrary per-file data (such as parameters or versioning information), and per-term data
-	///   (non-monotonic ones like pulsed postings data).
-	///  </li>
-	///  <li>
-	///   The format of TermData is determined by FST, typically monotonic metadata will be dense around shallow arcs,
-	///   while in deeper arcs only generic bytes and term statistics exist.
-	///  </li>
-	///  <li>
-	///   The byte Flag is used to indicate which part of metadata exists on current arc. Specially the monotonic part
-	///   is omitted when it is an array of 0s.
-	///  </li>
-	///  <li>
-	///   Since LongsSize is per-field fixed, it is only written once in field summary.
-	///  </li>
-	/// </ul>
-	/// 
-	/// @lucene.experimental
-	/// </summary>
+        public FSTTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter)
+        {
+            var termsFileName = IndexFileNames.SegmentFileName(state.SegmentInfo.Name, state.SegmentSuffix,
+                TERMS_EXTENSION);
 
-	public class FSTTermsWriter : FieldsConsumer
-	{
-	  internal const string TERMS_EXTENSION = "tmp";
-	  internal const string TERMS_CODEC_NAME = "FST_TERMS_DICT";
-	  public const int TERMS_VERSION_START = 0;
-	  public const int TERMS_VERSION_CHECKSUM = 1;
-	  public const int TERMS_VERSION_CURRENT = TERMS_VERSION_CHECKSUM;
+            _postingsWriter = postingsWriter;
+            _fieldInfos = state.FieldInfos;
+            _output = state.Directory.CreateOutput(termsFileName, state.Context);
 
-	  internal readonly PostingsWriterBase postingsWriter;
-	  internal readonly FieldInfos fieldInfos;
-	  internal IndexOutput @out;
-	  internal readonly IList<FieldMetaData> fields = new List<FieldMetaData>();
+            var success = false;
+            try
+            {
+                WriteHeader(_output);
+                _postingsWriter.Init(_output);
+                success = true;
+            }
+            finally
+            {
+                if (!success)
+                {
+                    IOUtils.CloseWhileHandlingException(_output);
+                }
+            }
+        }
 
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: public FSTTermsWriter(org.apache.lucene.index.SegmentWriteState state, org.apache.lucene.codecs.PostingsWriterBase postingsWriter) throws java.io.IOException
-	  public FSTTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter)
-	  {
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final String termsFileName = org.apache.lucene.index.IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
-		string termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
+        private void WriteHeader(IndexOutput output)
+        {
+            CodecUtil.WriteHeader(output, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);
+        }
 
-		this.postingsWriter = postingsWriter;
-		this.fieldInfos = state.fieldInfos;
-		this.@out = state.directory.createOutput(termsFileName, state.context);
+        private static void WriteTrailer(IndexOutput output, long dirStart)
+        {
+            output.WriteLong(dirStart);
+        }
 
-		bool success = false;
-		try
-		{
-		  writeHeader(@out);
-		  this.postingsWriter.init(@out);
-		  success = true;
-		}
-		finally
-		{
-		  if (!success)
-		  {
-			IOUtils.closeWhileHandlingException(@out);
-		  }
-		}
-	  }
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: private void writeHeader(org.apache.lucene.store.IndexOutput out) throws java.io.IOException
-	  private void writeHeader(IndexOutput @out)
-	  {
-		CodecUtil.writeHeader(@out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);
-	  }
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: private void writeTrailer(org.apache.lucene.store.IndexOutput out, long dirStart) throws java.io.IOException
-	  private void writeTrailer(IndexOutput @out, long dirStart)
-	  {
-		@out.writeLong(dirStart);
-	  }
+        public override TermsConsumer AddField(FieldInfo field)
+        {
+            return new TermsWriter(this, field);
+        }
 
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public org.apache.lucene.codecs.TermsConsumer addField(org.apache.lucene.index.FieldInfo field) throws java.io.IOException
-	  public override TermsConsumer addField(FieldInfo field)
-	  {
-		return new TermsWriter(this, field);
-	  }
+        public override void Dispose()
+        {
+            if (_output == null) return;
 
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public void close() throws java.io.IOException
-	  public override void close()
-	  {
-		if (@out != null)
-		{
-		  IOException ioe = null;
-		  try
-		  {
-			// write field summary
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final long dirStart = out.getFilePointer();
-			long dirStart = @out.FilePointer;
+            IOException ioe = null;
+            try
+            {
+                // write field summary
+                var dirStart = _output.FilePointer;
 
-			@out.writeVInt(fields.Count);
-			foreach (FieldMetaData field in fields)
-			{
-			  @out.writeVInt(field.fieldInfo.number);
-			  @out.writeVLong(field.numTerms);
-			  if (field.fieldInfo.IndexOptions != IndexOptions.DOCS_ONLY)
-			  {
-				@out.writeVLong(field.sumTotalTermFreq);
-			  }
-			  @out.writeVLong(field.sumDocFreq);
-			  @out.writeVInt(field.docCount);
-			  @out.writeVInt(field.longsSize);
-			  field.dict.save(@out);
-			}
-			writeTrailer(@out, dirStart);
-			CodecUtil.writeFooter(@out);
-		  }
-		  catch (IOException ioe2)
-		  {
-			ioe = ioe2;
-		  }
-		  finally
-		  {
-			IOUtils.closeWhileHandlingException(ioe, @out, postingsWriter);
-			@out = null;
-		  }
-		}
-	  }
+                _output.WriteVInt(_fields.Count);
+                foreach (var field in _fields)
+                {
+                    _output.WriteVInt(field.FieldInfo.Number);
+                    _output.WriteVLong(field.NumTerms);
+                    if (field.FieldInfo.FieldIndexOptions != IndexOptions.DOCS_ONLY)
+                    {
+                        _output.WriteVLong(field.SumTotalTermFreq);
+                    }
+                    _output.WriteVLong(field.SumDocFreq);
+                    _output.WriteVInt(field.DocCount);
+                    _output.WriteVInt(field.LongsSize);
+                    field.Dict.Save(_output);
+                }
+                WriteTrailer(_output, dirStart);
+                CodecUtil.WriteFooter(_output);
+            }
+            catch (IOException ioe2)
+            {
+                ioe = ioe2;
+            }
+            finally
+            {
+                IOUtils.CloseWhileHandlingException(ioe, _output, _postingsWriter);
+                _output = null;
+            }
+        }
 
-	  private class FieldMetaData
-	  {
-		public readonly FieldInfo fieldInfo;
-		public readonly long numTerms;
-		public readonly long sumTotalTermFreq;
-		public readonly long sumDocFreq;
-		public readonly int docCount;
-		public readonly int longsSize;
-		public readonly FST<FSTTermOutputs.TermData> dict;
+        private class FieldMetaData
+        {
+            public FieldInfo FieldInfo { get; private set; }
+            public long NumTerms { get; private set; }
+            public long SumTotalTermFreq { get; private set; }
+            public long SumDocFreq { get; private set; }
+            public int DocCount { get; private set; }
+            public int LongsSize { get; private set; }
+            public FST<FSTTermOutputs.TermData> Dict { get; private set; }
 
-		public FieldMetaData(FieldInfo fieldInfo, long numTerms, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize, FST<FSTTermOutputs.TermData> fst)
-		{
-		  this.fieldInfo = fieldInfo;
-		  this.numTerms = numTerms;
-		  this.sumTotalTermFreq = sumTotalTermFreq;
-		  this.sumDocFreq = sumDocFreq;
-		  this.docCount = docCount;
-		  this.longsSize = longsSize;
-		  this.dict = fst;
-		}
-	  }
+            public FieldMetaData(FieldInfo fieldInfo, long numTerms, long sumTotalTermFreq, long sumDocFreq,
+                int docCount, int longsSize, FST<FSTTermOutputs.TermData> fst)
+            {
+                FieldInfo = fieldInfo;
+                NumTerms = numTerms;
+                SumTotalTermFreq = sumTotalTermFreq;
+                SumDocFreq = sumDocFreq;
+                DocCount = docCount;
+                LongsSize = longsSize;
+                Dict = fst;
+            }
+        }
 
-	  internal sealed class TermsWriter : TermsConsumer
-	  {
-		  private readonly FSTTermsWriter outerInstance;
+        internal sealed class TermsWriter : TermsConsumer
+        {
+            private readonly FSTTermsWriter _outerInstance;
 
-		internal readonly Builder<FSTTermOutputs.TermData> builder;
-		internal readonly FSTTermOutputs outputs;
-		internal readonly FieldInfo fieldInfo;
-		internal readonly int longsSize;
-		internal long numTerms;
+            private readonly Builder<FSTTermOutputs.TermData> _builder;
+            private readonly FSTTermOutputs _outputs;
+            private readonly FieldInfo _fieldInfo;
+            private readonly int _longsSize;
+            private long _numTerms;
 
-		internal readonly IntsRef scratchTerm = new IntsRef();
-		internal readonly RAMOutputStream statsWriter = new RAMOutputStream();
-		internal readonly RAMOutputStream metaWriter = new RAMOutputStream();
+            private readonly IntsRef _scratchTerm = new IntsRef();
+            private readonly RAMOutputStream _statsWriter = new RAMOutputStream();
+            private readonly RAMOutputStream _metaWriter = new RAMOutputStream();
 
-		internal TermsWriter(FSTTermsWriter outerInstance, FieldInfo fieldInfo)
-		{
-			this.outerInstance = outerInstance;
-		  this.numTerms = 0;
-		  this.fieldInfo = fieldInfo;
-		  this.longsSize = outerInstance.postingsWriter.setField(fieldInfo);
-		  this.outputs = new FSTTermOutputs(fieldInfo, longsSize);
-		  this.builder = new Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
-		}
+            internal TermsWriter(FSTTermsWriter outerInstance, FieldInfo fieldInfo)
+            {
+                _outerInstance = outerInstance;
+                _numTerms = 0;
+                _fieldInfo = fieldInfo;
+                _longsSize = outerInstance._postingsWriter.SetField(fieldInfo);
+                _outputs = new FSTTermOutputs(fieldInfo, _longsSize);
+                _builder = new Builder<FSTTermOutputs.TermData>(FST.INPUT_TYPE.BYTE1, _outputs);
+            }
 
-		public override IComparer<BytesRef> Comparator
-		{
-			get
-			{
-			  return BytesRef.UTF8SortedAsUnicodeComparator;
-			}
-		}
+            public override IComparer<BytesRef> Comparator
+            {
+                get { return BytesRef.UTF8SortedAsUnicodeComparer; }
+            }
 
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public org.apache.lucene.codecs.PostingsConsumer startTerm(org.apache.lucene.util.BytesRef text) throws java.io.IOException
-		public override PostingsConsumer startTerm(BytesRef text)
-		{
-		  outerInstance.postingsWriter.startTerm();
-		  return outerInstance.postingsWriter;
-		}
+            public override PostingsConsumer StartTerm(BytesRef text)
+            {
+                _outerInstance._postingsWriter.StartTerm();
+                return _outerInstance._postingsWriter;
+            }
 
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public void finishTerm(org.apache.lucene.util.BytesRef text, org.apache.lucene.codecs.TermStats stats) throws java.io.IOException
-		public override void finishTerm(BytesRef text, TermStats stats)
-		{
-		  // write term meta data into fst
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final org.apache.lucene.codecs.BlockTermState state = postingsWriter.newTermState();
-		  BlockTermState state = outerInstance.postingsWriter.newTermState();
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final FSTTermOutputs.TermData meta = new FSTTermOutputs.TermData();
-		  FSTTermOutputs.TermData meta = new FSTTermOutputs.TermData();
-		  meta.longs = new long[longsSize];
-		  meta.bytes = null;
-		  meta.docFreq = state.docFreq = stats.docFreq;
-		  meta.totalTermFreq = state.totalTermFreq = stats.totalTermFreq;
-		  outerInstance.postingsWriter.finishTerm(state);
-		  outerInstance.postingsWriter.encodeTerm(meta.longs, metaWriter, fieldInfo, state, true);
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final int bytesSize = (int)metaWriter.getFilePointer();
-		  int bytesSize = (int)metaWriter.FilePointer;
-		  if (bytesSize > 0)
-		  {
-			meta.bytes = new sbyte[bytesSize];
-			metaWriter.writeTo(meta.bytes, 0);
-			metaWriter.reset();
-		  }
-		  builder.add(Util.toIntsRef(text, scratchTerm), meta);
-		  numTerms++;
-		}
+            public override void FinishTerm(BytesRef text, TermStats stats)
+            {
+                // write term meta data into fst
 
-//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
-//ORIGINAL LINE: @Override public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws java.io.IOException
-		public override void finish(long sumTotalTermFreq, long sumDocFreq, int docCount)
-		{
-		  // save FST dict
-		  if (numTerms > 0)
-		  {
-//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
-//ORIGINAL LINE: final org.apache.lucene.util.fst.FST<FSTTermOutputs.TermData> fst = builder.finish();
-			FST<FSTTermOutputs.TermData> fst = builder.finish();
-			outerInstance.fields.Add(new FieldMetaData(fieldInfo, numTerms, sumTotalTermFreq, sumDocFreq, docCount, longsSize, fst));
-		  }
-		}
-	  }
-	}
+                var state = _outerInstance._postingsWriter.NewTermState();
 
+                var meta = new FSTTermOutputs.TermData
+                {
+                    longs = new long[_longsSize],
+                    bytes = null,
+                    docFreq = state.DocFreq = stats.DocFreq,
+                    totalTermFreq = state.TotalTermFreq = stats.TotalTermFreq
+                };
+                _outerInstance._postingsWriter.FinishTerm(state);
+                _outerInstance._postingsWriter.EncodeTerm(meta.longs, _metaWriter, _fieldInfo, state, true);
+                var bytesSize = (int) _metaWriter.FilePointer;
+                if (bytesSize > 0)
+                {
+                    meta.bytes = new sbyte[bytesSize];
+                    _metaWriter.WriteTo(meta.bytes, 0);
+                    _metaWriter.Reset();
+                }
+                _builder.Add(Util.ToIntsRef(text, _scratchTerm), meta);
+                _numTerms++;
+            }
+
+            public override void Finish(long sumTotalTermFreq, long sumDocFreq, int docCount)
+            {
+                // save FST dict
+                if (_numTerms <= 0) return;
+
+                var fst = _builder.Finish();
+                _outerInstance._fields.Add(new FieldMetaData(_fieldInfo, _numTerms, sumTotalTermFreq, sumDocFreq,
+                    docCount, _longsSize, fst));
+            }
+        }
+    }
 }
\ No newline at end of file