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/28 10:50:11 UTC

[07/10] Lucene.Net.Codes/Sep fully ported, work done on SimpleText and Memory as well

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d852d5b0/src/Lucene.Net.Codecs/Memory/FSTOrdTermsWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Memory/FSTOrdTermsWriter.cs b/src/Lucene.Net.Codecs/Memory/FSTOrdTermsWriter.cs
index 76d0092..51c7278 100644
--- a/src/Lucene.Net.Codecs/Memory/FSTOrdTermsWriter.cs
+++ b/src/Lucene.Net.Codecs/Memory/FSTOrdTermsWriter.cs
@@ -1,374 +1,435 @@
-package 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.
- */
-
-import java.io.IOException;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Comparator;
-
-import index.FieldInfo.IndexOptions;
-import index.FieldInfo;
-import index.FieldInfos;
-import index.IndexFileNames;
-import index.SegmentWriteState;
-import store.DataOutput;
-import store.IndexOutput;
-import store.RAMOutputStream;
-import util.ArrayUtil;
-import util.BytesRef;
-import util.IOUtils;
-import util.IntsRef;
-import util.fst.Builder;
-import util.fst.FST;
-import util.fst.PositiveIntOutputs;
-import util.fst.Util;
-import codecs.BlockTermState;
-import codecs.PostingsWriterBase;
-import codecs.PostingsConsumer;
-import codecs.FieldsConsumer;
-import codecs.TermsConsumer;
-import codecs.TermStats;
-import codecs.CodecUtil;
-
-/** 
- * FST-based term dict, using ord as FST output.
- *
- * The FST holds the mapping between <term, ord>, and 
- * term's metadata is delta encoded into a single byte block.
- *
- * Typically the byte block consists of four 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 customized by postings base.
- * 4. single-level skip list to speed up metadata decoding by ord.
- *
- * <p>
- * Files:
- * <ul>
- *  <li><tt>.tix</tt>: <a href="#Termindex">Term Index</a></li>
- *  <li><tt>.tbk</tt>: <a href="#Termblock">Term Block</a></li>
- * </ul>
- * </p>
- *
- * <a name="Termindex" id="Termindex"></a>
- * <h3>Term Index</h3>
- * <p>
- *  The .tix contains a list of FSTs, one for each field.
- *  The FST maps a term to its corresponding order in current field.
- * </p>
- * 
- * <ul>
- *  <li>TermIndex(.tix) --&gt; Header, TermFST<sup>NumFields</sup>, Footer</li>
- *  <li>TermFST --&gt; {@link FST FST&lt;long&gt;}</li>
- *  <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- *  <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
- *
- * <p>Notes:</p>
- * <ul>
- *  <li>
- *  Since terms are already sorted before writing to <a href="#Termblock">Term Block</a>, 
- *  their ords can directly used to seek term metadata from term block.
- *  </li>
- * </ul>
- *
- * <a name="Termblock" id="Termblock"></a>
- * <h3>Term Block</h3>
- * <p>
- *  The .tbk contains all the statistics and metadata for terms, along with field summary (e.g. 
- *  per-field data like number of documents in current field). For each field, there are four blocks:
- *  <ul>
- *   <li>statistics bytes block: contains term statistics; </li>
- *   <li>metadata longs block: delta-encodes monotonic part of metadata; </li>
- *   <li>metadata bytes block: encodes other parts of metadata; </li>
- *   <li>skip block: contains skip data, to speed up metadata seeking and decoding</li>
- *  </ul>
- * </p>
- *
- * <p>File Format:</p>
- * <ul>
- *  <li>TermBlock(.tbk) --&gt; Header, <i>PostingsHeader</i>, FieldSummary, DirOffset</li>
- *  <li>FieldSummary --&gt; NumFields, &lt;FieldNumber, NumTerms, SumTotalTermFreq?, SumDocFreq,
- *                                         DocCount, LongsSize, DataBlock &gt; <sup>NumFields</sup>, Footer</li>
- *
- *  <li>DataBlock --&gt; StatsBlockLength, MetaLongsBlockLength, MetaBytesBlockLength, 
- *                       SkipBlock, StatsBlock, MetaLongsBlock, MetaBytesBlock </li>
- *  <li>SkipBlock --&gt; &lt; StatsFPDelta, MetaLongsSkipFPDelta, MetaBytesSkipFPDelta, 
- *                            MetaLongsSkipDelta<sup>LongsSize</sup> &gt;<sup>NumTerms</sup>
- *  <li>StatsBlock --&gt; &lt; DocFreq[Same?], (TotalTermFreq-DocFreq) ? &gt; <sup>NumTerms</sup>
- *  <li>MetaLongsBlock --&gt; &lt; LongDelta<sup>LongsSize</sup>, BytesSize &gt; <sup>NumTerms</sup>
- *  <li>MetaBytesBlock --&gt; Byte <sup>MetaBytesBlockLength</sup>
- *  <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- *  <li>DirOffset --&gt; {@link DataOutput#writeLong Uint64}</li>
- *  <li>NumFields, FieldNumber, DocCount, DocFreq, LongsSize, 
- *        FieldNumber, DocCount --&gt; {@link DataOutput#writeVInt VInt}</li>
- *  <li>NumTerms, SumTotalTermFreq, SumDocFreq, StatsBlockLength, MetaLongsBlockLength, MetaBytesBlockLength,
- *        StatsFPDelta, MetaLongsSkipFPDelta, MetaBytesSkipFPDelta, MetaLongsSkipStart, TotalTermFreq, 
- *        LongDelta,--&gt; {@link DataOutput#writeVLong VLong}</li>
- *  <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
- * <p>Notes: </p>
- * <ul>
- *  <li>
- *   The format of PostingsHeader and MetaBytes 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>
- *   During initialization the reader will load all the blocks into memory. SkipBlock will be decoded, so that during seek
- *   term dict can lookup file pointers directly. StatsFPDelta, MetaLongsSkipFPDelta, etc. are file offset
- *   for every SkipInterval's term. MetaLongsSkipDelta is the difference from previous one, which indicates
- *   the value of preceding metadata longs for every SkipInterval's term.
- *  </li>
- *  <li>
- *   DocFreq is the count of documents which contain the term. TotalTermFreq is the total number of occurrences of the term. 
- *   Usually these two values are the same for long tail terms, therefore one bit is stole from DocFreq to check this case,
- *   so that encoding of TotalTermFreq may be omitted.
- *  </li>
- * </ul>
- *
- * @lucene.experimental 
- */
-
-public class FSTOrdTermsWriter extends FieldsConsumer {
-  static final String TERMS_INDEX_EXTENSION = "tix";
-  static final String TERMS_BLOCK_EXTENSION = "tbk";
-  static final String TERMS_CODEC_NAME = "FST_ORD_TERMS_DICT";
-  public static final int TERMS_VERSION_START = 0;
-  public static final int TERMS_VERSION_CHECKSUM = 1;
-  public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_CHECKSUM;
-  public static final int SKIP_INTERVAL = 8;
-  
-  final PostingsWriterBase postingsWriter;
-  final FieldInfos fieldInfos;
-  final List<FieldMetaData> fields = new ArrayList<>();
-  IndexOutput blockOut = null;
-  IndexOutput indexOut = null;
-
-  public FSTOrdTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter)  {
-    final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION);
-    final String termsBlockFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_BLOCK_EXTENSION);
-
-    this.postingsWriter = postingsWriter;
-    this.fieldInfos = state.fieldInfos;
-
-    bool success = false;
-    try {
-      this.indexOut = state.directory.createOutput(termsIndexFileName, state.context);
-      this.blockOut = state.directory.createOutput(termsBlockFileName, state.context);
-      writeHeader(indexOut);
-      writeHeader(blockOut);
-      this.postingsWriter.init(blockOut); 
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeWhileHandlingException(indexOut, blockOut);
-      }
-    }
-  }
-
-  @Override
-  public TermsConsumer addField(FieldInfo field)  {
-    return new TermsWriter(field);
-  }
-
-  @Override
-  public void close()  {
-    if (blockOut != null) {
-      IOException ioe = null;
-      try {
-        final long blockDirStart = blockOut.getFilePointer();
-        
-        // write field summary
-        blockOut.writeVInt(fields.size());
-        for (FieldMetaData field : fields) {
-          blockOut.writeVInt(field.fieldInfo.number);
-          blockOut.writeVLong(field.numTerms);
-          if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
-            blockOut.writeVLong(field.sumTotalTermFreq);
-          }
-          blockOut.writeVLong(field.sumDocFreq);
-          blockOut.writeVInt(field.docCount);
-          blockOut.writeVInt(field.longsSize);
-          blockOut.writeVLong(field.statsOut.getFilePointer());
-          blockOut.writeVLong(field.metaLongsOut.getFilePointer());
-          blockOut.writeVLong(field.metaBytesOut.getFilePointer());
-          
-          field.skipOut.writeTo(blockOut);
-          field.statsOut.writeTo(blockOut);
-          field.metaLongsOut.writeTo(blockOut);
-          field.metaBytesOut.writeTo(blockOut);
-          field.dict.save(indexOut);
-        }
-        writeTrailer(blockOut, blockDirStart);
-        CodecUtil.writeFooter(indexOut);
-        CodecUtil.writeFooter(blockOut);
-      } catch (IOException ioe2) {
-        ioe = ioe2;
-      } finally {
-        IOUtils.closeWhileHandlingException(ioe, blockOut, indexOut, postingsWriter);
-        blockOut = null;
-      }
-    }
-  }
-
-  private void writeHeader(IndexOutput out)  {
-    CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);   
-  }
-  private void writeTrailer(IndexOutput out, long dirStart)  {
-    out.writeLong(dirStart);
-  }
-
-  private static class FieldMetaData {
-    public FieldInfo fieldInfo;
-    public long numTerms;
-    public long sumTotalTermFreq;
-    public long sumDocFreq;
-    public int docCount;
-    public int longsSize;
-    public FST<Long> dict;
-
-    // TODO: block encode each part 
-
-    // vint encode next skip point (fully decoded when reading)
-    public RAMOutputStream skipOut;
-    // vint encode df, (ttf-df)
-    public RAMOutputStream statsOut;
-    // vint encode monotonic long[] and length for corresponding byte[]
-    public RAMOutputStream metaLongsOut;
-    // generic byte[]
-    public RAMOutputStream metaBytesOut;
-  }
-
-  final class TermsWriter extends TermsConsumer {
-    private final Builder<Long> builder;
-    private final PositiveIntOutputs outputs;
-    private final FieldInfo fieldInfo;
-    private final int longsSize;
-    private long numTerms;
-
-    private final IntsRef scratchTerm = new IntsRef();
-    private final RAMOutputStream statsOut = new RAMOutputStream();
-    private final RAMOutputStream metaLongsOut = new RAMOutputStream();
-    private final RAMOutputStream metaBytesOut = new RAMOutputStream();
-
-    private final RAMOutputStream skipOut = new RAMOutputStream();
-    private long lastBlockStatsFP;
-    private long lastBlockMetaLongsFP;
-    private long lastBlockMetaBytesFP;
-    private long[] lastBlockLongs;
-
-    private long[] lastLongs;
-    private long lastMetaBytesFP;
-
-    TermsWriter(FieldInfo fieldInfo) {
-      this.numTerms = 0;
-      this.fieldInfo = fieldInfo;
-      this.longsSize = postingsWriter.setField(fieldInfo);
-      this.outputs = PositiveIntOutputs.getSingleton();
-      this.builder = new Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
-
-      this.lastBlockStatsFP = 0;
-      this.lastBlockMetaLongsFP = 0;
-      this.lastBlockMetaBytesFP = 0;
-      this.lastBlockLongs = new long[longsSize];
-
-      this.lastLongs = new long[longsSize];
-      this.lastMetaBytesFP = 0;
-    }
-
-    @Override
-    public Comparator<BytesRef> getComparator() {
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
-    }
-
-    @Override
-    public PostingsConsumer startTerm(BytesRef text)  {
-      postingsWriter.startTerm();
-      return postingsWriter;
-    }
-
-    @Override
-    public void finishTerm(BytesRef text, TermStats stats)  {
-      if (numTerms > 0 && numTerms % SKIP_INTERVAL == 0) {
-        bufferSkip();
-      }
-      // write term meta data into fst
-      final long longs[] = new long[longsSize];
-      final long delta = stats.totalTermFreq - stats.docFreq;
-      if (stats.totalTermFreq > 0) {
-        if (delta == 0) {
-          statsOut.writeVInt(stats.docFreq<<1|1);
-        } else {
-          statsOut.writeVInt(stats.docFreq<<1|0);
-          statsOut.writeVLong(stats.totalTermFreq-stats.docFreq);
-        }
-      } else {
-        statsOut.writeVInt(stats.docFreq);
-      }
-      BlockTermState state = postingsWriter.newTermState();
-      state.docFreq = stats.docFreq;
-      state.totalTermFreq = stats.totalTermFreq;
-      postingsWriter.finishTerm(state);
-      postingsWriter.encodeTerm(longs, metaBytesOut, fieldInfo, state, true);
-      for (int i = 0; i < longsSize; i++) {
-        metaLongsOut.writeVLong(longs[i] - lastLongs[i]);
-        lastLongs[i] = longs[i];
-      }
-      metaLongsOut.writeVLong(metaBytesOut.getFilePointer() - lastMetaBytesFP);
-
-      builder.add(Util.toIntsRef(text, scratchTerm), numTerms);
-      numTerms++;
-
-      lastMetaBytesFP = metaBytesOut.getFilePointer();
-    }
-
-    @Override
-    public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount)  {
-      if (numTerms > 0) {
-        final FieldMetaData metadata = new FieldMetaData();
-        metadata.fieldInfo = fieldInfo;
-        metadata.numTerms = numTerms;
-        metadata.sumTotalTermFreq = sumTotalTermFreq;
-        metadata.sumDocFreq = sumDocFreq;
-        metadata.docCount = docCount;
-        metadata.longsSize = longsSize;
-        metadata.skipOut = skipOut;
-        metadata.statsOut = statsOut;
-        metadata.metaLongsOut = metaLongsOut;
-        metadata.metaBytesOut = metaBytesOut;
-        metadata.dict = builder.finish();
-        fields.add(metadata);
-      }
-    }
-
-    private void bufferSkip()  {
-      skipOut.writeVLong(statsOut.getFilePointer() - lastBlockStatsFP);
-      skipOut.writeVLong(metaLongsOut.getFilePointer() - lastBlockMetaLongsFP);
-      skipOut.writeVLong(metaBytesOut.getFilePointer() - lastBlockMetaBytesFP);
-      for (int i = 0; i < longsSize; i++) {
-        skipOut.writeVLong(lastLongs[i] - lastBlockLongs[i]);
-      }
-      lastBlockStatsFP = statsOut.getFilePointer();
-      lastBlockMetaLongsFP = metaLongsOut.getFilePointer();
-      lastBlockMetaBytesFP = metaBytesOut.getFilePointer();
-      System.arraycopy(lastLongs, 0, lastBlockLongs, 0, longsSize);
-    }
-  }
-}
+using System;
+using System.Collections.Generic;
+
+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.
+	 */
+
+
+	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 PositiveIntOutputs = org.apache.lucene.util.fst.PositiveIntOutputs;
+	using Util = org.apache.lucene.util.fst.Util;
+
+	/// <summary>
+	/// FST-based term dict, using ord as FST output.
+	/// 
+	/// The FST holds the mapping between &lt;term, ord&gt;, and 
+	/// term's metadata is delta encoded into a single byte block.
+	/// 
+	/// Typically the byte block consists of four 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 customized by postings base.
+	/// 4. single-level skip list to speed up metadata decoding by ord.
+	/// 
+	/// <para>
+	/// Files:
+	/// <ul>
+	///  <li><tt>.tix</tt>: <a href="#Termindex">Term Index</a></li>
+	///  <li><tt>.tbk</tt>: <a href="#Termblock">Term Block</a></li>
+	/// </ul>
+	/// </para>
+	/// 
+	/// <a name="Termindex" id="Termindex"></a>
+	/// <h3>Term Index</h3>
+	/// <para>
+	///  The .tix contains a list of FSTs, one for each field.
+	///  The FST maps a term to its corresponding order in current field.
+	/// </para>
+	/// 
+	/// <ul>
+	///  <li>TermIndex(.tix) --&gt; Header, TermFST<sup>NumFields</sup>, Footer</li>
+	///  <li>TermFST --&gt; <seealso cref="FST FST&lt;long&gt;"/></li>
+	///  <li>Header --&gt; <seealso cref="CodecUtil#writeHeader CodecHeader"/></li>
+	///  <li>Footer --&gt; <seealso cref="CodecUtil#writeFooter CodecFooter"/></li>
+	/// </ul>
+	/// 
+	/// <para>Notes:</para>
+	/// <ul>
+	///  <li>
+	///  Since terms are already sorted before writing to <a href="#Termblock">Term Block</a>, 
+	///  their ords can directly used to seek term metadata from term block.
+	///  </li>
+	/// </ul>
+	/// 
+	/// <a name="Termblock" id="Termblock"></a>
+	/// <h3>Term Block</h3>
+	/// <para>
+	///  The .tbk contains all the statistics and metadata for terms, along with field summary (e.g. 
+	///  per-field data like number of documents in current field). For each field, there are four blocks:
+	///  <ul>
+	///   <li>statistics bytes block: contains term statistics; </li>
+	///   <li>metadata longs block: delta-encodes monotonic part of metadata; </li>
+	///   <li>metadata bytes block: encodes other parts of metadata; </li>
+	///   <li>skip block: contains skip data, to speed up metadata seeking and decoding</li>
+	///  </ul>
+	/// </para>
+	/// 
+	/// <para>File Format:</para>
+	/// <ul>
+	///  <li>TermBlock(.tbk) --&gt; Header, <i>PostingsHeader</i>, FieldSummary, DirOffset</li>
+	///  <li>FieldSummary --&gt; NumFields, &lt;FieldNumber, NumTerms, SumTotalTermFreq?, SumDocFreq,
+	///                                         DocCount, LongsSize, DataBlock &gt; <sup>NumFields</sup>, Footer</li>
+	/// 
+	///  <li>DataBlock --&gt; StatsBlockLength, MetaLongsBlockLength, MetaBytesBlockLength, 
+	///                       SkipBlock, StatsBlock, MetaLongsBlock, MetaBytesBlock </li>
+	///  <li>SkipBlock --&gt; &lt; StatsFPDelta, MetaLongsSkipFPDelta, MetaBytesSkipFPDelta, 
+	///                            MetaLongsSkipDelta<sup>LongsSize</sup> &gt;<sup>NumTerms</sup>
+	///  <li>StatsBlock --&gt; &lt; DocFreq[Same?], (TotalTermFreq-DocFreq) ? &gt; <sup>NumTerms</sup>
+	///  <li>MetaLongsBlock --&gt; &lt; LongDelta<sup>LongsSize</sup>, BytesSize &gt; <sup>NumTerms</sup>
+	///  <li>MetaBytesBlock --&gt; Byte <sup>MetaBytesBlockLength</sup>
+	///  <li>Header --&gt; <seealso cref="CodecUtil#writeHeader CodecHeader"/></li>
+	///  <li>DirOffset --&gt; <seealso cref="DataOutput#writeLong Uint64"/></li>
+	///  <li>NumFields, FieldNumber, DocCount, DocFreq, LongsSize, 
+	///        FieldNumber, DocCount --&gt; <seealso cref="DataOutput#writeVInt VInt"/></li>
+	///  <li>NumTerms, SumTotalTermFreq, SumDocFreq, StatsBlockLength, MetaLongsBlockLength, MetaBytesBlockLength,
+	///        StatsFPDelta, MetaLongsSkipFPDelta, MetaBytesSkipFPDelta, MetaLongsSkipStart, TotalTermFreq, 
+	///        LongDelta,--&gt; <seealso cref="DataOutput#writeVLong VLong"/></li>
+	///  <li>Footer --&gt; <seealso cref="CodecUtil#writeFooter CodecFooter"/></li>
+	/// </ul>
+	/// <para>Notes: </para>
+	/// <ul>
+	///  <li>
+	///   The format of PostingsHeader and MetaBytes 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>
+	///   During initialization the reader will load all the blocks into memory. SkipBlock will be decoded, so that during seek
+	///   term dict can lookup file pointers directly. StatsFPDelta, MetaLongsSkipFPDelta, etc. are file offset
+	///   for every SkipInterval's term. MetaLongsSkipDelta is the difference from previous one, which indicates
+	///   the value of preceding metadata longs for every SkipInterval's term.
+	///  </li>
+	///  <li>
+	///   DocFreq is the count of documents which contain the term. TotalTermFreq is the total number of occurrences of the term. 
+	///   Usually these two values are the same for long tail terms, therefore one bit is stole from DocFreq to check this case,
+	///   so that encoding of TotalTermFreq may be omitted.
+	///  </li>
+	/// </ul>
+	/// 
+	/// @lucene.experimental 
+	/// </summary>
+
+	public class FSTOrdTermsWriter : FieldsConsumer
+	{
+	  internal const string TERMS_INDEX_EXTENSION = "tix";
+	  internal const string TERMS_BLOCK_EXTENSION = "tbk";
+	  internal const string TERMS_CODEC_NAME = "FST_ORD_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;
+	  public const int SKIP_INTERVAL = 8;
+
+	  internal readonly PostingsWriterBase postingsWriter;
+	  internal readonly FieldInfos fieldInfos;
+	  internal readonly IList<FieldMetaData> fields = new List<FieldMetaData>();
+	  internal IndexOutput blockOut = null;
+	  internal IndexOutput indexOut = null;
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: public FSTOrdTermsWriter(org.apache.lucene.index.SegmentWriteState state, org.apache.lucene.codecs.PostingsWriterBase postingsWriter) throws java.io.IOException
+	  public FSTOrdTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter)
+	  {
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final String termsIndexFileName = org.apache.lucene.index.IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION);
+		string termsIndexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION);
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final String termsBlockFileName = org.apache.lucene.index.IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_BLOCK_EXTENSION);
+		string termsBlockFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_BLOCK_EXTENSION);
+
+		this.postingsWriter = postingsWriter;
+		this.fieldInfos = state.fieldInfos;
+
+		bool success = false;
+		try
+		{
+		  this.indexOut = state.directory.createOutput(termsIndexFileName, state.context);
+		  this.blockOut = state.directory.createOutput(termsBlockFileName, state.context);
+		  writeHeader(indexOut);
+		  writeHeader(blockOut);
+		  this.postingsWriter.init(blockOut);
+		  success = true;
+		}
+		finally
+		{
+		  if (!success)
+		  {
+			IOUtils.closeWhileHandlingException(indexOut, blockOut);
+		  }
+		}
+	  }
+
+//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);
+	  }
+
+//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 (blockOut != null)
+		{
+		  IOException ioe = null;
+		  try
+		  {
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final long blockDirStart = blockOut.getFilePointer();
+			long blockDirStart = blockOut.FilePointer;
+
+			// write field summary
+			blockOut.writeVInt(fields.Count);
+			foreach (FieldMetaData field in fields)
+			{
+			  blockOut.writeVInt(field.fieldInfo.number);
+			  blockOut.writeVLong(field.numTerms);
+			  if (field.fieldInfo.IndexOptions != IndexOptions.DOCS_ONLY)
+			  {
+				blockOut.writeVLong(field.sumTotalTermFreq);
+			  }
+			  blockOut.writeVLong(field.sumDocFreq);
+			  blockOut.writeVInt(field.docCount);
+			  blockOut.writeVInt(field.longsSize);
+			  blockOut.writeVLong(field.statsOut.FilePointer);
+			  blockOut.writeVLong(field.metaLongsOut.FilePointer);
+			  blockOut.writeVLong(field.metaBytesOut.FilePointer);
+
+			  field.skipOut.writeTo(blockOut);
+			  field.statsOut.writeTo(blockOut);
+			  field.metaLongsOut.writeTo(blockOut);
+			  field.metaBytesOut.writeTo(blockOut);
+			  field.dict.save(indexOut);
+			}
+			writeTrailer(blockOut, blockDirStart);
+			CodecUtil.writeFooter(indexOut);
+			CodecUtil.writeFooter(blockOut);
+		  }
+		  catch (IOException ioe2)
+		  {
+			ioe = ioe2;
+		  }
+		  finally
+		  {
+			IOUtils.closeWhileHandlingException(ioe, blockOut, indexOut, postingsWriter);
+			blockOut = null;
+		  }
+		}
+	  }
+
+//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);
+	  }
+
+	  private class FieldMetaData
+	  {
+		public FieldInfo fieldInfo;
+		public long numTerms;
+		public long sumTotalTermFreq;
+		public long sumDocFreq;
+		public int docCount;
+		public int longsSize;
+		public FST<long?> dict;
+
+		// TODO: block encode each part 
+
+		// vint encode next skip point (fully decoded when reading)
+		public RAMOutputStream skipOut;
+		// vint encode df, (ttf-df)
+		public RAMOutputStream statsOut;
+		// vint encode monotonic long[] and length for corresponding byte[]
+		public RAMOutputStream metaLongsOut;
+		// generic byte[]
+		public RAMOutputStream metaBytesOut;
+	  }
+
+	  internal sealed class TermsWriter : TermsConsumer
+	  {
+		  private readonly FSTOrdTermsWriter outerInstance;
+
+		internal readonly Builder<long?> builder;
+		internal readonly PositiveIntOutputs outputs;
+		internal readonly FieldInfo fieldInfo;
+		internal readonly int longsSize;
+		internal long numTerms;
+
+		internal readonly IntsRef scratchTerm = new IntsRef();
+		internal readonly RAMOutputStream statsOut = new RAMOutputStream();
+		internal readonly RAMOutputStream metaLongsOut = new RAMOutputStream();
+		internal readonly RAMOutputStream metaBytesOut = new RAMOutputStream();
+
+		internal readonly RAMOutputStream skipOut = new RAMOutputStream();
+		internal long lastBlockStatsFP;
+		internal long lastBlockMetaLongsFP;
+		internal long lastBlockMetaBytesFP;
+		internal long[] lastBlockLongs;
+
+		internal long[] lastLongs;
+		internal long lastMetaBytesFP;
+
+		internal TermsWriter(FSTOrdTermsWriter outerInstance, FieldInfo fieldInfo)
+		{
+			this.outerInstance = outerInstance;
+		  this.numTerms = 0;
+		  this.fieldInfo = fieldInfo;
+		  this.longsSize = outerInstance.postingsWriter.setField(fieldInfo);
+		  this.outputs = PositiveIntOutputs.Singleton;
+		  this.builder = new Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
+
+		  this.lastBlockStatsFP = 0;
+		  this.lastBlockMetaLongsFP = 0;
+		  this.lastBlockMetaBytesFP = 0;
+		  this.lastBlockLongs = new long[longsSize];
+
+		  this.lastLongs = new long[longsSize];
+		  this.lastMetaBytesFP = 0;
+		}
+
+		public override IComparer<BytesRef> Comparator
+		{
+			get
+			{
+			  return BytesRef.UTF8SortedAsUnicodeComparator;
+			}
+		}
+
+//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;
+		}
+
+//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)
+		{
+		  if (numTerms > 0 && numTerms % SKIP_INTERVAL == 0)
+		  {
+			bufferSkip();
+		  }
+		  // write term meta data into fst
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final long longs[] = new long[longsSize];
+		  long[] longs = new long[longsSize];
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final long delta = stats.totalTermFreq - stats.docFreq;
+		  long delta = stats.totalTermFreq - stats.docFreq;
+		  if (stats.totalTermFreq > 0)
+		  {
+			if (delta == 0)
+			{
+			  statsOut.writeVInt(stats.docFreq << 1 | 1);
+			}
+			else
+			{
+			  statsOut.writeVInt(stats.docFreq << 1 | 0);
+			  statsOut.writeVLong(stats.totalTermFreq - stats.docFreq);
+			}
+		  }
+		  else
+		  {
+			statsOut.writeVInt(stats.docFreq);
+		  }
+		  BlockTermState state = outerInstance.postingsWriter.newTermState();
+		  state.docFreq = stats.docFreq;
+		  state.totalTermFreq = stats.totalTermFreq;
+		  outerInstance.postingsWriter.finishTerm(state);
+		  outerInstance.postingsWriter.encodeTerm(longs, metaBytesOut, fieldInfo, state, true);
+		  for (int i = 0; i < longsSize; i++)
+		  {
+			metaLongsOut.writeVLong(longs[i] - lastLongs[i]);
+			lastLongs[i] = longs[i];
+		  }
+		  metaLongsOut.writeVLong(metaBytesOut.FilePointer - lastMetaBytesFP);
+
+		  builder.add(Util.toIntsRef(text, scratchTerm), numTerms);
+		  numTerms++;
+
+		  lastMetaBytesFP = metaBytesOut.FilePointer;
+		}
+
+//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)
+		{
+		  if (numTerms > 0)
+		  {
+//JAVA TO C# CONVERTER WARNING: The original Java variable was marked 'final':
+//ORIGINAL LINE: final FieldMetaData metadata = new FieldMetaData();
+			FieldMetaData metadata = new FieldMetaData();
+			metadata.fieldInfo = fieldInfo;
+			metadata.numTerms = numTerms;
+			metadata.sumTotalTermFreq = sumTotalTermFreq;
+			metadata.sumDocFreq = sumDocFreq;
+			metadata.docCount = docCount;
+			metadata.longsSize = longsSize;
+			metadata.skipOut = skipOut;
+			metadata.statsOut = statsOut;
+			metadata.metaLongsOut = metaLongsOut;
+			metadata.metaBytesOut = metaBytesOut;
+			metadata.dict = builder.finish();
+			outerInstance.fields.Add(metadata);
+		  }
+		}
+
+//JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+//ORIGINAL LINE: private void bufferSkip() throws java.io.IOException
+		internal void bufferSkip()
+		{
+		  skipOut.writeVLong(statsOut.FilePointer - lastBlockStatsFP);
+		  skipOut.writeVLong(metaLongsOut.FilePointer - lastBlockMetaLongsFP);
+		  skipOut.writeVLong(metaBytesOut.FilePointer - lastBlockMetaBytesFP);
+		  for (int i = 0; i < longsSize; i++)
+		  {
+			skipOut.writeVLong(lastLongs[i] - lastBlockLongs[i]);
+		  }
+		  lastBlockStatsFP = statsOut.FilePointer;
+		  lastBlockMetaLongsFP = metaLongsOut.FilePointer;
+		  lastBlockMetaBytesFP = metaBytesOut.FilePointer;
+		  Array.Copy(lastLongs, 0, lastBlockLongs, 0, longsSize);
+		}
+	  }
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d852d5b0/src/Lucene.Net.Codecs/Memory/FSTPostingsFormat.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Memory/FSTPostingsFormat.cs b/src/Lucene.Net.Codecs/Memory/FSTPostingsFormat.cs
index c5ea2f1..f41001b 100644
--- a/src/Lucene.Net.Codecs/Memory/FSTPostingsFormat.cs
+++ b/src/Lucene.Net.Codecs/Memory/FSTPostingsFormat.cs
@@ -1,83 +1,88 @@
-package codecs.memory;
+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.
+	 */
 
