You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by ni...@apache.org on 2016/10/11 18:35:06 UTC

[21/47] lucenenet git commit: Reverted Codecs.IntBlock API back to its original form

Reverted Codecs.IntBlock API back to its original form


Project: http://git-wip-us.apache.org/repos/asf/lucenenet/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucenenet/commit/4137444b
Tree: http://git-wip-us.apache.org/repos/asf/lucenenet/tree/4137444b
Diff: http://git-wip-us.apache.org/repos/asf/lucenenet/diff/4137444b

Branch: refs/heads/master
Commit: 4137444b2d5a74e7df9c5fa1117d9458279accdd
Parents: b5259f4
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Tue Oct 11 00:06:20 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Tue Oct 11 00:26:22 2016 +0700

----------------------------------------------------------------------
 .../Intblock/FixedIntBlockIndexInput.cs         | 189 +++++++++++++---
 .../Intblock/FixedIntBlockIndexOutput.cs        | 131 +++++++++---
 src/Lucene.Net.Codecs/Intblock/IBlockReader.cs  |  31 ---
 .../Intblock/IntBlockIndexInput.cs              |  84 --------
 .../Intblock/IntBlockIndexOutput.cs             |  85 --------
 .../Intblock/IntBlockIndexReader.cs             | 100 ---------
 .../Intblock/VariableIntBlockIndexInput.cs      | 213 ++++++++++++++++---
 .../Intblock/VariableIntBlockIndexOutput.cs     | 132 ++++++++----
 src/Lucene.Net.Codecs/Lucene.Net.Codecs.csproj  |  12 +-
 9 files changed, 535 insertions(+), 442 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4137444b/src/Lucene.Net.Codecs/Intblock/FixedIntBlockIndexInput.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Intblock/FixedIntBlockIndexInput.cs b/src/Lucene.Net.Codecs/Intblock/FixedIntBlockIndexInput.cs
index f992c77..f4e629c 100644
--- a/src/Lucene.Net.Codecs/Intblock/FixedIntBlockIndexInput.cs
+++ b/src/Lucene.Net.Codecs/Intblock/FixedIntBlockIndexInput.cs
@@ -1,26 +1,25 @@
-\ufeff/*
- * 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.Intblock
-{
+\ufeffusing Lucene.Net.Codecs.Sep;
+using Lucene.Net.Store;
+using System.Diagnostics;
 
-    using Sep;
-    using IntIndexInput = Sep.IntIndexInput;
-    using IndexInput = Store.IndexInput;
+namespace Lucene.Net.Codecs.IntBlock
+{
+    /*
+     * 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.
+     */
 
     /// <summary>
     /// Naive int block API that writes vInts.  This is
@@ -40,35 +39,161 @@ namespace Lucene.Net.Codecs.Intblock
     public abstract class FixedIntBlockIndexInput : IntIndexInput
     {
 
-        private readonly IndexInput _input;
-        protected internal readonly int BLOCK_SIZE;
+        private readonly IndexInput input;
+        protected internal readonly int blockSize;
 
         public FixedIntBlockIndexInput(IndexInput @in)
         {
-            _input = @in;
-            BLOCK_SIZE = @in.ReadVInt();
+            input = @in;
+            blockSize = @in.ReadVInt();
         }
 
         public override IntIndexInputReader Reader()
         {
-            var buffer = new int[BLOCK_SIZE];
-            var clone = (IndexInput) _input.Clone();
+            var buffer = new int[blockSize];
+            var clone = (IndexInput)input.Clone();
             // TODO: can this be simplified?
-            return new IntBlockIndexReader(clone, buffer, GetBlockReader(clone, buffer));
+            return new InputReader(clone, buffer, GetBlockReader(clone, buffer));
         }
 
         public override void Dispose()
         {
-            _input.Dispose();
+            input.Dispose();
         }
 
         public override IntIndexInputIndex Index()
         {
-            return new IntBlockIndexInput(this);
+            return new InputIndex(this);
         }
 
         protected internal abstract IBlockReader GetBlockReader(IndexInput @in, int[] buffer);
 
-    }
 
