You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2021/05/06 21:08:03 UTC

[lucene] branch main updated: LUCENE-9843: Remove compression option on default codec's docvalues

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

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


The following commit(s) were added to refs/heads/main by this push:
     new a7a0251  LUCENE-9843: Remove compression option on default codec's docvalues
a7a0251 is described below

commit a7a02519f0a5652110a186f4909347ac3349092d
Author: Robert Muir <rm...@apache.org>
AuthorDate: Thu May 6 17:07:41 2021 -0400

    LUCENE-9843: Remove compression option on default codec's docvalues
---
 lucene/CHANGES.txt                                 |   2 +
 .../lucene/codecs/lucene90/Lucene90Codec.java      |  12 +-
 .../codecs/lucene90/Lucene90DocValuesConsumer.java | 279 ++----------------
 .../codecs/lucene90/Lucene90DocValuesFormat.java   |  34 +--
 .../codecs/lucene90/Lucene90DocValuesProducer.java | 243 ++--------------
 ...TestBestCompressionLucene90DocValuesFormat.java |  33 ---
 .../TestBestSpeedLucene90DocValuesFormat.java      |  32 ---
 .../codecs/lucene90/TestDocValuesCompression.java  | 313 ---------------------
 ...tCase.java => TestLucene90DocValuesFormat.java} |  95 ++++++-
 .../java/org/apache/lucene/index/RandomCodec.java  |   2 +-
 10 files changed, 146 insertions(+), 899 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 45dde72..cd68983 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -114,6 +114,8 @@ API Changes
 * LUCENE-9948: No longer require the user to specify whether-or-not a field is multi-valued in
   LongValueFacetCounts (detect automatically based on what is indexed). (Greg Miller)
 
+* LUCENE-9843: Remove compression option on default codec's docvalues. (Jack Conradson)
+
 Improvements
 
 * LUCENE-9687: Hunspell support improvements: add API for spell-checking and suggestions, support compound words,
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java
index a0fbd56..4fef31d 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90Codec.java
@@ -47,18 +47,14 @@ public class Lucene90Codec extends Codec {
   /** Configuration option for the codec. */
   public enum Mode {
     /** Trade compression ratio for retrieval speed. */
-    BEST_SPEED(Lucene90StoredFieldsFormat.Mode.BEST_SPEED, Lucene90DocValuesFormat.Mode.BEST_SPEED),
+    BEST_SPEED(Lucene90StoredFieldsFormat.Mode.BEST_SPEED),
     /** Trade retrieval speed for compression ratio. */
-    BEST_COMPRESSION(
-        Lucene90StoredFieldsFormat.Mode.BEST_COMPRESSION,
-        Lucene90DocValuesFormat.Mode.BEST_COMPRESSION);
+    BEST_COMPRESSION(Lucene90StoredFieldsFormat.Mode.BEST_COMPRESSION);
 
     private final Lucene90StoredFieldsFormat.Mode storedMode;
-    private final Lucene90DocValuesFormat.Mode dvMode;
 
-    private Mode(Lucene90StoredFieldsFormat.Mode storedMode, Lucene90DocValuesFormat.Mode dvMode) {
+    private Mode(Lucene90StoredFieldsFormat.Mode storedMode) {
       this.storedMode = Objects.requireNonNull(storedMode);
-      this.dvMode = Objects.requireNonNull(dvMode);
     }
   }
 
@@ -110,7 +106,7 @@ public class Lucene90Codec extends Codec {
     this.storedFieldsFormat =
         new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
     this.defaultFormat = new Lucene90PostingsFormat();
-    this.defaultDVFormat = new Lucene90DocValuesFormat(mode.dvMode);
+    this.defaultDVFormat = new Lucene90DocValuesFormat();
   }
 
   @Override
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java
index 2e6389a..8122de1 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java
@@ -20,7 +20,6 @@ import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.DIRECT_M
 import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SHIFT;
 import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE;
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -31,7 +30,6 @@ import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.EmptyDocValuesProducer;
 import org.apache.lucene.index.FieldInfo;
@@ -46,9 +44,6 @@ import org.apache.lucene.search.SortedSetSelector;
 import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.ByteBuffersIndexOutput;
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
@@ -57,17 +52,14 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.MathUtil;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.compress.LZ4;
-import org.apache.lucene.util.compress.LZ4.FastCompressionHashTable;
 import org.apache.lucene.util.packed.DirectMonotonicWriter;
 import org.apache.lucene.util.packed.DirectWriter;
 
 /** writer for {@link Lucene90DocValuesFormat} */
 final class Lucene90DocValuesConsumer extends DocValuesConsumer {
 
-  final Lucene90DocValuesFormat.Mode mode;
   IndexOutput data, meta;
   final int maxDoc;
-  private final SegmentWriteState state;
   private byte[] termsDictBuffer;
 
   /** expert: Creates a new writer */
@@ -76,16 +68,11 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
       String dataCodec,
       String dataExtension,
       String metaCodec,
-      String metaExtension,
-      Lucene90DocValuesFormat.Mode mode)
+      String metaExtension)
       throws IOException {
-    this.mode = mode;
-    if (Lucene90DocValuesFormat.Mode.BEST_COMPRESSION == this.mode) {
-      this.termsDictBuffer = new byte[1 << 14];
-    }
+    this.termsDictBuffer = new byte[1 << 14];
     boolean success = false;
     try {
-      this.state = state;
       String dataName =
           IndexFileNames.segmentFileName(
               state.segmentInfo.name, state.segmentSuffix, dataExtension);
@@ -402,167 +389,11 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
     }
   }
 
-  class CompressedBinaryBlockWriter implements Closeable {
-    final FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();
-    int uncompressedBlockLength = 0;
-    int maxUncompressedBlockLength = 0;
-    int numDocsInCurrentBlock = 0;
-    final int[] docLengths = new int[Lucene90DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK];
-    byte[] block = BytesRef.EMPTY_BYTES;
-    int totalChunks = 0;
-    long maxPointer = 0;
-    final long blockAddressesStart;
-
-    private final IndexOutput tempBinaryOffsets;
-
-    public CompressedBinaryBlockWriter() throws IOException {
-      tempBinaryOffsets =
-          state.directory.createTempOutput(
-              state.segmentInfo.name, "binary_pointers", state.context);
-      boolean success = false;
-      try {
-        CodecUtil.writeHeader(
-            tempBinaryOffsets,
-            Lucene90DocValuesFormat.META_CODEC + "FilePointers",
-            Lucene90DocValuesFormat.VERSION_CURRENT);
-        blockAddressesStart = data.getFilePointer();
-        success = true;
-      } finally {
-        if (success == false) {
-          IOUtils.closeWhileHandlingException(this); // self-close because constructor caller can't
-        }
-      }
-    }
-
-    void addDoc(int doc, BytesRef v) throws IOException {
-      docLengths[numDocsInCurrentBlock] = v.length;
-      block = ArrayUtil.grow(block, uncompressedBlockLength + v.length);
-      System.arraycopy(v.bytes, v.offset, block, uncompressedBlockLength, v.length);
-      uncompressedBlockLength += v.length;
-      numDocsInCurrentBlock++;
-      if (numDocsInCurrentBlock == Lucene90DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK) {
-        flushData();
-      }
-    }
-
-    private void flushData() throws IOException {
-      if (numDocsInCurrentBlock > 0) {
-        // Write offset to this block to temporary offsets file
-        totalChunks++;
-        long thisBlockStartPointer = data.getFilePointer();
-
-        // Optimisation - check if all lengths are same
-        boolean allLengthsSame = true;
-        for (int i = 1; i < Lucene90DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) {
-          if (docLengths[i] != docLengths[i - 1]) {
-            allLengthsSame = false;
-            break;
-          }
-        }
-        if (allLengthsSame) {
-          // Only write one value shifted. Steal a bit to indicate all other lengths are the same
-          int onlyOneLength = (docLengths[0] << 1) | 1;
-          data.writeVInt(onlyOneLength);
-        } else {
-          for (int i = 0; i < Lucene90DocValuesFormat.BINARY_DOCS_PER_COMPRESSED_BLOCK; i++) {
-            if (i == 0) {
-              // Write first value shifted and steal a bit to indicate other lengths are to follow
-              int multipleLengths = (docLengths[0] << 1);
-              data.writeVInt(multipleLengths);
-            } else {
-              data.writeVInt(docLengths[i]);
-            }
-          }
-        }
-        maxUncompressedBlockLength = Math.max(maxUncompressedBlockLength, uncompressedBlockLength);
-        LZ4.compress(block, 0, uncompressedBlockLength, data, ht);
-        numDocsInCurrentBlock = 0;
-        // Ensure initialized with zeroes because full array is always written
-        Arrays.fill(docLengths, 0);
-        uncompressedBlockLength = 0;
-        maxPointer = data.getFilePointer();
-        tempBinaryOffsets.writeVLong(maxPointer - thisBlockStartPointer);
-      }
-    }
-
-    void writeMetaData() throws IOException {
-      if (totalChunks == 0) {
-        return;
-      }
-
-      long startDMW = data.getFilePointer();
-      meta.writeLong(startDMW);
-
-      meta.writeVInt(totalChunks);
-      meta.writeVInt(Lucene90DocValuesFormat.BINARY_BLOCK_SHIFT);
-      meta.writeVInt(maxUncompressedBlockLength);
-      meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
-
-      CodecUtil.writeFooter(tempBinaryOffsets);
-      IOUtils.close(tempBinaryOffsets);
-      // write the compressed block offsets info to the meta file by reading from temp file
-      try (ChecksumIndexInput filePointersIn =
-          state.directory.openChecksumInput(tempBinaryOffsets.getName(), IOContext.READONCE)) {
-        CodecUtil.checkHeader(
-            filePointersIn,
-            Lucene90DocValuesFormat.META_CODEC + "FilePointers",
-            Lucene90DocValuesFormat.VERSION_CURRENT,
-            Lucene90DocValuesFormat.VERSION_CURRENT);
-        Throwable priorE = null;
-        try {
-          final DirectMonotonicWriter filePointers =
-              DirectMonotonicWriter.getInstance(
-                  meta, data, totalChunks, DIRECT_MONOTONIC_BLOCK_SHIFT);
-          long fp = blockAddressesStart;
-          for (int i = 0; i < totalChunks; ++i) {
-            filePointers.add(fp);
-            fp += filePointersIn.readVLong();
-          }
-          if (maxPointer < fp) {
-            throw new CorruptIndexException(
-                "File pointers don't add up (" + fp + " vs expected " + maxPointer + ")",
-                filePointersIn);
-          }
-          filePointers.finish();
-        } catch (Throwable e) {
-          priorE = e;
-        } finally {
-          CodecUtil.checkFooter(filePointersIn, priorE);
-        }
-      }
-      // Write the length of the DMW block in the data
-      meta.writeLong(data.getFilePointer() - startDMW);
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (tempBinaryOffsets != null) {
-        IOUtils.close(tempBinaryOffsets);
-        state.directory.deleteFile(tempBinaryOffsets.getName());
-      }
-    }
-  }
-
   @Override
   public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
-    field.putAttribute(Lucene90DocValuesFormat.MODE_KEY, mode.name());
     meta.writeInt(field.number);
     meta.writeByte(Lucene90DocValuesFormat.BINARY);
 
-    switch (mode) {
-      case BEST_SPEED:
-        doAddUncompressedBinaryField(field, valuesProducer);
-        break;
-      case BEST_COMPRESSION:
-        doAddCompressedBinaryField(field, valuesProducer);
-        break;
-      default:
-        throw new AssertionError();
-    }
-  }
-
-  private void doAddUncompressedBinaryField(FieldInfo field, DocValuesProducer valuesProducer)
-      throws IOException {
     BinaryDocValues values = valuesProducer.getBinary(field);
     long start = data.getFilePointer();
     meta.writeLong(start); // dataOffset
@@ -626,59 +457,6 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
     }
   }
 
