You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2019/10/30 15:01:21 UTC

[lucene-solr] 03/03: Move to PFOR Delta.

This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch simd
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 0913d9f20cb80cd67d3609421e4de389c3a2c75d
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Wed Oct 30 16:00:31 2019 +0100

    Move to PFOR Delta.
---
 .../lucene/codecs/lucene84/EliasFanoSequence.java  | 113 -----------------
 .../lucene/codecs/lucene84/EliasFanoUtil.java      |  85 -------------
 .../codecs/lucene84/Lucene84PostingsReader.java    | 140 +++++++--------------
 .../codecs/lucene84/Lucene84PostingsWriter.java    |  15 +--
 4 files changed, 52 insertions(+), 301 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/EliasFanoSequence.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene84/EliasFanoSequence.java
deleted file mode 100644
index ec1de09..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/EliasFanoSequence.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.codecs.lucene84;
-
-final class EliasFanoSequence {
-
-  final long[] upperBits = new long[ForUtil.BLOCK_SIZE >>> 5];
-  final long[] lowerBits = new long[ForUtil.BLOCK_SIZE];
-  int lowBitsPerValue;
-  int upperIndex;
-  int i;
-
-  // Like FixedBitSet#nextSetBit
-  private int nextUpperSetBit(int index) {
-    int i = index >>> 6;
-    long word = upperBits[i] >>> index;
-    if (word != 0) {
-      return index + Long.numberOfTrailingZeros(word);
-    } else {
-      while (++i < upperBits.length) {
-        word = upperBits[i];
-        if (word != 0) {
-          return (i << 6) + Long.numberOfTrailingZeros(word);
-        }
-      }
-    }
-    return ForUtil.BLOCK_SIZE << 1;
-  }
-
-  long next() {
-    ++i;
-    upperIndex = nextUpperSetBit(upperIndex+1);
-    long highBits = upperIndex - i;
-    long lowBits = lowerBits[i];
-    return (highBits << lowBitsPerValue) | lowBits;
-  }
-
-  /*private void advanceToHighTarget(long highTarget) {
-    int upperLongIndex = (upperIndex + 1) >>> 6;
-    long word = upperBits[upperLongIndex] >>> upperIndex;
-
-    if (word != 0) {
-      int popCount = Long.bitCount(word);
-      long maxHighBits = (1 << (upperLongIndex + 1)) - 1 - i - popCount;
-      if (maxHighBits >= highTarget) {
-        // The looked up high target might be within the current long, stop here
-        // TODO: rank selection
-        ++i;
-        upperIndex += 1 + Long.numberOfTrailingZeros(word);
-        return;
-      }
-      i += popCount;
-    }
-
-    while (++upperLongIndex < upperBits.length) {
-      word = upperBits[upperLongIndex];
-      if (word != 0) {
-        int popCount = Long.bitCount(word);
-        long maxHighBits = (1 << (upperLongIndex + 1)) - 1 - i - popCount;
-        if (maxHighBits >= highTarget) {
-          // TODO: rank selection
-          ++i;
-          upperIndex = (upperLongIndex << 6) + Long.numberOfTrailingZeros(word);
-        }
-        i += popCount;
-      }
-    }
-  }
-
-  long advance(long target) {
-    final long highTarget = target >>> lowBitsPerValue;
-    advanceToHighTarget(highTarget);
-    long highBits = upperIndex - i;
-    while (highBits < highTarget) {
-      ++i;
-      upperIndex = nextUpperSetBit(upperIndex+1);
-      highBits = upperIndex - i;
-    }
-    long lowBits = lowerBits[i];
-    return (highBits << lowBitsPerValue) | lowBits;
-  }*/
-
-  long maybeAdvance(long target) {
-    final long highTarget = target >>> lowBitsPerValue;
-    final int minUpperIndex = (int) (highTarget + i);
-    final int nextUpperIndex = upperIndex + 1;
-    final int nextWordIndex = nextUpperIndex >>> 6;
-    final int targetWordIndex = Math.min(minUpperIndex >>> 6, 4);
-    if (targetWordIndex > nextWordIndex) {
-      i += Long.bitCount(upperBits[nextUpperIndex] >>> nextUpperIndex);
-      for (int j = nextWordIndex + 1; j < targetWordIndex; ++j) {
-        i += Long.bitCount(upperBits[j]);
-      }
-      upperIndex = (targetWordIndex << 6) - 1;
-    }
-    return next();
-  }
-
-}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/EliasFanoUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene84/EliasFanoUtil.java
deleted file mode 100644
index 51bf841..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/EliasFanoUtil.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.codecs.lucene84;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.packed.PackedInts;
-
-public class EliasFanoUtil {
-
-  private final ForUtil forUtil;
-
-  EliasFanoUtil(ForUtil forUtil) {
-    this.forUtil = forUtil;
-  }
-
-  /**
-   * Encode 128 8-bits integers from {@code data} into {@code out}.
-   */
-  void encode(long[] longs, DataOutput out) throws IOException {
-    final int highBitsPerValue = 7;
-    final int lowBitsPerValue = Math.max(PackedInts.bitsRequired(longs[ForUtil.BLOCK_SIZE - 1]) - highBitsPerValue, 0);
-    out.writeByte((byte) lowBitsPerValue);
-    encodeHighBits(longs, lowBitsPerValue, out);
-    if (lowBitsPerValue > 0) {
-      for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) {
-        longs[i] &= (1L << lowBitsPerValue) - 1;
-      }
-      forUtil.encode(longs, lowBitsPerValue, out);
-    }
-  }
-
-  void decode(DataInput in, EliasFanoSequence efSequence) throws IOException {
-    efSequence.upperIndex = -1;
-    efSequence.i = -1;
-    final int lowBitsPerValue = in.readByte();
-    efSequence.lowBitsPerValue = lowBitsPerValue;
-    for (int i = 0; i < ForUtil.BLOCK_SIZE >>> 5; ++i) {
-      efSequence.upperBits[i] = in.readLong();
-    }
-    if (lowBitsPerValue == 0) {
-      Arrays.fill(efSequence.lowerBits, 0L);
-    } else {
-      forUtil.decode(lowBitsPerValue, in, efSequence.lowerBits);
-    }
-  }
-
-  void skip(DataInput in) throws IOException {
-    final int lowBitsPerValue = in.readByte();
-    in.skipBytes((ForUtil.BLOCK_SIZE >>> 2) + forUtil.numBytes(lowBitsPerValue));
-  }
-
-  private void encodeHighBits(long[] longs, int lowBitsPerValue, DataOutput out) throws IOException {
-    final long[] encoded = new long[ForUtil.BLOCK_SIZE >>> 5];
-    long previousHighBits = 0;
-    int index = -1;
-    for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) {
-      long highBits = longs[i] >>> lowBitsPerValue;
-      index += 1 + (highBits - previousHighBits);
-      encoded[index >>> 6] |= 1L << index;
-      previousHighBits = highBits;
-    }
-    for (long l : encoded) {
-      out.writeLong(l);
-    }
-  }
-
-}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsReader.java
index b43a3e5..bbf313e 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsReader.java
@@ -28,7 +28,6 @@ import static org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat.VERSION_S
 
 import java.io.IOException;
 import java.nio.ByteOrder;
