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/17 17:11:47 UTC

[lucene-solr] branch branch_8x updated (7542168 -> 830bd18)

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

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


    from 7542168  LUCENE-9449 Skip docs with _doc sort and "after" (#1725) (#1856)
     new 54f1ddf  Further tune Lucene87StoredFieldsFormat for small documents. (#1888)
     new 830bd18  LUCENE-9529: Track dirtiness of stored fields via a number of docs, not chunks. (#1882)

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../compressing/CompressingStoredFieldsReader.java | 68 +++++++++++++---------
 .../compressing/CompressingStoredFieldsWriter.java | 13 +++--
 .../compressing/CompressingTermVectorsReader.java  | 32 ++++++----
 .../compressing/CompressingTermVectorsWriter.java  | 15 ++---
 .../lucene87/Lucene87StoredFieldsFormat.java       |  4 +-
 .../TestCompressingStoredFieldsFormat.java         |  3 +-
 .../TestCompressingTermVectorsFormat.java          |  2 +-
 7 files changed, 82 insertions(+), 55 deletions(-)


[lucene-solr] 02/02: LUCENE-9529: Track dirtiness of stored fields via a number of docs, not chunks. (#1882)

Posted by jp...@apache.org.
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 830bd186a8d72ce6cc96f2856c269ef02e98d3c5
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Thu Sep 17 18:59:08 2020 +0200

    LUCENE-9529: Track dirtiness of stored fields via a number of docs, not chunks. (#1882)
    
    The problem of tracking dirtiness via numbers of chunks is that larger
    chunks make stored fields readers more likely to be considered dirty, so
    I'm trying to work around it by tracking numbers of docs instead.
---
 .../compressing/CompressingStoredFieldsReader.java | 32 ++++++++++++++--------
 .../compressing/CompressingStoredFieldsWriter.java | 13 +++++----
 .../compressing/CompressingTermVectorsReader.java  | 32 ++++++++++++++--------
 .../compressing/CompressingTermVectorsWriter.java  | 15 +++++-----
 .../TestCompressingStoredFieldsFormat.java         |  3 +-
 .../TestCompressingTermVectorsFormat.java          |  2 +-
 6 files changed, 60 insertions(+), 37 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
index 0903100..1216b33 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
@@ -90,8 +90,8 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
   private final int numDocs;
   private final boolean merging;
   private final BlockState state;
-  private final long numChunks; // number of compressed blocks written
   private final long numDirtyChunks; // number of incomplete compressed blocks written
+  private final long numDirtyDocs; // cumulative number of missing docs in incomplete chunks
   private boolean closed;
 
   // used by clone
@@ -106,8 +106,8 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
     this.compressionMode = reader.compressionMode;
     this.decompressor = reader.decompressor.clone();
     this.numDocs = reader.numDocs;
-    this.numChunks = reader.numChunks;
     this.numDirtyChunks = reader.numDirtyChunks;
+    this.numDirtyDocs = reader.numDirtyDocs;
     this.merging = merging;
     this.state = new BlockState();
     this.closed = false;
@@ -187,15 +187,13 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
       this.indexReader = indexReader;
 
       if (version >= VERSION_META) {
-        numChunks = metaIn.readVLong();
         numDirtyChunks = metaIn.readVLong();
+        numDirtyDocs = metaIn.readVLong();
       } else {
-        fieldsStream.seek(maxPointer);
-        numChunks = fieldsStream.readVLong();
-        numDirtyChunks = fieldsStream.readVLong();
-      }
-      if (numDirtyChunks > numChunks) {
-        throw new CorruptIndexException("invalid chunk counts: dirty=" + numDirtyChunks + ", total=" + numChunks, fieldsStream);
+        // Old versions of this format did not record numDirtyDocs. Since bulk
+        // merges are disabled on version increments anyway, we make no effort
+        // to get valid values of numDirtyChunks and numDirtyDocs.
+        numDirtyChunks = numDirtyDocs = -1;
       }
 
       if (metaIn != null) {
@@ -693,14 +691,26 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
     return chunkSize;
   }
   
-  long getNumChunks() {
-    return numChunks;
+  long getNumDirtyDocs() {
+    if (version != VERSION_CURRENT) {
+      throw new IllegalStateException("getNumDirtyDocs should only ever get called when the reader is on the current version");
+    }
+    assert numDirtyDocs >= 0;
+    return numDirtyDocs;
   }
   
   long getNumDirtyChunks() {
+    if (version != VERSION_CURRENT) {
+      throw new IllegalStateException("getNumDirtyChunks should only ever get called when the reader is on the current version");
+    }
+    assert numDirtyChunks >= 0;
     return numDirtyChunks;
   }
 
+  int getNumDocs() {
+    return numDocs;
+  }
+
   int getPackedIntsVersion() {
     return packedIntsVersion;
   }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
index 23f8a01..ce782da 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
@@ -95,8 +95,8 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
   private int docBase; // doc ID at the beginning of the chunk
   private int numBufferedDocs; // docBase + numBufferedDocs == current doc ID
   
-  private long numChunks; // number of compressed blocks written
   private long numDirtyChunks; // number of incomplete compressed blocks written
+  private long numDirtyDocs; // cumulative number of missing docs in incomplete chunks
 
   /** Sole constructor. */
   CompressingStoredFieldsWriter(Directory directory, SegmentInfo si, String segmentSuffix, IOContext context,
@@ -252,7 +252,6 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
     docBase += numBufferedDocs;
     numBufferedDocs = 0;
     bufferedDocs.reset();
-    numChunks++;
   }
   
   @Override
@@ -468,8 +467,10 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
   @Override
   public void finish(FieldInfos fis, int numDocs) throws IOException {
     if (numBufferedDocs > 0) {
-      flush();
       numDirtyChunks++; // incomplete: we had to force this flush
+      final long expectedChunkDocs = Math.min(maxDocsPerChunk, (long) ((double) chunkSize / bufferedDocs.size() * numBufferedDocs));
+      numDirtyDocs += expectedChunkDocs - numBufferedDocs;
+      flush();
     } else {
       assert bufferedDocs.size() == 0;
     }
@@ -477,8 +478,8 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
       throw new RuntimeException("Wrote " + docBase + " docs, finish called with numDocs=" + numDocs);
     }
     indexWriter.finish(numDocs, fieldsStream.getFilePointer(), metaStream);
-    metaStream.writeVLong(numChunks);
     metaStream.writeVLong(numDirtyChunks);
+    metaStream.writeVLong(numDirtyDocs);
     CodecUtil.writeFooter(metaStream);
     CodecUtil.writeFooter(fieldsStream);
     assert bufferedDocs.size() == 0;
@@ -632,8 +633,8 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
         }
         
         // since we bulk merged all chunks, we inherit any dirty ones from this segment.
-        numChunks += matchingFieldsReader.getNumChunks();
         numDirtyChunks += matchingFieldsReader.getNumDirtyChunks();
+        numDirtyDocs += matchingFieldsReader.getNumDirtyDocs();
       } else {
         // optimized merge, we copy serialized (but decompressed) bytes directly
         // even on simple docs (1 stored field), it seems to help by about 20%
@@ -669,7 +670,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
   boolean tooDirty(CompressingStoredFieldsReader candidate) {
     // more than 1% dirty, or more than hard limit of 1024 dirty chunks
     return candidate.getNumDirtyChunks() > 1024 || 
-           candidate.getNumDirtyChunks() * 100 > candidate.getNumChunks();
+           candidate.getNumDirtyDocs() * 100 > candidate.getNumDocs();
   }
 
   private static class CompressingStoredFieldsMergeSub extends DocIDMerger.Sub {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
index d3bdc06..0d555a5 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
@@ -85,8 +85,8 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
   private final int numDocs;
   private boolean closed;
   private final BlockPackedReaderIterator reader;
-  private final long numChunks; // number of compressed blocks written
   private final long numDirtyChunks; // number of incomplete compressed blocks written
+  private final long numDirtyDocs; // cumulative number of missing docs in incomplete chunks
   private final long maxPointer; // end of the data section
 
   // used by clone
@@ -101,8 +101,8 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
     this.numDocs = reader.numDocs;
     this.reader = new BlockPackedReaderIterator(vectorsStream, packedIntsVersion, PACKED_BLOCK_SIZE, 0);
     this.version = reader.version;
-    this.numChunks = reader.numChunks;
     this.numDirtyChunks = reader.numDirtyChunks;
+    this.numDirtyDocs = reader.numDirtyDocs;
     this.maxPointer = reader.maxPointer;
     this.closed = false;
   }
@@ -178,15 +178,13 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
       this.maxPointer = maxPointer;
 
       if (version >= VERSION_META) {
-        numChunks = metaIn.readVLong();
         numDirtyChunks = metaIn.readVLong();
+        numDirtyDocs = metaIn.readVLong();
       } else {
-        vectorsStream.seek(maxPointer);
-        numChunks = vectorsStream.readVLong();
-        numDirtyChunks = vectorsStream.readVLong();
-      }
-      if (numDirtyChunks > numChunks) {
-        throw new CorruptIndexException("invalid chunk counts: dirty=" + numDirtyChunks + ", total=" + numChunks, vectorsStream);
+        // Old versions of this format did not record numDirtyDocs. Since bulk
+        // merges are disabled on version increments anyway, we make no effort
+        // to get valid values of numDirtyChunks and numDirtyDocs.
+        numDirtyChunks = numDirtyDocs = -1;
       }
 
       decompressor = compressionMode.newDecompressor();
@@ -240,14 +238,26 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
     return maxPointer;
   }
   
-  long getNumChunks() {
-    return numChunks;
+  long getNumDirtyDocs() {
+    if (version != VERSION_CURRENT) {
+      throw new IllegalStateException("getNumDirtyDocs should only ever get called when the reader is on the current version");
+    }
+    assert numDirtyDocs >= 0;
+    return numDirtyDocs;
   }
   
   long getNumDirtyChunks() {
+    if (version != VERSION_CURRENT) {
+      throw new IllegalStateException("getNumDirtyChunks should only ever get called when the reader is on the current version");
+    }
+    assert numDirtyChunks >= 0;
     return numDirtyChunks;
   }
 
+  int getNumDocs() {
+    return numDocs;
+  }
+
   /**
    * @throws AlreadyClosedException if this TermVectorsReader is closed
    */
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
index 3dc2ff6..702642d 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
@@ -86,9 +86,9 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
   private final CompressionMode compressionMode;
   private final Compressor compressor;
   private final int chunkSize;
-  
-  private long numChunks; // number of compressed blocks written
+
   private long numDirtyChunks; // number of incomplete compressed blocks written
+  private long numDirtyDocs; // cumulative number of missing docs in incomplete chunks
 
   /** a pending doc */
   private class DocData {
@@ -371,7 +371,6 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
     curDoc = null;
     curField = null;
     termSuffixes.reset();
-    numChunks++;
   }
 
   private int flushNumFields(int chunkDocs) throws IOException {
@@ -645,15 +644,17 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
   @Override
   public void finish(FieldInfos fis, int numDocs) throws IOException {
     if (!pendingDocs.isEmpty()) {
-      flush();
       numDirtyChunks++; // incomplete: we had to force this flush
+      final long expectedChunkDocs = Math.min(MAX_DOCUMENTS_PER_CHUNK, (long) ((double) chunkSize / termSuffixes.getPosition() * pendingDocs.size()));
+      numDirtyDocs += expectedChunkDocs - pendingDocs.size();
+      flush();
     }
     if (numDocs != this.numDocs) {
       throw new RuntimeException("Wrote " + this.numDocs + " docs, finish called with numDocs=" + numDocs);
     }
     indexWriter.finish(numDocs, vectorsStream.getFilePointer(), metaStream);
-    metaStream.writeVLong(numChunks);
     metaStream.writeVLong(numDirtyChunks);
+    metaStream.writeVLong(numDirtyDocs);
     CodecUtil.writeFooter(metaStream);
     CodecUtil.writeFooter(vectorsStream);
   }
@@ -817,8 +818,8 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
         }
         
         // since we bulk merged all chunks, we inherit any dirty ones from this segment.
-        numChunks += matchingVectorsReader.getNumChunks();
         numDirtyChunks += matchingVectorsReader.getNumDirtyChunks();
+        numDirtyDocs += matchingVectorsReader.getNumDirtyDocs();
       } else {        
         // naive merge...
         if (vectorsReader != null) {
@@ -853,7 +854,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
   boolean tooDirty(CompressingTermVectorsReader candidate) {
     // more than 1% dirty, or more than hard limit of 1024 dirty chunks
     return candidate.getNumDirtyChunks() > 1024 || 
-           candidate.getNumDirtyChunks() * 100 > candidate.getNumChunks();
+           candidate.getNumDirtyDocs() * 100 > candidate.getNumDocs();
   }
 
   @Override
diff --git a/lucene/test-framework/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java b/lucene/test-framework/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
index 1775e6d..07804c3 100644
--- a/lucene/test-framework/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
+++ b/lucene/test-framework/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
@@ -297,7 +297,8 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
       for (LeafReaderContext leaf : ir2.leaves()) {
         CodecReader sr = (CodecReader) leaf.reader();
         CompressingStoredFieldsReader reader = (CompressingStoredFieldsReader)sr.getFieldsReader();
-        assertEquals(1, reader.getNumChunks());
+        assertTrue(reader.getNumDirtyDocs() > 0);
+        assertTrue(reader.getNumDirtyDocs() < 100); // can't be gte the number of docs per chunk
         assertEquals(1, reader.getNumDirtyChunks());
       }
     }
diff --git a/lucene/test-framework/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java b/lucene/test-framework/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java
index e3477e9..e21ab6e 100644
--- a/lucene/test-framework/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java
+++ b/lucene/test-framework/src/test/org/apache/lucene/codecs/compressing/TestCompressingTermVectorsFormat.java
@@ -102,7 +102,7 @@ public class TestCompressingTermVectorsFormat extends BaseTermVectorsFormatTestC
       for (LeafReaderContext leaf : ir2.leaves()) {
         CodecReader sr = (CodecReader) leaf.reader();
         CompressingTermVectorsReader reader = (CompressingTermVectorsReader)sr.getTermVectorsReader();
-        assertEquals(1, reader.getNumChunks());
+        assertTrue(reader.getNumDirtyDocs() > 0);
         assertEquals(1, reader.getNumDirtyChunks());
       }
     }


[lucene-solr] 01/02: Further tune Lucene87StoredFieldsFormat for small documents. (#1888)

Posted by jp...@apache.org.
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 54f1ddf39741231c85b6c8b733b9208468f0a703
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Thu Sep 17 18:30:57 2020 +0200

    Further tune Lucene87StoredFieldsFormat for small documents. (#1888)
    
    The increase of the maximum number of chunks per doc done in previous
    issues was mostly random. I'd like to provide users with a similar
    trade-off with what the old versions of BEST_SPEED and BEST_COMPRESSION
    used to do. So since BEST_SPEED used to compress at most 128 docs at
    once, I think we should roughly make it 128*10 now since there are 10
    sub blocks. I made it 1024 to account for the fact that there is a preset
    dict as well that need decompressing. And similarly BEST_COMPRESSION used
    to allow 4x more docs than BEST_SPEED, so I made it 4096.
    
    With such larger numbers of docs per chunk, the decoding of metadata
    became a bottleneck for stored field access so I made it a bit faster by
    doing bulk decoding of the packed longs.
---
 .../compressing/CompressingStoredFieldsReader.java | 36 ++++++++++++----------
 .../lucene87/Lucene87StoredFieldsFormat.java       |  4 +--
 2 files changed, 22 insertions(+), 18 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
index 2ced3de..0903100 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
@@ -69,7 +69,7 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BitUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.LongsRef;
 import org.apache.lucene.util.packed.PackedInts;
 
 /**
@@ -402,8 +402,8 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
     // whether the block has been sliced, this happens for large documents
     private boolean sliced;
 
-    private int[] offsets = IntsRef.EMPTY_INTS;
-    private int[] numStoredFields = IntsRef.EMPTY_INTS;
+    private long[] offsets = LongsRef.EMPTY_LONGS;
+    private long[] numStoredFields = LongsRef.EMPTY_LONGS;
 
     // the start pointer at which you can read the compressed documents
     private long startPointer;
@@ -472,9 +472,11 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
         } else if (bitsPerStoredFields > 31) {
           throw new CorruptIndexException("bitsPerStoredFields=" + bitsPerStoredFields, fieldsStream);
         } else {
-          final PackedInts.ReaderIterator it = PackedInts.getReaderIteratorNoHeader(fieldsStream, PackedInts.Format.PACKED, packedIntsVersion, chunkDocs, bitsPerStoredFields, 1);
-          for (int i = 0; i < chunkDocs; ++i) {
-            numStoredFields[i] = (int) it.next();
+          final PackedInts.ReaderIterator it = PackedInts.getReaderIteratorNoHeader(fieldsStream, PackedInts.Format.PACKED, packedIntsVersion, chunkDocs, bitsPerStoredFields, 1024);
+          for (int i = 0; i < chunkDocs; ) {
+            final LongsRef next = it.next(Integer.MAX_VALUE);
+            System.arraycopy(next.longs, next.offset, numStoredFields, i, next.length);
+            i += next.length;
           }
         }
 
@@ -489,9 +491,11 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
         } else if (bitsPerStoredFields > 31) {
           throw new CorruptIndexException("bitsPerLength=" + bitsPerLength, fieldsStream);
         } else {
-          final PackedInts.ReaderIterator it = PackedInts.getReaderIteratorNoHeader(fieldsStream, PackedInts.Format.PACKED, packedIntsVersion, chunkDocs, bitsPerLength, 1);
-          for (int i = 0; i < chunkDocs; ++i) {
-            offsets[i + 1] = (int) it.next();
+          final PackedInts.ReaderIterator it = PackedInts.getReaderIteratorNoHeader(fieldsStream, PackedInts.Format.PACKED, packedIntsVersion, chunkDocs, bitsPerLength, 1024);
+          for (int i = 0; i < chunkDocs; ) {
+            final LongsRef next = it.next(Integer.MAX_VALUE);
+            System.arraycopy(next.longs, next.offset, offsets, i + 1, next.length);
+            i += next.length;
           }
           for (int i = 0; i < chunkDocs; ++i) {
             offsets[i + 1] += offsets[i];
@@ -500,8 +504,8 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
 
         // Additional validation: only the empty document has a serialized length of 0
         for (int i = 0; i < chunkDocs; ++i) {
-          final int len = offsets[i + 1] - offsets[i];
-          final int storedFields = numStoredFields[i];
+          final long len = offsets[i + 1] - offsets[i];
+          final long storedFields = numStoredFields[i];
           if ((len == 0) != (storedFields == 0)) {
             throw new CorruptIndexException("length=" + len + ", numStoredFields=" + storedFields, fieldsStream);
           }
@@ -512,7 +516,7 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
       startPointer = fieldsStream.getFilePointer();
 
       if (merging) {
-        final int totalLength = offsets[chunkDocs];
+        final int totalLength = Math.toIntExact(offsets[chunkDocs]);
         // decompress eagerly
         if (sliced) {
           bytes.offset = bytes.length = 0;
@@ -543,10 +547,10 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
       }
 
       final int index = docID - docBase;
-      final int offset = offsets[index];
-      final int length = offsets[index+1] - offset;
-      final int totalLength = offsets[chunkDocs];
-      final int numStoredFields = this.numStoredFields[index];
+      final int offset = Math.toIntExact(offsets[index]);
+      final int length = Math.toIntExact(offsets[index+1]) - offset;
+      final int totalLength = Math.toIntExact(offsets[chunkDocs]);
+      final int numStoredFields = Math.toIntExact(this.numStoredFields[index]);
 
       final BytesRef bytes;
       if (merging) {
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 0e9c9f5..1432df8 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
@@ -144,9 +144,9 @@ public class Lucene87StoredFieldsFormat extends StoredFieldsFormat {
   StoredFieldsFormat impl(Mode mode) {
     switch (mode) {
       case BEST_SPEED:
-        return new CompressingStoredFieldsFormat("Lucene87StoredFieldsFastData", BEST_SPEED_MODE, BEST_SPEED_BLOCK_LENGTH, 512, 10);
+        return new CompressingStoredFieldsFormat("Lucene87StoredFieldsFastData", BEST_SPEED_MODE, BEST_SPEED_BLOCK_LENGTH, 1024, 10);
       case BEST_COMPRESSION:
-        return new CompressingStoredFieldsFormat("Lucene87StoredFieldsHighData", BEST_COMPRESSION_MODE, BEST_COMPRESSION_BLOCK_LENGTH, 512, 10);
+        return new CompressingStoredFieldsFormat("Lucene87StoredFieldsHighData", BEST_COMPRESSION_MODE, BEST_COMPRESSION_BLOCK_LENGTH, 4096, 10);
       default: throw new AssertionError();
     }
   }