Skip to content

Commit e5193bc

Browse files
committed
cache preset dict in LZ4WithPresetDictDecompressor
1 parent 9b9fbd4 commit e5193bc

File tree

5 files changed

+174
-6
lines changed

5 files changed

+174
-6
lines changed

lucene/CHANGES.txt

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -200,6 +200,8 @@ Optimizations
200200

201201
* GITHUB#14373: Optimized `ParallelLeafReader` to improve term vector fetching efficiency. (Divyansh Agrawal)
202202

203+
* GITHUB#14397: Cache preset dict for LZ4WithPresetDictDecompressor. (kkewwei)
204+
203205
Bug Fixes
204206
---------------------
205207

lucene/core/src/java/org/apache/lucene/codecs/compressing/Decompressor.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818

1919
import java.io.IOException;
2020
import org.apache.lucene.store.DataInput;
21+
import org.apache.lucene.store.IndexInput;
2122
import org.apache.lucene.util.BytesRef;
2223

2324
/** A decompressor. */
@@ -42,6 +43,13 @@ protected Decompressor() {}
4243
public abstract void decompress(
4344
DataInput in, int originalLength, int offset, int length, BytesRef bytes) throws IOException;
4445

46+
public void decompress(IndexInput in, int originalLength, int offset, int length, BytesRef bytes)
47+
throws IOException {
48+
decompress(((DataInput) in), originalLength, offset, length, bytes);
49+
}
50+
4551
@Override
4652
public abstract Decompressor clone();
53+
54+
public void reset() {}
4755
}

lucene/core/src/java/org/apache/lucene/codecs/lucene90/LZ4WithPresetDictCompressionMode.java