-import java.util.Arrays;
 
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
@@ -129,42 +128,24 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
     }
   }
 
-  static void setSingletonDocID(int singletonDocID, EliasFanoSequence docBuffer) {
-    docBuffer.lowBitsPerValue = 31;
-    docBuffer.i = -1;
-    docBuffer.upperIndex = -1;
-    Arrays.fill(docBuffer.upperBits, ~0L);
-    docBuffer.lowerBits[0] = 1L + singletonDocID;
-  }
-
   /**
    * Read values that have been written using variable-length encoding instead of bit-packing.
    */
-  static void readVIntBlock(IndexInput docIn, EliasFanoSequence docBuffer,
+  static void readVIntBlock(IndexInput docIn, long[] docBuffer,
       long[] freqBuffer, int num, boolean indexHasFreq) throws IOException {
-    docBuffer.lowBitsPerValue = 31;
-    docBuffer.i = -1;
-    docBuffer.upperIndex = -1;
-    Arrays.fill(docBuffer.upperBits, ~0L);
     if (indexHasFreq) {
       for(int i=0;i<num;i++) {
         final int code = docIn.readVInt();
-        docBuffer.lowerBits[i] = code >>> 1;
+        docBuffer[i] = code >>> 1;
         if ((code & 1) != 0) {
           freqBuffer[i] = 1;
         } else {
           freqBuffer[i] = docIn.readVInt();
         }
-        if (i > 0) {
-          docBuffer.lowerBits[i] += docBuffer.lowerBits[i-1];
-        }
       }
     } else {
       for(int i=0;i<num;i++) {
-        docBuffer.lowerBits[i] = docIn.readVInt();
-        if (i > 0) {
-          docBuffer.lowerBits[i] += docBuffer.lowerBits[i-1];
-        }
+        docBuffer[i] = docIn.readVInt();
       }
     }
   }
@@ -272,10 +253,9 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
 
   final class BlockDocsEnum extends PostingsEnum {
 
-    final PForUtil pforUtil;
-    final EliasFanoUtil efUtil;
-    final EliasFanoSequence docIDSequence = new EliasFanoSequence();
+    final PForUtil pforUtil = new PForUtil(new ForUtil(byteOrder));
 
+    private final long[] docDeltaBuffer = new long[BLOCK_SIZE];
     private final long[] freqBuffer = new long[BLOCK_SIZE];
 
     private int docBufferUpto;
@@ -322,9 +302,6 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
       indexHasPos = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
       indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
       indexHasPayloads = fieldInfo.hasPayloads(); 
-      final ForUtil forUtil = new ForUtil(byteOrder);
-      pforUtil = new PForUtil(forUtil);
-      efUtil = new EliasFanoUtil(forUtil);
     }
 
     public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) {
@@ -356,7 +333,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
           freqBuffer[i] = 1;
         }
       }
