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 2020/02/05 18:55:12 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9147: Move the stored fields index off-heap. (#1179)

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 597141d  LUCENE-9147: Move the stored fields index off-heap. (#1179)
597141d is described below

commit 597141df6b6a017fced16ec27b8fd180e9a6fcc2
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Wed Feb 5 18:35:08 2020 +0100

    LUCENE-9147: Move the stored fields index off-heap. (#1179)
    
    This replaces the index of stored fields and term vectors with two
    `DirectMonotonic` arrays. `DirectMonotonicWriter` requires to know the number
    of values to write up-front, so incoming doc IDs and file pointers are buffered
    on disk using temporary files that never get fsynced, but have index headers
    and footers to make sure any corruption in these files wouldn't propagate to the
    index.
    
    `DirectMonotonicReader` gets a specialized `binarySearch` implementation that
    leverages the metadata in order to avoid going to the IndexInput as often as
    possible. Actually in the common case, it would only go to a single
    sub `DirectReader` which, combined with the size of blocks of 1k values, helps
    bound the number of page faults to 2.
---
 lucene/CHANGES.txt                                 |   3 +
 .../compressing/CompressingStoredFieldsFormat.java |  22 ++-
 .../CompressingStoredFieldsIndexWriter.java        | 213 ---------------------
 .../compressing/CompressingStoredFieldsReader.java |  90 ++++-----
 .../compressing/CompressingStoredFieldsWriter.java |  39 ++--
 .../compressing/CompressingTermVectorsReader.java  |  79 ++++----
 .../compressing/CompressingTermVectorsWriter.java  |  31 ++-
 .../lucene/codecs/compressing/FieldsIndex.java}    |  22 +--
 .../codecs/compressing/FieldsIndexReader.java      | 139 ++++++++++++++
 .../codecs/compressing/FieldsIndexWriter.java      | 206 ++++++++++++++++++++
 ...dexReader.java => LegacyFieldsIndexReader.java} |  15 +-
 .../lucene50/Lucene50StoredFieldsFormat.java       |  70 ++-----
 .../codecs/lucene50/Lucene50TermVectorsFormat.java |   6 +-
 .../index/TrackingTmpOutputDirectoryWrapper.java   |   3 +-
 .../lucene/util/packed/DirectMonotonicReader.java  | 112 +++++++++--
 .../org/apache/lucene/index/TestIndexWriter.java   |   4 +-
 .../lucene/store/TestFileSwitchDirectory.java      |   3 +-
 .../lucene/util/packed/TestDirectMonotonic.java    |  70 +++++++
 .../codecs/compressing/CompressingCodec.java       |  25 +--
 .../codecs/compressing/FastCompressingCodec.java   |   4 +-
 .../FastDecompressionCompressingCodec.java         |   4 +-
 .../HighCompressionCompressingCodec.java           |   8 +-
 .../compressing/dummy/DummyCompressingCodec.java   |   4 +-
 23 files changed, 710 insertions(+), 462 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 6a28efe..a3a59fd 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -68,6 +68,9 @@ Optimizations
 
 * LUCENE-9125: Optimize Automaton.step() with binary search and introduce Automaton.next(). (Bruno Roustant)
 
+* LUCENE-9147: The index of stored fields and term vectors in now off-heap.
+  (Adrien Grand)
+
 Bug Fixes
 ---------------------
 
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java
index 1b9f4b1..927865a 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java
@@ -28,6 +28,7 @@ import org.apache.lucene.index.MergePolicy;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
 
 
 /**
@@ -49,7 +50,7 @@ public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
   private final CompressionMode compressionMode;
   private final int chunkSize;
   private final int maxDocsPerChunk;
-  private final int blockSize;
+  private final int blockShift;
 
   /**
    * Create a new {@link CompressingStoredFieldsFormat} with an empty segment 
@@ -57,8 +58,8 @@ public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
    * 
    * @see CompressingStoredFieldsFormat#CompressingStoredFieldsFormat(String, String, CompressionMode, int, int, int)
    */
-  public CompressingStoredFieldsFormat(String formatName, CompressionMode compressionMode, int chunkSize, int maxDocsPerChunk, int blockSize) {
-    this(formatName, "", compressionMode, chunkSize, maxDocsPerChunk, blockSize);
+  public CompressingStoredFieldsFormat(String formatName, CompressionMode compressionMode, int chunkSize, int maxDocsPerChunk, int blockShift) {
+    this(formatName, "", compressionMode, chunkSize, maxDocsPerChunk, blockShift);
   }
   
   /**
@@ -93,11 +94,11 @@ public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
    * @param compressionMode the {@link CompressionMode} to use
    * @param chunkSize the minimum number of bytes of a single chunk of stored documents
    * @param maxDocsPerChunk the maximum number of documents in a single chunk
-   * @param blockSize the number of chunks to store in an index block
+   * @param blockShift the log in base 2 of number of chunks to store in an index block
    * @see CompressionMode
    */
   public CompressingStoredFieldsFormat(String formatName, String segmentSuffix, 
-                                       CompressionMode compressionMode, int chunkSize, int maxDocsPerChunk, int blockSize) {
+                                       CompressionMode compressionMode, int chunkSize, int maxDocsPerChunk, int blockShift) {
     this.formatName = formatName;
     this.segmentSuffix = segmentSuffix;
     this.compressionMode = compressionMode;
@@ -109,10 +110,11 @@ public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
       throw new IllegalArgumentException("maxDocsPerChunk must be >= 1");
     }
     this.maxDocsPerChunk = maxDocsPerChunk;
-    if (blockSize < 1) {
-      throw new IllegalArgumentException("blockSize must be >= 1");
+    if (blockShift < DirectMonotonicWriter.MIN_BLOCK_SHIFT || blockShift > DirectMonotonicWriter.MAX_BLOCK_SHIFT) {
+      throw new IllegalArgumentException("blockSize must be in " + DirectMonotonicWriter.MIN_BLOCK_SHIFT + "-" +
+          DirectMonotonicWriter.MAX_BLOCK_SHIFT + ", got " + blockShift);
     }
-    this.blockSize = blockSize;
+    this.blockShift = blockShift;
   }
 
   @Override
@@ -126,13 +128,13 @@ public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
   public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si,
       IOContext context) throws IOException {
     return new CompressingStoredFieldsWriter(directory, si, segmentSuffix, context,
-        formatName, compressionMode, chunkSize, maxDocsPerChunk, blockSize);
+        formatName, compressionMode, chunkSize, maxDocsPerChunk, blockShift);
   }
 
   @Override
   public String toString() {
     return getClass().getSimpleName() + "(compressionMode=" + compressionMode
-        + ", chunkSize=" + chunkSize + ", maxDocsPerChunk=" + maxDocsPerChunk + ", blockSize=" + blockSize + ")";
+        + ", chunkSize=" + chunkSize + ", maxDocsPerChunk=" + maxDocsPerChunk + ", blockShift=" + blockShift + ")";
   }
 
 }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java
