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 2021/08/30 13:27:49 UTC

[lucene] branch main updated: LUCENE-10035: Simple text codec add multi level skip list data (#224)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 6ade29c  LUCENE-10035: Simple text codec add multi level skip list data (#224)
6ade29c is described below

commit 6ade29c71a7088f6121e95035a7b1eb2724db584
Author: wuda <wu...@tiduyun.com>
AuthorDate: Mon Aug 30 21:27:42 2021 +0800

    LUCENE-10035: Simple text codec add multi level skip list data (#224)
---
 lucene/CHANGES.txt                                 |   3 +
 .../codecs/simpletext/SimpleTextFieldsReader.java  | 184 +++++++++++++++---
 .../codecs/simpletext/SimpleTextFieldsWriter.java  |  53 +++++-
 .../codecs/simpletext/SimpleTextSkipReader.java    | 206 +++++++++++++++++++++
 .../codecs/simpletext/SimpleTextSkipWriter.java    | 157 ++++++++++++++++
 .../lucene/codecs/MultiLevelSkipListReader.java    |  30 ++-
 .../lucene/codecs/MultiLevelSkipListWriter.java    |  24 ++-
 7 files changed, 619 insertions(+), 38 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index b048862..47a3b1c 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -15,6 +15,9 @@ New Features
 
 * LUCENE-9589: Swedish Minimal Stemmer (janhoy)
 
+* LUCENE-10035: The SimpleText codec now writes postings lists.
+  (wuda via Adrien Grand)
+
 System Requirements
 
 * LUCENE-8738: Move to Java 11 as minimum Java version.
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
index ac73b8d..c571875 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
@@ -25,6 +25,7 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.PAYLOAD
 import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.POS;
 import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.START_OFFSET;
 import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.TERM;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSkipWriter.SKIP_LIST;
 
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
@@ -38,6 +39,7 @@ import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.index.BaseTermsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.Impacts;
 import org.apache.lucene.index.ImpactsEnum;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.PostingsEnum;
@@ -45,6 +47,7 @@ import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SlowImpactsEnum;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.BufferedChecksumIndexInput;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
@@ -120,11 +123,15 @@ class SimpleTextFieldsReader extends FieldsProducer {
     private int docFreq;
     private long totalTermFreq;
     private long docsStart;
+    private long skipPointer;
     private boolean ended;
-    private final BytesRefFSTEnum<PairOutputs.Pair<Long, PairOutputs.Pair<Long, Long>>> fstEnum;
+    private final BytesRefFSTEnum<
+            PairOutputs.Pair<PairOutputs.Pair<Long, Long>, PairOutputs.Pair<Long, Long>>>
+        fstEnum;
 
     public SimpleTextTermsEnum(
-        FST<PairOutputs.Pair<Long, PairOutputs.Pair<Long, Long>>> fst, IndexOptions indexOptions) {
+        FST<PairOutputs.Pair<PairOutputs.Pair<Long, Long>, PairOutputs.Pair<Long, Long>>> fst,
+        IndexOptions indexOptions) {
       this.indexOptions = indexOptions;
       fstEnum = new BytesRefFSTEnum<>(fst);
     }
@@ -132,12 +139,16 @@ class SimpleTextFieldsReader extends FieldsProducer {
     @Override
     public boolean seekExact(BytesRef text) throws IOException {
 
-      final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long, PairOutputs.Pair<Long, Long>>>
+      final BytesRefFSTEnum.InputOutput<
+              PairOutputs.Pair<PairOutputs.Pair<Long, Long>, PairOutputs.Pair<Long, Long>>>
           result = fstEnum.seekExact(text);
       if (result != null) {
-        PairOutputs.Pair<Long, PairOutputs.Pair<Long, Long>> pair1 = result.output;
-        PairOutputs.Pair<Long, Long> pair2 = pair1.output2;
+        PairOutputs.Pair<PairOutputs.Pair<Long, Long>, PairOutputs.Pair<Long, Long>> pair =
+            result.output;
+        PairOutputs.Pair<Long, Long> pair1 = pair.output1;
+        PairOutputs.Pair<Long, Long> pair2 = pair.output2;
         docsStart = pair1.output1;
+        skipPointer = pair1.output2;
         docFreq = pair2.output1.intValue();
         totalTermFreq = pair2.output2;
         return true;
@@ -150,16 +161,20 @@ class SimpleTextFieldsReader extends FieldsProducer {
     public SeekStatus seekCeil(BytesRef text) throws IOException {
 
       // System.out.println("seek to text=" + text.utf8ToString());
-      final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long, PairOutputs.Pair<Long, Long>>>
+      final BytesRefFSTEnum.InputOutput<
+              PairOutputs.Pair<PairOutputs.Pair<Long, Long>, PairOutputs.Pair<Long, Long>>>
           result = fstEnum.seekCeil(text);
       if (result == null) {
         // System.out.println("  end");
         return SeekStatus.END;
       } else {
         // System.out.println("  got text=" + term.utf8ToString());
-        PairOutputs.Pair<Long, PairOutputs.Pair<Long, Long>> pair1 = result.output;
-        PairOutputs.Pair<Long, Long> pair2 = pair1.output2;
+        PairOutputs.Pair<PairOutputs.Pair<Long, Long>, PairOutputs.Pair<Long, Long>> pair =
+            result.output;
+        PairOutputs.Pair<Long, Long> pair1 = pair.output1;
+        PairOutputs.Pair<Long, Long> pair2 = pair.output2;
         docsStart = pair1.output1;
+        skipPointer = pair1.output2;
         docFreq = pair2.output1.intValue();
         totalTermFreq = pair2.output2;
 
@@ -176,12 +191,16 @@ class SimpleTextFieldsReader extends FieldsProducer {
     @Override
     public BytesRef next() throws IOException {
       assert !ended;
-      final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long, PairOutputs.Pair<Long, Long>>>
+      final BytesRefFSTEnum.InputOutput<
+              PairOutputs.Pair<PairOutputs.Pair<Long, Long>, PairOutputs.Pair<Long, Long>>>
           result = fstEnum.next();
       if (result != null) {
-        PairOutputs.Pair<Long, PairOutputs.Pair<Long, Long>> pair1 = result.output;
-        PairOutputs.Pair<Long, Long> pair2 = pair1.output2;
+        PairOutputs.Pair<PairOutputs.Pair<Long, Long>, PairOutputs.Pair<Long, Long>> pair =
+            result.output;
+        PairOutputs.Pair<Long, Long> pair1 = pair.output1;
+        PairOutputs.Pair<Long, Long> pair2 = pair.output2;
         docsStart = pair1.output1;
+        skipPointer = pair1.output2;
         docFreq = pair2.output1.intValue();
         totalTermFreq = pair2.output2;
         return result.input;
@@ -229,7 +248,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
         } else {
           docsAndPositionsEnum = new SimpleTextPostingsEnum();
         }
-        return docsAndPositionsEnum.reset(docsStart, indexOptions, docFreq);
+        return docsAndPositionsEnum.reset(docsStart, indexOptions, docFreq, skipPointer);
       }
 
       SimpleTextDocsEnum docsEnum;
@@ -240,16 +259,20 @@ class SimpleTextFieldsReader extends FieldsProducer {
       } else {
         docsEnum = new SimpleTextDocsEnum();
       }
-      return docsEnum.reset(docsStart, indexOptions == IndexOptions.DOCS, docFreq);
+      return docsEnum.reset(docsStart, indexOptions == IndexOptions.DOCS, docFreq, skipPointer);
     }
 
     @Override
     public ImpactsEnum impacts(int flags) throws IOException {
-      return new SlowImpactsEnum(postings(null, flags));
+      if (docFreq <= SimpleTextSkipWriter.BLOCK_SIZE) {
+        // no skip data
+        return new SlowImpactsEnum(postings(null, flags));
+      }
+      return (ImpactsEnum) postings(null, flags);
     }
   }
 
-  private class SimpleTextDocsEnum extends PostingsEnum {
+  private class SimpleTextDocsEnum extends ImpactsEnum {
     private final IndexInput inStart;
     private final IndexInput in;
     private boolean omitTF;
@@ -259,21 +282,31 @@ class SimpleTextFieldsReader extends FieldsProducer {
     private final CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
     private int cost;
 
+    // for skip list data
+    private SimpleTextSkipReader skipReader;
+    private int nextSkipDoc = 0;
+    private long seekTo = -1;
+
     public SimpleTextDocsEnum() {
       this.inStart = SimpleTextFieldsReader.this.in;
       this.in = this.inStart.clone();
+      this.skipReader = new SimpleTextSkipReader(this.inStart.clone());
     }
 
     public boolean canReuse(IndexInput in) {
       return in == inStart;
     }
 
-    public SimpleTextDocsEnum reset(long fp, boolean omitTF, int docFreq) throws IOException {
+    public SimpleTextDocsEnum reset(long fp, boolean omitTF, int docFreq, long skipPointer)
+        throws IOException {
       in.seek(fp);
       this.omitTF = omitTF;
       docID = -1;
       tf = 1;
       cost = docFreq;
+      skipReader.reset(skipPointer, docFreq);
+      nextSkipDoc = 0;
+      seekTo = -1;
       return this;
     }
 
@@ -309,6 +342,10 @@ class SimpleTextFieldsReader extends FieldsProducer {
 
     @Override
     public int nextDoc() throws IOException {
+      return advance(docID + 1);
+    }
+
+    private int readDoc() throws IOException {
       if (docID == NO_MORE_DOCS) {
         return docID;
       }
@@ -341,7 +378,8 @@ class SimpleTextFieldsReader extends FieldsProducer {
         } else if (StringHelper.startsWith(scratch.get(), PAYLOAD)) {
           // skip
         } else {
-          assert StringHelper.startsWith(scratch.get(), TERM)
+          assert StringHelper.startsWith(scratch.get(), SimpleTextSkipWriter.SKIP_LIST)
+                  || StringHelper.startsWith(scratch.get(), TERM)
                   || StringHelper.startsWith(scratch.get(), FIELD)
                   || StringHelper.startsWith(scratch.get(), END)
               : "scratch=" + scratch.get().utf8ToString();
@@ -357,19 +395,50 @@ class SimpleTextFieldsReader extends FieldsProducer {
       }
     }
 
+    private int advanceTarget(int target) throws IOException {
+      if (seekTo > 0) {
+        in.seek(seekTo);
+        seekTo = -1;
+      }
+      assert docID() < target;
+      int doc;
+      do {
+        doc = readDoc();
+      } while (doc < target);
+      return doc;
+    }
+
     @Override
     public int advance(int target) throws IOException {
-      // Naive -- better to index skip data
-      return slowAdvance(target);
+      advanceShallow(target);
+      return advanceTarget(target);
     }
 
     @Override
     public long cost() {
       return cost;
     }
+
+    @Override
+    public void advanceShallow(int target) throws IOException {
+      if (target > nextSkipDoc) {
+        skipReader.skipTo(target);
+        if (skipReader.getNextSkipDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+          seekTo = skipReader.getNextSkipDocFP();
+        }
+        nextSkipDoc = skipReader.getNextSkipDoc();
+      }
+      assert nextSkipDoc >= target;
+    }
+
+    @Override
+    public Impacts getImpacts() throws IOException {
+      advanceShallow(docID);
+      return skipReader.getImpacts();
+    }
   }
 
-  private class SimpleTextPostingsEnum extends PostingsEnum {
+  private class SimpleTextPostingsEnum extends ImpactsEnum {
     private final IndexInput inStart;
     private final IndexInput in;
     private int docID = -1;
@@ -387,16 +456,23 @@ class SimpleTextFieldsReader extends FieldsProducer {
     private int endOffset;
     private int cost;
 
+    // for skip list data
+    private SimpleTextSkipReader skipReader;
+    private int nextSkipDoc = 0;
+    private long seekTo = -1;
+
     public SimpleTextPostingsEnum() {
       this.inStart = SimpleTextFieldsReader.this.in;
       this.in = inStart.clone();
+      this.skipReader = new SimpleTextSkipReader(this.inStart.clone());
     }
 
     public boolean canReuse(IndexInput in) {
       return in == inStart;
     }
 
-    public SimpleTextPostingsEnum reset(long fp, IndexOptions indexOptions, int docFreq) {
+    public SimpleTextPostingsEnum reset(
+        long fp, IndexOptions indexOptions, int docFreq, long skipPointer) throws IOException {
       nextDocStart = fp;
       docID = -1;
       readPositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
@@ -407,6 +483,9 @@ class SimpleTextFieldsReader extends FieldsProducer {
         endOffset = -1;
       }
       cost = docFreq;
+      skipReader.reset(skipPointer, docFreq);
+      nextSkipDoc = 0;
+      seekTo = -1;
       return this;
     }
 
@@ -422,6 +501,10 @@ class SimpleTextFieldsReader extends FieldsProducer {
 
     @Override
     public int nextDoc() throws IOException {
+      return advance(docID + 1);
+    }
+
+    private int readDoc() throws IOException {
       boolean first = true;
       in.seek(nextDocStart);
       long posStart = 0;
@@ -452,7 +535,8 @@ class SimpleTextFieldsReader extends FieldsProducer {
         } else if (StringHelper.startsWith(scratch.get(), PAYLOAD)) {
           // skip
         } else {
-          assert StringHelper.startsWith(scratch.get(), TERM)
+          assert StringHelper.startsWith(scratch.get(), SimpleTextSkipWriter.SKIP_LIST)
+              || StringHelper.startsWith(scratch.get(), TERM)
               || StringHelper.startsWith(scratch.get(), FIELD)
               || StringHelper.startsWith(scratch.get(), END);
           if (!first) {
@@ -465,10 +549,23 @@ class SimpleTextFieldsReader extends FieldsProducer {
       }
     }
 
+    private int advanceTarget(int target) throws IOException {
+      if (seekTo > 0) {
+        nextDocStart = seekTo;
+        seekTo = -1;
+      }
+      assert docID() < target;
+      int doc;
+      do {
+        doc = readDoc();
+      } while (doc < target);
+      return doc;
+    }
+
     @Override
     public int advance(int target) throws IOException {
-      // Naive -- better to index skip data
-      return slowAdvance(target);
+      advanceShallow(target);
+      return advanceTarget(target);
     }
 
     @Override
@@ -533,6 +630,24 @@ class SimpleTextFieldsReader extends FieldsProducer {
     public long cost() {
       return cost;
     }
+
+    @Override
+    public void advanceShallow(int target) throws IOException {
+      if (target > nextSkipDoc) {
+        skipReader.skipTo(target);
+        if (skipReader.getNextSkipDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+          seekTo = skipReader.getNextSkipDocFP();
+        }
+      }
+      nextSkipDoc = skipReader.getNextSkipDoc();
+      assert nextSkipDoc >= target;
+    }
+
+    @Override
+    public Impacts getImpacts() throws IOException {
+      advanceShallow(docID);
+      return skipReader.getImpacts();
+    }
   }
 
   private static final long TERMS_BASE_RAM_BYTES_USED =
@@ -547,7 +662,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
     private long sumTotalTermFreq;
     private long sumDocFreq;
     private int docCount;
-    private FST<PairOutputs.Pair<Long, PairOutputs.Pair<Long, Long>>> fst;
+    private FST<PairOutputs.Pair<PairOutputs.Pair<Long, Long>, PairOutputs.Pair<Long, Long>>> fst;
     private int termCount;
     private final BytesRefBuilder scratch = new BytesRefBuilder();
     private final CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
@@ -561,10 +676,13 @@ class SimpleTextFieldsReader extends FieldsProducer {
 
     private void loadTerms() throws IOException {
       PositiveIntOutputs posIntOutputs = PositiveIntOutputs.getSingleton();
-      final FSTCompiler<PairOutputs.Pair<Long, PairOutputs.Pair<Long, Long>>> fstCompiler;
+      final FSTCompiler<
+              PairOutputs.Pair<PairOutputs.Pair<Long, Long>, PairOutputs.Pair<Long, Long>>>
+          fstCompiler;
+      final PairOutputs<Long, Long> outputsOuter = new PairOutputs<>(posIntOutputs, posIntOutputs);
       final PairOutputs<Long, Long> outputsInner = new PairOutputs<>(posIntOutputs, posIntOutputs);
-      final PairOutputs<Long, PairOutputs.Pair<Long, Long>> outputs =
-          new PairOutputs<>(posIntOutputs, outputsInner);
+      final PairOutputs<PairOutputs.Pair<Long, Long>, PairOutputs.Pair<Long, Long>> outputs =
+          new PairOutputs<>(outputsOuter, outputsInner);
       fstCompiler = new FSTCompiler<>(FST.INPUT_TYPE.BYTE1, outputs);
       IndexInput in = SimpleTextFieldsReader.this.in.clone();
       in.seek(termsStart);
@@ -572,6 +690,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
       long lastDocsStart = -1;
       int docFreq = 0;
       long totalTermFreq = 0;
+      long skipPointer = 0;
       FixedBitSet visitedDocs = new FixedBitSet(maxDoc);
       final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
       while (true) {
@@ -581,7 +700,8 @@ class SimpleTextFieldsReader extends FieldsProducer {
             fstCompiler.add(
                 Util.toIntsRef(lastTerm.get(), scratchIntsRef),
                 outputs.newPair(
-                    lastDocsStart, outputsInner.newPair((long) docFreq, totalTermFreq)));
+                    outputsOuter.newPair(lastDocsStart, skipPointer),
+                    outputsInner.newPair((long) docFreq, totalTermFreq)));
             sumTotalTermFreq += totalTermFreq;
           }
           break;
@@ -595,12 +715,15 @@ class SimpleTextFieldsReader extends FieldsProducer {
         } else if (StringHelper.startsWith(scratch.get(), FREQ)) {
           scratchUTF16.copyUTF8Bytes(scratch.bytes(), FREQ.length, scratch.length() - FREQ.length);
           totalTermFreq += ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length()) - 1;
+        } else if (StringHelper.startsWith(scratch.get(), SKIP_LIST)) {
+          skipPointer = in.getFilePointer();
         } else if (StringHelper.startsWith(scratch.get(), TERM)) {
           if (lastDocsStart != -1) {
             fstCompiler.add(
                 Util.toIntsRef(lastTerm.get(), scratchIntsRef),
                 outputs.newPair(
-                    lastDocsStart, outputsInner.newPair((long) docFreq, totalTermFreq)));
+                    outputsOuter.newPair(lastDocsStart, skipPointer),
+                    outputsInner.newPair((long) docFreq, totalTermFreq)));
           }
           lastDocsStart = in.getFilePointer();
           final int len = scratch.length() - TERM.length;
@@ -611,6 +734,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
           sumTotalTermFreq += totalTermFreq;
           totalTermFreq = 0;
           termCount++;
+          skipPointer = 0;
         }
       }
       docCount = visitedDocs.cardinality();
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
index 5453592..55f1ba0 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
@@ -17,11 +17,13 @@
 package org.apache.lucene.codecs.simpletext;
 
 import java.io.IOException;
+import org.apache.lucene.codecs.CompetitiveImpactAccumulator;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.NormsProducer;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
@@ -37,6 +39,14 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
   private final SegmentWriteState writeState;
   final String segment;
 
+  /** for write skip data. */
+  private int docCount = 0;
+
+  private final SimpleTextSkipWriter skipWriter;
+  private final CompetitiveImpactAccumulator competitiveImpactAccumulator =
+      new CompetitiveImpactAccumulator();
+  private long lastDocFilePointer = -1;
+
   static final BytesRef END = new BytesRef("END");
   static final BytesRef FIELD = new BytesRef("field ");
   static final BytesRef TERM = new BytesRef("  term ");
@@ -54,14 +64,16 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
     segment = writeState.segmentInfo.name;
     out = writeState.directory.createOutput(fileName, writeState.context);
     this.writeState = writeState;
+    this.skipWriter = new SimpleTextSkipWriter(writeState);
   }
 
   @Override
   public void write(Fields fields, NormsProducer norms) throws IOException {
-    write(writeState.fieldInfos, fields);
+    write(writeState.fieldInfos, fields, norms);
   }
 
-  public void write(FieldInfos fieldInfos, Fields fields) throws IOException {
+  public void write(FieldInfos fieldInfos, Fields fields, NormsProducer normsProducer)
+      throws IOException {
 
     // for each field
     for (String field : fields) {
@@ -78,6 +90,12 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
       boolean hasFreqs = terms.hasFreqs();
       boolean hasPayloads = fieldInfo.hasPayloads();
       boolean hasOffsets = terms.hasOffsets();
+      boolean fieldHasNorms = fieldInfo.hasNorms();
+
+      NumericDocValues norms = null;
+      if (fieldHasNorms && normsProducer != null) {
+        norms = normsProducer.getNorms(fieldInfo);
+      }
 
       int flags = 0;
       if (hasPositions) {
@@ -103,6 +121,10 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
         if (term == null) {
           break;
         }
+        docCount = 0;
+        skipWriter.resetSkip();
+        competitiveImpactAccumulator.clear();
+        lastDocFilePointer = -1;
 
         postingsEnum = termsEnum.postings(postingsEnum, flags);
 
@@ -136,7 +158,9 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
             newline();
             wroteTerm = true;
           }
-
+          if (lastDocFilePointer == -1) {
+            lastDocFilePointer = out.getFilePointer();
+          }
           write(DOC);
           write(Integer.toString(doc));
           newline();
@@ -183,7 +207,19 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
                 }
               }
             }
+            competitiveImpactAccumulator.add(freq, getNorm(doc, norms));
+          } else {
+            competitiveImpactAccumulator.add(1, getNorm(doc, norms));
           }
+          docCount++;
+          if (docCount != 0 && docCount % SimpleTextSkipWriter.BLOCK_SIZE == 0) {
+            skipWriter.bufferSkip(doc, lastDocFilePointer, docCount, competitiveImpactAccumulator);
+            competitiveImpactAccumulator.clear();
+            lastDocFilePointer = -1;
+          }
+        }
+        if (docCount >= SimpleTextSkipWriter.BLOCK_SIZE) {
+          skipWriter.writeSkip(out);
         }
       }
     }
@@ -214,4 +250,15 @@ class SimpleTextFieldsWriter extends FieldsConsumer {
       }
     }
   }
+
+  private long getNorm(int doc, NumericDocValues norms) throws IOException {
+    if (norms == null) {
+      return 1L;
+    }
+    boolean found = norms.advanceExact(doc);
+    if (found == false) {
+      return 1L;
+    }
+    return norms.longValue();
+  }
 }
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSkipReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSkipReader.java
new file mode 100644
index 0000000..d418021
--- /dev/null
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSkipReader.java
@@ -0,0 +1,206 @@
+/*
+ * 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.simpletext;
+
+import static org.apache.lucene.codecs.simpletext.SimpleTextSkipWriter.CHILD_POINTER;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSkipWriter.FREQ;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSkipWriter.IMPACT;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSkipWriter.IMPACTS;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSkipWriter.IMPACTS_END;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSkipWriter.LEVEL_LENGTH;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSkipWriter.NORM;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSkipWriter.SKIP_DOC;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSkipWriter.SKIP_DOC_FP;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSkipWriter.SKIP_LIST;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.codecs.MultiLevelSkipListReader;
+import org.apache.lucene.index.Impact;
+import org.apache.lucene.index.Impacts;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.CharsRefBuilder;
+import org.apache.lucene.util.StringHelper;
+
+/**
+ * This class reads skip lists with multiple levels.
+ *
+ * <p>See {@link SimpleTextFieldsWriter} for the information about the encoding of the multi level
+ * skip lists.
+ *
+ * @lucene.experimental
+ */
+class SimpleTextSkipReader extends MultiLevelSkipListReader {
+
+  private final CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
+  private final BytesRefBuilder scratch = new BytesRefBuilder();
+  private Impacts impacts;
+  private List<List<Impact>> perLevelImpacts;
+  private long nextSkipDocFP = -1;
+  private int numLevels = 1;
+  private boolean hasSkipList = false;
+
+  SimpleTextSkipReader(IndexInput skipStream) {
+    super(
+        skipStream,
+        SimpleTextSkipWriter.maxSkipLevels,
+        SimpleTextSkipWriter.BLOCK_SIZE,
+        SimpleTextSkipWriter.skipMultiplier);
+    impacts =
+        new Impacts() {
+          @Override
+          public int numLevels() {
+            return numLevels;
+          }
+
+          @Override
+          public int getDocIdUpTo(int level) {
+            return skipDoc[level];
+          }
+
+          @Override
+          public List<Impact> getImpacts(int level) {
+            assert level < numLevels;
+            return perLevelImpacts.get(level);
+          }
+        };
+    init();
+  }
+
+  @Override
+  public int skipTo(int target) throws IOException {
+    if (!hasSkipList) {
+      return -1;
+    }
+    int result = super.skipTo(target);
+    if (numberOfSkipLevels > 0) {
+      numLevels = numberOfSkipLevels;
+    } else {
+      // End of postings don't have skip data anymore, so we fill with dummy data
+      // like SlowImpactsEnum.
+      numLevels = 1;
+      perLevelImpacts.add(0, Collections.singletonList(new Impact(Integer.MAX_VALUE, 1L)));
+    }
+    return result;
+  }
+
+  @Override
+  protected int readSkipData(int level, IndexInput skipStream) throws IOException {
+    perLevelImpacts.get(level).clear();
+    int skipDoc = DocIdSetIterator.NO_MORE_DOCS;
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(skipStream);
+    int freq = 1;
+    while (true) {
+      SimpleTextUtil.readLine(input, scratch);
+      if (scratch.get().equals(SimpleTextFieldsWriter.END)) {
+        SimpleTextUtil.checkFooter(input);
+        break;
+      } else if (scratch.get().equals(IMPACTS_END)
+          || scratch.get().equals(SimpleTextFieldsWriter.TERM)
+          || scratch.get().equals(SimpleTextFieldsWriter.FIELD)) {
+        break;
+      } else if (StringHelper.startsWith(scratch.get(), SKIP_LIST)) {
+        // continue
+      } else if (StringHelper.startsWith(scratch.get(), SKIP_DOC)) {
+        scratchUTF16.copyUTF8Bytes(
+            scratch.bytes(), SKIP_DOC.length, scratch.length() - SKIP_DOC.length);
+        skipDoc = ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length());
+        // Because the MultiLevelSkipListReader stores doc id delta,but simple text codec stores doc
+        // id
+        skipDoc = skipDoc - super.skipDoc[level];
+      } else if (StringHelper.startsWith(scratch.get(), SKIP_DOC_FP)) {
+        scratchUTF16.copyUTF8Bytes(
+            scratch.bytes(), SKIP_DOC_FP.length, scratch.length() - SKIP_DOC_FP.length);
+        nextSkipDocFP = ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length());
+      } else if (StringHelper.startsWith(scratch.get(), IMPACTS)
+          || StringHelper.startsWith(scratch.get(), IMPACT)) {
+        // continue;
+      } else if (StringHelper.startsWith(scratch.get(), FREQ)) {
+        scratchUTF16.copyUTF8Bytes(scratch.bytes(), FREQ.length, scratch.length() - FREQ.length);
+        freq = ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length());
+      } else if (StringHelper.startsWith(scratch.get(), NORM)) {
+        scratchUTF16.copyUTF8Bytes(scratch.bytes(), NORM.length, scratch.length() - NORM.length);
+        long norm = Long.parseLong(scratchUTF16.toString());
+        Impact impact = new Impact(freq, norm);
+        perLevelImpacts.get(level).add(impact);
+      }
+    }
+    return skipDoc;
+  }
+
+  @Override
+  protected long readLevelLength(IndexInput skipStream) throws IOException {
+    SimpleTextUtil.readLine(skipStream, scratch);
+    scratchUTF16.copyUTF8Bytes(
+        scratch.bytes(), LEVEL_LENGTH.length, scratch.length() - LEVEL_LENGTH.length);
+    return Long.parseLong(scratchUTF16.toString());
+  }
+
+  @Override
+  protected long readChildPointer(IndexInput skipStream) throws IOException {
+    SimpleTextUtil.readLine(skipStream, scratch);
+    scratchUTF16.copyUTF8Bytes(
+        scratch.bytes(), CHILD_POINTER.length, scratch.length() - CHILD_POINTER.length);
+    return Long.parseLong(scratchUTF16.toString());
+  }
+
+  void reset(long skipPointer, int docFreq) throws IOException {
+    init();
+    if (skipPointer > 0) {
+      super.init(skipPointer, docFreq);
+      hasSkipList = true;
+    }
+  }
+
+  private void init() {
+    nextSkipDocFP = -1;
+    numLevels = 1;
+    perLevelImpacts = new ArrayList<>(maxNumberOfSkipLevels);
+    for (int level = 0; level < maxNumberOfSkipLevels; level++) {
+      List<Impact> impacts = new ArrayList<>();
+      impacts.add(new Impact(Integer.MAX_VALUE, 1L));
+      perLevelImpacts.add(level, impacts);
+    }
+    hasSkipList = false;
+  }
+
+  Impacts getImpacts() {
+    return impacts;
+  }
+
+  long getNextSkipDocFP() {
+    return nextSkipDocFP;
+  }
+
+  int getNextSkipDoc() {
+    if (!hasSkipList) {
+      return DocIdSetIterator.NO_MORE_DOCS;
+    }
+    return skipDoc[0];
+  }
+
+  boolean hasSkipList() {
+    return hasSkipList;
+  }
+}
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSkipWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSkipWriter.java
new file mode 100644
index 0000000..95ef8ca
--- /dev/null
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSkipWriter.java
@@ -0,0 +1,157 @@
+/*
+ * 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.simpletext;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.codecs.CompetitiveImpactAccumulator;
+import org.apache.lucene.codecs.MultiLevelSkipListWriter;
+import org.apache.lucene.index.Impact;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+
+/**
+ * plain text skip data.
+ *
+ * @lucene.experimental
+ */
+class SimpleTextSkipWriter extends MultiLevelSkipListWriter {
+
+  static final int skipMultiplier = 3;
+  static final int maxSkipLevels = 4;
+
+  static final int BLOCK_SIZE = 8;
+  private Map<Integer, Boolean> wroteHeaderPerLevelMap = new HashMap<>();
+  private int curDoc;
+  private long curDocFilePointer;
+  private CompetitiveImpactAccumulator[] curCompetitiveFreqNorms;
+  private final BytesRefBuilder scratch = new BytesRefBuilder();
+
+  static final BytesRef SKIP_LIST = new BytesRef("    skipList ");
+  static final BytesRef LEVEL_LENGTH = new BytesRef("      levelLength ");
+  static final BytesRef LEVEL = new BytesRef("      level ");
+  static final BytesRef SKIP_DOC = new BytesRef("        skipDoc ");
+  static final BytesRef SKIP_DOC_FP = new BytesRef("        skipDocFP ");
+  static final BytesRef IMPACTS = new BytesRef("        impacts ");
+  static final BytesRef IMPACT = new BytesRef("          impact ");
+  static final BytesRef FREQ = new BytesRef("            freq ");
+  static final BytesRef NORM = new BytesRef("            norm ");
+  static final BytesRef IMPACTS_END = new BytesRef("        impactsEnd ");
+  static final BytesRef CHILD_POINTER = new BytesRef("        childPointer ");
+
+  SimpleTextSkipWriter(SegmentWriteState writeState) throws IOException {
+    super(BLOCK_SIZE, skipMultiplier, maxSkipLevels, writeState.segmentInfo.maxDoc());
+    curCompetitiveFreqNorms = new CompetitiveImpactAccumulator[maxSkipLevels];
+    for (int i = 0; i < maxSkipLevels; ++i) {
+      curCompetitiveFreqNorms[i] = new CompetitiveImpactAccumulator();
+    }
+    resetSkip();
+  }
+
+  @Override
+  protected void writeSkipData(int level, DataOutput skipBuffer) throws IOException {
+    Boolean wroteHeader = wroteHeaderPerLevelMap.get(level);
+    if (wroteHeader == null || !wroteHeader) {
+      SimpleTextUtil.write(skipBuffer, LEVEL);
+      SimpleTextUtil.write(skipBuffer, level + "", scratch);
+      SimpleTextUtil.writeNewline(skipBuffer);
+
+      wroteHeaderPerLevelMap.put(level, true);
+    }
+    SimpleTextUtil.write(skipBuffer, SKIP_DOC);
+    SimpleTextUtil.write(skipBuffer, curDoc + "", scratch);
+    SimpleTextUtil.writeNewline(skipBuffer);
+
+    SimpleTextUtil.write(skipBuffer, SKIP_DOC_FP);
+    SimpleTextUtil.write(skipBuffer, curDocFilePointer + "", scratch);
+    SimpleTextUtil.writeNewline(skipBuffer);
+
+    CompetitiveImpactAccumulator competitiveFreqNorms = curCompetitiveFreqNorms[level];
+    Collection<Impact> impacts = competitiveFreqNorms.getCompetitiveFreqNormPairs();
+    assert impacts.size() > 0;
+    if (level + 1 < numberOfSkipLevels) {
+      curCompetitiveFreqNorms[level + 1].addAll(competitiveFreqNorms);
+    }
+    SimpleTextUtil.write(skipBuffer, IMPACTS);
+    SimpleTextUtil.writeNewline(skipBuffer);
+    for (Impact impact : impacts) {
+      SimpleTextUtil.write(skipBuffer, IMPACT);
+      SimpleTextUtil.writeNewline(skipBuffer);
+      SimpleTextUtil.write(skipBuffer, FREQ);
+      SimpleTextUtil.write(skipBuffer, impact.freq + "", scratch);
+      SimpleTextUtil.writeNewline(skipBuffer);
+      SimpleTextUtil.write(skipBuffer, NORM);
+      SimpleTextUtil.write(skipBuffer, impact.norm + "", scratch);
+      SimpleTextUtil.writeNewline(skipBuffer);
+    }
+    SimpleTextUtil.write(skipBuffer, IMPACTS_END);
+    SimpleTextUtil.writeNewline(skipBuffer);
+    competitiveFreqNorms.clear();
+  }
+
+  @Override
+  protected void resetSkip() {
+    super.resetSkip();
+    wroteHeaderPerLevelMap.clear();
+    this.curDoc = -1;
+    this.curDocFilePointer = -1;
+    for (CompetitiveImpactAccumulator acc : curCompetitiveFreqNorms) {
+      acc.clear();
+    }
+  }
+
+  @Override
+  public long writeSkip(IndexOutput output) throws IOException {
+    long skipOffset = output.getFilePointer();
+    SimpleTextUtil.write(output, SKIP_LIST);
+    SimpleTextUtil.writeNewline(output);
+    super.writeSkip(output);
+    return skipOffset;
+  }
+
+  void bufferSkip(
+      int doc,
+      long docFilePointer,
+      int numDocs,
+      final CompetitiveImpactAccumulator competitiveImpactAccumulator)
+      throws IOException {
+    assert doc > curDoc;
+    this.curDoc = doc;
+    this.curDocFilePointer = docFilePointer;
+    this.curCompetitiveFreqNorms[0].addAll(competitiveImpactAccumulator);
+    bufferSkip(numDocs);
+  }
+
+  @Override
+  protected void writeLevelLength(long levelLength, IndexOutput output) throws IOException {
+    SimpleTextUtil.write(output, LEVEL_LENGTH);
+    SimpleTextUtil.write(output, levelLength + "", scratch);
+    SimpleTextUtil.writeNewline(output);
+  }
+
+  @Override
+  protected void writeChildPointer(long childPointer, DataOutput skipBuffer) throws IOException {
+    SimpleTextUtil.write(skipBuffer, CHILD_POINTER);
+    SimpleTextUtil.write(skipBuffer, childPointer + "", scratch);
+    SimpleTextUtil.writeNewline(skipBuffer);
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java b/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java
index 353fd37..d454240 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListReader.java
@@ -19,7 +19,9 @@ package org.apache.lucene.codecs;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Arrays;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.MathUtil;
 
 /**
@@ -162,7 +164,7 @@ public abstract class MultiLevelSkipListReader implements Closeable {
 
     if (level != 0) {
       // read the child pointer if we are not on the leaf level
-      childPointer[level] = skipStream[level].readVLong() + skipPointer[level - 1];
+      childPointer[level] = readChildPointer(skipStream[level]) + skipPointer[level - 1];
     }
 
     return true;
@@ -174,7 +176,7 @@ public abstract class MultiLevelSkipListReader implements Closeable {
     numSkipped[level] = numSkipped[level + 1] - skipInterval[level + 1];
     skipDoc[level] = lastDoc;
     if (level > 0) {
-      childPointer[level] = skipStream[level].readVLong() + skipPointer[level - 1];
+      childPointer[level] = readChildPointer(skipStream[level]) + skipPointer[level - 1];
     }
   }
 
@@ -221,7 +223,7 @@ public abstract class MultiLevelSkipListReader implements Closeable {
 
     for (int i = numberOfSkipLevels - 1; i > 0; i--) {
       // the length of the current level
-      long length = skipStream[0].readVLong();
+      long length = readLevelLength(skipStream[0]);
 
       // the start pointer of the current level
       skipPointer[i] = skipStream[0].getFilePointer();
@@ -250,6 +252,28 @@ public abstract class MultiLevelSkipListReader implements Closeable {
    */
   protected abstract int readSkipData(int level, IndexInput skipStream) throws IOException;
 
+  /**
+   * read the length of the current level written via {@link
+   * MultiLevelSkipListWriter#writeLevelLength(long, IndexOutput)}.
+   *
+   * @param skipStream the IndexInput the length shall be read from
+   * @return level length
+   */
+  protected long readLevelLength(IndexInput skipStream) throws IOException {
+    return skipStream.readVLong();
+  }
+
+  /**
+   * read the child pointer written via {@link MultiLevelSkipListWriter#writeChildPointer(long,
+   * DataOutput)}.
+   *
+   * @param skipStream the IndexInput the child pointer shall be read from
+   * @return child pointer
+   */
+  protected long readChildPointer(IndexInput skipStream) throws IOException {
+    return skipStream.readVLong();
+  }
+
   /** Copies the values of the last read skip entry on this level */
   protected void setLastSkipData(int level) {
     lastDoc = skipDoc[level];
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java
index 7a13a63..f592b7c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java
@@ -147,7 +147,7 @@ public abstract class MultiLevelSkipListWriter {
 
       if (level != 0) {
         // store child pointers for all levels except the lowest
-        skipBuffer[level].writeVLong(childPointer);
+        writeChildPointer(childPointer, skipBuffer[level]);
       }
 
       // remember the childPointer for the next level
@@ -169,7 +169,7 @@ public abstract class MultiLevelSkipListWriter {
     for (int level = numberOfSkipLevels - 1; level > 0; level--) {
       long length = skipBuffer[level].size();
       if (length > 0) {
-        output.writeVLong(length);
+        writeLevelLength(length, output);
         skipBuffer[level].copyTo(output);
       }
     }
@@ -177,4 +177,24 @@ public abstract class MultiLevelSkipListWriter {
 
     return skipPointer;
   }
+
+  /**
+   * Writes the length of a level to the given output.
+   *
+   * @param levelLength the length of a level
+   * @param output the IndexOutput the length shall be written to
+   */
+  protected void writeLevelLength(long levelLength, IndexOutput output) throws IOException {
+    output.writeVLong(levelLength);
+  }
+
+  /**
+   * Writes the child pointer of a block to the given output.
+   *
+   * @param childPointer block of higher level point to the lower level
+   * @param skipBuffer the skip buffer to write to
+   */
+  protected void writeChildPointer(long childPointer, DataOutput skipBuffer) throws IOException {
+    skipBuffer.writeVLong(childPointer);
+  }
 }