You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by sy...@apache.org on 2014/09/06 21:36:15 UTC

[04/51] [abbrv] [partial] Cleaning up and getting ready to development towards v4.8

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/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
new file mode 100644
index 0000000..367b83d
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Memory/FSTTermsWriter.cs
@@ -0,0 +1,277 @@
+package 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.
+ */
+
+import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Comparator;
+
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.Util;
+import org.apache.lucene.codecs.BlockTermState;
+import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.codecs.TermStats;
+import org.apache.lucene.codecs.CodecUtil;
+
+/**
+ * FST-based term dict, using metadata as FST output.
+ *
+ * The FST directly holds the mapping between <term, metadata>.
+ *
+ * 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.
+ *
+ * <p>
+ * File:
+ * <ul>
+ *   <li><tt>.tst</tt>: <a href="#Termdictionary">Term Dictionary</a></li>
+ * </ul>
+ * <p>
+ *
+ * <a name="Termdictionary" id="Termdictionary"></a>
+ * <h3>Term Dictionary</h3>
+ * <p>
+ *  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).
+ * </p>
+ * <p>
+ *  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>
+ * </p>
+ *
+ * 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; {@link 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; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *  <li>DirOffset --&gt; {@link DataOutput#writeLong Uint64}</li>
+ *  <li>DocFreq, LongsSize, BytesSize, NumFields,
+ *        FieldNumber, DocCount --&gt; {@link DataOutput#writeVInt VInt}</li>
+ *  <li>TotalTermFreq, NumTerms, SumTotalTermFreq, SumDocFreq, LongDelta --&gt; 
+ *        {@link DataOutput#writeVLong VLong}</li>
+ * </ul>
+ * <p>Notes:</p>
+ * <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
+ */
+
+public class FSTTermsWriter extends FieldsConsumer {
+  static final String TERMS_EXTENSION = "tmp";
+  static final String TERMS_CODEC_NAME = "FST_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;
+  
+  final PostingsWriterBase postingsWriter;
+  final FieldInfos fieldInfos;
+  IndexOutput out;
+  final List<FieldMetaData> fields = new ArrayList<>();
+
+  public FSTTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter)  {
+    final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
+
+    this.postingsWriter = postingsWriter;
+    this.fieldInfos = state.fieldInfos;
+    this.out = state.directory.createOutput(termsFileName, state.context);
+
+    bool success = false;
+    try {
+      writeHeader(out);
+      this.postingsWriter.init(out); 
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(out);
+      }
+    }
+  }
+  private void writeHeader(IndexOutput out)  {
+    CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);   
+  }
+  private void writeTrailer(IndexOutput out, long dirStart)  {
+    out.writeLong(dirStart);
+  }
+
+  @Override
+  public TermsConsumer addField(FieldInfo field)  {
+    return new TermsWriter(field);
+  }
+
+  @Override
+  public void close()  {
+    if (out != null) {
+      IOException ioe = null;
+      try {
+        // write field summary
+        final long dirStart = out.getFilePointer();
+        
+        out.writeVInt(fields.size());
+        for (FieldMetaData field : fields) {
+          out.writeVInt(field.fieldInfo.number);
+          out.writeVLong(field.numTerms);
+          if (field.fieldInfo.getIndexOptions() != 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;
+      }
+    }
+  }
+
+  private static class FieldMetaData {
+    public final FieldInfo fieldInfo;
+    public final long numTerms;
+    public final long sumTotalTermFreq;
+    public final long sumDocFreq;
+    public final int docCount;
+    public final int longsSize;
+    public final FST<FSTTermOutputs.TermData> dict;
+
+    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;
+    }
+  }
+
+  final class TermsWriter extends TermsConsumer {
+    private final Builder<FSTTermOutputs.TermData> builder;
+    private final FSTTermOutputs outputs;
+    private final FieldInfo fieldInfo;
+    private final int longsSize;
+    private long numTerms;
+
+    private final IntsRef scratchTerm = new IntsRef();
+    private final RAMOutputStream statsWriter = new RAMOutputStream();
+    private final RAMOutputStream metaWriter = new RAMOutputStream();
+
+    TermsWriter(FieldInfo fieldInfo) {
+      this.numTerms = 0;
+      this.fieldInfo = fieldInfo;
+      this.longsSize = postingsWriter.setField(fieldInfo);
+      this.outputs = new FSTTermOutputs(fieldInfo, longsSize);
+      this.builder = new Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
+    }
+
+    @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)  {
+      // write term meta data into fst
+      final BlockTermState state = postingsWriter.newTermState();
+      final 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;
+      postingsWriter.finishTerm(state);
+      postingsWriter.encodeTerm(meta.longs, metaWriter, fieldInfo, state, true);
+      final int bytesSize = (int)metaWriter.getFilePointer();
+      if (bytesSize > 0) {
+        meta.bytes = new byte[bytesSize];
+        metaWriter.writeTo(meta.bytes, 0);
+        metaWriter.reset();
+      }
+      builder.add(Util.toIntsRef(text, scratchTerm), meta);
+      numTerms++;
+    }
+
+    @Override
+    public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount)  {
+      // save FST dict
+      if (numTerms > 0) {
+        final FST<FSTTermOutputs.TermData> fst = builder.finish();
+        fields.add(new FieldMetaData(fieldInfo, numTerms, sumTotalTermFreq, sumDocFreq, docCount, longsSize, fst));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/src/Lucene.Net.Codecs/Memory/MemoryDocValuesConsumer.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Memory/MemoryDocValuesConsumer.cs b/src/Lucene.Net.Codecs/Memory/MemoryDocValuesConsumer.cs
new file mode 100644
index 0000000..e8026df
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Memory/MemoryDocValuesConsumer.cs
@@ -0,0 +1,408 @@
+package 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.
+ */
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.MathUtil;
+import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.fst.FST.INPUT_TYPE;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+import org.apache.lucene.util.fst.Util;
+import org.apache.lucene.util.packed.BlockPackedWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.VERSION_CURRENT;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.BLOCK_SIZE;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.BYTES;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.NUMBER;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.FST;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.DELTA_COMPRESSED;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.GCD_COMPRESSED;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.TABLE_COMPRESSED;
+import static org.apache.lucene.codecs.memory.MemoryDocValuesProducer.UNCOMPRESSED;
+
+/**
+ * Writer for {@link MemoryDocValuesFormat}
+ */
+class MemoryDocValuesConsumer extends DocValuesConsumer {
+  IndexOutput data, meta;
+  final int maxDoc;
+  final float acceptableOverheadRatio;
+  
+  MemoryDocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension, float acceptableOverheadRatio)  {
+    this.acceptableOverheadRatio = acceptableOverheadRatio;
+    maxDoc = state.segmentInfo.getDocCount();
+    bool success = false;
+    try {
+      String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+      data = state.directory.createOutput(dataName, state.context);
+      CodecUtil.writeHeader(data, dataCodec, VERSION_CURRENT);
+      String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+      meta = state.directory.createOutput(metaName, state.context);
+      CodecUtil.writeHeader(meta, metaCodec, VERSION_CURRENT);
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  @Override
+  public void addNumericField(FieldInfo field, Iterable<Number> values)  {
+    addNumericField(field, values, true);
+  }
+
+  void addNumericField(FieldInfo field, Iterable<Number> values, bool optimizeStorage)  {
+    meta.writeVInt(field.number);
+    meta.writeByte(NUMBER);
+    meta.writeLong(data.getFilePointer());
+    long minValue = Long.MAX_VALUE;
+    long maxValue = Long.MIN_VALUE;
+    long gcd = 0;
+    bool missing = false;
+    // TODO: more efficient?
+    HashSet<Long> uniqueValues = null;
+    if (optimizeStorage) {
+      uniqueValues = new HashSet<>();
+
+      long count = 0;
+      for (Number nv : values) {
+        final long v;
+        if (nv == null) {
+          v = 0;
+          missing = true;
+        } else {
+          v = nv.longValue();
+        }
+
+        if (gcd != 1) {
+          if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
+            // in that case v - minValue might overflow and make the GCD computation return
+            // wrong results. Since these extreme values are unlikely, we just discard
+            // GCD computation for them
+            gcd = 1;
+          } else if (count != 0) { // minValue needs to be set first
+            gcd = MathUtil.gcd(gcd, v - minValue);
+          }
+        }
+
+        minValue = Math.min(minValue, v);
+        maxValue = Math.max(maxValue, v);
+
+        if (uniqueValues != null) {
+          if (uniqueValues.add(v)) {
+            if (uniqueValues.size() > 256) {
+              uniqueValues = null;
+            }
+          }
+        }
+
+        ++count;
+      }
+      Debug.Assert( count == maxDoc;
+    }
+    
+    if (missing) {
+      long start = data.getFilePointer();
+      writeMissingBitset(values);
+      meta.writeLong(start);
+      meta.writeLong(data.getFilePointer() - start);
+    } else {
+      meta.writeLong(-1L);
+    }
+
+    if (uniqueValues != null) {
+      // small number of unique values
+      final int bitsPerValue = PackedInts.bitsRequired(uniqueValues.size()-1);
+      FormatAndBits formatAndBits = PackedInts.fastestFormatAndBits(maxDoc, bitsPerValue, acceptableOverheadRatio);
+      if (formatAndBits.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
+        meta.writeByte(UNCOMPRESSED); // uncompressed
+        for (Number nv : values) {
+          data.writeByte(nv == null ? 0 : (byte) nv.longValue());
+        }
+      } else {
+        meta.writeByte(TABLE_COMPRESSED); // table-compressed
+        Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
+        final HashMap<Long,Integer> encode = new HashMap<>();
+        data.writeVInt(decode.length);
+        for (int i = 0; i < decode.length; i++) {
+          data.writeLong(decode[i]);
+          encode.put(decode[i], i);
+        }
+
+        meta.writeVInt(PackedInts.VERSION_CURRENT);
+        data.writeVInt(formatAndBits.format.getId());
+        data.writeVInt(formatAndBits.bitsPerValue);
+
+        final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, formatAndBits.format, maxDoc, formatAndBits.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
+        for(Number nv : values) {
+          writer.add(encode.get(nv == null ? 0 : nv.longValue()));
+        }
+        writer.finish();
+      }
+    } else if (gcd != 0 && gcd != 1) {
+      meta.writeByte(GCD_COMPRESSED);
+      meta.writeVInt(PackedInts.VERSION_CURRENT);
+      data.writeLong(minValue);
+      data.writeLong(gcd);
+      data.writeVInt(BLOCK_SIZE);
+
+      final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
+      for (Number nv : values) {
+        long value = nv == null ? 0 : nv.longValue();
+        writer.add((value - minValue) / gcd);
+      }
+      writer.finish();
+    } else {
+      meta.writeByte(DELTA_COMPRESSED); // delta-compressed
+
+      meta.writeVInt(PackedInts.VERSION_CURRENT);
+      data.writeVInt(BLOCK_SIZE);
+
+      final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
+      for (Number nv : values) {
+        writer.add(nv == null ? 0 : nv.longValue());
+      }
+      writer.finish();
+    }
+  }
+  
+  @Override
+  public void close()  {
+    bool success = false;
+    try {
+      if (meta != null) {
+        meta.writeVInt(-1); // write EOF marker
+        CodecUtil.writeFooter(meta); // write checksum
+      }
+      if (data != null) {
+        CodecUtil.writeFooter(data);
+      }
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(data, meta);
+      } else {
+        IOUtils.closeWhileHandlingException(data, meta);
+      }
+      data = meta = null;
+    }
+  }
+
+  @Override
+  public void addBinaryField(FieldInfo field, final Iterable<BytesRef> values)  {
+    // write the byte[] data
+    meta.writeVInt(field.number);
+    meta.writeByte(BYTES);
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    final long startFP = data.getFilePointer();
+    bool missing = false;
+    for(BytesRef v : values) {
+      final int length;
+      if (v == null) {
+        length = 0;
+        missing = true;
+      } else {
+        length = v.length;
+      }
+      if (length > MemoryDocValuesFormat.MAX_BINARY_FIELD_LENGTH) {
+        throw new IllegalArgumentException("DocValuesField \"" + field.name + "\" is too large, must be <= " + MemoryDocValuesFormat.MAX_BINARY_FIELD_LENGTH);
+      }
+      minLength = Math.min(minLength, length);
+      maxLength = Math.max(maxLength, length);
+      if (v != null) {
+        data.writeBytes(v.bytes, v.offset, v.length);
+      }
+    }
+    meta.writeLong(startFP);
+    meta.writeLong(data.getFilePointer() - startFP);
+    if (missing) {
+      long start = data.getFilePointer();
+      writeMissingBitset(values);
+      meta.writeLong(start);
+      meta.writeLong(data.getFilePointer() - start);
+    } else {
+      meta.writeLong(-1L);
+    }
+    meta.writeVInt(minLength);
+    meta.writeVInt(maxLength);
+    
+    // if minLength == maxLength, its a fixed-length byte[], we are done (the addresses are implicit)
+    // otherwise, we need to record the length fields...
+    if (minLength != maxLength) {
+      meta.writeVInt(PackedInts.VERSION_CURRENT);
+      meta.writeVInt(BLOCK_SIZE);
+
+      final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
+      long addr = 0;
+      for (BytesRef v : values) {
+        if (v != null) {
+          addr += v.length;
+        }
+        writer.add(addr);
+      }
+      writer.finish();
+    }
+  }
+  
+  private void writeFST(FieldInfo field, Iterable<BytesRef> values)  {
+    meta.writeVInt(field.number);
+    meta.writeByte(FST);
+    meta.writeLong(data.getFilePointer());
+    PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
+    Builder<Long> builder = new Builder<>(INPUT_TYPE.BYTE1, outputs);
+    IntsRef scratch = new IntsRef();
+    long ord = 0;
+    for (BytesRef v : values) {
+      builder.add(Util.toIntsRef(v, scratch), ord);
+      ord++;
+    }
+    FST<Long> fst = builder.finish();
+    if (fst != null) {
+      fst.save(data);
+    }
+    meta.writeVLong(ord);
+  }
+  
+  // TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
+  // but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
+  void writeMissingBitset(Iterable<?> values)  {
+    long bits = 0;
+    int count = 0;
+    for (Object v : values) {
+      if (count == 64) {
+        data.writeLong(bits);
+        count = 0;
+        bits = 0;
+      }
+      if (v != null) {
+        bits |= 1L << (count & 0x3f);
+      }
+      count++;
+    }
+    if (count > 0) {
+      data.writeLong(bits);
+    }
+  }
+
+  @Override
+  public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd)  {
+    // write the ordinals as numerics
+    addNumericField(field, docToOrd, false);
+    
+    // write the values as FST
+    writeFST(field, values);
+  }
+
+  // note: this might not be the most efficient... but its fairly simple
+  @Override
+  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords)  {
+    // write the ordinals as a binary field
+    addBinaryField(field, new Iterable<BytesRef>() {
+      @Override
+      public Iterator<BytesRef> iterator() {
+        return new SortedSetIterator(docToOrdCount.iterator(), ords.iterator());
+      }
+    });
+      
+    // write the values as FST
+    writeFST(field, values);
+  }
+  
+  // per-document vint-encoded byte[]
+  static class SortedSetIterator implements Iterator<BytesRef> {
+    byte[] buffer = new byte[10];
+    ByteArrayDataOutput out = new ByteArrayDataOutput();
+    BytesRef ref = new BytesRef();
+    
+    final Iterator<Number> counts;
+    final Iterator<Number> ords;
+    
+    SortedSetIterator(Iterator<Number> counts, Iterator<Number> ords) {
+      this.counts = counts;
+      this.ords = ords;
+    }
+    
+    @Override
+    public bool hasNext() {
+      return counts.hasNext();
+    }
+
+    @Override
+    public BytesRef next() {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+      
+      int count = counts.next().intValue();
+      int maxSize = count*9; // worst case
+      if (maxSize > buffer.length) {
+        buffer = ArrayUtil.grow(buffer, maxSize);
+      }
+      
+      try {
+        encodeValues(count);
+      } catch (IOException bogus) {
+        throw new RuntimeException(bogus);
+      }
+      
+      ref.bytes = buffer;
+      ref.offset = 0;
+      ref.length = out.getPosition();
+
+      return ref;
+    }
+    
+    // encodes count values to buffer
+    private void encodeValues(int count)  {
+      out.reset(buffer);
+      long lastOrd = 0;
+      for (int i = 0; i < count; i++) {
+        long ord = ords.next().longValue();
+        out.writeVLong(ord - lastOrd);
+        lastOrd = ord;
+      }
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/src/Lucene.Net.Codecs/Memory/MemoryDocValuesFormat.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Memory/MemoryDocValuesFormat.cs b/src/Lucene.Net.Codecs/Memory/MemoryDocValuesFormat.cs
new file mode 100644
index 0000000..efc1aba
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Memory/MemoryDocValuesFormat.cs
@@ -0,0 +1,72 @@
+package 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.packed.PackedInts;
+
+/** In-memory docvalues format */
+public class MemoryDocValuesFormat extends DocValuesFormat {
+
+  /** Maximum length for each binary doc values field. */
+  public static final int MAX_BINARY_FIELD_LENGTH = (1 << 15) - 2;
+  
+  final float acceptableOverheadRatio;
+  
+  /** 
+   * Calls {@link #MemoryDocValuesFormat(float) 
+   * MemoryDocValuesFormat(PackedInts.DEFAULT)} 
+   */
+  public MemoryDocValuesFormat() {
+    this(PackedInts.DEFAULT);
+  }
+  
+  /**
+   * Creates a new MemoryDocValuesFormat with the specified
+   * <code>acceptableOverheadRatio</code> for NumericDocValues.
+   * @param acceptableOverheadRatio compression parameter for numerics. 
+   *        Currently this is only used when the number of unique values is small.
+   *        
+   * @lucene.experimental
+   */
+  public MemoryDocValuesFormat(float acceptableOverheadRatio) {
+    super("Memory");
+    this.acceptableOverheadRatio = acceptableOverheadRatio;
+  }
+
+  @Override
+  public DocValuesConsumer fieldsConsumer(SegmentWriteState state)  {
+    return new MemoryDocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
+  }
+  
+  @Override
+  public DocValuesProducer fieldsProducer(SegmentReadState state)  {
+    return new MemoryDocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
+  }
+  
+  static final String DATA_CODEC = "MemoryDocValuesData";
+  static final String DATA_EXTENSION = "mdvd";
+  static final String METADATA_CODEC = "MemoryDocValuesMetadata";
+  static final String METADATA_EXTENSION = "mdvm";
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/src/Lucene.Net.Codecs/Memory/MemoryDocValuesProducer.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Memory/MemoryDocValuesProducer.cs b/src/Lucene.Net.Codecs/Memory/MemoryDocValuesProducer.cs
new file mode 100644
index 0000000..ce004b4
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Memory/MemoryDocValuesProducer.cs
@@ -0,0 +1,658 @@
+package 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.
+ */
+
+import java.io.IOException;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.fst.BytesRefFSTEnum;
+import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.FST.Arc;
+import org.apache.lucene.util.fst.FST.BytesReader;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+import org.apache.lucene.util.fst.Util;
+import org.apache.lucene.util.packed.BlockPackedReader;
+import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * Reader for {@link MemoryDocValuesFormat}
+ */
+class MemoryDocValuesProducer extends DocValuesProducer {
+  // metadata maps (just file pointers and minimal stuff)
+  private final Map<Integer,NumericEntry> numerics;
+  private final Map<Integer,BinaryEntry> binaries;
+  private final Map<Integer,FSTEntry> fsts;
+  private final IndexInput data;
+  
+  // ram instances we have already loaded
+  private final Map<Integer,NumericDocValues> numericInstances = 
+      new HashMap<>();
+  private final Map<Integer,BinaryDocValues> binaryInstances =
+      new HashMap<>();
+  private final Map<Integer,FST<Long>> fstInstances =
+      new HashMap<>();
+  private final Map<Integer,Bits> docsWithFieldInstances = new HashMap<>();
+  
+  private final int maxDoc;
+  private final AtomicLong ramBytesUsed;
+  private final int version;
+  
+  static final byte NUMBER = 0;
+  static final byte BYTES = 1;
+  static final byte FST = 2;
+
+  static final int BLOCK_SIZE = 4096;
+  
+  static final byte DELTA_COMPRESSED = 0;
+  static final byte TABLE_COMPRESSED = 1;
+  static final byte UNCOMPRESSED = 2;
+  static final byte GCD_COMPRESSED = 3;
+  
+  static final int VERSION_START = 0;
+  static final int VERSION_GCD_COMPRESSION = 1;
+  static final int VERSION_CHECKSUM = 2;
+  static final int VERSION_CURRENT = VERSION_CHECKSUM;
+    
+  MemoryDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension)  {
+    maxDoc = state.segmentInfo.getDocCount();
+    String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+    // read in the entries from the metadata file.
+    ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
+    bool success = false;
+    try {
+      version = CodecUtil.checkHeader(in, metaCodec, 
+                                      VERSION_START,
+                                      VERSION_CURRENT);
+      numerics = new HashMap<>();
+      binaries = new HashMap<>();
+      fsts = new HashMap<>();
+      readFields(in, state.fieldInfos);
+      if (version >= VERSION_CHECKSUM) {
+        CodecUtil.checkFooter(in);
+      } else {
+        CodecUtil.checkEOF(in);
+      }
+      ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(in);
+      } else {
+        IOUtils.closeWhileHandlingException(in);
+      }
+    }
+
+    success = false;
+    try {
+      String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
+      data = state.directory.openInput(dataName, state.context);
+      final int version2 = CodecUtil.checkHeader(data, dataCodec, 
+                                                 VERSION_START,
+                                                 VERSION_CURRENT);
+      if (version != version2) {
+        throw new CorruptIndexException("Format versions mismatch");
+      }
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this.data);
+      }
+    }
+  }
+  
+  private void readFields(IndexInput meta, FieldInfos infos)  {
+    int fieldNumber = meta.readVInt();
+    while (fieldNumber != -1) {
+      int fieldType = meta.readByte();
+      if (fieldType == NUMBER) {
+        NumericEntry entry = new NumericEntry();
+        entry.offset = meta.readLong();
+        entry.missingOffset = meta.readLong();
+        if (entry.missingOffset != -1) {
+          entry.missingBytes = meta.readLong();
+        } else {
+          entry.missingBytes = 0;
+        }
+        entry.format = meta.readByte();
+        switch(entry.format) {
+          case DELTA_COMPRESSED:
+          case TABLE_COMPRESSED:
+          case GCD_COMPRESSED:
+          case UNCOMPRESSED:
+               break;
+          default:
+               throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
+        }
+        if (entry.format != UNCOMPRESSED) {
+          entry.packedIntsVersion = meta.readVInt();
+        }
+        numerics.put(fieldNumber, entry);
+      } else if (fieldType == BYTES) {
+        BinaryEntry entry = new BinaryEntry();
+        entry.offset = meta.readLong();
+        entry.numBytes = meta.readLong();
+        entry.missingOffset = meta.readLong();
+        if (entry.missingOffset != -1) {
+          entry.missingBytes = meta.readLong();
+        } else {
+          entry.missingBytes = 0;
+        }
+        entry.minLength = meta.readVInt();
+        entry.maxLength = meta.readVInt();
+        if (entry.minLength != entry.maxLength) {
+          entry.packedIntsVersion = meta.readVInt();
+          entry.blockSize = meta.readVInt();
+        }
+        binaries.put(fieldNumber, entry);
+      } else if (fieldType == FST) {
+        FSTEntry entry = new FSTEntry();
+        entry.offset = meta.readLong();
+        entry.numOrds = meta.readVLong();
+        fsts.put(fieldNumber, entry);
+      } else {
+        throw new CorruptIndexException("invalid entry type: " + fieldType + ", input=" + meta);
+      }
+      fieldNumber = meta.readVInt();
+    }
+  }
+
+  @Override
+  public synchronized NumericDocValues getNumeric(FieldInfo field)  {
+    NumericDocValues instance = numericInstances.get(field.number);
+    if (instance == null) {
+      instance = loadNumeric(field);
+      numericInstances.put(field.number, instance);
+    }
+    return instance;
+  }
+  
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed.get();
+  }
+  
+  @Override
+  public void checkIntegrity()  {
+    if (version >= VERSION_CHECKSUM) {
+      CodecUtil.checksumEntireFile(data);
+    }
+  }
+  
+  private NumericDocValues loadNumeric(FieldInfo field)  {
+    NumericEntry entry = numerics.get(field.number);
+    data.seek(entry.offset + entry.missingBytes);
+    switch (entry.format) {
+      case TABLE_COMPRESSED:
+        int size = data.readVInt();
+        if (size > 256) {
+          throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, input=" + data);
+        }
+        final long decode[] = new long[size];
+        for (int i = 0; i < decode.length; i++) {
+          decode[i] = data.readLong();
+        }
+        final int formatID = data.readVInt();
+        final int bitsPerValue = data.readVInt();
+        final PackedInts.Reader ordsReader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, maxDoc, bitsPerValue);
+        ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(decode) + ordsReader.ramBytesUsed());
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return decode[(int)ordsReader.get(docID)];
+          }
+        };
+      case DELTA_COMPRESSED:
+        final int blockSize = data.readVInt();
+        final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
+        ramBytesUsed.addAndGet(reader.ramBytesUsed());
+        return reader;
+      case UNCOMPRESSED:
+        final byte bytes[] = new byte[maxDoc];
+        data.readBytes(bytes, 0, bytes.length);
+        ramBytesUsed.addAndGet(RamUsageEstimator.sizeOf(bytes));
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return bytes[docID];
+          }
+        };
+      case GCD_COMPRESSED:
+        final long min = data.readLong();
+        final long mult = data.readLong();
+        final int quotientBlockSize = data.readVInt();
+        final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, quotientBlockSize, maxDoc, false);
+        ramBytesUsed.addAndGet(quotientReader.ramBytesUsed());
+        return new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return min + mult * quotientReader.get(docID);
+          }
+        };
+      default:
+        throw new Debug.Assert(ionError();
+    }
+  }
+
+  @Override
+  public synchronized BinaryDocValues getBinary(FieldInfo field)  {
+    BinaryDocValues instance = binaryInstances.get(field.number);
+    if (instance == null) {
+      instance = loadBinary(field);
+      binaryInstances.put(field.number, instance);
+    }
+    return instance;
+  }
+  
+  private BinaryDocValues loadBinary(FieldInfo field)  {
+    BinaryEntry entry = binaries.get(field.number);
+    data.seek(entry.offset);
+    PagedBytes bytes = new PagedBytes(16);
+    bytes.copy(data, entry.numBytes);
+    final PagedBytes.Reader bytesReader = bytes.freeze(true);
+    if (entry.minLength == entry.maxLength) {
+      final int fixedLength = entry.minLength;
+      ramBytesUsed.addAndGet(bytes.ramBytesUsed());
+      return new BinaryDocValues() {
+        @Override
+        public void get(int docID, BytesRef result) {
+          bytesReader.fillSlice(result, fixedLength * (long)docID, fixedLength);
+        }
+      };
+    } else {
+      data.seek(data.getFilePointer() + entry.missingBytes);
+      final MonotonicBlockPackedReader addresses = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
+      ramBytesUsed.addAndGet(bytes.ramBytesUsed() + addresses.ramBytesUsed());
+      return new BinaryDocValues() {
+        @Override
+        public void get(int docID, BytesRef result) {
+          long startAddress = docID == 0 ? 0 : addresses.get(docID-1);
+          long endAddress = addresses.get(docID); 
+          bytesReader.fillSlice(result, startAddress, (int) (endAddress - startAddress));
+        }
+      };
+    }
+  }
+  
+  @Override
+  public SortedDocValues getSorted(FieldInfo field)  {
+    final FSTEntry entry = fsts.get(field.number);
+    if (entry.numOrds == 0) {
+      return DocValues.EMPTY_SORTED;
+    }
+    FST<Long> instance;
+    synchronized(this) {
+      instance = fstInstances.get(field.number);
+      if (instance == null) {
+        data.seek(entry.offset);
+        instance = new FST<>(data, PositiveIntOutputs.getSingleton());
+        ramBytesUsed.addAndGet(instance.sizeInBytes());
+        fstInstances.put(field.number, instance);
+      }
+    }
+    final NumericDocValues docToOrd = getNumeric(field);
+    final FST<Long> fst = instance;
+    
+    // per-thread resources
+    final BytesReader in = fst.getBytesReader();
+    final Arc<Long> firstArc = new Arc<>();
+    final Arc<Long> scratchArc = new Arc<>();
+    final IntsRef scratchInts = new IntsRef();
+    final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<>(fst);
+    
+    return new SortedDocValues() {
+      @Override
+      public int getOrd(int docID) {
+        return (int) docToOrd.get(docID);
+      }
+
+      @Override
+      public void lookupOrd(int ord, BytesRef result) {
+        try {
+          in.setPosition(0);
+          fst.getFirstArc(firstArc);
+          IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
+          result.bytes = new byte[output.length];
+          result.offset = 0;
+          result.length = 0;
+          Util.toBytesRef(output, result);
+        } catch (IOException bogus) {
+          throw new RuntimeException(bogus);
+        }
+      }
+
+      @Override
+      public int lookupTerm(BytesRef key) {
+        try {
+          InputOutput<Long> o = fstEnum.seekCeil(key);
+          if (o == null) {
+            return -getValueCount()-1;
+          } else if (o.input.equals(key)) {
+            return o.output.intValue();
+          } else {
+            return (int) -o.output-1;
+          }
+        } catch (IOException bogus) {
+          throw new RuntimeException(bogus);
+        }
+      }
+
+      @Override
+      public int getValueCount() {
+        return (int)entry.numOrds;
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        return new FSTTermsEnum(fst);
+      }
+    };
+  }
+  
+  @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field)  {
+    final FSTEntry entry = fsts.get(field.number);
+    if (entry.numOrds == 0) {
+      return DocValues.EMPTY_SORTED_SET; // empty FST!
+    }
+    FST<Long> instance;
+    synchronized(this) {
+      instance = fstInstances.get(field.number);
+      if (instance == null) {
+        data.seek(entry.offset);
+        instance = new FST<>(data, PositiveIntOutputs.getSingleton());
+        ramBytesUsed.addAndGet(instance.sizeInBytes());
+        fstInstances.put(field.number, instance);
+      }
+    }
+    final BinaryDocValues docToOrds = getBinary(field);
+    final FST<Long> fst = instance;
+    
+    // per-thread resources
+    final BytesReader in = fst.getBytesReader();
+    final Arc<Long> firstArc = new Arc<>();
+    final Arc<Long> scratchArc = new Arc<>();
+    final IntsRef scratchInts = new IntsRef();
+    final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<>(fst);
+    final BytesRef ref = new BytesRef();
+    final ByteArrayDataInput input = new ByteArrayDataInput();
+    return new SortedSetDocValues() {
+      long currentOrd;
+
+      @Override
+      public long nextOrd() {
+        if (input.eof()) {
+          return NO_MORE_ORDS;
+        } else {
+          currentOrd += input.readVLong();
+          return currentOrd;
+        }
+      }
+      
+      @Override
+      public void setDocument(int docID) {
+        docToOrds.get(docID, ref);
+        input.reset(ref.bytes, ref.offset, ref.length);
+        currentOrd = 0;
+      }
+
+      @Override
+      public void lookupOrd(long ord, BytesRef result) {
+        try {
+          in.setPosition(0);
+          fst.getFirstArc(firstArc);
+          IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
+          result.bytes = new byte[output.length];
+          result.offset = 0;
+          result.length = 0;
+          Util.toBytesRef(output, result);
+        } catch (IOException bogus) {
+          throw new RuntimeException(bogus);
+        }
+      }
+
+      @Override
+      public long lookupTerm(BytesRef key) {
+        try {
+          InputOutput<Long> o = fstEnum.seekCeil(key);
+          if (o == null) {
+            return -getValueCount()-1;
+          } else if (o.input.equals(key)) {
+            return o.output.intValue();
+          } else {
+            return -o.output-1;
+          }
+        } catch (IOException bogus) {
+          throw new RuntimeException(bogus);
+        }
+      }
+
+      @Override
+      public long getValueCount() {
+        return entry.numOrds;
+      }
+
+      @Override
+      public TermsEnum termsEnum() {
+        return new FSTTermsEnum(fst);
+      }
+    };
+  }
+  
+  private Bits getMissingBits(int fieldNumber, final long offset, final long length)  {
+    if (offset == -1) {
+      return new Bits.MatchAllBits(maxDoc);
+    } else {
+      Bits instance;
+      synchronized(this) {
+        instance = docsWithFieldInstances.get(fieldNumber);
+        if (instance == null) {
+          IndexInput data = this.data.clone();
+          data.seek(offset);
+          Debug.Assert( length % 8 == 0;
+          long bits[] = new long[(int) length >> 3];
+          for (int i = 0; i < bits.length; i++) {
+            bits[i] = data.readLong();
+          }
+          instance = new FixedBitSet(bits, maxDoc);
+          docsWithFieldInstances.put(fieldNumber, instance);
+        }
+      }
+      return instance;
+    }
+  }
+  
+  @Override
+  public Bits getDocsWithField(FieldInfo field)  {
+    switch(field.getDocValuesType()) {
+      case SORTED_SET:
+        return DocValues.docsWithValue(getSortedSet(field), maxDoc);
+      case SORTED:
+        return DocValues.docsWithValue(getSorted(field), maxDoc);
+      case BINARY:
+        BinaryEntry be = binaries.get(field.number);
+        return getMissingBits(field.number, be.missingOffset, be.missingBytes);
+      case NUMERIC:
+        NumericEntry ne = numerics.get(field.number);
+        return getMissingBits(field.number, ne.missingOffset, ne.missingBytes);
+      default: 
+        throw new Debug.Assert(ionError();
+    }
+  }
+
+  @Override
+  public void close()  {
+    data.close();
+  }
+  
+  static class NumericEntry {
+    long offset;
+    long missingOffset;
+    long missingBytes;
+    byte format;
+    int packedIntsVersion;
+  }
+  
+  static class BinaryEntry {
+    long offset;
+    long missingOffset;
+    long missingBytes;
+    long numBytes;
+    int minLength;
+    int maxLength;
+    int packedIntsVersion;
+    int blockSize;
+  }
+  
+  static class FSTEntry {
+    long offset;
+    long numOrds;
+  }
+  
+  // exposes FSTEnum directly as a TermsEnum: avoids binary-search next()
+  static class FSTTermsEnum extends TermsEnum {
+    final BytesRefFSTEnum<Long> in;
+    
+    // this is all for the complicated seek(ord)...
+    // maybe we should add a FSTEnum that supports this operation?
+    final FST<Long> fst;
+    final FST.BytesReader bytesReader;
+    final Arc<Long> firstArc = new Arc<>();
+    final Arc<Long> scratchArc = new Arc<>();
+    final IntsRef scratchInts = new IntsRef();
+    final BytesRef scratchBytes = new BytesRef();
+    
+    FSTTermsEnum(FST<Long> fst) {
+      this.fst = fst;
+      in = new BytesRefFSTEnum<>(fst);
+      bytesReader = fst.getBytesReader();
+    }
+
+    @Override
+    public BytesRef next()  {
+      InputOutput<Long> io = in.next();
+      if (io == null) {
+        return null;
+      } else {
+        return io.input;
+      }
+    }
+
+    @Override
+    public Comparator<BytesRef> getComparator() {
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+
+    @Override
+    public SeekStatus seekCeil(BytesRef text)  {
+      if (in.seekCeil(text) == null) {
+        return SeekStatus.END;
+      } else if (term().equals(text)) {
+        // TODO: add SeekStatus to FSTEnum like in https://issues.apache.org/jira/browse/LUCENE-3729
+        // to remove this comparision?
+        return SeekStatus.FOUND;
+      } else {
+        return SeekStatus.NOT_FOUND;
+      }
+    }
+
+    @Override
+    public bool seekExact(BytesRef text)  {
+      if (in.seekExact(text) == null) {
+        return false;
+      } else {
+        return true;
+      }
+    }
+
+    @Override
+    public void seekExact(long ord)  {
+      // TODO: would be better to make this simpler and faster.
+      // but we dont want to introduce a bug that corrupts our enum state!
+      bytesReader.setPosition(0);
+      fst.getFirstArc(firstArc);
+      IntsRef output = Util.getByOutput(fst, ord, bytesReader, firstArc, scratchArc, scratchInts);
+      scratchBytes.bytes = new byte[output.length];
+      scratchBytes.offset = 0;
+      scratchBytes.length = 0;
+      Util.toBytesRef(output, scratchBytes);
+      // TODO: we could do this lazily, better to try to push into FSTEnum though?
+      in.seekExact(scratchBytes);
+    }
+
+    @Override
+    public BytesRef term()  {
+      return in.current().input;
+    }
+
+    @Override
+    public long ord()  {
+      return in.current().output;
+    }
+
+    @Override
+    public int docFreq()  {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long totalTermFreq()  {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags)  {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags)  {
+      throw new UnsupportedOperationException();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/src/Lucene.Net.Codecs/Memory/MemoryPostingsFormat.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Memory/MemoryPostingsFormat.cs b/src/Lucene.Net.Codecs/Memory/MemoryPostingsFormat.cs
new file mode 100644
index 0000000..ae5abca
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Memory/MemoryPostingsFormat.cs
@@ -0,0 +1,936 @@
+package 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.
+ */
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.TermStats;
+import org.apache.lucene.codecs.TermsConsumer;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.fst.ByteSequenceOutputs;
+import org.apache.lucene.util.fst.BytesRefFSTEnum;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.Util;
+import org.apache.lucene.util.packed.PackedInts;
+
+// TODO: would be nice to somehow allow this to act like
+// InstantiatedIndex, by never writing to disk; ie you write
+// to this Codec in RAM only and then when you open a reader
+// it pulls the FST directly from what you wrote w/o going
+// to disk.
+
+/** Stores terms & postings (docs, positions, payloads) in
+ *  RAM, using an FST.
+ *
+ * <p>Note that this codec implements advance as a linear
+ * scan!  This means if you store large fields in here,
+ * queries that rely on advance will (AND boolQuery,
+ * PhraseQuery) will be relatively slow!
+ *
+ * @lucene.experimental */
+
+// TODO: Maybe name this 'Cached' or something to reflect
+// the reality that it is actually written to disk, but
+// loads itself in ram?
+public final class MemoryPostingsFormat extends PostingsFormat {
+
+  private final bool doPackFST;
+  private final float acceptableOverheadRatio;
+
+  public MemoryPostingsFormat() {
+    this(false, PackedInts.DEFAULT);
+  }
+
+  /**
+   * Create MemoryPostingsFormat, specifying advanced FST options.
+   * @param doPackFST true if a packed FST should be built.
+   *        NOTE: packed FSTs are limited to ~2.1 GB of postings.
+   * @param acceptableOverheadRatio allowable overhead for packed ints
+   *        during FST construction.
+   */
+  public MemoryPostingsFormat(bool doPackFST, float acceptableOverheadRatio) {
+    super("Memory");
+    this.doPackFST = doPackFST;
+    this.acceptableOverheadRatio = acceptableOverheadRatio;
+  }
+  
+  @Override
+  public String toString() {
+    return "PostingsFormat(name=" + getName() + " doPackFST= " + doPackFST + ")";
+  }
+
+  private final static class TermsWriter extends TermsConsumer {
+    private final IndexOutput out;
+    private final FieldInfo field;
+    private final Builder<BytesRef> builder;
+    private final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
+    private final bool doPackFST;
+    private final float acceptableOverheadRatio;
+    private int termCount;
+
+    public TermsWriter(IndexOutput out, FieldInfo field, bool doPackFST, float acceptableOverheadRatio) {
+      this.out = out;
+      this.field = field;
+      this.doPackFST = doPackFST;
+      this.acceptableOverheadRatio = acceptableOverheadRatio;
+      builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doPackFST, acceptableOverheadRatio, true, 15);
+    }
+
+    private class PostingsWriter extends PostingsConsumer {
+      private int lastDocID;
+      private int lastPos;
+      private int lastPayloadLen;
+
+      // NOTE: not private so we don't pay access check at runtime:
+      int docCount;
+      RAMOutputStream buffer = new RAMOutputStream();
+      
+      int lastOffsetLength;
+      int lastOffset;
+
+      @Override
+      public void startDoc(int docID, int termDocFreq)  {
+        //System.out.println("    startDoc docID=" + docID + " freq=" + termDocFreq);
+        final int delta = docID - lastDocID;
+        Debug.Assert( docID == 0 || delta > 0;
+        lastDocID = docID;
+        docCount++;
+
+        if (field.getIndexOptions() == IndexOptions.DOCS_ONLY) {
+          buffer.writeVInt(delta);
+        } else if (termDocFreq == 1) {
+          buffer.writeVInt((delta<<1) | 1);
+        } else {
+          buffer.writeVInt(delta<<1);
+          Debug.Assert( termDocFreq > 0;
+          buffer.writeVInt(termDocFreq);
+        }
+
+        lastPos = 0;
+        lastOffset = 0;
+      }
+
+      @Override
+      public void addPosition(int pos, BytesRef payload, int startOffset, int endOffset)  {
+        Debug.Assert( payload == null || field.hasPayloads();
+
+        //System.out.println("      addPos pos=" + pos + " payload=" + payload);
+
+        final int delta = pos - lastPos;
+        Debug.Assert( delta >= 0;
+        lastPos = pos;
+        
+        int payloadLen = 0;
+        
+        if (field.hasPayloads()) {
+          payloadLen = payload == null ? 0 : payload.length;
+          if (payloadLen != lastPayloadLen) {
+            lastPayloadLen = payloadLen;
+            buffer.writeVInt((delta<<1)|1);
+            buffer.writeVInt(payloadLen);
+          } else {
+            buffer.writeVInt(delta<<1);
+          }
+        } else {
+          buffer.writeVInt(delta);
+        }
+        
+        if (field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) {
+          // don't use startOffset - lastEndOffset, because this creates lots of negative vints for synonyms,
+          // and the numbers aren't that much smaller anyways.
+          int offsetDelta = startOffset - lastOffset;
+          int offsetLength = endOffset - startOffset;
+          if (offsetLength != lastOffsetLength) {
+            buffer.writeVInt(offsetDelta << 1 | 1);
+            buffer.writeVInt(offsetLength);
+          } else {
+            buffer.writeVInt(offsetDelta << 1);
+          }
+          lastOffset = startOffset;
+          lastOffsetLength = offsetLength;
+        }
+        
+        if (payloadLen > 0) {
+          buffer.writeBytes(payload.bytes, payload.offset, payloadLen);
+        }
+      }
+
+      @Override
+      public void finishDoc() {
+      }
+
+      public PostingsWriter reset() {
+        Debug.Assert( buffer.getFilePointer() == 0;
+        lastDocID = 0;
+        docCount = 0;
+        lastPayloadLen = 0;
+        // force first offset to write its length
+        lastOffsetLength = -1;
+        return this;
+      }
+    }
+
+    private final PostingsWriter postingsWriter = new PostingsWriter();
+
+    @Override
+    public PostingsConsumer startTerm(BytesRef text) {
+      //System.out.println("  startTerm term=" + text.utf8ToString());
+      return postingsWriter.reset();
+    }
+
+    private final RAMOutputStream buffer2 = new RAMOutputStream();
+    private final BytesRef spare = new BytesRef();
+    private byte[] finalBuffer = new byte[128];
+
+    private final IntsRef scratchIntsRef = new IntsRef();
+
+    @Override
+    public void finishTerm(BytesRef text, TermStats stats)  {
+
+      Debug.Assert( postingsWriter.docCount == stats.docFreq;
+
+      Debug.Assert( buffer2.getFilePointer() == 0;
+
+      buffer2.writeVInt(stats.docFreq);
+      if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+        buffer2.writeVLong(stats.totalTermFreq-stats.docFreq);
+      }
+      int pos = (int) buffer2.getFilePointer();
+      buffer2.writeTo(finalBuffer, 0);
+      buffer2.reset();
+
+      final int totalBytes = pos + (int) postingsWriter.buffer.getFilePointer();
+      if (totalBytes > finalBuffer.length) {
+        finalBuffer = ArrayUtil.grow(finalBuffer, totalBytes);
+      }
+      postingsWriter.buffer.writeTo(finalBuffer, pos);
+      postingsWriter.buffer.reset();
+
+      spare.bytes = finalBuffer;
+      spare.length = totalBytes;
+
+      //System.out.println("    finishTerm term=" + text.utf8ToString() + " " + totalBytes + " bytes totalTF=" + stats.totalTermFreq);
+      //for(int i=0;i<totalBytes;i++) {
+      //  System.out.println("      " + Integer.toHexString(finalBuffer[i]&0xFF));
+      //}
+
+      builder.add(Util.toIntsRef(text, scratchIntsRef), BytesRef.deepCopyOf(spare));
+      termCount++;
+    }
+
+    @Override
+    public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount)  {
+      if (termCount > 0) {
+        out.writeVInt(termCount);
+        out.writeVInt(field.number);
+        if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+          out.writeVLong(sumTotalTermFreq);
+        }
+        out.writeVLong(sumDocFreq);
+        out.writeVInt(docCount);
+        FST<BytesRef> fst = builder.finish();
+        fst.save(out);
+        //System.out.println("finish field=" + field.name + " fp=" + out.getFilePointer());
+      }
+    }
+
+    @Override
+    public Comparator<BytesRef> getComparator() {
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+  }
+
+  private static String EXTENSION = "ram";
+  private static final String CODEC_NAME = "MemoryPostings";
+  private static final int VERSION_START = 0;
+  private static final int VERSION_CURRENT = VERSION_START;
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state)  {
+
+    final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
+    final IndexOutput out = state.directory.createOutput(fileName, state.context);
+    bool success = false;
+    try {
+      CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(out);
+      }
+    }
+    
+    return new FieldsConsumer() {
+      @Override
+      public TermsConsumer addField(FieldInfo field) {
+        //System.out.println("\naddField field=" + field.name);
+        return new TermsWriter(out, field, doPackFST, acceptableOverheadRatio);
+      }
+
+      @Override
+      public void close()  {
+        // EOF marker:
+        try {
+          out.writeVInt(0);
+          CodecUtil.writeFooter(out);
+        } finally {
+          out.close();
+        }
+      }
+    };
+  }
+
+  private final static class FSTDocsEnum extends DocsEnum {
+    private final IndexOptions indexOptions;
+    private final bool storePayloads;
+    private byte[] buffer = new byte[16];
+    private final ByteArrayDataInput in = new ByteArrayDataInput(buffer);
+
+    private Bits liveDocs;
+    private int docUpto;
+    private int docID = -1;
+    private int accum;
+    private int freq;
+    private int payloadLen;
+    private int numDocs;
+
+    public FSTDocsEnum(IndexOptions indexOptions, bool storePayloads) {
+      this.indexOptions = indexOptions;
+      this.storePayloads = storePayloads;
+    }
+
+    public bool canReuse(IndexOptions indexOptions, bool storePayloads) {
+      return indexOptions == this.indexOptions && storePayloads == this.storePayloads;
+    }
+    
+    public FSTDocsEnum reset(BytesRef bufferIn, Bits liveDocs, int numDocs) {
+      Debug.Assert( numDocs > 0;
+      if (buffer.length < bufferIn.length) {
+        buffer = ArrayUtil.grow(buffer, bufferIn.length);
+      }
+      in.reset(buffer, 0, bufferIn.length);
+      System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length);
+      this.liveDocs = liveDocs;
+      docID = -1;
+      accum = 0;
+      docUpto = 0;
+      freq = 1;
+      payloadLen = 0;
+      this.numDocs = numDocs;
+      return this;
+    }
+
+    @Override
+    public int nextDoc() {
+      while(true) {
+        //System.out.println("  nextDoc cycle docUpto=" + docUpto + " numDocs=" + numDocs + " fp=" + in.getPosition() + " this=" + this);
+        if (docUpto == numDocs) {
+          // System.out.println("    END");
+          return docID = NO_MORE_DOCS;
+        }
+        docUpto++;
+        if (indexOptions == IndexOptions.DOCS_ONLY) {
+          accum += in.readVInt();
+        } else {
+          final int code = in.readVInt();
+          accum += code >>> 1;
+          //System.out.println("  docID=" + accum + " code=" + code);
+          if ((code & 1) != 0) {
+            freq = 1;
+          } else {
+            freq = in.readVInt();
+            Debug.Assert( freq > 0;
+          }
+
+          if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+            // Skip positions/payloads
+            for(int posUpto=0;posUpto<freq;posUpto++) {
+              if (!storePayloads) {
+                in.readVInt();
+              } else {
+                final int posCode = in.readVInt();
+                if ((posCode & 1) != 0) {
+                  payloadLen = in.readVInt();
+                }
+                in.skipBytes(payloadLen);
+              }
+            }
+          } else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
+            // Skip positions/offsets/payloads
+            for(int posUpto=0;posUpto<freq;posUpto++) {
+              int posCode = in.readVInt();
+              if (storePayloads && ((posCode & 1) != 0)) {
+                payloadLen = in.readVInt();
+              }
+              if ((in.readVInt() & 1) != 0) {
+                // new offset length
+                in.readVInt();
+              }
+              if (storePayloads) {
+                in.skipBytes(payloadLen);
+              }
+            }
+          }
+        }
+
+        if (liveDocs == null || liveDocs.get(accum)) {
+          //System.out.println("    return docID=" + accum + " freq=" + freq);
+          return (docID = accum);
+        }
+      }
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int advance(int target)  {
+      // TODO: we could make more efficient version, but, it
+      // should be rare that this will matter in practice
+      // since usually apps will not store "big" fields in
+      // this codec!
+      return slowAdvance(target);
+    }
+
+    @Override
+    public int freq() {
+      return freq;
+    }
+    
+    @Override
+    public long cost() {
+      return numDocs;
+    }
+  }
+
+  private final static class FSTDocsAndPositionsEnum extends DocsAndPositionsEnum {
+    private final bool storePayloads;
+    private byte[] buffer = new byte[16];
+    private final ByteArrayDataInput in = new ByteArrayDataInput(buffer);
+
+    private Bits liveDocs;
+    private int docUpto;
+    private int docID = -1;
+    private int accum;
+    private int freq;
+    private int numDocs;
+    private int posPending;
+    private int payloadLength;
+    final bool storeOffsets;
+    int offsetLength;
+    int startOffset;
+
+    private int pos;
+    private final BytesRef payload = new BytesRef();
+
+    public FSTDocsAndPositionsEnum(bool storePayloads, bool storeOffsets) {
+      this.storePayloads = storePayloads;
+      this.storeOffsets = storeOffsets;
+    }
+
+    public bool canReuse(bool storePayloads, bool storeOffsets) {
+      return storePayloads == this.storePayloads && storeOffsets == this.storeOffsets;
+    }
+    
+    public FSTDocsAndPositionsEnum reset(BytesRef bufferIn, Bits liveDocs, int numDocs) {
+      Debug.Assert( numDocs > 0;
+
+      // System.out.println("D&P reset bytes this=" + this);
+      // for(int i=bufferIn.offset;i<bufferIn.length;i++) {
+      //   System.out.println("  " + Integer.toHexString(bufferIn.bytes[i]&0xFF));
+      // }
+
+      if (buffer.length < bufferIn.length) {
+        buffer = ArrayUtil.grow(buffer, bufferIn.length);
+      }
+      in.reset(buffer, 0, bufferIn.length - bufferIn.offset);
+      System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length);
+      this.liveDocs = liveDocs;
+      docID = -1;
+      accum = 0;
+      docUpto = 0;
+      payload.bytes = buffer;
+      payloadLength = 0;
+      this.numDocs = numDocs;
+      posPending = 0;
+      startOffset = storeOffsets ? 0 : -1; // always return -1 if no offsets are stored
+      offsetLength = 0;
+      return this;
+    }
+
+    @Override
+    public int nextDoc() {
+      while (posPending > 0) {
+        nextPosition();
+      }
+      while(true) {
+        //System.out.println("  nextDoc cycle docUpto=" + docUpto + " numDocs=" + numDocs + " fp=" + in.getPosition() + " this=" + this);
+        if (docUpto == numDocs) {
+          //System.out.println("    END");
+          return docID = NO_MORE_DOCS;
+        }
+        docUpto++;
+        
+        final int code = in.readVInt();
+        accum += code >>> 1;
+        if ((code & 1) != 0) {
+          freq = 1;
+        } else {
+          freq = in.readVInt();
+          Debug.Assert( freq > 0;
+        }
+
+        if (liveDocs == null || liveDocs.get(accum)) {
+          pos = 0;
+          startOffset = storeOffsets ? 0 : -1;
+          posPending = freq;
+          //System.out.println("    return docID=" + accum + " freq=" + freq);
+          return (docID = accum);
+        }
+
+        // Skip positions
+        for(int posUpto=0;posUpto<freq;posUpto++) {
+          if (!storePayloads) {
+            in.readVInt();
+          } else {
+            final int skipCode = in.readVInt();
+            if ((skipCode & 1) != 0) {
+              payloadLength = in.readVInt();
+              //System.out.println("    new payloadLen=" + payloadLength);
+            }
+          }
+          
+          if (storeOffsets) {
+            if ((in.readVInt() & 1) != 0) {
+              // new offset length
+              offsetLength = in.readVInt();
+            }
+          }
+          
+          if (storePayloads) {
+            in.skipBytes(payloadLength);
+          }
+        }
+      }
+    }
+
+    @Override
+    public int nextPosition() {
+      //System.out.println("    nextPos storePayloads=" + storePayloads + " this=" + this);
+      Debug.Assert( posPending > 0;
+      posPending--;
+      if (!storePayloads) {
+        pos += in.readVInt();
+      } else {
+        final int code = in.readVInt();
+        pos += code >>> 1;
+        if ((code & 1) != 0) {
+          payloadLength = in.readVInt();
+          //System.out.println("      new payloadLen=" + payloadLength);
+          //} else {
+          //System.out.println("      same payloadLen=" + payloadLength);
+        }
+      }
+      
+      if (storeOffsets) {
+        int offsetCode = in.readVInt();
+        if ((offsetCode & 1) != 0) {
+          // new offset length
+          offsetLength = in.readVInt();
+        }
+        startOffset += offsetCode >>> 1;
+      }
+      
+      if (storePayloads) {
+        payload.offset = in.getPosition();
+        in.skipBytes(payloadLength);
+        payload.length = payloadLength;
+      }
+
+      //System.out.println("      pos=" + pos + " payload=" + payload + " fp=" + in.getPosition());
+      return pos;
+    }
+
+    @Override
+    public int startOffset() {
+      return startOffset;
+    }
+
+    @Override
+    public int endOffset() {
+      return startOffset + offsetLength;
+    }
+
+    @Override
+    public BytesRef getPayload() {
+      return payload.length > 0 ? payload : null;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int advance(int target)  {
+      // TODO: we could make more efficient version, but, it
+      // should be rare that this will matter in practice
+      // since usually apps will not store "big" fields in
+      // this codec!
+      return slowAdvance(target);
+    }
+
+    @Override
+    public int freq() {
+      return freq;
+    }
+    
+    @Override
+    public long cost() {
+      return numDocs;
+    }
+  }
+
+  private final static class FSTTermsEnum extends TermsEnum {
+    private final FieldInfo field;
+    private final BytesRefFSTEnum<BytesRef> fstEnum;
+    private final ByteArrayDataInput buffer = new ByteArrayDataInput();
+    private bool didDecode;
+
+    private int docFreq;
+    private long totalTermFreq;
+    private BytesRefFSTEnum.InputOutput<BytesRef> current;
+    private BytesRef postingsSpare = new BytesRef();
+
+    public FSTTermsEnum(FieldInfo field, FST<BytesRef> fst) {
+      this.field = field;
+      fstEnum = new BytesRefFSTEnum<>(fst);
+    }
+
+    private void decodeMetaData() {
+      if (!didDecode) {
+        buffer.reset(current.output.bytes, current.output.offset, current.output.length);
+        docFreq = buffer.readVInt();
+        if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+          totalTermFreq = docFreq + buffer.readVLong();
+        } else {
+          totalTermFreq = -1;
+        }
+        postingsSpare.bytes = current.output.bytes;
+        postingsSpare.offset = buffer.getPosition();
+        postingsSpare.length = current.output.length - (buffer.getPosition() - current.output.offset);
+        //System.out.println("  df=" + docFreq + " totTF=" + totalTermFreq + " offset=" + buffer.getPosition() + " len=" + current.output.length);
+        didDecode = true;
+      }
+    }
+
+    @Override
+    public bool seekExact(BytesRef text)  {
+      //System.out.println("te.seekExact text=" + field.name + ":" + text.utf8ToString() + " this=" + this);
+      current = fstEnum.seekExact(text);
+      didDecode = false;
+      return current != null;
+    }
+
+    @Override
+    public SeekStatus seekCeil(BytesRef text)  {
+      //System.out.println("te.seek text=" + field.name + ":" + text.utf8ToString() + " this=" + this);
+      current = fstEnum.seekCeil(text);
+      if (current == null) {
+        return SeekStatus.END;
+      } else {
+
+        // System.out.println("  got term=" + current.input.utf8ToString());
+        // for(int i=0;i<current.output.length;i++) {
+        //   System.out.println("    " + Integer.toHexString(current.output.bytes[i]&0xFF));
+        // }
+
+        didDecode = false;
+
+        if (text.equals(current.input)) {
+          //System.out.println("  found!");
+          return SeekStatus.FOUND;
+        } else {
+          //System.out.println("  not found: " + current.input.utf8ToString());
+          return SeekStatus.NOT_FOUND;
+        }
+      }
+    }
+    
+    @Override
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) {
+      decodeMetaData();
+      FSTDocsEnum docsEnum;
+
+      if (reuse == null || !(reuse instanceof FSTDocsEnum)) {
+        docsEnum = new FSTDocsEnum(field.getIndexOptions(), field.hasPayloads());
+      } else {
+        docsEnum = (FSTDocsEnum) reuse;        
+        if (!docsEnum.canReuse(field.getIndexOptions(), field.hasPayloads())) {
+          docsEnum = new FSTDocsEnum(field.getIndexOptions(), field.hasPayloads());
+        }
+      }
+      return docsEnum.reset(this.postingsSpare, liveDocs, docFreq);
+    }
+
+    @Override
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
+
+      bool hasOffsets = field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+      if (field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
+        return null;
+      }
+      decodeMetaData();
+      FSTDocsAndPositionsEnum docsAndPositionsEnum;
+      if (reuse == null || !(reuse instanceof FSTDocsAndPositionsEnum)) {
+        docsAndPositionsEnum = new FSTDocsAndPositionsEnum(field.hasPayloads(), hasOffsets);
+      } else {
+        docsAndPositionsEnum = (FSTDocsAndPositionsEnum) reuse;        
+        if (!docsAndPositionsEnum.canReuse(field.hasPayloads(), hasOffsets)) {
+          docsAndPositionsEnum = new FSTDocsAndPositionsEnum(field.hasPayloads(), hasOffsets);
+        }
+      }
+      //System.out.println("D&P reset this=" + this);
+      return docsAndPositionsEnum.reset(postingsSpare, liveDocs, docFreq);
+    }
+
+    @Override
+    public BytesRef term() {
+      return current.input;
+    }
+
+    @Override
+    public BytesRef next()  {
+      //System.out.println("te.next");
+      current = fstEnum.next();
+      if (current == null) {
+        //System.out.println("  END");
+        return null;
+      }
+      didDecode = false;
+      //System.out.println("  term=" + field.name + ":" + current.input.utf8ToString());
+      return current.input;
+    }
+
+    @Override
+    public int docFreq() {
+      decodeMetaData();
+      return docFreq;
+    }
+
+    @Override
+    public long totalTermFreq() {
+      decodeMetaData();
+      return totalTermFreq;
+    }
+
+    @Override
+    public Comparator<BytesRef> getComparator() {
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+
+    @Override
+    public void seekExact(long ord) {
+      // NOTE: we could add this...
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long ord() {
+      // NOTE: we could add this...
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  private final static class TermsReader extends Terms {
+
+    private final long sumTotalTermFreq;
+    private final long sumDocFreq;
+    private final int docCount;
+    private final int termCount;
+    private FST<BytesRef> fst;
+    private final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
+    private final FieldInfo field;
+
+    public TermsReader(FieldInfos fieldInfos, IndexInput in, int termCount)  {
+      this.termCount = termCount;
+      final int fieldNumber = in.readVInt();
+      field = fieldInfos.fieldInfo(fieldNumber);
+      if (field.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+        sumTotalTermFreq = in.readVLong();
+      } else {
+        sumTotalTermFreq = -1;
+      }
+      sumDocFreq = in.readVLong();
+      docCount = in.readVInt();
+      
+      fst = new FST<>(in, outputs);
+    }
+
+    @Override
+    public long getSumTotalTermFreq() {
+      return sumTotalTermFreq;
+    }
+
+    @Override
+    public long getSumDocFreq() {
+      return sumDocFreq;
+    }
+
+    @Override
+    public int getDocCount() {
+      return docCount;
+    }
+
+    @Override
+    public long size() {
+      return termCount;
+    }
+
+    @Override
+    public TermsEnum iterator(TermsEnum reuse) {
+      return new FSTTermsEnum(field, fst);
+    }
+
+    @Override
+    public Comparator<BytesRef> getComparator() {
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
+    }
+
+    @Override
+    public bool hasFreqs() {
+      return field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+    }
+
+    @Override
+    public bool hasOffsets() {
+      return field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+    }
+
+    @Override
+    public bool hasPositions() {
+      return field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+    }
+    
+    @Override
+    public bool hasPayloads() {
+      return field.hasPayloads();
+    }
+
+    public long ramBytesUsed() {
+      return ((fst!=null) ? fst.sizeInBytes() : 0);
+    }
+  }
+
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state)  {
+    final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
+    final ChecksumIndexInput in = state.directory.openChecksumInput(fileName, IOContext.READONCE);
+
+    final SortedMap<String,TermsReader> fields = new TreeMap<>();
+
+    try {
+      CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START, VERSION_CURRENT);
+      while(true) {
+        final int termCount = in.readVInt();
+        if (termCount == 0) {
+          break;
+        }
+        final TermsReader termsReader = new TermsReader(state.fieldInfos, in, termCount);
+        // System.out.println("load field=" + termsReader.field.name);
+        fields.put(termsReader.field.name, termsReader);
+      }
+      CodecUtil.checkFooter(in);
+    } finally {
+      in.close();
+    }
+
+    return new FieldsProducer() {
+      @Override
+      public Iterator<String> iterator() {
+        return Collections.unmodifiableSet(fields.keySet()).iterator();
+      }
+
+      @Override
+      public Terms terms(String field) {
+        return fields.get(field);
+      }
+      
+      @Override
+      public int size() {
+        return fields.size();
+      }
+
+      @Override
+      public void close() {
+        // Drop ref to FST:
+        for(TermsReader termsReader : fields.values()) {
+          termsReader.fst = null;
+        }
+      }
+
+      @Override
+      public long ramBytesUsed() {
+        long sizeInBytes = 0;
+        for(Map.Entry<String,TermsReader> entry: fields.entrySet()) {
+          sizeInBytes += (entry.getKey().length() * RamUsageEstimator.NUM_BYTES_CHAR);
+          sizeInBytes += entry.getValue().ramBytesUsed();
+        }
+        return sizeInBytes;
+      }
+      
+      @Override
+      public void checkIntegrity()  {}
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/src/Lucene.Net.Codecs/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Properties/AssemblyInfo.cs b/src/Lucene.Net.Codecs/Properties/AssemblyInfo.cs
new file mode 100644
index 0000000..152a9c0
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Properties/AssemblyInfo.cs
@@ -0,0 +1,36 @@
+using System.Reflection;
+using System.Runtime.CompilerServices;
+using System.Runtime.InteropServices;
+
+// General Information about an assembly is controlled through the following 
+// set of attributes. Change these attribute values to modify the information
+// associated with an assembly.
+[assembly: AssemblyTitle("Codecs")]
+[assembly: AssemblyDescription("")]
+[assembly: AssemblyConfiguration("")]
+[assembly: AssemblyCompany("")]
+[assembly: AssemblyProduct("Codecs")]
+[assembly: AssemblyCopyright("Copyright ©  2014")]
+[assembly: AssemblyTrademark("")]
+[assembly: AssemblyCulture("")]
+
+// Setting ComVisible to false makes the types in this assembly not visible 
+// to COM components.  If you need to access a type in this assembly from 
+// COM, set the ComVisible attribute to true on that type.
+[assembly: ComVisible(false)]
+
+// The following GUID is for the ID of the typelib if this project is exposed to COM
+[assembly: Guid("03ad88e5-c647-4821-9c75-ca5507ab18f0")]
+
+// Version information for an assembly consists of the following four values:
+//
+//      Major Version
+//      Minor Version 
+//      Build Number
+//      Revision
+//
+// You can specify all the values or you can default the Build and Revision Numbers 
+// by using the '*' as shown below:
+// [assembly: AssemblyVersion("1.0.*")]
+[assembly: AssemblyVersion("1.0.0.0")]
+[assembly: AssemblyFileVersion("1.0.0.0")]

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/src/Lucene.Net.Codecs/Pulsing/Pulsing41PostingsFormat.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Pulsing/Pulsing41PostingsFormat.cs b/src/Lucene.Net.Codecs/Pulsing/Pulsing41PostingsFormat.cs
new file mode 100644
index 0000000..f2634f4
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Pulsing/Pulsing41PostingsFormat.cs
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Lucene.Net.Codecs.Pulsing
+{
+    using Lucene.Net.Codecs.Lucene41;
+
+    /// <summary>
+    /// Concrete pulsing implementation over {@link Lucene41PostingsFormat}.
+    /// 
+    /// @lucene.experimental 
+    /// </summary>
+    public class Pulsing41PostingsFormat : PulsingPostingsFormat
+    {
+
+        /// <summary>Inlines docFreq=1 terms, otherwise uses the normal "Lucene41" format.</summary>
+        public Pulsing41PostingsFormat() : this(1)
+        {
+        }
+
+        /// <summary>Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene41" format.</summary>
+        public Pulsing41PostingsFormat(int freqCutoff) :
+            this(freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE)
+        {
+        }
+
+        /// <summary>Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene41" format.</summary>
+        public Pulsing41PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) :
+            base("Pulsing41", new Lucene41PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize)
+        {
+        }
+    }
+}