deleted file mode 100644
index 0372f2c..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexWriter.java
+++ /dev/null
@@ -1,213 +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.compressing;
-
-
-import static org.apache.lucene.util.BitUtil.zigZagEncode;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.packed.PackedInts;
-
-/**
- * Efficient index format for block-based {@link Codec}s.
- * <p> This writer generates a file which can be loaded into memory using
- * memory-efficient data structures to quickly locate the block that contains
- * any document.
- * <p>In order to have a compact in-memory representation, for every block of
- * 1024 chunks, this index computes the average number of bytes per
- * chunk and for every chunk, only stores the difference between<ul>
- * <li>${chunk number} * ${average length of a chunk}</li>
- * <li>and the actual start offset of the chunk</li></ul>
- * <p>Data is written as follows:
- * <ul>
- * <li>PackedIntsVersion, &lt;Block&gt;<sup>BlockCount</sup>, BlocksEndMarker</li>
- * <li>PackedIntsVersion --&gt; {@link PackedInts#VERSION_CURRENT} as a {@link DataOutput#writeVInt VInt}</li>
- * <li>BlocksEndMarker --&gt; <tt>0</tt> as a {@link DataOutput#writeVInt VInt}, this marks the end of blocks since blocks are not allowed to start with <tt>0</tt></li>
- * <li>Block --&gt; BlockChunks, &lt;DocBases&gt;, &lt;StartPointers&gt;</li>
- * <li>BlockChunks --&gt; a {@link DataOutput#writeVInt VInt} which is the number of chunks encoded in the block</li>
- * <li>DocBases --&gt; DocBase, AvgChunkDocs, BitsPerDocBaseDelta, DocBaseDeltas</li>
- * <li>DocBase --&gt; first document ID of the block of chunks, as a {@link DataOutput#writeVInt VInt}</li>
- * <li>AvgChunkDocs --&gt; average number of documents in a single chunk, as a {@link DataOutput#writeVInt VInt}</li>
- * <li>BitsPerDocBaseDelta --&gt; number of bits required to represent a delta from the average using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a></li>
- * <li>DocBaseDeltas --&gt; {@link PackedInts packed} array of BlockChunks elements of BitsPerDocBaseDelta bits each, representing the deltas from the average doc base using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a>.</li>
- * <li>StartPointers --&gt; StartPointerBase, AvgChunkSize, BitsPerStartPointerDelta, StartPointerDeltas</li>
- * <li>StartPointerBase --&gt; the first start pointer of the block, as a {@link DataOutput#writeVLong VLong}</li>
- * <li>AvgChunkSize --&gt; the average size of a chunk of compressed documents, as a {@link DataOutput#writeVLong VLong}</li>
- * <li>BitsPerStartPointerDelta --&gt; number of bits required to represent a delta from the average using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a></li>
- * <li>StartPointerDeltas --&gt; {@link PackedInts packed} array of BlockChunks elements of BitsPerStartPointerDelta bits each, representing the deltas from the average start pointer using <a href="https://developers.google.com/protocol-buffers/docs/encoding#types">ZigZag encoding</a></li>
- * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
- * <p>Notes
- * <ul>
- * <li>For any block, the doc base of the n-th chunk can be restored with
- * <code>DocBase + AvgChunkDocs * n + DocBaseDeltas[n]</code>.</li>
- * <li>For any block, the start pointer of the n-th chunk can be restored with
- * <code>StartPointerBase + AvgChunkSize * n + StartPointerDeltas[n]</code>.</li>
- * <li>Once data is loaded into memory, you can lookup the start pointer of any
- * document chunk by performing two binary searches: a first one based on the values
- * of DocBase in order to find the right block, and then inside the block based
- * on DocBaseDeltas (by reconstructing the doc bases for every chunk).</li>
- * </ul>
- * @lucene.internal
- */
-public final class CompressingStoredFieldsIndexWriter implements Closeable {
-  
-  final IndexOutput fieldsIndexOut;
-  final int blockSize;
-  int totalDocs;
-  int blockDocs;
-  int blockChunks;
-  long firstStartPointer;
-  long maxStartPointer;
-  final int[] docBaseDeltas;
-  final long[] startPointerDeltas;
-
-  CompressingStoredFieldsIndexWriter(IndexOutput indexOutput, int blockSize) throws IOException {
-    if (blockSize <= 0) {
-      throw new IllegalArgumentException("blockSize must be positive");
-    }
-    this.blockSize = blockSize;
-    this.fieldsIndexOut = indexOutput;
-    reset();
-    totalDocs = 0;
-    docBaseDeltas = new int[blockSize];
-    startPointerDeltas = new long[blockSize];
-    fieldsIndexOut.writeVInt(PackedInts.VERSION_CURRENT);
-  }
-
-  private void reset() {
-    blockChunks = 0;
-    blockDocs = 0;
-    firstStartPointer = -1; // means unset
-  }
-
-  private void writeBlock() throws IOException {
-    assert blockChunks > 0;
-    fieldsIndexOut.writeVInt(blockChunks);
-
-    // The trick here is that we only store the difference from the average start
-    // pointer or doc base, this helps save bits per value.
-    // And in order to prevent a few chunks that would be far from the average to
-    // raise the number of bits per value for all of them, we only encode blocks
-    // of 1024 chunks at once
-    // See LUCENE-4512
-
-    // doc bases
-    final int avgChunkDocs;
-    if (blockChunks == 1) {
-      avgChunkDocs = 0;
-    } else {
-      avgChunkDocs = Math.round((float) (blockDocs - docBaseDeltas[blockChunks - 1]) / (blockChunks - 1));
-    }
-    fieldsIndexOut.writeVInt(totalDocs - blockDocs); // docBase
-    fieldsIndexOut.writeVInt(avgChunkDocs);
-    int docBase = 0;
-    long maxDelta = 0;
-    for (int i = 0; i < blockChunks; ++i) {
-      final int delta = docBase - avgChunkDocs * i;
-      maxDelta |= zigZagEncode(delta);
-      docBase += docBaseDeltas[i];
-    }
-
-    final int bitsPerDocBase = PackedInts.bitsRequired(maxDelta);
-    fieldsIndexOut.writeVInt(bitsPerDocBase);
-    PackedInts.Writer writer = PackedInts.getWriterNoHeader(fieldsIndexOut,
-        PackedInts.Format.PACKED, blockChunks, bitsPerDocBase, 1);
-    docBase = 0;
-    for (int i = 0; i < blockChunks; ++i) {
-      final long delta = docBase - avgChunkDocs * i;
-      assert PackedInts.bitsRequired(zigZagEncode(delta)) <= writer.bitsPerValue();
-      writer.add(zigZagEncode(delta));
-      docBase += docBaseDeltas[i];
-    }
-    writer.finish();
-
-    // start pointers
-    fieldsIndexOut.writeVLong(firstStartPointer);
-    final long avgChunkSize;
-    if (blockChunks == 1) {
-      avgChunkSize = 0;
-    } else {
-      avgChunkSize = (maxStartPointer - firstStartPointer) / (blockChunks - 1);
-    }
-    fieldsIndexOut.writeVLong(avgChunkSize);
-    long startPointer = 0;
-    maxDelta = 0;
-    for (int i = 0; i < blockChunks; ++i) {
-      startPointer += startPointerDeltas[i];
-      final long delta = startPointer - avgChunkSize * i;
-      maxDelta |= zigZagEncode(delta);
-    }
-
-    final int bitsPerStartPointer = PackedInts.bitsRequired(maxDelta);
-    fieldsIndexOut.writeVInt(bitsPerStartPointer);
-    writer = PackedInts.getWriterNoHeader(fieldsIndexOut, PackedInts.Format.PACKED,
-        blockChunks, bitsPerStartPointer, 1);
-    startPointer = 0;
-    for (int i = 0; i < blockChunks; ++i) {
-      startPointer += startPointerDeltas[i];
-      final long delta = startPointer - avgChunkSize * i;
-      assert PackedInts.bitsRequired(zigZagEncode(delta)) <= writer.bitsPerValue();
-      writer.add(zigZagEncode(delta));
-    }
-    writer.finish();
-  }
-
-  void writeIndex(int numDocs, long startPointer) throws IOException {
-    if (blockChunks == blockSize) {
-      writeBlock();
-      reset();
-    }
-
-    if (firstStartPointer == -1) {
-      firstStartPointer = maxStartPointer = startPointer;
-    }
-    assert firstStartPointer > 0 && startPointer >= firstStartPointer;
-
-    docBaseDeltas[blockChunks] = numDocs;
-    startPointerDeltas[blockChunks] = startPointer - maxStartPointer;
-
-    ++blockChunks;
-    blockDocs += numDocs;
-    totalDocs += numDocs;
-    maxStartPointer = startPointer;
-  }
-
-  void finish(int numDocs, long maxPointer) throws IOException {
-    if (numDocs != totalDocs) {
-      throw new IllegalStateException("Expected " + numDocs + " docs, but got " + totalDocs);
-    }
-    if (blockChunks > 0) {
-      writeBlock();
-    }
-    fieldsIndexOut.writeVInt(0); // end marker
-    fieldsIndexOut.writeVLong(maxPointer);
-    CodecUtil.writeFooter(fieldsIndexOut);
-  }
-
-  @Override
-  public void close() throws IOException {
-    fieldsIndexOut.close();
-  }
-
-}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
index 62508f8..1af4133 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
@@ -18,14 +18,13 @@ package org.apache.lucene.codecs.compressing;
 
 
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.BYTE_ARR;
-import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.CODEC_SFX_DAT;
-import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.CODEC_SFX_IDX;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.DAY;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.DAY_ENCODING;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.FIELDS_EXTENSION;
-import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.FIELDS_INDEX_EXTENSION;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.HOUR;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.HOUR_ENCODING;
+import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.INDEX_CODEC_NAME;
+import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.INDEX_EXTENSION_PREFIX;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.NUMERIC_DOUBLE;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.NUMERIC_FLOAT;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.NUMERIC_INT;
@@ -36,6 +35,7 @@ import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.TYPE_BITS;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.TYPE_MASK;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_CURRENT;
+import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_OFFHEAP_INDEX;
 import static org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter.VERSION_START;
 
 import java.io.EOFException;
@@ -77,7 +77,7 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
 
   private final int version;
   private final FieldInfos fieldInfos;
-  private final CompressingStoredFieldsIndexReader indexReader;
+  private final FieldsIndex indexReader;
   private final long maxPointer;
   private final IndexInput fieldsStream;
   private final int chunkSize;
@@ -118,42 +118,13 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
     boolean success = false;
     fieldInfos = fn;
     numDocs = si.maxDoc();