-import java.io.IOException;
+	using Lucene41PostingsWriter = org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
+	using Lucene41PostingsReader = org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
+	using IndexOptions = org.apache.lucene.index.FieldInfo.IndexOptions;
+	using SegmentReadState = org.apache.lucene.index.SegmentReadState;
+	using SegmentWriteState = org.apache.lucene.index.SegmentWriteState;
+	using IOUtils = org.apache.lucene.util.IOUtils;
 
-import codecs.FieldsConsumer;
-import codecs.FieldsProducer;
-import codecs.PostingsFormat;
-import codecs.PostingsReaderBase;
-import codecs.PostingsWriterBase;
-import codecs.lucene41.Lucene41PostingsWriter;
-import codecs.lucene41.Lucene41PostingsReader;
-import index.FieldInfo.IndexOptions;
-import index.SegmentReadState;
-import index.SegmentWriteState;
-import util.IOUtils;
+	/// <summary>
+	/// FST term dict + Lucene41PBF
+	/// </summary>
 
-/**
- * FST term dict + Lucene41PBF
- */
+	public sealed class FSTPostingsFormat : PostingsFormat
+	{
+	  public FSTPostingsFormat() : base("FST41")
+	  {
+	  }
 
-public final class FSTPostingsFormat extends PostingsFormat {
-  public FSTPostingsFormat() {
-    super("FST41");
-  }
+	  public override string ToString()
+	  {
+		return Name;
+	  }
 
-  @Override
-  public String toString() {
-    return getName();
-  }
+//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 postingsWriter = new Lucene41PostingsWriter(state);
 
-  @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state)  {
-    PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
+		bool success = false;
+		try
+		{
+		  FieldsConsumer ret = new FSTTermsWriter(state, postingsWriter);
+		  success = true;
+		  return ret;
+		}
+		finally
+		{
+		  if (!success)
+		  {
+			IOUtils.closeWhileHandlingException(postingsWriter);
+		  }
+		}
+	  }
 
-    bool success = false;
-    try {
-      FieldsConsumer ret = new FSTTermsWriter(state, postingsWriter);
-      success = true;
-      return ret;
-    } finally {
-      if (!success) {
-        IOUtils.closeWhileHandlingException(postingsWriter);
-      }
-    }
-  }
+//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 postingsReader = new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+		bool success = false;
+		try
+		{
+		  FieldsProducer ret = new FSTTermsReader(state, postingsReader);
+		  success = true;
+		  return ret;
+		}
+		finally
+		{
+		  if (!success)
+		  {
+			IOUtils.closeWhileHandlingException(postingsReader);
+		  }
+		}
+	  }
+	}
 