-      accum = -1;
+      accum = 0;
       docUpto = 0;
       nextSkipDoc = BLOCK_SIZE - 1; // we won't skip if target is found in first block
       docBufferUpto = BLOCK_SIZE;
@@ -409,7 +386,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
       assert left > 0;
 
       if (left >= BLOCK_SIZE) {
-        efUtil.decode(docIn, docIDSequence);
+        pforUtil.decode(docIn, docDeltaBuffer);
 
         if (indexHasFreq) {
           if (needsFreq) {
@@ -419,11 +396,11 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
           }
         }
       } else if (docFreq == 1) {
-        setSingletonDocID(singletonDocID, docIDSequence);
+        docDeltaBuffer[0] = singletonDocID;
         freqBuffer[0] = totalTermFreq;
       } else {
         // Read vInts:
-        readVIntBlock(docIn, docIDSequence, freqBuffer, left, indexHasFreq);
+        readVIntBlock(docIn, docDeltaBuffer, freqBuffer, left, indexHasFreq);
       }
       docBufferUpto = 0;
     }
@@ -434,13 +411,13 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
         return doc = NO_MORE_DOCS;
       }
       if (docBufferUpto == BLOCK_SIZE) {
-        accum = doc;
         refillDocs(); // we don't need to load freqBuffer for now (will be loaded later if necessary)
       }
 
+      accum += docDeltaBuffer[docBufferUpto];
       docUpto++;
 
-      doc = (int) (accum + docIDSequence.next());
+      doc = (int) accum;
       docBufferUpto++;
       return doc;
     }
@@ -479,7 +456,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
 
           // Force to read next block
           docBufferUpto = BLOCK_SIZE;
-          doc = skipper.getDoc();                 // actually, this is just lastSkipEntry
+          accum = skipper.getDoc();               // actually, this is just lastSkipEntry
           docIn.seek(skipper.getDocPointer());    // now point to the block we want to search
           // even if freqBuffer were not read from the previous block, we will mark them as read,
           // as we don't need to skip the previous block freqBuffer in refillDocs,
@@ -494,18 +471,16 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
         return doc = NO_MORE_DOCS;
       }
       if (docBufferUpto == BLOCK_SIZE) {
-        accum = doc;
         refillDocs();
       }
 
       // Now scan... this is an inlined/pared down version
       // of nextDoc():