-  private void doAddCompressedBinaryField(FieldInfo field, DocValuesProducer valuesProducer)
-      throws IOException {
-    try (CompressedBinaryBlockWriter blockWriter = new CompressedBinaryBlockWriter()) {
-      BinaryDocValues values = valuesProducer.getBinary(field);
-      long start = data.getFilePointer();
-      meta.writeLong(start); // dataOffset
-      int numDocsWithField = 0;
-      int minLength = Integer.MAX_VALUE;
-      int maxLength = 0;
-      for (int doc = values.nextDoc();
-          doc != DocIdSetIterator.NO_MORE_DOCS;
-          doc = values.nextDoc()) {
-        numDocsWithField++;
-        BytesRef v = values.binaryValue();
-        blockWriter.addDoc(doc, v);
-        int length = v.length;
-        minLength = Math.min(length, minLength);
-        maxLength = Math.max(length, maxLength);
-      }
-      blockWriter.flushData();
-
-      assert numDocsWithField <= maxDoc;
-      meta.writeLong(data.getFilePointer() - start); // dataLength
-
-      if (numDocsWithField == 0) {
-        meta.writeLong(-2); // docsWithFieldOffset
-        meta.writeLong(0L); // docsWithFieldLength
-        meta.writeShort((short) -1); // jumpTableEntryCount
-        meta.writeByte((byte) -1); // denseRankPower
-      } else if (numDocsWithField == maxDoc) {
-        meta.writeLong(-1); // docsWithFieldOffset
-        meta.writeLong(0L); // docsWithFieldLength
-        meta.writeShort((short) -1); // jumpTableEntryCount
-        meta.writeByte((byte) -1); // denseRankPower
-      } else {
-        long offset = data.getFilePointer();
-        meta.writeLong(offset); // docsWithFieldOffset
-        values = valuesProducer.getBinary(field);
-        final short jumpTableEntryCount =
-            IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-        meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
-        meta.writeShort(jumpTableEntryCount);
-        meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
-      }
-
-      meta.writeInt(numDocsWithField);
-      meta.writeInt(minLength);
-      meta.writeInt(maxLength);
-
-      blockWriter.writeMetaData();
-    }
-  }
-
   @Override
   public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
     meta.writeInt(field.number);