-  @Override
-  public FieldsProducer fieldsProducer(SegmentReadState state)  {
-    PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
-                                                                state.fieldInfos,
-                                                                state.segmentInfo,
-                                                                state.context,
-                                                                state.segmentSuffix);
-    bool success = false;
-    try {
-      FieldsProducer ret = new FSTTermsReader(state, postingsReader);
-      success = true;
-      return ret;
-    } finally {
-      if (!success) {
-        IOUtils.closeWhileHandlingException(postingsReader);
-      }
-    }
-  }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d852d5b0/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 0db3cbd..7ff2a8c 100644
--- a/src/Lucene.Net.Codecs/Memory/FSTPulsing41PostingsFormat.cs
+++ b/src/Lucene.Net.Codecs/Memory/FSTPulsing41PostingsFormat.cs
@@ -1,92 +1,102 @@
-package codecs.memory;
+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.
+	 */
 
-import java.io.IOException;
+	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;
 
-import codecs.FieldsConsumer;
-import codecs.FieldsProducer;
-import codecs.PostingsBaseFormat;
-import codecs.PostingsFormat;
-import codecs.PostingsReaderBase;
-import codecs.PostingsWriterBase;
-import codecs.lucene41.Lucene41PostingsWriter;
-import codecs.lucene41.Lucene41PostingsReader;
-import codecs.lucene41.Lucene41PostingsBaseFormat;
-import codecs.lucene41.Lucene41PostingsFormat;
-import codecs.pulsing.PulsingPostingsWriter;
-import codecs.pulsing.PulsingPostingsReader;
-import index.SegmentReadState;
-import index.SegmentWriteState;
-import util.IOUtils;
+	/// <summary>
+	/// FST + Pulsing41, test only, since
+	///  FST does no delta encoding here!
+	///  @lucene.experimental 
+	/// </summary>
 
