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/09/03 10:54:16 UTC

[lucene-solr] 01/02: LUCENE-9486: Use preset dictionaries with LZ4 for BEST_SPEED. (#1793)

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

commit 4cedd92dee0ad1e9e3c8f655574d2af8ab1abd37
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Thu Sep 3 12:17:04 2020 +0200

    LUCENE-9486: Use preset dictionaries with LZ4 for BEST_SPEED. (#1793)
---
 lucene/CHANGES.txt                                 |   4 +-
 .../codecs/blocktree/CompressionAlgorithm.java     |   2 +-
 .../lucene/codecs/compressing/CompressionMode.java |   2 +-
 .../codecs/lucene80/Lucene80DocValuesProducer.java |   2 +-
 .../DeflateWithPresetDictCompressionMode.java      | 227 +++++++++++++++++++++
 .../lucene87/LZ4WithPresetDictCompressionMode.java | 199 ++++++++++++++++++
 .../lucene87/Lucene87StoredFieldsFormat.java       | 221 ++------------------
 .../java/org/apache/lucene/util/compress/LZ4.java  |  80 ++++++--
 .../apache/lucene/util/compress/LZ4TestCase.java   |  81 +++++++-
 .../luke/models/commits/CommitsImplTest.java       |   7 +-
 .../codecs/compressing/CompressingCodec.java       |   4 +-
 .../DeflateWithPresetCompressingCodec.java         |   6 +-
 ...dec.java => LZ4WithPresetCompressingCodec.java} |  14 +-
 .../services/org.apache.lucene.codecs.Codec        |   1 +
 14 files changed, 597 insertions(+), 253 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index ba969ea..92f94fc 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -45,8 +45,8 @@ Optimizations
 * LUCENE-9395: ConstantValuesSource now shares a single DoubleValues
   instance across all segments (Tony Xu)
 
-* LUCENE-9447: BEST_COMPRESSION now provides higher compression ratios on highly
-  compressible data. (Adrien Grand)
+* LUCENE-9447, LUCENE-9486: Stored fields now get higer compression ratios on
+  highly compressible data. (Adrien Grand)
 
 * LUCENE-9373: FunctionMatchQuery now accepts a "matchCost" optimization hint.
   (Maxim Glazkov, David Smiley)
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/CompressionAlgorithm.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/CompressionAlgorithm.java
index 98e2a42..a98a3ca 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/CompressionAlgorithm.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/CompressionAlgorithm.java
@@ -48,7 +48,7 @@ enum CompressionAlgorithm {
 
     @Override
     void read(DataInput in, byte[] out, int len) throws IOException {
-      org.apache.lucene.util.compress.LZ4.decompress(in, len, out);
+      org.apache.lucene.util.compress.LZ4.decompress(in, len, out, 0);
     }
 
   };
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java
index 5850ec9..e5a3c4e 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressionMode.java
@@ -136,7 +136,7 @@ public abstract class CompressionMode {
       if (bytes.bytes.length < originalLength + 7) {
         bytes.bytes = new byte[ArrayUtil.oversize(originalLength + 7, 1)];
       }
-      final int decompressedLength = LZ4.decompress(in, offset + length, bytes.bytes);
+      final int decompressedLength = LZ4.decompress(in, offset + length, bytes.bytes, 0);
       if (decompressedLength > originalLength) {
         throw new CorruptIndexException("Corrupted: lengths mismatch: " + decompressedLength + " > " + originalLength, in);
       }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
index 625ec94..5e721e8 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene80/Lucene80DocValuesProducer.java
@@ -832,7 +832,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
         }
         
         assert uncompressedBlockLength <= uncompressedBlock.length;
-        LZ4.decompress(compressedData, uncompressedBlockLength, uncompressedBlock);
+        LZ4.decompress(compressedData, uncompressedBlockLength, uncompressedBlock, 0);
       }
       
       uncompressedBytesRef.offset = uncompressedDocStarts[docInBlockId];        
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/DeflateWithPresetDictCompressionMode.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/DeflateWithPresetDictCompressionMode.java
new file mode 100644
index 0000000..09d52ad
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/DeflateWithPresetDictCompressionMode.java
@@ -0,0 +1,227 @@
+/*
+ * 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.lucene87;
+
+import java.io.IOException;
+import java.util.zip.DataFormatException;
+import java.util.zip.Deflater;
+import java.util.zip.Inflater;
+
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.codecs.compressing.Compressor;
+import org.apache.lucene.codecs.compressing.Decompressor;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A compression mode that trades speed for compression ratio. Although
+ * compression and decompression might be slow, this compression mode should
+ * provide a good compression ratio. This mode might be interesting if/when
+ * your index size is much bigger than your OS cache.
+ * @lucene.internal
+ */
+public final class DeflateWithPresetDictCompressionMode extends CompressionMode {
+
+  private final int dictLength, subBlockLength;
+
+  /** Sole constructor. */
+  public DeflateWithPresetDictCompressionMode(int dictLength, int subBlockLength) {
+    this.dictLength = dictLength;
+    this.subBlockLength = subBlockLength;
+  }
+
+  @Override
+  public Compressor newCompressor() {
+    // notes:
+    // 3 is the highest level that doesn't have lazy match evaluation
+    // 6 is the default, higher than that is just a waste of cpu
+    return new DeflateWithPresetDictCompressor(6, dictLength, subBlockLength);
+  }
+
+  @Override
+  public Decompressor newDecompressor() {
+    return new DeflateWithPresetDictDecompressor();
+  }
+
+  @Override
+  public String toString() {
+    return "BEST_COMPRESSION";
+  }
+
+  private static final class DeflateWithPresetDictDecompressor extends Decompressor {
+
+    byte[] compressed;
+
+    DeflateWithPresetDictDecompressor() {
+      compressed = new byte[0];
+    }
+
+    private void doDecompress(DataInput in, Inflater decompressor, BytesRef bytes) throws IOException {
+      final int compressedLength = in.readVInt();
+      if (compressedLength == 0) {
+        return;
+      }
+      // pad with extra "dummy byte": see javadocs for using Inflater(true)
+      // we do it for compliance, but it's unnecessary for years in zlib.
+      final int paddedLength = compressedLength + 1;
+      compressed = ArrayUtil.grow(compressed, paddedLength);
+      in.readBytes(compressed, 0, compressedLength);
+      compressed[compressedLength] = 0; // explicitly set dummy byte to 0
+
+      // extra "dummy byte"
+      decompressor.setInput(compressed, 0, paddedLength);
+      try {
+        bytes.length += decompressor.inflate(bytes.bytes, bytes.length, bytes.bytes.length - bytes.length);
+      } catch (DataFormatException e) {
+        throw new IOException(e);
+      }
+      if (decompressor.finished() == false) {
+        throw new CorruptIndexException("Invalid decoder state: needsInput=" + decompressor.needsInput()
+        + ", needsDict=" + decompressor.needsDictionary(), in);
+      }
+    }
+
+    @Override
+    public void decompress(DataInput in, int originalLength, int offset, int length, BytesRef bytes) throws IOException {
+      assert offset + length <= originalLength;
+      if (length == 0) {
+        bytes.length = 0;
+        return;
+      }
+      final int dictLength = in.readVInt();
+      final int blockLength = in.readVInt();
+      bytes.bytes = ArrayUtil.grow(bytes.bytes, dictLength);
+      bytes.offset = bytes.length = 0;
+
+      final Inflater decompressor = new Inflater(true);
+      try {
+        // Read the dictionary
+        doDecompress(in, decompressor, bytes);
+        if (dictLength != bytes.length) {
+          throw new CorruptIndexException("Unexpected dict length", in);
+        }
+
+        int offsetInBlock = dictLength;
+        int offsetInBytesRef = offset;
+
+        // Skip unneeded blocks
+        while (offsetInBlock + blockLength < offset) {
+          final int compressedLength = in.readVInt();
+          in.skipBytes(compressedLength);
+          offsetInBlock += blockLength;
+          offsetInBytesRef -= blockLength;
+        }
+
+        // Read blocks that intersect with the interval we need
+        while (offsetInBlock < offset + length) {
+          bytes.bytes = ArrayUtil.grow(bytes.bytes, bytes.length + blockLength);
+          decompressor.reset();
+          decompressor.setDictionary(bytes.bytes, 0, dictLength);
+          doDecompress(in, decompressor, bytes);
+          offsetInBlock += blockLength;
+        }
+
+        bytes.offset = offsetInBytesRef;
+        bytes.length = length;
+        assert bytes.isValid();
+      } finally {
+        decompressor.end();
+      }
+    }
+
+    @Override
+    public Decompressor clone() {
+      return new DeflateWithPresetDictDecompressor();
+    }
+
+  }
+
+  private static class DeflateWithPresetDictCompressor extends Compressor {
+
+    final byte[] dictBytes;
+    final int blockLength;
+    final Deflater compressor;
+    byte[] compressed;
+    boolean closed;
+
+    DeflateWithPresetDictCompressor(int level, int dictLength, int blockLength) {
+      compressor = new Deflater(level, true);
+      compressed = new byte[64];
+      this.dictBytes = new byte[dictLength];
+      this.blockLength = blockLength;
+    }
+
+    private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+      if (len == 0) {
+        out.writeVInt(0);
+        return;
+      }
+      compressor.setInput(bytes, off, len);
+      compressor.finish();
+      if (compressor.needsInput()) {
+        throw new IllegalStateException();
+      }
+
+      int totalCount = 0;
+      for (;;) {
+        final int count = compressor.deflate(compressed, totalCount, compressed.length - totalCount);
+        totalCount += count;
+        assert totalCount <= compressed.length;
+        if (compressor.finished()) {
+          break;
+        } else {
+          compressed = ArrayUtil.grow(compressed);
+        }
+      }
+
+      out.writeVInt(totalCount);
+      out.writeBytes(compressed, totalCount);
+    }
+
+    @Override
+    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+      final int dictLength = Math.min(dictBytes.length, len);
+      System.arraycopy(bytes, off, dictBytes, 0, dictLength);
+      out.writeVInt(dictLength);
+      out.writeVInt(blockLength);
+      final int end = off + len;
+
+      // Compress the dictionary first
+      compressor.reset();
+      doCompress(bytes, off, dictLength, out);
+
+      // And then sub blocks
+      for (int start = off + dictLength; start < end; start += blockLength) {
+        compressor.reset();
+        // NOTE: offset MUST be 0 when setting the dictionary in order to work around JDK-8252739
+        compressor.setDictionary(dictBytes, 0, dictLength);
+        doCompress(bytes, start, Math.min(blockLength, off + len - start), out);
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (closed == false) {
+        compressor.end();
+        closed = true;
+      }
+    }
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/LZ4WithPresetDictCompressionMode.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/LZ4WithPresetDictCompressionMode.java
new file mode 100644
index 0000000..0d10cfd
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/LZ4WithPresetDictCompressionMode.java
@@ -0,0 +1,199 @@
+/*
+ * 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.lucene87;
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.codecs.compressing.Compressor;
+import org.apache.lucene.codecs.compressing.Decompressor;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.compress.LZ4;
+
+/**
+ * A compression mode that compromises on the compression ratio to provide
+ * fast compression and decompression.
+ * @lucene.internal
+ */
+public final class LZ4WithPresetDictCompressionMode extends CompressionMode {
+
+  private final int dictLength, subBlockLength;
+
+  /** Sole constructor. */
+  public LZ4WithPresetDictCompressionMode(int dictLength, int subBlockLength) {
+    this.dictLength = dictLength;
+    this.subBlockLength = subBlockLength;
+  }
+
+  @Override
+  public Compressor newCompressor() {
+    return new LZ4WithPresetDictCompressor(dictLength, subBlockLength);
+  }
+
+  @Override
+  public Decompressor newDecompressor() {
+    return new LZ4WithPresetDictDecompressor();
+  }
+
+  @Override
+  public String toString() {
+    return "BEST_SPEED";
+  }
+
+  private static final class LZ4WithPresetDictDecompressor extends Decompressor {
+
+    private int[] compressedLengths;
+    private byte[] buffer;
+
+    LZ4WithPresetDictDecompressor() {
+      compressedLengths = new int[0];
+      buffer = new byte[0];
+    }
+
+    private int readCompressedLengths(DataInput in,  int originalLength, int dictLength, int blockLength) throws IOException {
+      in.readVInt(); // compressed length of the dictionary, unused
+      int totalLength = dictLength;
+      int i = 0;
+      while (totalLength < originalLength) {
+        compressedLengths = ArrayUtil.grow(compressedLengths, i+1);
+        compressedLengths[i++] = in.readVInt();
+        totalLength += blockLength;
+      }
+      return i;
+    }
+
+    @Override
+    public void decompress(DataInput in, int originalLength, int offset, int length, BytesRef bytes) throws IOException {
+      assert offset + length <= originalLength;
+
+      if (length == 0) {
+        bytes.length = 0;
+        return;
+      }
+
+      final int dictLength = in.readVInt();
+      final int blockLength = in.readVInt();
+
+      final int numBlocks = readCompressedLengths(in, originalLength, dictLength, blockLength);
+
+      buffer = ArrayUtil.grow(buffer, dictLength + blockLength);
+      bytes.length = 0;
+      // Read the dictionary
+      if (LZ4.decompress(in, dictLength, buffer, 0) != dictLength) {
+        throw new CorruptIndexException("Illegal dict length", in);
+      }
+
+      int offsetInBlock = dictLength;
+      int offsetInBytesRef = offset;
+      if (offset >= dictLength) {
+        offsetInBytesRef -= dictLength;
+
+        // Skip unneeded blocks
+        int numBytesToSkip = 0;
+        for (int i = 0; i < numBlocks && offsetInBlock + blockLength < offset; ++i) {
+          int compressedBlockLength = compressedLengths[i];
+          numBytesToSkip += compressedBlockLength;
+          offsetInBlock += blockLength;
+          offsetInBytesRef -= blockLength;
+        }
+        in.skipBytes(numBytesToSkip);
+      } else {
+        // The dictionary contains some bytes we need, copy its content to the BytesRef
+        bytes.bytes = ArrayUtil.grow(bytes.bytes, dictLength);
+        System.arraycopy(buffer, 0, bytes.bytes, 0, dictLength);
+        bytes.length = dictLength;
+      }
+
+      // Read blocks that intersect with the interval we need
+      while (offsetInBlock < offset + length) {
+        final int bytesToDecompress = Math.min(blockLength, offset + length - offsetInBlock);
+        LZ4.decompress(in, bytesToDecompress, buffer, dictLength);
+        bytes.bytes = ArrayUtil.grow(bytes.bytes, bytes.length + bytesToDecompress);
+        System.arraycopy(buffer, dictLength, bytes.bytes, bytes.length, bytesToDecompress);
+        bytes.length += bytesToDecompress;
+        offsetInBlock += blockLength;
+      }
+
+      bytes.offset = offsetInBytesRef;
+      bytes.length = length;
+      assert bytes.isValid();
+    }
+
+    @Override
+    public Decompressor clone() {
+      return new LZ4WithPresetDictDecompressor();
+    }
+
+  }
+
+  private static class LZ4WithPresetDictCompressor extends Compressor {
+
+    final int dictLength;
+    final int blockLength;
+    final ByteBuffersDataOutput compressed;
+    final LZ4.FastCompressionHashTable hashTable;
+    final byte[] buffer;
+
+    LZ4WithPresetDictCompressor(int dictLength, int blockLength) {
+      compressed = ByteBuffersDataOutput.newResettableInstance();
+      hashTable = new LZ4.FastCompressionHashTable();
+      this.dictLength = dictLength;
+      this.blockLength = blockLength;
+      buffer = new byte[dictLength + blockLength];
+    }
+
+    private void doCompress(byte[] bytes, int dictLen, int len, DataOutput out) throws IOException {
+      long prevCompressedSize = compressed.size();
+      LZ4.compressWithDictionary(bytes, 0, dictLen, len, compressed, hashTable);
+      // Write the number of compressed bytes
+      out.writeVInt(Math.toIntExact(compressed.size() - prevCompressedSize));
+    }
+
+    @Override
+    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
+      final int dictLength = Math.min(this.dictLength, len);
+      out.writeVInt(dictLength);
+      out.writeVInt(blockLength);
+      final int end = off + len;
+
+      compressed.reset();
+      // Compress the dictionary first
+      System.arraycopy(bytes, off, buffer, 0, dictLength);
+      doCompress(buffer, 0, dictLength, out);
+
+      // And then sub blocks
+      for (int start = off + dictLength; start < end; start += blockLength) {
+        int l = Math.min(blockLength, off + len - start);
+        System.arraycopy(bytes, start, buffer, dictLength, l);
+        doCompress(buffer, dictLength, l, out);
+      }
+
+      // We only wrote lengths so far, now write compressed data
+      compressed.copyTo(out);
+    }
+
+    @Override
+    public void close() throws IOException {
+      // no-op
+    }
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87StoredFieldsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87StoredFieldsFormat.java
index 915b949..bf9a267 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87StoredFieldsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene87/Lucene87StoredFieldsFormat.java
@@ -18,27 +18,17 @@ package org.apache.lucene.codecs.lucene87;
 
 import java.io.IOException;
 import java.util.Objects;
-import java.util.zip.DataFormatException;
-import java.util.zip.Deflater;
-import java.util.zip.Inflater;
 
 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.CompressionMode;
-import org.apache.lucene.codecs.compressing.Compressor;
-import org.apache.lucene.codecs.compressing.Decompressor;
-import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.StoredFieldVisitor;
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.packed.DirectMonotonicWriter;
 
 /**
@@ -154,7 +144,7 @@ public class Lucene87StoredFieldsFormat extends StoredFieldsFormat {
   StoredFieldsFormat impl(Mode mode) {
     switch (mode) {
       case BEST_SPEED:
-        return new CompressingStoredFieldsFormat("Lucene87StoredFieldsFastData", CompressionMode.FAST, 16*1024, 128, 10);
+        return new CompressingStoredFieldsFormat("Lucene87StoredFieldsFastData", BEST_SPEED_MODE, BEST_SPEED_BLOCK_LENGTH, 512, 10);
       case BEST_COMPRESSION:
         return new CompressingStoredFieldsFormat("Lucene87StoredFieldsHighData", BEST_COMPRESSION_MODE, BEST_COMPRESSION_BLOCK_LENGTH, 512, 10);
       default: throw new AssertionError();
@@ -179,202 +169,17 @@ public class Lucene87StoredFieldsFormat extends StoredFieldsFormat {
   private static final int BEST_COMPRESSION_BLOCK_LENGTH = BEST_COMPRESSION_DICT_LENGTH + 10 * BEST_COMPRESSION_SUB_BLOCK_LENGTH - 8 * 1024;
 
   /** Compression mode for {@link Mode#BEST_COMPRESSION} */
-  public static final DeflateWithPresetDict BEST_COMPRESSION_MODE = new DeflateWithPresetDict(BEST_COMPRESSION_DICT_LENGTH, BEST_COMPRESSION_SUB_BLOCK_LENGTH);
-
-  /**
-   * A compression mode that trades speed for compression ratio. Although
-   * compression and decompression might be slow, this compression mode should
-   * provide a good compression ratio. This mode might be interesting if/when
-   * your index size is much bigger than your OS cache.
-   */
-  public static class DeflateWithPresetDict extends CompressionMode {
-
-    private final int dictLength, subBlockLength;
-
-    /** Sole constructor. */
-    public DeflateWithPresetDict(int dictLength, int subBlockLength) {
-      this.dictLength = dictLength;
-      this.subBlockLength = subBlockLength;
-    }
-
-    @Override
-    public Compressor newCompressor() {
-      // notes:
-      // 3 is the highest level that doesn't have lazy match evaluation
-      // 6 is the default, higher than that is just a waste of cpu
-      return new DeflateWithPresetDictCompressor(6, dictLength, subBlockLength);
-    }
-
-    @Override
-    public Decompressor newDecompressor() {
-      return new DeflateWithPresetDictDecompressor();
-    }
-
-    @Override
-    public String toString() {
-      return "BEST_COMPRESSION";
-    }
-
-  };
-
-  private static final class DeflateWithPresetDictDecompressor extends Decompressor {
-
-    byte[] compressed;
-
-    DeflateWithPresetDictDecompressor() {
-      compressed = new byte[0];
-    }
-
-    private void doDecompress(DataInput in, Inflater decompressor, BytesRef bytes) throws IOException {
-      final int compressedLength = in.readVInt();
-      if (compressedLength == 0) {
-        return;
-      }
-      // pad with extra "dummy byte": see javadocs for using Inflater(true)
-      // we do it for compliance, but it's unnecessary for years in zlib.
-      final int paddedLength = compressedLength + 1;
-      compressed = ArrayUtil.grow(compressed, paddedLength);
-      in.readBytes(compressed, 0, compressedLength);
-      compressed[compressedLength] = 0; // explicitly set dummy byte to 0
-
-      // extra "dummy byte"
-      decompressor.setInput(compressed, 0, paddedLength);
-      try {
-        bytes.length += decompressor.inflate(bytes.bytes, bytes.length, bytes.bytes.length - bytes.length);
-      } catch (DataFormatException e) {
-        throw new IOException(e);
-      }
-      if (decompressor.finished() == false) {
-        throw new CorruptIndexException("Invalid decoder state: needsInput=" + decompressor.needsInput()
-        + ", needsDict=" + decompressor.needsDictionary(), in);
-      }
-    }
-
-    @Override
-    public void decompress(DataInput in, int originalLength, int offset, int length, BytesRef bytes) throws IOException {
-      assert offset + length <= originalLength;
-      if (length == 0) {
-        bytes.length = 0;
-        return;
-      }
-      final int dictLength = in.readVInt();
-      final int blockLength = in.readVInt();
-      bytes.bytes = ArrayUtil.grow(bytes.bytes, dictLength);
-      bytes.offset = bytes.length = 0;
-
-      final Inflater decompressor = new Inflater(true);
-      try {
-        // Read the dictionary
-        doDecompress(in, decompressor, bytes);
-        if (dictLength != bytes.length) {
-          throw new CorruptIndexException("Unexpected dict length", in);
-        }
-
-        int offsetInBlock = dictLength;
-        int offsetInBytesRef = offset;
-
-        // Skip unneeded blocks
-        while (offsetInBlock + blockLength < offset) {
-          final int compressedLength = in.readVInt();
-          in.skipBytes(compressedLength);
-          offsetInBlock += blockLength;
-          offsetInBytesRef -= blockLength;
-        }
-
-        // Read blocks that intersect with the interval we need
-        while (offsetInBlock < offset + length) {
-          bytes.bytes = ArrayUtil.grow(bytes.bytes, bytes.length + blockLength);
-          decompressor.reset();
-          decompressor.setDictionary(bytes.bytes, 0, dictLength);
-          doDecompress(in, decompressor, bytes);
-          offsetInBlock += blockLength;
-        }
-
-        bytes.offset = offsetInBytesRef;
-        bytes.length = length;
-        assert bytes.isValid();
-      } finally {
-        decompressor.end();
-      }
-    }
-
-    @Override
-    public Decompressor clone() {
-      return new DeflateWithPresetDictDecompressor();
-    }
-
-  }
-
-  private static class DeflateWithPresetDictCompressor extends Compressor {
-
-    final byte[] dictBytes;
-    final int blockLength;
-    final Deflater compressor;
-    byte[] compressed;
-    boolean closed;
-
-    DeflateWithPresetDictCompressor(int level, int dictLength, int blockLength) {
-      compressor = new Deflater(level, true);
-      compressed = new byte[64];
-      this.dictBytes = new byte[dictLength];
-      this.blockLength = blockLength;
-    }
-
-    private void doCompress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
-      if (len == 0) {
-        out.writeVInt(0);
-        return;
-      }
-      compressor.setInput(bytes, off, len);
-      compressor.finish();
-      if (compressor.needsInput()) {
-        throw new IllegalStateException();
-      }
-
-      int totalCount = 0;
-      for (;;) {
-        final int count = compressor.deflate(compressed, totalCount, compressed.length - totalCount);
-        totalCount += count;
-        assert totalCount <= compressed.length;
-        if (compressor.finished()) {
-          break;
-        } else {
-          compressed = ArrayUtil.grow(compressed);
-        }
-      }
-
-      out.writeVInt(totalCount);
-      out.writeBytes(compressed, totalCount);
-    }
-
-    @Override
-    public void compress(byte[] bytes, int off, int len, DataOutput out) throws IOException {
-      final int dictLength = Math.min(dictBytes.length, len);
-      System.arraycopy(bytes, off, dictBytes, 0, dictLength);
-      out.writeVInt(dictLength);
-      out.writeVInt(blockLength);
-      final int end = off + len;
-
-      // Compress the dictionary first
-      compressor.reset();
-      doCompress(bytes, off, dictLength, out);
-
-      // And then sub blocks
-      for (int start = off + dictLength; start < end; start += blockLength) {
-        compressor.reset();
-        // NOTE: offset MUST be 0 when setting the dictionary in order to work around JDK-8252739
-        compressor.setDictionary(dictBytes, 0, dictLength);
-        doCompress(bytes, start, Math.min(blockLength, off + len - start), out);
-      }
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (closed == false) {
-        compressor.end();
-        closed = true;
-      }
-    }
-  }
+  public static final CompressionMode BEST_COMPRESSION_MODE = new DeflateWithPresetDictCompressionMode(BEST_COMPRESSION_DICT_LENGTH, BEST_COMPRESSION_SUB_BLOCK_LENGTH);
+
+  // We need to re-initialize the hash table for every sub block with the
+  // content of the dictionary, so we keep it small to not hurt indexing.
+  private static final int BEST_SPEED_DICT_LENGTH = 4 * 1024;
+  // 60kB so that dict_length + block_length == max window size
+  private static final int BEST_SPEED_SUB_BLOCK_LENGTH = 60 * 1024;
+  // shoot for 10 sub blocks in addition to the dictionary
+  private static final int BEST_SPEED_BLOCK_LENGTH = BEST_SPEED_DICT_LENGTH + 10 * BEST_SPEED_SUB_BLOCK_LENGTH - 8 * 1024;
+
+  /** Compression mode for {@link Mode#BEST_SPEED} */
+  public static final CompressionMode BEST_SPEED_MODE = new LZ4WithPresetDictCompressionMode(BEST_SPEED_DICT_LENGTH, BEST_SPEED_SUB_BLOCK_LENGTH);
 
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/compress/LZ4.java b/lucene/core/src/java/org/apache/lucene/util/compress/LZ4.java
index 19021f5..2a471ea 100644
--- a/lucene/core/src/java/org/apache/lucene/util/compress/LZ4.java
+++ b/lucene/core/src/java/org/apache/lucene/util/compress/LZ4.java
@@ -56,7 +56,6 @@ public final class LZ4 {
   static final int LAST_LITERALS = 5; // the last 5 bytes must be encoded as literals
   static final int HASH_LOG_HC = 15; // log size of the dictionary for compressHC
   static final int HASH_TABLE_SIZE_HC = 1 << HASH_LOG_HC;
-  static final int OPTIMAL_ML = 0x0F + 4 - 1; // match length that doesn't require an additional byte
 
 
   private static int hash(int i, int hashBits) {
@@ -78,14 +77,15 @@ public final class LZ4 {
   }
 
   /**
-   * Decompress at least <code>decompressedLen</code> bytes into
-   * <code>dest[dOff:]</code>. Please note that <code>dest</code> must be large
+   * Decompress at least {@code decompressedLen} bytes into
+   * {@code dest[dOff:]}. Please note that {@code dest} must be large
    * enough to be able to hold <b>all</b> decompressed data (meaning that you
    * need to know the total decompressed length).
+   * If the given bytes were compressed using a preset dictionary then the same
+   * dictionary must be provided in {@code dest[dOff-dictLen:dOff]}.
    */
-  public static int decompress(DataInput compressed, int decompressedLen, byte[] dest) throws IOException {
-    int dOff = 0;
-    final int destEnd = dest.length;
+  public static int decompress(DataInput compressed, int decompressedLen, byte[] dest, int dOff) throws IOException {
+    final int destEnd = dOff + decompressedLen;
 
     do {
       // literals
@@ -104,7 +104,7 @@ public final class LZ4 {
         dOff += literalLen;
       }
 
-      if (dOff >= decompressedLen) {
+      if (dOff >= destEnd) {
         break;
       }
 
@@ -134,7 +134,7 @@ public final class LZ4 {
         System.arraycopy(dest, dOff - matchDec, dest, dOff, fastLen);
         dOff += matchLen;
       }
-    } while (dOff < decompressedLen);
+    } while (dOff < destEnd);
 
     return dOff;
   }
@@ -191,6 +191,9 @@ public final class LZ4 {
     /** Reset this hash table in order to compress the given content. */
     abstract void reset(byte[] b, int off, int len);
 
+    /** Init {@code dictLen} bytes to be used as a dictionary. */
+    abstract void initDictionary(int dictLen);
+
     /**
      * Advance the cursor to {@off} and return an index that stored the same
      * 4 bytes as {@code b[o:o+4)}. This may only be called on strictly
@@ -230,7 +233,6 @@ public final class LZ4 {
       FutureObjects.checkFromIndexSize(off, len, bytes.length);
       this.bytes = bytes;
       this.base = off;
-      this.lastOff = off - 1;
       this.end = off + len;
       final int bitsPerOffset = PackedInts.bitsRequired(len - LAST_LITERALS);
       final int bitsPerOffsetLog = 32 - Integer.numberOfLeadingZeros(bitsPerOffset - 1);
@@ -240,8 +242,18 @@ public final class LZ4 {
       } else {
         // Avoid calling hashTable.clear(), this makes it costly to compress many short sequences otherwise.
         // Instead, get() checks that references are less than the current offset.
-        get(off); // this sets the hashTable for the first 4 bytes as a side-effect
       }
+      this.lastOff = off - 1;
+    }
+
+    @Override
+    void initDictionary(int dictLen) {
+      for (int i = 0; i < dictLen; ++i) {
+        final int v = readInt(bytes, base + i);
+        final int h = hash(v, hashLog);
+        hashTable.set(h, i);
+      }
+      lastOff += dictLen;
     }
 
     @Override
@@ -329,8 +341,17 @@ public final class LZ4 {
     }
 
     @Override
+    void initDictionary(int dictLen) {
+      assert next == base;
+      for (int i = 0; i < dictLen; ++i) {
+        addHash(base + i);
+      }
+      next += dictLen;
+    }
+
+    @Override
     int get(int off) {
-      assert off > next;
+      assert off >= next;
       assert off < end;
 
       for (; next < off; next++) {
@@ -390,23 +411,40 @@ public final class LZ4 {
   }
 
   /**
-   * Compress <code>bytes[off:off+len]</code> into <code>out</code> using
-   * at most 16KB of memory. <code>ht</code> shouldn't be shared across threads
-   * but can safely be reused.
+   * Compress {@code bytes[off:off+len]} into {@code out} using at most 16kB of
+   * memory. {@code ht} shouldn't be shared across threads but can safely be
+   * reused.
    */
   public static void compress(byte[] bytes, int off, int len, DataOutput out, HashTable ht) throws IOException {
-    FutureObjects.checkFromIndexSize(off, len, bytes.length);
+    compressWithDictionary(bytes, off, 0, len, out, ht);
+  }
+
+  /**
+   * Compress {@code bytes[dictOff+dictLen:dictOff+dictLen+len]} into
+   * {@code out} using at most 16kB of memory.
+   * {@code bytes[dictOff:dictOff+dictLen]} will be used as a dictionary.
+   * {@code dictLen} must not be greater than 64kB, the maximum window size.
+   *
+   * {@code ht} shouldn't be shared across threads but can safely be reused.
+   */
+  public static void compressWithDictionary(byte[] bytes, int dictOff, int dictLen, int len, DataOutput out, HashTable ht) throws IOException {
+    FutureObjects.checkFromIndexSize(dictOff, dictLen, bytes.length);
+    FutureObjects.checkFromIndexSize(dictOff + dictLen, len, bytes.length);
+    if (dictLen > MAX_DISTANCE) {
+      throw new IllegalArgumentException("dictLen must not be greater than 64kB, but got " + dictLen);
+    }
 
-    final int base = off;
-    final int end = off + len;
+    final int end = dictOff + dictLen + len;
 
-    int anchor = off++;
+    int off = dictOff + dictLen;
+    int anchor = off;
 
     if (len > LAST_LITERALS + MIN_MATCH) {
 
       final int limit = end - LAST_LITERALS;
       final int matchLimit = limit - MIN_MATCH;
-      ht.reset(bytes, base, len);
+      ht.reset(bytes, dictOff, dictLen + len);
+      ht.initDictionary(dictLen);
 
       main:
       while (off <= limit) {
@@ -418,7 +456,7 @@ public final class LZ4 {
           }
           ref = ht.get(off);
           if (ref != -1) {
-            assert ref >= base && ref < off;
+            assert ref >= dictOff && ref < off;
             assert readInt(bytes, ref) == readInt(bytes, off);
             break;
           }
@@ -429,7 +467,7 @@ public final class LZ4 {
         int matchLen = MIN_MATCH + commonBytes(bytes, ref + MIN_MATCH, off + MIN_MATCH, limit);
 
         // try to find a better match
-        for (int r = ht.previous(ref), min = Math.max(off - MAX_DISTANCE + 1, base); r >= min; r = ht.previous(r)) {
+        for (int r = ht.previous(ref), min = Math.max(off - MAX_DISTANCE + 1, dictOff); r >= min; r = ht.previous(r)) {
           assert readInt(bytes, r) == readInt(bytes, off);
           int rMatchLen = MIN_MATCH + commonBytes(bytes, r + MIN_MATCH, off + MIN_MATCH, limit);
           if (rMatchLen > matchLen) {
diff --git a/lucene/core/src/test/org/apache/lucene/util/compress/LZ4TestCase.java b/lucene/core/src/test/org/apache/lucene/util/compress/LZ4TestCase.java
index 43f8c63..2d0e70b3 100644
--- a/lucene/core/src/test/org/apache/lucene/util/compress/LZ4TestCase.java
+++ b/lucene/core/src/test/org/apache/lucene/util/compress/LZ4TestCase.java
@@ -47,6 +47,12 @@ public abstract class LZ4TestCase extends LuceneTestCase {
     }
 
     @Override
+    void initDictionary(int dictLen) {
+      assertTrue(in.assertReset());
+      in.initDictionary(dictLen);
+    }
+
+    @Override
     int get(int off) {
       return in.get(off);
     }
@@ -64,7 +70,7 @@ public abstract class LZ4TestCase extends LuceneTestCase {
   }
 
   private void doTest(byte[] data, LZ4.HashTable hashTable) throws IOException {
-    int offset = random().nextBoolean()
+    int offset = data.length >= (1 << 16) || random().nextBoolean()
         ? random().nextInt(10)
         : (1<<16) - data.length / 2; // this triggers special reset logic for high compression
     byte[] copy = new byte[data.length + offset + random().nextInt(10)];
@@ -135,8 +141,57 @@ public abstract class LZ4TestCase extends LuceneTestCase {
 
     // Now restore and compare bytes
     byte[] restored = new byte[length + random().nextInt(10)];
-    LZ4.decompress(new ByteArrayDataInput(compressed), length, restored);
+    LZ4.decompress(new ByteArrayDataInput(compressed), length, restored, 0);
     assertArrayEquals(ArrayUtil.copyOfSubArray(data, offset, offset+length), ArrayUtil.copyOfSubArray(restored, 0, length));
+
+    // Now restore with an offset
+    int restoreOffset = TestUtil.nextInt(random(), 1, 10);
+    restored = new byte[restoreOffset + length + random().nextInt(10)];
+    LZ4.decompress(new ByteArrayDataInput(compressed), length, restored, restoreOffset);
+    assertArrayEquals(ArrayUtil.copyOfSubArray(data, offset, offset+length), ArrayUtil.copyOfSubArray(restored, restoreOffset, restoreOffset+length));
+  }
+
+  private void doTestWithDictionary(byte[] data, LZ4.HashTable hashTable) throws IOException {
+    ByteBuffersDataOutput copy = new ByteBuffersDataOutput();
+    int dictOff = TestUtil.nextInt(random(), 0, 10);
+    copy.writeBytes(new byte[dictOff]);
+
+    // Create a dictionary from substrings of the input to compress
+    int dictLen = 0;
+    for (int i = TestUtil.nextInt(random(), 0, data.length); i < data.length && dictLen < LZ4.MAX_DISTANCE; ) {
+      int l = Math.min(data.length - i, TestUtil.nextInt(random(), 1, 32));
+      l = Math.min(l, LZ4.MAX_DISTANCE - dictLen);
+      copy.writeBytes(data, i, l);
+      dictLen += l;
+      i += l;
+      i += TestUtil.nextInt(random(), 1, 32);
+    }
+
+    copy.writeBytes(data);
+    copy.writeBytes(new byte[random().nextInt(10)]);
+
+    byte[] copyBytes = copy.toArrayCopy();
+    doTestWithDictionary(copyBytes, dictOff, dictLen, data.length, hashTable);
+  }
+
+  private void doTestWithDictionary(byte[] data, int dictOff, int dictLen, int length, LZ4.HashTable hashTable) throws IOException {
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    LZ4.compressWithDictionary(data, dictOff, dictLen, length, out, hashTable);
+    byte[] compressed = out.toArrayCopy();
+
+    // Compress once again with the same hash table to test reuse
+    ByteBuffersDataOutput out2 = new ByteBuffersDataOutput();
+    LZ4.compressWithDictionary(data, dictOff, dictLen, length, out2, hashTable);
+    assertArrayEquals(compressed, out2.toArrayCopy());
+
+    // Now restore and compare bytes
+    int restoreOffset = TestUtil.nextInt(random(), 1, 10);
+    byte[] restored = new byte[restoreOffset + dictLen + length + random().nextInt(10)];
+    System.arraycopy(data, dictOff, restored, restoreOffset, dictLen);
+    LZ4.decompress(new ByteArrayDataInput(compressed), length, restored, dictLen + restoreOffset);
+    assertArrayEquals(
+        ArrayUtil.copyOfSubArray(data, dictOff+dictLen, dictOff+dictLen+length),
+        ArrayUtil.copyOfSubArray(restored, restoreOffset+dictLen, restoreOffset+dictLen+length));
   }
 
   public void testEmpty() throws IOException {
@@ -149,6 +204,7 @@ public abstract class LZ4TestCase extends LuceneTestCase {
     // literals and matchs lengths <= 15
     final byte[] data = "1234562345673456745678910123".getBytes(StandardCharsets.UTF_8);
     doTest(data, newHashTable());
+    doTestWithDictionary(data, newHashTable());
   }
 
   public void testLongMatchs() throws IOException {
@@ -179,10 +235,11 @@ public abstract class LZ4TestCase extends LuceneTestCase {
     byte[] b = new byte[TestUtil.nextInt(random(), 1, 1 << 32)];
     random().nextBytes(b);
     doTest(b, newHashTable());
+    doTestWithDictionary(b, newHashTable());
   }
 
   public void testCompressibleRandom() throws IOException {
-    byte[] b = new byte[TestUtil.nextInt(random(), 1, 1 << 32)];
+    byte[] b = new byte[TestUtil.nextInt(random(), 1, 1 << 18)];
     final int base = random().nextInt(256);
     final int maxDelta = 1 + random().nextInt(8);
     Random r = random();
@@ -190,6 +247,7 @@ public abstract class LZ4TestCase extends LuceneTestCase {
       b[i] = (byte) (base + r.nextInt(maxDelta));
     }
     doTest(b, newHashTable());
+    doTestWithDictionary(b, newHashTable());
   }
 
   public void testLUCENE5201() throws IOException {
@@ -245,4 +303,21 @@ public abstract class LZ4TestCase extends LuceneTestCase {
       };
     doTest(data, 9, data.length - 9, newHashTable());
   }
+
+  public void testUseDictionary() throws IOException {
+    byte[] b = new byte[] {
+        1, 2, 3, 4, 5, 6, // dictionary
+        0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12
+    };
+    int dictOff = 0;
+    int dictLen = 6;
+    int len = b.length - dictLen;
+
+    doTestWithDictionary(b, dictOff, dictLen, len, newHashTable());
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    LZ4.compressWithDictionary(b, dictOff, dictLen, len, out, newHashTable());
+
+    // The compressed output is smaller than the original input despite being incompressible on its own
+    assertTrue(out.size() < len);
+  }
 }
diff --git a/lucene/luke/src/test/org/apache/lucene/luke/models/commits/CommitsImplTest.java b/lucene/luke/src/test/org/apache/lucene/luke/models/commits/CommitsImplTest.java
index 5588754..84a9b43 100644
--- a/lucene/luke/src/test/org/apache/lucene/luke/models/commits/CommitsImplTest.java
+++ b/lucene/luke/src/test/org/apache/lucene/luke/models/commits/CommitsImplTest.java
@@ -33,14 +33,11 @@ import org.apache.lucene.index.NoDeletionPolicy;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-// See: https://github.com/DmitryKey/luke/issues/111
-@LuceneTestCase.SuppressCodecs({
-   "SimpleText", "DeflateWithPresetCompressingStoredFieldsData", "DummyCompressingStoredFieldsData", "HighCompressionCompressingStoredFieldsData", "FastCompressingStoredFieldsData", "FastDecompressionCompressingStoredFieldsData"
-})
 public class CommitsImplTest extends LuceneTestCase {
 
   private DirectoryReader reader;
@@ -63,7 +60,7 @@ public class CommitsImplTest extends LuceneTestCase {
 
     Directory dir = newFSDirectory(indexDir);
 
-    IndexWriterConfig config = new IndexWriterConfig(new MockAnalyzer(random()));
+    IndexWriterConfig config = new IndexWriterConfig(new MockAnalyzer(random())).setCodec(TestUtil.getDefaultCodec());
     config.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir, config);
 
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 9fd243f..1038d23 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
@@ -37,7 +37,7 @@ public abstract class CompressingCodec extends FilterCodec {
    * Create a random instance.
    */
   public static CompressingCodec randomInstance(Random random, int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockShift) {
-    switch (random.nextInt(5)) {
+    switch (random.nextInt(6)) {
     case 0:
       return new FastCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockShift);
     case 1:
@@ -48,6 +48,8 @@ public abstract class CompressingCodec extends FilterCodec {
       return new DummyCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockShift);
     case 4:
       return new DeflateWithPresetCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockShift);
+    case 5:
+      return new LZ4WithPresetCompressingCodec(chunkSize, maxDocsPerChunk, withSegmentSuffix, blockShift);
     default:
       throw new AssertionError();
     }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java
index 9d1791e..b65d355 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java
@@ -16,16 +16,16 @@
  */
 package org.apache.lucene.codecs.compressing;
 
-import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat.DeflateWithPresetDict;
+import org.apache.lucene.codecs.lucene87.DeflateWithPresetDictCompressionMode;
 
-/** CompressionCodec that uses {@link DeflateWithPresetDict}. */
+/** CompressionCodec that uses {@link DeflateWithPresetDictCompressionMode}. */
 public class DeflateWithPresetCompressingCodec extends CompressingCodec {
 
   /** Constructor that allows to configure the chunk size. */
   public DeflateWithPresetCompressingCodec(int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockSize) {
     super("DeflateWithPresetCompressingStoredFieldsData", 
           withSegmentSuffix ? "DeflateWithPresetCompressingStoredFields" : "",
-          new DeflateWithPresetDict(chunkSize/10, chunkSize/3+1), chunkSize, maxDocsPerChunk, blockSize);
+          new DeflateWithPresetDictCompressionMode(chunkSize/10, chunkSize/3+1), chunkSize, maxDocsPerChunk, blockSize);
   }
 
   /** No-arg constructor. */
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/LZ4WithPresetCompressingCodec.java
similarity index 65%
copy from lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java
copy to lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/LZ4WithPresetCompressingCodec.java
index 9d1791e..ea413fc 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/DeflateWithPresetCompressingCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/LZ4WithPresetCompressingCodec.java
@@ -16,20 +16,20 @@
  */
 package org.apache.lucene.codecs.compressing;
 
-import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat.DeflateWithPresetDict;
+import org.apache.lucene.codecs.lucene87.LZ4WithPresetDictCompressionMode;
 
-/** CompressionCodec that uses {@link DeflateWithPresetDict}. */
-public class DeflateWithPresetCompressingCodec extends CompressingCodec {
+/** CompressionCodec that uses {@link LZ4WithPresetDictCompressionMode}. */
+public class LZ4WithPresetCompressingCodec extends CompressingCodec {
 
   /** Constructor that allows to configure the chunk size. */
-  public DeflateWithPresetCompressingCodec(int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockSize) {
-    super("DeflateWithPresetCompressingStoredFieldsData", 
+  public LZ4WithPresetCompressingCodec(int chunkSize, int maxDocsPerChunk, boolean withSegmentSuffix, int blockSize) {
+    super("LZ4WithPresetCompressingStoredFieldsData", 
           withSegmentSuffix ? "DeflateWithPresetCompressingStoredFields" : "",
-          new DeflateWithPresetDict(chunkSize/10, chunkSize/3+1), chunkSize, maxDocsPerChunk, blockSize);
+          new LZ4WithPresetDictCompressionMode(chunkSize/10, chunkSize/3+1), chunkSize, maxDocsPerChunk, blockSize);
   }
 
   /** No-arg constructor. */
-  public DeflateWithPresetCompressingCodec() {
+  public LZ4WithPresetCompressingCodec() {
     this(1<<18, 512, false, 10);
   }
 
diff --git a/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
index 5892cb0..618b607 100644
--- a/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
+++ b/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
@@ -19,4 +19,5 @@ org.apache.lucene.codecs.compressing.DeflateWithPresetCompressingCodec
 org.apache.lucene.codecs.compressing.FastCompressingCodec
 org.apache.lucene.codecs.compressing.FastDecompressionCompressingCodec
 org.apache.lucene.codecs.compressing.HighCompressionCompressingCodec
+org.apache.lucene.codecs.compressing.LZ4WithPresetCompressingCodec
 org.apache.lucene.codecs.compressing.dummy.DummyCompressingCodec