-    
-    int version = -1;
-    long maxPointer = -1;
-    CompressingStoredFieldsIndexReader indexReader = null;
-    
-    // Load the index into memory
-    final String indexName = IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_INDEX_EXTENSION);    
-    try (ChecksumIndexInput indexStream = d.openChecksumInput(indexName, context)) {
-      Throwable priorE = null;
-      try {
-        final String codecNameIdx = formatName + CODEC_SFX_IDX;
-        version = CodecUtil.checkIndexHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
-        assert CodecUtil.indexHeaderLength(codecNameIdx, segmentSuffix) == indexStream.getFilePointer();
-        indexReader = new CompressingStoredFieldsIndexReader(indexStream, si);
-        maxPointer = indexStream.readVLong();
-      } catch (Throwable exception) {
-        priorE = exception;
-      } finally {
-        CodecUtil.checkFooter(indexStream, priorE);
-      }
-    }
-    
-    this.version = version;
-    this.maxPointer = maxPointer;
-    this.indexReader = indexReader;
 
     final String fieldsStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION);
     try {
       // Open the data file and read metadata
       fieldsStream = d.openInput(fieldsStreamFN, context);
-      final String codecNameDat = formatName + CODEC_SFX_DAT;
-      final int fieldsVersion = CodecUtil.checkIndexHeader(fieldsStream, codecNameDat, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
-      if (version != fieldsVersion) {
-        throw new CorruptIndexException("Version mismatch between stored fields index and data: " + version + " != " + fieldsVersion, fieldsStream);
-      }
-      assert CodecUtil.indexHeaderLength(codecNameDat, segmentSuffix) == fieldsStream.getFilePointer();
+      version = CodecUtil.checkIndexHeader(fieldsStream, formatName, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
+      assert CodecUtil.indexHeaderLength(formatName, segmentSuffix) == fieldsStream.getFilePointer();
 
       chunkSize = fieldsStream.readVInt();
       packedIntsVersion = fieldsStream.readVInt();
@@ -161,6 +132,45 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
       this.merging = false;
       this.state = new BlockState();
 
+      // NOTE: data file is too costly to verify checksum against all the bytes on open,
+      // but for now we at least verify proper structure of the checksum footer: which looks
+      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
+      // such as file truncation.
+      CodecUtil.retrieveChecksum(fieldsStream);
+
+      long maxPointer = -1;
+      FieldsIndex indexReader = null;
+
+      if (version < VERSION_OFFHEAP_INDEX) {
+        // Load the index into memory
+        final String indexName = IndexFileNames.segmentFileName(segment, segmentSuffix, "fdx");
+        try (ChecksumIndexInput indexStream = d.openChecksumInput(indexName, context)) {
+          Throwable priorE = null;
+          try {
+            assert formatName.endsWith("Data");
+            final String codecNameIdx = formatName.substring(0, formatName.length() - "Data".length()) + "Index";
+            final int version2 = CodecUtil.checkIndexHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
+            if (version != version2) {
+              throw new CorruptIndexException("Version mismatch between stored fields index and data: " + version2 + " != " + version, indexStream);
+            }
+            assert CodecUtil.indexHeaderLength(codecNameIdx, segmentSuffix) == indexStream.getFilePointer();
+            indexReader = new LegacyFieldsIndexReader(indexStream, si);
+            maxPointer = indexStream.readVLong();
+          } catch (Throwable exception) {
+            priorE = exception;
+          } finally {
+            CodecUtil.checkFooter(indexStream, priorE);
+          }
+        }
+      } else {
+        FieldsIndexReader fieldsIndexReader = new FieldsIndexReader(d, si.name, segmentSuffix, INDEX_EXTENSION_PREFIX, INDEX_CODEC_NAME, si.getId());
+        indexReader = fieldsIndexReader;
+        maxPointer = fieldsIndexReader.getMaxPointer();
+      }
+
+      this.maxPointer = maxPointer;
+      this.indexReader = indexReader;
+
       fieldsStream.seek(maxPointer);
       numChunks = fieldsStream.readVLong();
       numDirtyChunks = fieldsStream.readVLong();
@@ -168,12 +178,6 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
         throw new CorruptIndexException("invalid chunk counts: dirty=" + numDirtyChunks + ", total=" + numChunks, fieldsStream);
       }
 
-      // NOTE: data file is too costly to verify checksum against all the bytes on open,
-      // but for now we at least verify proper structure of the checksum footer: which looks
-      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
-      // such as file truncation.
-      CodecUtil.retrieveChecksum(fieldsStream);
-
       success = true;
     } finally {
       if (!success) {
@@ -197,7 +201,7 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
   @Override
   public void close() throws IOException {
     if (!closed) {
-      IOUtils.close(fieldsStream);
+      IOUtils.close(indexReader, fieldsStream);
       closed = true;
     }
   }
@@ -621,7 +625,7 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
     return compressionMode;
   }
   
-  CompressingStoredFieldsIndexReader getIndexReader() {
+  FieldsIndex getIndexReader() {
     return indexReader;
   }
   
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
index 5b8eb9e..7d3ef38 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
@@ -56,9 +56,10 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
 
   /** Extension of stored fields file */
   public static final String FIELDS_EXTENSION = "fdt";
-  
-  /** Extension of stored fields index file */
-  public static final String FIELDS_INDEX_EXTENSION = "fdx";
+  /** Extension of stored fields index */
+  public static final String INDEX_EXTENSION_PREFIX = "fd";
+  /** Codec name for the index. */
+  public static final String INDEX_CODEC_NAME = "Lucene85FieldsIndex";
 
   static final int         STRING = 0x00;
   static final int       BYTE_ARR = 0x01;
@@ -70,13 +71,12 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
   static final int TYPE_BITS = PackedInts.bitsRequired(NUMERIC_DOUBLE);
   static final int TYPE_MASK = (int) PackedInts.maxValue(TYPE_BITS);
 
-  static final String CODEC_SFX_IDX = "Index";
-  static final String CODEC_SFX_DAT = "Data";
   static final int VERSION_START = 1;
-  static final int VERSION_CURRENT = VERSION_START;
+  static final int VERSION_OFFHEAP_INDEX = 2;
+  static final int VERSION_CURRENT = VERSION_OFFHEAP_INDEX;
 
   private final String segment;
-  private CompressingStoredFieldsIndexWriter indexWriter;
+  private FieldsIndexWriter indexWriter;
   private IndexOutput fieldsStream;
 
   private Compressor compressor;
@@ -94,8 +94,8 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
   private long numDirtyChunks; // number of incomplete compressed blocks written
 
   /** Sole constructor. */
-  public CompressingStoredFieldsWriter(Directory directory, SegmentInfo si, String segmentSuffix, IOContext context,
-      String formatName, CompressionMode compressionMode, int chunkSize, int maxDocsPerChunk, int blockSize) throws IOException {
+  CompressingStoredFieldsWriter(Directory directory, SegmentInfo si, String segmentSuffix, IOContext context,
+      String formatName, CompressionMode compressionMode, int chunkSize, int maxDocsPerChunk, int blockShift) throws IOException {
     assert directory != null;
     this.segment = si.name;
     this.compressionMode = compressionMode;
@@ -109,21 +109,12 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
     this.numBufferedDocs = 0;
 
     boolean success = false;
-    IndexOutput indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_INDEX_EXTENSION), 
-                                                                     context);
     try {
-      fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION),
-                                                    context);
-
-      final String codecNameIdx = formatName + CODEC_SFX_IDX;
-      final String codecNameDat = formatName + CODEC_SFX_DAT;
-      CodecUtil.writeIndexHeader(indexStream, codecNameIdx, VERSION_CURRENT, si.getId(), segmentSuffix);
-      CodecUtil.writeIndexHeader(fieldsStream, codecNameDat, VERSION_CURRENT, si.getId(), segmentSuffix);
-      assert CodecUtil.indexHeaderLength(codecNameDat, segmentSuffix) == fieldsStream.getFilePointer();
-      assert CodecUtil.indexHeaderLength(codecNameIdx, segmentSuffix) == indexStream.getFilePointer();
+      fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION), context);
+      CodecUtil.writeIndexHeader(fieldsStream, formatName, VERSION_CURRENT, si.getId(), segmentSuffix);
+      assert CodecUtil.indexHeaderLength(formatName, segmentSuffix) == fieldsStream.getFilePointer();
 
-      indexWriter = new CompressingStoredFieldsIndexWriter(indexStream, blockSize);
-      indexStream = null;
+      indexWriter = new FieldsIndexWriter(directory, segment, segmentSuffix, INDEX_EXTENSION_PREFIX, INDEX_CODEC_NAME, si.getId(), blockShift, context);
 
       fieldsStream.writeVInt(chunkSize);
       fieldsStream.writeVInt(PackedInts.VERSION_CURRENT);
@@ -131,7 +122,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
       success = true;
     } finally {
       if (!success) {
-        IOUtils.closeWhileHandlingException(fieldsStream, indexStream, indexWriter);
+        IOUtils.closeWhileHandlingException(fieldsStream, indexWriter);
       }
     }
   }
@@ -583,7 +574,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
         // read the docstart + doccount from the chunk header (we write a new header, since doc numbers will change),
         // and just copy the bytes directly.
         IndexInput rawDocs = matchingFieldsReader.getFieldsStream();
