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

[03/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/Pulsing/PulsingPostingsFormat.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Pulsing/PulsingPostingsFormat.cs b/src/Lucene.Net.Codecs/Pulsing/PulsingPostingsFormat.cs
new file mode 100644
index 0000000..552ecbc
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Pulsing/PulsingPostingsFormat.cs
@@ -0,0 +1,127 @@
+/*
+ * 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 System;
+    using System.Diagnostics;
+    using Lucene.Net.Index;
+    using Lucene.Net.Util;
+
+    /// <summary>
+    /// This postings format "inlines" the postings for terms that have
+    /// low docFreq.  It wraps another postings format, which is used for
+    /// writing the non-inlined terms.
+    /// @lucene.experimental 
+    /// </summary>
+    public abstract class PulsingPostingsFormat : PostingsFormat
+    {
+
+        private readonly int _freqCutoff;
+        private readonly int _minBlockSize;
+        private readonly int _maxBlockSize;
+        private readonly PostingsBaseFormat _wrappedPostingsBaseFormat;
+
+        public int FreqCutoff
+        {
+            get { return _freqCutoff; }
+        }
+
+        protected PulsingPostingsFormat(String name, PostingsBaseFormat wrappedPostingsBaseFormat, int freqCutoff) :
+            this(name, wrappedPostingsBaseFormat, freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
+            BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE)
+        {
+        }
+
+        /// <summary>Terms with freq <= freqCutoff are inlined into terms dict.</summary>
+        protected PulsingPostingsFormat(String name, PostingsBaseFormat wrappedPostingsBaseFormat, int freqCutoff,
+            int minBlockSize, int maxBlockSize) : base(name)
+        {
+            Debug.Debug.Assert((minBlockSize > 1);
+
+            _freqCutoff = freqCutoff;
+            _minBlockSize = minBlockSize;
+            _maxBlockSize = maxBlockSize;
+            _wrappedPostingsBaseFormat = wrappedPostingsBaseFormat;
+        }
+
+        public override String ToString()
+        {
+            return string.Format("{0} (freqCutoff={1}, minBlockSize={2}, maxBlockSize={3})", Name, _freqCutoff, _minBlockSize, _maxBlockSize);
+        }
+
+        public override FieldsConsumer FieldsConsumer(SegmentWriteState state)
+        {
+            PostingsWriterBase docsWriter = null;
+
+            // Terms that have <= freqCutoff number of docs are
+            // "pulsed" (inlined):
+            PostingsWriterBase pulsingWriter = null;
+
+            // Terms dict
+            bool success = false;
+            try
+            {
+                docsWriter = _wrappedPostingsBaseFormat.PostingsWriterBase(state);
+
+                // Terms that have <= freqCutoff number of docs are
+                // "pulsed" (inlined):
+                pulsingWriter = new PulsingPostingsWriter(state, _freqCutoff, docsWriter);
+                FieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter, _minBlockSize, _maxBlockSize);
+                success = true;
+                return ret;
+            }
+            finally
+            {
+                if (!success)
+                {
+                    IOUtils.CloseWhileHandlingException(docsWriter, pulsingWriter);
+                }
+            }
+        }
+
+        public override FieldsProducer FieldsProducer(SegmentReadState state)
+        {
+            PostingsReaderBase docsReader = null;
+            PostingsReaderBase pulsingReader = null;
+
+            bool success = false;
+            try
+            {
+                docsReader = _wrappedPostingsBaseFormat.PostingsReaderBase(state);
+                pulsingReader = new PulsingPostingsReader(state, docsReader);
+                FieldsProducer ret = new BlockTreeTermsReader(
+                    state.Directory, state.FieldInfos, state.SegmentInfo,
+                    pulsingReader,
+                    state.Context,
+                    state.SegmentSuffix,
+                    state.TermsIndexDivisor);
+                success = true;
+                return ret;
+            }
+            finally
+            {
+                if (!success)
+                {
+                    IOUtils.CloseWhileHandlingException(docsReader, pulsingReader);
+                }
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/src/Lucene.Net.Codecs/Pulsing/PulsingPostingsReader.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Pulsing/PulsingPostingsReader.cs b/src/Lucene.Net.Codecs/Pulsing/PulsingPostingsReader.cs
new file mode 100644
index 0000000..9f4599b
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Pulsing/PulsingPostingsReader.cs
@@ -0,0 +1,780 @@
+/*
+ * 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 System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Lucene.Net.Index;
+    using Lucene.Net.Store;
+    using Lucene.Net.Util;
+
+    /// <summary>
+    /// 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?
+    /// </summary>
+    public class PulsingPostingsReader : PostingsReaderBase
+    {
+
+        // Fallback reader for non-pulsed terms:
+        private readonly PostingsReaderBase _wrappedPostingsReader;
+        private readonly SegmentReadState segmentState;
+        private int maxPositions;
+        private int version;
+        private SortedDictionary<int, int> fields;
+
+        public PulsingPostingsReader(SegmentReadState state, PostingsReaderBase wrappedPostingsReader)
+        {
+            this._wrappedPostingsReader = wrappedPostingsReader;
+            this.segmentState = state;
+        }
+
+        public override void Init(IndexInput termsIn)
+        {
+            version = CodecUtil.CheckHeader(termsIn, PulsingPostingsWriter.CODEC,
+                PulsingPostingsWriter.VERSION_START,
+                PulsingPostingsWriter.VERSION_CURRENT);
+
+            maxPositions = termsIn.ReadVInt();
+            _wrappedPostingsReader.Init(termsIn);
+
+            if (_wrappedPostingsReader is PulsingPostingsReader || version < PulsingPostingsWriter.VERSION_META_ARRAY)
+            {
+                fields = null;
+            }
+            else
+            {
+                fields = new SortedDictionary<int, int>();
+                String summaryFileName = IndexFileNames.SegmentFileName(segmentState.SegmentInfo.Name,
+                    segmentState.SegmentSuffix, PulsingPostingsWriter.SUMMARY_EXTENSION);
+                IndexInput input = null;
+
+                try
+                {
+                    input =
+                        segmentState.Directory.OpenInput(summaryFileName, segmentState.Context);
+                    CodecUtil.CheckHeader(input,
+                        PulsingPostingsWriter.CODEC,
+                        version,
+                        PulsingPostingsWriter.VERSION_CURRENT);
+
+                    int numField = input.ReadVInt();
+                    for (int i = 0; i < numField; i++)
+                    {
+                        int fieldNum = input.ReadVInt();
+                        int longsSize = input.ReadVInt();
+                        fields.Add(fieldNum, longsSize);
+                    }
+                }
+                finally
+                {
+                    IOUtils.CloseWhileHandlingException(input);
+                }
+            }
+        }
+
+        public override BlockTermState NewTermState()
+        {
+            var state = new PulsingTermState {WrappedTermState = _wrappedPostingsReader.NewTermState()};
+            return state;
+        }
+
+        public override void DecodeTerm(long[] empty, DataInput input, FieldInfo fieldInfo, BlockTermState _termState,
+            bool absolute)
+        {
+            PulsingTermState termState = (PulsingTermState) _termState;
+
+            Debug.Debug.Assert((empty.Length == 0);
+            termState.Absolute = termState.Absolute || absolute;
+            // if we have positions, its total TF, otherwise its computed based on docFreq.
+            // TODO Double check this is right..
+            long count = FieldInfo.IndexOptions_e.DOCS_AND_FREQS_AND_POSITIONS.CompareTo(fieldInfo.IndexOptions) <= 0
+                ? termState.TotalTermFreq
+                : termState.DocFreq;
+            //System.out.println("  count=" + count + " threshold=" + maxPositions);
+
+            if (count <= maxPositions)
+            {
+                // Inlined into terms dict -- just read the byte[] blob in,
+                // but don't decode it now (we only decode when a DocsEnum
+                // or D&PEnum is pulled):
+                termState.PostingsSize = input.ReadVInt();
+                if (termState.Postings == null || termState.Postings.Length < termState.PostingsSize)
+                {
+                    termState.Postings = new byte[ArrayUtil.Oversize(termState.PostingsSize, 1)];
+                }
+                // TODO: sort of silly to copy from one big byte[]
+                // (the blob holding all inlined terms' blobs for
+                // current term block) into another byte[] (just the
+                // blob for this term)...
+                input.ReadBytes(termState.Postings, 0, termState.PostingsSize);
+                //System.out.println("  inlined bytes=" + termState.postingsSize);
+                termState.Absolute = termState.Absolute || absolute;
+            }
+            else
+            {
+                int longsSize = fields == null ? 0 : fields[fieldInfo.Number];
+                if (termState.Longs == null)
+                {
+                    termState.Longs = new long[longsSize];
+                }
+                for (int i = 0; i < longsSize; i++)
+                {
+                    termState.Longs[i] = input.ReadVLong();
+                }
+                termState.PostingsSize = -1;
+                termState.WrappedTermState.DocFreq = termState.DocFreq;
+                termState.WrappedTermState.TotalTermFreq = termState.TotalTermFreq;
+                _wrappedPostingsReader.DecodeTerm(termState.Longs, input, fieldInfo,
+                    termState.WrappedTermState,
+                    termState.Absolute);
+                termState.Absolute = false;
+            }
+        }
+
+        public override DocsEnum Docs(FieldInfo field, BlockTermState _termState, Bits liveDocs, DocsEnum reuse,
+            int flags)
+        {
+            PulsingTermState termState = (PulsingTermState) _termState;
+            if (termState.PostingsSize != -1)
+            {
+                PulsingDocsEnum postings;
+                if (reuse is PulsingDocsEnum)
+                {
+                    postings = (PulsingDocsEnum) reuse;
+                    if (!postings.CanReuse(field))
+                    {
+                        postings = new PulsingDocsEnum(field);
+                    }
+                }
+                else
+                {
+                    // the 'reuse' is actually the wrapped enum
+                    PulsingDocsEnum previous = (PulsingDocsEnum) GetOther(reuse);
+                    if (previous != null && previous.CanReuse(field))
+                    {
+                        postings = previous;
+                    }
+                    else
+                    {
+                        postings = new PulsingDocsEnum(field);
+                    }
+                }
+                if (reuse != postings)
+                {
+                    SetOther(postings, reuse); // postings.other = reuse
+                }
+                return postings.Reset(liveDocs, termState);
+            }
+            else
+            {
+                if (reuse is PulsingDocsEnum)
+                {
+                    DocsEnum wrapped = _wrappedPostingsReader.Docs(field, termState.WrappedTermState, liveDocs,
+                        GetOther(reuse), flags);
+                    SetOther(wrapped, reuse); // wrapped.other = reuse
+                    return wrapped;
+                }
+                else
+                {
+                    return _wrappedPostingsReader.Docs(field, termState.WrappedTermState, liveDocs, reuse, flags);
+                }
+            }
+        }
+
+        public override DocsAndPositionsEnum DocsAndPositions(FieldInfo field, BlockTermState _termState, Bits liveDocs,
+            DocsAndPositionsEnum reuse,
+            int flags)
+        {
+
+            PulsingTermState termState = (PulsingTermState) _termState;
+
+            if (termState.PostingsSize != -1)
+            {
+                PulsingDocsAndPositionsEnum postings;
+                if (reuse is PulsingDocsAndPositionsEnum)
+                {
+                    postings = (PulsingDocsAndPositionsEnum) reuse;
+                    if (!postings.CanReuse(field))
+                    {
+                        postings = new PulsingDocsAndPositionsEnum(field);
+                    }
+                }
+                else
+                {
+                    // the 'reuse' is actually the wrapped enum
+                    PulsingDocsAndPositionsEnum previous = (PulsingDocsAndPositionsEnum) GetOther(reuse);
+                    if (previous != null && previous.CanReuse(field))
+                    {
+                        postings = previous;
+                    }
+                    else
+                    {
+                        postings = new PulsingDocsAndPositionsEnum(field);
+                    }
+                }
+                if (reuse != postings)
+                {
+                    SetOther(postings, reuse); // postings.other = reuse 
+                }
+                return postings.reset(liveDocs, termState);
+            }
+            else
+            {
+                if (reuse is PulsingDocsAndPositionsEnum)
+                {
+                    DocsAndPositionsEnum wrapped = _wrappedPostingsReader.DocsAndPositions(field,
+                        termState.WrappedTermState,
+                        liveDocs, (DocsAndPositionsEnum) GetOther(reuse),
+                        flags);
+                    SetOther(wrapped, reuse); // wrapped.other = reuse
+                    return wrapped;
+                }
+                else
+                {
+                    return _wrappedPostingsReader.DocsAndPositions(field, termState.WrappedTermState, liveDocs, reuse,
+                        flags);
+                }
+            }
+        }
+
+        public override long RamBytesUsed()
+        {
+            return ((_wrappedPostingsReader != null) ? _wrappedPostingsReader.RamBytesUsed() : 0);
+        }
+
+        public override void CheckIntegrity()
+        {
+            _wrappedPostingsReader.CheckIntegrity();
+        }
+
+        protected override void Dispose(bool disposing)
+        {
+            if (!disposing)
+                _wrappedPostingsReader.Dispose();
+        }
+        
+        /// <summary>
+        /// for a docsenum, gets the 'other' reused enum.
+        /// Example: Pulsing(Standard).
+        /// when doing a term range query you are switching back and forth
+        /// between Pulsing and Standard
+        ///  
+        /// The way the reuse works is that Pulsing.other = Standard and
+        /// Standard.other = Pulsing.
+        /// </summary>
+        private DocsEnum GetOther(DocsEnum de)
+        {
+            if (de == null)
+            {
+                return null;
+            }
+            else
+            {
+                AttributeSource atts = de.Attributes();
+                return atts.AddAttribute(PulsingEnumAttribute.Enums().get(this);
+            }
+        }
+
+        /// <summary>
+        /// for a docsenum, sets the 'other' reused enum.
+        /// see GetOther for an example.
+        /// </summary>
+        private DocsEnum SetOther(DocsEnum de, DocsEnum other)
+        {
+            AttributeSource atts = de.Attributes();
+            return atts.AddAttribute(PulsingEnumAttributeImpl.Enums().put(this, other));
+        }
+
+        ///<summary>
+        /// A per-docsenum attribute that stores additional reuse information
+        /// so that pulsing enums can keep a reference to their wrapped enums,
+        /// and vice versa. this way we can always reuse.
+        /// 
+        /// @lucene.internal 
+        /// </summary>
+        public interface IPulsingEnumAttribute : IAttribute
+        {
+            Dictionary<PulsingPostingsReader, DocsEnum> Enums();
+        }
+
+        internal class PulsingTermState : BlockTermState
+        {
+            public bool Absolute { get; set; }
+            public long[] Longs { get; set; }
+            public byte[] Postings { get; set; }
+            public int PostingsSize { get; set; } // -1 if this term was not inlined
+            public BlockTermState WrappedTermState { get; set; }
+
+            public override object Clone()
+            {
+                PulsingTermState clone = (PulsingTermState) base.Clone();
+                if (PostingsSize != -1)
+                {
+                    clone.Postings = new byte[PostingsSize];
+                    Array.Copy(Postings, 0, clone.Postings, 0, PostingsSize);
+                }
+                else
+                {
+                    Debug.Debug.Assert((WrappedTermState != null);
+                    clone.WrappedTermState = (BlockTermState) WrappedTermState.Clone();
+                    clone.Absolute = Absolute;
+                    if (Longs != null)
+                    {
+                        clone.Longs = new long[Longs.Length];
+                        Array.Copy(Longs, 0, clone.Longs, 0, Longs.Length);
+                    }
+                }
+                return clone;
+            }
+
+            public override void CopyFrom(TermState other)
+            {
+                base.CopyFrom(other);
+                var _other = (PulsingTermState) other;
+                PostingsSize = _other.PostingsSize;
+                if (_other.PostingsSize != -1)
+                {
+                    if (Postings == null || Postings.Length < _other.PostingsSize)
+                    {
+                        Postings = new byte[ArrayUtil.Oversize(_other.PostingsSize, 1)];
+                    }
+                    System.Array.Copy(_other.Postings, 0, Postings, 0, _other.PostingsSize);
+                }
+                else
+                {
+                    WrappedTermState.CopyFrom(_other.WrappedTermState);
+                }
+            }
+
+            public override String ToString()
+            {
+                if (PostingsSize == -1)
+                {
+                    return "PulsingTermState: not inlined: wrapped=" + WrappedTermState;
+                }
+                else
+                {
+                    return "PulsingTermState: inlined size=" + PostingsSize + " " + base.ToString();
+                }
+            }
+        }
+
+        internal class PulsingDocsEnum : DocsEnum
+        {
+            private byte[] postingsBytes;
+            private readonly ByteArrayDataInput postings = new ByteArrayDataInput();
+            private readonly FieldInfo.IndexOptions_e? indexOptions;
+            private readonly bool storePayloads;
+            private readonly bool storeOffsets;
+            private Bits liveDocs;
+
+            private int docID = -1;
+            private int accum;
+            private int freq;
+            private int payloadLength;
+            private int cost;
+
+            public PulsingDocsEnum(FieldInfo fieldInfo)
+            {
+                indexOptions = fieldInfo.IndexOptions;
+                storePayloads = fieldInfo.HasPayloads();
+                storeOffsets = indexOptions.Value.CompareTo(FieldInfo.IndexOptions_e.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+            }
+
+            public PulsingDocsEnum Reset(Bits liveDocs, PulsingTermState termState)
+            {
+                Debug.Debug.Assert((termState.PostingsSize != -1);
+
+                // Must make a copy of termState's byte[] so that if
+                // app does TermsEnum.next(), this DocsEnum is not affected
+                if (postingsBytes == null)
+                {
+                    postingsBytes = new byte[termState.PostingsSize];
+                }
+                else if (postingsBytes.Length < termState.PostingsSize)
+                {
+                    postingsBytes = ArrayUtil.Grow(postingsBytes, termState.PostingsSize);
+                }
+                System.Array.Copy(termState.Postings, 0, postingsBytes, 0, termState.PostingsSize);
+                postings.Reset(postingsBytes, 0, termState.PostingsSize);
+                docID = -1;
+                accum = 0;
+                freq = 1;
+                cost = termState.DocFreq;
+                payloadLength = 0;
+                this.liveDocs = liveDocs;
+                return this;
+            }
+
+            public bool CanReuse(FieldInfo fieldInfo)
+            {
+                return indexOptions == fieldInfo.IndexOptions && storePayloads == fieldInfo.HasPayloads();
+            }
+
+            public override int DocID()
+            {
+                return docID;
+            }
+
+            public override int NextDoc()
+            {
+                //System.out.println("PR nextDoc this= "+ this);
+                while (true)
+                {
+                    if (postings.Eof())
+                    {
+                        return docID = NO_MORE_DOCS;
+                    }
+
+                    int code = postings.ReadVInt();
+                    if (indexOptions == FieldInfo.IndexOptions_e.DOCS_ONLY)
+                    {
+                        accum += code;
+                    }
+                    else
+                    {
+                        accum += (int)((uint)code >> 1); ; // shift off low bit
+                        if ((code & 1) != 0)
+                        {
+                            // if low bit is set
+                            freq = 1; // freq is one
+                        }
+                        else
+                        {
+                            freq = postings.ReadVInt(); // else read freq
+                        }
+
+                        if (indexOptions.Value.CompareTo(FieldInfo.IndexOptions_e.DOCS_AND_FREQS_AND_POSITIONS) >= 0)
+                        {
+                            // Skip positions
+                            if (storePayloads)
+                            {
+                                for (int pos = 0; pos < freq; pos++)
+                                {
+                                    int posCode = postings.ReadVInt();
+                                    if ((posCode & 1) != 0)
+                                    {
+                                        payloadLength = postings.ReadVInt();
+                                    }
+                                    if (storeOffsets && (postings.ReadVInt() & 1) != 0)
+                                    {
+                                        // new offset length
+                                        postings.ReadVInt();
+                                    }
+                                    if (payloadLength != 0)
+                                    {
+                                        postings.SkipBytes(payloadLength);
+                                    }
+                                }
+                            }
+                            else
+                            {
+                                for (int pos = 0; pos < freq; pos++)
+                                {
+                                    // TODO: skipVInt
+                                    postings.ReadVInt();
+                                    if (storeOffsets && (postings.ReadVInt() & 1) != 0)
+                                    {
+                                        // new offset length
+                                        postings.ReadVInt();
+                                    }
+                                }
+                            }
+                        }
+                    }
+
+                    if (liveDocs == null || liveDocs.Get(accum))
+                    {
+                        return (docID = accum);
+                    }
+
+                }
+            }
+
+            public override int Advance(int target)
+            {
+                return docID = SlowAdvance(target);
+            }
+
+            public override long Cost()
+            {
+                return cost;
+            }
+
+            public override int Freq()
+            {
+                return freq;
+            }
+        }
+
+        internal class PulsingDocsAndPositionsEnum : DocsAndPositionsEnum
+        {
+            private byte[] postingsBytes;
+            private readonly ByteArrayDataInput postings = new ByteArrayDataInput();
+            private readonly bool storePayloads;
+            private readonly bool storeOffsets;
+            // note: we could actually reuse across different options, if we passed this to reset()
+            // and re-init'ed storeOffsets accordingly (made it non-final)
+            private readonly FieldInfo.IndexOptions_e? indexOptions;
+
+            private Bits liveDocs;
+            private int docID = -1;
+            private int accum;
+            private int freq;
+            private int posPending;
+            private int position;
+            private int payloadLength;
+            private BytesRef payload;
+            private int startOffset;
+            private int offsetLength;
+
+            private bool payloadRetrieved;
+            private int cost;
+
+            public PulsingDocsAndPositionsEnum(FieldInfo fieldInfo)
+            {
+                indexOptions = fieldInfo.IndexOptions;
+                storePayloads = fieldInfo.HasPayloads();
+                storeOffsets =
+                    indexOptions.Value.CompareTo(FieldInfo.IndexOptions_e.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+            }
+
+            public PulsingDocsAndPositionsEnum reset(Bits liveDocs, PulsingTermState termState)
+            {
+                Debug.Debug.Assert((termState.PostingsSize != -1);
+
+                if (postingsBytes == null)
+                {
+                    postingsBytes = new byte[termState.PostingsSize];
+                }
+                else if (postingsBytes.Length < termState.PostingsSize)
+                {
+                    postingsBytes = ArrayUtil.Grow(postingsBytes, termState.PostingsSize);
+                }
+
+                System.Array.Copy(termState.Postings, 0, postingsBytes, 0, termState.PostingsSize);
+                postings.Reset(postingsBytes, 0, termState.PostingsSize);
+                this.liveDocs = liveDocs;
+                payloadLength = 0;
+                posPending = 0;
+                docID = -1;
+                accum = 0;
+                cost = termState.DocFreq;
+                startOffset = storeOffsets ? 0 : -1; // always return -1 if no offsets are stored
+                offsetLength = 0;
+                //System.out.println("PR d&p reset storesPayloads=" + storePayloads + " bytes=" + bytes.length + " this=" + this);
+                return this;
+            }
+
+            public bool CanReuse(FieldInfo fieldInfo)
+            {
+                return indexOptions == fieldInfo.IndexOptions && storePayloads == fieldInfo.HasPayloads();
+            }
+
+            public override int NextDoc()
+            {
+
+                while (true)
+                {
+
+                    SkipPositions();
+
+                    if (postings.Eof())
+                    {
+                        return docID = NO_MORE_DOCS;
+                    }
+
+                    int code = postings.ReadVInt();
+                    accum += (int)((uint)code >> 1); // shift off low bit 
+                    if ((code & 1) != 0)
+                    {
+                        // if low bit is set
+                        freq = 1; // freq is one
+                    }
+                    else
+                    {
+                        freq = postings.ReadVInt(); // else read freq
+                    }
+                    posPending = freq;
+                    startOffset = storeOffsets ? 0 : -1; // always return -1 if no offsets are stored
+
+                    if (liveDocs == null || liveDocs.Get(accum))
+                    {
+                        position = 0;
+                        return (docID = accum);
+                    }
+                }
+            }
+
+            public override int Freq()
+            {
+                return freq;
+            }
+
+            public override int DocID()
+            {
+                return docID;
+            }
+
+            public override int Advance(int target)
+            {
+                return docID = SlowAdvance(target);
+            }
+
+            public override int NextPosition()
+            {
+                Debug.Debug.Assert((posPending > 0);
+
+                posPending--;
+
+                if (storePayloads)
+                {
+                    if (!payloadRetrieved)
+                    {
+                        postings.SkipBytes(payloadLength);
+                    }
+                    int code = postings.ReadVInt();
+                    if ((code & 1) != 0)
+                    {
+                        payloadLength = postings.ReadVInt();
+                    }
+                    position += (int)((uint)code >> 1);
+                    payloadRetrieved = false;
+                }
+                else
+                {
+                    position += postings.ReadVInt();
+                }
+
+                if (storeOffsets)
+                {
+                    int offsetCode = postings.ReadVInt();
+                    if ((offsetCode & 1) != 0)
+                    {
+                        // new offset length
+                        offsetLength = postings.ReadVInt();
+                    }
+                    startOffset += (int)((uint)offsetCode >> 1);
+                }
+
+                return position;
+            }
+
+            public override int StartOffset()
+            {
+                return startOffset;
+            }
+
+            public override int EndOffset()
+            {
+                return startOffset + offsetLength;
+            }
+
+            public override BytesRef Payload
+            {
+                get
+                {
+                    if (payloadRetrieved)
+                    {
+                        return payload;
+                    }
+                    else if (storePayloads && payloadLength > 0)
+                    {
+                        payloadRetrieved = true;
+                        if (payload == null)
+                        {
+                            payload = new BytesRef(payloadLength);
+                        }
+                        else
+                        {
+                            payload.Grow(payloadLength);
+                        }
+                        postings.ReadBytes(payload.Bytes, 0, payloadLength);
+                        payload.Length = payloadLength;
+                        return payload;
+                    }
+                    else
+                    {
+                        return null;
+                    }
+                }
+            }
+
+            private void SkipPositions()
+            {
+                while (posPending != 0)
+                {
+                    NextPosition();
+                }
+                if (storePayloads && !payloadRetrieved)
+                {
+                    postings.SkipBytes(payloadLength);
+                    payloadRetrieved = true;
+                }
+            }
+            
+            public override long Cost()
+            {
+                return cost;
+            }
+        }
+        
+        /// <summary>
+        /// Implementation of {@link PulsingEnumAttribute} for reuse of
+        /// wrapped postings readers underneath pulsing.
+        /// 
+        /// @lucene.internal
+        /// </summary>
+        internal sealed class PulsingEnumAttributeImpl : AttributeImpl, IPulsingEnumAttribute
+        {
+            // we could store 'other', but what if someone 'chained' multiple postings readers,
+            // this could cause problems?
+            // TODO: we should consider nuking this map and just making it so if you do this,
+            // you don't reuse? and maybe pulsingPostingsReader should throw an exc if it wraps
+            // another pulsing, because this is just stupid and wasteful. 
+            // we still have to be careful in case someone does Pulsing(Stomping(Pulsing(...
+            private readonly Dictionary<PulsingPostingsReader, DocsEnum> _enums = new Dictionary<PulsingPostingsReader, DocsEnum>();
+
+            public Dictionary<PulsingPostingsReader, DocsEnum> Enums()
+            {
+                return _enums;
+            }
+            public override void Clear()
+            {
+                // our state is per-docsenum, so this makes no sense.
+                // its best not to clear, in case a wrapped enum has a per-doc attribute or something
+                // and is calling clearAttributes(), so they don't nuke the reuse information!
+            }
+
+            public override void CopyTo(AttributeImpl target)
+            {
+                // this makes no sense for us, because our state is per-docsenum.
+                // we don't want to copy any stuff over to another docsenum ever!
+            }
+
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/src/Lucene.Net.Codecs/Pulsing/PulsingPostingsWriter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Pulsing/PulsingPostingsWriter.cs b/src/Lucene.Net.Codecs/Pulsing/PulsingPostingsWriter.cs
new file mode 100644
index 0000000..528f8de
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Pulsing/PulsingPostingsWriter.cs
@@ -0,0 +1,511 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+using Lucene.Net.Index;
+using Lucene.Net.Store;
+using Lucene.Net.Util;
+
+namespace Lucene.Net.Codecs.Pulsing
+{
+
+    /// <summary>
+    /// TODO: we now inline based on total TF of the term,
+    /// but it might be better to inline by "net bytes used"
+    /// so that a term that has only 1 posting but a huge
+    /// payload would not be inlined.  Though this is
+    /// presumably rare in practice...
+    /// 
+    /// Writer for the pulsing format. 
+    ///
+    /// Wraps another postings implementation and decides 
+    /// (based on total number of occurrences), whether a terms 
+    /// postings should be inlined into the term dictionary,
+    /// or passed through to the wrapped writer.
+    ///
+    /// @lucene.experimental
+    /// </summary>
+    public sealed class PulsingPostingsWriter : PostingsWriterBase
+    {
+
+        internal static readonly String CODEC = "PulsedPostingsWriter";
+        internal static readonly String SUMMARY_EXTENSION = "smy";         // recording field summary
+        
+        // To add a new version, increment from the last one, and
+        // change VERSION_CURRENT to point to your new version:
+        internal static readonly int VERSION_START = 0;
+        internal static readonly int VERSION_META_ARRAY = 1;
+        internal static readonly int VERSION_CURRENT = VERSION_META_ARRAY;
+
+        private SegmentWriteState segmentState;
+        private IndexOutput termsOut;
+        private List<FieldMetaData> fields;
+        private FieldInfo.IndexOptions_e? indexOptions;
+        private bool storePayloads;
+
+        // information for wrapped PF, in current field
+        private int longsSize;
+        private long[] longs;
+        private bool absolute;
+
+        private class PulsingTermState : BlockTermState
+        {
+            internal byte[] bytes;
+            internal BlockTermState wrappedState;
+
+            public override String ToString()
+            {
+                if (bytes != null)
+                {
+                    return "inlined";
+                }
+                return "not inlined wrapped=" + wrappedState;
+            }
+        }
+
+        // one entry per position
+        private Position[] pending;
+        private int pendingCount = 0;   // -1 once we've hit too many positions
+        private Position currentDoc;    // first Position entry of current doc
+
+        private sealed class Position
+        {
+            internal BytesRef payload;
+            internal int termFreq; // only incremented on first position for a given doc
+            internal int pos;
+            internal int docID;
+            internal int startOffset;
+            internal int endOffset;
+        }
+
+        private class FieldMetaData
+        {
+            public int FieldNumber { get; private set; }
+            public int LongsSize { get; private set; }
+
+            public FieldMetaData(int number, int size)
+            {
+                FieldNumber = number;
+                LongsSize = size;
+            }
+        }
+
+        // TODO: -- lazy init this?  ie, if every single term
+        // was inlined (eg for a "primary key" field) then we
+        // never need to use this fallback?  Fallback writer for
+        // non-inlined terms:
+        private readonly PostingsWriterBase _wrappedPostingsWriter;
+
+        /// <summary>
+        /// If the total number of positions (summed across all docs
+        /// for this term) is <= maxPositions, then the postings are
+        /// inlined into terms dict
+        /// </summary>
+        public PulsingPostingsWriter(SegmentWriteState state, int maxPositions, PostingsWriterBase wrappedPostingsWriter)
+        {
+
+            pending = new Position[maxPositions];
+            for (int i = 0; i < maxPositions; i++)
+            {
+                pending[i] = new Position();
+            }
+            fields = new List<FieldMetaData>();
+
+            // We simply wrap another postings writer, but only call
+            // on it when tot positions is >= the cutoff:
+            this._wrappedPostingsWriter = wrappedPostingsWriter;
+            this.segmentState = state;
+        }
+
+        public override void Init(IndexOutput termsOut)
+        {
+            this.termsOut = termsOut;
+            CodecUtil.WriteHeader(termsOut, CODEC, VERSION_CURRENT);
+            termsOut.WriteVInt(pending.Length); // encode maxPositions in header
+            _wrappedPostingsWriter.Init(termsOut);
+        }
+
+        public override BlockTermState NewTermState()
+        {
+            PulsingTermState state = new PulsingTermState();
+            state.wrappedState = _wrappedPostingsWriter.NewTermState();
+            return state;
+        }
+
+        public override void StartTerm()
+        {
+            Debug.Debug.Assert((pendingCount == 0);
+        }
+
+        /// <summary>
+        /// TODO: -- should we NOT reuse across fields?  would
+        /// be cleaner
+        /// Currently, this instance is re-used across fields, so
+        /// our parent calls setField whenever the field changes
+        /// </summary>
+        /// <param name="fieldInfo"></param>
+        /// <returns></returns>
+        public override int SetField(FieldInfo fieldInfo)
+        {
+            this.indexOptions = fieldInfo.IndexOptions;
+            storePayloads = fieldInfo.HasPayloads();
+            absolute = false;
+            longsSize = _wrappedPostingsWriter.SetField(fieldInfo);
+            longs = new long[longsSize];
+            fields.Add(new FieldMetaData(fieldInfo.Number, longsSize));
+            return 0;
+        }
+
+        public override void StartDoc(int docID, int termDocFreq)
+        {
+            Debug.Debug.Assert((docID >= 0, "Got DocID=" + docID);
+
+            if (pendingCount == pending.Length)
+            {
+                push();
+                _wrappedPostingsWriter.FinishDoc();
+            }
+
+            if (pendingCount != -1)
+            {
+                Debug.Debug.Assert((pendingCount < pending.Length);
+                currentDoc = pending[pendingCount];
+                currentDoc.docID = docID;
+                if (indexOptions == FieldInfo.IndexOptions_e.DOCS_ONLY)
+                {
+                    pendingCount++;
+                }
+                else if (indexOptions == FieldInfo.IndexOptions_e.DOCS_AND_FREQS)
+                {
+                    pendingCount++;
+                    currentDoc.termFreq = termDocFreq;
+                }
+                else
+                {
+                    currentDoc.termFreq = termDocFreq;
+                }
+            }
+            else
+            {
+                // We've already seen too many docs for this term --
+                // just forward to our fallback writer
+                _wrappedPostingsWriter.StartDoc(docID, termDocFreq);
+            }
+        }
+
+        public override void AddPosition(int position, BytesRef payload, int startOffset, int endOffset)
+        {
+
+            if (pendingCount == pending.Length)
+            {
+                push();
+            }
+
+            if (pendingCount == -1)
+            {
+                // We've already seen too many docs for this term --
+                // just forward to our fallback writer
+                _wrappedPostingsWriter.AddPosition(position, payload, startOffset, endOffset);
+            }
+            else
+            {
+                // buffer up
+                Position pos = pending[pendingCount++];
+                pos.pos = position;
+                pos.startOffset = startOffset;
+                pos.endOffset = endOffset;
+                pos.docID = currentDoc.docID;
+                if (payload != null && payload.Length > 0)
+                {
+                    if (pos.payload == null)
+                    {
+                        pos.payload = BytesRef.DeepCopyOf(payload);
+                    }
+                    else
+                    {
+                        pos.payload.CopyBytes(payload);
+                    }
+                }
+                else if (pos.payload != null)
+                {
+                    pos.payload.Length = 0;
+                }
+            }
+        }
+
+        public override void FinishDoc()
+        {
+            if (pendingCount == -1)
+            {
+                _wrappedPostingsWriter.FinishDoc();
+            }
+        }
+
+        private readonly RAMOutputStream buffer = new RAMOutputStream();
+
+        /// <summary>
+        /// Called when we are done adding docs to this term
+        /// </summary>
+        /// <param name="_state"></param>
+        public override void FinishTerm(BlockTermState _state)
+        {
+            PulsingTermState state = (PulsingTermState) _state;
+
+            Debug.Debug.Assert((pendingCount > 0 || pendingCount == -1);
+
+            if (pendingCount == -1)
+            {
+                state.wrappedState.DocFreq = state.DocFreq;
+                state.wrappedState.TotalTermFreq = state.TotalTermFreq;
+                state.bytes = null;
+                _wrappedPostingsWriter.FinishTerm(state.wrappedState);
+            }
+            else
+            {
+                // There were few enough total occurrences for this
+                // term, so we fully inline our postings data into
+                // terms dict, now:
+
+                // TODO: it'd be better to share this encoding logic
+                // in some inner codec that knows how to write a
+                // single doc / single position, etc.  This way if a
+                // given codec wants to store other interesting
+                // stuff, it could use this pulsing codec to do so
+
+                if (indexOptions.Value.CompareTo(FieldInfo.IndexOptions_e.DOCS_AND_FREQS_AND_POSITIONS) >= 0)
+                {
+                    int lastDocID = 0;
+                    int pendingIDX = 0;
+                    int lastPayloadLength = -1;
+                    int lastOffsetLength = -1;
+                    while (pendingIDX < pendingCount)
+                    {
+                        Position doc = pending[pendingIDX];
+
+                        int delta = doc.docID - lastDocID;
+                        lastDocID = doc.docID;
+
+                        // if (DEBUG) System.out.println("  write doc=" + doc.docID + " freq=" + doc.termFreq);
+
+                        if (doc.termFreq == 1)
+                        {
+                            buffer.WriteVInt((delta << 1) | 1);
+                        }
+                        else
+                        {
+                            buffer.WriteVInt(delta << 1);
+                            buffer.WriteVInt(doc.termFreq);
+                        }
+
+                        int lastPos = 0;
+                        int lastOffset = 0;
+                        for (int posIDX = 0; posIDX < doc.termFreq; posIDX++)
+                        {
+                            Position pos = pending[pendingIDX++];
+                            Debug.Debug.Assert((pos.docID == doc.docID);
+                            int posDelta = pos.pos - lastPos;
+                            lastPos = pos.pos;
+                            
+                            int payloadLength = pos.payload == null ? 0 : pos.payload.Length;
+                            if (storePayloads)
+                            {
+                                if (payloadLength != lastPayloadLength)
+                                {
+                                    buffer.WriteVInt((posDelta << 1) | 1);
+                                    buffer.WriteVInt(payloadLength);
+                                    lastPayloadLength = payloadLength;
+                                }
+                                else
+                                {
+                                    buffer.WriteVInt(posDelta << 1);
+                                }
+                            }
+                            else
+                            {
+                                buffer.WriteVInt(posDelta);
+                            }
+
+                            if (indexOptions.Value.CompareTo(FieldInfo.IndexOptions_e.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0)
+                            {
+                                //System.out.println("write=" + pos.startOffset + "," + pos.endOffset);
+                                int offsetDelta = pos.startOffset - lastOffset;
+                                int offsetLength = pos.endOffset - pos.startOffset;
+                                if (offsetLength != lastOffsetLength)
+                                {
+                                    buffer.WriteVInt(offsetDelta << 1 | 1);
+                                    buffer.WriteVInt(offsetLength);
+                                }
+                                else
+                                {
+                                    buffer.WriteVInt(offsetDelta << 1);
+                                }
+                                lastOffset = pos.startOffset;
+                                lastOffsetLength = offsetLength;
+                            }
+
+                            if (payloadLength > 0)
+                            {
+                                Debug.Debug.Assert((storePayloads);
+                                buffer.WriteBytes(pos.payload.Bytes, 0, pos.payload.Length);
+                            }
+                        }
+                    }
+                }
+                else if (indexOptions == FieldInfo.IndexOptions_e.DOCS_AND_FREQS)
+                {
+                    int lastDocID = 0;
+                    for (int posIDX = 0; posIDX < pendingCount; posIDX++)
+                    {
+                        Position doc = pending[posIDX];
+                        int delta = doc.docID - lastDocID;
+                        Debug.Debug.Assert((doc.termFreq != 0);
+
+                        if (doc.termFreq == 1)
+                        {
+                            buffer.WriteVInt((delta << 1) | 1);
+                        }
+                        else
+                        {
+                            buffer.WriteVInt(delta << 1);
+                            buffer.WriteVInt(doc.termFreq);
+                        }
+                        lastDocID = doc.docID;
+                    }
+                }
+                else if (indexOptions == FieldInfo.IndexOptions_e.DOCS_ONLY)
+                {
+                    int lastDocID = 0;
+                    for (int posIDX = 0; posIDX < pendingCount; posIDX++)
+                    {
+                        Position doc = pending[posIDX];
+                        buffer.WriteVInt(doc.docID - lastDocID);
+                        lastDocID = doc.docID;
+                    }
+                }
+
+                state.bytes = new byte[(int) buffer.FilePointer];
+                buffer.WriteTo((sbyte[])(Array)state.bytes, 0);
+                buffer.Reset();
+            }
+            pendingCount = 0;
+        }
+
+        public override void EncodeTerm(long[] empty, DataOutput output, FieldInfo fieldInfo, BlockTermState _state,
+            bool absolute)
+        {
+            PulsingTermState state = (PulsingTermState) _state;
+            Debug.Debug.Assert((empty.Length == 0);
+            this.absolute = this.absolute || absolute;
+            if (state.bytes == null)
+            {
+                _wrappedPostingsWriter.EncodeTerm(longs, buffer, fieldInfo, state.wrappedState, this.absolute);
+                for (int i = 0; i < longsSize; i++)
+                {
+                    output.WriteVLong(longs[i]);
+                }
+                buffer.WriteTo(output);
+                buffer.Reset();
+                this.absolute = false;
+            }
+            else
+            {
+                output.WriteVInt(state.bytes.Length);
+                output.WriteBytes(state.bytes, 0, state.bytes.Length);
+                this.absolute = this.absolute || absolute;
+            }
+        }
+
+        protected override void Dispose(bool disposing)
+        {
+            _wrappedPostingsWriter.Dispose();
+
+            if (_wrappedPostingsWriter is PulsingPostingsWriter ||
+                VERSION_CURRENT < VERSION_META_ARRAY)
+            {
+                return;
+            }
+
+            String summaryFileName = IndexFileNames.SegmentFileName(segmentState.SegmentInfo.Name,
+                segmentState.SegmentSuffix, SUMMARY_EXTENSION);
+            IndexOutput output = null;
+            try
+            {
+                output =
+                    segmentState.Directory.CreateOutput(summaryFileName, segmentState.Context);
+                CodecUtil.WriteHeader(output, CODEC, VERSION_CURRENT);
+                output.WriteVInt(fields.Count);
+                foreach (FieldMetaData field in fields)
+                {
+                    output.WriteVInt(field.FieldNumber);
+                    output.WriteVInt(field.LongsSize);
+                }
+                output.Dispose();
+            }
+            finally
+            {
+                IOUtils.CloseWhileHandlingException(output);
+            }
+        }
+
+        // Pushes pending positions to the wrapped codec
+        private void push()
+        {
+            // if (DEBUG) System.out.println("PW now push @ " + pendingCount + " wrapped=" + wrappedPostingsWriter);
+            Debug.Debug.Assert((pendingCount == pending.Length);
+
+            _wrappedPostingsWriter.StartTerm();
+
+            // Flush all buffered docs
+            if (indexOptions.Value.CompareTo(FieldInfo.IndexOptions_e.DOCS_AND_FREQS_AND_POSITIONS) >= 0)
+            {
+                Position doc = null;
+
+                foreach(Position pos in pending)
+                {
+                    if (doc == null)
+                    {
+                        doc = pos;
+                        // if (DEBUG) System.out.println("PW: wrapped.startDoc docID=" + doc.docID + " tf=" + doc.termFreq);
+                        _wrappedPostingsWriter.StartDoc(doc.docID, doc.termFreq);
+                    }
+                    else if (doc.docID != pos.docID)
+                    {
+                        Debug.Debug.Assert((pos.docID > doc.docID);
+                        // if (DEBUG) System.out.println("PW: wrapped.finishDoc");
+                        _wrappedPostingsWriter.FinishDoc();
+                        doc = pos;
+                        // if (DEBUG) System.out.println("PW: wrapped.startDoc docID=" + doc.docID + " tf=" + doc.termFreq);
+                        _wrappedPostingsWriter.StartDoc(doc.docID, doc.termFreq);
+                    }
+                    // if (DEBUG) System.out.println("PW:   wrapped.addPos pos=" + pos.pos);
+                    _wrappedPostingsWriter.AddPosition(pos.pos, pos.payload, pos.startOffset, pos.endOffset);
+                }
+                //wrappedPostingsWriter.finishDoc();
+            }
+            else
+            {
+                foreach(Position doc in pending)
+                {
+                    _wrappedPostingsWriter.StartDoc(doc.docID, indexOptions == FieldInfo.IndexOptions_e.DOCS_ONLY ? 0 : doc.termFreq);
+                }
+            }
+            pendingCount = -1;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/src/Lucene.Net.Codecs/Sep/IntIndexInput.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Sep/IntIndexInput.cs b/src/Lucene.Net.Codecs/Sep/IntIndexInput.cs
new file mode 100644
index 0000000..ebe0d27
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Sep/IntIndexInput.cs
@@ -0,0 +1,59 @@
+package org.apache.lucene.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.
+ * 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.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.store.DataInput;
+
+/** Defines basic API for writing ints to an IndexOutput.
+ *  IntBlockCodec interacts with this API. @see
+ *  IntBlockReader
+ *
+ * @lucene.experimental */
+public abstract class IntIndexInput implements Closeable {
+
+  public abstract Reader reader() ;
+
+  @Override
+  public abstract void close() ;
+
+  public abstract Index index() ;
+  
+  /** Records a single skip-point in the {@link IntIndexInput.Reader}. */
+  public abstract static class Index {
+
+    public abstract void read(DataInput indexIn, bool absolute) ;
+
+    /** Seeks primary stream to the last read offset */
+    public abstract void seek(IntIndexInput.Reader stream) ;
+
+    public abstract void copyFrom(Index other);
+    
+    @Override
+    public abstract Index clone();
+  }
+
+  /** Reads int values. */
+  public abstract static class Reader {
+
+    /** Reads next single int */
+    public abstract int next() ;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/src/Lucene.Net.Codecs/Sep/IntIndexOutput.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Sep/IntIndexOutput.cs b/src/Lucene.Net.Codecs/Sep/IntIndexOutput.cs
new file mode 100644
index 0000000..3f608d5
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Sep/IntIndexOutput.cs
@@ -0,0 +1,61 @@
+package org.apache.lucene.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.
+ * 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.
+ */
+
+// TODO: we may want tighter integration w/ IndexOutput --
+// may give better perf:
+
+import org.apache.lucene.store.DataOutput;
+
+import java.io.IOException;
+import java.io.Closeable;
+
+/** Defines basic API for writing ints to an IndexOutput.
+ *  IntBlockCodec interacts with this API. @see
+ *  IntBlockReader.
+ *
+ * <p>NOTE: block sizes could be variable
+ *
+ * @lucene.experimental */
+public abstract class IntIndexOutput implements Closeable {
+
+  /** Write an int to the primary file.  The value must be
+   * >= 0.  */
+  public abstract void write(int v) ;
+
+  /** Records a single skip-point in the IndexOutput. */
+  public abstract static class Index {
+
+    /** Internally records the current location */
+    public abstract void mark() ;
+
+    /** Copies index from other */
+    public abstract void copyFrom(Index other, bool copyLast) ;
+
+    /** Writes "location" of current output pointer of primary
+     *  output to different output (out) */
+    public abstract void write(DataOutput indexOut, bool absolute) ;
+  }
+
+  /** If you are indexing the primary output file, call
+   *  this and interact with the returned IndexWriter. */
+  public abstract Index index();
+
+  @Override
+  public abstract void close() ;
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/src/Lucene.Net.Codecs/Sep/IntStreamFactory.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Sep/IntStreamFactory.cs b/src/Lucene.Net.Codecs/Sep/IntStreamFactory.cs
new file mode 100644
index 0000000..a6d42ba
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Sep/IntStreamFactory.cs
@@ -0,0 +1,36 @@
+package org.apache.lucene.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.
+ * 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 org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+import java.io.IOException;
+
+/** Provides int reader and writer to specified files.
+ *
+ * @lucene.experimental */
+public abstract class IntStreamFactory {
+  /** Create an {@link IntIndexInput} on the provided
+   *  fileName. */
+  public abstract IntIndexInput openInput(Directory dir, String fileName, IOContext context) ;
+
+  /** Create an {@link IntIndexOutput} on the provided
+   *  fileName. */
+  public abstract IntIndexOutput createOutput(Directory dir, String fileName, IOContext context) ;
+}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/1da1cb5b/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
new file mode 100644
index 0000000..3aa45fa
--- /dev/null
+++ b/src/Lucene.Net.Codecs/Sep/SepPostingsReader.cs
@@ -0,0 +1,714 @@
+package org.apache.lucene.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.
+ * 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.BlockTermState;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.PostingsReaderBase;
+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.SegmentInfo;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.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);
+        }
+      } 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;
+        }
+
+        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);
+    }
+
+    @Override
+    public int freq()  {
+      return freq;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int advance(int target)  {
+
+      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) {
+          // This DocsEnum has never done any skipping
+          skipper = new SepSkipListReader(skipIn.clone(),
+                                          freqIn,
+                                          docIn,
+                                          posIn,
+                                          maxSkipLevels, skipInterval);
+
+        }
+
+        if (!skipped) {
+          // We haven't yet skipped for this posting
+          skipper.init(skipFP,
+                       docIndex,
+                       freqIndex,
+                       posIndex,
+                       0,
+                       docFreq,
+                       storePayloads);
+          skipper.setIndexOptions(indexOptions);
+
+          skipped = true;
+        }
+
+        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;
+        }
+      } 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;
+        }
+
+        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;
+        }
+      }
+
+      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);
+        }
+
+        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);
+        }
+      }
+        
+      // 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;
+        }
+        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;
+    }
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return 0;
+  }
+
+  @Override
+  public void checkIntegrity()  {
+    // TODO: remove sep layout, its fallen behind on features...
+  }
+}