-/** FST + Pulsing41, test only, since
- *  FST does no delta encoding here!
- *  @lucene.experimental */
+	public class FSTPulsing41PostingsFormat : PostingsFormat
+	{
+	  private readonly PostingsBaseFormat wrappedPostingsBaseFormat;
+	  private readonly int freqCutoff;
 
-public class FSTPulsing41PostingsFormat extends PostingsFormat {
-  private final PostingsBaseFormat wrappedPostingsBaseFormat;
-  private final int freqCutoff;
+	  public FSTPulsing41PostingsFormat() : this(1)
+	  {
+	  }
 
-  public FSTPulsing41PostingsFormat() {
-    this(1);
-  }
-  
-  public FSTPulsing41PostingsFormat(int freqCutoff) {
-    super("FSTPulsing41");
-    this.wrappedPostingsBaseFormat = new Lucene41PostingsBaseFormat();
-    this.freqCutoff = freqCutoff;
-  }
+	  public FSTPulsing41PostingsFormat(int freqCutoff) : base("FSTPulsing41")
+	  {
+		this.wrappedPostingsBaseFormat = new Lucene41PostingsBaseFormat();
+		this.freqCutoff = freqCutoff;
+	  }
 
-  @Override
-  public 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.FieldsConsumer fieldsConsumer(org.apache.lucene.index.SegmentWriteState state) throws java.io.IOException
+	  public override FieldsConsumer fieldsConsumer(SegmentWriteState state)
+	  {
+		PostingsWriterBase docsWriter = null;
+		PostingsWriterBase pulsingWriter = null;
 
-    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);
-      }
-    }
-  }
+		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);
+		  }
+		}
+	  }
 