@@ -742,21 +520,10 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
   private void addTermsDict(SortedSetDocValues values) throws IOException {
     final long size = values.getValueCount();
     meta.writeVLong(size);
-    boolean compress =
-        Lucene90DocValuesFormat.Mode.BEST_COMPRESSION == mode
-            && values.getValueCount()
-                > Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD;
-    int code, blockMask, shift;
-    if (compress) {
-      code = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_CODE;
-      blockMask = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_MASK;
-      shift = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT;
-    } else {
-      code = shift = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_SHIFT;
-      blockMask = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_MASK;
-    }
 
-    meta.writeInt(code);
+    int blockMask = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_MASK;
+    int shift = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT;
+
     meta.writeInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
     ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
     ByteBuffersIndexOutput addressOutput =
@@ -772,16 +539,12 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
     int maxLength = 0, maxBlockLength = 0;
     TermsEnum iterator = values.termsEnum();
 
-    LZ4.FastCompressionHashTable ht = null;
-    ByteArrayDataOutput bufferedOutput = null;
-    if (compress) {
-      ht = new LZ4.FastCompressionHashTable();
-      bufferedOutput = new ByteArrayDataOutput(termsDictBuffer);
-    }
+    LZ4.FastCompressionHashTable ht = new LZ4.FastCompressionHashTable();
+    ByteArrayDataOutput bufferedOutput = new ByteArrayDataOutput(termsDictBuffer);
 
     for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
       if ((ord & blockMask) == 0) {
-        if (compress && bufferedOutput.getPosition() > 0) {
+        if (bufferedOutput.getPosition() > 0) {
           maxBlockLength =
               Math.max(maxBlockLength, compressAndGetTermsDictBlockLength(bufferedOutput, ht));
           bufferedOutput.reset(termsDictBuffer);
@@ -794,40 +557,32 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
         final int prefixLength = StringHelper.bytesDifference(previous.get(), term);
         final int suffixLength = term.length - prefixLength;
         assert suffixLength > 0; // terms are unique
-        DataOutput blockOutput;
-        if (compress) {
-          // Will write (suffixLength + 1 byte + 2 vint) bytes. Grow the buffer in need.
-          bufferedOutput = maybeGrowBuffer(bufferedOutput, suffixLength + 11);
-          blockOutput = bufferedOutput;
-        } else {
-          blockOutput = data;
-        }
-        blockOutput.writeByte(
+        // Will write (suffixLength + 1 byte + 2 vint) bytes. Grow the buffer in need.
+        bufferedOutput = maybeGrowBuffer(bufferedOutput, suffixLength + 11);
+        bufferedOutput.writeByte(
             (byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength - 1) << 4)));
         if (prefixLength >= 15) {
-          blockOutput.writeVInt(prefixLength - 15);
+          bufferedOutput.writeVInt(prefixLength - 15);
         }
         if (suffixLength >= 16) {
-          blockOutput.writeVInt(suffixLength - 16);
+          bufferedOutput.writeVInt(suffixLength - 16);
         }
-        blockOutput.writeBytes(term.bytes, term.offset + prefixLength, suffixLength);
+        bufferedOutput.writeBytes(term.bytes, term.offset + prefixLength, suffixLength);
       }
       maxLength = Math.max(maxLength, term.length);
       previous.copyBytes(term);
       ++ord;
     }
     // Compress and write out the last block
-    if (compress && bufferedOutput.getPosition() > 0) {
+    if (bufferedOutput.getPosition() > 0) {
       maxBlockLength =
           Math.max(maxBlockLength, compressAndGetTermsDictBlockLength(bufferedOutput, ht));
     }
 
     writer.finish();
     meta.writeInt(maxLength);
-    if (compress) {
-      // Write one more int for storing max block length. For compressed terms dict only.
-      meta.writeInt(maxBlockLength);
-    }
+    // Write one more int for storing max block length.
+    meta.writeInt(maxBlockLength);
     meta.writeLong(start);
     meta.writeLong(data.getFilePointer() - start);
     start = data.getFilePointer();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java
index 6c6788c..de0d6e5 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java
@@ -17,7 +17,6 @@
 package org.apache.lucene.codecs.lucene90;
 
 import java.io.IOException;
-import java.util.Objects;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.DocValuesProducer;
@@ -139,34 +138,15 @@ import org.apache.lucene.util.packed.DirectWriter;
  */
 public final class Lucene90DocValuesFormat extends DocValuesFormat {
 
-  /** Configuration option for doc values. */
-  public enum Mode {
-    /** Trade compression ratio for retrieval speed. */
-    BEST_SPEED,
-    /** Trade retrieval speed for compression ratio. */
-    BEST_COMPRESSION
-  }
-
-  /** Attribute key for compression mode. */
-  public static final String MODE_KEY = Lucene90DocValuesFormat.class.getSimpleName() + ".mode";
-
-  private final Mode mode;
-
   /** Default constructor. */
   public Lucene90DocValuesFormat() {
-    this(Mode.BEST_SPEED);
-  }
-
-  /** Constructor */
-  public Lucene90DocValuesFormat(Mode mode) {
     super("Lucene90");
-    this.mode = Objects.requireNonNull(mode);
   }
 
   @Override
   public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     return new Lucene90DocValuesConsumer(
-        state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION, mode);
+        state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
   }
 
   @Override
@@ -194,21 +174,9 @@ public final class Lucene90DocValuesFormat extends DocValuesFormat {
   static final int NUMERIC_BLOCK_SHIFT = 14;
   static final int NUMERIC_BLOCK_SIZE = 1 << NUMERIC_BLOCK_SHIFT;
 
-  static final int BINARY_BLOCK_SHIFT = 5;
-  static final int BINARY_DOCS_PER_COMPRESSED_BLOCK = 1 << BINARY_BLOCK_SHIFT;
-
-  static final int TERMS_DICT_BLOCK_SHIFT = 4;
-  static final int TERMS_DICT_BLOCK_SIZE = 1 << TERMS_DICT_BLOCK_SHIFT;
-  static final int TERMS_DICT_BLOCK_MASK = TERMS_DICT_BLOCK_SIZE - 1;
-
-  static final int TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD = 32;
   static final int TERMS_DICT_BLOCK_LZ4_SHIFT = 6;
   static final int TERMS_DICT_BLOCK_LZ4_SIZE = 1 << TERMS_DICT_BLOCK_LZ4_SHIFT;
   static final int TERMS_DICT_BLOCK_LZ4_MASK = TERMS_DICT_BLOCK_LZ4_SIZE - 1;
-  static final int TERMS_DICT_COMPRESSOR_LZ4_CODE = 1;
-  // Writing a special code so we know this is a LZ4-compressed block.
-  static final int TERMS_DICT_BLOCK_LZ4_CODE =
-      TERMS_DICT_BLOCK_LZ4_SHIFT << 16 | TERMS_DICT_COMPRESSOR_LZ4_CODE;
 
   static final int TERMS_DICT_REVERSE_INDEX_SHIFT = 10;
   static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java
index 4eeba84..7d86ce7 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java
@@ -16,6 +16,8 @@
  */
 package org.apache.lucene.codecs.lucene90;
 
+import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT;
+
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -86,7 +88,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
                 state.segmentInfo.getId(),
                 state.segmentSuffix);
 
-        readFields(state.segmentInfo.name, in, state.fieldInfos);
+        readFields(in, state.fieldInfos);
 
       } catch (Throwable exception) {
         priorE = exception;
@@ -127,8 +129,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
     }
   }
 