Lines changed: 79 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import org.apache.lucene.store.ByteBuffersDataOutput;
2626
import org.apache.lucene.store.DataInput;
2727
import org.apache.lucene.store.DataOutput;
28+
import org.apache.lucene.store.IndexInput;
2829
import org.apache.lucene.util.ArrayUtil;
2930
import org.apache.lucene.util.BytesRef;
3031
import org.apache.lucene.util.compress.LZ4;
@@ -64,6 +65,9 @@ private static final class LZ4WithPresetDictDecompressor extends Decompressor {
6465

6566
private int[] compressedLengths;
6667
private byte[] buffer;
68+
private long cachedDictFilPointer = -1;
69+
private int cachedDictLength = -1;
70+
private long dictEndFilePointer = -1;
6771

6872
LZ4WithPresetDictDecompressor() {
6973
compressedLengths = new int[0];
@@ -144,10 +148,85 @@ public void decompress(DataInput in, int originalLength, int offset, int length,
144148
assert bytes.isValid();
145149
}
146150

151+
@Override
152+
public void decompress(
153+
IndexInput in, int originalLength, int offset, int length, BytesRef bytes)
154+
throws IOException {
155+
assert offset + length <= originalLength;
156+
157+
if (length == 0) {
158+
bytes.length = 0;
159+
return;
160+
}
161+
162+
final int dictLength = in.readVInt();
163+
final int blockLength = in.readVInt();
164+
165+
final int numBlocks = readCompressedLengths(in, originalLength, dictLength, blockLength);
166+
167+
buffer = ArrayUtil.grow(buffer, dictLength + blockLength);
168+
long startPointer = in.getFilePointer();
169+
bytes.length = 0;
170+
if (cachedDictFilPointer == startPointer) {
171+
assert cachedDictLength == dictLength && dictEndFilePointer > 0;
172+
in.seek(dictEndFilePointer);
173+
} else {
174+
// Read the dictionary
175+
if (LZ4.decompress(in, dictLength, buffer, 0) != dictLength) {
176+
throw new CorruptIndexException("Illegal dict length", in);
177+
}
178+
cachedDictLength = dictLength;
179+
dictEndFilePointer = in.getFilePointer();
180+
cachedDictFilPointer = startPointer;
181+
}
182+
183+
int offsetInBlock = dictLength;
184+
int offsetInBytesRef = offset;
185+
if (offset >= dictLength) {
186+
offsetInBytesRef -= dictLength;
187+
188+
// Skip unneeded blocks
189+
int numBytesToSkip = 0;
190+
for (int i = 0; i < numBlocks && offsetInBlock + blockLength < offset; ++i) {
191+
int compressedBlockLength = compressedLengths[i];
192+
numBytesToSkip += compressedBlockLength;
193+
offsetInBlock += blockLength;
194+
offsetInBytesRef -= blockLength;
195+
}
196+
in.skipBytes(numBytesToSkip);
197+
} else {
198+
// The dictionary contains some bytes we need, copy its content to the BytesRef
199+
bytes.bytes = ArrayUtil.grow(bytes.bytes, dictLength);
200+
System.arraycopy(buffer, 0, bytes.bytes, 0, dictLength);
201+
bytes.length = dictLength;
202+
}
203+
204+
// Read blocks that intersect with the interval we need
205+
while (offsetInBlock < offset + length) {
206+
final int bytesToDecompress = Math.min(blockLength, offset + length - offsetInBlock);
207+
LZ4.decompress(in, bytesToDecompress, buffer, dictLength);
208+
bytes.bytes = ArrayUtil.grow(bytes.bytes, bytes.length + bytesToDecompress);
209+
System.arraycopy(buffer, dictLength, bytes.bytes, bytes.length, bytesToDecompress);
210+
bytes.length += bytesToDecompress;
211+
offsetInBlock += blockLength;
212+
}
213+
214+
bytes.offset = offsetInBytesRef;
215+
bytes.length = length;
216+
assert bytes.isValid();
217+
}
218+
147219
@Override
148220
public Decompressor clone() {
149221
return new LZ4WithPresetDictDecompressor();
150222
}
223+
224+
@Override
225+
public void reset() {
226+
cachedDictFilPointer = -1;
227+
cachedDictLength = -1;
228+
dictEndFilePointer = -1;
229+
}
151230
}
152231

153232
private static class LZ4WithPresetDictCompressor extends Compressor {

lucene/core/src/java/org/apache/lucene/codecs/lucene90/compressing/Lucene90CompressingStoredFieldsReader.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -512,6 +512,7 @@ private void doReset(int docID) throws IOException {
512512
bytes.offset = bytes.length = 0;
513513
for (int decompressed = 0; decompressed < totalLength; ) {
514514
final int toDecompress = Math.min(totalLength - decompressed, chunkSize);
515+
decompressor.reset();
515516
decompressor.decompress(fieldsStream, toDecompress, 0, toDecompress, spare);
516517
bytes.bytes = ArrayUtil.grow(bytes.bytes, bytes.length + spare.length);
517518
System.arraycopy(spare.bytes, spare.offset, bytes.bytes, bytes.length, spare.length);
@@ -573,6 +574,7 @@ void fillBuffer() throws IOException {
573574
throw new EOFException();
574575
}
575576
final int toDecompress = Math.min(length - decompressed, chunkSize);
577+
decompressor.reset();
576578
decompressor.decompress(fieldsStream, toDecompress, 0, toDecompress, bytes);
577579
decompressed += toDecompress;
578580
}
@@ -644,6 +646,7 @@ SerializedDocument serializedDocument(int docID) throws IOException {
644646
if (state.contains(docID) == false) {
645647
fieldsStream.seek(indexReader.getStartPointer(docID));
646648
state.reset(docID);
649+
decompressor.reset();
647650
}
648651
assert state.contains(docID);
649652
return state.document(docID);

lucene/core/src/test/org/apache/lucene/codecs/compressing/AbstractTestCompressionMode.java

Lines changed: 82 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -17,13 +17,16 @@
1717
package org.apache.lucene.codecs.compressing;
1818

1919
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
20+
import java.io.EOFException;
2021
import java.io.IOException;
2122
import java.nio.ByteBuffer;
2223
import java.util.Arrays;
2324
import java.util.Random;
25+
import org.apache.lucene.codecs.lucene90.LZ4WithPresetDictCompressionMode;
2426
import org.apache.lucene.store.ByteArrayDataInput;
2527
import org.apache.lucene.store.ByteArrayDataOutput;
2628
import org.apache.lucene.store.ByteBuffersDataInput;
29+
import org.apache.lucene.store.IndexInput;
2730
import org.apache.lucene.tests.util.LuceneTestCase;
2831
import org.apache.lucene.tests.util.TestUtil;
2932
import org.apache.lucene.util.ArrayUtil;
@@ -68,23 +71,34 @@ static byte[] compress(Compressor compressor, byte[] decompressed, int off, int
6871

6972
byte[] decompress(byte[] compressed, int originalLength) throws IOException {
7073
Decompressor decompressor = mode.newDecompressor();
71-
return decompress(decompressor, compressed, originalLength);
74+
return decompress(mode, decompressor, compressed, originalLength);
7275
}
7376

74-
static byte[] decompress(Decompressor decompressor, byte[] compressed, int originalLength)
77+
static byte[] decompress(
78+
CompressionMode mode, Decompressor decompressor, byte[] compressed, int originalLength)
7579
throws IOException {
7680
final BytesRef bytes = new BytesRef();
77-
decompressor.decompress(
78-
new ByteArrayDataInput(compressed), originalLength, 0, originalLength, bytes);
81+
if ((mode instanceof LZ4WithPresetDictCompressionMode == false) || random().nextBoolean()) {
82+
decompressor.decompress(
83+
new ByteArrayDataInput(compressed), originalLength, 0, originalLength, bytes);
84+
} else {
85+
decompressor.decompress(
86+
new MockIndexInput(compressed), originalLength, 0, originalLength, bytes);
87+
}
7988
return BytesRef.deepCopyOf(bytes).bytes;
8089
}
8190

8291
byte[] decompress(byte[] compressed, int originalLength, int offset, int length)
8392
throws IOException {
8493
Decompressor decompressor = mode.newDecompressor();
8594
final BytesRef bytes = new BytesRef();
86-
decompressor.decompress(
87-
new ByteArrayDataInput(compressed), originalLength, offset, length, bytes);
95+
if ((mode instanceof LZ4WithPresetDictCompressionMode == false) || random().nextBoolean()) {
96+
decompressor.decompress(
97+
new ByteArrayDataInput(compressed), originalLength, offset, length, bytes);
98+
} else {
99+
decompressor.decompress(
100+
new MockIndexInput(compressed), originalLength, offset, length, bytes);
101+
}
88102
return BytesRef.deepCopyOf(bytes).bytes;
89103
}
90104

@@ -162,4 +176,66 @@ public void testExtremelyLargeInput() throws IOException {
162176
}
163177
test(decompressed);
164178
}
179+
180+
private static class MockIndexInput extends IndexInput {
181+
private byte[] bytes;
182+
private final long length;
183+
private final int startOffset;
184+
private int pos;
185+
186+
MockIndexInput(byte[] bytes) throws EOFException {
187+
this(bytes, 0, bytes.length);
188+
}
189+
190+
MockIndexInput(byte[] bytes, int startOffset, long length) throws EOFException {
191+
super("MockIndexInput");
192+
if ((length + startOffset) > bytes.length) {
193+
throw new EOFException();
194+
}
195+
this.bytes = bytes;
196+
this.startOffset = startOffset;
197+
this.length = length;
198+
this.pos = 0;
199+
}
200+
201+
@Override
202+
public byte readByte() throws IOException {
203+
if (this.startOffset + this.pos + 1 > length) {
204+
throw new EOFException();
205+
}
206+
return bytes[startOffset + pos++];
207+
}
208+
209+
@Override
210+
public void readBytes(byte[] b, int offset, int len) throws IOException {
211+
if (this.startOffset + this.pos + len > length) {
212+
throw new EOFException();
213+
}
214+
System.arraycopy(bytes, startOffset + pos, b, offset, len);
215+
pos += len;
216+
}
217+
218+
@Override
219+
public void close() throws IOException {}
220+
221+
@Override
222+
public long getFilePointer() {
223+
return pos;
224+
}
225+
226+
@Override
227+
public void seek(long pos) throws IOException {
228+
this.pos = startOffset + (int) pos;
229+
}
230+
231+
@Override
232+
public long length() {
233+
return length;
234+
}
235+
236+
@Override
237+
public IndexInput slice(String sliceDescription, long offset, long length) throws IOException {
238+
return new MockIndexInput(bytes, startOffset + (int) offset, length);
239+
}
240+
}
165241
}

0 commit comments

Comments
 (0)