-  @Override
-  public 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);
-      }
-    }
-  }
-}
+//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);
+		  }
+		}
+	  }
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d852d5b0/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 55a2bd6..104ff61 100644
--- a/src/Lucene.Net.Codecs/Memory/FSTTermOutputs.cs
+++ b/src/Lucene.Net.Codecs/Memory/FSTTermOutputs.cs
@@ -1,331 +1,423 @@
-package codecs.memory;
+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
+{
 
-import java.io.IOException;
-import java.util.Arrays;
+	/*
+	 * 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.
+	 */
 
-import index.FieldInfo;
-import index.FieldInfo.IndexOptions;
-import store.DataInput;
-import store.DataOutput;
-import util.fst.Outputs;
-import util.LongsRef;
 
-/**
- * An FST {@link Outputs} implementation for 
- * {@link FSTTermsWriter}.
- *
- * @lucene.experimental
- */
+	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;
 
-// NOTE: outputs should be per-field, since
-// longsSize is fixed for each field
-class FSTTermOutputs extends Outputs<FSTTermOutputs.TermData> {
-  private final static TermData NO_OUTPUT = new TermData();
-  //private static bool TEST = false;
-  private final bool hasPos;
-  private final int longsSize;
+	/// <summary>
+	/// An FST <seealso cref="Outputs"/> implementation for 
+	/// <seealso cref="FSTTermsWriter"/>.
+	/// 
+	/// @lucene.experimental
+	/// </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.
-   */
-  static class TermData {
-    long[] longs;
-    byte[] bytes;
-    int docFreq;
-    long totalTermFreq;
-    TermData() {
-      this.longs = null;
-      this.bytes = null;
-      this.docFreq = 0;
-      this.totalTermFreq = -1;
-    }
-    TermData(long[] longs, byte[] bytes, int docFreq, long totalTermFreq) {
-      this.longs = longs;
-      this.bytes = bytes;
-      this.docFreq = docFreq;
-      this.totalTermFreq = totalTermFreq;
-    }
+	// 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;
 
-    // NOTE: actually, FST nodes are seldom 
-    // identical when outputs on their arcs 
-    // aren't NO_OUTPUTs.
-    @Override
-    public int hashCode() {
-      int hash = 0;
-      if (longs != null) {
-        final int end = longs.length;
-        for (int i = 0; i < end; i++) {
-          hash -= longs[i];
-        }
-      }
-      if (bytes != null) {
-        hash = -hash;
-        final int end = bytes.length;
-        for (int i = 0; i < end; i++) {
-          hash += bytes[i];
-        }
-      }
-      hash += docFreq + totalTermFreq;
-      return hash;
-    }
+	  /// <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;
+		}
 
-    @Override
-    public bool equals(Object other_) {
-      if (other_ == this) {
-        return true;
-      } else if (!(other_ instanceof FSTTermOutputs.TermData)) {
-        return false;
-      }
-      TermData other = (TermData) other_;
-      return statsEqual(this, other) && 
-             longsEqual(this, other) && 
-             bytesEqual(this, other);
+		// 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;
+		}
 
-    }
-  }
-  
-  protected FSTTermOutputs(FieldInfo fieldInfo, int longsSize) {
-    this.hasPos = (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY);
-    this.longsSize = longsSize;
-  }
+		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);
 
