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:06 UTC

[02/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/Sep/SepPostingsReader.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Sep/SepPostingsReader.cs b/src/Lucene.Net.Codecs/Sep/SepPostingsReader.cs
index e7b09e4..cd8c2d6 100644
--- a/src/Lucene.Net.Codecs/Sep/SepPostingsReader.cs
+++ b/src/Lucene.Net.Codecs/Sep/SepPostingsReader.cs
@@ -1,6 +1,4 @@
-package codecs.sep;
-
-/*
+/*
  * 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.
@@ -17,698 +15,762 @@ package codecs.sep;
  * limitations under the License.
  */
 
-import java.io.IOException;
-
-import codecs.BlockTermState;
-import codecs.CodecUtil;
-import codecs.PostingsReaderBase;
-import index.DocsAndPositionsEnum;
-import index.DocsEnum;
-import index.FieldInfo.IndexOptions;
-import index.FieldInfo;
-import index.FieldInfos;
-import index.IndexFileNames;
-import index.SegmentInfo;
-import index.TermState;
-import store.ByteArrayDataInput;
-import store.DataInput;
-import store.Directory;
-import store.IOContext;
-import store.IndexInput;
-import util.ArrayUtil;
-import util.Bits;
-import util.BytesRef;
-import util.IOUtils;
-
-/** Concrete class that reads the current doc/freq/skip
- *  postings format.    
- *
- * @lucene.experimental
- */
-
-// TODO: -- should we switch "hasProx" higher up?  and
-// create two separate docs readers, one that also reads
-// prox and one that doesn't?
-
-public class SepPostingsReader extends PostingsReaderBase {
-
-  final IntIndexInput freqIn;
-  final IntIndexInput docIn;
-  final IntIndexInput posIn;
-  final IndexInput payloadIn;
-  final IndexInput skipIn;
-
-  int skipInterval;
-  int maxSkipLevels;
-  int skipMinimum;
-
-  public SepPostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext context, IntStreamFactory intFactory, String segmentSuffix)  {
-    bool success = false;
-    try {
-
-      final String docFileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.DOC_EXTENSION);
-      docIn = intFactory.openInput(dir, docFileName, context);
-
-      skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.SKIP_EXTENSION), context);
-
-      if (fieldInfos.hasFreq()) {
-        freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION), context);        
-      } else {
-        freqIn = null;
-      }
-      if (fieldInfos.hasProx()) {
-        posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.POS_EXTENSION), context);
-        payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.PAYLOAD_EXTENSION), context);
-      } else {
-        posIn = null;
-        payloadIn = null;
-      }
-      success = true;
-    } finally {
-      if (!success) {
-        close();
-      }
-    }
-  }
-
-  @Override
-  public void init(IndexInput termsIn)  {
-    // Make sure we are talking to the matching past writer
-    CodecUtil.checkHeader(termsIn, SepPostingsWriter.CODEC,
-      SepPostingsWriter.VERSION_START, SepPostingsWriter.VERSION_START);
-    skipInterval = termsIn.readInt();
-    maxSkipLevels = termsIn.readInt();
-    skipMinimum = termsIn.readInt();
-  }
-
-  @Override
-  public void close()  {
-    IOUtils.close(freqIn, docIn, skipIn, posIn, payloadIn);
-  }
-
-  private static final class SepTermState extends BlockTermState {
-    // We store only the seek point to the docs file because
-    // the rest of the info (freqIndex, posIndex, etc.) is
-    // stored in the docs file:
-    IntIndexInput.Index docIndex;
-    IntIndexInput.Index posIndex;
-    IntIndexInput.Index freqIndex;
-    long payloadFP;
-    long skipFP;
-
-    @Override
-    public SepTermState clone() {
-      SepTermState other = new SepTermState();
-      other.copyFrom(this);
-      return other;
-    }
-
-    @Override
-    public void copyFrom(TermState _other) {
-      super.copyFrom(_other);
-      SepTermState other = (SepTermState) _other;
-      if (docIndex == null) {
-        docIndex = other.docIndex.clone();
-      } else {
-        docIndex.copyFrom(other.docIndex);
-      }
-      if (other.freqIndex != null) {
-        if (freqIndex == null) {
-          freqIndex = other.freqIndex.clone();
-        } else {
-          freqIndex.copyFrom(other.freqIndex);
+namespace Lucene.Net.Codecs.Sep
+{
+
+    using System.Diagnostics;
+    using Index;
+    using Store;
+    using Util;
+
+    /// <summary>
+    /// Concrete class that reads the current doc/freq/skip
+    /// postings format.    
+    /// 
+    /// @lucene.experimental
+    /// </summary>
+    /// <remarks>
+    /// TODO: -- should we switch "hasProx" higher up?  and
+    /// create two separate docs readers, one that also reads
+    /// prox and one that doesn't?
+    /// </remarks>
+    public class SepPostingsReader : PostingsReaderBase
+    {
+        private readonly IntIndexInput _freqIn;
+        private readonly IntIndexInput _docIn;
+        private readonly IntIndexInput _posIn;
+        private readonly IndexInput _payloadIn;
+        private readonly IndexInput _skipIn;
+
+        private int _skipInterval;
+        private int _maxSkipLevels;
+        private int _skipMinimum;
+
+        public SepPostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext context,
+            IntStreamFactory intFactory, string segmentSuffix)
+        {
+            var success = false;
+            try
+            {
+
+                var docFileName = IndexFileNames.SegmentFileName(segmentInfo.Name, segmentSuffix,
+                    SepPostingsWriter.DOC_EXTENSION);
+                _docIn = intFactory.OpenInput(dir, docFileName, context);
+
+                _skipIn =
+                    dir.OpenInput(
+                        IndexFileNames.SegmentFileName(segmentInfo.Name, segmentSuffix, SepPostingsWriter.SKIP_EXTENSION),
+                        context);
+
+                if (fieldInfos.HasFreq())
+                {
+                    _freqIn = intFactory.OpenInput(dir,
+                        IndexFileNames.SegmentFileName(segmentInfo.Name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION),
+                        context);
+                }
+                else
+                {
+                    _freqIn = null;
+                }
+                if (fieldInfos.HasProx())
+                {
+                    _posIn = intFactory.OpenInput(dir,
+                        IndexFileNames.SegmentFileName(segmentInfo.Name, segmentSuffix, SepPostingsWriter.POS_EXTENSION),
+                        context);
+                    _payloadIn =
+                        dir.OpenInput(
+                            IndexFileNames.SegmentFileName(segmentInfo.Name, segmentSuffix,
+                                SepPostingsWriter.PAYLOAD_EXTENSION), context);
+                }
+                else
+                {
+                    _posIn = null;
+                    _payloadIn = null;
+                }
+                success = true;
+            }
+            finally
+            {
+                if (!success)
+                {
+                    Dispose();
+                }
+            }
         }
-      } else {
-        freqIndex = null;
-      }
-      if (other.posIndex != null) {
-        if (posIndex == null) {
-          posIndex = other.posIndex.clone();
-        } else {
-          posIndex.copyFrom(other.posIndex);
-        }
-      } else {
-        posIndex = null;
-      }
-      payloadFP = other.payloadFP;
-      skipFP = other.skipFP;
-    }
-
-    @Override
-    public String toString() {
-      return super.toString() + " docIndex=" + docIndex + " freqIndex=" + freqIndex + " posIndex=" + posIndex + " payloadFP=" + payloadFP + " skipFP=" + skipFP;
-    }
-  }
-
-  @Override
-  public BlockTermState newTermState()  {
-    final SepTermState state = new SepTermState();
-    state.docIndex = docIn.index();
-    if (freqIn != null) {
-      state.freqIndex = freqIn.index();
-    }
-    if (posIn != null) {
-      state.posIndex = posIn.index();
-    }
-    return state;
-  }
-
-  @Override
-  public void decodeTerm(long[] empty, DataInput in, FieldInfo fieldInfo, BlockTermState _termState, bool absolute) 
-     {
-    final SepTermState termState = (SepTermState) _termState;
-    termState.docIndex.read(in, absolute);
-    if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
-      termState.freqIndex.read(in, absolute);
-      if (fieldInfo.getIndexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-        //System.out.println("  freqIndex=" + termState.freqIndex);
-        termState.posIndex.read(in, absolute);
-        //System.out.println("  posIndex=" + termState.posIndex);
-        if (fieldInfo.hasPayloads()) {
-          if (absolute) {
-            termState.payloadFP = in.readVLong();
-          } else {
-            termState.payloadFP += in.readVLong();
-          }
-          //System.out.println("  payloadFP=" + termState.payloadFP);
-        }
-      }
-    }
-
-    if (termState.docFreq >= skipMinimum) {
-      //System.out.println("   readSkip @ " + in.getPosition());
-      if (absolute) {
-        termState.skipFP = in.readVLong();
-      } else {
-        termState.skipFP += in.readVLong();
-      }
-      //System.out.println("  skipFP=" + termState.skipFP);
-    } else if (absolute) {
-      termState.skipFP = 0;
-    }
-  }
-
-  @Override
-  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs, DocsEnum reuse, int flags)  {
-    final SepTermState termState = (SepTermState) _termState;
-    SepDocsEnum docsEnum;
-    if (reuse == null || !(reuse instanceof SepDocsEnum)) {
-      docsEnum = new SepDocsEnum();
-    } else {
-      docsEnum = (SepDocsEnum) reuse;
-      if (docsEnum.startDocIn != docIn) {
-        // If you are using ParellelReader, and pass in a
-        // reused DocsAndPositionsEnum, it could have come
-        // from another reader also using sep codec
-        docsEnum = new SepDocsEnum();        
-      }
-    }
-
-    return docsEnum.init(fieldInfo, termState, liveDocs);
-  }
-
-  @Override
-  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs,
-                                               DocsAndPositionsEnum reuse, int flags)
-     {
-
-    Debug.Assert( fieldInfo.getIndexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
-    final SepTermState termState = (SepTermState) _termState;
-    SepDocsAndPositionsEnum postingsEnum;
-    if (reuse == null || !(reuse instanceof SepDocsAndPositionsEnum)) {
-      postingsEnum = new SepDocsAndPositionsEnum();
-    } else {
-      postingsEnum = (SepDocsAndPositionsEnum) reuse;
-      if (postingsEnum.startDocIn != docIn) {
-        // If you are using ParellelReader, and pass in a
-        // reused DocsAndPositionsEnum, it could have come
-        // from another reader also using sep codec
-        postingsEnum = new SepDocsAndPositionsEnum();        
-      }
-    }
-
-    return postingsEnum.init(fieldInfo, termState, liveDocs);
-  }
-
-  class SepDocsEnum extends DocsEnum {
-    int docFreq;
-    int doc = -1;
-    int accum;
-    int count;
-    int freq;
-    long freqStart;
-
-    // TODO: -- should we do omitTF with 2 different enum classes?
-    private bool omitTF;
-    private IndexOptions indexOptions;
-    private bool storePayloads;
-    private Bits liveDocs;
-    private final IntIndexInput.Reader docReader;
-    private final IntIndexInput.Reader freqReader;
-    private long skipFP;
-
-    private final IntIndexInput.Index docIndex;
-    private final IntIndexInput.Index freqIndex;
-    private final IntIndexInput.Index posIndex;
-    private final IntIndexInput startDocIn;
-
-    // TODO: -- should we do hasProx with 2 different enum classes?
-
-    bool skipped;
-    SepSkipListReader skipper;
-
-    SepDocsEnum()  {
-      startDocIn = docIn;
-      docReader = docIn.reader();
-      docIndex = docIn.index();
-      if (freqIn != null) {
-        freqReader = freqIn.reader();
-        freqIndex = freqIn.index();
-      } else {
-        freqReader = null;
-        freqIndex = null;
-      }
-      if (posIn != null) {
-        posIndex = posIn.index();                 // only init this so skipper can read it
-      } else {
-        posIndex = null;
-      }
-    }
 
-    SepDocsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs)  {
-      this.liveDocs = liveDocs;
-      this.indexOptions = fieldInfo.getIndexOptions();
-      omitTF = indexOptions == IndexOptions.DOCS_ONLY;
-      storePayloads = fieldInfo.hasPayloads();
-
-      // TODO: can't we only do this if consumer
-      // skipped consuming the previous docs?
-      docIndex.copyFrom(termState.docIndex);
-      docIndex.seek(docReader);
-
-      if (!omitTF) {
-        freqIndex.copyFrom(termState.freqIndex);
-        freqIndex.seek(freqReader);
-      }
-
-      docFreq = termState.docFreq;
-      // NOTE: unused if docFreq < skipMinimum:
-      skipFP = termState.skipFP;
-      count = 0;
-      doc = -1;
-      accum = 0;
-      freq = 1;
-      skipped = false;
-
-      return this;
-    }
-
-    @Override
-    public int nextDoc()  {
-
-      while(true) {
-        if (count == docFreq) {
-          return doc = NO_MORE_DOCS;
+        public override void Init(IndexInput termsIn)
+        {
+            // Make sure we are talking to the matching past writer
+            CodecUtil.CheckHeader(termsIn, SepPostingsWriter.CODEC, SepPostingsWriter.VERSION_START,
+                SepPostingsWriter.VERSION_START);
+            _skipInterval = termsIn.ReadInt();
+            _maxSkipLevels = termsIn.ReadInt();
+            _skipMinimum = termsIn.ReadInt();
         }
 
-        count++;
+        protected override void Dispose(bool disposing)
+        {
+            if (disposing) return;
 
-        // Decode next doc
-        //System.out.println("decode docDelta:");
-        accum += docReader.next();
-          
-        if (!omitTF) {
-          //System.out.println("decode freq:");
-          freq = freqReader.next();
+            IOUtils.Close(_freqIn, _docIn, _skipIn, _posIn, _payloadIn);
         }
 
-        if (liveDocs == null || liveDocs.get(accum)) {
-          break;
-        }
-      }
-      return (doc = accum);
-    }
-
-    @Override
-    public int freq()  {
-      return freq;
-    }
-
-    @Override
-    public int docID() {
-      return doc;
-    }
-
-    @Override
-    public int advance(int target)  {
+        public override BlockTermState NewTermState()
+        {
+            var state = new SepTermState {DOC_INDEX = _docIn.Index()};
 
-      if ((target - skipInterval) >= doc && docFreq >= skipMinimum) {
+            if (_freqIn != null)
+                state.FREQ_INDEX = _freqIn.Index();
 
-        // There are enough docs in the posting to have
-        // skip data, and its not too close
-
-        if (skipper == null) {
-          // This DocsEnum has never done any skipping
-          skipper = new SepSkipListReader(skipIn.clone(),
-                                          freqIn,
-                                          docIn,
-                                          posIn,
-                                          maxSkipLevels, skipInterval);
+            if (_posIn != null)
+                state.POS_INDEX = _posIn.Index();
 
+            return state;
         }
 
-        if (!skipped) {
-          // We haven't yet skipped for this posting
-          skipper.init(skipFP,
-                       docIndex,
-                       freqIndex,
-                       posIndex,
-                       0,
-                       docFreq,
-                       storePayloads);
-          skipper.setIndexOptions(indexOptions);
-
-          skipped = true;
+        public override void DecodeTerm(long[] empty, DataInput input, FieldInfo fieldInfo, BlockTermState bTermState,
+            bool absolute)
+        {
+            var termState = (SepTermState) bTermState;
+            termState.DOC_INDEX.Read(input, absolute);
+            if (fieldInfo.FieldIndexOptions != FieldInfo.IndexOptions.DOCS_ONLY)
+            {
+                termState.FREQ_INDEX.Read(input, absolute);
+                if (fieldInfo.FieldIndexOptions == FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS)
+                {
+                    termState.POS_INDEX.Read(input, absolute);
+                    
+                    if (fieldInfo.HasPayloads())
+                    {
+                        if (absolute)
+                        {
+                            termState.PAYLOAD_FP = input.ReadVLong();
+                        }
+                        else
+                        {
+                            termState.PAYLOAD_FP += input.ReadVLong();
+                        }
+                    }
+                }
+            }
+
+            if (termState.DocFreq >= _skipMinimum)
+            {
+                if (absolute)
+                {
+                    termState.SKIP_FP = input.ReadVLong();
+                }
+                else
+                {
+                    termState.SKIP_FP += input.ReadVLong();
+                }
+            }
+            else if (absolute)
+            {
+                termState.SKIP_FP = 0;
+            }
         }
 
-        final int newCount = skipper.skipTo(target); 
-
-        if (newCount > count) {
-
-          // Skipper did move
-          if (!omitTF) {
-            skipper.getFreqIndex().seek(freqReader);
-          }
-          skipper.getDocIndex().seek(docReader);
-          count = newCount;
-          doc = accum = skipper.getDoc();
-        }
-      }
-        
-      // Now, linear scan for the rest:
-      do {
-        if (nextDoc() == NO_MORE_DOCS) {
-          return NO_MORE_DOCS;
+        public override DocsEnum Docs(FieldInfo fieldInfo, BlockTermState bTermState, Bits liveDocs, DocsEnum reuse,
+            int flags)
+        {
+            var termState = (SepTermState)bTermState;
+
+            SepDocsEnum docsEnum;
+            if (!(reuse is SepDocsEnum))
+            {
+                docsEnum = new SepDocsEnum(this);
+            }
+            else
+            {
+                docsEnum = (SepDocsEnum) reuse;
+                if (docsEnum.START_DOC_IN != _docIn)
+                {
+                    // If you are using ParellelReader, and pass in a
+                    // reused DocsAndPositionsEnum, it could have come
+                    // from another reader also using sep codec
+                    docsEnum = new SepDocsEnum(this);
+                }
+            }
+
+            return docsEnum.Init(fieldInfo, termState, liveDocs);
         }
-      } while (target > doc);
-
-      return doc;
-    }
-    
-    @Override
-    public long cost() {
-      return docFreq;
-    }
-  }
-
-  class SepDocsAndPositionsEnum extends DocsAndPositionsEnum {
-    int docFreq;
-    int doc = -1;
-    int accum;
-    int count;
-    int freq;
-    long freqStart;
-
-    private bool storePayloads;
-    private Bits liveDocs;
-    private final IntIndexInput.Reader docReader;
-    private final IntIndexInput.Reader freqReader;
-    private final IntIndexInput.Reader posReader;
-    private final IndexInput payloadIn;
-    private long skipFP;
-
-    private final IntIndexInput.Index docIndex;
-    private final IntIndexInput.Index freqIndex;
-    private final IntIndexInput.Index posIndex;
-    private final IntIndexInput startDocIn;
-
-    private long payloadFP;
-
-    private int pendingPosCount;
-    private int position;
-    private int payloadLength;
-    private long pendingPayloadBytes;
-
-    private bool skipped;
-    private SepSkipListReader skipper;
-    private bool payloadPending;
-    private bool posSeekPending;
-
-    SepDocsAndPositionsEnum()  {
-      startDocIn = docIn;
-      docReader = docIn.reader();
-      docIndex = docIn.index();
-      freqReader = freqIn.reader();
-      freqIndex = freqIn.index();
-      posReader = posIn.reader();
-      posIndex = posIn.index();
-      payloadIn = SepPostingsReader.this.payloadIn.clone();
-    }
-
-    SepDocsAndPositionsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs)  {
-      this.liveDocs = liveDocs;
-      storePayloads = fieldInfo.hasPayloads();
-      //System.out.println("Sep D&P init");
-
-      // TODO: can't we only do this if consumer
-      // skipped consuming the previous docs?
-      docIndex.copyFrom(termState.docIndex);
-      docIndex.seek(docReader);
-      //System.out.println("  docIndex=" + docIndex);
-
-      freqIndex.copyFrom(termState.freqIndex);
-      freqIndex.seek(freqReader);
-      //System.out.println("  freqIndex=" + freqIndex);
-
-      posIndex.copyFrom(termState.posIndex);
-      //System.out.println("  posIndex=" + posIndex);
-      posSeekPending = true;
-      payloadPending = false;
-
-      payloadFP = termState.payloadFP;
-      skipFP = termState.skipFP;
-      //System.out.println("  skipFP=" + skipFP);
-
-      docFreq = termState.docFreq;
-      count = 0;
-      doc = -1;
-      accum = 0;
-      pendingPosCount = 0;
-      pendingPayloadBytes = 0;
-      skipped = false;
-
-      return this;
-    }
-
-    @Override
-    public int nextDoc()  {
 
-      while(true) {
-        if (count == docFreq) {
-          return doc = NO_MORE_DOCS;
+        public override DocsAndPositionsEnum DocsAndPositions(FieldInfo fieldInfo, BlockTermState bTermState,
+            Bits liveDocs, DocsAndPositionsEnum reuse, int flags)
+        {
+
+            Debug.Assert(fieldInfo.FieldIndexOptions == FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+            var termState = (SepTermState)bTermState;
+            SepDocsAndPositionsEnum postingsEnum;
+            if (!(reuse is SepDocsAndPositionsEnum))
+            {
+                postingsEnum = new SepDocsAndPositionsEnum(this);
+            }
+            else
+            {
+                postingsEnum = (SepDocsAndPositionsEnum) reuse;
+                if (postingsEnum.START_DOC_IN != _docIn)
+                {
+                    // If you are using ParellelReader, and pass in a
+                    // reused DocsAndPositionsEnum, it could have come
+                    // from another reader also using sep codec
+                    postingsEnum = new SepDocsAndPositionsEnum(this);
+                }
+            }
+
+            return postingsEnum.Init(fieldInfo, termState, liveDocs);
         }
 
-        count++;
-
-        // TODO: maybe we should do the 1-bit trick for encoding
-        // freq=1 case?
-
-        // Decode next doc
-        //System.out.println("  sep d&p read doc");
-        accum += docReader.next();
-
-        //System.out.println("  sep d&p read freq");
-        freq = freqReader.next();
-
-        pendingPosCount += freq;
-
-        if (liveDocs == null || liveDocs.get(accum)) {
-          break;
+        public override long RamBytesUsed()
+        {
+            return 0;
         }
-      }
-
-      position = 0;
-      return (doc = accum);
-    }
-
-    @Override
-    public int freq()  {
-      return freq;
-    }
-
-    @Override
-    public int docID() {
-      return doc;
-    }
-
-    @Override
-    public int advance(int target)  {
-      //System.out.println("SepD&P advance target=" + target + " vs current=" + doc + " this=" + this);
-
-      if ((target - skipInterval) >= doc && docFreq >= skipMinimum) {
 
-        // There are enough docs in the posting to have
-        // skip data, and its not too close
-
-        if (skipper == null) {
-          //System.out.println("  create skipper");
-          // This DocsEnum has never done any skipping
-          skipper = new SepSkipListReader(skipIn.clone(),
-                                          freqIn,
-                                          docIn,
-                                          posIn,
-                                          maxSkipLevels, skipInterval);
+        public override void CheckIntegrity()
+        {
+            // TODO: remove sep layout, its fallen behind on features...
         }
 
-        if (!skipped) {
-          //System.out.println("  init skip data skipFP=" + skipFP);
-          // We haven't yet skipped for this posting
-          skipper.init(skipFP,
-                       docIndex,
-                       freqIndex,
-                       posIndex,
-                       payloadFP,
-                       docFreq,
-                       storePayloads);
-          skipper.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
-          skipped = true;
-        }
-        final int newCount = skipper.skipTo(target); 
-        //System.out.println("  skip newCount=" + newCount + " vs " + count);
-
-        if (newCount > count) {
-
-          // Skipper did move
-          skipper.getFreqIndex().seek(freqReader);
-          skipper.getDocIndex().seek(docReader);
-          //System.out.println("  doc seek'd to " + skipper.getDocIndex());
-          // NOTE: don't seek pos here; do it lazily
-          // instead.  Eg a PhraseQuery may skip to many
-          // docs before finally asking for positions...
-          posIndex.copyFrom(skipper.getPosIndex());
-          posSeekPending = true;
-          count = newCount;
-          doc = accum = skipper.getDoc();
-          //System.out.println("    moved to doc=" + doc);
-          //payloadIn.seek(skipper.getPayloadPointer());
-          payloadFP = skipper.getPayloadPointer();
-          pendingPosCount = 0;
-          pendingPayloadBytes = 0;
-          payloadPending = false;
-          payloadLength = skipper.getPayloadLength();
-          //System.out.println("    move payloadLen=" + payloadLength);
+        internal sealed class SepTermState : BlockTermState
+        {
+            // We store only the seek point to the docs file because
+            // the rest of the info (freqIndex, posIndex, etc.) is
+            // stored in the docs file:
+            internal IntIndexInputIndex DOC_INDEX;
+            internal IntIndexInputIndex POS_INDEX;
+            internal IntIndexInputIndex FREQ_INDEX;
+            internal long PAYLOAD_FP;
+            internal long SKIP_FP;
+
+            public override object Clone()
+            {
+                var other = new SepTermState();
+                other.CopyFrom(this);
+                return other;
+            }
+
+            public override void CopyFrom(TermState tsOther)
+            {
+                base.CopyFrom(tsOther);
+
+                var other = (SepTermState)tsOther;
+                if (DOC_INDEX == null)
+                {
+                    DOC_INDEX = other.DOC_INDEX.Clone();
+                }
+                else
+                {
+                    DOC_INDEX.CopyFrom(other.DOC_INDEX);
+                }
+                if (other.FREQ_INDEX != null)
+                {
+                    if (FREQ_INDEX == null)
+                    {
+                        FREQ_INDEX = other.FREQ_INDEX.Clone();
+                    }
+                    else
+                    {
+                        FREQ_INDEX.CopyFrom(other.FREQ_INDEX);
+                    }
+                }
+                else
+                {
+                    FREQ_INDEX = null;
+                }
+                if (other.POS_INDEX != null)
+                {
+                    if (POS_INDEX == null)
+                    {
+                        POS_INDEX = other.POS_INDEX.Clone();
+                    }
+                    else
+                    {
+                        POS_INDEX.CopyFrom(other.POS_INDEX);
+                    }
+                }
+                else
+                {
+                    POS_INDEX = null;
+                }
+                PAYLOAD_FP = other.PAYLOAD_FP;
+                SKIP_FP = other.SKIP_FP;
+            }
+
+            public override string ToString()
+            {
+                return base.ToString() + " docIndex=" + DOC_INDEX + " freqIndex=" + FREQ_INDEX + " posIndex=" + POS_INDEX +
+                       " payloadFP=" + PAYLOAD_FP + " skipFP=" + SKIP_FP;
+            }
         }
-      }
-        
-      // Now, linear scan for the rest:
-      do {
-        if (nextDoc() == NO_MORE_DOCS) {
-          //System.out.println("  advance nextDoc=END");
-          return NO_MORE_DOCS;
-        }
-        //System.out.println("  advance nextDoc=" + doc);
-      } while (target > doc);
 
-      //System.out.println("  return doc=" + doc);
-      return doc;
-    }
 
-    @Override
-    public int nextPosition()  {
-      if (posSeekPending) {
-        posIndex.seek(posReader);
-        payloadIn.seek(payloadFP);
-        posSeekPending = false;
-      }
-
-      // scan over any docs that were iterated without their
-      // positions
-      while (pendingPosCount > freq) {
-        final int code = posReader.next();
-        if (storePayloads && (code & 1) != 0) {
-          // Payload length has changed
-          payloadLength = posReader.next();
-          Debug.Assert( payloadLength >= 0;
+        internal class SepDocsEnum : DocsEnum
+        {
+            private readonly SepPostingsReader _outerInstance;
+
+            private int _docFreq;
+            private int _doc = -1;
+            private int _accum;
+            private int _count;
+            private int _freq;
+
+            // TODO: -- should we do omitTF with 2 different enum classes?
+            private bool _omitTf;
+            private FieldInfo.IndexOptions _indexOptions;
+            private bool _storePayloads;
+            private Bits _liveDocs;
+            private readonly IntIndexInputReader _docReader;
+            private readonly IntIndexInputReader _freqReader;
+            private long _skipFp;
+
+            private readonly IntIndexInputIndex _docIndex;
+            private readonly IntIndexInputIndex _freqIndex;
+            private readonly IntIndexInputIndex _posIndex;
+
+            // TODO: -- should we do hasProx with 2 different enum classes?
+
+            private bool _skipped;
+            private SepSkipListReader _skipper;
+
+            internal IntIndexInput START_DOC_IN;
+
+            internal SepDocsEnum(SepPostingsReader outerInstance)
+            {
+                _outerInstance = outerInstance;
+                _docReader = outerInstance._docIn.Reader();
+                _docIndex = outerInstance._docIn.Index();
+                if (outerInstance._freqIn != null)
+                {
+                    _freqReader = outerInstance._freqIn.Reader();
+                    _freqIndex = outerInstance._freqIn.Index();
+                }
+                else
+                {
+                    _freqReader = null;
+                    _freqIndex = null;
+                }
+                _posIndex = outerInstance._posIn != null ? outerInstance._posIn.Index() : null;
+
+                START_DOC_IN = outerInstance._docIn;
+            }
+
+            internal virtual SepDocsEnum Init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs)
+            {
+                _liveDocs = liveDocs;
+                if (fieldInfo.FieldIndexOptions.HasValue)
+                    _indexOptions = fieldInfo.FieldIndexOptions.Value;
+
+                _omitTf = _indexOptions == FieldInfo.IndexOptions.DOCS_ONLY;
+                _storePayloads = fieldInfo.HasPayloads();
+
+                // TODO: can't we only do this if consumer
+                // skipped consuming the previous docs?
+                _docIndex.CopyFrom(termState.DOC_INDEX);
+                _docIndex.Seek(_docReader);
+
+                if (!_omitTf)
+                {
+                    _freqIndex.CopyFrom(termState.FREQ_INDEX);
+                    _freqIndex.Seek(_freqReader);
+                }
+
+                _docFreq = termState.DocFreq;
+                // NOTE: unused if docFreq < skipMinimum:
+                _skipFp = termState.SKIP_FP;
+                _count = 0;
+                _doc = -1;
+                _accum = 0;
+                _freq = 1;
+                _skipped = false;
+
+                return this;
+            }
+
+            public override int NextDoc()
+            {
+
+                while (true)
+                {
+                    if (_count == _docFreq)
+                    {
+                        return _doc = NO_MORE_DOCS;
+                    }
+
+                    _count++;
+
+                    // Decode next doc
+                    //System.out.println("decode docDelta:");
+                    _accum += _docReader.Next();
+
+                    if (!_omitTf)
+                    {
+                        //System.out.println("decode freq:");
+                        _freq = _freqReader.Next();
+                    }
+
+                    if (_liveDocs == null || _liveDocs.Get(_accum))
+                    {
+                        break;
+                    }
+                }
+                return (_doc = _accum);
+            }
+
+            public override int Freq()
+            {
+                return _freq;
+            }
+
+            public override int DocID()
+            {
+                return _doc;
+            }
+
+            public override int Advance(int target)
+            {
+
+                if ((target - _outerInstance._skipInterval) >= _doc && _docFreq >= _outerInstance._skipMinimum)
+                {
+
+                    // There are enough docs in the posting to have
+                    // skip data, and its not too close
+
+                    if (_skipper == null)
+                    {
+                        // This DocsEnum has never done any skipping
+                        _skipper = new SepSkipListReader((IndexInput) _outerInstance._skipIn.Clone(),
+                            _outerInstance._freqIn,
+                            _outerInstance._docIn, _outerInstance._posIn, _outerInstance._maxSkipLevels,
+                            _outerInstance._skipInterval);
+
+                    }
+
+                    if (!_skipped)
+                    {
+                        // We haven't yet skipped for this posting
+                        _skipper.Init(_skipFp, _docIndex, _freqIndex, _posIndex, 0, _docFreq, _storePayloads);
+                        _skipper.IndexOptions = _indexOptions;
+
+                        _skipped = true;
+                    }
+
+                    int newCount = _skipper.SkipTo(target);
+
+                    if (newCount > _count)
+                    {
+
+                        // Skipper did move
+                        if (!_omitTf)
+                        {
+                            _skipper.FreqIndex.Seek(_freqReader);
+                        }
+                        _skipper.DocIndex.Seek(_docReader);
+                        _count = newCount;
+                        _doc = _accum = _skipper.Doc;
+                    }
+                }
+
+                // Now, linear scan for the rest:
+                do
+                {
+                    if (NextDoc() == NO_MORE_DOCS)
+                    {
+                        return NO_MORE_DOCS;
+                    }
+                } while (target > _doc);
+
+                return _doc;
+            }
+
+            public override long Cost()
+            {
+                return _docFreq;
+            }
         }
-        pendingPosCount--;
-        position = 0;
-        pendingPayloadBytes += payloadLength;
-      }
-
-      final int code = posReader.next();
-
-      if (storePayloads) {
-        if ((code & 1) != 0) {
-          // Payload length has changed
-          payloadLength = posReader.next();
-          Debug.Assert( payloadLength >= 0;
-        }
-        position += code >>> 1;
-        pendingPayloadBytes += payloadLength;
-        payloadPending = payloadLength > 0;
-      } else {
-        position += code;
-      }
-    
-      pendingPosCount--;
-      Debug.Assert( pendingPosCount >= 0;
-      return position;
-    }
-
-    @Override
-    public int startOffset() {
-      return -1;
-    }
 
-    @Override
-    public int endOffset() {
-      return -1;
-    }
-
-    private BytesRef payload;
-
-    @Override
-    public BytesRef getPayload()  {
-      if (!payloadPending) {
-        return null;
-      }
-      
-      if (pendingPayloadBytes == 0) {
-        return payload;
-      }
-
-      Debug.Assert( pendingPayloadBytes >= payloadLength;
-
-      if (pendingPayloadBytes > payloadLength) {
-        payloadIn.seek(payloadIn.getFilePointer() + (pendingPayloadBytes - payloadLength));
-      }
-
-      if (payload == null) {
-        payload = new BytesRef();
-        payload.bytes = new byte[payloadLength];
-      } else if (payload.bytes.length < payloadLength) {
-        payload.grow(payloadLength);
-      }
-
-      payloadIn.readBytes(payload.bytes, 0, payloadLength);
-      payload.length = payloadLength;
-      pendingPayloadBytes = 0;
-      return payload;
-    }
-    
-    @Override
-    public long cost() {
-      return docFreq;
+        internal class SepDocsAndPositionsEnum : DocsAndPositionsEnum
+        {
+            private readonly SepPostingsReader _outerInstance;
+            private BytesRef _payload;
+
+            private int _docFreq;
+            private int _doc = -1;
+            private int _accum;
+            private int _count;
+            private int _freq;
+
+            private bool _storePayloads;
+            private Bits _liveDocs;
+            private readonly IntIndexInputReader _docReader;
+            private readonly IntIndexInputReader _freqReader;
+            private readonly IntIndexInputReader _posReader;
+            private readonly IndexInput _payloadIn;
+            private long _skipFp;
+
+            private readonly IntIndexInputIndex _docIndex;
+            private readonly IntIndexInputIndex _freqIndex;
+            private readonly IntIndexInputIndex _posIndex;
+
+            private long _payloadFp;
+
+            private int _pendingPosCount;
+            private int _position;
+            private int _payloadLength;
+            private long _pendingPayloadBytes;
+
+            private bool _skipped;
+            private SepSkipListReader _skipper;
+            private bool _payloadPending;
+            private bool _posSeekPending;
+
+            internal IntIndexInput START_DOC_IN;
+
+            internal SepDocsAndPositionsEnum(SepPostingsReader outerInstance)
+            {
+                _outerInstance = outerInstance;
+                _docReader = outerInstance._docIn.Reader();
+                _docIndex = outerInstance._docIn.Index();
+                _freqReader = outerInstance._freqIn.Reader();
+                _freqIndex = outerInstance._freqIn.Index();
+                _posReader = outerInstance._posIn.Reader();
+                _posIndex = outerInstance._posIn.Index();
+                _payloadIn = (IndexInput) outerInstance._payloadIn.Clone();
+
+                START_DOC_IN = outerInstance._docIn;
+            }
+
+            internal virtual SepDocsAndPositionsEnum Init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs)
+            {
+                _liveDocs = liveDocs;
+                _storePayloads = fieldInfo.HasPayloads();
+
+                // TODO: can't we only do this if consumer skipped consuming the previous docs?
+                _docIndex.CopyFrom(termState.DOC_INDEX);
+                _docIndex.Seek(_docReader);
+
+                _freqIndex.CopyFrom(termState.FREQ_INDEX);
+                _freqIndex.Seek(_freqReader);
+
+                _posIndex.CopyFrom(termState.POS_INDEX);
+                _posSeekPending = true;
+                _payloadPending = false;
+
+                _payloadFp = termState.PAYLOAD_FP;
+                _skipFp = termState.SKIP_FP;
+
+                _docFreq = termState.DocFreq;
+                _count = 0;
+                _doc = -1;
+                _accum = 0;
+                _pendingPosCount = 0;
+                _pendingPayloadBytes = 0;
+                _skipped = false;
+
+                return this;
+            }
+
+            public override int NextDoc()
+            {
+                while (true)
+                {
+                    if (_count == _docFreq)
+                        return _doc = NO_MORE_DOCS;
+
+                    _count++;
+
+                    // Decode next doc
+                    _accum += _docReader.Next();
+                    _freq = _freqReader.Next();
+                    _pendingPosCount += _freq;
+
+                    if (_liveDocs == null || _liveDocs.Get(_accum))
+                        break;
+                }
+
+                _position = 0;
+                return (_doc = _accum);
+            }
+
+            public override int Freq()
+            {
+                return _freq;
+            }
+
+            public override int DocID()
+            {
+                return _doc;
+            }
+
+            public override int Advance(int target)
+            {
+                //System.out.println("SepD&P advance target=" + target + " vs current=" + doc + " this=" + this);
+
+                if ((target - _outerInstance._skipInterval) >= _doc && _docFreq >= _outerInstance._skipMinimum)
+                {
+
+                    // There are enough docs in the posting to have
+                    // skip data, and its not too close
+
+                    if (_skipper == null)
+                    {
+                        //System.out.println("  create skipper");
+                        // This DocsEnum has never done any skipping
+                        _skipper = new SepSkipListReader((IndexInput) _outerInstance._skipIn.Clone(),
+                            _outerInstance._freqIn,
+                            _outerInstance._docIn, _outerInstance._posIn, _outerInstance._maxSkipLevels,
+                            _outerInstance._skipInterval);
+                    }
+
+                    if (!_skipped)
+                    {
+                        //System.out.println("  init skip data skipFP=" + skipFP);
+                        // We haven't yet skipped for this posting
+                        _skipper.Init(_skipFp, _docIndex, _freqIndex, _posIndex, _payloadFp, _docFreq, _storePayloads);
+                        _skipper.IndexOptions = FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+                        _skipped = true;
+                    }
+
+                    int newCount = _skipper.SkipTo(target);
+
+                    if (newCount > _count)
+                    {
+
+                        // Skipper did move
+                        _skipper.FreqIndex.Seek(_freqReader);
+                        _skipper.DocIndex.Seek(_docReader);
+
+                        // NOTE: don't seek pos here; do it lazily
+                        // instead.  Eg a PhraseQuery may skip to many
+                        // docs before finally asking for positions...
+
+                        _posIndex.CopyFrom(_skipper.PosIndex);
+                        _posSeekPending = true;
+                        _count = newCount;
+                        _doc = _accum = _skipper.Doc;
+
+                        _payloadFp = _skipper.PayloadPointer;
+                        _pendingPosCount = 0;
+                        _pendingPayloadBytes = 0;
+                        _payloadPending = false;
+                        _payloadLength = _skipper.PayloadLength;
+                    }
+                }
+
+                // Now, linear scan for the rest:
+                do
+                {
+                    if (NextDoc() == NO_MORE_DOCS)
+                    {
+                        return NO_MORE_DOCS;
+                    }
+
+                } while (target > _doc);
+
+                return _doc;
+            }
+
+            public override int NextPosition()
+            {
+                if (_posSeekPending)
+                {
+                    _posIndex.Seek(_posReader);
+                    _payloadIn.Seek(_payloadFp);
+                    _posSeekPending = false;
+                }
+
+                int code;
+
+                // scan over any docs that were iterated without their positions
+                while (_pendingPosCount > _freq)
+                {
+                    code = _posReader.Next();
+                    if (_storePayloads && (code & 1) != 0)
+                    {
+                        // Payload length has changed
+                        _payloadLength = _posReader.Next();
+                        Debug.Assert(_payloadLength >= 0);
+                    }
+                    _pendingPosCount--;
+                    _position = 0;
+                    _pendingPayloadBytes += _payloadLength;
+                }
+
+                code = _posReader.Next();
+
+                if (_storePayloads)
+                {
+                    if ((code & 1) != 0)
+                    {
+                        // Payload length has changed
+                        _payloadLength = _posReader.Next();
+                        Debug.Assert(_payloadLength >= 0);
+                    }
+                    _position += (int) ((uint) code >> 1);
+                    _pendingPayloadBytes += _payloadLength;
+                    _payloadPending = _payloadLength > 0;
+                }
+                else
+                {
+                    _position += code;
+                }
+
+                _pendingPosCount--;
+                Debug.Assert(_pendingPosCount >= 0);
+                return _position;
+            }
+
+            public override int StartOffset()
+            {
+                return -1;
+            }
+
+            public override int EndOffset()
+            {
+                return -1;
+            }
+
+            public override BytesRef Payload
+            {
+                get
+                {
+                    if (!_payloadPending)
+                    {
+                        return null;
+                    }
+
+                    if (_pendingPayloadBytes == 0)
+                    {
+                        return _payload;
+                    }
+
+                    Debug.Assert(_pendingPayloadBytes >= _payloadLength);
+
+                    if (_pendingPayloadBytes > _payloadLength)
+                    {
+                        _payloadIn.Seek(_payloadIn.FilePointer + (_pendingPayloadBytes - _payloadLength));
+                    }
+
+                    if (_payload == null)
+                    {
+                        _payload = new BytesRef {Bytes = new sbyte[_payloadLength]};
+                    }
+                    else if (_payload.Bytes.Length < _payloadLength)
+                    {
+                        _payload.Grow(_payloadLength);
+                    }
+
+                    _payloadIn.ReadBytes(_payload.Bytes, 0, _payloadLength);
+                    _payload.Length = _payloadLength;
+                    _pendingPayloadBytes = 0;
+                    return _payload;
+                }
+            }
+
+            public override long Cost()
+            {
+                return _docFreq;
+            }
+        }
     }
-  }
-
-  @Override
-  public long ramBytesUsed() {
-    return 0;
-  }
-
-  @Override
-  public void checkIntegrity()  {
-    // TODO: remove sep layout, its fallen behind on features...
-  }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d852d5b0/src/Lucene.Net.Codecs/Sep/SepPostingsWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Sep/SepPostingsWriter.cs b/src/Lucene.Net.Codecs/Sep/SepPostingsWriter.cs
index eba7022..602f724 100644
--- a/src/Lucene.Net.Codecs/Sep/SepPostingsWriter.cs
+++ b/src/Lucene.Net.Codecs/Sep/SepPostingsWriter.cs
@@ -1,6 +1,4 @@
-package codecs.sep;
-
-/*
+/*
  * 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.
@@ -17,355 +15,397 @@ package codecs.sep;
  * limitations under the License.
  */
 
-import java.io.IOException;
-
-import codecs.BlockTermState;
-import codecs.CodecUtil;
-import codecs.PostingsWriterBase;
-import index.CorruptIndexException;
-import index.DocsEnum;
-import index.FieldInfo;
-import index.FieldInfo.IndexOptions;
-import index.IndexFileNames;
-import index.SegmentWriteState;
-import store.DataOutput;
-import store.IndexOutput;
-import store.RAMOutputStream;
-import util.BytesRef;
-import util.IOUtils;
-
-/** Writes frq to .frq, docs to .doc, pos to .pos, payloads
- *  to .pyl, skip data to .skp
- *
- * @lucene.experimental */
-public final class SepPostingsWriter extends PostingsWriterBase {
-  final static String CODEC = "SepPostingsWriter";
-
-  final static String DOC_EXTENSION = "doc";
-  final static String SKIP_EXTENSION = "skp";
-  final static String FREQ_EXTENSION = "frq";
-  final static String POS_EXTENSION = "pos";
-  final static String PAYLOAD_EXTENSION = "pyl";
-
-  // Increment version to change it:
-  final static int VERSION_START = 0;
-  final static int VERSION_CURRENT = VERSION_START;
-
-  IntIndexOutput freqOut;
-  IntIndexOutput.Index freqIndex;
-
-  IntIndexOutput posOut;
-  IntIndexOutput.Index posIndex;
-
-  IntIndexOutput docOut;
-  IntIndexOutput.Index docIndex;
-
-  IndexOutput payloadOut;
-
-  IndexOutput skipOut;
-
-  final SepSkipListWriter skipListWriter;
-  /** Expert: The fraction of TermDocs entries stored in skip tables,
-   * used to accelerate {@link DocsEnum#advance(int)}.  Larger values result in
-   * smaller indexes, greater acceleration, but fewer accelerable cases, while
-   * smaller values result in bigger indexes, less acceleration and more
-   * accelerable cases. More detailed experiments would be useful here. */
-  final int skipInterval;
-  static final int DEFAULT_SKIP_INTERVAL = 16;
-  
-  /**
-   * Expert: minimum docFreq to write any skip data at all
-   */
-  final int skipMinimum;
-
-  /** Expert: The maximum number of skip levels. Smaller values result in 
-   * slightly smaller indexes, but slower skipping in big posting lists.
-   */
-  final int maxSkipLevels = 10;
-
-  final int totalNumDocs;
-
-  bool storePayloads;
-  IndexOptions indexOptions;
-
-  FieldInfo fieldInfo;
-
-  int lastPayloadLength;
-  int lastPosition;
-  long payloadStart;
-  int lastDocID;
-  int df;
-
-  SepTermState lastState;
-  long lastPayloadFP;
-  long lastSkipFP;
-
-  public SepPostingsWriter(SegmentWriteState state, IntStreamFactory factory)  {
-    this(state, factory, DEFAULT_SKIP_INTERVAL);
-  }
-
-  public SepPostingsWriter(SegmentWriteState state, IntStreamFactory factory, int skipInterval)  {
-    freqOut = null;
-    freqIndex = null;
-    posOut = null;
-    posIndex = null;
-    payloadOut = null;
-    bool success = false;
-    try {
-      this.skipInterval = skipInterval;
-      this.skipMinimum = skipInterval; /* set to the same for now */
-      final String docFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, DOC_EXTENSION);
-
-      docOut = factory.createOutput(state.directory, docFileName, state.context);
-      docIndex = docOut.index();
-
-      if (state.fieldInfos.hasFreq()) {
-        final String frqFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, FREQ_EXTENSION);
-        freqOut = factory.createOutput(state.directory, frqFileName, state.context);
-        freqIndex = freqOut.index();
-      }
-
-      if (state.fieldInfos.hasProx()) {      
-        final String posFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, POS_EXTENSION);
-        posOut = factory.createOutput(state.directory, posFileName, state.context);
-        posIndex = posOut.index();
-        
-        // TODO: -- only if at least one field stores payloads?
-        final String payloadFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, PAYLOAD_EXTENSION);
-        payloadOut = state.directory.createOutput(payloadFileName, state.context);
-      }
-
-      final String skipFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, SKIP_EXTENSION);
-      skipOut = state.directory.createOutput(skipFileName, state.context);
-      
-      totalNumDocs = state.segmentInfo.getDocCount();
-      
-      skipListWriter = new SepSkipListWriter(skipInterval,
-          maxSkipLevels,
-          totalNumDocs,
-          freqOut, docOut,
-          posOut, payloadOut);
-      
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeWhileHandlingException(docOut, skipOut, freqOut, posOut, payloadOut);
-      }
-    }
-  }
-
-  @Override
-  public void init(IndexOutput termsOut)  {
-    CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT);
-    // TODO: -- just ask skipper to "start" here
-    termsOut.writeInt(skipInterval);                // write skipInterval
-    termsOut.writeInt(maxSkipLevels);               // write maxSkipLevels
-    termsOut.writeInt(skipMinimum);                 // write skipMinimum
-  }
-
-  @Override
-  public BlockTermState newTermState() {
-    return new SepTermState();
-  }
-
-  @Override
-  public void startTerm()  {
-    docIndex.mark();
-    //System.out.println("SEPW: startTerm docIndex=" + docIndex);
-
-    if (indexOptions != IndexOptions.DOCS_ONLY) {
-      freqIndex.mark();
-    }
-    
-    if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-      posIndex.mark();
-      payloadStart = payloadOut.getFilePointer();
-      lastPayloadLength = -1;
-    }
+namespace Lucene.Net.Codecs.Sep
+{
+    using System.Diagnostics;
+    using Index;
+    using Store;
+    using Util;
+
+    /// <summary>
+    /// Writes frq to .frq, docs to .doc, pos to .pos, payloads
+    /// to .pyl, skip data to .skp
+    /// 
+    /// @lucene.experimental 
+    /// </summary>
+    public sealed class SepPostingsWriter : PostingsWriterBase
+    {
+        internal const string CODEC = "SepPostingsWriter";
+
+        internal const string DOC_EXTENSION = "doc";
+        internal const string SKIP_EXTENSION = "skp";
+        internal const string FREQ_EXTENSION = "frq";
+        internal const string POS_EXTENSION = "pos";
+        internal const string PAYLOAD_EXTENSION = "pyl";
+
+        // Increment version to change it:
+        internal const int VERSION_START = 0;
+        internal const int VERSION_CURRENT = VERSION_START;
+
+        internal IntIndexOutput FREQ_OUT;
+        internal IntIndexOutputIndex FREQ_INDEX;
+
+        internal IntIndexOutput POS_OUT;
+        internal IntIndexOutputIndex POS_INDEX;
+
+        internal IntIndexOutput DOC_OUT;
+        internal IntIndexOutputIndex DOC_INDEX;
+
+        internal IndexOutput PAYLOAD_OUT;
+
+        internal IndexOutput SKIP_OUT;
+
+        internal readonly SepSkipListWriter SKIP_LIST_WRITER;
+
+        /// <summary>
+        /// Expert: The fraction of TermDocs entries stored in skip tables,
+        /// used to accelerate <seealso cref="DocsEnum#advance(int)"/>.  Larger values result in
+        /// smaller indexes, greater acceleration, but fewer accelerable cases, while
+        /// smaller values result in bigger indexes, less acceleration and more
+        /// accelerable cases. More detailed experiments would be useful here. 
+        /// </summary>
+        internal readonly int SKIP_INTERVAL;
+
+        internal const int DEFAULT_SKIP_INTERVAL = 16;
+
+        /// <summary>
+        /// Expert: minimum docFreq to write any skip data at all
+        /// </summary>
+        internal readonly int SKIP_MINIMUM;
+
+        /// <summary>
+        /// Expert: The maximum number of skip levels. Smaller values result in 
+        /// slightly smaller indexes, but slower skipping in big posting lists.
+        /// </summary>
+        internal readonly int MAX_SKIP_LEVELS = 10;
+
+        internal readonly int TOTAL_NUM_DOCS;
+
+        internal bool STORE_PAYLOADS;
+        internal FieldInfo.IndexOptions INDEX_OPTIONS;
+
+        internal FieldInfo FIELD_INFO;
+
+        internal int LAST_PAYLOAD_LENGTH;
+        internal int LAST_POSITION;
+        internal long PAYLOAD_START;
+        internal int LAST_DOC_ID;
+        internal int DF;
+
+        private SepTermState _lastState;
+        internal long LAST_PAYLOAD_FP;
+        internal long LAST_SKIP_FP;
+
+        public SepPostingsWriter(SegmentWriteState state, IntStreamFactory factory)
+            : this(state, factory, DEFAULT_SKIP_INTERVAL)
+        {
+        }
 
-    skipListWriter.resetSkip(docIndex, freqIndex, posIndex);
-  }
-
-  // Currently, this instance is re-used across fields, so
-  // our parent calls setField whenever the field changes
-  @Override
-  public int setField(FieldInfo fieldInfo) {
-    this.fieldInfo = fieldInfo;
-    this.indexOptions = fieldInfo.getIndexOptions();
-    if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) {
-      throw new UnsupportedOperationException("this codec cannot index offsets");
-    }
-    skipListWriter.setIndexOptions(indexOptions);
-    storePayloads = indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS && fieldInfo.hasPayloads();
-    lastPayloadFP = 0;
-    lastSkipFP = 0;
-    lastState = setEmptyState();
-    return 0;
-  }
-
-  private SepTermState setEmptyState() {
-    SepTermState emptyState = new SepTermState();
-    emptyState.docIndex = docOut.index();
-    if (indexOptions != IndexOptions.DOCS_ONLY) {
-      emptyState.freqIndex = freqOut.index();
-      if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-        emptyState.posIndex = posOut.index();
-      }
-    }
-    emptyState.payloadFP = 0;
-    emptyState.skipFP = 0;
-    return emptyState;
-  }
+        public SepPostingsWriter(SegmentWriteState state, IntStreamFactory factory, int skipInterval)
+        {
+            FREQ_OUT = null;
+            FREQ_INDEX = null;
+            POS_OUT = null;
+            POS_INDEX = null;
+            PAYLOAD_OUT = null;
+            var success = false;
+            try
+            {
+                SKIP_INTERVAL = skipInterval;
+                SKIP_MINIMUM = skipInterval; // set to the same for now
+                var docFileName = IndexFileNames.SegmentFileName(state.SegmentInfo.Name, state.SegmentSuffix, DOC_EXTENSION);
+
+                DOC_OUT = factory.CreateOutput(state.Directory, docFileName, state.Context);
+                DOC_INDEX = DOC_OUT.Index();
+
+                if (state.FieldInfos.HasFreq())
+                {
+                    var frqFileName = IndexFileNames.SegmentFileName(state.SegmentInfo.Name, state.SegmentSuffix, FREQ_EXTENSION);
+                    FREQ_OUT = factory.CreateOutput(state.Directory, frqFileName, state.Context);
+                    FREQ_INDEX = FREQ_OUT.Index();
+                }
+
+                if (state.FieldInfos.HasProx())
+                {
+                    var posFileName = IndexFileNames.SegmentFileName(state.SegmentInfo.Name, state.SegmentSuffix, POS_EXTENSION);
+                    POS_OUT = factory.CreateOutput(state.Directory, posFileName, state.Context);
+                    POS_INDEX = POS_OUT.Index();
+
+                    // TODO: -- only if at least one field stores payloads?
+                    var payloadFileName = IndexFileNames.SegmentFileName(state.SegmentInfo.Name, state.SegmentSuffix,PAYLOAD_EXTENSION);
+                    PAYLOAD_OUT = state.Directory.CreateOutput(payloadFileName, state.Context);
+                }
+
+                var skipFileName = IndexFileNames.SegmentFileName(state.SegmentInfo.Name, state.SegmentSuffix, SKIP_EXTENSION);
+                SKIP_OUT = state.Directory.CreateOutput(skipFileName, state.Context);
+
+                TOTAL_NUM_DOCS = state.SegmentInfo.DocCount;
+
+                SKIP_LIST_WRITER = new SepSkipListWriter(skipInterval, MAX_SKIP_LEVELS, TOTAL_NUM_DOCS, FREQ_OUT, DOC_OUT,
+                    POS_OUT, PAYLOAD_OUT);
+
+                success = true;
+            }
+            finally
+            {
+                if (!success)
+                {
+                    IOUtils.CloseWhileHandlingException(DOC_OUT, SKIP_OUT, FREQ_OUT, POS_OUT, PAYLOAD_OUT);
+                }
+            }
+        }
+        public override void Init(IndexOutput termsOut)
+        {
+            CodecUtil.WriteHeader(termsOut, CODEC, VERSION_CURRENT);
+            // TODO: -- just ask skipper to "start" here
+            termsOut.WriteInt(SKIP_INTERVAL);    // write skipInterval
+            termsOut.WriteInt(MAX_SKIP_LEVELS);   // write maxSkipLevels
+            termsOut.WriteInt(SKIP_MINIMUM);     // write skipMinimum
+        }
 
-  /** Adds a new doc in this term.  If this returns null
-   *  then we just skip consuming positions/payloads. */
-  @Override
-  public void startDoc(int docID, int termDocFreq)  {
+        public override BlockTermState NewTermState()
+        {
+            return new SepTermState();
+        }
 
-    final int delta = docID - lastDocID;
-    //System.out.println("SEPW: startDoc: write doc=" + docID + " delta=" + delta + " out.fp=" + docOut);
+        public override void StartTerm()
+        {
+            DOC_INDEX.Mark();
+            
+            if (INDEX_OPTIONS != FieldInfo.IndexOptions.DOCS_ONLY)
+            {
+                FREQ_INDEX.Mark();
+            }
+
+            if (INDEX_OPTIONS == FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS)
+            {
+                POS_INDEX.Mark();
+                PAYLOAD_START = PAYLOAD_OUT.FilePointer;
+                LAST_PAYLOAD_LENGTH = -1;
+            }
+
+            SKIP_LIST_WRITER.ResetSkip(DOC_INDEX, FREQ_INDEX, POS_INDEX);
+        }
 
-    if (docID < 0 || (df > 0 && delta <= 0)) {
-      throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " ) (docOut: " + docOut + ")");
-    }
+        // Currently, this instance is re-used across fields, so
+        // our parent calls setField whenever the field changes
+        public override int SetField(FieldInfo fi)
+        {
+            FIELD_INFO = fi;
+            
+            if (FIELD_INFO.FieldIndexOptions.HasValue)
+                INDEX_OPTIONS = FIELD_INFO.FieldIndexOptions.Value;
+
+            if (INDEX_OPTIONS >= FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS)
+            {
+                throw new System.NotSupportedException("this codec cannot index offsets");
+            }
+            SKIP_LIST_WRITER.IndexOptions = INDEX_OPTIONS;
+            STORE_PAYLOADS = INDEX_OPTIONS == FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS &&
+                            FIELD_INFO.HasPayloads();
+            LAST_PAYLOAD_FP = 0;
+            LAST_SKIP_FP = 0;
+            _lastState = SetEmptyState();
+            return 0;
+        }
 
-    if ((++df % skipInterval) == 0) {
-      // TODO: -- awkward we have to make these two
-      // separate calls to skipper
-      //System.out.println("    buffer skip lastDocID=" + lastDocID);
-      skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength);
-      skipListWriter.bufferSkip(df);
-    }
+        private SepTermState SetEmptyState()
+        {
+            var emptyState = new SepTermState {DocIndex = DOC_OUT.Index()};
+            if (INDEX_OPTIONS != FieldInfo.IndexOptions.DOCS_ONLY)
+            {
+                emptyState.FreqIndex = FREQ_OUT.Index();
+                if (INDEX_OPTIONS == FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS)
+                {
+                    emptyState.PosIndex = POS_OUT.Index();
+                }
+            }
+            emptyState.PayloadFp = 0;
+            emptyState.SkipFp = 0;
+            return emptyState;
+        }
 
-    lastDocID = docID;
-    docOut.write(delta);
-    if (indexOptions != IndexOptions.DOCS_ONLY) {
-      //System.out.println("    sepw startDoc: write freq=" + termDocFreq);
-      freqOut.write(termDocFreq);
-    }
-  }
-
-  /** Add a new position & payload */
-  @Override
-  public void addPosition(int position, BytesRef payload, int startOffset, int endOffset)  {
-    Debug.Assert( indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
-
-    final int delta = position - lastPosition;
-    Debug.Assert( delta >= 0: "position=" + position + " lastPosition=" + lastPosition;            // not quite right (if pos=0 is repeated twice we don't catch it)
-    lastPosition = position;
-
-    if (storePayloads) {
-      final int payloadLength = payload == null ? 0 : payload.length;
-      if (payloadLength != lastPayloadLength) {
-        lastPayloadLength = payloadLength;
-        // TODO: explore whether we get better compression
-        // by not storing payloadLength into prox stream?
-        posOut.write((delta<<1)|1);
-        posOut.write(payloadLength);
-      } else {
-        posOut.write(delta << 1);
-      }
-
-      if (payloadLength > 0) {
-        payloadOut.writeBytes(payload.bytes, payload.offset, payloadLength);
-      }
-    } else {
-      posOut.write(delta);
-    }
+        /// <summary>
+        /// Adds a new doc in this term.  If this returns null
+        ///  then we just skip consuming positions/payloads. 
+        /// </summary>
+        public override void StartDoc(int docId, int termDocFreq)
+        {
+            var delta = docId - LAST_DOC_ID;
+            
+            if (docId < 0 || (DF > 0 && delta <= 0))
+            {
+                throw new CorruptIndexException("docs out of order (" + docId + " <= " + LAST_DOC_ID + " ) (docOut: " +
+                                                DOC_OUT + ")");
+            }
+
+            if ((++DF%SKIP_INTERVAL) == 0)
+            {
+                // TODO: -- awkward we have to make these two separate calls to skipper
+                SKIP_LIST_WRITER.SetSkipData(LAST_DOC_ID, STORE_PAYLOADS, LAST_PAYLOAD_LENGTH);
+                SKIP_LIST_WRITER.BufferSkip(DF);
+            }
+
+            LAST_DOC_ID = docId;
+            DOC_OUT.Write(delta);
+            if (INDEX_OPTIONS != FieldInfo.IndexOptions.DOCS_ONLY)
+            {
+                //System.out.println("    sepw startDoc: write freq=" + termDocFreq);
+                FREQ_OUT.Write(termDocFreq);
+            }
+        }
 
-    lastPosition = position;
-  }
-
-  /** Called when we are done adding positions & payloads */
-  @Override
-  public void finishDoc() {       
-    lastPosition = 0;
-  }
-
-  private static class SepTermState extends BlockTermState {
-    public IntIndexOutput.Index docIndex;
-    public IntIndexOutput.Index freqIndex;
-    public IntIndexOutput.Index posIndex;
-    public long payloadFP;
-    public long skipFP;
-  }
-
-  /** Called when we are done adding docs to this term */
-  @Override
-  public void finishTerm(BlockTermState _state)  {
-    SepTermState state = (SepTermState)_state;
-    // TODO: -- wasteful we are counting this in two places?
-    Debug.Assert( state.docFreq > 0;
-    Debug.Assert( state.docFreq == df;
-
-    state.docIndex = docOut.index();
-    state.docIndex.copyFrom(docIndex, false);
-    if (indexOptions != IndexOptions.DOCS_ONLY) {
-      state.freqIndex = freqOut.index();
-      state.freqIndex.copyFrom(freqIndex, false);
-      if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-        state.posIndex = posOut.index();
-        state.posIndex.copyFrom(posIndex, false);
-      } else {
-        state.posIndex = null;
-      }
-    } else {
-      state.freqIndex = null;
-      state.posIndex = null;
-    }
+        /// <summary>
+        /// Add a new position & payload </summary>
+        public override void AddPosition(int position, BytesRef payload, int startOffset, int endOffset)
+        {
+            Debug.Assert(INDEX_OPTIONS == FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+            int delta = position - LAST_POSITION;
+            Debug.Assert(delta >= 0, "position=" + position + " lastPosition=" + LAST_POSITION);
+            // not quite right (if pos=0 is repeated twice we don't catch it)
+            LAST_POSITION = position;
+
+            if (STORE_PAYLOADS)
+            {
+                int payloadLength = payload == null ? 0 : payload.Length;
+                if (payloadLength != LAST_PAYLOAD_LENGTH)
+                {
+                    LAST_PAYLOAD_LENGTH = payloadLength;
+                    // TODO: explore whether we get better compression
+                    // by not storing payloadLength into prox stream?
+                    POS_OUT.Write((delta << 1) | 1);
+                    POS_OUT.Write(payloadLength);
+                }
+                else
+                {
+                    POS_OUT.Write(delta << 1);
+                }
+
+                if (payloadLength > 0 && payload != null)
+                {
+                    PAYLOAD_OUT.WriteBytes(payload.Bytes, payload.Offset, payloadLength);
+                }
+            }
+            else
+            {
+                POS_OUT.Write(delta);
+            }
+
+            LAST_POSITION = position;
+        }
 
-    if (df >= skipMinimum) {
-      state.skipFP = skipOut.getFilePointer();
-      //System.out.println("  skipFP=" + skipFP);
-      skipListWriter.writeSkip(skipOut);
-      //System.out.println("    numBytes=" + (skipOut.getFilePointer()-skipFP));
-    } else {
-      state.skipFP = -1;
-    }
-    state.payloadFP = payloadStart;
-
-    lastDocID = 0;
-    df = 0;
-  }
-
-  @Override
-  public void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, bool absolute)  {
-    SepTermState state = (SepTermState)_state;
-    if (absolute) {
-      lastSkipFP = 0;
-      lastPayloadFP = 0;
-      lastState = state;
-    }
-    lastState.docIndex.copyFrom(state.docIndex, false);
-    lastState.docIndex.write(out, absolute);
-    if (indexOptions != IndexOptions.DOCS_ONLY) {
-      lastState.freqIndex.copyFrom(state.freqIndex, false);
-      lastState.freqIndex.write(out, absolute);
-      if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-        lastState.posIndex.copyFrom(state.posIndex, false);
-        lastState.posIndex.write(out, absolute);
-        if (storePayloads) {
-          if (absolute) {
-            out.writeVLong(state.payloadFP);
-          } else {
-            out.writeVLong(state.payloadFP - lastPayloadFP);
-          }
-          lastPayloadFP = state.payloadFP;
+        /// <summary>Called when we are done adding positions & payloads </summary>
+        public override void FinishDoc()
+        {
+            LAST_POSITION = 0;
+        }
+
+        private class SepTermState : BlockTermState
+        {
+            public IntIndexOutputIndex DocIndex { get; set; }
+            public IntIndexOutputIndex FreqIndex { get; set; }
+            public IntIndexOutputIndex PosIndex { get; set; }
+            public long PayloadFp { get; set; }
+            public long SkipFp { get; set; }
+        }
+
+        /// <summary>Called when we are done adding docs to this term </summary>
+        public override void FinishTerm(BlockTermState bstate)
+        {
+            var state = (SepTermState)bstate;
+            // TODO: -- wasteful we are counting this in two places?
+            Debug.Assert(state.DocFreq > 0);
+            Debug.Assert(state.DocFreq == DF);
+
+            state.DocIndex = DOC_OUT.Index();
+            state.DocIndex.CopyFrom(DOC_INDEX, false);
+            if (INDEX_OPTIONS != FieldInfo.IndexOptions.DOCS_ONLY)
+            {
+                state.FreqIndex = FREQ_OUT.Index();
+                state.FreqIndex.CopyFrom(FREQ_INDEX, false);
+                if (INDEX_OPTIONS == FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS)
+                {
+                    state.PosIndex = POS_OUT.Index();
+                    state.PosIndex.CopyFrom(POS_INDEX, false);
+                }
+                else
+                {
+                    state.PosIndex = null;
+                }
+            }
+            else
+            {
+                state.FreqIndex = null;
+                state.PosIndex = null;
+            }
+
+            if (DF >= SKIP_MINIMUM)
+            {
+                state.SkipFp = SKIP_OUT.FilePointer;
+                SKIP_LIST_WRITER.WriteSkip(SKIP_OUT);
+            }
+            else
+            {
+                state.SkipFp = -1;
+            }
+            state.PayloadFp = PAYLOAD_START;
+
+            LAST_DOC_ID = 0;
+            DF = 0;
+        }
+
+        public override void EncodeTerm(long[] longs, DataOutput output, FieldInfo fi, BlockTermState bstate, bool absolute)
+        {
+            var state = (SepTermState) bstate;
+            if (absolute)
+            {
+                LAST_SKIP_FP = 0;
+                LAST_PAYLOAD_FP = 0;
+                _lastState = state;
+            }
+            _lastState.DocIndex.CopyFrom(state.DocIndex, false);
+            _lastState.DocIndex.Write(output, absolute);
+            if (INDEX_OPTIONS != FieldInfo.IndexOptions.DOCS_ONLY)
+            {
+                _lastState.FreqIndex.CopyFrom(state.FreqIndex, false);
+                _lastState.FreqIndex.Write(output, absolute);
+                if (INDEX_OPTIONS == FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS)
+                {
+                    _lastState.PosIndex.CopyFrom(state.PosIndex, false);
+                    _lastState.PosIndex.Write(output, absolute);
+                    if (STORE_PAYLOADS)
+                    {
+                        if (absolute)
+                        {
+                            output.WriteVLong(state.PayloadFp);
+                        }
+                        else
+                        {
+                            output.WriteVLong(state.PayloadFp - LAST_PAYLOAD_FP);
+                        }
+                        LAST_PAYLOAD_FP = state.PayloadFp;
+                    }
+                }
+            }
+            if (state.SkipFp == -1) return;
+
+            if (absolute)
+            {
+                output.WriteVLong(state.SkipFp);
+            }
+            else
+            {
+                output.WriteVLong(state.SkipFp - LAST_SKIP_FP);
+            }
+            LAST_SKIP_FP = state.SkipFp;
+        }
+
+        protected override void Dispose(bool disposing)
+        {
+            if (disposing) return;
+
+            IOUtils.Close(DOC_OUT, SKIP_OUT, FREQ_OUT, POS_OUT, PAYLOAD_OUT);
         }
-      }
-    }
-    if (state.skipFP != -1) {
-      if (absolute) {
-        out.writeVLong(state.skipFP);
-      } else {
-        out.writeVLong(state.skipFP - lastSkipFP);
-      }
-      lastSkipFP = state.skipFP;
     }
-  }
 
-  @Override
-  public void close()  {
-    IOUtils.close(docOut, skipOut, freqOut, posOut, payloadOut);
-  }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/d852d5b0/src/Lucene.Net.Codecs/Sep/SepSkipListReader.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Sep/SepSkipListReader.cs b/src/Lucene.Net.Codecs/Sep/SepSkipListReader.cs
index aee9180..b194b67 100644
--- a/src/Lucene.Net.Codecs/Sep/SepSkipListReader.cs
+++ b/src/Lucene.Net.Codecs/Sep/SepSkipListReader.cs
@@ -1,6 +1,4 @@
-package codecs.sep;
-
-/*
+/*
  * 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.
@@ -17,193 +15,202 @@ package codecs.sep;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Arrays;
-
-import store.IndexInput;
-import codecs.MultiLevelSkipListReader;
-import index.FieldInfo.IndexOptions;
-
-/**
- * Implements the skip list reader for the default posting list format
- * that stores positions and payloads.
- *
- * @lucene.experimental
- */
-
-// TODO: rewrite this as recursive classes?
-class SepSkipListReader extends MultiLevelSkipListReader {
-  private bool currentFieldStoresPayloads;
-  private IntIndexInput.Index freqIndex[];
-  private IntIndexInput.Index docIndex[];
-  private IntIndexInput.Index posIndex[];
-  private long payloadPointer[];
-  private int payloadLength[];
-
-  private final IntIndexInput.Index lastFreqIndex;
-  private final IntIndexInput.Index lastDocIndex;
-  // TODO: -- make private again
-  final IntIndexInput.Index lastPosIndex;
-  
-  private long lastPayloadPointer;
-  private int lastPayloadLength;
-                           
-  SepSkipListReader(IndexInput skipStream,
-                    IntIndexInput freqIn,
-                    IntIndexInput docIn,
-                    IntIndexInput posIn,
-                    int maxSkipLevels,
-                    int skipInterval)
-     {
-    super(skipStream, maxSkipLevels, skipInterval);
-    if (freqIn != null) {
-      freqIndex = new IntIndexInput.Index[maxSkipLevels];
-    }
-    docIndex = new IntIndexInput.Index[maxSkipLevels];
-    if (posIn != null) {
-      posIndex = new IntIndexInput.Index[maxNumberOfSkipLevels];
-    }
-    for(int i=0;i<maxSkipLevels;i++) {
-      if (freqIn != null) {
-        freqIndex[i] = freqIn.index();
-      }
-      docIndex[i] = docIn.index();
-      if (posIn != null) {
-        posIndex[i] = posIn.index();
-      }
-    }
-    payloadPointer = new long[maxSkipLevels];
-    payloadLength = new int[maxSkipLevels];
-
-    if (freqIn != null) {
-      lastFreqIndex = freqIn.index();
-    } else {
-      lastFreqIndex = null;
-    }
-    lastDocIndex = docIn.index();
-    if (posIn != null) {
-      lastPosIndex = posIn.index();
-    } else {
-      lastPosIndex = null;
-    }
-  }
-  
-  IndexOptions indexOptions;
-
-  void setIndexOptions(IndexOptions v) {
-    indexOptions = v;
-  }
-
-  void init(long skipPointer,
-            IntIndexInput.Index docBaseIndex,
-            IntIndexInput.Index freqBaseIndex,
-            IntIndexInput.Index posBaseIndex,
-            long payloadBasePointer,
-            int df,
-            bool storesPayloads) {
-
-    super.init(skipPointer, df);
-    this.currentFieldStoresPayloads = storesPayloads;
-
-    lastPayloadPointer = payloadBasePointer;
-
-    for(int i=0;i<maxNumberOfSkipLevels;i++) {
-      docIndex[i].copyFrom(docBaseIndex);
-      if (freqIndex != null) {
-        freqIndex[i].copyFrom(freqBaseIndex);
-      }
-      if (posBaseIndex != null) {
-        posIndex[i].copyFrom(posBaseIndex);
-      }
-    }
-    Arrays.fill(payloadPointer, payloadBasePointer);
-    Arrays.fill(payloadLength, 0);
-  }
-
-  long getPayloadPointer() {
-    return lastPayloadPointer;
-  }
-  
-  /** Returns the payload length of the payload stored just before 
-   * the doc to which the last call of {@link MultiLevelSkipListReader#skipTo(int)} 
-   * has skipped.  */
-  int getPayloadLength() {
-    return lastPayloadLength;
-  }
-  
-  @Override
-  protected void seekChild(int level)  {
-    super.seekChild(level);
-    payloadPointer[level] = lastPayloadPointer;
-    payloadLength[level] = lastPayloadLength;
-  }
-  
-  @Override
-  protected void setLastSkipData(int level) {
-    super.setLastSkipData(level);
-
-    lastPayloadPointer = payloadPointer[level];
-    lastPayloadLength = payloadLength[level];
-    if (freqIndex != null) {
-      lastFreqIndex.copyFrom(freqIndex[level]);
-    }
-    lastDocIndex.copyFrom(docIndex[level]);
-    if (lastPosIndex != null) {
-      lastPosIndex.copyFrom(posIndex[level]);
+namespace Lucene.Net.Codecs.Sep
+{
+    using System.Diagnostics;
+    using Index;
+    using Store;
+    using Support;
+
+    /// <summary>
+    /// Implements the skip list reader for the default posting list format
+    /// that stores positions and payloads.
+    /// 
+    /// @lucene.experimental
+    /// </summary>
+
+    // TODO: rewrite this as recursive classes?
+    internal class SepSkipListReader : MultiLevelSkipListReader
+    {
+        private bool _currentFieldStoresPayloads;
+        private readonly IntIndexInputIndex[] _freqIndex;
+        private readonly IntIndexInputIndex[] _docIndex;
+        private readonly IntIndexInputIndex[] _posIndex;
+        private readonly long[] _payloadPointer;
+        private readonly int[] _payloadLength;
+
+        private readonly IntIndexInputIndex _lastFreqIndex;
+        private readonly IntIndexInputIndex _lastDocIndex;
+        private readonly IntIndexInputIndex _lastPosIndex;
+
+        private FieldInfo.IndexOptions _indexOptions;
+
+        private long _lastPayloadPointer;
+        private int _lastPayloadLength;
+
+        internal SepSkipListReader(IndexInput skipStream, IntIndexInput freqIn, IntIndexInput docIn, IntIndexInput posIn,
+            int maxSkipLevels, int skipInterval)
+            : base(skipStream, maxSkipLevels, skipInterval)
+        {
+            if (freqIn != null)
+                _freqIndex = new IntIndexInputIndex[maxSkipLevels];
+
+            _docIndex = new IntIndexInputIndex[maxSkipLevels];
+
+            if (posIn != null)
+                _posIndex = new IntIndexInputIndex[MaxNumberOfSkipLevels];
+
+            for (var i = 0; i < maxSkipLevels; i++)
+            {
+                if (freqIn != null)
+                    _freqIndex[i] = freqIn.Index();
+
+                _docIndex[i] = docIn.Index();
+
+                if (posIn != null)
+                    _posIndex[i] = posIn.Index();
+            }
+
+            _payloadPointer = new long[maxSkipLevels];
+            _payloadLength = new int[maxSkipLevels];
+
+            _lastFreqIndex = freqIn != null ? freqIn.Index() : null;
+            _lastDocIndex = docIn.Index();
+            _lastPosIndex = posIn != null ? posIn.Index() : null;
+        }
+
+
+        internal virtual FieldInfo.IndexOptions IndexOptions
+        {
+            set { _indexOptions = value; }
+        }
+
+        internal virtual void Init(long skipPointer, IntIndexInputIndex docBaseIndex, IntIndexInputIndex freqBaseIndex,
+            IntIndexInputIndex posBaseIndex, long payloadBasePointer, int df, bool storesPayloads)
+        {
+
+            base.Init(skipPointer, df);
+            _currentFieldStoresPayloads = storesPayloads;
+
+            _lastPayloadPointer = payloadBasePointer;
+
+            for (var i = 0; i < MaxNumberOfSkipLevels; i++)
+            {
+                _docIndex[i].CopyFrom(docBaseIndex);
+                if (_freqIndex != null)
+                    _freqIndex[i].CopyFrom(freqBaseIndex);
+
+                if (posBaseIndex != null)
+                    _posIndex[i].CopyFrom(posBaseIndex);
+            }
+            Arrays.Fill(_payloadPointer, payloadBasePointer);
+            Arrays.Fill(_payloadLength, 0);
+        }
+
+        internal virtual long PayloadPointer
+        {
+            get { return _lastPayloadPointer; }
+        }
+
+        /// <summary>
+        /// Returns the payload length of the payload stored just before 
+        /// the doc to which the last call of <seealso cref="MultiLevelSkipListReader#skipTo(int)"/> 
+        /// has skipped.  
+        /// </summary>
+        internal virtual int PayloadLength
+        {
+            get { return _lastPayloadLength; }
+        }
+
+        protected override void SeekChild(int level)
+        {
+            base.SeekChild(level);
+            _payloadPointer[level] = _lastPayloadPointer;
+            _payloadLength[level] = _lastPayloadLength;
+        }
+
+        protected override int LastSkipData
+        {
+            set
+            {
+                base.LastSkipData = value;
+
+                _lastPayloadPointer = _payloadPointer[value];
+                _lastPayloadLength = _payloadLength[value];
+                
+                if (_freqIndex != null)
+                    _lastFreqIndex.CopyFrom(_freqIndex[value]);
+                
+                _lastDocIndex.CopyFrom(_docIndex[value]);
+
+                if (_lastPosIndex != null)
+                    _lastPosIndex.CopyFrom(_posIndex[value]);
+
+                if (value <= 0) return;
+
+                if (_freqIndex != null)
+                    _freqIndex[value - 1].CopyFrom(_freqIndex[value]);
+                
+                _docIndex[value - 1].CopyFrom(_docIndex[value]);
+            
+                if (_posIndex != null)
+                    _posIndex[value - 1].CopyFrom(_posIndex[value]);
+                
+            }
+        }
+
+        internal virtual IntIndexInputIndex FreqIndex
+        {
+            get { return _lastFreqIndex; }
+        }
+
+        internal virtual IntIndexInputIndex PosIndex
+        {
+            get { return _lastPosIndex; }
+        }
+
+        internal virtual IntIndexInputIndex DocIndex
+        {
+            get { return _lastDocIndex; }
+        }
+
+        protected override int ReadSkipData(int level, IndexInput skipStream)
+        {
+            int delta;
+            Debug.Assert(_indexOptions == FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS ||
+                         !_currentFieldStoresPayloads);
+            if (_currentFieldStoresPayloads)
+            {
+                // the current field stores payloads.
+                // if the doc delta is odd then we have
+                // to read the current payload length
+                // because it differs from the length of the
+                // previous payload
+                delta = skipStream.ReadVInt();
+                if ((delta & 1) != 0)
+                {
+                    _payloadLength[level] = skipStream.ReadVInt();
+                }
+                delta = (int) ((uint) delta >> 1);
+            }
+            else
+            {
+                delta = skipStream.ReadVInt();
+            }
+
+            if (_indexOptions != FieldInfo.IndexOptions.DOCS_ONLY)
+                _freqIndex[level].Read(skipStream, false);
+            
+            _docIndex[level].Read(skipStream, false);
+            if (_indexOptions != FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) return delta;
+
+            _posIndex[level].Read(skipStream, false);
+            
+            if (_currentFieldStoresPayloads)
+                _payloadPointer[level] += skipStream.ReadVInt();
+            
+            return delta;
+        }
     }
 
-    if (level > 0) {
-      if (freqIndex != null) {
-        freqIndex[level-1].copyFrom(freqIndex[level]);
-      }
-      docIndex[level-1].copyFrom(docIndex[level]);
-      if (posIndex != null) {
-        posIndex[level-1].copyFrom(posIndex[level]);
-      }
-    }
-  }
-
-  IntIndexInput.Index getFreqIndex() {
-    return lastFreqIndex;
-  }
-
-  IntIndexInput.Index getPosIndex() {
-    return lastPosIndex;
-  }
-
-  IntIndexInput.Index getDocIndex() {
-    return lastDocIndex;
-  }
-
-  @Override
-  protected int readSkipData(int level, IndexInput skipStream)  {
-    int delta;
-    Debug.Assert( indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS || !currentFieldStoresPayloads;
-    if (currentFieldStoresPayloads) {
-      // the current field stores payloads.
-      // if the doc delta is odd then we have
-      // to read the current payload length
-      // because it differs from the length of the
-      // previous payload
-      delta = skipStream.readVInt();
-      if ((delta & 1) != 0) {
-        payloadLength[level] = skipStream.readVInt();
-      }
-      delta >>>= 1;
-    } else {
-      delta = skipStream.readVInt();
-    }
-    if (indexOptions != IndexOptions.DOCS_ONLY) {
-      freqIndex[level].read(skipStream, false);
-    }
-    docIndex[level].read(skipStream, false);
-    if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-      posIndex[level].read(skipStream, false);
-      if (currentFieldStoresPayloads) {
-        payloadPointer[level] += skipStream.readVInt();
-      }
-    }
-    
-    return delta;
-  }
-}
+}
\ No newline at end of file