-  private void readFields(String segmentName, IndexInput meta, FieldInfos infos)
-      throws IOException {
+  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
     for (int fieldNumber = meta.readInt(); fieldNumber != -1; fieldNumber = meta.readInt()) {
       FieldInfo info = infos.fieldInfo(fieldNumber);
       if (info == null) {
@@ -138,19 +139,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
       if (type == Lucene90DocValuesFormat.NUMERIC) {
         numerics.put(info.name, readNumeric(meta));
       } else if (type == Lucene90DocValuesFormat.BINARY) {
-        String value = info.getAttribute(Lucene90DocValuesFormat.MODE_KEY);
-        if (value == null) {
-          throw new IllegalStateException(
-              "missing value for "
-                  + Lucene90DocValuesFormat.MODE_KEY
-                  + " for field: "
-                  + info.name
-                  + " in segment: "
-                  + segmentName);
-        }
-        Lucene90DocValuesFormat.Mode mode = Lucene90DocValuesFormat.Mode.valueOf(value);
-        final boolean compressed = mode == Lucene90DocValuesFormat.Mode.BEST_COMPRESSION;
-        binaries.put(info.name, readBinary(meta, compressed));
+        binaries.put(info.name, readBinary(meta));
       } else if (type == Lucene90DocValuesFormat.SORTED) {
         sorted.put(info.name, readSorted(meta));
       } else if (type == Lucene90DocValuesFormat.SORTED_SET) {
@@ -198,9 +187,8 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
     entry.valueJumpTableOffset = meta.readLong();
   }
 
-  private BinaryEntry readBinary(IndexInput meta, boolean compressed) throws IOException {
+  private BinaryEntry readBinary(IndexInput meta) throws IOException {
     final BinaryEntry entry = new BinaryEntry();
-    entry.compressed = compressed;
     entry.dataOffset = meta.readLong();
     entry.dataLength = meta.readLong();
     entry.docsWithFieldOffset = meta.readLong();
@@ -210,18 +198,11 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
     entry.numDocsWithField = meta.readInt();
     entry.minLength = meta.readInt();
     entry.maxLength = meta.readInt();
-    if ((entry.compressed && entry.numDocsWithField > 0) || entry.minLength < entry.maxLength) {
+    if (entry.minLength < entry.maxLength) {
       entry.addressesOffset = meta.readLong();
 
       // Old count of uncompressed addresses
       long numAddresses = entry.numDocsWithField + 1L;
-      // New count of compressed addresses - the number of compresseed blocks
-      if (entry.compressed) {
-        entry.numCompressedChunks = meta.readVInt();
-        entry.docsPerChunkShift = meta.readVInt();
-        entry.maxUncompressedChunkSize = meta.readVInt();
-        numAddresses = entry.numCompressedChunks;
-      }
 
       final int blockShift = meta.readVInt();
       entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, numAddresses, blockShift);
@@ -275,24 +256,13 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
 
   private static void readTermDict(IndexInput meta, TermsDictEntry entry) throws IOException {
     entry.termsDictSize = meta.readVLong();
-    int termsDictBlockCode = meta.readInt();
-    if (Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_CODE == termsDictBlockCode) {
-      // This is a LZ4 compressed block.
-      entry.compressed = true;
-      entry.termsDictBlockShift = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SHIFT;
-    } else {
-      entry.termsDictBlockShift = termsDictBlockCode;
-    }
-
     final int blockShift = meta.readInt();
     final long addressesSize =
-        (entry.termsDictSize + (1L << entry.termsDictBlockShift) - 1) >>> entry.termsDictBlockShift;
+        (entry.termsDictSize + (1L << TERMS_DICT_BLOCK_LZ4_SHIFT) - 1)
+            >>> TERMS_DICT_BLOCK_LZ4_SHIFT;
     entry.termsAddressesMeta = DirectMonotonicReader.loadMeta(meta, addressesSize, blockShift);
     entry.maxTermLength = meta.readInt();
-    // Read one more int for compressed term dict.
-    if (entry.compressed) {
-      entry.maxBlockLength = meta.readInt();
-    }
+    entry.maxBlockLength = meta.readInt();
     entry.termsDataOffset = meta.readLong();
     entry.termsDataLength = meta.readLong();
     entry.termsAddressesOffset = meta.readLong();
@@ -343,7 +313,6 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
   }
 
   private static class BinaryEntry {
-    boolean compressed;
     long dataOffset;
     long dataLength;
     long docsWithFieldOffset;
@@ -356,14 +325,10 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
     long addressesOffset;
     long addressesLength;
     DirectMonotonicReader.Meta addressesMeta;
-    int numCompressedChunks;
-    int docsPerChunkShift;
-    int maxUncompressedChunkSize;
   }
 
   private static class TermsDictEntry {
     long termsDictSize;
-    int termsDictBlockShift;
     DirectMonotonicReader.Meta termsAddressesMeta;
     int maxTermLength;
     long termsDataOffset;
@@ -377,7 +342,6 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
     long termsIndexAddressesOffset;
     long termsIndexAddressesLength;
 
-    boolean compressed;
     int maxBlockLength;
   }
 
@@ -728,7 +692,10 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
     }
   }
 
-  private BinaryDocValues getUncompressedBinary(BinaryEntry entry) throws IOException {
+  @Override
+  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+    BinaryEntry entry = binaries.get(field.name);
+
     if (entry.docsWithFieldOffset == -2) {
       return DocValues.emptyBinary();
     }
@@ -815,148 +782,6 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
     }
   }
 
-  // Decompresses blocks of binary values to retrieve content
-  static class BinaryDecoder {
-
-    private final LongValues addresses;
-    private final IndexInput compressedData;
-    // Cache of last uncompressed block
-    private long lastBlockId = -1;
-    private final int[] uncompressedDocStarts;
-    private int uncompressedBlockLength = 0;
-    private final byte[] uncompressedBlock;
-    private final BytesRef uncompressedBytesRef;
-    private final int docsPerChunk;
-    private final int docsPerChunkShift;
-
-    public BinaryDecoder(
-        LongValues addresses,
-        IndexInput compressedData,
-        int biggestUncompressedBlockSize,
-        int docsPerChunkShift) {
-      super();
-      this.addresses = addresses;
-      this.compressedData = compressedData;
-      // pre-allocate a byte array large enough for the biggest uncompressed block needed.
-      this.uncompressedBlock = new byte[biggestUncompressedBlockSize];
-      uncompressedBytesRef = new BytesRef(uncompressedBlock);
-      this.docsPerChunk = 1 << docsPerChunkShift;
-      this.docsPerChunkShift = docsPerChunkShift;
-      uncompressedDocStarts = new int[docsPerChunk + 1];
-    }
-
-    BytesRef decode(int docNumber) throws IOException {
-      int blockId = docNumber >> docsPerChunkShift;
-      int docInBlockId = docNumber % docsPerChunk;
-      assert docInBlockId < docsPerChunk;
-
-      // already read and uncompressed?
-      if (blockId != lastBlockId) {
-        lastBlockId = blockId;
-        long blockStartOffset = addresses.get(blockId);
-        compressedData.seek(blockStartOffset);
-
-        uncompressedBlockLength = 0;
-
-        int onlyLength = -1;
-        for (int i = 0; i < docsPerChunk; i++) {
-          if (i == 0) {
-            // The first length value is special. It is shifted and has a bit to denote if
-            // all other values are the same length
-            int lengthPlusSameInd = compressedData.readVInt();
-            int sameIndicator = lengthPlusSameInd & 1;
-            int firstValLength = lengthPlusSameInd >>> 1;
-            if (sameIndicator == 1) {
-              onlyLength = firstValLength;
-            }
-            uncompressedBlockLength += firstValLength;
-          } else {
-            if (onlyLength == -1) {
-              // Various lengths are stored - read each from disk
-              uncompressedBlockLength += compressedData.readVInt();
-            } else {
-              // Only one length
-              uncompressedBlockLength += onlyLength;
-            }
-          }
-          uncompressedDocStarts[i + 1] = uncompressedBlockLength;
-        }
-
-        if (uncompressedBlockLength == 0) {
-          uncompressedBytesRef.offset = 0;
-          uncompressedBytesRef.length = 0;
-          return uncompressedBytesRef;
-        }
-
-        assert uncompressedBlockLength <= uncompressedBlock.length;
-        LZ4.decompress(compressedData, uncompressedBlockLength, uncompressedBlock, 0);
-      }
-
-      uncompressedBytesRef.offset = uncompressedDocStarts[docInBlockId];
-      uncompressedBytesRef.length =
-          uncompressedDocStarts[docInBlockId + 1] - uncompressedBytesRef.offset;
-      return uncompressedBytesRef;
-    }
-  }
-
-  @Override
-  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
-    BinaryEntry entry = binaries.get(field.name);
-    if (entry.compressed) {
-      return getCompressedBinary(entry);
-    } else {
-      return getUncompressedBinary(entry);
-    }
-  }
-
-  private BinaryDocValues getCompressedBinary(BinaryEntry entry) throws IOException {
-
-    if (entry.docsWithFieldOffset == -2) {
-      return DocValues.emptyBinary();
-    }
-    if (entry.docsWithFieldOffset == -1) {
-      // dense
-      final RandomAccessInput addressesData =
-          this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
-      final LongValues addresses =
-          DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData);
-      return new DenseBinaryDocValues(maxDoc) {
-        BinaryDecoder decoder =
-            new BinaryDecoder(
-                addresses, data.clone(), entry.maxUncompressedChunkSize, entry.docsPerChunkShift);
-
-        @Override
-        public BytesRef binaryValue() throws IOException {
-          return decoder.decode(doc);
-        }
-      };
-    } else {
-      // sparse
-      final IndexedDISI disi =
-          new IndexedDISI(
-              data,
-              entry.docsWithFieldOffset,
-              entry.docsWithFieldLength,
-              entry.jumpTableEntryCount,
-              entry.denseRankPower,
-              entry.numDocsWithField);
-      final RandomAccessInput addressesData =
-          this.data.randomAccessSlice(entry.addressesOffset, entry.addressesLength);
-      final LongValues addresses =
-          DirectMonotonicReader.getInstance(entry.addressesMeta, addressesData);
-      return new SparseBinaryDocValues(disi) {
-        BinaryDecoder decoder =
-            new BinaryDecoder(
-                addresses, data.clone(), entry.maxUncompressedChunkSize, entry.docsPerChunkShift);
-
-        @Override
-        public BytesRef binaryValue() throws IOException {
-          return decoder.decode(disi.index());
-        }
-      };
-    }
-  }
-
   @Override
   public SortedDocValues getSorted(FieldInfo field) throws IOException {
     SortedEntry entry = sorted.get(field.name);
@@ -1174,7 +999,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
           data.randomAccessSlice(entry.termsAddressesOffset, entry.termsAddressesLength);
       blockAddresses = DirectMonotonicReader.getInstance(entry.termsAddressesMeta, addressesSlice);
       bytes = data.slice("terms", entry.termsDataOffset, entry.termsDataLength);
-      blockMask = (1L << entry.termsDictBlockShift) - 1;
+      blockMask = (1L << TERMS_DICT_BLOCK_LZ4_SHIFT) - 1;
       RandomAccessInput indexAddressesSlice =
           data.randomAccessSlice(entry.termsIndexAddressesOffset, entry.termsIndexAddressesLength);
       indexAddresses =
@@ -1182,11 +1007,9 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
       indexBytes = data.slice("terms-index", entry.termsIndexOffset, entry.termsIndexLength);
       term = new BytesRef(entry.maxTermLength);
 
-      if (entry.compressed) {
-        // add 7 padding bytes can help decompression run faster.
-        int bufferSize = entry.maxBlockLength + LZ4_DECOMPRESSOR_PADDING;
-        blockBuffer = new BytesRef(new byte[bufferSize], 0, bufferSize);
-      }
+      // add 7 padding bytes can help decompression run faster.
+      int bufferSize = entry.maxBlockLength + LZ4_DECOMPRESSOR_PADDING;
+      blockBuffer = new BytesRef(new byte[bufferSize], 0, bufferSize);
     }
 
     @Override
@@ -1196,14 +1019,9 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
       }
 
       if ((ord & blockMask) == 0L) {
-        if (this.entry.compressed) {
-          decompressBlock();
-        } else {
-          term.length = bytes.readVInt();
-          bytes.readBytes(term.bytes, 0, term.length);
-        }
+        decompressBlock();
       } else {
-        DataInput input = this.entry.compressed ? blockInput : bytes;
+        DataInput input = blockInput;
         final int token = Byte.toUnsignedInt(input.readByte());
         int prefixLength = token & 0x0F;
         int suffixLength = 1 + (token >>> 4);
@@ -1224,10 +1042,10 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
       if (ord < 0 || ord >= entry.termsDictSize) {
         throw new IndexOutOfBoundsException();
       }
-      final long blockIndex = ord >>> entry.termsDictBlockShift;
+      final long blockIndex = ord >>> TERMS_DICT_BLOCK_LZ4_SHIFT;
       final long blockAddress = blockAddresses.get(blockIndex);
       bytes.seek(blockAddress);
-      this.ord = (blockIndex << entry.termsDictBlockShift) - 1;
+      this.ord = (blockIndex << TERMS_DICT_BLOCK_LZ4_SHIFT) - 1;
       do {
         next();
       } while (this.ord < ord);
@@ -1264,7 +1082,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
     }
 
     private BytesRef getFirstTermFromBlock(long block) throws IOException {
-      assert block >= 0 && block <= (entry.termsDictSize - 1) >>> entry.termsDictBlockShift;
+      assert block >= 0 && block <= (entry.termsDictSize - 1) >>> TERMS_DICT_BLOCK_LZ4_SHIFT;
       final long blockAddress = blockAddresses.get(block);
       bytes.seek(blockAddress);
       term.length = bytes.readVInt();
@@ -1281,8 +1099,8 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
       long ordLo = index << entry.termsDictIndexShift;
       long ordHi = Math.min(entry.termsDictSize, ordLo + (1L << entry.termsDictIndexShift)) - 1L;
 
-      long blockLo = ordLo >>> entry.termsDictBlockShift;
-      long blockHi = ordHi >>> entry.termsDictBlockShift;
+      long blockLo = ordLo >>> TERMS_DICT_BLOCK_LZ4_SHIFT;
+      long blockHi = ordHi >>> TERMS_DICT_BLOCK_LZ4_SHIFT;
 
       while (blockLo <= blockHi) {
         final long blockMid = (blockLo + blockHi) >>> 1;
@@ -1296,7 +1114,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
       }
 
       assert blockHi < 0 || getFirstTermFromBlock(blockHi).compareTo(text) <= 0;
-      assert blockHi == ((entry.termsDictSize - 1) >>> entry.termsDictBlockShift)
+      assert blockHi == ((entry.termsDictSize - 1) >>> TERMS_DICT_BLOCK_LZ4_SHIFT)
           || getFirstTermFromBlock(blockHi + 1).compareTo(text) > 0;
 
       return blockHi;
@@ -1311,14 +1129,9 @@ final class Lucene90DocValuesProducer extends DocValuesProducer {
         return SeekStatus.NOT_FOUND;
       }
       final long blockAddress = blockAddresses.get(block);
-      this.ord = block << entry.termsDictBlockShift;
+      this.ord = block << TERMS_DICT_BLOCK_LZ4_SHIFT;
       bytes.seek(blockAddress);
-      if (this.entry.compressed) {
-        decompressBlock();
-      } else {
-        term.length = bytes.readVInt();
-        bytes.readBytes(term.bytes, 0, term.length);
-      }
+      decompressBlock();
 
       while (true) {
         int cmp = term.compareTo(text);
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestCompressionLucene90DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestCompressionLucene90DocValuesFormat.java
deleted file mode 100644
index 8747701..0000000
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestCompressionLucene90DocValuesFormat.java
+++ /dev/null
@@ -1,33 +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.lucene90;
-
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.util.TestUtil;
-
-/** Tests Lucene90DocValuesFormat */
-public class TestBestCompressionLucene90DocValuesFormat
-    extends BaseLucene90DocValuesFormatTestCase {
-  private final Codec codec =
-      TestUtil.alwaysDocValuesFormat(
-          new Lucene90DocValuesFormat(Lucene90DocValuesFormat.Mode.BEST_COMPRESSION));
-
-  @Override
-  protected Codec getCodec() {
-    return codec;
-  }
-}
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestSpeedLucene90DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestSpeedLucene90DocValuesFormat.java
deleted file mode 100644
index 6303795..0000000
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestBestSpeedLucene90DocValuesFormat.java
+++ /dev/null
@@ -1,32 +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.lucene90;
-
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.util.TestUtil;
-
-/** Tests Lucene90DocValuesFormat */
-public class TestBestSpeedLucene90DocValuesFormat extends BaseLucene90DocValuesFormatTestCase {
-  private final Codec codec =
-      TestUtil.alwaysDocValuesFormat(
-          new Lucene90DocValuesFormat(Lucene90DocValuesFormat.Mode.BEST_SPEED));
-
-  @Override
-  protected Codec getCodec() {
-    return codec;
-  }
-}
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestDocValuesCompression.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestDocValuesCompression.java
deleted file mode 100644
index 8df5260..0000000
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestDocValuesCompression.java
+++ /dev/null
@@ -1,313 +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.lucene90;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.SortedDocValuesField;
-import org.apache.lucene.document.SortedSetDocValuesField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
-
-public class TestDocValuesCompression extends LuceneTestCase {
-  private final Codec bestSpeed = new Lucene90Codec(Lucene90Codec.Mode.BEST_SPEED);
-  private final Codec bestCompression = new Lucene90Codec(Lucene90Codec.Mode.BEST_COMPRESSION);
-
-  public void testTermsDictCompressionForLowCardinalityFields() throws IOException {
-    final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD - 1;
-    Set<String> valuesSet = new HashSet<>();
-    for (int i = 0; i < CARDINALITY; ++i) {
-      final int length = TestUtil.nextInt(random(), 10, 30);
-      String value = TestUtil.randomSimpleString(random(), length);
-      valuesSet.add(value);
-    }
-
-    List<String> values = new ArrayList<>(valuesSet);
-    long sizeForBestSpeed = writeAndGetDocValueFileSize(bestSpeed, values);
-    long sizeForBestCompression = writeAndGetDocValueFileSize(bestCompression, values);
-
-    // Ensure terms dict data was not compressed for low-cardinality fields.
-    assertEquals(sizeForBestSpeed, sizeForBestCompression);
-  }
-
-  public void testTermsDictCompressionForHighCardinalityFields() throws IOException {
-    final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD << 1;
-    Set<String> valuesSet = new HashSet<>();
-    for (int i = 0; i < CARDINALITY; ++i) {
-      final int length = TestUtil.nextInt(random(), 10, 30);
-      String value = TestUtil.randomSimpleString(random(), length);
-      // Add common suffix for better compression ratio.
-      valuesSet.add(value + "_CommonPartBetterForCompression");
-    }
-
-    List<String> values = new ArrayList<>(valuesSet);
-    long sizeForBestSpeed = writeAndGetDocValueFileSize(bestSpeed, values);
-    long sizeForBestCompression = writeAndGetDocValueFileSize(bestCompression, values);
-
-    // Compression happened.
-    assertTrue(sizeForBestCompression < sizeForBestSpeed);
-  }
-
-  public void testReseekAfterSkipDecompression() throws IOException {
-    final int CARDINALITY = (Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE << 1) + 11;
-    Set<String> valueSet = new HashSet<>(CARDINALITY);
-    for (int i = 0; i < CARDINALITY; i++) {
-      valueSet.add(TestUtil.randomSimpleString(random(), 64));
-    }
-    List<String> values = new ArrayList<>(valueSet);
-    Collections.sort(values);
-    // Create one non-existent value just between block-1 and block-2.
-    String nonexistentValue =
-        values.get(Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE - 1)
-            + TestUtil.randomSimpleString(random(), 64, 128);
-    int docValues = values.size();
-
-    try (Directory directory = newDirectory()) {
-      Analyzer analyzer = new StandardAnalyzer();
-      IndexWriterConfig config = new IndexWriterConfig(analyzer);
-      config.setCodec(bestCompression);
-      config.setUseCompoundFile(false);
-      IndexWriter writer = new IndexWriter(directory, config);
-      for (int i = 0; i < 280; i++) {
-        Document doc = new Document();
-        doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
-        doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % docValues))));
-        writer.addDocument(doc);
-      }
-      writer.commit();
-      writer.forceMerge(1);
-      DirectoryReader dReader = DirectoryReader.open(writer);
-      writer.close();
-
-      LeafReader reader = getOnlyLeafReader(dReader);
-      // Check values count.
-      SortedDocValues ssdvMulti = reader.getSortedDocValues("sdv");
-      assertEquals(docValues, ssdvMulti.getValueCount());
-
-      // Seek to first block.
-      int ord1 = ssdvMulti.lookupTerm(new BytesRef(values.get(0)));
-      assertTrue(ord1 >= 0);
-      int ord2 = ssdvMulti.lookupTerm(new BytesRef(values.get(1)));
-      assertTrue(ord2 >= ord1);
-      // Ensure re-seek logic is correct after skip-decompression.
-      int nonexistentOrd2 = ssdvMulti.lookupTerm(new BytesRef(nonexistentValue));
-      assertTrue(nonexistentOrd2 < 0);
-      dReader.close();
-    }
-  }
-
-  public void testLargeTermsCompression() throws IOException {
-    final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD << 1;
-    Set<String> valuesSet = new HashSet<>();
-    for (int i = 0; i < CARDINALITY; ++i) {
-      final int length = TestUtil.nextInt(random(), 512, 1024);
-      valuesSet.add(TestUtil.randomSimpleString(random(), length));
-    }
-    int valuesCount = valuesSet.size();
-    List<String> values = new ArrayList<>(valuesSet);
-
-    try (Directory directory = newDirectory()) {
-      Analyzer analyzer = new StandardAnalyzer();
-      IndexWriterConfig config = new IndexWriterConfig(analyzer);
-      config.setCodec(bestCompression);
-      config.setUseCompoundFile(false);
-      IndexWriter writer = new IndexWriter(directory, config);
-      for (int i = 0; i < 256; i++) {
-        Document doc = new Document();
-        doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
-        doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
-        writer.addDocument(doc);
-      }
-      writer.commit();
-      writer.forceMerge(1);
-      DirectoryReader ireader = DirectoryReader.open(writer);
-      writer.close();
-
-      LeafReader reader = getOnlyLeafReader(ireader);
-      // Check values count.
-      SortedDocValues ssdvMulti = reader.getSortedDocValues("sdv");
-      assertEquals(valuesCount, ssdvMulti.getValueCount());
-      ireader.close();
-    }
-  }
-
-  // Ensure the old segment can be merged together with the new compressed segment.
-  public void testMergeWithUncompressedSegment() throws IOException {
-    final int CARDINALITY = Lucene90DocValuesFormat.TERMS_DICT_BLOCK_COMPRESSION_THRESHOLD << 1;
-    Set<String> valuesSet = new HashSet<>();
-    for (int i = 0; i < CARDINALITY; ++i) {
-      final int length = TestUtil.nextInt(random(), 10, 30);
-      // Add common suffix for better compression ratio.
-      valuesSet.add(TestUtil.randomSimpleString(random(), length));
-    }
-    List<String> values = new ArrayList<>(valuesSet);
-    int valuesCount = values.size();
-
-    try (Directory directory = newDirectory()) {
-      // 1. Write 256 documents without terms dict compression.
-      Analyzer analyzer = new StandardAnalyzer();
-      IndexWriterConfig config = new IndexWriterConfig(analyzer);
-      config.setCodec(bestSpeed);
-      config.setUseCompoundFile(false);
-      IndexWriter writer = new IndexWriter(directory, config);
-      for (int i = 0; i < 256; i++) {
-        Document doc = new Document();
-        doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
-        doc.add(new SortedSetDocValuesField("ssdv", new BytesRef(values.get(i % valuesCount))));
-        doc.add(
-            new SortedSetDocValuesField("ssdv", new BytesRef(values.get((i + 1) % valuesCount))));
-        doc.add(
-            new SortedSetDocValuesField("ssdv", new BytesRef(values.get((i + 2) % valuesCount))));
-        doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
-        writer.addDocument(doc);
-      }
-      writer.commit();
-      DirectoryReader ireader = DirectoryReader.open(writer);
-      assertEquals(256, ireader.numDocs());
-      LeafReader reader = getOnlyLeafReader(ireader);
-      SortedSetDocValues ssdv = reader.getSortedSetDocValues("ssdv");
-      assertEquals(valuesCount, ssdv.getValueCount());
-      SortedDocValues sdv = reader.getSortedDocValues("sdv");
-      assertEquals(valuesCount, sdv.getValueCount());
-      ireader.close();
-      writer.close();
-
-      // 2. Add another 100 documents, and enabling terms dict compression.
-      config = new IndexWriterConfig(analyzer);
-      config.setCodec(bestCompression);
-      config.setUseCompoundFile(false);
-      writer = new IndexWriter(directory, config);
-      // Add 2 new values.
-      valuesSet.add(TestUtil.randomSimpleString(random(), 10));
-      valuesSet.add(TestUtil.randomSimpleString(random(), 10));
-      values = new ArrayList<>(valuesSet);
-      valuesCount = valuesSet.size();
-
-      for (int i = 256; i < 356; i++) {
-        Document doc = new Document();
-        doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
-        doc.add(new SortedSetDocValuesField("ssdv", new BytesRef(values.get(i % valuesCount))));
-        doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
-        writer.addDocument(doc);
-      }
-      writer.commit();
-      writer.forceMerge(1);
-      ireader = DirectoryReader.open(writer);
-      assertEquals(356, ireader.numDocs());
-      reader = getOnlyLeafReader(ireader);
-      ssdv = reader.getSortedSetDocValues("ssdv");
-      assertEquals(valuesCount, ssdv.getValueCount());
-      ireader.close();
-      writer.close();
-    }
-  }
-
-  private static long writeAndGetDocValueFileSize(Codec codec, List<String> values)
-      throws IOException {
-    int valuesCount = values.size();
-    long dvdFileSize = -1;
-    try (Directory directory = newDirectory()) {
-      Analyzer analyzer = new StandardAnalyzer();
-      IndexWriterConfig config = new IndexWriterConfig(analyzer);
-      config.setCodec(codec);
-      config.setUseCompoundFile(false);
-      IndexWriter writer = new IndexWriter(directory, config);
-      for (int i = 0; i < 256; i++) {
-        Document doc = new Document();
-        doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
-        // Multi value sorted-set field.
-        doc.add(
-            new SortedSetDocValuesField("ssdv_multi_", new BytesRef(values.get(i % valuesCount))));
-        doc.add(
-            new SortedSetDocValuesField(
-                "ssdv_multi_", new BytesRef(values.get((i + 1) % valuesCount))));
-        doc.add(
-            new SortedSetDocValuesField(
-                "ssdv_multi_", new BytesRef(values.get((i + 2) % valuesCount))));
-        // Single value sorted-set field.
-        doc.add(
-            new SortedSetDocValuesField("ssdv_single_", new BytesRef(values.get(i % valuesCount))));
-        // Sorted field.
-        doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
-        writer.addDocument(doc);
-      }
-      writer.commit();
-      writer.forceMerge(1);
-      DirectoryReader ireader = DirectoryReader.open(writer);
-      writer.close();
-
-      LeafReader reader = getOnlyLeafReader(ireader);
-      // Check values count.
-      SortedSetDocValues ssdvMulti = reader.getSortedSetDocValues("ssdv_multi_");
-      assertEquals(valuesCount, ssdvMulti.getValueCount());
-      for (int i = 0; i < valuesCount; i++) {
-        BytesRef term = ssdvMulti.lookupOrd(i);
-        assertTrue(term.bytes.length > 0);
-      }
-      for (int i = 0; i < valuesCount; i++) {
-        for (int j = 0; j < 3; j++) {
-          assertTrue(ssdvMulti.lookupTerm(new BytesRef(values.get((i + j) % valuesCount))) >= 0);
-        }
-      }
-
-      SortedSetDocValues ssdvSingle = reader.getSortedSetDocValues("ssdv_single_");
-      assertEquals(valuesCount, ssdvSingle.getValueCount());
-      for (int i = 0; i < valuesCount; i++) {
-        assertTrue(ssdvSingle.lookupTerm(new BytesRef(values.get(i % valuesCount))) >= 0);
-      }
-
-      SortedDocValues sdv = reader.getSortedDocValues("sdv");
-      assertEquals(valuesCount, sdv.getValueCount());
-      for (int i = 0; i < valuesCount; i++) {
-        assertTrue(sdv.lookupTerm(new BytesRef(values.get(i % valuesCount))) >= 0);
-      }
-
-      dvdFileSize = docValueFileSize(directory);
-      assertTrue(dvdFileSize > 0);
-      ireader.close();
-    }
-
-    return dvdFileSize;
-  }
-
-  static long docValueFileSize(Directory d) throws IOException {
-    for (String file : d.listAll()) {
-      if (file.endsWith(Lucene90DocValuesFormat.DATA_EXTENSION)) {
-        return d.fileLength(file);
-      }
-    }
-    return -1;
-  }
-}
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/BaseLucene90DocValuesFormatTestCase.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormat.java
similarity index 88%
rename from lucene/core/src/test/org/apache/lucene/codecs/lucene90/BaseLucene90DocValuesFormatTestCase.java
rename to lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormat.java
index 19bc542..79c744a 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/BaseLucene90DocValuesFormatTestCase.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormat.java
@@ -26,7 +26,10 @@ import java.util.Set;
 import java.util.TreeSet;
 import java.util.function.LongSupplier;
 import java.util.function.Supplier;
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.asserting.AssertingCodec;
@@ -68,8 +71,13 @@ import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.TestUtil;
 
 /** Tests Lucene90DocValuesFormat */