-  @Override
-  //
-  // 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.
-  //
-  public TermData common(TermData t1, TermData 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;
+	  protected internal FSTTermOutputs(FieldInfo fieldInfo, int longsSize)
+	  {
+		this.hasPos = (fieldInfo.IndexOptions != FieldInfo.IndexOptions.DOCS_ONLY);
+		this.longsSize = longsSize;
+	  }
 
-    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);
-      }
-    } else {  // equal long[]
-      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 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);
 
-  @Override
-  public 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;
+		long[] min = t1.longs, max = t2.longs;
+		int pos = 0;
+		TermData ret;
 
-    int pos = 0;
-    long diff = 0;
-    long[] share = new long[longsSize];
+		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;
+	  }
 
-    while (pos < longsSize) {
-      share[pos] = t1.longs[pos] - t2.longs[pos];
-      diff += share[pos];
-      pos++;
-    }
+	  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);
 
-    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;
-  }
+		int pos = 0;
+		long diff = 0;
+		long[] share = new long[longsSize];
 
-  // 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?
-  @Override
-  public 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;
+		while (pos < longsSize)
+		{
+		  share[pos] = t1.longs[pos] - t2.longs[pos];
+		  diff += share[pos];
+		  pos++;
+		}
 
-    int pos = 0;
-    long[] accum = new long[longsSize];
+		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;
+	  }
 