-      
       while (true) {
-        doc = (int) (accum + docIDSequence.next());
+        accum += docDeltaBuffer[docBufferUpto];
         docUpto++;
 
-        if (doc >= target) {
+        if (accum >= target) {
           break;
         }
         docBufferUpto++;
@@ -515,7 +490,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
       }
 
       docBufferUpto++;
-      return doc;
+      return doc = (int) accum;
     }
     
     @Override
@@ -527,10 +502,9 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
   // Also handles payloads + offsets
   final class EverythingEnum extends PostingsEnum {
 
-    final PForUtil pforUtil;
-    final EliasFanoUtil efUtil;
-    final EliasFanoSequence docIDSequence = new EliasFanoSequence();
+    final PForUtil pforUtil = new PForUtil(new ForUtil(byteOrder));
 
+    private final long[] docDeltaBuffer = new long[BLOCK_SIZE];
     private final long[] freqBuffer = new long[BLOCK_SIZE];
     private final long[] posDeltaBuffer = new long[BLOCK_SIZE];
 
@@ -566,7 +540,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
     private long totalTermFreq;                       // number of positions in this posting list
     private int docUpto;                              // how many docs we've read
     private int doc;                                  // doc we last read
-    private long accum;                               // accumulator for doc deltas
+    private int accum;                                // accumulator for doc deltas
     private int freq;                                 // freq we last read
     private int position;                             // current position
 
@@ -639,10 +613,6 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
         payloadBytes = null;
         payload = null;
       }
-
-      final ForUtil forUtil = new ForUtil(byteOrder);
-      pforUtil = new PForUtil(forUtil);
-      efUtil = new EliasFanoUtil(forUtil);
     }
 
     public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) {
@@ -681,7 +651,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
       this.needsPayloads = PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS);
 
       doc = -1;
-      accum = -1;
+      accum = 0;
       docUpto = 0;
       if (docFreq > BLOCK_SIZE) {
         nextSkipDoc = BLOCK_SIZE - 1; // we won't skip if target is found in first block
@@ -708,13 +678,13 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
       assert left > 0;
 
       if (left >= BLOCK_SIZE) {
-        efUtil.decode(docIn, docIDSequence);
+        pforUtil.decode(docIn, docDeltaBuffer);
         pforUtil.decode(docIn, freqBuffer);
       } else if (docFreq == 1) {
-        setSingletonDocID(singletonDocID, docIDSequence);
+        docDeltaBuffer[0] = singletonDocID;
         freqBuffer[0] = totalTermFreq;
       } else {
-        readVIntBlock(docIn, docIDSequence, freqBuffer, left, true);
+        readVIntBlock(docIn, docDeltaBuffer, freqBuffer, left, true);
       }
       docBufferUpto = 0;
     }
@@ -794,16 +764,16 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
         return doc = NO_MORE_DOCS;
       }
       if (docBufferUpto == BLOCK_SIZE) {
-        accum = doc;
         refillDocs();
       }
 
+      accum += docDeltaBuffer[docBufferUpto];
       freq = (int) freqBuffer[docBufferUpto];
       posPendingCount += freq;
       docBufferUpto++;
       docUpto++;
 
-      doc = (int) (accum + docIDSequence.next());
+      doc = accum;
       position = 0;
       lastStartOffset = 0;
       return doc;
@@ -840,7 +810,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
 
           // Force to read next block
           docBufferUpto = BLOCK_SIZE;
-          doc = skipper.getDoc();
+          accum = skipper.getDoc();
           docIn.seek(skipper.getDocPointer());
           posPendingFP = skipper.getPosPointer();
           payPendingFP = skipper.getPayPointer();