-        CompressingStoredFieldsIndexReader index = matchingFieldsReader.getIndexReader();
+        FieldsIndex index = matchingFieldsReader.getIndexReader();
         rawDocs.seek(index.getStartPointer(0));
         int docID = 0;
         while (docID < maxDoc) {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
index 866899c..4b045f9 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
@@ -53,15 +53,15 @@ import org.apache.lucene.util.LongsRef;
 import org.apache.lucene.util.packed.BlockPackedReaderIterator;
 import org.apache.lucene.util.packed.PackedInts;
 
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.CODEC_SFX_DAT;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.CODEC_SFX_IDX;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_OFFHEAP_INDEX;
 import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.FLAGS_BITS;
 import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.OFFSETS;
 import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.PACKED_BLOCK_SIZE;
 import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.PAYLOADS;
 import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.POSITIONS;
 import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_EXTENSION;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_INDEX_EXTENSION;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_INDEX_CODEC_NAME;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_INDEX_EXTENSION_PREFIX;
 import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_CURRENT;
 import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_START;
 
@@ -72,7 +72,7 @@ import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.
 public final class CompressingTermVectorsReader extends TermVectorsReader implements Closeable {
 
   private final FieldInfos fieldInfos;
-  final CompressingStoredFieldsIndexReader indexReader;
+  final FieldsIndex indexReader;
   final IndexInput vectorsStream;
   private final int version;
   private final int packedIntsVersion;
@@ -112,45 +112,48 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
     boolean success = false;
     fieldInfos = fn;
     numDocs = si.maxDoc();
-    int version = -1;
-    CompressingStoredFieldsIndexReader indexReader = null;
-    
-    long maxPointer = -1;
-    
-    // Load the index into memory
-    final String indexName = IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_INDEX_EXTENSION);
-    try (ChecksumIndexInput input = d.openChecksumInput(indexName, context)) {
-      Throwable priorE = null;
-      try {
-        final String codecNameIdx = formatName + CODEC_SFX_IDX;
-        version = CodecUtil.checkIndexHeader(input, codecNameIdx, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
-        assert CodecUtil.indexHeaderLength(codecNameIdx, segmentSuffix) == input.getFilePointer();
-        indexReader = new CompressingStoredFieldsIndexReader(input, si);
-        maxPointer = input.readVLong(); // the end of the data section
-      } catch (Throwable exception) {
-        priorE = exception;
-      } finally {
-        CodecUtil.checkFooter(input, priorE);
-      }
-    }
-    
-    this.version = version;
-    this.indexReader = indexReader;
-    this.maxPointer = maxPointer;
 
     try {
       // Open the data file and read metadata
       final String vectorsStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION);
       vectorsStream = d.openInput(vectorsStreamFN, context);
-      final String codecNameDat = formatName + CODEC_SFX_DAT;
-      int version2 = CodecUtil.checkIndexHeader(vectorsStream, codecNameDat, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
-      if (version != version2) {
-        throw new CorruptIndexException("Version mismatch between stored fields index and data: " + version + " != " + version2, vectorsStream);
+      version = CodecUtil.checkIndexHeader(vectorsStream, formatName, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
+      assert CodecUtil.indexHeaderLength(formatName, segmentSuffix) == vectorsStream.getFilePointer();
+
+      FieldsIndex indexReader = null;
+      long maxPointer = -1;
+
+      if (version < VERSION_OFFHEAP_INDEX) {
+        // Load the index into memory
+        final String indexName = IndexFileNames.segmentFileName(segment, segmentSuffix, "tvx");
+        try (ChecksumIndexInput indexStream = d.openChecksumInput(indexName, context)) {
+          Throwable priorE = null;
+          try {
+            assert formatName.endsWith("Data");
+            final String codecNameIdx = formatName.substring(0, formatName.length() - "Data".length()) + "Index";
+            final int version2 = CodecUtil.checkIndexHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
+            if (version != version2) {
+              throw new CorruptIndexException("Version mismatch between stored fields index and data: " + version + " != " + version2, indexStream);
+            }
+            assert CodecUtil.indexHeaderLength(codecNameIdx, segmentSuffix) == indexStream.getFilePointer();
+            indexReader = new LegacyFieldsIndexReader(indexStream, si);
+            maxPointer = indexStream.readVLong(); // the end of the data section
+          } catch (Throwable exception) {
+            priorE = exception;
+          } finally {
+            CodecUtil.checkFooter(indexStream, priorE);
+          }
+        }
+      } else {
+        FieldsIndexReader fieldsIndexReader = new FieldsIndexReader(d, si.name, segmentSuffix, VECTORS_INDEX_EXTENSION_PREFIX, VECTORS_INDEX_CODEC_NAME, si.getId());
+        indexReader = fieldsIndexReader;
+        maxPointer = fieldsIndexReader.getMaxPointer();
       }
-      assert CodecUtil.indexHeaderLength(codecNameDat, segmentSuffix) == vectorsStream.getFilePointer();
-      
-      long pos = vectorsStream.getFilePointer();
 
+      this.indexReader = indexReader;
+      this.maxPointer = maxPointer;
+
+      long pos = vectorsStream.getFilePointer();
       vectorsStream.seek(maxPointer);
       numChunks = vectorsStream.readVLong();
       numDirtyChunks = vectorsStream.readVLong();
@@ -194,7 +197,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
     return version;
   }
 
-  CompressingStoredFieldsIndexReader getIndexReader() {
+  FieldsIndex getIndexReader() {
     return indexReader;
   }
 
@@ -226,7 +229,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
   @Override
   public void close() throws IOException {
     if (!closed) {
-      IOUtils.close(vectorsStream);
+      IOUtils.close(indexReader, vectorsStream);
       closed = true;
     }
   }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
index 6156fd5..def9d52 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
@@ -59,13 +59,12 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
   static final int MAX_DOCUMENTS_PER_CHUNK = 128;
 
   static final String VECTORS_EXTENSION = "tvd";
-  static final String VECTORS_INDEX_EXTENSION = "tvx";
-
-  static final String CODEC_SFX_IDX = "Index";
-  static final String CODEC_SFX_DAT = "Data";
+  static final String VECTORS_INDEX_EXTENSION_PREFIX = "tv";
+  static final String VECTORS_INDEX_CODEC_NAME = "Lucene85TermVectorsIndex";
 
   static final int VERSION_START = 1;
-  static final int VERSION_CURRENT = VERSION_START;
+  static final int VERSION_OFFHEAP_INDEX = 2;
+  static final int VERSION_CURRENT = VERSION_OFFHEAP_INDEX;
 
   static final int PACKED_BLOCK_SIZE = 64;
 
@@ -75,7 +74,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
   static final int FLAGS_BITS = PackedInts.bitsRequired(POSITIONS | OFFSETS | PAYLOADS);
 
   private final String segment;
-  private CompressingStoredFieldsIndexWriter indexWriter;
+  private FieldsIndexWriter indexWriter;
   private IndexOutput vectorsStream;
 
   private final CompressionMode compressionMode;
@@ -204,7 +203,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
 
   /** Sole constructor. */
   public CompressingTermVectorsWriter(Directory directory, SegmentInfo si, String segmentSuffix, IOContext context,
-      String formatName, CompressionMode compressionMode, int chunkSize, int blockSize) throws IOException {
+      String formatName, CompressionMode compressionMode, int chunkSize, int blockShift) throws IOException {
     assert directory != null;
     this.segment = si.name;
     this.compressionMode = compressionMode;
@@ -218,21 +217,13 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
     lastTerm = new BytesRef(ArrayUtil.oversize(30, 1));
 
     boolean success = false;
-    IndexOutput indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_INDEX_EXTENSION), 
-                                                                     context);
     try {
       vectorsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION),
                                                      context);
+      CodecUtil.writeIndexHeader(vectorsStream, formatName, VERSION_CURRENT, si.getId(), segmentSuffix);
+      assert CodecUtil.indexHeaderLength(formatName, segmentSuffix) == vectorsStream.getFilePointer();
 
-      final String codecNameIdx = formatName + CODEC_SFX_IDX;
-      final String codecNameDat = formatName + CODEC_SFX_DAT;
-      CodecUtil.writeIndexHeader(indexStream, codecNameIdx, VERSION_CURRENT, si.getId(), segmentSuffix);
-      CodecUtil.writeIndexHeader(vectorsStream, codecNameDat, VERSION_CURRENT, si.getId(), segmentSuffix);
-      assert CodecUtil.indexHeaderLength(codecNameDat, segmentSuffix) == vectorsStream.getFilePointer();
-      assert CodecUtil.indexHeaderLength(codecNameIdx, segmentSuffix) == indexStream.getFilePointer();
-
-      indexWriter = new CompressingStoredFieldsIndexWriter(indexStream, blockSize);
-      indexStream = null;
+      indexWriter = new FieldsIndexWriter(directory, segment, segmentSuffix, VECTORS_INDEX_EXTENSION_PREFIX, VECTORS_INDEX_CODEC_NAME, si.getId(), blockShift, context);
 
       vectorsStream.writeVInt(PackedInts.VERSION_CURRENT);
       vectorsStream.writeVInt(chunkSize);
@@ -246,7 +237,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
       success = true;
     } finally {
       if (!success) {
-        IOUtils.closeWhileHandlingException(vectorsStream, indexStream, indexWriter);
+        IOUtils.closeWhileHandlingException(vectorsStream, indexWriter, indexWriter);
       }
     }
   }
@@ -774,7 +765,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
         // read the docstart + doccount from the chunk header (we write a new header, since doc numbers will change),
         // and just copy the bytes directly.
         IndexInput rawDocs = matchingVectorsReader.getVectorsStream();
-        CompressingStoredFieldsIndexReader index = matchingVectorsReader.getIndexReader();
+        FieldsIndex index = matchingVectorsReader.getIndexReader();
         rawDocs.seek(index.getStartPointer(0));
         int docID = 0;
         while (docID < maxDoc) {
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/FieldsIndex.java
similarity index 59%
copy from lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java
copy to lucene/core/src/java/org/apache/lucene/codecs/compressing/FieldsIndex.java
index 007948c..e86e749 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/FieldsIndex.java
@@ -16,18 +16,16 @@
  */
 package org.apache.lucene.codecs.compressing;
 
-/** CompressionCodec that uses {@link CompressionMode#FAST} */
-public class FastCompressingCodec extends CompressingCodec {
+import java.io.Closeable;
 
-  /** Constructor that allows to configure the chunk size. */
-  public FastCompressingCodec(int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockSize) {
-    super("FastCompressingStoredFields", 
-          withSegmentSuffix ? "FastCompressingStoredFields" : "",
-          CompressionMode.FAST, chunkSize, maxDocsPerChunk, blockSize);
-  }
+import org.apache.lucene.util.Accountable;
+
+abstract class FieldsIndex implements Accountable, Cloneable, Closeable {
+
+  /** Get the start pointer for the block that contains the given docID. */
+  abstract long getStartPointer(int docID);
+
+  @Override
+  public abstract FieldsIndex clone();
 
-  /** Default constructor. */
-  public FastCompressingCodec() {
-    this(1 << 14, 128, false, 1024);
-  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/FieldsIndexReader.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/FieldsIndexReader.java
new file mode 100644
index 0000000..566c49c
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/FieldsIndexReader.java
@@ -0,0 +1,139 @@
+/*
+ * 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.compressing;
+
+import static org.apache.lucene.codecs.compressing.FieldsIndexWriter.FIELDS_INDEX_EXTENSION_SUFFIX;
+import static org.apache.lucene.codecs.compressing.FieldsIndexWriter.FIELDS_META_EXTENSION_SUFFIX;
+import static org.apache.lucene.codecs.compressing.FieldsIndexWriter.VERSION_CURRENT;
+import static org.apache.lucene.codecs.compressing.FieldsIndexWriter.VERSION_START;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.FutureObjects;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.packed.DirectMonotonicReader;
+
+final class FieldsIndexReader extends FieldsIndex {
+
+  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FieldsIndexReader.class);
+
+  private final int maxDoc;
+  private final int blockShift;
+  private final int numChunks;
+  private final DirectMonotonicReader.Meta docsMeta;
+  private final DirectMonotonicReader.Meta startPointersMeta;
+  private final IndexInput indexInput;
+  private final long docsStartPointer, docsEndPointer, startPointersStartPointer, startPointersEndPointer;
+  private final DirectMonotonicReader docs, startPointers;
+  private final long maxPointer;
+
+  FieldsIndexReader(Directory dir, String name, String suffix, String extensionPrefix, String codecName, byte[] id) throws IOException {
+    try (ChecksumIndexInput metaIn = dir.openChecksumInput(IndexFileNames.segmentFileName(name, suffix, extensionPrefix + FIELDS_META_EXTENSION_SUFFIX), IOContext.READONCE)) {
+      Throwable priorE = null;
+      try {
+        CodecUtil.checkIndexHeader(metaIn, codecName + "Meta", VERSION_START, VERSION_CURRENT, id, suffix);
+        maxDoc = metaIn.readInt();
+        blockShift = metaIn.readInt();
+        numChunks = metaIn.readInt();
+        docsStartPointer = metaIn.readLong();
+        docsMeta = DirectMonotonicReader.loadMeta(metaIn, numChunks, blockShift);
+        docsEndPointer = startPointersStartPointer = metaIn.readLong();
+        startPointersMeta = DirectMonotonicReader.loadMeta(metaIn, numChunks, blockShift);
+        startPointersEndPointer = metaIn.readLong();
+        maxPointer = metaIn.readLong();
+      } finally {
+        CodecUtil.checkFooter(metaIn, priorE);
+      }
+    }
+
+    indexInput = dir.openInput(IndexFileNames.segmentFileName(name, suffix, extensionPrefix + FIELDS_INDEX_EXTENSION_SUFFIX), IOContext.READ);
+    boolean success = false;
+    try {
+      CodecUtil.checkIndexHeader(indexInput, codecName + "Idx", VERSION_START, VERSION_CURRENT, id, suffix);
+      CodecUtil.retrieveChecksum(indexInput);
+      success = true;
+    } finally {
+      if (success == false) {
+        indexInput.close();
+      }
+    }
+    final RandomAccessInput docsSlice = indexInput.randomAccessSlice(docsStartPointer, docsEndPointer - docsStartPointer);
+    final RandomAccessInput startPointersSlice = indexInput.randomAccessSlice(startPointersStartPointer, startPointersEndPointer - startPointersStartPointer);
+    docs = DirectMonotonicReader.getInstance(docsMeta, docsSlice);
+    startPointers = DirectMonotonicReader.getInstance(startPointersMeta, startPointersSlice);
+  }
+
+  private FieldsIndexReader(FieldsIndexReader other) throws IOException {
+    maxDoc = other.maxDoc;
+    numChunks = other.numChunks;
+    blockShift = other.blockShift;
+    docsMeta = other.docsMeta;
+    startPointersMeta = other.startPointersMeta;
+    indexInput = other.indexInput.clone();
+    docsStartPointer = other.docsStartPointer;
+    docsEndPointer = other.docsEndPointer;
+    startPointersStartPointer = other.startPointersStartPointer;
+    startPointersEndPointer = other.startPointersEndPointer;
+    maxPointer = other.maxPointer;
+    final RandomAccessInput docsSlice = indexInput.randomAccessSlice(docsStartPointer, docsEndPointer - docsStartPointer);
+    final RandomAccessInput startPointersSlice = indexInput.randomAccessSlice(startPointersStartPointer, startPointersEndPointer - startPointersStartPointer);
+    docs = DirectMonotonicReader.getInstance(docsMeta, docsSlice);
+    startPointers = DirectMonotonicReader.getInstance(startPointersMeta, startPointersSlice);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return BASE_RAM_BYTES_USED + docsMeta.ramBytesUsed() + startPointersMeta.ramBytesUsed() +
+        docs.ramBytesUsed() + startPointers.ramBytesUsed();
+  }
+
+  @Override
+  public void close() throws IOException {
+    indexInput.close();
+  }
+
+  @Override
+  long getStartPointer(int docID) {
+    FutureObjects.checkIndex(docID, maxDoc);
+    long blockIndex = docs.binarySearch(0, numChunks, docID);
+    if (blockIndex < 0) {
+      blockIndex = -2 - blockIndex;
+    }
+    return startPointers.get(blockIndex);
+  }
+
+  @Override
+  public FieldsIndex clone() {
+    try {
+      return new FieldsIndexReader(this);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  public long getMaxPointer() {
+    return maxPointer;
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/FieldsIndexWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/FieldsIndexWriter.java
new file mode 100644
index 0000000..e67195e
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/FieldsIndexWriter.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.compressing;
+
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.packed.DirectMonotonicReader;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
+
+/**
+ * Efficient index format for block-based {@link Codec}s.
+ * <p>For each block of compressed stored fields, this stores the first document
+ * of the block and the start pointer of the block in a
+ * {@link DirectMonotonicWriter}. At read time, the docID is binary-searched in
+ * the {@link DirectMonotonicReader} that records doc IDS, and the returned
+ * index is used to look up the start pointer in the
+ * {@link DirectMonotonicReader} that records start pointers.
+ * @lucene.internal
+ */
+public final class FieldsIndexWriter implements Closeable {
+
+  /** Extension of stored fields index file. */
+  public static final String FIELDS_INDEX_EXTENSION_SUFFIX = "x";
+
+  /** Extension of stored fields meta file. */
+  public static final String FIELDS_META_EXTENSION_SUFFIX = "m";
+
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = 0;
+
+  private final Directory dir;
+  private final String name;
+  private final String suffix;
+  private final String extension;
+  private final String codecName;
+  private final byte[] id;
+  private final int blockShift;
+  private final IOContext ioContext;
+  private IndexOutput docsOut;
+  private IndexOutput filePointersOut;
+  private int totalDocs;
+  private int totalChunks;
+  private long previousFP;
+
+  FieldsIndexWriter(Directory dir, String name, String suffix, String extension,
+      String codecName, byte[] id, int blockShift, IOContext ioContext) throws IOException {
+    this.dir = dir;
+    this.name = name;
+    this.suffix = suffix;
+    this.extension = extension;
+    this.codecName = codecName;
+    this.id = id;
+    this.blockShift = blockShift;
+    this.ioContext = ioContext;
+    this.docsOut = dir.createTempOutput(name, codecName + "-doc_ids", ioContext);
+    IndexOutput filePointersOut = null;
+    boolean success = false;
+    try {
+      CodecUtil.writeHeader(docsOut, codecName + "Docs", VERSION_CURRENT);
+      this.filePointersOut = filePointersOut = dir.createTempOutput(name, codecName + "file_pointers", ioContext);
+      CodecUtil.writeHeader(filePointersOut, codecName + "FilePointers", VERSION_CURRENT);
+      success = true;
+    } finally {
+      if (success == false) {
+        docsOut.close();
+        dir.deleteFile(docsOut.getName());
+        if (filePointersOut != null) {
+          filePointersOut.close();
+          dir.deleteFile(filePointersOut.getName());
+        }
+      }
+    }
+  }
+
+  void writeIndex(int numDocs, long startPointer) throws IOException {
+    assert startPointer >= previousFP;
+    docsOut.writeVInt(numDocs);
+    filePointersOut.writeVLong(startPointer - previousFP);
+    previousFP = startPointer;
+    totalDocs += numDocs;
+    totalChunks++;
+  }
+
+  void finish(int numDocs, long maxPointer) throws IOException {
+    if (numDocs != totalDocs) {
+      throw new IllegalStateException("Expected " + numDocs + " docs, but got " + totalDocs);
+    }
+    CodecUtil.writeFooter(docsOut);
+    CodecUtil.writeFooter(filePointersOut);
+    IOUtils.close(docsOut, filePointersOut);
+
+    try (IndexOutput metaOut = dir.createOutput(IndexFileNames.segmentFileName(name, suffix, extension + FIELDS_META_EXTENSION_SUFFIX), ioContext);
+        IndexOutput dataOut = dir.createOutput(IndexFileNames.segmentFileName(name, suffix, extension + FIELDS_INDEX_EXTENSION_SUFFIX), ioContext)) {
+
+      CodecUtil.writeIndexHeader(metaOut, codecName + "Meta", VERSION_CURRENT, id, suffix);
+      CodecUtil.writeIndexHeader(dataOut, codecName + "Idx", VERSION_CURRENT, id, suffix);
+
+      metaOut.writeInt(numDocs);
+      metaOut.writeInt(blockShift);
+      metaOut.writeInt(totalChunks + 1);
+      metaOut.writeLong(dataOut.getFilePointer());
+
+      try (ChecksumIndexInput docsIn = dir.openChecksumInput(docsOut.getName(), IOContext.READONCE)) {
+        CodecUtil.checkHeader(docsIn, codecName + "Docs", VERSION_CURRENT, VERSION_CURRENT);
+        Throwable priorE = null;
+        try {
+          final DirectMonotonicWriter docs = DirectMonotonicWriter.getInstance(metaOut, dataOut, totalChunks + 1, blockShift);
+          long doc = 0;
+          docs.add(doc);
+          for (int i = 0; i < totalChunks; ++i) {
+            doc += docsIn.readVInt();
+            docs.add(doc);
+          }
+          docs.finish();
+          if (doc != totalDocs) {
+            throw new CorruptIndexException("Docs don't add up", docsIn);
+          }
+        } catch (Throwable e) {
+          priorE = e;
+        } finally {
+          CodecUtil.checkFooter(docsIn, priorE);
+        }
+      }
+      dir.deleteFile(docsOut.getName());
+      docsOut = null;
+
+      metaOut.writeLong(dataOut.getFilePointer());
+      try (ChecksumIndexInput filePointersIn = dir.openChecksumInput(filePointersOut.getName(), IOContext.READONCE)) {
+        CodecUtil.checkHeader(filePointersIn, codecName + "FilePointers", VERSION_CURRENT, VERSION_CURRENT);
+        Throwable priorE = null;
+        try {
+          final DirectMonotonicWriter filePointers = DirectMonotonicWriter.getInstance(metaOut, dataOut, totalChunks + 1, blockShift);
+          long fp = 0;
+          for (int i = 0; i < totalChunks; ++i) {
+            fp += filePointersIn.readVLong();
+            filePointers.add(fp);
+          }
+          if (maxPointer < fp) {
+            throw new CorruptIndexException("File pointers don't add up", filePointersIn);
+          }
+          filePointers.add(maxPointer);
+          filePointers.finish();
+        } catch (Throwable e) {
+          priorE = e;
+        } finally {
+          CodecUtil.checkFooter(filePointersIn, priorE);
+        }
+      }
+      dir.deleteFile(filePointersOut.getName());
+      filePointersOut = null;
+
+      metaOut.writeLong(dataOut.getFilePointer());
+      metaOut.writeLong(maxPointer);
+
+      CodecUtil.writeFooter(metaOut);
+      CodecUtil.writeFooter(dataOut);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      IOUtils.close(docsOut, filePointersOut);
+    } finally {
+      List<String> fileNames = new ArrayList<>();
+      if (docsOut != null) {
+        fileNames.add(docsOut.getName());
+      }
+      if (filePointersOut != null) {
+        fileNames.add(filePointersOut.getName());
+      }
+      try {
+        IOUtils.deleteFiles(dir, fileNames);
+      } finally {
+        docsOut = filePointersOut = null;
+      }
+    }
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/LegacyFieldsIndexReader.java
similarity index 95%
rename from lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java
rename to lucene/core/src/java/org/apache/lucene/codecs/compressing/LegacyFieldsIndexReader.java
index 61410b6..3e0bc48 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsIndexReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/LegacyFieldsIndexReader.java
@@ -35,12 +35,12 @@ import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.packed.PackedInts;
 
 /**
- * Random-access reader for {@link CompressingStoredFieldsIndexWriter}.
+ * Random-access reader for {@link FieldsIndexWriter}.
  * @lucene.internal
  */
-public final class CompressingStoredFieldsIndexReader implements Cloneable, Accountable {
+final class LegacyFieldsIndexReader extends FieldsIndex {
 
-  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(CompressingStoredFieldsIndexReader.class);
+  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(LegacyFieldsIndexReader.class);
 
   final int maxDoc;
   final int[] docBases;
@@ -52,7 +52,7 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable, Acco
 
   // It is the responsibility of the caller to close fieldsIndexIn after this constructor
   // has been called
-  CompressingStoredFieldsIndexReader(IndexInput fieldsIndexIn, SegmentInfo si) throws IOException {
+  LegacyFieldsIndexReader(IndexInput fieldsIndexIn, SegmentInfo si) throws IOException {
     maxDoc = si.maxDoc();
     int[] docBases = new int[16];
     long[] startPointers = new long[16];
@@ -163,7 +163,7 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable, Acco
   }
 
   @Override
-  public CompressingStoredFieldsIndexReader clone() {
+  public LegacyFieldsIndexReader clone() {
     return this;
   }
 
@@ -211,4 +211,9 @@ public final class CompressingStoredFieldsIndexReader implements Cloneable, Acco
   public String toString() {
     return getClass().getSimpleName() + "(blocks=" + docBases.length + ")";
   }
+
+  @Override
+  public void close() throws IOException {
+    // nothing to do
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java
index fdfba5b..ee91c9c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java
@@ -20,20 +20,17 @@ package org.apache.lucene.codecs.lucene50;
 import java.io.IOException;
 import java.util.Objects;
 
-import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.codecs.compressing.CompressingStoredFieldsFormat;
-import org.apache.lucene.codecs.compressing.CompressingStoredFieldsIndexWriter;
 import org.apache.lucene.codecs.compressing.CompressionMode;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.StoredFieldVisitor;
-import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
 
 /**
  * Lucene 5.0 stored fields format.
@@ -58,7 +55,7 @@ import org.apache.lucene.util.packed.PackedInts;
  *   // indexWriterConfig.setCodec(new Lucene54Codec(Mode.BEST_COMPRESSION));
  * </pre>
  * <p><b>File formats</b>
- * <p>Stored fields are represented by two files:
+ * <p>Stored fields are represented by three files:
  * <ol>
  * <li><a name="field_data"></a>
  * <p>A fields data file (extension <tt>.fdt</tt>). This file stores a compact
@@ -67,45 +64,8 @@ import org.apache.lucene.util.packed.PackedInts;
  * buffer. When its size reaches 16KB or more, some metadata about the documents
  * is flushed to disk, immediately followed by a compressed representation of
  * the buffer using the
- * <a href="http://code.google.com/p/lz4/">LZ4</a>
+ * <a href="https://github.com/lz4/lz4">LZ4</a>
  * <a href="http://fastcompression.blogspot.fr/2011/05/lz4-explained.html">compression format</a>.</p>
- * <p>Here is a more detailed description of the field data file format:</p>
- * <ul>
- * <li>FieldData (.fdt) --&gt; &lt;Header&gt;, PackedIntsVersion, &lt;Chunk&gt;<sup>ChunkCount</sup>, ChunkCount, DirtyChunkCount, Footer</li>
- * <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
- * <li>PackedIntsVersion --&gt; {@link PackedInts#VERSION_CURRENT} as a {@link DataOutput#writeVInt VInt}</li>
- * <li>ChunkCount is not known in advance and is the number of chunks necessary to store all document of the segment</li>
- * <li>Chunk --&gt; DocBase, ChunkDocs, DocFieldCounts, DocLengths, &lt;CompressedDocs&gt;</li>
- * <li>DocBase --&gt; the ID of the first document of the chunk as a {@link DataOutput#writeVInt VInt}</li>
- * <li>ChunkDocs --&gt; the number of documents in the chunk as a {@link DataOutput#writeVInt VInt}</li>
- * <li>DocFieldCounts --&gt; the number of stored fields of every document in the chunk, encoded as followed:<ul>
- *   <li>if chunkDocs=1, the unique value is encoded as a {@link DataOutput#writeVInt VInt}</li>
- *   <li>else read a {@link DataOutput#writeVInt VInt} (let's call it <tt>bitsRequired</tt>)<ul>
- *     <li>if <tt>bitsRequired</tt> is <tt>0</tt> then all values are equal, and the common value is the following {@link DataOutput#writeVInt VInt}</li>
- *     <li>else <tt>bitsRequired</tt> is the number of bits required to store any value, and values are stored in a {@link PackedInts packed} array where every value is stored on exactly <tt>bitsRequired</tt> bits</li>
- *   </ul></li>
- * </ul></li>
- * <li>DocLengths --&gt; the lengths of all documents in the chunk, encoded with the same method as DocFieldCounts</li>
- * <li>CompressedDocs --&gt; a compressed representation of &lt;Docs&gt; using the LZ4 compression format</li>
- * <li>Docs --&gt; &lt;Doc&gt;<sup>ChunkDocs</sup></li>
- * <li>Doc --&gt; &lt;FieldNumAndType, Value&gt;<sup>DocFieldCount</sup></li>
- * <li>FieldNumAndType --&gt; a {@link DataOutput#writeVLong VLong}, whose 3 last bits are Type and other bits are FieldNum</li>
- * <li>Type --&gt;<ul>
- *   <li>0: Value is String</li>
- *   <li>1: Value is BinaryValue</li>
- *   <li>2: Value is Int</li>
- *   <li>3: Value is Float</li>
- *   <li>4: Value is Long</li>
- *   <li>5: Value is Double</li>
- *   <li>6, 7: unused</li>
- * </ul></li>
- * <li>FieldNum --&gt; an ID of the field</li>
- * <li>Value --&gt; {@link DataOutput#writeString(String) String} | BinaryValue | Int | Float | Long | Double depending on Type</li>
- * <li>BinaryValue --&gt; ValueLength &lt;Byte&gt;<sup>ValueLength</sup></li>
- * <li>ChunkCount --&gt; the number of chunks in this file</li>
- * <li>DirtyChunkCount --&gt; the number of prematurely flushed chunks in this file</li>
- * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
  * <p>Notes
  * <ul>
  * <li>If documents are larger than 16KB then chunks will likely contain only
@@ -119,18 +79,20 @@ import org.apache.lucene.util.packed.PackedInts;
  * <li>Given that the original lengths are written in the metadata of the chunk,
  * the decompressor can leverage this information to stop decoding as soon as
  * enough data has been decompressed.</li>
- * <li>In case documents are incompressible, CompressedDocs will be less than
- * 0.5% larger than Docs.</li>
+ * <li>In case documents are incompressible, the overhead of the compression format
+ * is less than 0.5%.</li>
  * </ul>
  * </li>
  * <li><a name="field_index"></a>
- * <p>A fields index file (extension <tt>.fdx</tt>).</p>
- * <ul>
- * <li>FieldsIndex (.fdx) --&gt; &lt;Header&gt;, &lt;ChunkIndex&gt;, Footer</li>
- * <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
- * <li>ChunkIndex: See {@link CompressingStoredFieldsIndexWriter}</li>
- * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
+ * <p>A fields index file (extension <tt>.fdx</tt>). This file stores two
+ * {@link DirectMonotonicWriter monotonic arrays}, one for the first doc IDs of
+ * each block of compressed documents, and another one for the corresponding
+ * offsets on disk. At search time, the array containing doc IDs is
+ * binary-searched in order to find the block that contains the expected doc ID,
+ * and the associated offset on disk is retrieved from the second array.</p>
+ * <li><a name="field_meta"></a>
+ * <p>A fields meta file (extension <tt>.fdm</tt>). This file stores metadata
+ * about the monotonic arrays stored in the index file.</p>
  * </li>
  * </ol>
  * <p><b>Known limitations</b>
@@ -186,9 +148,9 @@ public final class Lucene50StoredFieldsFormat extends StoredFieldsFormat {
   StoredFieldsFormat impl(Mode mode) {
     switch (mode) {
       case BEST_SPEED: 
-        return new CompressingStoredFieldsFormat("Lucene50StoredFieldsFast", CompressionMode.FAST, 1 << 14, 128, 1024);
+        return new CompressingStoredFieldsFormat("Lucene50StoredFieldsFastData", CompressionMode.FAST, 1 << 14, 128, 10);
       case BEST_COMPRESSION: 
-        return new CompressingStoredFieldsFormat("Lucene50StoredFieldsHigh", CompressionMode.HIGH_COMPRESSION, 61440, 512, 1024);
+        return new CompressingStoredFieldsFormat("Lucene50StoredFieldsHighData", CompressionMode.HIGH_COMPRESSION, 61440, 512, 10);
       default: throw new AssertionError();
     }
   }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java
index 5ff4956..40889bf 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.lucene50;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.compressing.CompressingStoredFieldsIndexWriter;
+import org.apache.lucene.codecs.compressing.FieldsIndexWriter;
 import org.apache.lucene.codecs.compressing.CompressingTermVectorsFormat;
 import org.apache.lucene.codecs.compressing.CompressionMode;
 import org.apache.lucene.store.DataOutput;
@@ -116,7 +116,7 @@ import org.apache.lucene.util.packed.PackedInts;
  * <ul>
  * <li>VectorIndex (.tvx) --&gt; &lt;Header&gt;, &lt;ChunkIndex&gt;, Footer</li>
  * <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
- * <li>ChunkIndex: See {@link CompressingStoredFieldsIndexWriter}</li>
+ * <li>ChunkIndex: See {@link FieldsIndexWriter}</li>
  * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
  * </li>
@@ -127,7 +127,7 @@ public final class Lucene50TermVectorsFormat extends CompressingTermVectorsForma
 
   /** Sole constructor. */
   public Lucene50TermVectorsFormat() {
-    super("Lucene50TermVectors", "", CompressionMode.FAST, 1 << 12, 1024);
+    super("Lucene50TermVectorsData", "", CompressionMode.FAST, 1 << 12, 10);
   }
 
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/TrackingTmpOutputDirectoryWrapper.java b/lucene/core/src/java/org/apache/lucene/index/TrackingTmpOutputDirectoryWrapper.java
index 2827840..fdb886c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/TrackingTmpOutputDirectoryWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/index/TrackingTmpOutputDirectoryWrapper.java
@@ -43,7 +43,8 @@ final class TrackingTmpOutputDirectoryWrapper extends FilterDirectory {
 
   @Override
   public IndexInput openInput(String name, IOContext context) throws IOException {
-    String tmpName = fileNames.get(name);
+    // keep the original file name if no match, it might be a temp file already
+    String tmpName = fileNames.getOrDefault(name, name);
     return super.openInput(tmpName, context);
   }
 
diff --git a/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java b/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java
index 676efcd..8d9fc84 100644
--- a/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java
+++ b/lucene/core/src/java/org/apache/lucene/util/packed/DirectMonotonicReader.java
@@ -18,6 +18,7 @@ package org.apache.lucene.util.packed;
 
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.RandomAccessInput;
@@ -29,7 +30,9 @@ import org.apache.lucene.util.RamUsageEstimator;
  * Retrieves an instance previously written by {@link DirectMonotonicWriter}.
  * @see DirectMonotonicWriter 
  */
-public final class DirectMonotonicReader {
+public final class DirectMonotonicReader extends LongValues implements Accountable {
+
+  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(DirectMonotonicReader.class);
 
   /** An instance that always returns {@code 0}. */
   private static final LongValues EMPTY = new LongValues() {
@@ -92,7 +95,7 @@ public final class DirectMonotonicReader {
   /**
    * Retrieves an instance from the specified slice.
    */
-  public static LongValues getInstance(Meta meta, RandomAccessInput data) throws IOException {
+  public static DirectMonotonicReader getInstance(Meta meta, RandomAccessInput data) throws IOException {
     final LongValues[] readers = new LongValues[meta.numBlocks];
     for (int i = 0; i < meta.mins.length; ++i) {
       if (meta.bpvs[i] == 0) {
@@ -101,20 +104,99 @@ public final class DirectMonotonicReader {
         readers[i] = DirectReader.getInstance(data, meta.bpvs[i], meta.offsets[i]);
       }
     }
-    final int blockShift = meta.blockShift;
-
-    final long[] mins = meta.mins;
-    final float[] avgs = meta.avgs;
-    return new LongValues() {
-
-      @Override
-      public long get(long index) {
-        final int block = (int) (index >>> blockShift);
-        final long blockIndex = index & ((1 << blockShift) - 1);
-        final long delta = readers[block].get(blockIndex);
-        return mins[block] + (long) (avgs[block] * blockIndex) + delta;
+
+    return new DirectMonotonicReader(meta.blockShift, readers, meta.mins, meta.avgs, meta.bpvs);
+  }
+
+  private final int blockShift;
+  private final LongValues[] readers;
+  private final long[] mins;
+  private final float[] avgs;
+  private final byte[] bpvs;
+  private final int nonZeroBpvs;
+
+  private DirectMonotonicReader(int blockShift, LongValues[] readers, long[] mins, float[] avgs, byte[] bpvs) {
+    this.blockShift = blockShift;
+    this.readers = readers;
+    this.mins = mins;
+    this.avgs = avgs;
+    this.bpvs = bpvs;
+    if (readers.length != mins.length || readers.length != avgs.length || readers.length != bpvs.length) {
+      throw new IllegalArgumentException();
+    }
+    int nonZeroBpvs = 0;
+    for (byte b : bpvs) {
+      if (b != 0) {
+        nonZeroBpvs++;
+      }
+    }
+    this.nonZeroBpvs = nonZeroBpvs;
+  }
+
+  @Override
+  public long get(long index) {
+    final int block = (int) (index >>> blockShift);
+    final long blockIndex = index & ((1 << blockShift) - 1);
+    final long delta = readers[block].get(blockIndex);
+    return mins[block] + (long) (avgs[block] * blockIndex) + delta;
+  }
+
+  /** Get lower/upper bounds for the value at a given index without hitting the direct reader. */
+  private long[] getBounds(long index) {
+    final int block = Math.toIntExact(index >>> blockShift);
+    final long blockIndex = index & ((1 << blockShift) - 1);
+    final long lowerBound = mins[block] + (long) (avgs[block] * blockIndex);
+    final long upperBound = lowerBound + (1L << bpvs[block]) - 1;
+    if (bpvs[block] == 64 || upperBound < lowerBound) { // overflow
+      return new long[] { Long.MIN_VALUE, Long.MAX_VALUE };
+    } else {
+      return new long[] { lowerBound, upperBound };
+    }
+  }
+
+  /**
+   * Return the index of a key if it exists, or its insertion point otherwise
+   * like {@link Arrays#binarySearch(long[], int, int, long)}.
+   *
+   * @see Arrays#binarySearch(long[], int, int, long)
+   */
+  public long binarySearch(long fromIndex, long toIndex, long key) {
+    if (fromIndex < 0 || fromIndex > toIndex) {
+      throw new IllegalArgumentException("fromIndex=" + fromIndex + ",toIndex=" + toIndex);
+    }
+    long lo = fromIndex;
+    long hi = toIndex - 1;
+
+    while (lo <= hi) {
+      final long mid = (lo + hi) >>> 1;
+      // Try to run as many iterations of the binary search as possible without
+      // hitting the direct readers, since they might hit a page fault.
+      final long[] bounds = getBounds(mid);
+      if (bounds[1] < key) {
+        lo = mid + 1;
+      } else if (bounds[0] > key) {
+        hi = mid - 1;
+      } else {
+        final long midVal = get(mid);
+        if (midVal < key) {
+          lo = mid + 1;
+        } else if (midVal > key) {
+          hi = mid - 1;
+        } else {
+          return mid;
+        }
       }
+    }
 
-    };
+    return -1 - lo;
   }
+
+  @Override
+  public long ramBytesUsed() {
+    // Don't include meta, which should be accounted separately
+    return BASE_RAM_BYTES_USED + RamUsageEstimator.shallowSizeOf(readers) +
+        // Assume empty objects for the readers
+        nonZeroBpvs * RamUsageEstimator.alignObjectSize(RamUsageEstimator.NUM_BYTES_ARRAY_HEADER);
+  }
+
 }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
index 7167b47..1db3dd8 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -1408,8 +1408,8 @@ public class TestIndexWriter extends LuceneTestCase {
           file.startsWith(IndexFileNames.SEGMENTS) || 
           IndexFileNames.CODEC_FILE_PATTERN.matcher(file).matches()) {
         if (file.lastIndexOf('.') < 0
-            // don't count stored fields and term vectors in
-            || !Arrays.asList("fdx", "fdt", "tvx", "tvd", "tvf").contains(file.substring(file.lastIndexOf('.') + 1))) {
+            // don't count stored fields and term vectors in, or any temporary files they might
+            || !Arrays.asList("fdt", "tvd", "tmp").contains(file.substring(file.lastIndexOf('.') + 1))) {
           ++computedExtraFileCount;
         }
       }
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java b/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java
index db95019..26e9980 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java
@@ -49,7 +49,8 @@ public class TestFileSwitchDirectory extends BaseDirectoryTestCase {
   public void testBasic() throws IOException {
     Set<String> fileExtensions = new HashSet<>();
     fileExtensions.add(CompressingStoredFieldsWriter.FIELDS_EXTENSION);
-    fileExtensions.add(CompressingStoredFieldsWriter.FIELDS_INDEX_EXTENSION);
+    fileExtensions.add("fdx");
+    fileExtensions.add("fdm");
     
     MockDirectoryWrapper primaryDir = new MockDirectoryWrapper(random(), new RAMDirectory());
     primaryDir.setCheckIndexOnClose(false); // only part of an index
diff --git a/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java b/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java
index 300e1e0..46526f0 100644
--- a/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java
+++ b/lucene/core/src/test/org/apache/lucene/util/packed/TestDirectMonotonic.java
@@ -188,4 +188,74 @@ public class TestDirectMonotonic extends LuceneTestCase {
     }
   }
 
+  public void testMonotonicBinarySearch() throws IOException {
+    try (Directory dir = newDirectory()) {
+      doTestMonotonicBinarySearchAgainstLongArray(dir, new long[] {4, 7, 8, 10, 19, 30, 55, 78, 100}, 2);
+    }
+  }
+
+  public void testMonotonicBinarySearchRandom() throws IOException {
+    try (Directory dir = newDirectory()) {
+      final int iters = atLeast(100);
+      for (int iter = 0; iter < iters; ++iter) {
+        final int arrayLength = random().nextInt(1 << random().nextInt(14));
+        final long[] array = new long[arrayLength];
+        final long base = random().nextLong();
+        final int bpv = TestUtil.nextInt(random(), 4, 61);
+        for (int i = 0; i < array.length; ++i) {
+          array[i] = base + TestUtil.nextLong(random(), 0, (1L << bpv) - 1);
+        }
+        Arrays.sort(array);
+        doTestMonotonicBinarySearchAgainstLongArray(dir, array, TestUtil.nextInt(random(), 2, 10));
+      }
+    }
+  }
+
+  private void doTestMonotonicBinarySearchAgainstLongArray(Directory dir, long[] array, int blockShift) throws IOException {
+    try (IndexOutput metaOut = dir.createOutput("meta", IOContext.DEFAULT);
+        IndexOutput dataOut = dir.createOutput("data", IOContext.DEFAULT)) {
+      DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(metaOut, dataOut, array.length, blockShift);
+      for (long l : array) {
+        writer.add(l);
+      }
+      writer.finish();
+    }
+
+    try (IndexInput metaIn = dir.openInput("meta", IOContext.READONCE);
+        IndexInput dataIn = dir.openInput("data", IOContext.READ)) {
+      DirectMonotonicReader.Meta meta = DirectMonotonicReader.loadMeta(metaIn, array.length, blockShift);
+      DirectMonotonicReader reader = DirectMonotonicReader.getInstance(meta, dataIn.randomAccessSlice(0L, dir.fileLength("data")));
+
+      if (array.length == 0) {
+        assertEquals(-1, reader.binarySearch(0, array.length, 42L));
+      } else {
+        for (int i = 0; i < array.length; ++i) {
+          final long index = reader.binarySearch(0, array.length, array[i]);
+          assertTrue(index >= 0);
+          assertTrue(index < array.length);
+          assertEquals(array[i], array[(int) index]);
+        }
+        if (array[0] != Long.MIN_VALUE) {
+          assertEquals(-1, reader.binarySearch(0, array.length, array[0] - 1));
+        }
+        if (array[array.length - 1] != Long.MAX_VALUE) {
+          assertEquals(-1 - array.length, reader.binarySearch(0, array.length, array[array.length - 1] + 1));
+        }
+        for (int i = 0; i < array.length - 2; ++i) {
+          if (array[i] + 1 < array[i+1]) {
+            final long intermediate = random().nextBoolean() ? array[i] + 1 : array[i+1] - 1;
+            final long index = reader.binarySearch(0, array.length, intermediate);
+            assertTrue(index < 0);
+            final int insertionPoint = Math.toIntExact(-1 -index);
+            assertTrue(insertionPoint > 0);
+            assertTrue(insertionPoint < array.length);
+            assertTrue(array[insertionPoint] > intermediate);
+            assertTrue(array[insertionPoint-1] < intermediate);
+          }
+        }
+      }
+    }
+    dir.deleteFile("meta");
+    dir.deleteFile("data");
+  }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
index 4fd5e16..4f334ae 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
@@ -23,6 +23,7 @@ import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.compressing.dummy.DummyCompressingCodec;
 import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
 
 import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
 
@@ -35,16 +36,16 @@ public abstract class CompressingCodec extends FilterCodec {
   /**
    * Create a random instance.
    */
-  public static CompressingCodec randomInstance(Random random, int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockSize) {
+  public static CompressingCodec randomInstance(Random random, int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockShift) {
     switch (random.nextInt(4)) {
     case 0:
-      return new FastCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockSize);
+      return new FastCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockShift);
     case 1:
-      return new FastDecompressionCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockSize);
+      return new FastDecompressionCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockShift);
     case 2:
-      return new HighCompressionCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockSize);
+      return new HighCompressionCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockShift);
     case 3:
-      return new DummyCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockSize);
+      return new DummyCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockShift);
     default:
       throw new AssertionError();
     }
@@ -57,7 +58,9 @@ public abstract class CompressingCodec extends FilterCodec {
   public static CompressingCodec randomInstance(Random random) {
     final int chunkSize = random.nextBoolean() ? RandomNumbers.randomIntBetween(random, 1, 10) : RandomNumbers.randomIntBetween(random, 1, 1 << 15);
     final int chunkDocs = random.nextBoolean() ? RandomNumbers.randomIntBetween(random, 1, 10) : RandomNumbers.randomIntBetween(random, 64, 1024);
-    final int blockSize = random.nextBoolean() ? RandomNumbers.randomIntBetween(random, 1, 10) : RandomNumbers.randomIntBetween(random, 1, 1024);
+    final int blockSize = random.nextBoolean()
+        ? RandomNumbers.randomIntBetween(random, DirectMonotonicWriter.MIN_BLOCK_SHIFT, 10)
+        : RandomNumbers.randomIntBetween(random, DirectMonotonicWriter.MIN_BLOCK_SHIFT, DirectMonotonicWriter.MAX_BLOCK_SHIFT);
     return randomInstance(random, chunkSize, chunkDocs, false, blockSize);
   }
 
@@ -70,8 +73,8 @@ public abstract class CompressingCodec extends FilterCodec {
     // e.g. defaults use 128 for FAST and 512 for HIGH
     final int chunkDocs = TestUtil.nextInt(random, 1<<6, 1<<10);
     // e.g. defaults use 1024 for both cases
-    final int blockSize = TestUtil.nextInt(random, 1<<9, 1<<11);
-    return randomInstance(random, chunkSize, chunkDocs, false, blockSize);
+    final int blockShift = TestUtil.nextInt(random, 8, 12);
+    return randomInstance(random, chunkSize, chunkDocs, false, blockShift);
   }
   
   /**
@@ -91,10 +94,10 @@ public abstract class CompressingCodec extends FilterCodec {
   /**
    * Creates a compressing codec with a given segment suffix
    */
-  public CompressingCodec(String name, String segmentSuffix, CompressionMode compressionMode, int chunkSize, int maxDocsPerChunk, int blockSize) {
+  public CompressingCodec(String name, String segmentSuffix, CompressionMode compressionMode, int chunkSize, int maxDocsPerChunk, int blockShift) {
     super(name, TestUtil.getDefaultCodec());
-    this.storedFieldsFormat = new CompressingStoredFieldsFormat(name, segmentSuffix, compressionMode, chunkSize, maxDocsPerChunk, blockSize);
-    this.termVectorsFormat = new CompressingTermVectorsFormat(name, segmentSuffix, compressionMode, chunkSize, blockSize);
+    this.storedFieldsFormat = new CompressingStoredFieldsFormat(name, segmentSuffix, compressionMode, chunkSize, maxDocsPerChunk, blockShift);
+    this.termVectorsFormat = new CompressingTermVectorsFormat(name, segmentSuffix, compressionMode, chunkSize, blockShift);
   }
   
   /**
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java
index 007948c..337eaea 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java
@@ -21,13 +21,13 @@ public class FastCompressingCodec extends CompressingCodec {
 
   /** Constructor that allows to configure the chunk size. */
   public FastCompressingCodec(int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockSize) {
-    super("FastCompressingStoredFields", 
+    super("FastCompressingStoredFieldsData", 
           withSegmentSuffix ? "FastCompressingStoredFields" : "",
           CompressionMode.FAST, chunkSize, maxDocsPerChunk, blockSize);
   }
 
   /** Default constructor. */
   public FastCompressingCodec() {
-    this(1 << 14, 128, false, 1024);
+    this(1 << 14, 128, false, 10);
   }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java
index d569b16..64454f4 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java
@@ -21,13 +21,13 @@ public class FastDecompressionCompressingCodec extends CompressingCodec {
 
   /** Constructor that allows to configure the chunk size. */
   public FastDecompressionCompressingCodec(int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockSize) {
-    super("FastDecompressionCompressingStoredFields",
+    super("FastDecompressionCompressingStoredFieldsData",
           withSegmentSuffix ? "FastDecompressionCompressingStoredFields" : "",
           CompressionMode.FAST_DECOMPRESSION, chunkSize, maxDocsPerChunk, blockSize);
   }
 
   /** Default constructor. */
   public FastDecompressionCompressingCodec() {
-    this(1 << 14, 256, false, 1024);
+    this(1 << 14, 256, false, 10);
   }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java
index 85b4f0d..7453ff3 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java
@@ -20,16 +20,16 @@ package org.apache.lucene.codecs.compressing;
 public class HighCompressionCompressingCodec extends CompressingCodec {
 
   /** Constructor that allows to configure the chunk size. */
-  public HighCompressionCompressingCodec(int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockSize) {
-    super("HighCompressionCompressingStoredFields",
+  public HighCompressionCompressingCodec(int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockShift) {
+    super("HighCompressionCompressingStoredFieldsData",
           withSegmentSuffix ? "HighCompressionCompressingStoredFields" : "",
-          CompressionMode.HIGH_COMPRESSION, chunkSize, maxDocsPerChunk, blockSize);
+          CompressionMode.HIGH_COMPRESSION, chunkSize, maxDocsPerChunk, blockShift);
   }
 
   /** Default constructor. */
   public HighCompressionCompressingCodec() {
     // we don't worry about zlib block overhead as it's
     // not bad and try to save space instead:
-    this(61440, 512, false, 1024);
+    this(61440, 512, false, 10);
   }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/dummy/DummyCompressingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/dummy/DummyCompressingCodec.java
index 167418e..4b414da 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/dummy/DummyCompressingCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/dummy/DummyCompressingCodec.java
@@ -86,14 +86,14 @@ public class DummyCompressingCodec extends CompressingCodec {
 
   /** Constructor that allows to configure the chunk size. */
   public DummyCompressingCodec(int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockSize) {
-    super("DummyCompressingStoredFields",
+    super("DummyCompressingStoredFieldsData",
           withSegmentSuffix ? "DummyCompressingStoredFields" : "",
           DUMMY, chunkSize, maxDocsPerChunk, blockSize);
   }
 
   /** Default constructor. */
   public DummyCompressingCodec() {
-    this(1 << 14, 128, false, 1024);
+    this(1 << 14, 128, false, 10);
   }
 
 }