+        /// <summary>
+        /// Interface for fixed-size block decoders.
+        /// <para>
+        /// Implementations should decode into the buffer in <see cref="ReadBlock()"/>.
+        /// </para>
+        /// </summary>
+        public interface IBlockReader
+        {
+            void ReadBlock();
+        }
+
+        private class InputReader : IntIndexInputReader
+        {
+            private readonly IndexInput input;
+            private readonly IBlockReader blockReader;
+            private readonly int blockSize;
+            private readonly int[] pending;
+
+            private int upto;
+            private bool seekPending;
+            private long pendingFP;
+            private long lastBlockFP = -1;
+
+            public InputReader(IndexInput input, int[] pending, IBlockReader blockReader)
+            {
+                this.input = input;
+                this.pending = pending;
+                this.blockSize = pending.Length;
+                this.blockReader = blockReader;
+                upto = blockSize;
+            }
+
+            internal void Seek(long fp, int upto)
+            {
+                Debug.Assert(upto < blockSize);
+                if (seekPending || fp != lastBlockFP)
+                {
+                    pendingFP = fp;
+                    seekPending = true;
+                }
+                this.upto = upto;
+            }
+
+            public override int Next()
+            {
+                if (seekPending)
+                {
+                    // Seek & load new block
+                    input.Seek(pendingFP);
+                    lastBlockFP = pendingFP;
+                    blockReader.ReadBlock();
+                    seekPending = false;
+                }
+                else if (upto == blockSize)
+                {
+                    // Load new block
+                    lastBlockFP = input.FilePointer;
+                    blockReader.ReadBlock();
+                    upto = 0;
+                }
+                return pending[upto++];
+            }
+        }
+
+        private class InputIndex : IntIndexInputIndex
+        {
+            private readonly FixedIntBlockIndexInput outerInstance;
+
+            public InputIndex(FixedIntBlockIndexInput outerInstance)
+            {
+                this.outerInstance = outerInstance;
+            }
+
+            private long fp;
+            private int upto;
+
+            public override void Read(DataInput indexIn, bool absolute)
+            {
+                if (absolute)
+                {
+                    upto = indexIn.ReadVInt();
+                    fp = indexIn.ReadVLong();
+                }
+                else
+                {
+                    int uptoDelta = indexIn.ReadVInt();
+                    if ((uptoDelta & 1) == 1)
+                    {
+                        // same block
+                        upto += (int)((uint)uptoDelta >> 1);
+                    }
+                    else
+                    {
+                        // new block
+                        upto = (int)((uint)uptoDelta >> 1);
+                        fp += indexIn.ReadVLong();
+                    }
+                }
+                Debug.Assert(upto < outerInstance.blockSize);
+            }
+
+            public override void Seek(IntIndexInputReader other)
+            {
+                ((InputReader)other).Seek(fp, upto);
+            }
+
+            public override void CopyFrom(IntIndexInputIndex other)
+            {
+                InputIndex idx = (InputIndex)other;
+                fp = idx.fp;
+                upto = idx.upto;
+            }
+
+            public override IntIndexInputIndex Clone()
+            {
+                InputIndex other = new InputIndex(outerInstance);
+                other.fp = fp;
+                other.upto = upto;
+                return other;
+            }
+
+            public override string ToString()
+            {
+                return "fp=" + fp + " upto=" + upto;
+            }
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4137444b/src/Lucene.Net.Codecs/Intblock/FixedIntBlockIndexOutput.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Intblock/FixedIntBlockIndexOutput.cs b/src/Lucene.Net.Codecs/Intblock/FixedIntBlockIndexOutput.cs
index 96d08d1..b98f92d 100644
--- a/src/Lucene.Net.Codecs/Intblock/FixedIntBlockIndexOutput.cs
+++ b/src/Lucene.Net.Codecs/Intblock/FixedIntBlockIndexOutput.cs
@@ -1,25 +1,25 @@
-\ufeff/*
- * 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.Intblock
+\ufeffusing Lucene.Net.Codecs.Sep;
+using Lucene.Net.Store;
+using System.Diagnostics;
+
+namespace Lucene.Net.Codecs.IntBlock
 {
-    using Sep;
-    using IntIndexOutput = Sep.IntIndexOutput;
-    using IndexOutput = Store.IndexOutput;
+    /*
+     * 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.
+     */
 
     /// <summary>
     /// Naive int block API that writes vInts.  This is
@@ -39,31 +39,95 @@ namespace Lucene.Net.Codecs.Intblock
     /// </summary>
     public abstract class FixedIntBlockIndexOutput : IntIndexOutput
     {
-        private readonly int _blockSize;
-        protected internal readonly int[] BUFFER;
+        protected readonly IndexOutput output;
+        private readonly int blockSize;
+        protected readonly int[] buffer;
+        private int upto;
 
         protected internal FixedIntBlockIndexOutput(IndexOutput output, int fixedBlockSize)
         {
-            _blockSize = fixedBlockSize;
-            OUTPUT = output;
-            output.WriteVInt(_blockSize);
-            BUFFER = new int[_blockSize];
+            blockSize = fixedBlockSize;
+            this.output = output;
+            output.WriteVInt(blockSize);
+            buffer = new int[blockSize];
         }
 
         protected internal abstract void FlushBlock();
 
         public override IntIndexOutputIndex Index()
         {
-            return new IntBlockIndexOuput(this);
+            return new OutputIndex(this);
+        }
+
+        private class OutputIndex : IntIndexOutputIndex
+        {
+            private readonly FixedIntBlockIndexOutput outerInstance;
+
+            public OutputIndex(FixedIntBlockIndexOutput outerInstance)
+            {
+                this.outerInstance = outerInstance;
+            }
+
+            internal long fp;
+            internal int upto;
+            internal long lastFP;
+            internal int lastUpto;
+
+            public override void Mark()
+            {
+                fp = outerInstance.output.FilePointer;
+                upto = outerInstance.upto;
+            }
+
+            public override void CopyFrom(IntIndexOutputIndex other, bool copyLast)
+            {
+                OutputIndex idx = (OutputIndex)other;
+                fp = idx.fp;
+                upto = idx.upto;
+                if (copyLast)
+                {
+                    lastFP = fp;
+                    lastUpto = upto;
+                }
+            }
+
+            public override void Write(DataOutput indexOut, bool absolute)
+            {
+                if (absolute)
+                {
+                    indexOut.WriteVInt(upto);
+                    indexOut.WriteVLong(fp);
+                }
+                else if (fp == lastFP)
+                {
+                    // same block
+                    Debug.Assert(upto >= lastUpto);
+                    int uptoDelta = upto - lastUpto;
+                    indexOut.WriteVInt(uptoDelta << 1 | 1);
+                }
+                else
+                {
+                    // new block
+                    indexOut.WriteVInt(upto << 1);
+                    indexOut.WriteVLong(fp - lastFP);
+                }
+                lastUpto = upto;
+                lastFP = fp;
+            }
+
+            public override string ToString()
+            {
+                return "fp=" + fp + " upto=" + upto;
+            }
         }
 
         public override void Write(int v)
         {
-            BUFFER[_upto++] = v;
-            if (_upto == _blockSize)
+            buffer[upto++] = v;
+            if (upto == blockSize)
             {
                 FlushBlock();
-                _upto = 0;
+                upto = 0;
             }
         }
 
@@ -71,7 +135,7 @@ namespace Lucene.Net.Codecs.Intblock
         {
             try
             {
-                if (_upto > 0)
+                if (upto > 0)
                 {
                     // NOTE: entries in the block after current upto are
                     // invalid
@@ -80,9 +144,8 @@ namespace Lucene.Net.Codecs.Intblock
             }
             finally
             {
-                OUTPUT.Dispose();
+                output.Dispose();
             }
         }
     }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4137444b/src/Lucene.Net.Codecs/Intblock/IBlockReader.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Intblock/IBlockReader.cs b/src/Lucene.Net.Codecs/Intblock/IBlockReader.cs
deleted file mode 100644
index 6218502..0000000
--- a/src/Lucene.Net.Codecs/Intblock/IBlockReader.cs
+++ /dev/null
@@ -1,31 +0,0 @@
-\ufeff/*
- * 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.Intblock
-{
-    /// <summary>
-    /// Interface for variable-size block decoders.
-    /// <para>
-    /// Implementations should decode into the buffer in <seealso cref="#readBlock"/>.
-    /// </para>
-    /// </summary>
-    public interface IBlockReader
-    {
-        int ReadBlock();
-        void Seek(long pos);
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4137444b/src/Lucene.Net.Codecs/Intblock/IntBlockIndexInput.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Intblock/IntBlockIndexInput.cs b/src/Lucene.Net.Codecs/Intblock/IntBlockIndexInput.cs
deleted file mode 100644
index 0e3e67f..0000000
--- a/src/Lucene.Net.Codecs/Intblock/IntBlockIndexInput.cs
+++ /dev/null
@@ -1,84 +0,0 @@
-\ufeff/*
- * 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.Intblock
-{
-    using Sep;
-    using Store;
-
-    internal class IntBlockIndexInput : IntIndexInputIndex
-    {
-        private readonly IntIndexInput _outerInstance;
-        private long _fp;
-        private int _upto;
-
-        public IntBlockIndexInput(IntIndexInput outerInstance)
-        {
-            _outerInstance = outerInstance;
-        }
-
-        public override void Read(DataInput indexIn, bool absolute)
-        {
-            if (absolute)
-            {
-                _upto = indexIn.ReadVInt();
-                _fp = indexIn.ReadVLong();
-            }
-            else
-            {
-                int uptoDelta = indexIn.ReadVInt();
-                if ((uptoDelta & 1) == 1)
-                {
-                    // same block
-                    _upto += (int) ((uint) uptoDelta >> 1);
-                }
-                else
-                {
-                    // new block
-                    _upto = (int) ((uint) uptoDelta >> 1);
-                    _fp += indexIn.ReadVLong();
-                }
-            }
-            // TODO: we can't do this assert because non-causal
-            // int encoders can have upto over the buffer size
-            //assert upto < maxBlockSize: "upto=" + upto + " max=" + maxBlockSize;
-        }
-
-        public override string ToString()
-        {
-            return "VarIntBlock.Index fp=" + _fp + " upto=" + _upto;
-        }
-
-        public override void Seek(IntIndexInputReader other)
-        {
-            ((IntBlockIndexReader)other).Seek(_fp, _upto);
-        }
-
-        public override void CopyFrom(IntIndexInputIndex other)
-        {
-            var idx = (IntBlockIndexInput)other;
-            _fp = idx._fp;
-            _upto = idx._upto;
-        }
-
-        public override IntIndexInputIndex Clone()
-        {
-            var other = new IntBlockIndexInput(_outerInstance) {_fp = _fp, _upto = _upto};
-            return other;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4137444b/src/Lucene.Net.Codecs/Intblock/IntBlockIndexOutput.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Intblock/IntBlockIndexOutput.cs b/src/Lucene.Net.Codecs/Intblock/IntBlockIndexOutput.cs
deleted file mode 100644
index b5cec95..0000000
--- a/src/Lucene.Net.Codecs/Intblock/IntBlockIndexOutput.cs
+++ /dev/null
@@ -1,85 +0,0 @@
-\ufeff/*
- * 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.Intblock
-{
-    using Sep;
-    using Store;
-    using System.Diagnostics;
-
-    internal class IntBlockIndexOuput : IntIndexOutputIndex
-    {
-        private readonly IntIndexOutput _outerInstance;
-        private long _fp;
-        private int _upto;
-        private long _lastFp;
-        private int _lastUpto;
-
-        public IntBlockIndexOuput(IntIndexOutput outerInstance)
-        {
-            _outerInstance = outerInstance;
-        }
-
-        public override void Mark()
-        {
-            _fp = _outerInstance.OUTPUT.FilePointer;
-            _upto = _outerInstance._upto;
-        }
-
-        public override void CopyFrom(IntIndexOutputIndex other, bool copyLast)
-        {
-            var idx = (IntBlockIndexOuput)other;
-            _fp = idx._fp;
-            _upto = idx._upto;
-            if (copyLast)
-            {
-                _lastFp = _fp;
-                _lastUpto = _upto;
-            }
-        }
-
-        public override void Write(DataOutput indexOut, bool absolute)
-        {
-            Debug.Assert(_upto >= 0);
-            if (absolute)
-            {
-                indexOut.WriteVInt(_upto);
-                indexOut.WriteVLong(_fp);
-            }
-            else if (_fp == _lastFp)
-            {
-                // same block
-                Debug.Assert(_upto >= _lastUpto);
-                var uptoDelta = _upto - _lastUpto;
-                indexOut.WriteVInt(uptoDelta << 1 | 1);
-            }
-            else
-            {
-                // new block
-                indexOut.WriteVInt(_upto << 1);
-                indexOut.WriteVLong(_fp - _lastFp);
-            }
-            _lastUpto = _upto;
-            _lastFp = _fp;
-        }
-
-        public override string ToString()
-        {
-            return "fp=" + _fp + " upto=" + _upto;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4137444b/src/Lucene.Net.Codecs/Intblock/IntBlockIndexReader.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Intblock/IntBlockIndexReader.cs b/src/Lucene.Net.Codecs/Intblock/IntBlockIndexReader.cs
deleted file mode 100644
index 22a4aa7..0000000
--- a/src/Lucene.Net.Codecs/Intblock/IntBlockIndexReader.cs
+++ /dev/null
@@ -1,100 +0,0 @@
-\ufeff/*
- * 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.Intblock
-{
-    using Sep;
-    using Store;
-    using System.Diagnostics;
-
-    internal class IntBlockIndexReader : IntIndexInputReader
-    {
-        private readonly IndexInput _input;
-
-        public readonly int[] PENDING;
-        private int _upto;
-
-        private bool _seekPending;
-        private long _pendingFp;
-        private int _pendingUpto;
-        private long _lastBlockFp;
-        private int _blockSize;
-        private readonly IBlockReader _blockReader;
-
-        public IntBlockIndexReader(IndexInput input, int[] pending, IBlockReader blockReader)
-        {
-            _input = input;
-            PENDING = pending;
-            _blockReader = blockReader;
-            _blockSize = pending.Length;
-            _upto = _blockSize;
-        }
-
-        internal virtual void Seek(long fp, int upto)
-        {
-            // TODO: should we do this in real-time, not lazy?
-            _pendingFp = fp;
-            _pendingUpto = upto;
-            Debug.Assert(_pendingUpto >= 0, "pendingUpto=" + _pendingUpto);
-            _seekPending = true;
-        }
-
-        internal void MaybeSeek()
-        {
-            if (!_seekPending) return;
-
-            if (_pendingFp != _lastBlockFp)
-            {
-                // need new block
-                _input.Seek(_pendingFp);
-                _lastBlockFp = _pendingFp;
-                _blockReader.Seek(_pendingFp);
-                _blockSize = _blockReader.ReadBlock();
-            }
-            _upto = _pendingUpto;
-
-            // TODO: if we were more clever when writing the
-            // index, such that a seek point wouldn't be written
-            // until the int encoder "committed", we could avoid
-            // this (likely minor) inefficiency:
-
-            // This is necessary for int encoders that are
-            // non-causal, ie must see future int values to
-            // encode the current ones.
-            while (_upto >= _blockSize)
-            {
-                _upto -= _blockSize;
-                _lastBlockFp = _input.FilePointer;
-                _blockSize = _blockReader.ReadBlock();
-            }
-            _seekPending = false;
-        }
-
-        public override int Next()
-        {
-            MaybeSeek();
-            if (_upto == _blockSize)
-            {
-                _lastBlockFp = _input.FilePointer;
-                _blockSize = _blockReader.ReadBlock();
-                _upto = 0;
-            }
-
-            return PENDING[_upto++];
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4137444b/src/Lucene.Net.Codecs/Intblock/VariableIntBlockIndexInput.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Intblock/VariableIntBlockIndexInput.cs b/src/Lucene.Net.Codecs/Intblock/VariableIntBlockIndexInput.cs
index 2ed4c47..f3d4b2f 100644
--- a/src/Lucene.Net.Codecs/Intblock/VariableIntBlockIndexInput.cs
+++ b/src/Lucene.Net.Codecs/Intblock/VariableIntBlockIndexInput.cs
@@ -1,25 +1,25 @@
-\ufeff/*
- * 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.Intblock
+\ufeffusing Lucene.Net.Codecs.Sep;
+using Lucene.Net.Store;
+using System.Diagnostics;
+
+namespace Lucene.Net.Codecs.IntBlock
 {
-    using Sep;
-    using IntIndexInput = Sep.IntIndexInput;
-    using IndexInput = Store.IndexInput;
+    /*
+     * 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.
+     */
 
     /// <summary>
     /// Naive int block API that writes vInts.  This is
@@ -40,35 +40,186 @@ namespace Lucene.Net.Codecs.Intblock
     /// </summary>
     public abstract class VariableIntBlockIndexInput : IntIndexInput
     {
-        private readonly IndexInput _input;
-        protected internal readonly int MAX_BLOCK_SIZE;
+        private readonly IndexInput input;
+        protected internal readonly int maxBlockSize;
 
         protected internal VariableIntBlockIndexInput(IndexInput input)
         {
-            _input = input;
-            MAX_BLOCK_SIZE = input.ReadInt();
+            this.input = input;
+            maxBlockSize = input.ReadInt();
         }
 
         public override IntIndexInputReader Reader()
         {
-            var buffer = new int[MAX_BLOCK_SIZE];
-            var clone = (IndexInput)_input.Clone();
+            var buffer = new int[maxBlockSize];
+            var clone = (IndexInput)input.Clone();
             // TODO: can this be simplified?
-            return new IntBlockIndexReader(clone, buffer, GetBlockReader(clone, buffer));
+            return new InputReader(clone, buffer, GetBlockReader(clone, buffer));
         }
 
         public override void Dispose()
         {
-            _input.Dispose();
+            input.Dispose();
         }
 
         public override IntIndexInputIndex Index()
         {
-            return new IntBlockIndexInput(this);
+            return new InputIndex(this);
         }
 
         protected internal abstract IBlockReader GetBlockReader(IndexInput @in, int[] buffer);
 
-    }
+        /// <summary>
+        /// Interface for variable-size block decoders.
+        /// <para>
+        /// Implementations should decode into the buffer in <see cref="ReadBlock()"/>.
+        /// </para>
+        /// </summary>
+        public interface IBlockReader
+        {
+            int ReadBlock();
+            void Seek(long pos);
+        }
+
+        private class InputReader : IntIndexInputReader
+        {
+            private readonly IndexInput input;
+
+            public readonly int[] pending;
+            private int upto;
+
+            private bool seekPending;
+            private long pendingFP;
+            private int pendingUpto;
+            private long lastBlockFP;
+            private int blockSize;
+            private readonly IBlockReader blockReader;
+
+            public InputReader(IndexInput input, int[] pending, IBlockReader blockReader)
+            {
+                this.input = input;
+                this.pending = pending;
+                this.blockReader = blockReader;
+            }
+
+            internal void Seek(long fp, int upto)
+            {
+                // TODO: should we do this in real-time, not lazy?
+                pendingFP = fp;
+                pendingUpto = upto;
+                Debug.Assert(pendingUpto >= 0, "pendingUpto=" + pendingUpto);
+                seekPending = true;
+            }
+
+            private void MaybeSeek()
+            {
+                if (seekPending)
+                {
+                    if (pendingFP != lastBlockFP)
+                    {
+                        // need new block
+                        input.Seek(pendingFP);
+                        blockReader.Seek(pendingFP);
+                        lastBlockFP = pendingFP;
+                        blockSize = blockReader.ReadBlock();
+                    }
+                    upto = pendingUpto;
+
+                    // TODO: if we were more clever when writing the
+                    // index, such that a seek point wouldn't be written
+                    // until the int encoder "committed", we could avoid
+                    // this (likely minor) inefficiency:
+
+                    // This is necessary for int encoders that are
+                    // non-causal, ie must see future int values to
+                    // encode the current ones.
+                    while (upto >= blockSize)
+                    {
+                        upto -= blockSize;
+                        lastBlockFP = input.FilePointer;
+                        blockSize = blockReader.ReadBlock();
+                    }
+                    seekPending = false;
+                }
+            }
 
+            public override int Next()
+            {
+                this.MaybeSeek();
+                if (upto == blockSize)
+                {
+                    lastBlockFP = input.FilePointer;
+                    blockSize = blockReader.ReadBlock();
+                    upto = 0;
+                }
+
+                return pending[upto++];
+            }
+        }
+
+        private class InputIndex : IntIndexInputIndex
+        {
+            private readonly VariableIntBlockIndexInput outerInstance;
+
+            public InputIndex(VariableIntBlockIndexInput outerInstance)
+            {
+                this.outerInstance = outerInstance;
+            }
+
+            private long fp;
+            private int upto;
+
+            public override void Read(DataInput indexIn, bool absolute)
+            {
+                if (absolute)
+                {
+                    upto = indexIn.ReadVInt();
+                    fp = indexIn.ReadVLong();
+                }
+                else
+                {
+                    int uptoDelta = indexIn.ReadVInt();
+                    if ((uptoDelta & 1) == 1)
+                    {
+                        // same block
+                        upto += (int)((uint)uptoDelta >> 1);
+                    }
+                    else
+                    {
+                        // new block
+                        upto = (int)((uint)uptoDelta >> 1);
+                        fp += indexIn.ReadVLong();
+                    }
+                }
+                // TODO: we can't do this assert because non-causal
+                // int encoders can have upto over the buffer size
+                //assert upto < maxBlockSize: "upto=" + upto + " max=" + maxBlockSize;
+            }
+
+            public override string ToString()
+            {
+                return "VarIntBlock.Index fp=" + fp + " upto=" + upto + " maxBlock=" + outerInstance.maxBlockSize;
+            }
+
+            public override void Seek(IntIndexInputReader other)
+            {
+                ((InputReader)other).Seek(fp, upto);
+            }
+
+            public override void CopyFrom(IntIndexInputIndex other)
+            {
+                InputIndex idx = (InputIndex)other;
+                fp = idx.fp;
+                upto = idx.upto;
+            }
+
+            public override IntIndexInputIndex Clone()
+            {
+                InputIndex other = new InputIndex(outerInstance);
+                other.fp = fp;
+                other.upto = upto;
+                return other;
+            }
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4137444b/src/Lucene.Net.Codecs/Intblock/VariableIntBlockIndexOutput.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Intblock/VariableIntBlockIndexOutput.cs b/src/Lucene.Net.Codecs/Intblock/VariableIntBlockIndexOutput.cs
index dd30ba7..131ffcc 100644
--- a/src/Lucene.Net.Codecs/Intblock/VariableIntBlockIndexOutput.cs
+++ b/src/Lucene.Net.Codecs/Intblock/VariableIntBlockIndexOutput.cs
@@ -1,27 +1,25 @@
-\ufeff/*
- * 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.Intblock
-{
+\ufeffusing Lucene.Net.Codecs.Sep;
+using Lucene.Net.Store;
+using System.Diagnostics;
 
-    using System.Diagnostics;
-    using Sep;
-    using IntIndexOutput = Sep.IntIndexOutput;
-    using IndexOutput = Store.IndexOutput;
+namespace Lucene.Net.Codecs.IntBlock
+{
+    /*
+     * 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.
+     */
 
     /// <summary>
     /// Naive int block API that writes vInts.  This is
@@ -43,7 +41,10 @@ namespace Lucene.Net.Codecs.Intblock
     /// </summary>
     public abstract class VariableIntBlockIndexOutput : IntIndexOutput
     {
-        private bool _hitExcDuringWrite;
+        protected readonly IndexOutput output;
+
+        private int upto;
+        private bool hitExcDuringWrite;
 
         // TODO what Var-Var codecs exist in practice... and what are there blocksizes like?
         // if its less than 128 we should set that as max and use byte?
@@ -56,8 +57,8 @@ namespace Lucene.Net.Codecs.Intblock
         /// </summary>
         protected internal VariableIntBlockIndexOutput(IndexOutput output, int maxBlockSize)
         {
-            OUTPUT = output;
-            output.WriteInt(maxBlockSize);
+            this.output = output;
+            this.output.WriteInt(maxBlockSize);
         }
 
         /// <summary>
@@ -68,37 +69,94 @@ namespace Lucene.Net.Codecs.Intblock
 
         public override IntIndexOutputIndex Index()
         {
-            return new IntBlockIndexOuput(this);
+            return new OutputIndex(this);
+        }
+
+        private class OutputIndex : IntIndexOutputIndex
+        {
+            private readonly VariableIntBlockIndexOutput outerInstance;
+
+            public OutputIndex(VariableIntBlockIndexOutput outerInstance)
+            {
+                this.outerInstance = outerInstance;
+            }
+
+            long fp;
+            int upto;
+            long lastFP;
+            int lastUpto;
+
+            public override void Mark()
+            {
+                fp = outerInstance.output.FilePointer;
+                upto = outerInstance.upto;
+            }
+
+            public override void CopyFrom(IntIndexOutputIndex other, bool copyLast)
+            {
+                OutputIndex idx = (OutputIndex)other;
+                fp = idx.fp;
+                upto = idx.upto;
+                if (copyLast)
+                {
+                    lastFP = fp;
+                    lastUpto = upto;
+                }
+            }
+
+            public override void Write(DataOutput indexOut, bool absolute)
+            {
+                Debug.Assert(upto >= 0);
+                if (absolute)
+                {
+                    indexOut.WriteVInt(upto);
+                    indexOut.WriteVLong(fp);
+                }
+                else if (fp == lastFP)
+                {
+                    // same block
+                    Debug.Assert(upto >= lastUpto);
+                    int uptoDelta = upto - lastUpto;
+                    indexOut.WriteVInt(uptoDelta << 1 | 1);
+                }
+                else
+                {
+                    // new block
+                    indexOut.WriteVInt(upto << 1);
+                    indexOut.WriteVLong(fp - lastFP);
+                }
+                lastUpto = upto;
+                lastFP = fp;
+            }
         }
 
         public override void Write(int v)
         {
-            _hitExcDuringWrite = true;
-            _upto -= Add(v) - 1;
-            _hitExcDuringWrite = false;
-            Debug.Assert(_upto >= 0);
+            hitExcDuringWrite = true;
+            upto -= Add(v) - 1;
+            hitExcDuringWrite = false;
+            Debug.Assert(upto >= 0);
         }
 
         public override void Dispose()
         {
             try
             {
-                if (_hitExcDuringWrite) return;
+                if (hitExcDuringWrite) return;
 
                 // stuff 0s in until the "real" data is flushed:
                 var stuffed = 0;
-                while (_upto > stuffed)
+                while (upto > stuffed)
                 {
-                    _upto -= Add(0) - 1;
-                    Debug.Assert(_upto >= 0);
+                    upto -= Add(0) - 1;
+                    Debug.Assert(upto >= 0);
                     stuffed += 1;
                 }
             }
             finally
             {
-                OUTPUT.Dispose();
+                output.Dispose();
             }
         }
     }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/4137444b/src/Lucene.Net.Codecs/Lucene.Net.Codecs.csproj
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Codecs/Lucene.Net.Codecs.csproj b/src/Lucene.Net.Codecs/Lucene.Net.Codecs.csproj
index f9a9902..4b17b1e 100644
--- a/src/Lucene.Net.Codecs/Lucene.Net.Codecs.csproj
+++ b/src/Lucene.Net.Codecs/Lucene.Net.Codecs.csproj
@@ -62,14 +62,10 @@
     <Compile Include="DiskDV\DiskDocValuesProducer.cs" />
     <Compile Include="DiskDV\DiskNormsFormat.cs" />
     <Compile Include="HashMapHelperClass.cs" />
-    <Compile Include="Intblock\FixedIntBlockIndexInput.cs" />
-    <Compile Include="Intblock\FixedIntBlockIndexOutput.cs" />
-    <Compile Include="Intblock\IBlockReader.cs" />
-    <Compile Include="Intblock\IntBlockIndexInput.cs" />
-    <Compile Include="Intblock\IntBlockIndexOutput.cs" />
-    <Compile Include="Intblock\IntBlockIndexReader.cs" />
-    <Compile Include="Intblock\VariableIntBlockIndexInput.cs" />
-    <Compile Include="Intblock\VariableIntBlockIndexOutput.cs" />
+    <Compile Include="IntBlock\FixedIntBlockIndexInput.cs" />
+    <Compile Include="IntBlock\FixedIntBlockIndexOutput.cs" />
+    <Compile Include="IntBlock\VariableIntBlockIndexInput.cs" />
+    <Compile Include="IntBlock\VariableIntBlockIndexOutput.cs" />
     <Compile Include="Memory\DirectDocValuesConsumer.cs" />
     <Compile Include="Memory\DirectDocValuesFormat.cs" />
     <Compile Include="Memory\DirectDocValuesProducer.cs" />