-public abstract class BaseLucene90DocValuesFormatTestCase
-    extends BaseCompressingDocValuesFormatTestCase {
+public class TestLucene90DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
+  private final Codec codec = TestUtil.getDefaultCodec();
+
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  }
 
   // TODO: these big methods can easily blow up some of the other ram-hungry codecs...
   // for now just keep them here, as we want to test this for this format.
@@ -761,4 +769,87 @@ public abstract class BaseLucene90DocValuesFormatTestCase
     }
     ir.close();
   }
+
+  public void testReseekAfterSkipDecompression() throws IOException {
+    final int CARDINALITY = (Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE << 1) + 11;
+    Set<String> valueSet = new HashSet<>(CARDINALITY);
+    for (int i = 0; i < CARDINALITY; i++) {
+      valueSet.add(TestUtil.randomSimpleString(random(), 64));
+    }
+    List<String> values = new ArrayList<>(valueSet);
+    Collections.sort(values);
+    // Create one non-existent value just between block-1 and block-2.
+    String nonexistentValue =
+        values.get(Lucene90DocValuesFormat.TERMS_DICT_BLOCK_LZ4_SIZE - 1)
+            + TestUtil.randomSimpleString(random(), 64, 128);
+    int docValues = values.size();
+
+    try (Directory directory = newDirectory()) {
+      Analyzer analyzer = new StandardAnalyzer();
+      IndexWriterConfig config = new IndexWriterConfig(analyzer);
+      config.setCodec(getCodec());
+      config.setUseCompoundFile(false);
+      IndexWriter writer = new IndexWriter(directory, config);
+      for (int i = 0; i < 280; i++) {
+        Document doc = new Document();
+        doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
+        doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % docValues))));
+        writer.addDocument(doc);
+      }
+      writer.commit();
+      writer.forceMerge(1);
+      DirectoryReader dReader = DirectoryReader.open(writer);
+      writer.close();
+
+      LeafReader reader = getOnlyLeafReader(dReader);
+      // Check values count.
+      SortedDocValues ssdvMulti = reader.getSortedDocValues("sdv");
+      assertEquals(docValues, ssdvMulti.getValueCount());
+
+      // Seek to first block.
+      int ord1 = ssdvMulti.lookupTerm(new BytesRef(values.get(0)));
+      assertTrue(ord1 >= 0);
+      int ord2 = ssdvMulti.lookupTerm(new BytesRef(values.get(1)));
+      assertTrue(ord2 >= ord1);
+      // Ensure re-seek logic is correct after skip-decompression.
+      int nonexistentOrd2 = ssdvMulti.lookupTerm(new BytesRef(nonexistentValue));
+      assertTrue(nonexistentOrd2 < 0);
+      dReader.close();
+    }
+  }
+
+  public void testLargeTermsCompression() throws IOException {
+    final int CARDINALITY = 64;
+    Set<String> valuesSet = new HashSet<>();
+    for (int i = 0; i < CARDINALITY; ++i) {
+      final int length = TestUtil.nextInt(random(), 512, 1024);
+      valuesSet.add(TestUtil.randomSimpleString(random(), length));
+    }
+    int valuesCount = valuesSet.size();
+    List<String> values = new ArrayList<>(valuesSet);
+
+    try (Directory directory = newDirectory()) {
+      Analyzer analyzer = new StandardAnalyzer();
+      IndexWriterConfig config = new IndexWriterConfig(analyzer);
+      config.setCodec(getCodec());
+      config.setUseCompoundFile(false);
+      IndexWriter writer = new IndexWriter(directory, config);
+      for (int i = 0; i < 256; i++) {
+        Document doc = new Document();
+        doc.add(new StringField("id", "Doc" + i, Field.Store.NO));
+        doc.add(new SortedDocValuesField("sdv", new BytesRef(values.get(i % valuesCount))));
+        writer.addDocument(doc);
+      }
+      writer.commit();
+      writer.forceMerge(1);
+      DirectoryReader ireader = DirectoryReader.open(writer);
+      writer.close();
+
+      LeafReader reader = getOnlyLeafReader(ireader);
+      // Check values count.
+      SortedDocValues ssdvMulti = reader.getSortedDocValues("sdv");
+      assertEquals(valuesCount, ssdvMulti.getValueCount());
+      ireader.close();
+    }
+  }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
index cdd85da..1bd2656 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
@@ -228,7 +228,7 @@ public class RandomCodec extends AssertingCodec {
     addDocValues(
         avoidCodecs,
         TestUtil.getDefaultDocValuesFormat(),
-        new Lucene90DocValuesFormat(Lucene90DocValuesFormat.Mode.BEST_COMPRESSION),
+        new Lucene90DocValuesFormat(),
         new AssertingDocValuesFormat());
 
     Collections.shuffle(formats, random);