@@ -854,19 +824,18 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
         return doc = NO_MORE_DOCS;
       }
       if (docBufferUpto == BLOCK_SIZE) {
-        accum = doc;
         refillDocs();
       }
 
       // Now scan:
       while (true) {
-        doc = (int) (accum + docIDSequence.next());
+        accum += docDeltaBuffer[docBufferUpto];
         freq = (int) freqBuffer[docBufferUpto];
         posPendingCount += freq;
         docBufferUpto++;
         docUpto++;
 
-        if (doc >= target) {
+        if (accum >= target) {
           break;
         }
         if (docUpto == docFreq) {
@@ -876,7 +845,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
 
       position = 0;
       lastStartOffset = 0;
-      return doc;
+      return doc = accum;
     }
 
     // TODO: in theory we could avoid loading frq block
@@ -1009,10 +978,9 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
 
   final class BlockImpactsPostingsEnum extends ImpactsEnum {
 
-    final PForUtil pforUtil;
-    final EliasFanoUtil efUtil;
-    final EliasFanoSequence docIDSequence = new EliasFanoSequence();
+    final PForUtil pforUtil = new PForUtil(new ForUtil(byteOrder));
 
+    private final long[] docDeltaBuffer = new long[BLOCK_SIZE];
     private final long[] freqBuffer = new long[BLOCK_SIZE];
     private final long[] posDeltaBuffer = new long[BLOCK_SIZE];
 
@@ -1087,7 +1055,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
       }
 
       doc = -1;
-      accum = -1;
+      accum = 0;
       docUpto = 0;
       docBufferUpto = BLOCK_SIZE;
 
@@ -1097,10 +1065,6 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
           indexHasOffsets,
           indexHasPayloads);
       skipper.init(docTermStartFP+termState.skipOffset, docTermStartFP, posTermStartFP, payTermStartFP, docFreq);
-
-      final ForUtil forUtil = new ForUtil(byteOrder);
-      pforUtil = new PForUtil(forUtil);
-      efUtil = new EliasFanoUtil(forUtil);
     }
 
     @Override
@@ -1118,10 +1082,10 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
       assert left > 0;
 
       if (left >= BLOCK_SIZE) {
-        efUtil.decode(docIn, docIDSequence);
+        pforUtil.decode(docIn, docDeltaBuffer);
         pforUtil.decode(docIn, freqBuffer);
       } else {
-        readVIntBlock(docIn, docIDSequence, freqBuffer, left, true);
+        readVIntBlock(docIn, docDeltaBuffer, freqBuffer, left, true);
       }
       docBufferUpto = 0;
     }
@@ -1194,9 +1158,6 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
 
     @Override
     public int advance(int target) throws IOException {
-      if (docBufferUpto == BLOCK_SIZE) {
-        accum = Math.max(accum, doc);
-      }
       if (target > nextSkipDoc) {
         advanceShallow(target);
       }
@@ -1212,26 +1173,25 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
       }
 
       // Now scan:
-      long doc, freq;
+      long freq;
       while (true) {
-        doc = accum + docIDSequence.next();
+        accum += docDeltaBuffer[docBufferUpto];
         freq = freqBuffer[docBufferUpto];
         posPendingCount += freq;
         docBufferUpto++;
         docUpto++;
 
-        if (doc >= target) {
+        if (accum >= target) {
           break;
         }
         if (docUpto == docFreq) {
-          return this.doc = NO_MORE_DOCS;
+          return doc = NO_MORE_DOCS;
         }
       }
-      this.doc = (int) doc;
       this.freq = (int) freq;
       position = 0;
 
-      return this.doc;
+      return doc = (int) accum;
     }
 
     // TODO: in theory we could avoid loading frq block
@@ -1310,10 +1270,9 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
 
   final class BlockImpactsEverythingEnum extends ImpactsEnum {
 
-    final PForUtil pforUtil;
-    final EliasFanoUtil efUtil;
-    final EliasFanoSequence docIDSequence = new EliasFanoSequence();
+    final PForUtil pforUtil = new PForUtil(new ForUtil(byteOrder));
 
+    private final long[] docDeltaBuffer = new long[BLOCK_SIZE];
     private final long[] freqBuffer = new long[BLOCK_SIZE];
     private final long[] posDeltaBuffer = new long[BLOCK_SIZE];
 
@@ -1451,7 +1410,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
       }
 
       doc = -1;
-      accum = -1;
+      accum = 0;
       docUpto = 0;
       posDocUpTo = 0;
       isFreqsRead = true;
@@ -1469,10 +1428,6 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
           freqBuffer[i] = 1;
         }
       }
-
-      final ForUtil forUtil = new ForUtil(byteOrder);
-      pforUtil = new PForUtil(forUtil);
-      efUtil = new EliasFanoUtil(forUtil);
     }
     
     @Override