-    while (pos < longsSize) {
-      accum[pos] = t1.longs[pos] + t2.longs[pos];
-      pos++;
-    }
+	  // 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);
 
-    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;
-  }
+		int pos = 0;
+		long[] accum = new long[longsSize];
 
-  @Override
-  public 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((byte)bits);
-      } else {
-        out.writeByte((byte)bits);
-        out.writeVInt(data.bytes.length);
-      }
-    } else {
-      out.writeByte((byte)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);
-      }
-    }
-  }
+		while (pos < longsSize)
+		{
+		  accum[pos] = t1.longs[pos] + t2.longs[pos];
+		  pos++;
+		}
 
-  @Override
-  public TermData read(DataInput in)  {
-    long[] longs = new long[longsSize];
-    byte[] 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 = (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 byte[bytesSize];
-      in.readBytes(bytes, 0, bytesSize);
-    }
-    if (bit2 > 0) {  // stats exist
-      int code = in.readVInt();
-      if (hasPos) {
-        totalTermFreq = docFreq = code >>> 1;
-        if ((code & 1) == 0) {
-          totalTermFreq += in.readVLong();
-        }
-      } else {
-        docFreq = code;
-      }
-    }
-    return new TermData(longs, bytes, docFreq, totalTermFreq);
-  }
+		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;
+	  }
 
-  @Override
-  public TermData getNoOutput() {
-    return NO_OUTPUT;
-  }
+//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);
+		  }
+		}
+	  }
 
-  @Override
-  public String outputToString(TermData data) {
-    return data.toString();
-  }
+//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);
+	  }
 
-  static bool statsEqual(final TermData t1, final TermData t2) {
-    return t1.docFreq == t2.docFreq && t1.totalTermFreq == t2.totalTermFreq;
-  }
-  static bool bytesEqual(final TermData t1, final TermData t2) {
-    if (t1.bytes == null && t2.bytes == null) {
-      return true;
-    }
-    return t1.bytes != null && t2.bytes != null && Arrays.equals(t1.bytes, t2.bytes);
-  }
-  static bool longsEqual(final TermData t1, final TermData t2) {
-    if (t1.longs == null && t2.longs == null) {
-      return true;
-    }
-    return t1.longs != null && t2.longs != null && Arrays.equals(t1.longs, t2.longs);
-  }
-  static bool allZero(final long[] l) {
-    for (int i = 0; i < l.length; i++) {
-      if (l[i] != 0) {
-        return false;
-      }
-    }
-    return true;
-  }
-}
+	  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;
+	  }
+	}
+
+}
\ No newline at end of file