@@ -1512,12 +1467,12 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
       assert left > 0;
 
       if (left >= BLOCK_SIZE) {
-        efUtil.decode(docIn, docIDSequence);
+        pforUtil.decode(docIn, docDeltaBuffer);
         if (indexHasFreq) {
           isFreqsRead = false; // freq block will be loaded lazily when necessary, we don't load it here
         }
       } else {
-        readVIntBlock(docIn, docIDSequence, freqBuffer, left, indexHasFreq);
+        readVIntBlock(docIn, docDeltaBuffer, freqBuffer, left, indexHasFreq);
       }
       docBufferUpto = 0;
     }
@@ -1634,9 +1589,6 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
 
     @Override
     public int advance(int target) throws IOException {
-      if (docBufferUpto == BLOCK_SIZE) {
-        accum = Math.max(accum, doc);
-      }
       if (target > nextSkipDoc) {
         advanceShallow(target);
       }
@@ -1654,11 +1606,11 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
 
       // Now scan:
       while (true) {
-        doc = (int) (accum + docIDSequence.next());
+        accum += docDeltaBuffer[docBufferUpto];
         docBufferUpto++;
         docUpto++;
 
-        if (doc >= target) {
+        if (accum >= target) {
           break;
         }
         if (docUpto == docFreq) {
@@ -1668,7 +1620,7 @@ public final class Lucene84PostingsReader extends PostingsReaderBase {
       position = 0;
       lastStartOffset = 0;
 
-      return doc;
+      return doc = accum;
     }
 
     // TODO: in theory we could avoid loading frq block
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsWriter.java
index 70eca5e..f59e6b6 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene84/Lucene84PostingsWriter.java
@@ -92,7 +92,6 @@ public final class Lucene84PostingsWriter extends PushPostingsWriterBase {
   private int docCount;
 
   private final PForUtil pforUtil;
-  private final EliasFanoUtil efUtil;
   private final Lucene84SkipWriter skipWriter;
 
   private boolean fieldHasNorms;
@@ -118,9 +117,7 @@ public final class Lucene84PostingsWriter extends PushPostingsWriterBase {
       } else {
         throw new Error();
       }
-      final ForUtil forUtil = new ForUtil(byteOrder);
-      pforUtil = new PForUtil(forUtil);
-      efUtil = new EliasFanoUtil(forUtil);
+      pforUtil = new PForUtil(new ForUtil(byteOrder));
       if (state.fieldInfos.hasProx()) {
         posDeltaBuffer = new long[BLOCK_SIZE];
         String posFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene84PostingsFormat.POS_EXTENSION);
@@ -233,9 +230,9 @@ public final class Lucene84PostingsWriter extends PushPostingsWriterBase {
       competitiveFreqNormAccumulator.clear();
     }
 
-    final int docDelta = docID - lastBlockDocID;
+    final int docDelta = docID - lastDocID;
 
-    if (docID < 0 || (docCount > 0 && docID <= lastDocID)) {
+    if (docID < 0 || (docCount > 0 && docDelta <= 0)) {
       throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )", docOut);
     }
 
@@ -248,7 +245,7 @@ public final class Lucene84PostingsWriter extends PushPostingsWriterBase {
     docCount++;
 
     if (docBufferUpto == BLOCK_SIZE) {
-      efUtil.encode(docDeltaBuffer, docOut);
+      pforUtil.encode(docDeltaBuffer, docOut);
       if (writeFreqs) {
         pforUtil.encode(freqBuffer, docOut);
       }
@@ -364,12 +361,12 @@ public final class Lucene84PostingsWriter extends PushPostingsWriterBase {
     final int singletonDocID;
     if (state.docFreq == 1) {
       // pulse the singleton docid into the term dictionary, freq is implicitly totalTermFreq
-      singletonDocID = (int) (docDeltaBuffer[0] - 1);
+      singletonDocID = (int) docDeltaBuffer[0];
     } else {
       singletonDocID = -1;
       // vInt encode the remaining doc deltas and freqs:
       for(int i=0;i<docBufferUpto;i++) {
-        final int docDelta = (int) (docDeltaBuffer[i] - (i == 0 ? 0 : docDeltaBuffer[i-1]));
+        final int docDelta = (int) docDeltaBuffer[i];
         final int freq = (int) freqBuffer[i];
         if (!writeFreqs) {
           docOut.writeVInt(docDelta);