From 05e06e51ec3429d45d75c4c7d61e55c6c6dee2f2 Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Mon, 21 Oct 2024 07:36:46 +0200 Subject: [PATCH 01/37] Use growNoCopy when copying bytes in BytesRefBuilder (#13930) --- lucene/CHANGES.txt | 2 ++ .../org/apache/lucene/util/BytesRefBuilder.java | 16 ++++++++-------- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 5d9343bca2a..15a72c66a63 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -53,6 +53,8 @@ Optimizations * GITHUB#13800: MaxScoreBulkScorer now recomputes scorer partitions when the minimum competitive allows for a more favorable partitioning. (Adrien Grand) +* GITHUB#13930: Use growNoCopy when copying bytes in BytesRefBuilder. (Ignacio Vera) + Bug Fixes --------------------- * GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended diff --git a/lucene/core/src/java/org/apache/lucene/util/BytesRefBuilder.java b/lucene/core/src/java/org/apache/lucene/util/BytesRefBuilder.java index 21adf0c60da..27fc4337d25 100644 --- a/lucene/core/src/java/org/apache/lucene/util/BytesRefBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/BytesRefBuilder.java @@ -100,8 +100,10 @@ public class BytesRefBuilder { * #clear()} and then {@link #append(byte[], int, int)}. */ public void copyBytes(byte[] b, int off, int len) { - clear(); - append(b, off, len); + assert ref.offset == 0; + ref.length = len; + growNoCopy(len); + System.arraycopy(b, off, ref.bytes, 0, len); } /** @@ -109,8 +111,7 @@ public class BytesRefBuilder { * #clear()} and then {@link #append(BytesRef)}. */ public void copyBytes(BytesRef ref) { - clear(); - append(ref); + copyBytes(ref.bytes, ref.offset, ref.length); } /** @@ -118,8 +119,7 @@ public class BytesRefBuilder { * #clear()} and then {@link #append(BytesRefBuilder)}. */ public void copyBytes(BytesRefBuilder builder) { - clear(); - append(builder); + copyBytes(builder.get()); } /** @@ -135,7 +135,7 @@ public class BytesRefBuilder { * text. */ public void copyChars(CharSequence text, int off, int len) { - grow(UnicodeUtil.maxUTF8Length(len)); + growNoCopy(UnicodeUtil.maxUTF8Length(len)); ref.length = UnicodeUtil.UTF16toUTF8(text, off, len, ref.bytes); } @@ -144,7 +144,7 @@ public class BytesRefBuilder { * text. */ public void copyChars(char[] text, int off, int len) { - grow(UnicodeUtil.maxUTF8Length(len)); + growNoCopy(UnicodeUtil.maxUTF8Length(len)); ref.length = UnicodeUtil.UTF16toUTF8(text, off, len, ref.bytes); } From 66f22fa0fd01518adc05045a6d95d53e82e77d1b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 21 Oct 2024 15:10:08 +0200 Subject: [PATCH 02/37] Simplify PForUtil construction and cleanup its code gen a little (#13932) Generate cleaner code for PForUtil that has no dead parameters. Also: PForUtil instances always create their own `ForUtil`, so we can inline that into the field declaration. Also, we can save cycles for accessing the input on PostingsDecodingUtil. Surprisingly, the combination of these cleanups yields a small but statistically fully visible speedup that the compiler isn't able to get to on its own it seems. --- .../checksums/generateForDeltaUtil.json | 4 ++-- .../generated/checksums/generateForUtil.json | 4 ++-- .../lucene/codecs/lucene912/ForDeltaUtil.java | 16 +++++++-------- .../lucene/codecs/lucene912/ForUtil.java | 20 +++++++++---------- .../lucene912/Lucene912PostingsReader.java | 6 +++--- .../lucene912/Lucene912PostingsWriter.java | 3 +-- .../lucene/codecs/lucene912/PForUtil.java | 16 +++++++-------- .../codecs/lucene912/gen_ForDeltaUtil.py | 15 +++++++++++--- .../lucene/codecs/lucene912/gen_ForUtil.py | 9 +++++++-- .../lucene/codecs/lucene912/TestPForUtil.java | 5 ++--- 10 files changed, 54 insertions(+), 44 deletions(-) diff --git a/lucene/core/src/generated/checksums/generateForDeltaUtil.json b/lucene/core/src/generated/checksums/generateForDeltaUtil.json index 26ebc1198d9..bb88ff10324 100644 --- a/lucene/core/src/generated/checksums/generateForDeltaUtil.json +++ b/lucene/core/src/generated/checksums/generateForDeltaUtil.json @@ -1,4 +1,4 @@ { - "lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java": "f561578ccb6a95364bb62c5ed86b38ff0b4a009d", - "lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForDeltaUtil.py": "eea1a71be9da8a13fdd979354dc4a8c6edf21be1" + "lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java": "b662da5848b0decc8bceb4225f433875ae9e3c11", + "lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForDeltaUtil.py": "01787b97bbe79edb7703498cef8ddb85901a6b1e" } \ No newline at end of file diff --git a/lucene/core/src/generated/checksums/generateForUtil.json b/lucene/core/src/generated/checksums/generateForUtil.json index 3a1c02b35bb..e862c737e5d 100644 --- a/lucene/core/src/generated/checksums/generateForUtil.json +++ b/lucene/core/src/generated/checksums/generateForUtil.json @@ -1,4 +1,4 @@ { - "lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java": "159e82388346fde147924d5e15ca65df4dd63b9a", - "lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py": "66dc8813160feae2a37d8b50474f5f9830b6cb22" + "lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java": "02e0c8c290e65d0314664fde24c9331bdec44925", + "lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py": "d7850f37e52a16c6592322950d0f6219cad23a33" } \ No newline at end of file diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java index f13b3cde69c..fe0378b388b 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java @@ -286,11 +286,11 @@ public final class ForDeltaUtil { throws IOException { switch (bitsPerValue) { case 1: - decode1(pdu, tmp, longs); + decode1(pdu, longs); prefixSum8(longs, base); break; case 2: - decode2(pdu, tmp, longs); + decode2(pdu, longs); prefixSum8(longs, base); break; case 3: @@ -298,7 +298,7 @@ public final class ForDeltaUtil { prefixSum8(longs, base); break; case 4: - decode4(pdu, tmp, longs); + decode4(pdu, longs); prefixSum8(longs, base); break; case 5: @@ -314,7 +314,7 @@ public final class ForDeltaUtil { prefixSum16(longs, base); break; case 8: - decode8To16(pdu, tmp, longs); + decode8To16(pdu, longs); prefixSum16(longs, base); break; case 9: @@ -346,7 +346,7 @@ public final class ForDeltaUtil { prefixSum32(longs, base); break; case 16: - decode16To32(pdu, tmp, longs); + decode16To32(pdu, longs); prefixSum32(longs, base); break; case 17: @@ -431,8 +431,7 @@ public final class ForDeltaUtil { } } - private static void decode8To16(PostingDecodingUtil pdu, long[] tmp, long[] longs) - throws IOException { + private static void decode8To16(PostingDecodingUtil pdu, long[] longs) throws IOException { pdu.splitLongs(16, longs, 8, 8, MASK16_8, longs, 16, MASK16_8); } @@ -522,8 +521,7 @@ public final class ForDeltaUtil { } } - private static void decode16To32(PostingDecodingUtil pdu, long[] tmp, long[] longs) - throws IOException { + private static void decode16To32(PostingDecodingUtil pdu, long[] longs) throws IOException { pdu.splitLongs(32, longs, 16, 16, MASK32_16, longs, 32, MASK32_16); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java index ca7747b58bd..fd5afa235f7 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java @@ -291,11 +291,11 @@ public final class ForUtil { void decode(int bitsPerValue, PostingDecodingUtil pdu, long[] longs) throws IOException { switch (bitsPerValue) { case 1: - decode1(pdu, tmp, longs); + decode1(pdu, longs); expand8(longs); break; case 2: - decode2(pdu, tmp, longs); + decode2(pdu, longs); expand8(longs); break; case 3: @@ -303,7 +303,7 @@ public final class ForUtil { expand8(longs); break; case 4: - decode4(pdu, tmp, longs); + decode4(pdu, longs); expand8(longs); break; case 5: @@ -319,7 +319,7 @@ public final class ForUtil { expand8(longs); break; case 8: - decode8(pdu, tmp, longs); + decode8(pdu, longs); expand8(longs); break; case 9: @@ -351,7 +351,7 @@ public final class ForUtil { expand16(longs); break; case 16: - decode16(pdu, tmp, longs); + decode16(pdu, longs); expand16(longs); break; case 17: @@ -393,11 +393,11 @@ public final class ForUtil { } } - static void decode1(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { + static void decode1(PostingDecodingUtil pdu, long[] longs) throws IOException { pdu.splitLongs(2, longs, 7, 1, MASK8_1, longs, 14, MASK8_1); } - static void decode2(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { + static void decode2(PostingDecodingUtil pdu, long[] longs) throws IOException { pdu.splitLongs(4, longs, 6, 2, MASK8_2, longs, 12, MASK8_2); } @@ -413,7 +413,7 @@ public final class ForUtil { } } - static void decode4(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { + static void decode4(PostingDecodingUtil pdu, long[] longs) throws IOException { pdu.splitLongs(8, longs, 4, 4, MASK8_4, longs, 8, MASK8_4); } @@ -457,7 +457,7 @@ public final class ForUtil { } } - static void decode8(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { + static void decode8(PostingDecodingUtil pdu, long[] longs) throws IOException { pdu.in.readLongs(longs, 0, 16); } @@ -601,7 +601,7 @@ public final class ForUtil { } } - static void decode16(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { + static void decode16(PostingDecodingUtil pdu, long[] longs) throws IOException { pdu.in.readLongs(longs, 0, 32); } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java index bdb4dc4db08..cea329c93ca 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java @@ -427,7 +427,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { public PostingsEnum reset(IntBlockTermState termState, int flags) throws IOException { resetIndexInput(termState); if (pforUtil == null && docFreq >= BLOCK_SIZE) { - pforUtil = new PForUtil(new ForUtil()); + pforUtil = new PForUtil(); forDeltaUtil = new ForDeltaUtil(); } totalTermFreq = indexHasFreq ? termState.totalTermFreq : docFreq; @@ -727,7 +727,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } totalTermFreq = termState.totalTermFreq; if (pforUtil == null && totalTermFreq >= BLOCK_SIZE) { - pforUtil = new PForUtil(new ForUtil()); + pforUtil = new PForUtil(); } // Where this term's postings start in the .pos file: final long posTermStartFP = termState.posStartFP; @@ -1142,7 +1142,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { private abstract class BlockImpactsEnum extends ImpactsEnum { protected final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(); - protected final PForUtil pforUtil = new PForUtil(new ForUtil()); + protected final PForUtil pforUtil = new PForUtil(); protected final long[] docBuffer = new long[BLOCK_SIZE + 1]; protected final long[] freqBuffer = new long[BLOCK_SIZE]; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java index df34510de07..b0da9347700 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java @@ -142,9 +142,8 @@ public class Lucene912PostingsWriter extends PushPostingsWriterBase { metaOut, META_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); CodecUtil.writeIndexHeader( docOut, DOC_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); - final ForUtil forUtil = new ForUtil(); forDeltaUtil = new ForDeltaUtil(); - pforUtil = new PForUtil(forUtil); + pforUtil = new PForUtil(); if (state.fieldInfos.hasProx()) { posDeltaBuffer = new long[BLOCK_SIZE]; String posFileName = diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java index 3857eabbe44..063f385ba42 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java @@ -38,11 +38,10 @@ final class PForUtil { return true; } - private final ForUtil forUtil; + private final ForUtil forUtil = new ForUtil(); - PForUtil(ForUtil forUtil) { + static { assert ForUtil.BLOCK_SIZE <= 256 : "blocksize must fit in one byte. got " + ForUtil.BLOCK_SIZE; - this.forUtil = forUtil; } /** Encode 128 integers from {@code longs} into {@code out}. */ @@ -106,17 +105,18 @@ final class PForUtil { /** Decode 128 integers into {@code ints}. */ void decode(PostingDecodingUtil pdu, long[] longs) throws IOException { - final int token = Byte.toUnsignedInt(pdu.in.readByte()); + var in = pdu.in; + final int token = Byte.toUnsignedInt(in.readByte()); final int bitsPerValue = token & 0x1f; - final int numExceptions = token >>> 5; if (bitsPerValue == 0) { - Arrays.fill(longs, 0, ForUtil.BLOCK_SIZE, pdu.in.readVLong()); + Arrays.fill(longs, 0, ForUtil.BLOCK_SIZE, in.readVLong()); } else { forUtil.decode(bitsPerValue, pdu, longs); } + final int numExceptions = token >>> 5; for (int i = 0; i < numExceptions; ++i) { - longs[Byte.toUnsignedInt(pdu.in.readByte())] |= - Byte.toUnsignedLong(pdu.in.readByte()) << bitsPerValue; + longs[Byte.toUnsignedInt(in.readByte())] |= + Byte.toUnsignedLong(in.readByte()) << bitsPerValue; } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForDeltaUtil.py b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForDeltaUtil.py index 56c402372a6..134c17c0b8f 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForDeltaUtil.py +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForDeltaUtil.py @@ -361,7 +361,10 @@ def writeRemainder(bpv, next_primitive, remaining_bits_per_long, o, num_values, def writeDecode(bpv, f): next_primitive = primitive_size_for_bpv(bpv) - f.write(' private static void decode%dTo%d(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException {\n' %(bpv, next_primitive)) + if next_primitive % bpv == 0: + f.write(' private static void decode%dTo%d(PostingDecodingUtil pdu, long[] longs) throws IOException {\n' %(bpv, next_primitive)) + else: + f.write(' private static void decode%dTo%d(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException {\n' %(bpv, next_primitive)) if bpv == next_primitive: f.write(' pdu.in.readLongs(longs, 0, %d);\n' %(bpv*2)) else: @@ -390,9 +393,15 @@ if __name__ == '__main__': primitive_size = primitive_size_for_bpv(bpv) f.write(' case %d:\n' %bpv) if next_primitive(bpv) == primitive_size: - f.write(' decode%d(pdu, tmp, longs);\n' %bpv) + if primitive_size % bpv == 0: + f.write(' decode%d(pdu, longs);\n' %bpv) + else: + f.write(' decode%d(pdu, tmp, longs);\n' %bpv) else: - f.write(' decode%dTo%d(pdu, tmp, longs);\n' %(bpv, primitive_size)) + if primitive_size % bpv == 0: + f.write(' decode%dTo%d(pdu, longs);\n' %(bpv, primitive_size)) + else: + f.write(' decode%dTo%d(pdu, tmp, longs);\n' %(bpv, primitive_size)) f.write(' prefixSum%d(longs, base);\n' %primitive_size) f.write(' break;\n') f.write(' default:\n') diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py index 1049608ca19..9851ec99a61 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py @@ -287,8 +287,8 @@ def writeDecode(bpv, f): next_primitive = 8 elif bpv <= 16: next_primitive = 16 - f.write(' static void decode%d(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException {\n' %bpv) if bpv == next_primitive: + f.write(' static void decode%d(PostingDecodingUtil pdu, long[] longs) throws IOException {\n' %bpv) f.write(' pdu.in.readLongs(longs, 0, %d);\n' %(bpv*2)) else: num_values_per_long = 64 / next_primitive @@ -296,8 +296,10 @@ def writeDecode(bpv, f): num_iters = (next_primitive - 1) // bpv o = 2 * bpv * num_iters if remaining_bits == 0: + f.write(' static void decode%d(PostingDecodingUtil pdu, long[] longs) throws IOException {\n' %bpv) f.write(' pdu.splitLongs(%d, longs, %d, %d, MASK%d_%d, longs, %d, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, bpv, next_primitive, bpv, o, next_primitive, next_primitive - num_iters * bpv)) else: + f.write(' static void decode%d(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException {\n' %bpv) f.write(' pdu.splitLongs(%d, longs, %d, %d, MASK%d_%d, tmp, 0, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, bpv, next_primitive, bpv, next_primitive, next_primitive - num_iters * bpv)) writeRemainder(bpv, next_primitive, remaining_bits, o, 128/num_values_per_long - o, f) f.write(' }\n') @@ -334,7 +336,10 @@ if __name__ == '__main__': elif bpv <= 16: next_primitive = 16 f.write(' case %d:\n' %bpv) - f.write(' decode%d(pdu, tmp, longs);\n' %bpv) + if next_primitive % bpv == 0: + f.write(' decode%d(pdu, longs);\n' %bpv) + else: + f.write(' decode%d(pdu, tmp, longs);\n' %bpv) f.write(' expand%d(longs);\n' %next_primitive) f.write(' break;\n') f.write(' default:\n') diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java index ccc786293a1..2dd3f5ece75 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java @@ -39,11 +39,10 @@ public class TestPForUtil extends LuceneTestCase { final Directory d = new ByteBuffersDirectory(); final long endPointer = encodeTestData(iterations, values, d); - ForUtil forUtil = new ForUtil(); IndexInput in = d.openInput("test.bin", IOContext.READONCE); PostingDecodingUtil pdu = Lucene912PostingsReader.VECTORIZATION_PROVIDER.newPostingDecodingUtil(in); - final PForUtil pforUtil = new PForUtil(forUtil); + final PForUtil pforUtil = new PForUtil(); for (int i = 0; i < iterations; ++i) { if (random().nextInt(5) == 0) { PForUtil.skip(in); @@ -91,7 +90,7 @@ public class TestPForUtil extends LuceneTestCase { private long encodeTestData(int iterations, int[] values, Directory d) throws IOException { IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT); - final PForUtil pforUtil = new PForUtil(new ForUtil()); + final PForUtil pforUtil = new PForUtil(); for (int i = 0; i < iterations; ++i) { long[] source = new long[ForUtil.BLOCK_SIZE]; From d1e017f1927e2539ead23e36e7136bf6bdf3993a Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 21 Oct 2024 16:38:12 +0200 Subject: [PATCH 03/37] Speedup OrderIntervalsSource some more (#13937) Follow-up to #13871, getting another speedup from relatively trivial changes: * avoid redundant `end()` call by directly storing the end value for sub-iterator that we don't use for anything else * also save most `get(...)` calls for this sub-iterator * avoid redundant `start()` call by grabbing `start()` directly from `nextInterval` * replace `getFirst()` with `get(0)`, it looks nice but has needless overhead in my testing (not sure why, but profiling clearly shows it to be slower, maybe just a result of having `get()`'s code hot in the cache with a higher likelihood or something esoteric like that) * avoid using an iterator for loop for a random access list, this is probably the biggest win in this PR --- .../intervals/OrderedIntervalsSource.java | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/OrderedIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/OrderedIntervalsSource.java index c855d61d051..fc1588c8bdf 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/OrderedIntervalsSource.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/OrderedIntervalsSource.java @@ -127,9 +127,9 @@ class OrderedIntervalsSource extends MinimizingConjunctionIntervalsSource { final var subIterators = this.subIterators; int currentIndex = i; while (true) { + int prevEnd = subIterators.get(currentIndex - 1).end(); while (true) { - var prev = subIterators.get(currentIndex - 1); - if (prev.end() >= lastStart) { + if (prevEnd >= lastStart) { i = currentIndex; return start; } @@ -137,19 +137,21 @@ class OrderedIntervalsSource extends MinimizingConjunctionIntervalsSource { break; } final IntervalIterator current = subIterators.get(currentIndex); - if (minimizing && (current.start() > prev.end())) { + if (minimizing && (current.start() > prevEnd)) { break; } + int currentStart; do { if (current.end() >= lastStart - || current.nextInterval() == IntervalIterator.NO_MORE_INTERVALS) { + || (currentStart = current.nextInterval()) == IntervalIterator.NO_MORE_INTERVALS) { i = currentIndex; return start; } - } while (current.start() <= prev.end()); + } while (currentStart <= prevEnd); currentIndex++; + prevEnd = current.end(); } - var first = subIterators.getFirst(); + var first = subIterators.get(0); final int start = first.start(); this.start = start; if (start == NO_MORE_INTERVALS) { @@ -161,8 +163,10 @@ class OrderedIntervalsSource extends MinimizingConjunctionIntervalsSource { final int end = last.end(); this.end = end; int slop = end - start + 1; - for (IntervalIterator subIterator : subIterators) { - slop -= subIterator.width(); + // use indexed loop since this is always a random access capable list to avoid allocations + // in a hot nested loop + for (int j = 0, n = subIterators.size(); j < n; j++) { + slop -= subIterators.get(j).width(); } this.slop = slop; onMatch.onMatch(); From 86457a5f330ceaba3505c1acbf3cbb0541cc9e51 Mon Sep 17 00:00:00 2001 From: linfn Date: Mon, 21 Oct 2024 22:54:13 +0800 Subject: [PATCH 04/37] Fix StoredFieldsConsumer finish (#13927) --- lucene/CHANGES.txt | 1 + .../lucene/index/StoredFieldsConsumer.java | 3 +- .../index/TestStoredFieldsConsumer.java | 84 +++++++++++++++++++ 3 files changed, 86 insertions(+), 2 deletions(-) create mode 100644 lucene/core/src/test/org/apache/lucene/index/TestStoredFieldsConsumer.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 15a72c66a63..5376ffeccc9 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -61,6 +61,7 @@ Bug Fixes when they were not sorted by startOffset. (Seunghan Jung) * GITHUB#13884: Remove broken .toArray from Long/CharObjectHashMap entirely. (Pan Guixin) * GITHUB#12686: Added support for highlighting IndexOrDocValuesQuery. (Prudhvi Godithi) +* GITHUB#13927: Fix StoredFieldsConsumer finish. (linfn) Build --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java b/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java index 1bde9d4d597..78f2f726a2f 100644 --- a/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java @@ -93,9 +93,8 @@ class StoredFieldsConsumer { void finish(int maxDoc) throws IOException { while (lastDoc < maxDoc - 1) { - startDocument(lastDoc); + startDocument(lastDoc + 1); finishDocument(); - ++lastDoc; } } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestStoredFieldsConsumer.java b/lucene/core/src/test/org/apache/lucene/index/TestStoredFieldsConsumer.java new file mode 100644 index 00000000000..e0c46bc58a7 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/index/TestStoredFieldsConsumer.java @@ -0,0 +1,84 @@ +/* + * 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.index; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FlushInfo; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.util.StringHelper; +import org.apache.lucene.util.Version; + +public class TestStoredFieldsConsumer extends LuceneTestCase { + + public void testFinish() throws IOException { + Directory dir = newDirectory(); + IndexWriterConfig iwc = new IndexWriterConfig(); + SegmentInfo si = + new SegmentInfo( + dir, + Version.LATEST, + null, + "_0", + -1, + false, + false, + iwc.getCodec(), + Collections.emptyMap(), + StringHelper.randomId(), + new HashMap<>(), + null); + + AtomicInteger startDocCounter = new AtomicInteger(), finishDocCounter = new AtomicInteger(); + StoredFieldsConsumer consumer = + new StoredFieldsConsumer(iwc.getCodec(), dir, si) { + @Override + void startDocument(int docID) throws IOException { + super.startDocument(docID); + startDocCounter.incrementAndGet(); + } + + @Override + void finishDocument() throws IOException { + super.finishDocument(); + finishDocCounter.incrementAndGet(); + } + }; + + int numDocs = 3; + consumer.finish(numDocs); + + si.setMaxDoc(numDocs); + SegmentWriteState state = + new SegmentWriteState( + null, + dir, + si, + new FieldInfos(new FieldInfo[0]), + null, + new IOContext(new FlushInfo(numDocs, 10))); + consumer.flush(state, null); + dir.close(); + + assertEquals(numDocs, startDocCounter.get()); + assertEquals(numDocs, finishDocCounter.get()); + } +} From a779a64d7b1c6f63f489d5c6a11693169b9e4362 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 21 Oct 2024 16:55:04 +0200 Subject: [PATCH 05/37] Move BooleanScorer to work on top of Scorers rather than BulkScorers. (#13931) I was looking at some queries where Lucene performs significantly worse than Tantivy at https://tantivy-search.github.io/bench/, and found out that we get quite some overhead from implementing `BooleanScorer` on top of `BulkScorer` (effectively implemented by `DefaultBulkScorer` since it only runs term queries as boolean clauses) rather than `Scorer` directly. The `CountOrHighHigh` and `CountOrHighMed` tasks are a bit noisy on my machine, so I did 3 runs on wikibigall, and all of them had speedups for these two tasks, often with a very low p-value. In theory, this change could make things slower when the inner query has a specialized bulk scorer, such as `MatchAllDocsQuery` or a conjunction. It does feel right to optimize for term queries though. --- lucene/CHANGES.txt | 5 + .../apache/lucene/search/BooleanScorer.java | 191 ++++++++---------- .../lucene/search/BooleanScorerSupplier.java | 4 +- .../apache/lucene/search/BooleanWeight.java | 64 ------ .../apache/lucene/search/TestBooleanOr.java | 48 ++--- .../lucene/search/TestMinShouldMatch2.java | 3 +- 6 files changed, 118 insertions(+), 197 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 5376ffeccc9..04563553106 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -55,6 +55,11 @@ Optimizations * GITHUB#13930: Use growNoCopy when copying bytes in BytesRefBuilder. (Ignacio Vera) +* GITHUB#13931: Refactored `BooleanScorer` to evaluate matches of sub clauses + using the `Scorer` abstraction rather than the `BulkScorer` abstraction. This + speeds up exhaustive evaluation of disjunctions of term queries. + (Adrien Grand) + Bug Fixes --------------------- * GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java index e6e4f456bf3..7be1558dd71 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java @@ -20,13 +20,14 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.Objects; +import org.apache.lucene.internal.hppc.LongArrayList; import org.apache.lucene.util.Bits; import org.apache.lucene.util.PriorityQueue; /** * {@link BulkScorer} that is used for pure disjunctions and disjunctions that have low values of * {@link BooleanQuery.Builder#setMinimumNumberShouldMatch(int)} and dense clauses. This scorer - * scores documents by batches of 2048 docs. + * scores documents by batches of 4,096 docs. */ final class BooleanScorer extends BulkScorer { @@ -41,71 +42,32 @@ final class BooleanScorer extends BulkScorer { int freq; } - private class BulkScorerAndDoc { - final BulkScorer scorer; - final long cost; - int next; - - BulkScorerAndDoc(BulkScorer scorer) { - this.scorer = scorer; - this.cost = scorer.cost(); - this.next = -1; - } - - void advance(int min) throws IOException { - score(orCollector, null, min, min); - } - - void score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException { - next = scorer.score(collector, acceptDocs, min, max); - } - } - - // See WANDScorer for an explanation - private static long cost(Collection scorers, int minShouldMatch) { - final PriorityQueue pq = - new PriorityQueue(scorers.size() - minShouldMatch + 1) { - @Override - protected boolean lessThan(BulkScorer a, BulkScorer b) { - return a.cost() > b.cost(); - } - }; - for (BulkScorer scorer : scorers) { - pq.insertWithOverflow(scorer); - } - long cost = 0; - for (BulkScorer scorer = pq.pop(); scorer != null; scorer = pq.pop()) { - cost += scorer.cost(); - } - return cost; - } - - static final class HeadPriorityQueue extends PriorityQueue { + static final class HeadPriorityQueue extends PriorityQueue { public HeadPriorityQueue(int maxSize) { super(maxSize); } @Override - protected boolean lessThan(BulkScorerAndDoc a, BulkScorerAndDoc b) { - return a.next < b.next; + protected boolean lessThan(DisiWrapper a, DisiWrapper b) { + return a.doc < b.doc; } } - static final class TailPriorityQueue extends PriorityQueue { + static final class TailPriorityQueue extends PriorityQueue { public TailPriorityQueue(int maxSize) { super(maxSize); } @Override - protected boolean lessThan(BulkScorerAndDoc a, BulkScorerAndDoc b) { + protected boolean lessThan(DisiWrapper a, DisiWrapper b) { return a.cost < b.cost; } - public BulkScorerAndDoc get(int i) { + public DisiWrapper get(int i) { Objects.checkIndex(i, size()); - return (BulkScorerAndDoc) getHeapArray()[1 + i]; + return (DisiWrapper) getHeapArray()[1 + i]; } } @@ -115,7 +77,7 @@ final class BooleanScorer extends BulkScorer { // This is basically an inlined FixedBitSet... seems to help with bound checks final long[] matching = new long[SET_SIZE]; - final BulkScorerAndDoc[] leads; + final DisiWrapper[] leads; final HeadPriorityQueue head; final TailPriorityQueue tail; final Score score = new Score(); @@ -123,31 +85,6 @@ final class BooleanScorer extends BulkScorer { final long cost; final boolean needsScores; - final class OrCollector implements LeafCollector { - Scorable scorer; - - @Override - public void setScorer(Scorable scorer) { - this.scorer = scorer; - } - - @Override - public void collect(int doc) throws IOException { - final int i = doc & MASK; - final int idx = i >>> 6; - matching[idx] |= 1L << i; - if (buckets != null) { - final Bucket bucket = buckets[i]; - bucket.freq++; - if (needsScores) { - bucket.score += scorer.score(); - } - } - } - } - - final OrCollector orCollector = new OrCollector(); - final class DocIdStreamView extends DocIdStream { int base; @@ -194,7 +131,7 @@ final class BooleanScorer extends BulkScorer { private final DocIdStreamView docIdStreamView = new DocIdStreamView(); - BooleanScorer(Collection scorers, int minShouldMatch, boolean needsScores) { + BooleanScorer(Collection scorers, int minShouldMatch, boolean needsScores) { if (minShouldMatch < 1 || minShouldMatch > scorers.size()) { throw new IllegalArgumentException( "minShouldMatch should be within 1..num_scorers. Got " + minShouldMatch); @@ -211,18 +148,21 @@ final class BooleanScorer extends BulkScorer { } else { buckets = null; } - this.leads = new BulkScorerAndDoc[scorers.size()]; + this.leads = new DisiWrapper[scorers.size()]; this.head = new HeadPriorityQueue(scorers.size() - minShouldMatch + 1); this.tail = new TailPriorityQueue(minShouldMatch - 1); this.minShouldMatch = minShouldMatch; this.needsScores = needsScores; - for (BulkScorer scorer : scorers) { - final BulkScorerAndDoc evicted = tail.insertWithOverflow(new BulkScorerAndDoc(scorer)); + LongArrayList costs = new LongArrayList(scorers.size()); + for (Scorer scorer : scorers) { + DisiWrapper w = new DisiWrapper(scorer); + costs.add(w.cost); + final DisiWrapper evicted = tail.insertWithOverflow(w); if (evicted != null) { head.add(evicted); } } - this.cost = cost(scorers, minShouldMatch); + this.cost = ScorerUtil.costWithMinShouldMatch(costs.stream(), costs.size(), minShouldMatch); } @Override @@ -230,19 +170,49 @@ final class BooleanScorer extends BulkScorer { return cost; } + private void scoreDisiWrapperIntoBitSet(DisiWrapper w, Bits acceptDocs, int min, int max) + throws IOException { + boolean needsScores = BooleanScorer.this.needsScores; + long[] matching = BooleanScorer.this.matching; + Bucket[] buckets = BooleanScorer.this.buckets; + + DocIdSetIterator it = w.iterator; + Scorer scorer = w.scorer; + int doc = w.doc; + if (doc < min) { + doc = it.advance(min); + } + for (; doc < max; doc = it.nextDoc()) { + if (acceptDocs == null || acceptDocs.get(doc)) { + final int i = doc & MASK; + final int idx = i >> 6; + matching[idx] |= 1L << i; + if (buckets != null) { + final Bucket bucket = buckets[i]; + bucket.freq++; + if (needsScores) { + bucket.score += scorer.score(); + } + } + } + } + + w.doc = doc; + } + private void scoreWindowIntoBitSetAndReplay( LeafCollector collector, Bits acceptDocs, int base, int min, int max, - BulkScorerAndDoc[] scorers, + DisiWrapper[] scorers, int numScorers) throws IOException { for (int i = 0; i < numScorers; ++i) { - final BulkScorerAndDoc scorer = scorers[i]; - assert scorer.next < max; - scorer.score(orCollector, acceptDocs, min, max); + final DisiWrapper w = scorers[i]; + assert w.doc < max; + scoreDisiWrapperIntoBitSet(w, acceptDocs, min, max); } docIdStreamView.base = base; @@ -251,20 +221,20 @@ final class BooleanScorer extends BulkScorer { Arrays.fill(matching, 0L); } - private BulkScorerAndDoc advance(int min) throws IOException { + private DisiWrapper advance(int min) throws IOException { assert tail.size() == minShouldMatch - 1; final HeadPriorityQueue head = this.head; final TailPriorityQueue tail = this.tail; - BulkScorerAndDoc headTop = head.top(); - BulkScorerAndDoc tailTop = tail.top(); - while (headTop.next < min) { + DisiWrapper headTop = head.top(); + DisiWrapper tailTop = tail.top(); + while (headTop.doc < min) { if (tailTop == null || headTop.cost <= tailTop.cost) { - headTop.advance(min); + headTop.doc = headTop.iterator.advance(min); headTop = head.updateTop(); } else { // swap the top of head and tail - final BulkScorerAndDoc previousHeadTop = headTop; - tailTop.advance(min); + final DisiWrapper previousHeadTop = headTop; + tailTop.doc = tailTop.iterator.advance(min); headTop = head.updateTop(tailTop); tailTop = tail.updateTop(previousHeadTop); } @@ -282,9 +252,11 @@ final class BooleanScorer extends BulkScorer { throws IOException { while (maxFreq < minShouldMatch && maxFreq + tail.size() >= minShouldMatch) { // a match is still possible - final BulkScorerAndDoc candidate = tail.pop(); - candidate.advance(windowMin); - if (candidate.next < windowMax) { + final DisiWrapper candidate = tail.pop(); + if (candidate.doc < windowMin) { + candidate.doc = candidate.iterator.advance(windowMin); + } + if (candidate.doc < windowMax) { leads[maxFreq++] = candidate; } else { head.add(candidate); @@ -304,7 +276,7 @@ final class BooleanScorer extends BulkScorer { // Push back scorers into head and tail for (int i = 0; i < maxFreq; ++i) { - final BulkScorerAndDoc evicted = head.insertWithOverflow(leads[i]); + final DisiWrapper evicted = head.insertWithOverflow(leads[i]); if (evicted != null) { tail.add(evicted); } @@ -312,7 +284,7 @@ final class BooleanScorer extends BulkScorer { } private void scoreWindowSingleScorer( - BulkScorerAndDoc bulkScorer, + DisiWrapper w, LeafCollector collector, Bits acceptDocs, int windowMin, @@ -320,33 +292,44 @@ final class BooleanScorer extends BulkScorer { int max) throws IOException { assert tail.size() == 0; - final int nextWindowBase = head.top().next & ~MASK; + final int nextWindowBase = head.top().doc & ~MASK; final int end = Math.max(windowMax, Math.min(max, nextWindowBase)); - bulkScorer.score(collector, acceptDocs, windowMin, end); + DocIdSetIterator it = w.iterator; + int doc = w.doc; + if (doc < windowMin) { + doc = it.advance(windowMin); + } + collector.setScorer(w.scorer); + for (; doc < end; doc = it.nextDoc()) { + if (acceptDocs == null || acceptDocs.get(doc)) { + collector.collect(doc); + } + } + w.doc = doc; // reset the scorer that should be used for the general case collector.setScorer(score); } - private BulkScorerAndDoc scoreWindow( - BulkScorerAndDoc top, LeafCollector collector, Bits acceptDocs, int min, int max) + private DisiWrapper scoreWindow( + DisiWrapper top, LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException { - final int windowBase = top.next & ~MASK; // find the window that the next match belongs to + final int windowBase = top.doc & ~MASK; // find the window that the next match belongs to final int windowMin = Math.max(min, windowBase); final int windowMax = Math.min(max, windowBase + SIZE); // Fill 'leads' with all scorers from 'head' that are in the right window leads[0] = head.pop(); int maxFreq = 1; - while (head.size() > 0 && head.top().next < windowMax) { + while (head.size() > 0 && head.top().doc < windowMax) { leads[maxFreq++] = head.pop(); } if (minShouldMatch == 1 && maxFreq == 1) { // special case: only one scorer can match in the current window, // we can collect directly - final BulkScorerAndDoc bulkScorer = leads[0]; + final DisiWrapper bulkScorer = leads[0]; scoreWindowSingleScorer(bulkScorer, collector, acceptDocs, windowMin, windowMax, max); return head.add(bulkScorer); } else { @@ -360,11 +343,11 @@ final class BooleanScorer extends BulkScorer { public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException { collector.setScorer(score); - BulkScorerAndDoc top = advance(min); - while (top.next < max) { + DisiWrapper top = advance(min); + while (top.doc < max) { top = scoreWindow(top, collector, acceptDocs, min, max); } - return top.next; + return top.doc; } } diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java index b07d8ebb88e..a8169ad227f 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java @@ -289,9 +289,9 @@ final class BooleanScorerSupplier extends ScorerSupplier { return new MaxScoreBulkScorer(maxDoc, optionalScorers); } - List optional = new ArrayList(); + List optional = new ArrayList(); for (ScorerSupplier ss : subs.get(Occur.SHOULD)) { - optional.add(ss.bulkScorer()); + optional.add(ss.get(Long.MAX_VALUE)); } return new BooleanScorer(optional, Math.max(1, minShouldMatch), scoreMode.needsScores()); diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java index f73f504f3cf..08fc1296d10 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java @@ -153,70 +153,6 @@ final class BooleanWeight extends Weight { return MatchesUtils.fromSubMatches(matches); } - // Return a BulkScorer for the optional clauses only, - // or null if it is not applicable - // pkg-private for forcing use of BooleanScorer in tests - BulkScorer optionalBulkScorer(LeafReaderContext context) throws IOException { - if (scoreMode == ScoreMode.TOP_SCORES) { - if (!query.isPureDisjunction()) { - return null; - } - - List optional = new ArrayList<>(); - for (WeightedBooleanClause wc : weightedClauses) { - Weight w = wc.weight; - BooleanClause c = wc.clause; - if (c.occur() != Occur.SHOULD) { - continue; - } - ScorerSupplier scorer = w.scorerSupplier(context); - if (scorer != null) { - optional.add(scorer); - } - } - - if (optional.size() <= 1) { - return null; - } - - List optionalScorers = new ArrayList<>(); - for (ScorerSupplier ss : optional) { - optionalScorers.add(ss.get(Long.MAX_VALUE)); - } - - return new MaxScoreBulkScorer(context.reader().maxDoc(), optionalScorers); - } - - List optional = new ArrayList(); - for (WeightedBooleanClause wc : weightedClauses) { - Weight w = wc.weight; - BooleanClause c = wc.clause; - if (c.occur() != Occur.SHOULD) { - continue; - } - BulkScorer subScorer = w.bulkScorer(context); - - if (subScorer != null) { - optional.add(subScorer); - } - } - - if (optional.size() == 0) { - return null; - } - - if (query.getMinimumNumberShouldMatch() > optional.size()) { - return null; - } - - if (optional.size() == 1) { - return optional.get(0); - } - - return new BooleanScorer( - optional, Math.max(1, query.getMinimumNumberShouldMatch()), scoreMode.needsScores()); - } - @Override public int count(LeafReaderContext context) throws IOException { final int numDocs = context.reader().numDocs(); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java index d957e22d7b2..6fe375c086d 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java @@ -16,7 +16,6 @@ */ package org.apache.lucene.search; -import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -33,8 +32,9 @@ import org.apache.lucene.tests.index.RandomIndexWriter; import org.apache.lucene.tests.search.QueryUtils; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.tests.util.TestUtil; -import org.apache.lucene.util.Bits; +import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.IntArrayDocIdSet; public class TestBooleanOr extends LuceneTestCase { @@ -205,34 +205,30 @@ public class TestBooleanOr extends LuceneTestCase { dir.close(); } - private static BulkScorer scorer(int... matches) { - return new BulkScorer() { - final Score scorer = new Score(); - int i = 0; + private static Scorer scorer(int... matches) throws IOException { + matches = ArrayUtil.growExact(matches, matches.length + 1); + matches[matches.length - 1] = DocIdSetIterator.NO_MORE_DOCS; + DocIdSetIterator it = new IntArrayDocIdSet(matches, matches.length - 1).iterator(); + return new Scorer() { @Override - public int score(LeafCollector collector, Bits acceptDocs, int min, int max) - throws IOException { - collector.setScorer(scorer); - while (i < matches.length && matches[i] < min) { - i += 1; - } - while (i < matches.length && matches[i] < max) { - int doc = matches[i]; - if (acceptDocs == null || acceptDocs.get(doc)) { - collector.collect(doc); - } - i += 1; - } - if (i == matches.length) { - return DocIdSetIterator.NO_MORE_DOCS; - } - return RandomNumbers.randomIntBetween(random(), max, matches[i]); + public DocIdSetIterator iterator() { + return it; } @Override - public long cost() { - return matches.length; + public int docID() { + return it.docID(); + } + + @Override + public float getMaxScore(int upTo) throws IOException { + return Float.MAX_VALUE; + } + + @Override + public float score() throws IOException { + return 0; } }; } @@ -240,7 +236,7 @@ public class TestBooleanOr extends LuceneTestCase { // Make sure that BooleanScorer keeps working even if the sub clauses return // next matching docs which are less than the actual next match public void testSubScorerNextIsNotMatch() throws IOException { - final List optionalScorers = + final List optionalScorers = Arrays.asList( scorer(100000, 1000001, 9999999), scorer(4000, 1000051), diff --git a/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java b/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java index 95f6f7ee124..96572226c20 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java @@ -128,7 +128,8 @@ public class TestMinShouldMatch2 extends LuceneTestCase { case SCORER: return weight.scorer(reader.getContext()); case BULK_SCORER: - final BulkScorer bulkScorer = weight.optionalBulkScorer(reader.getContext()); + final ScorerSupplier ss = weight.scorerSupplier(reader.getContext()); + final BulkScorer bulkScorer = ss.bulkScorer(); if (bulkScorer == null) { if (weight.scorer(reader.getContext()) != null) { throw new AssertionError("BooleanScorer should be applicable for this query"); From f8ea1305144374923a8c2c7feeb5cd44d3cec1af Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 22 Oct 2024 16:32:28 +0200 Subject: [PATCH 06/37] Reduce the compiled size of the collect() method on `TopScoreDocCollector`. (#13939) This comes from observations on https://tantivy-search.github.io/bench/ for exhaustive evaluation like `TOP_100_COUNT`. `collect()` is often inlined, but other methods that we'd like to see inlined like `PostingsEnum#nextDoc()` are not always inlined. This PR decreases the compiled size of `collect()` to make more room for other methods to be inlined. It does so by moving an assertion to `AssertingScorable` and extracting an uncommon code path to a method. --- .../lucene/search/TopScoreDocCollector.java | 46 ++++++++++++------- .../tests/search/AssertingScorable.java | 3 +- 2 files changed, 31 insertions(+), 18 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java index 3469276982b..1524036b20d 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java @@ -70,17 +70,16 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { public void collect(int doc) throws IOException { float score = scorer.score(); - // This collector relies on the fact that scorers produce positive values: - assert score >= 0; // NOTE: false for NaN - - totalHits++; + int hitCountSoFar = ++totalHits; hitsThresholdChecker.incrementHitCount(); - if (minScoreAcc != null && (totalHits & minScoreAcc.modInterval) == 0) { + if (minScoreAcc != null && (hitCountSoFar & minScoreAcc.modInterval) == 0) { updateGlobalMinCompetitiveScore(scorer); } if (score <= pqTop.score) { + // Note: for queries that match lots of hits, this is the common case: most hits are not + // competitive. if (totalHitsRelation == TotalHits.Relation.EQUAL_TO) { // we just reached totalHitsThreshold, we can start setting the min // competitive score now @@ -89,8 +88,12 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { // Since docs are returned in-order (i.e., increasing doc Id), a document // with equal score to pqTop.score cannot compete since HitQueue favors // documents with lower doc Ids. Therefore reject those docs too. - return; + } else { + collectCompetitiveHit(doc, score); } + } + + private void collectCompetitiveHit(int doc, float score) throws IOException { pqTop.doc = doc + docBase; pqTop.score = score; pqTop = pq.updateTop(); @@ -103,7 +106,6 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { static class PagingTopScoreDocCollector extends TopScoreDocCollector { private final ScoreDoc after; - private int collectedHits; PagingTopScoreDocCollector( int numHits, @@ -112,12 +114,19 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { MaxScoreAccumulator minScoreAcc) { super(numHits, hitsThresholdChecker, minScoreAcc); this.after = after; - this.collectedHits = 0; } @Override protected int topDocsSize() { - return collectedHits < pq.size() ? collectedHits : pq.size(); + // Note: this relies on sentinel values having Integer.MAX_VALUE as a doc ID. + int[] validTopHitCount = new int[1]; + pq.forEach( + scoreDoc -> { + if (scoreDoc.doc != Integer.MAX_VALUE) { + validTopHitCount[0]++; + } + }); + return validTopHitCount[0]; } @Override @@ -148,17 +157,15 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { public void collect(int doc) throws IOException { float score = scorer.score(); - // This collector relies on the fact that scorers produce positive values: - assert score >= 0; // NOTE: false for NaN - - totalHits++; + int hitCountSoFar = ++totalHits; hitsThresholdChecker.incrementHitCount(); - if (minScoreAcc != null && (totalHits & minScoreAcc.modInterval) == 0) { + if (minScoreAcc != null && (hitCountSoFar & minScoreAcc.modInterval) == 0) { updateGlobalMinCompetitiveScore(scorer); } - if (score > after.score || (score == after.score && doc <= afterDoc)) { + float afterScore = after.score; + if (score > afterScore || (score == afterScore && doc <= afterDoc)) { // hit was collected on a previous page if (totalHitsRelation == TotalHits.Relation.EQUAL_TO) { // we just reached totalHitsThreshold, we can start setting the min @@ -169,6 +176,8 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { } if (score <= pqTop.score) { + // Note: for queries that match lots of hits, this is the common case: most hits are not + // competitive. if (totalHitsRelation == TotalHits.Relation.EQUAL_TO) { // we just reached totalHitsThreshold, we can start setting the min // competitive score now @@ -178,9 +187,12 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { // Since docs are returned in-order (i.e., increasing doc Id), a document // with equal score to pqTop.score cannot compete since HitQueue favors // documents with lower doc Ids. Therefore reject those docs too. - return; + } else { + collectCompetitiveHit(doc, score); } - collectedHits++; + } + + private void collectCompetitiveHit(int doc, float score) throws IOException { pqTop.doc = doc + docBase; pqTop.score = score; pqTop = pq.updateTop(); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorable.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorable.java index 3a87d4c0f1a..2f3d604ad25 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorable.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingScorable.java @@ -33,7 +33,8 @@ public class AssertingScorable extends FilterScorable { @Override public float score() throws IOException { final float score = in.score(); - assert !Float.isNaN(score) : "NaN score for in=" + in; + // Note: score >= 0 returns false for NaN + assert score >= 0 : "score=" + score + " for in=" + in; return score; } From bb3f1822517f6a4b16fca6363d62cfcc8a541cef Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 22 Oct 2024 16:52:36 +0200 Subject: [PATCH 07/37] Introduce a heuristic to amortize the per-window overhead in MaxScoreBulkScorer. (#13941) It is sometimes possible for `MaxScoreBulkScorer` to compute windows that don't contain many candidate matches, resulting in more time spent evaluating maximum scores per window than evaluating candidate matches on this window. This PR introduces a heuristic that tries to require at least 32 candidate matches per clause per window to amortize the per-window overhead. This results in a speedup for the `OrMany` task. --- lucene/CHANGES.txt | 3 ++ .../lucene/search/MaxScoreBulkScorer.java | 30 +++++++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 04563553106..47a5802da39 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -60,6 +60,9 @@ Optimizations speeds up exhaustive evaluation of disjunctions of term queries. (Adrien Grand) +* GITHUB#13941: Optimized computation of top-hits on disjunctive queries with + many clauses. (Adrien Grand) + Bug Fixes --------------------- * GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended diff --git a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java index 18f5b83e93a..56857bc67cc 100644 --- a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java @@ -66,6 +66,15 @@ final class MaxScoreBulkScorer extends BulkScorer { maxScoreSums = new double[allScorers.length]; } + // Number of outer windows that have been evaluated + private int numOuterWindows; + // Number of candidate matches so far + private int numCandidates; + // Minimum window size. See #computeOuterWindowMax where we have heuristics that adjust the + // minimum window size based on the average number of candidate matches per outer window, to keep + // the per-window overhead under control. + private int minWindowSize = 1; + @Override public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException { collector.setScorer(scorable); @@ -124,6 +133,7 @@ final class MaxScoreBulkScorer extends BulkScorer { } outerWindowMin = Math.min(top.doc, outerWindowMax); + ++numOuterWindows; } return nextCandidate(max); @@ -278,6 +288,23 @@ final class MaxScoreBulkScorer extends BulkScorer { windowMax = (int) Math.min(windowMax, upTo + 1L); // upTo is inclusive } + if (allScorers.length - firstWindowLead > 1) { + // The more clauses we consider to compute outer windows, the higher chances that one of these + // clauses has a block boundary in the next few doc IDs. This situation can result in more + // time spent computing maximum scores per outer window than evaluating hits. To avoid such + // situations, we target at least 32 candidate matches per clause per outer window on average, + // to make sure we amortize the cost of computing maximum scores. + long threshold = numOuterWindows * 32L * allScorers.length; + if (numCandidates < threshold) { + minWindowSize = Math.min(minWindowSize << 1, INNER_WINDOW_SIZE); + } else { + minWindowSize = 1; + } + + int minWindowMax = (int) Math.min(Integer.MAX_VALUE, (long) windowMin + minWindowSize); + windowMax = Math.max(windowMax, minWindowMax); + } + return windowMax; } @@ -300,6 +327,9 @@ final class MaxScoreBulkScorer extends BulkScorer { private void scoreNonEssentialClauses( LeafCollector collector, int doc, double essentialScore, int numNonEssentialClauses) throws IOException { + + ++numCandidates; + double score = essentialScore; for (int i = numNonEssentialClauses - 1; i >= 0; --i) { float maxPossibleScore = From 2ec5cc8c175806df4634ebee9a4d4687fbc59872 Mon Sep 17 00:00:00 2001 From: Jakub Slowinski <32519034+slow-J@users.noreply.github.com> Date: Thu, 24 Oct 2024 10:46:09 +0100 Subject: [PATCH 08/37] Removing the deprecated parameters, -fast, -slow, -crossCheckTermVectors from CheckIndex. (#11023) (#13942) Their usage is replaced with -level with respective values of 1, 3, 3. --- lucene/CHANGES.txt | 2 +- .../java/org/apache/lucene/index/CheckIndex.java | 13 ------------- 2 files changed, 1 insertion(+), 14 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 47a5802da39..9aa1dfa5233 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -7,7 +7,7 @@ http://s.apache.org/luceneversions API Changes --------------------- -(No changes) +* GITHUB#11023: Removing deprecated parameters from CheckIndex. (Jakub Slowinski) New Features --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java index becb00cbb5b..d957af01d0a 100644 --- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java +++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java @@ -4284,21 +4284,8 @@ public final class CheckIndex implements Closeable { int level = Integer.parseInt(args[i]); Level.checkIfLevelInBounds(level); opts.level = level; - } else if ("-fast".equals(arg)) { - // Deprecated. Remove in Lucene 11. - System.err.println( - "-fast is deprecated, use '-level 1' for explicitly verifying file checksums only. This is also now the default " - + "behaviour!"); - } else if ("-slow".equals(arg)) { - // Deprecated. Remove in Lucene 11. - System.err.println("-slow is deprecated, use '-level 3' instead for slow checks"); - opts.level = Level.MIN_LEVEL_FOR_SLOW_CHECKS; } else if ("-exorcise".equals(arg)) { opts.doExorcise = true; - } else if ("-crossCheckTermVectors".equals(arg)) { - // Deprecated. Remove in Lucene 11. - System.err.println("-crossCheckTermVectors is deprecated, use '-level 3' instead"); - opts.level = Level.MAX_VALUE; } else if (arg.equals("-verbose")) { opts.verbose = true; } else if (arg.equals("-segment")) { From 0bbef32cf500b4c4e630b05c744d8cfc44460c37 Mon Sep 17 00:00:00 2001 From: Lu Xugang Date: Fri, 25 Oct 2024 15:05:51 +0800 Subject: [PATCH 09/37] Check ahead if we can get the count (#13899) Currently, we traverse the BKD tree or perform a binary search using DocValues first, and then check whether the count can be obtained in the count() method of IndexSortSortedNumericDocValuesRangeQuery. we should consider providing a mechanism to perform this check beforehand, avoid unnecessary processing when dealing with a sparseRange --- lucene/CHANGES.txt | 2 ++ ...xSortSortedNumericDocValuesRangeQuery.java | 36 ++++++++++++++++++- 2 files changed, 37 insertions(+), 1 deletion(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 9aa1dfa5233..941e58814cf 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -63,6 +63,8 @@ Optimizations * GITHUB#13941: Optimized computation of top-hits on disjunctive queries with many clauses. (Adrien Grand) +* GITHUB#13899: Check ahead if we can get the count. (Lu Xugang) + Bug Fixes --------------------- * GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSortSortedNumericDocValuesRangeQuery.java b/lucene/core/src/java/org/apache/lucene/search/IndexSortSortedNumericDocValuesRangeQuery.java index e8b341cdcfb..3b6ef588dfb 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSortSortedNumericDocValuesRangeQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSortSortedNumericDocValuesRangeQuery.java @@ -186,10 +186,44 @@ public class IndexSortSortedNumericDocValuesRangeQuery extends Query { @Override public int count(LeafReaderContext context) throws IOException { if (context.reader().hasDeletions() == false) { - IteratorAndCount itAndCount = getDocIdSetIteratorOrNull(context); + if (lowerValue > upperValue) { + return 0; + } + IteratorAndCount itAndCount = null; + LeafReader reader = context.reader(); + + // first use bkd optimization if possible + SortedNumericDocValues sortedNumericValues = DocValues.getSortedNumeric(reader, field); + NumericDocValues numericValues = DocValues.unwrapSingleton(sortedNumericValues); + PointValues pointValues = reader.getPointValues(field); + if (pointValues != null && pointValues.getDocCount() == reader.maxDoc()) { + itAndCount = getDocIdSetIteratorOrNullFromBkd(context, numericValues); + } if (itAndCount != null && itAndCount.count != -1) { return itAndCount.count; } + + // use index sort optimization if possible + Sort indexSort = reader.getMetaData().sort(); + if (indexSort != null + && indexSort.getSort().length > 0 + && indexSort.getSort()[0].getField().equals(field)) { + final SortField sortField = indexSort.getSort()[0]; + final SortField.Type sortFieldType = getSortFieldType(sortField); + // The index sort optimization is only supported for Type.INT and Type.LONG + if (sortFieldType == Type.INT || sortFieldType == Type.LONG) { + Object missingValue = sortField.getMissingValue(); + final long missingLongValue = missingValue == null ? 0L : (long) missingValue; + // all documents have docValues or missing value falls outside the range + if ((pointValues != null && pointValues.getDocCount() == reader.maxDoc()) + || (missingLongValue < lowerValue || missingLongValue > upperValue)) { + itAndCount = getDocIdSetIterator(sortField, sortFieldType, context, numericValues); + } + if (itAndCount != null && itAndCount.count != -1) { + return itAndCount.count; + } + } + } } return fallbackWeight.count(context); } From 8362f609b256960a7c8410d213f12e94cc2f23b0 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 25 Oct 2024 13:30:29 +0200 Subject: [PATCH 10/37] Add MIGRATE entry about the fact that readVLong() may now read up to 10 bytes. (#13956) This may be of interest for custom `DataInput`/`IndexInput` implementations that extend `readVLong()`. --- lucene/MIGRATE.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/lucene/MIGRATE.md b/lucene/MIGRATE.md index 456e867800f..1db50b7fdd8 100644 --- a/lucene/MIGRATE.md +++ b/lucene/MIGRATE.md @@ -19,6 +19,13 @@ ## Migration from Lucene 9.x to Lucene 10.0 +### DataInput#readVLong() may now read negative vlongs + +LUCENE-10376 started allowing `DataInput#readVLong()` to read negative vlongs. +In particular, this feature is used by the `DataInput#readZLong()` method. A +practical implication is that `DataInput#readVLong()` may now read up to 10 +bytes, while it would never read more than 9 bytes in Lucene 9.x. + ### Changes to DataInput.readGroupVInt and readGroupVInts methods As part of GITHUB#13820, GITHUB#13825, GITHUB#13830, this issue corrects DataInput.readGroupVInts From 226824241210b74f16173e1dd312b3f3ea9b46ec Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 25 Oct 2024 13:30:49 +0200 Subject: [PATCH 11/37] Remove some useless code in TopScoreDocCollector. (#13955) --- .../org/apache/lucene/search/TopScoreDocCollector.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java index 1524036b20d..025e7937601 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java @@ -254,11 +254,13 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { } protected void updateMinCompetitiveScore(Scorable scorer) throws IOException { - if (hitsThresholdChecker.isThresholdReached() - && pqTop != null - && pqTop.score != Float.NEGATIVE_INFINITY) { // -Infinity is the score of sentinels + if (hitsThresholdChecker.isThresholdReached()) { // since we tie-break on doc id and collect in doc id order, we can require // the next float + // pqTop is never null since TopScoreDocCollector fills the priority queue with sentinel + // values + // if the top element is a sentinel value, its score will be -Infty and the below logic is + // still valid float localMinScore = Math.nextUp(pqTop.score); if (localMinScore > minCompetitiveScore) { scorer.setMinCompetitiveScore(localMinScore); From bf69c3cf4370c1afc932e3d94703b3cea75e5415 Mon Sep 17 00:00:00 2001 From: Shubham Chaudhary <36742242+shubhamvishu@users.noreply.github.com> Date: Fri, 25 Oct 2024 17:01:34 +0530 Subject: [PATCH 12/37] Make some BooleanQuery methods public and a new `#add(Collection)` method for BQ builder (#13950) --- lucene/CHANGES.txt | 2 ++ .../org/apache/lucene/search/BooleanQuery.java | 18 +++++++++++++++++- 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 941e58814cf..9da41e6e4cb 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -36,6 +36,8 @@ API Changes * GITHUB#13859: Allow open-ended ranges in Intervals range queries. (Mayya Sharipova) +* GITHUB#13950: Make BooleanQuery#getClauses public and add #add(Collection) to BQ builder. (Shubham Chaudhary) + New Features --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java b/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java index 4798745ef67..0d6daa07ba1 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java @@ -87,6 +87,22 @@ public class BooleanQuery extends Query implements Iterable { return this; } + /** + * Add a collection of BooleanClauses to this {@link Builder}. Note that the order in which + * clauses are added does not have any impact on matching documents or query performance. + * + * @throws IndexSearcher.TooManyClauses if the new number of clauses exceeds the maximum clause + * number + */ + public Builder add(Collection collection) { + // see #addClause(BooleanClause) + if ((clauses.size() + collection.size()) > IndexSearcher.maxClauseCount) { + throw new IndexSearcher.TooManyClauses(); + } + clauses.addAll(collection); + return this; + } + /** * Add a new clause to this {@link Builder}. Note that the order in which clauses are added does * not have any impact on matching documents or query performance. @@ -136,7 +152,7 @@ public class BooleanQuery extends Query implements Iterable { } /** Return the collection of queries for the given {@link Occur}. */ - Collection getClauses(Occur occur) { + public Collection getClauses(Occur occur) { return clauseSets.get(occur); } From 618bfa59a6df4f775488351d5258dce29420cba8 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Fri, 25 Oct 2024 07:42:26 -0400 Subject: [PATCH 13/37] Ensure doc order for TestCommonTermsQuery#testMinShouldMatch (#13953) --- .../org/apache/lucene/queries/TestCommonTermsQuery.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/lucene/queries/src/test/org/apache/lucene/queries/TestCommonTermsQuery.java b/lucene/queries/src/test/org/apache/lucene/queries/TestCommonTermsQuery.java index b94960d587a..9282fdb7946 100644 --- a/lucene/queries/src/test/org/apache/lucene/queries/TestCommonTermsQuery.java +++ b/lucene/queries/src/test/org/apache/lucene/queries/TestCommonTermsQuery.java @@ -194,7 +194,12 @@ public class TestCommonTermsQuery extends LuceneTestCase { public void testMinShouldMatch() throws IOException { Directory dir = newDirectory(); MockAnalyzer analyzer = new MockAnalyzer(random()); - RandomIndexWriter w = new RandomIndexWriter(random(), dir, analyzer); + RandomIndexWriter w = + new RandomIndexWriter( + random(), + dir, + LuceneTestCase.newIndexWriterConfig(analyzer) + .setMergePolicy(LuceneTestCase.newMergePolicy(random(), false))); String[] docs = new String[] { "this is the end of the world right", From 5c080156f57bd16b7a877fdfa36feb2b439e4870 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 25 Oct 2024 14:02:04 +0200 Subject: [PATCH 14/37] Disable exchanging minimum scores across slices for exhaustive evaluation. (#13954) When `totalHitsThreshold` is `Integer.MAX_VALUE`, dynamic pruning is never used and all hits get evaluated. Thus, the minimum competitive score always stays at zero, and there is nothing to exchange across slices. --- lucene/CHANGES.txt | 4 ++++ .../org/apache/lucene/search/TopFieldCollectorManager.java | 5 ++++- .../apache/lucene/search/TopScoreDocCollectorManager.java | 5 ++++- 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 9da41e6e4cb..f3e359b7630 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -65,8 +65,12 @@ Optimizations * GITHUB#13941: Optimized computation of top-hits on disjunctive queries with many clauses. (Adrien Grand) +* GITHUB#13954: Disabled exchanging scores across slices for exhaustive + top-hits evaluation. (Adrien Grand) + * GITHUB#13899: Check ahead if we can get the count. (Lu Xugang) + Bug Fixes --------------------- * GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java index d09a589be97..2d4a0c0e215 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java @@ -93,7 +93,10 @@ public class TopFieldCollectorManager implements CollectorManager(); } diff --git a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java index 4e3181abdf7..94a4e1a82d3 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java @@ -75,7 +75,10 @@ public class TopScoreDocCollectorManager supportsConcurrency ? HitsThresholdChecker.createShared(Math.max(totalHitsThreshold, numHits)) : HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits)); - this.minScoreAcc = supportsConcurrency ? new MaxScoreAccumulator() : null; + this.minScoreAcc = + supportsConcurrency && totalHitsThreshold != Integer.MAX_VALUE + ? new MaxScoreAccumulator() + : null; } /** From b12ee529997fba510fdb3b0b40104b6e4ddccebe Mon Sep 17 00:00:00 2001 From: ljak Date: Fri, 25 Oct 2024 12:21:58 -0400 Subject: [PATCH 15/37] Ensure stability of clause order for DisjunctionMaxQuery toString (#13944) Co-authored-by: Laurent --- lucene/CHANGES.txt | 1 + .../lucene/search/DisjunctionMaxQuery.java | 34 +++++++++---------- .../search/TestDisjunctionMaxQuery.java | 22 ++++++++++++ 3 files changed, 39 insertions(+), 18 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index f3e359b7630..eacb70642eb 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -78,6 +78,7 @@ Bug Fixes * GITHUB#13884: Remove broken .toArray from Long/CharObjectHashMap entirely. (Pan Guixin) * GITHUB#12686: Added support for highlighting IndexOrDocValuesQuery. (Prudhvi Godithi) * GITHUB#13927: Fix StoredFieldsConsumer finish. (linfn) +* GITHUB#13944: Ensure deterministic order of clauses for `DisjunctionMaxQuery#toString`. (Laurent Jakubina) Build --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java index 5a53749b667..8d6bae3e7ea 100644 --- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java @@ -23,6 +23,7 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Objects; +import java.util.stream.Collectors; import org.apache.lucene.index.LeafReaderContext; /** @@ -44,6 +45,7 @@ public final class DisjunctionMaxQuery extends Query implements Iterable /* The subqueries */ private final Multiset disjuncts = new Multiset<>(); + private final List orderedQueries; // used for toString() /* Multiple of the non-max disjunct scores added into our final score. Non-zero values support tie-breaking. */ private final float tieBreakerMultiplier; @@ -65,6 +67,7 @@ public final class DisjunctionMaxQuery extends Query implements Iterable } this.tieBreakerMultiplier = tieBreakerMultiplier; this.disjuncts.addAll(disjuncts); + this.orderedQueries = new ArrayList<>(disjuncts); // order from the caller } /** @@ -295,24 +298,19 @@ public final class DisjunctionMaxQuery extends Query implements Iterable */ @Override public String toString(String field) { - StringBuilder buffer = new StringBuilder(); - buffer.append("("); - Iterator it = disjuncts.iterator(); - for (int i = 0; it.hasNext(); i++) { - Query subquery = it.next(); - if (subquery instanceof BooleanQuery) { // wrap sub-bools in parens - buffer.append("("); - buffer.append(subquery.toString(field)); - buffer.append(")"); - } else buffer.append(subquery.toString(field)); - if (i != disjuncts.size() - 1) buffer.append(" | "); - } - buffer.append(")"); - if (tieBreakerMultiplier != 0.0f) { - buffer.append("~"); - buffer.append(tieBreakerMultiplier); - } - return buffer.toString(); + return this.orderedQueries.stream() + .map( + subquery -> { + if (subquery instanceof BooleanQuery) { // wrap sub-bools in parens + return "(" + subquery.toString(field) + ")"; + } + return subquery.toString(field); + }) + .collect( + Collectors.joining( + " | ", + "(", + ")" + ((tieBreakerMultiplier != 0.0f) ? "~" + tieBreakerMultiplier : ""))); } /** diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java index 12dc0ffc96b..1464aa84bd3 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java @@ -25,6 +25,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Locale; +import java.util.stream.Collectors; import org.apache.lucene.analysis.standard.StandardAnalyzer; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; @@ -489,6 +490,27 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase { assertEquals(q1, q2); } + /* Inspired from TestIntervals.testIntervalDisjunctionToStringStability */ + public void testToStringOrderMatters() { + final int clauseNbr = + random().nextInt(22) + 4; // ensure a reasonably large minimum number of clauses + final String[] terms = new String[clauseNbr]; + for (int i = 0; i < clauseNbr; i++) { + terms[i] = Character.toString((char) ('a' + i)); + } + + final String expected = + Arrays.stream(terms) + .map((term) -> "test:" + term) + .collect(Collectors.joining(" | ", "(", ")~1.0")); + + DisjunctionMaxQuery source = + new DisjunctionMaxQuery( + Arrays.stream(terms).map((term) -> tq("test", term)).toList(), 1.0f); + + assertEquals(expected, source.toString("")); + } + public void testRandomTopDocs() throws Exception { doTestRandomTopDocs(2, 0.05f, 0.05f); doTestRandomTopDocs(2, 1.0f, 0.05f); From a3a00f3f58f16a05d1e67243af9822583bedd649 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Sat, 26 Oct 2024 13:45:54 +0200 Subject: [PATCH 16/37] Remove LeafSimScorer abstraction. (#13957) `LeafSimScorer` is a specialization of a `SimScorer` for a given segment. It doesn't add much value, but benchmarks suggest that it adds measurable overhead to queries sorted by score. --- lucene/CHANGES.txt | 3 + .../apache/lucene/document/FeatureQuery.java | 6 +- .../apache/lucene/search/LeafSimScorer.java | 72 ------------------- .../apache/lucene/search/PhraseScorer.java | 21 ++++-- .../apache/lucene/search/PhraseWeight.java | 15 ++-- .../apache/lucene/search/SynonymQuery.java | 56 ++++++++++----- .../org/apache/lucene/search/TermQuery.java | 28 ++++---- .../org/apache/lucene/search/TermScorer.java | 37 +++++++--- .../lucene/search/TestMinShouldMatch2.java | 16 +++-- .../function/valuesource/NormValueSource.java | 11 +-- .../queries/payloads/PayloadScoreQuery.java | 12 ++-- .../payloads/SpanPayloadCheckQuery.java | 6 +- .../queries/spans/SpanContainingQuery.java | 6 +- .../lucene/queries/spans/SpanNearQuery.java | 5 +- .../lucene/queries/spans/SpanScorer.java | 21 ++++-- .../lucene/queries/spans/SpanTermQuery.java | 2 +- .../lucene/queries/spans/SpanWeight.java | 27 ++++--- .../lucene/queries/spans/TermSpans.java | 3 +- .../queries/spans/AssertingSpanWeight.java | 6 +- .../sandbox/search/CombinedFieldQuery.java | 5 +- .../search/MultiNormsLeafSimScorer.java | 6 +- .../sandbox/search/TermAutomatonQuery.java | 18 +++-- .../sandbox/search/TermAutomatonScorer.java | 27 ++++--- 23 files changed, 204 insertions(+), 205 deletions(-) delete mode 100644 lucene/core/src/java/org/apache/lucene/search/LeafSimScorer.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index eacb70642eb..360e47079fe 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -9,6 +9,9 @@ API Changes --------------------- * GITHUB#11023: Removing deprecated parameters from CheckIndex. (Jakub Slowinski) +* GITHUB#13957: Removed LeafSimScorer class, to save its overhead. Scorers now + compute scores directly from a SimScorer, postings and norms. (Adrien Grand) + New Features --------------------- (No changes) diff --git a/lucene/core/src/java/org/apache/lucene/document/FeatureQuery.java b/lucene/core/src/java/org/apache/lucene/document/FeatureQuery.java index ce530815345..255895705ca 100644 --- a/lucene/core/src/java/org/apache/lucene/document/FeatureQuery.java +++ b/lucene/core/src/java/org/apache/lucene/document/FeatureQuery.java @@ -27,7 +27,6 @@ import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryVisitor; import org.apache.lucene.search.ScoreMode; @@ -120,7 +119,6 @@ final class FeatureQuery extends Query { @Override public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { - final Weight thisWeight = this; Terms terms = Terms.getTerms(context.reader(), fieldName); TermsEnum termsEnum = terms.iterator(); if (termsEnum.seekExact(new BytesRef(featureName)) == false) { @@ -135,10 +133,8 @@ final class FeatureQuery extends Query { @Override public Scorer get(long leadCost) throws IOException { final SimScorer scorer = function.scorer(boost); - final LeafSimScorer simScorer = - new LeafSimScorer(scorer, context.reader(), fieldName, false); final ImpactsEnum impacts = termsEnum.impacts(PostingsEnum.FREQS); - return new TermScorer(thisWeight, impacts, simScorer, topLevelScoringClause); + return new TermScorer(impacts, scorer, null, topLevelScoringClause); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/search/LeafSimScorer.java b/lucene/core/src/java/org/apache/lucene/search/LeafSimScorer.java deleted file mode 100644 index 5d8d2e92286..00000000000 --- a/lucene/core/src/java/org/apache/lucene/search/LeafSimScorer.java +++ /dev/null @@ -1,72 +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.search; - -import java.io.IOException; -import java.util.Objects; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.search.similarities.Similarity.SimScorer; - -/** {@link SimScorer} on a specific {@link LeafReader}. */ -public final class LeafSimScorer { - - private final SimScorer scorer; - private final NumericDocValues norms; - - /** Sole constructor: Score documents of {@code reader} with {@code scorer}. */ - public LeafSimScorer(SimScorer scorer, LeafReader reader, String field, boolean needsScores) - throws IOException { - this.scorer = Objects.requireNonNull(scorer); - norms = needsScores ? reader.getNormValues(field) : null; - } - - /** Return the wrapped {@link SimScorer}. */ - public SimScorer getSimScorer() { - return scorer; - } - - private long getNormValue(int doc) throws IOException { - if (norms != null) { - boolean found = norms.advanceExact(doc); - assert found; - return norms.longValue(); - } else { - return 1L; // default norm - } - } - - /** - * Score the provided document assuming the given term document frequency. This method must be - * called on non-decreasing sequences of doc ids. - * - * @see SimScorer#score(float, long) - */ - public float score(int doc, float freq) throws IOException { - return scorer.score(freq, getNormValue(doc)); - } - - /** - * Explain the score for the provided document assuming the given term document frequency. This - * method must be called on non-decreasing sequences of doc ids. - * - * @see SimScorer#explain(Explanation, long) - */ - public Explanation explain(int doc, Explanation freqExpl) throws IOException { - return scorer.explain(freqExpl, getNormValue(doc)); - } -} diff --git a/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java b/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java index 26a1387acda..69f07e624f2 100644 --- a/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/PhraseScorer.java @@ -18,6 +18,8 @@ package org.apache.lucene.search; import java.io.IOException; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.search.similarities.Similarity.SimScorer; class PhraseScorer extends Scorer { @@ -26,16 +28,19 @@ class PhraseScorer extends Scorer { final MaxScoreCache maxScoreCache; final PhraseMatcher matcher; final ScoreMode scoreMode; - private final LeafSimScorer simScorer; + private final SimScorer simScorer; + private final NumericDocValues norms; final float matchCost; private float minCompetitiveScore = 0; private float freq = 0; - PhraseScorer(PhraseMatcher matcher, ScoreMode scoreMode, LeafSimScorer simScorer) { + PhraseScorer( + PhraseMatcher matcher, ScoreMode scoreMode, SimScorer simScorer, NumericDocValues norms) { this.matcher = matcher; this.scoreMode = scoreMode; this.simScorer = simScorer; + this.norms = norms; this.matchCost = matcher.getMatchCost(); this.approximation = matcher.approximation(); this.impactsApproximation = matcher.impactsApproximation(); @@ -50,7 +55,11 @@ class PhraseScorer extends Scorer { matcher.reset(); if (scoreMode == ScoreMode.TOP_SCORES && minCompetitiveScore > 0) { float maxFreq = matcher.maxFreq(); - if (simScorer.score(docID(), maxFreq) < minCompetitiveScore) { + long norm = 1L; + if (norms != null && norms.advanceExact(docID())) { + norm = norms.longValue(); + } + if (simScorer.score(maxFreq, norm) < minCompetitiveScore) { // The maximum score we could get is less than the min competitive score return false; } @@ -79,7 +88,11 @@ class PhraseScorer extends Scorer { freq += matcher.sloppyWeight(); } } - return simScorer.score(docID(), freq); + long norm = 1L; + if (norms != null && norms.advanceExact(docID())) { + norm = norms.longValue(); + } + return simScorer.score(freq, norm); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java b/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java index 6aa1d684968..75439ca7eb0 100644 --- a/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java +++ b/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java @@ -19,6 +19,7 @@ package org.apache.lucene.search; import java.io.IOException; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.Similarity.SimScorer; @@ -63,9 +64,8 @@ public abstract class PhraseWeight extends Weight { public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { PhraseMatcher matcher = getPhraseMatcher(context, stats, false); if (matcher == null) return null; - LeafSimScorer simScorer = - new LeafSimScorer(stats, context.reader(), field, scoreMode.needsScores()); - final var scorer = new PhraseScorer(matcher, scoreMode, simScorer); + NumericDocValues norms = scoreMode.needsScores() ? context.reader().getNormValues(field) : null; + final var scorer = new PhraseScorer(matcher, scoreMode, stats, norms); return new DefaultScorerSupplier(scorer); } @@ -83,10 +83,13 @@ public abstract class PhraseWeight extends Weight { while (matcher.nextMatch()) { freq += matcher.sloppyWeight(); } - LeafSimScorer docScorer = - new LeafSimScorer(stats, context.reader(), field, scoreMode.needsScores()); Explanation freqExplanation = Explanation.match(freq, "phraseFreq=" + freq); - Explanation scoreExplanation = docScorer.explain(doc, freqExplanation); + NumericDocValues norms = scoreMode.needsScores() ? context.reader().getNormValues(field) : null; + long norm = 1L; + if (norms != null && norms.advanceExact(doc)) { + norm = norms.longValue(); + } + Explanation scoreExplanation = stats.explain(freqExplanation, norm); return Explanation.match( scoreExplanation.getValue(), "weight(" diff --git a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java index 360fc854140..357f97019f8 100644 --- a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java @@ -30,6 +30,7 @@ import org.apache.lucene.index.Impacts; import org.apache.lucene.index.ImpactsEnum; import org.apache.lucene.index.ImpactsSource; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.SlowImpactsEnum; import org.apache.lucene.index.Term; @@ -38,6 +39,7 @@ import org.apache.lucene.index.TermStates; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.similarities.Similarity; +import org.apache.lucene.search.similarities.Similarity.SimScorer; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.PriorityQueue; @@ -259,9 +261,13 @@ public final class SynonymQuery extends Query { assert scorer instanceof TermScorer; freq = ((TermScorer) scorer).freq(); } - LeafSimScorer docScorer = new LeafSimScorer(simWeight, context.reader(), field, true); Explanation freqExplanation = Explanation.match(freq, "termFreq=" + freq); - Explanation scoreExplanation = docScorer.explain(doc, freqExplanation); + NumericDocValues norms = context.reader().getNormValues(field); + long norm = 1L; + if (norms != null && norms.advanceExact(doc)) { + norm = norms.longValue(); + } + Explanation scoreExplanation = simWeight.explain(freqExplanation, norm); return Explanation.match( scoreExplanation.getValue(), "weight(" @@ -334,27 +340,27 @@ public final class SynonymQuery extends Query { return new ConstantScoreScorer(0f, scoreMode, DocIdSetIterator.empty()); } - LeafSimScorer simScorer = new LeafSimScorer(simWeight, context.reader(), field, true); + NumericDocValues norms = context.reader().getNormValues(field); // we must optimize this case (term not in segment), disjunctions require >= 2 subs if (iterators.size() == 1) { final TermScorer scorer; if (scoreMode == ScoreMode.TOP_SCORES) { - scorer = new TermScorer(impacts.get(0), simScorer); + scorer = new TermScorer(impacts.get(0), simWeight, norms); } else { - scorer = new TermScorer(iterators.get(0), simScorer); + scorer = new TermScorer(iterators.get(0), simWeight, norms); } float boost = termBoosts.get(0); return scoreMode == ScoreMode.COMPLETE_NO_SCORES || boost == 1f ? scorer - : new FreqBoostTermScorer(boost, scorer, simScorer); + : new FreqBoostTermScorer(boost, scorer, simWeight, norms); } else { // we use termscorers + disjunction as an impl detail DisiPriorityQueue queue = new DisiPriorityQueue(iterators.size()); for (int i = 0; i < iterators.size(); i++) { PostingsEnum postings = iterators.get(i); - final TermScorer termScorer = new TermScorer(postings, simScorer); + final TermScorer termScorer = new TermScorer(postings, simWeight, norms); float boost = termBoosts.get(i); final DisiWrapperFreq wrapper = new DisiWrapperFreq(termScorer, boost); queue.add(wrapper); @@ -368,8 +374,7 @@ public final class SynonymQuery extends Query { boosts[i] = termBoosts.get(i); } ImpactsSource impactsSource = mergeImpacts(impacts.toArray(new ImpactsEnum[0]), boosts); - MaxScoreCache maxScoreCache = - new MaxScoreCache(impactsSource, simScorer.getSimScorer()); + MaxScoreCache maxScoreCache = new MaxScoreCache(impactsSource, simWeight); ImpactsDISI impactsDisi = new ImpactsDISI(iterator, maxScoreCache); if (scoreMode == ScoreMode.TOP_SCORES) { @@ -379,7 +384,7 @@ public final class SynonymQuery extends Query { iterator = impactsDisi; } - return new SynonymScorer(queue, iterator, impactsDisi, simScorer); + return new SynonymScorer(queue, iterator, impactsDisi, simWeight, norms); } } @@ -575,18 +580,21 @@ public final class SynonymQuery extends Query { private final DocIdSetIterator iterator; private final MaxScoreCache maxScoreCache; private final ImpactsDISI impactsDisi; - private final LeafSimScorer simScorer; + private final SimScorer scorer; + private final NumericDocValues norms; SynonymScorer( DisiPriorityQueue queue, DocIdSetIterator iterator, ImpactsDISI impactsDisi, - LeafSimScorer simScorer) { + SimScorer scorer, + NumericDocValues norms) { this.queue = queue; this.iterator = iterator; this.maxScoreCache = impactsDisi.getMaxScoreCache(); this.impactsDisi = impactsDisi; - this.simScorer = simScorer; + this.scorer = scorer; + this.norms = norms; } @Override @@ -605,7 +613,11 @@ public final class SynonymQuery extends Query { @Override public float score() throws IOException { - return simScorer.score(iterator.docID(), freq()); + long norm = 1L; + if (norms != null && norms.advanceExact(iterator.docID())) { + norm = norms.longValue(); + } + return scorer.score(freq(), norm); } @Override @@ -647,9 +659,11 @@ public final class SynonymQuery extends Query { private static class FreqBoostTermScorer extends FilterScorer { final float boost; final TermScorer in; - final LeafSimScorer docScorer; + final SimScorer scorer; + final NumericDocValues norms; - public FreqBoostTermScorer(float boost, TermScorer in, LeafSimScorer docScorer) { + public FreqBoostTermScorer( + float boost, TermScorer in, SimScorer scorer, NumericDocValues norms) { super(in); if (Float.isNaN(boost) || Float.compare(boost, 0f) < 0 || Float.compare(boost, 1f) > 0) { throw new IllegalArgumentException( @@ -657,7 +671,8 @@ public final class SynonymQuery extends Query { } this.boost = boost; this.in = in; - this.docScorer = docScorer; + this.scorer = scorer; + this.norms = norms; } float freq() throws IOException { @@ -666,8 +681,11 @@ public final class SynonymQuery extends Query { @Override public float score() throws IOException { - assert docID() != DocIdSetIterator.NO_MORE_DOCS; - return docScorer.score(in.docID(), freq()); + long norm = 1L; + if (norms != null && norms.advanceExact(in.docID())) { + norm = norms.longValue(); + } + return scorer.score(freq(), norm); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java index 3a843addcc3..814f74d8e78 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java @@ -22,6 +22,7 @@ import java.util.Objects; import org.apache.lucene.index.IndexReaderContext; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.ReaderUtil; import org.apache.lucene.index.Term; @@ -150,19 +151,17 @@ public class TermQuery extends Query { return new ConstantScoreScorer(0f, scoreMode, DocIdSetIterator.empty()); } - LeafSimScorer scorer = - new LeafSimScorer(simScorer, context.reader(), term.field(), scoreMode.needsScores()); + NumericDocValues norms = null; + if (scoreMode.needsScores()) { + norms = context.reader().getNormValues(term.field()); + } + if (scoreMode == ScoreMode.TOP_SCORES) { return new TermScorer( - TermWeight.this, - termsEnum.impacts(PostingsEnum.FREQS), - scorer, - topLevelScoringClause); + termsEnum.impacts(PostingsEnum.FREQS), simScorer, norms, topLevelScoringClause); } else { - return new TermScorer( - termsEnum.postings( - null, scoreMode.needsScores() ? PostingsEnum.FREQS : PostingsEnum.NONE), - scorer); + int flags = scoreMode.needsScores() ? PostingsEnum.FREQS : PostingsEnum.NONE; + return new TermScorer(termsEnum.postings(null, flags), simScorer, norms); } } @@ -223,11 +222,14 @@ public class TermQuery extends Query { int newDoc = scorer.iterator().advance(doc); if (newDoc == doc) { float freq = ((TermScorer) scorer).freq(); - LeafSimScorer docScorer = - new LeafSimScorer(simScorer, context.reader(), term.field(), true); + NumericDocValues norms = context.reader().getNormValues(term.field()); + long norm = 1L; + if (norms != null && norms.advanceExact(doc)) { + norm = norms.longValue(); + } Explanation freqExplanation = Explanation.match(freq, "freq, occurrences of term within document"); - Explanation scoreExplanation = docScorer.explain(doc, freqExplanation); + Explanation scoreExplanation = simScorer.explain(freqExplanation, norm); return Explanation.match( scoreExplanation.getValue(), "weight(" diff --git a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java index 22c6102b958..4b53788f233 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java @@ -18,8 +18,10 @@ package org.apache.lucene.search; import java.io.IOException; import org.apache.lucene.index.ImpactsEnum; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.SlowImpactsEnum; +import org.apache.lucene.search.similarities.Similarity.SimScorer; /** * Expert: A Scorer for documents matching a Term. @@ -29,17 +31,19 @@ import org.apache.lucene.index.SlowImpactsEnum; public final class TermScorer extends Scorer { private final PostingsEnum postingsEnum; private final DocIdSetIterator iterator; - private final LeafSimScorer docScorer; + private final SimScorer scorer; + private final NumericDocValues norms; private final ImpactsDISI impactsDisi; private final MaxScoreCache maxScoreCache; /** Construct a {@link TermScorer} that will iterate all documents. */ - public TermScorer(PostingsEnum postingsEnum, LeafSimScorer docScorer) { + public TermScorer(PostingsEnum postingsEnum, SimScorer scorer, NumericDocValues norms) { iterator = this.postingsEnum = postingsEnum; ImpactsEnum impactsEnum = new SlowImpactsEnum(postingsEnum); - maxScoreCache = new MaxScoreCache(impactsEnum, docScorer.getSimScorer()); + maxScoreCache = new MaxScoreCache(impactsEnum, scorer); impactsDisi = null; - this.docScorer = docScorer; + this.scorer = scorer; + this.norms = norms; } /** @@ -47,12 +51,12 @@ public final class TermScorer extends Scorer { * documents. */ public TermScorer( - Weight weight, ImpactsEnum impactsEnum, - LeafSimScorer docScorer, + SimScorer scorer, + NumericDocValues norms, boolean topLevelScoringClause) { postingsEnum = impactsEnum; - maxScoreCache = new MaxScoreCache(impactsEnum, docScorer.getSimScorer()); + maxScoreCache = new MaxScoreCache(impactsEnum, scorer); if (topLevelScoringClause) { impactsDisi = new ImpactsDISI(impactsEnum, maxScoreCache); iterator = impactsDisi; @@ -60,7 +64,8 @@ public final class TermScorer extends Scorer { impactsDisi = null; iterator = impactsEnum; } - this.docScorer = docScorer; + this.scorer = scorer; + this.norms = norms; } @Override @@ -80,13 +85,23 @@ public final class TermScorer extends Scorer { @Override public float score() throws IOException { - assert docID() != DocIdSetIterator.NO_MORE_DOCS; - return docScorer.score(postingsEnum.docID(), postingsEnum.freq()); + var postingsEnum = this.postingsEnum; + var norms = this.norms; + + long norm = 1L; + if (norms != null && norms.advanceExact(postingsEnum.docID())) { + norm = norms.longValue(); + } + return scorer.score(postingsEnum.freq(), norm); } @Override public float smoothingScore(int docId) throws IOException { - return docScorer.score(docId, 0); + long norm = 1L; + if (norms != null && norms.advanceExact(docId)) { + norm = norms.longValue(); + } + return scorer.score(0, norm); } @Override diff --git a/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java b/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java index 96572226c20..e06951df164 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java @@ -29,6 +29,7 @@ import org.apache.lucene.document.SortedSetDocValuesField; import org.apache.lucene.document.StringField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.SortedSetDocValues; import org.apache.lucene.index.Term; import org.apache.lucene.index.TermStates; @@ -345,7 +346,8 @@ public class TestMinShouldMatch2 extends LuceneTestCase { final int maxDoc; final Set ords = new HashSet<>(); - final LeafSimScorer[] sims; + final SimScorer[] sims; + final NumericDocValues norms; final int minNrShouldMatch; double score = Float.NaN; @@ -356,7 +358,7 @@ public class TestMinShouldMatch2 extends LuceneTestCase { this.maxDoc = reader.maxDoc(); BooleanQuery bq = (BooleanQuery) weight.getQuery(); this.minNrShouldMatch = bq.getMinimumNumberShouldMatch(); - this.sims = new LeafSimScorer[(int) dv.getValueCount()]; + this.sims = new SimScorer[(int) dv.getValueCount()]; for (BooleanClause clause : bq.clauses()) { assert !clause.isProhibited(); assert !clause.isRequired(); @@ -366,14 +368,14 @@ public class TestMinShouldMatch2 extends LuceneTestCase { boolean success = ords.add(ord); assert success; // no dups TermStates ts = TermStates.build(searcher, term, true); - SimScorer w = + sims[(int) ord] = weight.similarity.scorer( 1f, searcher.collectionStatistics("field"), searcher.termStatistics(term, ts.docFreq(), ts.totalTermFreq())); - sims[(int) ord] = new LeafSimScorer(w, reader, "field", true); } } + norms = reader.getNormValues("field"); } @Override @@ -409,11 +411,15 @@ public class TestMinShouldMatch2 extends LuceneTestCase { continue; } long ord; + long norm = 1L; + if (norms != null && norms.advanceExact(currentDoc)) { + norm = norms.longValue(); + } for (int i = 0; i < dv.docValueCount(); i++) { ord = dv.nextOrd(); if (ords.contains(ord)) { currentMatched++; - score += sims[(int) ord].score(currentDoc, 1); + score += sims[(int) ord].score(1, norm); } } if (currentMatched >= minNrShouldMatch) { diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java b/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java index 3e87a19c20e..4373d77d381 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/NormValueSource.java @@ -19,12 +19,12 @@ package org.apache.lucene.queries.function.valuesource; import java.io.IOException; import java.util.Map; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.queries.function.FunctionValues; import org.apache.lucene.queries.function.ValueSource; import org.apache.lucene.queries.function.docvalues.FloatDocValues; import org.apache.lucene.search.CollectionStatistics; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.TermStatistics; import org.apache.lucene.search.similarities.Similarity.SimScorer; import org.apache.lucene.search.similarities.TFIDFSimilarity; @@ -76,8 +76,7 @@ public class NormValueSource extends ValueSource { 1f, new CollectionStatistics(field, 1, 1, 1, 1), new TermStatistics(new BytesRef("bogus"), 1, 1)); - final LeafSimScorer leafSimScorer = - new LeafSimScorer(simScorer, readerContext.reader(), field, true); + final NumericDocValues norms = readerContext.reader().getNormValues(field); return new FloatDocValues(this) { int lastDocID = -1; @@ -88,7 +87,11 @@ public class NormValueSource extends ValueSource { throw new AssertionError("docs out of order: lastDocID=" + lastDocID + " docID=" + docID); } lastDocID = docID; - return leafSimScorer.score(docID, 1f); + long norm = 1L; + if (norms != null && norms.advanceExact(docID)) { + norm = norms.longValue(); + } + return simScorer.score(1f, norm); } }; } diff --git a/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java index 72d6622b4e3..59e35aa7604 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/payloads/PayloadScoreQuery.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Map; import java.util.Objects; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.Term; import org.apache.lucene.index.TermStates; @@ -32,11 +33,11 @@ import org.apache.lucene.queries.spans.Spans; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryVisitor; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.ScorerSupplier; +import org.apache.lucene.search.similarities.Similarity.SimScorer; import org.apache.lucene.util.BytesRef; /** @@ -190,9 +191,9 @@ public class PayloadScoreQuery extends SpanQuery { if (spans == null) { return null; } - LeafSimScorer docScorer = innerWeight.getSimScorer(context); + NumericDocValues norms = context.reader().getNormValues(field); PayloadSpans payloadSpans = new PayloadSpans(spans, decoder); - final var scorer = new PayloadSpanScorer(payloadSpans, docScorer); + final var scorer = new PayloadSpanScorer(payloadSpans, innerWeight.getSimScorer(), norms); return new DefaultScorerSupplier(scorer); } } @@ -248,8 +249,9 @@ public class PayloadScoreQuery extends SpanQuery { private final PayloadSpans spans; - private PayloadSpanScorer(PayloadSpans spans, LeafSimScorer docScorer) throws IOException { - super(spans, docScorer); + private PayloadSpanScorer(PayloadSpans spans, SimScorer scorer, NumericDocValues norms) + throws IOException { + super(spans, scorer, norms); this.spans = spans; } diff --git a/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java index 6cb2dbd2b0f..5792e106516 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/payloads/SpanPayloadCheckQuery.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.Term; import org.apache.lucene.index.TermStates; @@ -34,7 +35,6 @@ import org.apache.lucene.queries.spans.SpanWeight; import org.apache.lucene.queries.spans.Spans; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryVisitor; import org.apache.lucene.search.ScoreMode; @@ -191,8 +191,8 @@ public class SpanPayloadCheckQuery extends SpanQuery { if (spans == null) { return null; } - final LeafSimScorer docScorer = getSimScorer(context); - final var scorer = new SpanScorer(spans, docScorer); + final NumericDocValues norms = context.reader().getNormValues(field); + final var scorer = new SpanScorer(spans, getSimScorer(), norms); return new DefaultScorerSupplier(scorer); } diff --git a/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanContainingQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanContainingQuery.java index 4752d6bee38..aa7137cba4f 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanContainingQuery.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanContainingQuery.java @@ -20,10 +20,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Map; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.Term; import org.apache.lucene.index.TermStates; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.ScorerSupplier; @@ -144,8 +144,8 @@ public final class SpanContainingQuery extends SpanContainQuery { if (spans == null) { return null; } - final LeafSimScorer docScorer = getSimScorer(context); - final var scorer = new SpanScorer(spans, docScorer); + final NumericDocValues norms = context.reader().getNormValues(field); + final var scorer = new SpanScorer(spans, getSimScorer(), norms); return new DefaultScorerSupplier(scorer); } } diff --git a/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanNearQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanNearQuery.java index 69e1a74c80b..9e7f4a049ed 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanNearQuery.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanNearQuery.java @@ -29,7 +29,6 @@ import org.apache.lucene.index.TermStates; import org.apache.lucene.index.Terms; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryVisitor; import org.apache.lucene.search.ScoreMode; @@ -247,8 +246,8 @@ public class SpanNearQuery extends SpanQuery implements Cloneable { if (spans == null) { return null; } - final LeafSimScorer docScorer = getSimScorer(context); - final var scorer = new SpanScorer(spans, docScorer); + final var scorer = + new SpanScorer(spans, getSimScorer(), context.reader().getNormValues(field)); return new DefaultScorerSupplier(scorer); } } diff --git a/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanScorer.java b/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanScorer.java index 9737ae5308e..686ce7c91b1 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanScorer.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanScorer.java @@ -18,10 +18,11 @@ package org.apache.lucene.queries.spans; import java.io.IOException; import java.util.Objects; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.TwoPhaseIterator; +import org.apache.lucene.search.similarities.Similarity.SimScorer; /** * A basic {@link Scorer} over {@link Spans}. @@ -31,7 +32,8 @@ import org.apache.lucene.search.TwoPhaseIterator; public class SpanScorer extends Scorer { protected final Spans spans; - protected final LeafSimScorer docScorer; + protected final SimScorer scorer; + protected final NumericDocValues norms; /** accumulated sloppy freq (computed in setFreqCurrentDoc) */ private float freq; @@ -39,9 +41,10 @@ public class SpanScorer extends Scorer { private int lastScoredDoc = -1; // last doc we called setFreqCurrentDoc() for /** Sole constructor. */ - public SpanScorer(Spans spans, LeafSimScorer docScorer) { + public SpanScorer(Spans spans, SimScorer scorer, NumericDocValues norms) { this.spans = Objects.requireNonNull(spans); - this.docScorer = docScorer; + this.scorer = scorer; + this.norms = norms; } /** return the Spans for this Scorer * */ @@ -69,8 +72,12 @@ public class SpanScorer extends Scorer { * slop-adjusted {@link #freq}. */ protected float scoreCurrentDoc() throws IOException { - assert docScorer != null : getClass() + " has a null docScorer!"; - return docScorer.score(docID(), freq); + assert scorer != null : getClass() + " has a null docScorer!"; + long norm = 1L; + if (norms != null && norms.advanceExact(docID())) { + norm = norms.longValue(); + } + return scorer.score(freq, norm); } /** @@ -98,7 +105,7 @@ public class SpanScorer extends Scorer { // assert (startPos != prevStartPos) || (endPos > prevEndPos) : "non increased // endPos="+endPos; assert (startPos != prevStartPos) || (endPos >= prevEndPos) : "decreased endPos=" + endPos; - if (docScorer == null) { // scores not required, break out here + if (scorer == null) { // scores not required, break out here freq = 1; return; } diff --git a/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanTermQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanTermQuery.java index 359f6ae7e95..82c0f9b4aee 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanTermQuery.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanTermQuery.java @@ -160,7 +160,7 @@ public class SpanTermQuery extends SpanQuery { final PostingsEnum postings = termsEnum.postings(null, requiredPostings.getRequiredPostings()); float positionsCost = termPositionsCost(termsEnum) * PHRASE_TO_SPAN_TERM_POSITIONS_COST; - return new TermSpans(getSimScorer(context), postings, term, positionsCost); + return new TermSpans(postings, term, positionsCost); } } diff --git a/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanWeight.java b/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanWeight.java index 29e1c13e773..8c7d8dbdb3a 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanWeight.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/spans/SpanWeight.java @@ -22,13 +22,13 @@ import java.util.Comparator; import java.util.Locale; import java.util.Map; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.Term; import org.apache.lucene.index.TermStates; import org.apache.lucene.search.CollectionStatistics; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.Matches; import org.apache.lucene.search.MatchesIterator; import org.apache.lucene.search.MatchesUtils; @@ -38,6 +38,7 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermStatistics; import org.apache.lucene.search.Weight; import org.apache.lucene.search.similarities.Similarity; +import org.apache.lucene.search.similarities.Similarity.SimScorer; import org.apache.lucene.util.ArrayUtil; /** Expert-only. Public for use by other weight implementations */ @@ -142,8 +143,8 @@ public abstract class SpanWeight extends Weight { if (spans == null) { return null; } - final LeafSimScorer docScorer = getSimScorer(context); - final var scorer = new SpanScorer(spans, docScorer); + final NumericDocValues norms = context.reader().getNormValues(field); + final var scorer = new SpanScorer(spans, simScorer, norms); return new ScorerSupplier() { @Override public SpanScorer get(long leadCost) throws IOException { @@ -157,15 +158,9 @@ public abstract class SpanWeight extends Weight { }; } - /** - * Return a LeafSimScorer for this context - * - * @param context the LeafReaderContext - * @return a SimWeight - * @throws IOException on error - */ - public LeafSimScorer getSimScorer(LeafReaderContext context) throws IOException { - return simScorer == null ? null : new LeafSimScorer(simScorer, context.reader(), field, true); + /** Return the SimScorer */ + public SimScorer getSimScorer() { + return simScorer; } @Override @@ -176,9 +171,13 @@ public abstract class SpanWeight extends Weight { if (newDoc == doc) { if (simScorer != null) { float freq = scorer.sloppyFreq(); - LeafSimScorer docScorer = new LeafSimScorer(simScorer, context.reader(), field, true); Explanation freqExplanation = Explanation.match(freq, "phraseFreq=" + freq); - Explanation scoreExplanation = docScorer.explain(doc, freqExplanation); + NumericDocValues norms = context.reader().getNormValues(field); + long norm = 1L; + if (norms != null && norms.advanceExact(doc)) { + norm = norms.longValue(); + } + Explanation scoreExplanation = simScorer.explain(freqExplanation, norm); return Explanation.match( scoreExplanation.getValue(), "weight(" diff --git a/lucene/queries/src/java/org/apache/lucene/queries/spans/TermSpans.java b/lucene/queries/src/java/org/apache/lucene/queries/spans/TermSpans.java index b81b1846a09..995b242d90b 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/spans/TermSpans.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/spans/TermSpans.java @@ -21,7 +21,6 @@ import java.util.Objects; import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.Term; import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.LeafSimScorer; /** * Expert: Public for extension only. This does not work correctly for terms that indexed at @@ -37,7 +36,7 @@ public class TermSpans extends Spans { protected boolean readPayload; private final float positionsCost; - public TermSpans(LeafSimScorer scorer, PostingsEnum postings, Term term, float positionsCost) { + public TermSpans(PostingsEnum postings, Term term, float positionsCost) { this.postings = Objects.requireNonNull(postings); this.term = Objects.requireNonNull(term); this.doc = -1; diff --git a/lucene/queries/src/test/org/apache/lucene/queries/spans/AssertingSpanWeight.java b/lucene/queries/src/test/org/apache/lucene/queries/spans/AssertingSpanWeight.java index 4cbb0f39f72..4e1f16b9a02 100644 --- a/lucene/queries/src/test/org/apache/lucene/queries/spans/AssertingSpanWeight.java +++ b/lucene/queries/src/test/org/apache/lucene/queries/spans/AssertingSpanWeight.java @@ -23,8 +23,8 @@ import org.apache.lucene.index.Term; import org.apache.lucene.index.TermStates; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.ScorerSupplier; +import org.apache.lucene.search.similarities.Similarity.SimScorer; /** Wraps a SpanWeight with additional asserts */ public class AssertingSpanWeight extends SpanWeight { @@ -55,8 +55,8 @@ public class AssertingSpanWeight extends SpanWeight { } @Override - public LeafSimScorer getSimScorer(LeafReaderContext context) throws IOException { - return in.getSimScorer(context); + public SimScorer getSimScorer() { + return in.getSimScorer(); } @Override diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/CombinedFieldQuery.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/CombinedFieldQuery.java index 417119fa9c8..8f21c8e1850 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/CombinedFieldQuery.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/CombinedFieldQuery.java @@ -45,7 +45,6 @@ import org.apache.lucene.search.DisjunctionDISIApproximation; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.Matches; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryVisitor; @@ -402,14 +401,12 @@ public final class CombinedFieldQuery extends Query implements Accountable { MultiNormsLeafSimScorer scoringSimScorer = new MultiNormsLeafSimScorer(simWeight, context.reader(), fieldAndWeights.values(), true); - LeafSimScorer nonScoringSimScorer = - new LeafSimScorer(simWeight, context.reader(), "pseudo_field", false); // we use termscorers + disjunction as an impl detail DisiPriorityQueue queue = new DisiPriorityQueue(iterators.size()); for (int i = 0; i < iterators.size(); i++) { float weight = fields.get(i).weight; queue.add( - new WeightedDisiWrapper(new TermScorer(iterators.get(i), nonScoringSimScorer), weight)); + new WeightedDisiWrapper(new TermScorer(iterators.get(i), simWeight, null), weight)); } // Even though it is called approximation, it is accurate since none of // the sub iterators are two-phase iterators. diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/MultiNormsLeafSimScorer.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/MultiNormsLeafSimScorer.java index 1454e1cdaaa..026fa4130d3 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/MultiNormsLeafSimScorer.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/MultiNormsLeafSimScorer.java @@ -16,8 +16,6 @@ */ package org.apache.lucene.sandbox.search; -import static org.apache.lucene.sandbox.search.CombinedFieldQuery.FieldAndWeight; - import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -27,13 +25,13 @@ import java.util.Objects; import java.util.Set; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.sandbox.search.CombinedFieldQuery.FieldAndWeight; import org.apache.lucene.search.Explanation; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.similarities.Similarity.SimScorer; import org.apache.lucene.util.SmallFloat; /** - * Copy of {@link LeafSimScorer} that sums document's norms from multiple fields. + * Scorer that sums document's norms from multiple fields. * *

For all fields, norms must be encoded using {@link SmallFloat#intToByte4}. This scorer also * requires that either all fields or no fields have norms enabled. Having only some fields with diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/TermAutomatonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/TermAutomatonQuery.java index 46386b52249..1ec26113960 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/TermAutomatonQuery.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/TermAutomatonQuery.java @@ -24,6 +24,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.ReaderUtil; import org.apache.lucene.index.Term; @@ -35,7 +36,6 @@ import org.apache.lucene.queries.spans.SpanNearQuery; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.MultiPhraseQuery; import org.apache.lucene.search.PhraseQuery; @@ -429,9 +429,8 @@ public class TermAutomatonQuery extends Query implements Accountable { } if (any) { - scorer = - new TermAutomatonScorer( - this, enums, anyTermID, new LeafSimScorer(stats, context.reader(), field, true)); + NumericDocValues norms = context.reader().getNormValues(field); + scorer = new TermAutomatonScorer(this, enums, anyTermID, stats, norms); } else { return null; } @@ -456,15 +455,20 @@ public class TermAutomatonQuery extends Query implements Accountable { } float score = scorer.score(); - LeafSimScorer leafSimScorer = ((TermAutomatonScorer) scorer).getLeafSimScorer(); EnumAndScorer[] originalSubsOnDoc = ((TermAutomatonScorer) scorer).getOriginalSubsOnDoc(); + NumericDocValues norms = context.reader().getNormValues(field); + long norm = 1L; + if (norms != null && norms.advanceExact(doc)) { + norm = norms.longValue(); + } + List termExplanations = new ArrayList<>(); for (EnumAndScorer enumAndScorer : originalSubsOnDoc) { if (enumAndScorer != null) { PostingsEnum postingsEnum = enumAndScorer.posEnum; if (postingsEnum.docID() == doc) { - float termScore = leafSimScorer.score(doc, postingsEnum.freq()); + float termScore = stats.score(postingsEnum.freq(), norm); termExplanations.add( Explanation.match( postingsEnum.freq(), @@ -482,7 +486,7 @@ public class TermAutomatonQuery extends Query implements Accountable { Explanation freqExplanation = Explanation.match(score, "TermAutomatonQuery, sum of:", termExplanations); - return leafSimScorer.explain(doc, freqExplanation); + return stats.explain(freqExplanation, norm); } } diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/TermAutomatonScorer.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/TermAutomatonScorer.java index 2d0b7ecb85b..7c6d6c9b6bb 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/TermAutomatonScorer.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/TermAutomatonScorer.java @@ -17,11 +17,12 @@ package org.apache.lucene.sandbox.search; import java.io.IOException; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.sandbox.search.TermAutomatonQuery.EnumAndScorer; import org.apache.lucene.sandbox.search.TermAutomatonQuery.TermAutomatonWeight; import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.LeafSimScorer; import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.similarities.Similarity.SimScorer; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.PriorityQueue; import org.apache.lucene.util.RamUsageEstimator; @@ -44,7 +45,8 @@ class TermAutomatonScorer extends Scorer { // This is -1 if wildcard (null) terms were not used, else it's the id // of the wildcard term: private final int anyTermID; - private final LeafSimScorer docScorer; + private final SimScorer scorer; + private final NumericDocValues norms; private int numSubsOnDoc; @@ -61,11 +63,16 @@ class TermAutomatonScorer extends Scorer { private final EnumAndScorer[] originalSubsOnDoc; public TermAutomatonScorer( - TermAutomatonWeight weight, EnumAndScorer[] subs, int anyTermID, LeafSimScorer docScorer) + TermAutomatonWeight weight, + EnumAndScorer[] subs, + int anyTermID, + SimScorer scorer, + NumericDocValues norms) throws IOException { // System.out.println(" automaton:\n" + weight.automaton.toDot()); this.runAutomaton = new TermRunAutomaton(weight.automaton, subs.length); - this.docScorer = docScorer; + this.scorer = scorer; + this.norms = norms; this.docIDQueue = new DocIDQueue(subs.length); this.posQueue = new PositionQueue(subs.length); this.anyTermID = anyTermID; @@ -356,10 +363,6 @@ class TermAutomatonScorer extends Scorer { return originalSubsOnDoc; } - LeafSimScorer getLeafSimScorer() { - return docScorer; - } - @Override public int docID() { return docID; @@ -369,12 +372,16 @@ class TermAutomatonScorer extends Scorer { public float score() throws IOException { // TODO: we could probably do better here, e.g. look @ freqs of actual terms involved in this // doc and score differently - return docScorer.score(docID, freq); + long norm = 1L; + if (norms != null && norms.advanceExact(docID)) { + norm = norms.longValue(); + } + return scorer.score(freq, norm); } @Override public float getMaxScore(int upTo) throws IOException { - return docScorer.getSimScorer().score(Float.MAX_VALUE, 1L); + return scorer.score(Float.MAX_VALUE, 1L); } static class TermRunAutomaton extends RunAutomaton { From 81ab3b9722461c24ea031fea0dc7265f85bb7b66 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Sat, 26 Oct 2024 13:52:55 +0200 Subject: [PATCH 17/37] Move CHANGES entry to correct version. --- lucene/CHANGES.txt | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 360e47079fe..079707feae7 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -9,9 +9,6 @@ API Changes --------------------- * GITHUB#11023: Removing deprecated parameters from CheckIndex. (Jakub Slowinski) -* GITHUB#13957: Removed LeafSimScorer class, to save its overhead. Scorers now - compute scores directly from a SimScorer, postings and norms. (Adrien Grand) - New Features --------------------- (No changes) @@ -41,6 +38,8 @@ API Changes * GITHUB#13950: Make BooleanQuery#getClauses public and add #add(Collection) to BQ builder. (Shubham Chaudhary) +* GITHUB#13957: Removed LeafSimScorer class, to save its overhead. Scorers now + compute scores directly from a SimScorer, postings and norms. (Adrien Grand) New Features --------------------- From 937432acd8d0483604fc144bc97e423dd2df7156 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 28 Oct 2024 15:50:53 +0100 Subject: [PATCH 18/37] Remove HitsThresholdChecker. (#13943) `TopScoreDocCollectorManager` has a dependency on `HitsThresholdChecker`, which is essentially a shared counter that is incremented until it reaches the total hits threshold, when the scorer can start dynamically pruning hits. A consequence of this removal is that dynamic pruning may start later, as soon as: - either the current slice collected `totalHitsThreshold` hits, - or another slice collected `totalHitsThreshold` hits and the current slice collected enough hits (up to 1,024) to check the shared `MaxScoreAccumulator`. So in short, it exchanges a bit more work globally in favor of a bit less contention. A longer-term goal of mine is to stop specializing our `CollectorManager`s based on whether they are going to be used concurrently or not. --- lucene/CHANGES.txt | 2 + .../lucene/search/HitsThresholdChecker.java | 147 ------------------ .../lucene/search/TopFieldCollector.java | 37 +++-- .../search/TopFieldCollectorManager.java | 11 +- .../lucene/search/TopScoreDocCollector.java | 32 ++-- .../search/TopScoreDocCollectorManager.java | 11 +- .../lucene/search/TestTopDocsCollector.java | 14 +- .../lucene/search/TestTopFieldCollector.java | 14 +- 8 files changed, 52 insertions(+), 216 deletions(-) delete mode 100644 lucene/core/src/java/org/apache/lucene/search/HitsThresholdChecker.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 079707feae7..4e300574fa5 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -72,6 +72,8 @@ Optimizations * GITHUB#13899: Check ahead if we can get the count. (Lu Xugang) +* GITHUB#13943: Removed shared `HitsThresholdChecker`, which reduces overhead + but may delay a bit when dynamic pruning kicks in. (Adrien Grand) Bug Fixes --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/HitsThresholdChecker.java b/lucene/core/src/java/org/apache/lucene/search/HitsThresholdChecker.java deleted file mode 100644 index 78e1a589a77..00000000000 --- a/lucene/core/src/java/org/apache/lucene/search/HitsThresholdChecker.java +++ /dev/null @@ -1,147 +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.search; - -import java.util.concurrent.atomic.LongAdder; - -/** Used for defining custom algorithms to allow searches to early terminate */ -abstract class HitsThresholdChecker { - /** Implementation of HitsThresholdChecker which allows global hit counting */ - private static class GlobalHitsThresholdChecker extends HitsThresholdChecker { - private final LongAdder globalHitCount = new LongAdder(); - // Cache whether the threshold has been reached already. It is not volatile or synchronized on - // purpose to contain the overhead of reading the value similarly to what String#hashCode() - // does. This does not affect correctness. - private boolean thresholdReached = false; - - GlobalHitsThresholdChecker(int totalHitsThreshold) { - super(totalHitsThreshold); - assert totalHitsThreshold != Integer.MAX_VALUE; - } - - @Override - void incrementHitCount() { - if (thresholdReached == false) { - globalHitCount.increment(); - } - } - - @Override - boolean isThresholdReached() { - if (thresholdReached) { - return true; - } - if (globalHitCount.longValue() > getHitsThreshold()) { - thresholdReached = true; - return true; - } - return false; - } - - @Override - ScoreMode scoreMode() { - return ScoreMode.TOP_SCORES; - } - } - - /** Default implementation of HitsThresholdChecker to be used for single threaded execution */ - private static class LocalHitsThresholdChecker extends HitsThresholdChecker { - private int hitCount; - - LocalHitsThresholdChecker(int totalHitsThreshold) { - super(totalHitsThreshold); - assert totalHitsThreshold != Integer.MAX_VALUE; - } - - @Override - void incrementHitCount() { - ++hitCount; - } - - @Override - boolean isThresholdReached() { - return hitCount > getHitsThreshold(); - } - - @Override - ScoreMode scoreMode() { - return ScoreMode.TOP_SCORES; - } - } - - /** - * No-op implementation of {@link HitsThresholdChecker} that does no counting, as the threshold - * can never be reached. This is useful for cases where early termination is never desired, so - * that the overhead of counting hits can be avoided. - */ - private static final HitsThresholdChecker EXACT_HITS_COUNT_THRESHOLD_CHECKER = - new HitsThresholdChecker(Integer.MAX_VALUE) { - @Override - void incrementHitCount() { - // noop - } - - @Override - boolean isThresholdReached() { - return false; - } - - @Override - ScoreMode scoreMode() { - return ScoreMode.COMPLETE; - } - }; - - /* - * Returns a threshold checker that is useful for single threaded searches - */ - static HitsThresholdChecker create(final int totalHitsThreshold) { - return totalHitsThreshold == Integer.MAX_VALUE - ? HitsThresholdChecker.EXACT_HITS_COUNT_THRESHOLD_CHECKER - : new LocalHitsThresholdChecker(totalHitsThreshold); - } - - /* - * Returns a threshold checker that is based on a shared counter - */ - static HitsThresholdChecker createShared(final int totalHitsThreshold) { - return totalHitsThreshold == Integer.MAX_VALUE - ? HitsThresholdChecker.EXACT_HITS_COUNT_THRESHOLD_CHECKER - : new GlobalHitsThresholdChecker(totalHitsThreshold); - } - - private final int totalHitsThreshold; - - HitsThresholdChecker(int totalHitsThreshold) { - if (totalHitsThreshold < 0) { - throw new IllegalArgumentException( - "totalHitsThreshold must be >= 0, got " + totalHitsThreshold); - } - this.totalHitsThreshold = totalHitsThreshold; - } - - final int getHitsThreshold() { - return totalHitsThreshold; - } - - abstract boolean isThresholdReached(); - - abstract ScoreMode scoreMode(); - - abstract void incrementHitCount(); -} diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java index eac31bf89d0..384f5fa1168 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java @@ -71,15 +71,14 @@ public abstract class TopFieldCollector extends TopDocsCollector { } void countHit(int doc) throws IOException { - ++totalHits; - hitsThresholdChecker.incrementHitCount(); + int hitCountSoFar = ++totalHits; - if (minScoreAcc != null && (totalHits & minScoreAcc.modInterval) == 0) { + if (minScoreAcc != null && (hitCountSoFar & minScoreAcc.modInterval) == 0) { updateGlobalMinCompetitiveScore(scorer); } if (scoreMode.isExhaustive() == false && totalHitsRelation == TotalHits.Relation.EQUAL_TO - && hitsThresholdChecker.isThresholdReached()) { + && totalHits > totalHitsThreshold) { // for the first time hitsThreshold is reached, notify comparator about this comparator.setHitsThresholdReached(); totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO; @@ -92,7 +91,7 @@ public abstract class TopFieldCollector extends TopDocsCollector { // this document is larger than anything else in the queue, and // therefore not competitive. if (searchSortPartOfIndexSort) { - if (hitsThresholdChecker.isThresholdReached()) { + if (totalHits > totalHitsThreshold) { totalHitsRelation = Relation.GREATER_THAN_OR_EQUAL_TO; throw new CollectionTerminatedException(); } else { @@ -180,9 +179,9 @@ public abstract class TopFieldCollector extends TopDocsCollector { Sort sort, FieldValueHitQueue queue, int numHits, - HitsThresholdChecker hitsThresholdChecker, + int totalHitsThreshold, MaxScoreAccumulator minScoreAcc) { - super(queue, numHits, hitsThresholdChecker, sort.needsScores(), minScoreAcc); + super(queue, numHits, totalHitsThreshold, sort.needsScores(), minScoreAcc); this.sort = sort; this.queue = queue; } @@ -235,9 +234,9 @@ public abstract class TopFieldCollector extends TopDocsCollector { FieldValueHitQueue queue, FieldDoc after, int numHits, - HitsThresholdChecker hitsThresholdChecker, + int totalHitsThreshold, MaxScoreAccumulator minScoreAcc) { - super(queue, numHits, hitsThresholdChecker, sort.needsScores(), minScoreAcc); + super(queue, numHits, totalHitsThreshold, sort.needsScores(), minScoreAcc); this.sort = sort; this.queue = queue; this.after = after; @@ -301,7 +300,7 @@ public abstract class TopFieldCollector extends TopDocsCollector { private static final ScoreDoc[] EMPTY_SCOREDOCS = new ScoreDoc[0]; final int numHits; - final HitsThresholdChecker hitsThresholdChecker; + final int totalHitsThreshold; final FieldComparator firstComparator; final boolean canSetMinScore; @@ -327,25 +326,25 @@ public abstract class TopFieldCollector extends TopDocsCollector { private TopFieldCollector( FieldValueHitQueue pq, int numHits, - HitsThresholdChecker hitsThresholdChecker, + int totalHitsThreshold, boolean needsScores, MaxScoreAccumulator minScoreAcc) { super(pq); this.needsScores = needsScores; this.numHits = numHits; - this.hitsThresholdChecker = hitsThresholdChecker; + this.totalHitsThreshold = Math.max(totalHitsThreshold, numHits); this.numComparators = pq.getComparators().length; this.firstComparator = pq.getComparators()[0]; int reverseMul = pq.reverseMul[0]; if (firstComparator.getClass().equals(FieldComparator.RelevanceComparator.class) && reverseMul == 1 // if the natural sort is preserved (sort by descending relevance) - && hitsThresholdChecker.getHitsThreshold() != Integer.MAX_VALUE) { + && totalHitsThreshold != Integer.MAX_VALUE) { scoreMode = ScoreMode.TOP_SCORES; canSetMinScore = true; } else { canSetMinScore = false; - if (hitsThresholdChecker.getHitsThreshold() != Integer.MAX_VALUE) { + if (totalHitsThreshold != Integer.MAX_VALUE) { scoreMode = needsScores ? ScoreMode.TOP_DOCS_WITH_SCORES : ScoreMode.TOP_DOCS; } else { scoreMode = needsScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES; @@ -361,10 +360,10 @@ public abstract class TopFieldCollector extends TopDocsCollector { protected void updateGlobalMinCompetitiveScore(Scorable scorer) throws IOException { assert minScoreAcc != null; - if (canSetMinScore && hitsThresholdChecker.isThresholdReached()) { - // we can start checking the global maximum score even - // if the local queue is not full because the threshold - // is reached. + if (canSetMinScore) { + // we can start checking the global maximum score even if the local queue is not full or if + // the threshold is not reached on the local competitor: the fact that there is a shared min + // competitive score implies that one of the collectors hit its totalHitsThreshold already long maxMinScore = minScoreAcc.getRaw(); float score; if (maxMinScore != Long.MIN_VALUE @@ -377,7 +376,7 @@ public abstract class TopFieldCollector extends TopDocsCollector { } protected void updateMinCompetitiveScore(Scorable scorer) throws IOException { - if (canSetMinScore && queueFull && hitsThresholdChecker.isThresholdReached()) { + if (canSetMinScore && queueFull && totalHits > totalHitsThreshold) { assert bottom != null; float minScore = (float) firstComparator.value(bottom.slot); if (minScore > minCompetitiveScore) { diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java index 2d4a0c0e215..48c1ca98091 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java @@ -32,7 +32,7 @@ public class TopFieldCollectorManager implements CollectorManager collectors; private final boolean supportsConcurrency; @@ -89,10 +89,7 @@ public class TopFieldCollectorManager implements CollectorManager { static class SimpleTopScoreDocCollector extends TopScoreDocCollector { SimpleTopScoreDocCollector( - int numHits, HitsThresholdChecker hitsThresholdChecker, MaxScoreAccumulator minScoreAcc) { - super(numHits, hitsThresholdChecker, minScoreAcc); + int numHits, int totalHitsThreshold, MaxScoreAccumulator minScoreAcc) { + super(numHits, totalHitsThreshold, minScoreAcc); } @Override @@ -71,7 +71,6 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { float score = scorer.score(); int hitCountSoFar = ++totalHits; - hitsThresholdChecker.incrementHitCount(); if (minScoreAcc != null && (hitCountSoFar & minScoreAcc.modInterval) == 0) { updateGlobalMinCompetitiveScore(scorer); @@ -80,7 +79,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { if (score <= pqTop.score) { // Note: for queries that match lots of hits, this is the common case: most hits are not // competitive. - if (totalHitsRelation == TotalHits.Relation.EQUAL_TO) { + if (hitCountSoFar == totalHitsThreshold + 1) { // we just reached totalHitsThreshold, we can start setting the min // competitive score now updateMinCompetitiveScore(scorer); @@ -108,11 +107,8 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { private final ScoreDoc after; PagingTopScoreDocCollector( - int numHits, - ScoreDoc after, - HitsThresholdChecker hitsThresholdChecker, - MaxScoreAccumulator minScoreAcc) { - super(numHits, hitsThresholdChecker, minScoreAcc); + int numHits, ScoreDoc after, int totalHitsThreshold, MaxScoreAccumulator minScoreAcc) { + super(numHits, totalHitsThreshold, minScoreAcc); this.after = after; } @@ -158,7 +154,6 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { float score = scorer.score(); int hitCountSoFar = ++totalHits; - hitsThresholdChecker.incrementHitCount(); if (minScoreAcc != null && (hitCountSoFar & minScoreAcc.modInterval) == 0) { updateGlobalMinCompetitiveScore(scorer); @@ -178,8 +173,8 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { if (score <= pqTop.score) { // Note: for queries that match lots of hits, this is the common case: most hits are not // competitive. - if (totalHitsRelation == TotalHits.Relation.EQUAL_TO) { - // we just reached totalHitsThreshold, we can start setting the min + if (hitCountSoFar == totalHitsThreshold + 1) { + // we just exceeded totalHitsThreshold, we can start setting the min // competitive score now updateMinCompetitiveScore(scorer); } @@ -204,20 +199,18 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { int docBase; ScoreDoc pqTop; - final HitsThresholdChecker hitsThresholdChecker; + final int totalHitsThreshold; final MaxScoreAccumulator minScoreAcc; float minCompetitiveScore; // prevents instantiation - TopScoreDocCollector( - int numHits, HitsThresholdChecker hitsThresholdChecker, MaxScoreAccumulator minScoreAcc) { + TopScoreDocCollector(int numHits, int totalHitsThreshold, MaxScoreAccumulator minScoreAcc) { super(new HitQueue(numHits, true)); - assert hitsThresholdChecker != null; // HitQueue implements getSentinelObject to return a ScoreDoc, so we know // that at this point top() is already initialized. pqTop = pq.top(); - this.hitsThresholdChecker = hitsThresholdChecker; + this.totalHitsThreshold = totalHitsThreshold; this.minScoreAcc = minScoreAcc; } @@ -232,7 +225,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { @Override public ScoreMode scoreMode() { - return hitsThresholdChecker.scoreMode(); + return totalHitsThreshold == Integer.MAX_VALUE ? ScoreMode.COMPLETE : ScoreMode.TOP_SCORES; } protected void updateGlobalMinCompetitiveScore(Scorable scorer) throws IOException { @@ -245,7 +238,6 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { float score = MaxScoreAccumulator.toScore(maxMinScore); score = docBase >= MaxScoreAccumulator.docId(maxMinScore) ? Math.nextUp(score) : score; if (score > minCompetitiveScore) { - assert hitsThresholdChecker.isThresholdReached(); scorer.setMinCompetitiveScore(score); minCompetitiveScore = score; totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO; @@ -254,7 +246,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { } protected void updateMinCompetitiveScore(Scorable scorer) throws IOException { - if (hitsThresholdChecker.isThresholdReached()) { + if (totalHits > totalHitsThreshold) { // since we tie-break on doc id and collect in doc id order, we can require // the next float // pqTop is never null since TopScoreDocCollector fills the priority queue with sentinel diff --git a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java index 94a4e1a82d3..7933313febe 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java @@ -29,7 +29,7 @@ public class TopScoreDocCollectorManager implements CollectorManager { private final int numHits; private final ScoreDoc after; - private final HitsThresholdChecker hitsThresholdChecker; + private final int totalHitsThreshold; private final MaxScoreAccumulator minScoreAcc; private final boolean supportsConcurrency; private boolean collectorCreated; @@ -71,10 +71,7 @@ public class TopScoreDocCollectorManager this.numHits = numHits; this.after = after; this.supportsConcurrency = supportsConcurrency; - this.hitsThresholdChecker = - supportsConcurrency - ? HitsThresholdChecker.createShared(Math.max(totalHitsThreshold, numHits)) - : HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits)); + this.totalHitsThreshold = Math.max(totalHitsThreshold, numHits); this.minScoreAcc = supportsConcurrency && totalHitsThreshold != Integer.MAX_VALUE ? new MaxScoreAccumulator() @@ -141,10 +138,10 @@ public class TopScoreDocCollectorManager if (after == null) { return new TopScoreDocCollector.SimpleTopScoreDocCollector( - numHits, hitsThresholdChecker, minScoreAcc); + numHits, totalHitsThreshold, minScoreAcc); } else { return new TopScoreDocCollector.PagingTopScoreDocCollector( - numHits, after, hitsThresholdChecker, minScoreAcc); + numHits, after, totalHitsThreshold, minScoreAcc); } } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java index 14b51ca214e..44bd66148cd 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java @@ -529,26 +529,24 @@ public class TestTopDocsCollector extends LuceneTestCase { scorer.score = 2; leafCollector.collect(1); - assertEquals(2f, MaxScoreAccumulator.toScore(minValueChecker.getRaw()), 0f); - assertEquals(Math.nextUp(2f), scorer.minCompetitiveScore, 0f); - assertNull(scorer2.minCompetitiveScore); + assertEquals(Long.MIN_VALUE, minValueChecker.getRaw()); + assertNull(scorer.minCompetitiveScore); scorer2.score = 9; leafCollector2.collect(1); - assertEquals(6f, MaxScoreAccumulator.toScore(minValueChecker.getRaw()), 0f); - assertEquals(Math.nextUp(2f), scorer.minCompetitiveScore, 0f); - assertEquals(Math.nextUp(6f), scorer2.minCompetitiveScore, 0f); + assertEquals(Long.MIN_VALUE, minValueChecker.getRaw()); + assertNull(scorer2.minCompetitiveScore); scorer2.score = 7; leafCollector2.collect(2); assertEquals(MaxScoreAccumulator.toScore(minValueChecker.getRaw()), 7f, 0f); - assertEquals(Math.nextUp(2f), scorer.minCompetitiveScore, 0f); + assertNull(scorer.minCompetitiveScore); assertEquals(Math.nextUp(7f), scorer2.minCompetitiveScore, 0f); scorer2.score = 1; leafCollector2.collect(3); assertEquals(MaxScoreAccumulator.toScore(minValueChecker.getRaw()), 7f, 0f); - assertEquals(Math.nextUp(2f), scorer.minCompetitiveScore, 0f); + assertNull(scorer.minCompetitiveScore); assertEquals(Math.nextUp(7f), scorer2.minCompetitiveScore, 0f); scorer.score = 10; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java index c507eb0f647..bd0d408e0b2 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java @@ -587,26 +587,24 @@ public class TestTopFieldCollector extends LuceneTestCase { scorer.score = 2; leafCollector.collect(1); - assertEquals(2f, MaxScoreAccumulator.toScore(minValueChecker.getRaw()), 0f); - assertEquals(2f, scorer.minCompetitiveScore, 0f); - assertNull(scorer2.minCompetitiveScore); + assertEquals(Long.MIN_VALUE, minValueChecker.getRaw()); + assertNull(scorer.minCompetitiveScore); scorer2.score = 9; leafCollector2.collect(1); - assertEquals(6f, MaxScoreAccumulator.toScore(minValueChecker.getRaw()), 0f); - assertEquals(2f, scorer.minCompetitiveScore, 0f); - assertEquals(6f, scorer2.minCompetitiveScore, 0f); + assertEquals(Long.MIN_VALUE, minValueChecker.getRaw()); + assertNull(scorer2.minCompetitiveScore); scorer2.score = 7; leafCollector2.collect(2); assertEquals(7f, MaxScoreAccumulator.toScore(minValueChecker.getRaw()), 0f); - assertEquals(2f, scorer.minCompetitiveScore, 0f); + assertNull(scorer.minCompetitiveScore); assertEquals(7f, scorer2.minCompetitiveScore, 0f); scorer2.score = 1; leafCollector2.collect(3); assertEquals(7f, MaxScoreAccumulator.toScore(minValueChecker.getRaw()), 0f); - assertEquals(2f, scorer.minCompetitiveScore, 0f); + assertNull(scorer.minCompetitiveScore); assertEquals(7f, scorer2.minCompetitiveScore, 0f); scorer.score = 10; From 60ddd08c95776f11c70057c19463c0709b1ce7a2 Mon Sep 17 00:00:00 2001 From: panguixin Date: Tue, 29 Oct 2024 23:30:15 +0800 Subject: [PATCH 19/37] replace Map with IntObjectHashMap for DV producer (#13961) --- lucene/CHANGES.txt | 2 + .../lucene80/Lucene80DocValuesProducer.java | 33 +++++----- .../lucene90/Lucene90DocValuesProducer.java | 63 +++++++++---------- .../perfield/PerFieldDocValuesFormat.java | 23 +++---- .../index/SegmentDocValuesProducer.java | 21 +++---- 5 files changed, 71 insertions(+), 71 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 4e300574fa5..7146a97195e 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -75,6 +75,8 @@ Optimizations * GITHUB#13943: Removed shared `HitsThresholdChecker`, which reduces overhead but may delay a bit when dynamic pruning kicks in. (Adrien Grand) +* GITHUB#13961: Replace Map with IntObjectHashMap for DV producer. (Pan Guixin) + Bug Fixes --------------------- * GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesProducer.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesProducer.java index c5754e5d1e5..211267d4c03 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesProducer.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene80/Lucene80DocValuesProducer.java @@ -17,8 +17,6 @@ package org.apache.lucene.backward_codecs.lucene80; import java.io.IOException; -import java.util.HashMap; -import java.util.Map; import org.apache.lucene.backward_codecs.packed.LegacyDirectMonotonicReader; import org.apache.lucene.backward_codecs.packed.LegacyDirectReader; import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil; @@ -41,6 +39,7 @@ import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.SortedSetDocValues; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.TermsEnum.SeekStatus; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.store.ByteArrayDataInput; import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.DataInput; @@ -53,11 +52,11 @@ import org.apache.lucene.util.compress.LZ4; /** reader for {@link Lucene80DocValuesFormat} */ final class Lucene80DocValuesProducer extends DocValuesProducer { - private final Map numerics = new HashMap<>(); - private final Map binaries = new HashMap<>(); - private final Map sorted = new HashMap<>(); - private final Map sortedSets = new HashMap<>(); - private final Map sortedNumerics = new HashMap<>(); + private final IntObjectHashMap numerics = new IntObjectHashMap<>(); + private final IntObjectHashMap binaries = new IntObjectHashMap<>(); + private final IntObjectHashMap sorted = new IntObjectHashMap<>(); + private final IntObjectHashMap sortedSets = new IntObjectHashMap<>(); + private final IntObjectHashMap sortedNumerics = new IntObjectHashMap<>(); private final IndexInput data; private final int maxDoc; private int version = -1; @@ -139,7 +138,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer { } byte type = meta.readByte(); if (type == Lucene80DocValuesFormat.NUMERIC) { - numerics.put(info.name, readNumeric(meta)); + numerics.put(info.number, readNumeric(meta)); } else if (type == Lucene80DocValuesFormat.BINARY) { final boolean compressed; if (version >= Lucene80DocValuesFormat.VERSION_CONFIGURABLE_COMPRESSION) { @@ -158,13 +157,13 @@ final class Lucene80DocValuesProducer extends DocValuesProducer { } else { compressed = version >= Lucene80DocValuesFormat.VERSION_BIN_COMPRESSED; } - binaries.put(info.name, readBinary(meta, compressed)); + binaries.put(info.number, readBinary(meta, compressed)); } else if (type == Lucene80DocValuesFormat.SORTED) { - sorted.put(info.name, readSorted(meta)); + sorted.put(info.number, readSorted(meta)); } else if (type == Lucene80DocValuesFormat.SORTED_SET) { - sortedSets.put(info.name, readSortedSet(meta)); + sortedSets.put(info.number, readSortedSet(meta)); } else if (type == Lucene80DocValuesFormat.SORTED_NUMERIC) { - sortedNumerics.put(info.name, readSortedNumeric(meta)); + sortedNumerics.put(info.number, readSortedNumeric(meta)); } else { throw new CorruptIndexException("invalid type: " + type, meta); } @@ -426,7 +425,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer { @Override public NumericDocValues getNumeric(FieldInfo field) throws IOException { - NumericEntry entry = numerics.get(field.name); + NumericEntry entry = numerics.get(field.number); return getNumeric(entry); } @@ -915,7 +914,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer { @Override public BinaryDocValues getBinary(FieldInfo field) throws IOException { - BinaryEntry entry = binaries.get(field.name); + BinaryEntry entry = binaries.get(field.number); if (entry.compressed) { return getCompressedBinary(entry); } else { @@ -973,7 +972,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer { @Override public SortedDocValues getSorted(FieldInfo field) throws IOException { - SortedEntry entry = sorted.get(field.name); + SortedEntry entry = sorted.get(field.number); return getSorted(entry); } @@ -1407,7 +1406,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer { @Override public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { - SortedNumericEntry entry = sortedNumerics.get(field.name); + SortedNumericEntry entry = sortedNumerics.get(field.number); if (entry.numValues == entry.numDocsWithField) { return DocValues.singleton(getNumeric(entry)); } @@ -1543,7 +1542,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer { @Override public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { - SortedSetEntry entry = sortedSets.get(field.name); + SortedSetEntry entry = sortedSets.get(field.number); if (entry.singleValueEntry != null) { return DocValues.singleton(getSorted(entry.singleValueEntry)); } 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 da027a35f17..11e83b3f03c 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 @@ -21,8 +21,6 @@ import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.SKIP_IND 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; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.index.BaseTermsEnum; @@ -43,6 +41,7 @@ import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.SortedSetDocValues; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.TermsEnum.SeekStatus; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.store.ByteArrayDataInput; import org.apache.lucene.store.ChecksumIndexInput; @@ -59,12 +58,12 @@ import org.apache.lucene.util.packed.DirectReader; /** reader for {@link Lucene90DocValuesFormat} */ final class Lucene90DocValuesProducer extends DocValuesProducer { - private final Map numerics; - private final Map binaries; - private final Map sorted; - private final Map sortedSets; - private final Map sortedNumerics; - private final Map skippers; + private final IntObjectHashMap numerics; + private final IntObjectHashMap binaries; + private final IntObjectHashMap sorted; + private final IntObjectHashMap sortedSets; + private final IntObjectHashMap sortedNumerics; + private final IntObjectHashMap skippers; private final IndexInput data; private final int maxDoc; private int version = -1; @@ -81,12 +80,12 @@ final class Lucene90DocValuesProducer extends DocValuesProducer { String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); this.maxDoc = state.segmentInfo.maxDoc(); - numerics = new HashMap<>(); - binaries = new HashMap<>(); - sorted = new HashMap<>(); - sortedSets = new HashMap<>(); - sortedNumerics = new HashMap<>(); - skippers = new HashMap<>(); + numerics = new IntObjectHashMap<>(); + binaries = new IntObjectHashMap<>(); + sorted = new IntObjectHashMap<>(); + sortedSets = new IntObjectHashMap<>(); + sortedNumerics = new IntObjectHashMap<>(); + skippers = new IntObjectHashMap<>(); merging = false; // read in the entries from the metadata file. @@ -149,12 +148,12 @@ final class Lucene90DocValuesProducer extends DocValuesProducer { // Used for cloning private Lucene90DocValuesProducer( - Map numerics, - Map binaries, - Map sorted, - Map sortedSets, - Map sortedNumerics, - Map skippers, + IntObjectHashMap numerics, + IntObjectHashMap binaries, + IntObjectHashMap sorted, + IntObjectHashMap sortedSets, + IntObjectHashMap sortedNumerics, + IntObjectHashMap skippers, IndexInput data, int maxDoc, int version, @@ -194,18 +193,18 @@ final class Lucene90DocValuesProducer extends DocValuesProducer { } byte type = meta.readByte(); if (info.docValuesSkipIndexType() != DocValuesSkipIndexType.NONE) { - skippers.put(info.name, readDocValueSkipperMeta(meta)); + skippers.put(info.number, readDocValueSkipperMeta(meta)); } if (type == Lucene90DocValuesFormat.NUMERIC) { - numerics.put(info.name, readNumeric(meta)); + numerics.put(info.number, readNumeric(meta)); } else if (type == Lucene90DocValuesFormat.BINARY) { - binaries.put(info.name, readBinary(meta)); + binaries.put(info.number, readBinary(meta)); } else if (type == Lucene90DocValuesFormat.SORTED) { - sorted.put(info.name, readSorted(meta)); + sorted.put(info.number, readSorted(meta)); } else if (type == Lucene90DocValuesFormat.SORTED_SET) { - sortedSets.put(info.name, readSortedSet(meta)); + sortedSets.put(info.number, readSortedSet(meta)); } else if (type == Lucene90DocValuesFormat.SORTED_NUMERIC) { - sortedNumerics.put(info.name, readSortedNumeric(meta)); + sortedNumerics.put(info.number, readSortedNumeric(meta)); } else { throw new CorruptIndexException("invalid type: " + type, meta); } @@ -430,7 +429,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer { @Override public NumericDocValues getNumeric(FieldInfo field) throws IOException { - NumericEntry entry = numerics.get(field.name); + NumericEntry entry = numerics.get(field.number); return getNumeric(entry); } @@ -786,7 +785,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer { @Override public BinaryDocValues getBinary(FieldInfo field) throws IOException { - BinaryEntry entry = binaries.get(field.name); + BinaryEntry entry = binaries.get(field.number); if (entry.docsWithFieldOffset == -2) { return DocValues.emptyBinary(); @@ -887,7 +886,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer { @Override public SortedDocValues getSorted(FieldInfo field) throws IOException { - SortedEntry entry = sorted.get(field.name); + SortedEntry entry = sorted.get(field.number); return getSorted(entry); } @@ -1363,7 +1362,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer { @Override public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { - SortedNumericEntry entry = sortedNumerics.get(field.name); + SortedNumericEntry entry = sortedNumerics.get(field.number); return getSortedNumeric(entry); } @@ -1508,7 +1507,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer { @Override public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { - SortedSetEntry entry = sortedSets.get(field.name); + SortedSetEntry entry = sortedSets.get(field.number); if (entry.singleValueEntry != null) { return DocValues.singleton(getSorted(entry.singleValueEntry)); } @@ -1782,7 +1781,7 @@ final class Lucene90DocValuesProducer extends DocValuesProducer { @Override public DocValuesSkipper getSkipper(FieldInfo field) throws IOException { - final DocValuesSkipperEntry entry = skippers.get(field.name); + final DocValuesSkipperEntry entry = skippers.get(field.number); final IndexInput input = data.slice("doc value skipper", entry.offset, entry.length); // Prefetch the first page of data. Following pages are expected to get prefetched through diff --git a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java index 2e45e232b5f..9350c016f67 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java @@ -38,6 +38,7 @@ import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SortedDocValues; import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.util.IOUtils; /** @@ -256,7 +257,7 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat { private static class FieldsReader extends DocValuesProducer { - private final Map fields = new HashMap<>(); + private final IntObjectHashMap fields = new IntObjectHashMap<>(); private final Map formats = new HashMap<>(); // clone for merge @@ -270,10 +271,10 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat { } // Then rebuild fields: - for (Map.Entry ent : other.fields.entrySet()) { - DocValuesProducer producer = oldToNew.get(ent.getValue()); + for (IntObjectHashMap.IntObjectCursor ent : other.fields) { + DocValuesProducer producer = oldToNew.get(ent.value); assert producer != null; - fields.put(ent.getKey(), producer); + fields.put(ent.key, producer); } } @@ -302,7 +303,7 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat { segmentSuffix, format.fieldsProducer(new SegmentReadState(readState, segmentSuffix))); } - fields.put(fieldName, formats.get(segmentSuffix)); + fields.put(fi.number, formats.get(segmentSuffix)); } } } @@ -316,37 +317,37 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat { @Override public NumericDocValues getNumeric(FieldInfo field) throws IOException { - DocValuesProducer producer = fields.get(field.name); + DocValuesProducer producer = fields.get(field.number); return producer == null ? null : producer.getNumeric(field); } @Override public BinaryDocValues getBinary(FieldInfo field) throws IOException { - DocValuesProducer producer = fields.get(field.name); + DocValuesProducer producer = fields.get(field.number); return producer == null ? null : producer.getBinary(field); } @Override public SortedDocValues getSorted(FieldInfo field) throws IOException { - DocValuesProducer producer = fields.get(field.name); + DocValuesProducer producer = fields.get(field.number); return producer == null ? null : producer.getSorted(field); } @Override public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { - DocValuesProducer producer = fields.get(field.name); + DocValuesProducer producer = fields.get(field.number); return producer == null ? null : producer.getSortedNumeric(field); } @Override public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { - DocValuesProducer producer = fields.get(field.name); + DocValuesProducer producer = fields.get(field.number); return producer == null ? null : producer.getSortedSet(field); } @Override public DocValuesSkipper getSkipper(FieldInfo field) throws IOException { - DocValuesProducer producer = fields.get(field.name); + DocValuesProducer producer = fields.get(field.number); return producer == null ? null : producer.getSkipper(field); } diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentDocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/index/SegmentDocValuesProducer.java index 1d9878fe0db..0f4df818ddc 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentDocValuesProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentDocValuesProducer.java @@ -18,11 +18,10 @@ package org.apache.lucene.index; import java.io.IOException; import java.util.Collections; -import java.util.HashMap; import java.util.IdentityHashMap; -import java.util.Map; import java.util.Set; import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.internal.hppc.LongArrayList; import org.apache.lucene.store.Directory; @@ -32,7 +31,7 @@ import org.apache.lucene.store.Directory; // producer? class SegmentDocValuesProducer extends DocValuesProducer { - final Map dvProducersByField = new HashMap<>(); + final IntObjectHashMap dvProducersByField = new IntObjectHashMap<>(); final Set dvProducers = Collections.newSetFromMap(new IdentityHashMap()); final LongArrayList dvGens = new LongArrayList(); @@ -67,7 +66,7 @@ class SegmentDocValuesProducer extends DocValuesProducer { dvGens.add(docValuesGen); dvProducers.add(baseProducer); } - dvProducersByField.put(fi.name, baseProducer); + dvProducersByField.put(fi.number, baseProducer); } else { assert !dvGens.contains(docValuesGen); // otherwise, producer sees only the one fieldinfo it wrote @@ -76,7 +75,7 @@ class SegmentDocValuesProducer extends DocValuesProducer { docValuesGen, si, dir, new FieldInfos(new FieldInfo[] {fi})); dvGens.add(docValuesGen); dvProducers.add(dvp); - dvProducersByField.put(fi.name, dvp); + dvProducersByField.put(fi.number, dvp); } } } catch (Throwable t) { @@ -91,42 +90,42 @@ class SegmentDocValuesProducer extends DocValuesProducer { @Override public NumericDocValues getNumeric(FieldInfo field) throws IOException { - DocValuesProducer dvProducer = dvProducersByField.get(field.name); + DocValuesProducer dvProducer = dvProducersByField.get(field.number); assert dvProducer != null; return dvProducer.getNumeric(field); } @Override public BinaryDocValues getBinary(FieldInfo field) throws IOException { - DocValuesProducer dvProducer = dvProducersByField.get(field.name); + DocValuesProducer dvProducer = dvProducersByField.get(field.number); assert dvProducer != null; return dvProducer.getBinary(field); } @Override public SortedDocValues getSorted(FieldInfo field) throws IOException { - DocValuesProducer dvProducer = dvProducersByField.get(field.name); + DocValuesProducer dvProducer = dvProducersByField.get(field.number); assert dvProducer != null; return dvProducer.getSorted(field); } @Override public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { - DocValuesProducer dvProducer = dvProducersByField.get(field.name); + DocValuesProducer dvProducer = dvProducersByField.get(field.number); assert dvProducer != null; return dvProducer.getSortedNumeric(field); } @Override public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { - DocValuesProducer dvProducer = dvProducersByField.get(field.name); + DocValuesProducer dvProducer = dvProducersByField.get(field.number); assert dvProducer != null; return dvProducer.getSortedSet(field); } @Override public DocValuesSkipper getSkipper(FieldInfo field) throws IOException { - DocValuesProducer dvProducer = dvProducersByField.get(field.name); + DocValuesProducer dvProducer = dvProducersByField.get(field.number); assert dvProducer != null; return dvProducer.getSkipper(field); } From 9359cfd32f1f2c9ec1fd7e3cabd75ada4fc91204 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 29 Oct 2024 18:16:52 +0100 Subject: [PATCH 20/37] Speed up Lucene912PostingsReader nextDoc() impls. (#13963) 127 times out of 128, nextDoc() returns the next doc ID in the buffer. Currently, we check if the current doc is equal to the last doc ID in the block to know if we need to refill. We can do better by comparing the current index in the block with the block size, which is a bit more efficient since the latter is a constant. --- lucene/CHANGES.txt | 3 +++ .../lucene912/Lucene912PostingsReader.java | 21 +++++++++++-------- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 7146a97195e..66fe8b4ebd5 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -77,6 +77,9 @@ Optimizations * GITHUB#13961: Replace Map with IntObjectHashMap for DV producer. (Pan Guixin) +* GITHUB#13963: Speed up nextDoc() implementations in Lucene912PostingsReader. + (Adrien Grand) + Bug Fixes --------------------- * GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java index cea329c93ca..6da66e4ddf2 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java @@ -580,7 +580,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { @Override public int nextDoc() throws IOException { - if (doc == level0LastDocID) { // advance skip data on level 0 + if (docBufferUpto == BLOCK_SIZE) { // advance skip data on level 0 moveToNextLevel0Block(); } @@ -875,7 +875,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { @Override public int nextDoc() throws IOException { - if (doc == level0LastDocID) { // advance level 0 skip data + if (docBufferUpto == BLOCK_SIZE) { // advance level 0 skip data moveToNextLevel0Block(); } @@ -1417,11 +1417,13 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { @Override public int nextDoc() throws IOException { - if (doc == level0LastDocID) { - moveToNextLevel0Block(); - } else if (needsRefilling) { - refillDocs(); - needsRefilling = false; + if (docBufferUpto == BLOCK_SIZE) { + if (needsRefilling) { + refillDocs(); + needsRefilling = false; + } else { + moveToNextLevel0Block(); + } } return this.doc = (int) docBuffer[docBufferUpto++]; @@ -1644,8 +1646,9 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { @Override public int nextDoc() throws IOException { - advanceShallow(doc + 1); - if (needsRefilling) { + if (docBufferUpto == BLOCK_SIZE) { + advanceShallow(doc + 1); + assert needsRefilling; refillDocs(); needsRefilling = false; } From 3041af7a9477883cef7d5e2897df06ee1b63b3a5 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 30 Oct 2024 12:51:36 +0100 Subject: [PATCH 21/37] Speed up advancing within a block, take 2. (#13958) PR #13692 tried to speed up advancing by using branchless binary search, but while this yielded a speedup on my machine, this yielded a slowdown on nightly benchmarks. This PR tries a different approach using vectorization. Experimentation suggests that it speeds up queries that advance to the next few doc IDs, such as `AndHighHigh`. --- lucene/CHANGES.txt | 2 + .../benchmark/jmh/AdvanceBenchmark.java | 180 ++++++++++++++++++ .../lucene912/Lucene912PostingsReader.java | 36 ++-- .../DefaultVectorUtilSupport.java | 10 + .../vectorization/VectorUtilSupport.java | 8 + .../org/apache/lucene/search/ImpactsDISI.java | 4 + .../org/apache/lucene/util/VectorUtil.java | 10 + .../PanamaVectorUtilSupport.java | 28 +++ .../apache/lucene/util/TestVectorUtil.java | 31 +++ 9 files changed, 292 insertions(+), 17 deletions(-) create mode 100644 lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/AdvanceBenchmark.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 66fe8b4ebd5..23ee3e92a85 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -80,6 +80,8 @@ Optimizations * GITHUB#13963: Speed up nextDoc() implementations in Lucene912PostingsReader. (Adrien Grand) +* GITHUB#13958: Speed up advancing within a block. (Adrien Grand) + Bug Fixes --------------------- * GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended diff --git a/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/AdvanceBenchmark.java b/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/AdvanceBenchmark.java new file mode 100644 index 00000000000..7046e10c06e --- /dev/null +++ b/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/AdvanceBenchmark.java @@ -0,0 +1,180 @@ +/* + * 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.benchmark.jmh; + +import java.util.Arrays; +import java.util.Random; +import java.util.concurrent.TimeUnit; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.util.VectorUtil; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.CompilerControl; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +@State(Scope.Benchmark) +@Warmup(iterations = 5, time = 1) +@Measurement(iterations = 5, time = 1) +@Fork( + value = 3, + jvmArgsAppend = { + "-Xmx1g", + "-Xms1g", + "-XX:+AlwaysPreTouch", + "--add-modules", + "jdk.incubator.vector" + }) +public class AdvanceBenchmark { + + private final long[] values = new long[129]; + private final int[] startIndexes = new int[1_000]; + private final long[] targets = new long[startIndexes.length]; + + @Setup(Level.Trial) + public void setup() throws Exception { + for (int i = 0; i < 128; ++i) { + values[i] = i; + } + values[128] = DocIdSetIterator.NO_MORE_DOCS; + Random r = new Random(0); + for (int i = 0; i < startIndexes.length; ++i) { + startIndexes[i] = r.nextInt(64); + targets[i] = startIndexes[i] + 1 + r.nextInt(1 << r.nextInt(7)); + } + } + + @Benchmark + public void binarySearch() { + for (int i = 0; i < startIndexes.length; ++i) { + binarySearch(values, targets[i], startIndexes[i]); + } + } + + @CompilerControl(CompilerControl.Mode.DONT_INLINE) + private static int binarySearch(long[] values, long target, int startIndex) { + // Standard binary search + int i = Arrays.binarySearch(values, startIndex, values.length, target); + if (i < 0) { + i = -1 - i; + } + return i; + } + + @Benchmark + public void inlinedBranchlessBinarySearch() { + for (int i = 0; i < targets.length; ++i) { + inlinedBranchlessBinarySearch(values, targets[i]); + } + } + + @CompilerControl(CompilerControl.Mode.DONT_INLINE) + private static int inlinedBranchlessBinarySearch(long[] values, long target) { + // This compiles to cmov instructions. + int start = 0; + + if (values[63] < target) { + start += 64; + } + if (values[start + 31] < target) { + start += 32; + } + if (values[start + 15] < target) { + start += 16; + } + if (values[start + 7] < target) { + start += 8; + } + if (values[start + 3] < target) { + start += 4; + } + if (values[start + 1] < target) { + start += 2; + } + if (values[start] < target) { + start += 1; + } + + return start; + } + + @Benchmark + public void linearSearch() { + for (int i = 0; i < startIndexes.length; ++i) { + linearSearch(values, targets[i], startIndexes[i]); + } + } + + @CompilerControl(CompilerControl.Mode.DONT_INLINE) + private static int linearSearch(long[] values, long target, int startIndex) { + // Naive linear search. + for (int i = startIndex; i < values.length; ++i) { + if (values[i] >= target) { + return i; + } + } + return values.length; + } + + @Benchmark + public void vectorUtilSearch() { + for (int i = 0; i < startIndexes.length; ++i) { + VectorUtil.findNextGEQ(values, 128, targets[i], startIndexes[i]); + } + } + + @CompilerControl(CompilerControl.Mode.DONT_INLINE) + private static int vectorUtilSearch(long[] values, long target, int startIndex) { + return VectorUtil.findNextGEQ(values, 128, target, startIndex); + } + + private static void assertEquals(int expected, int actual) { + if (expected != actual) { + throw new AssertionError("Expected: " + expected + ", got " + actual); + } + } + + public static void main(String[] args) { + // For testing purposes + long[] values = new long[129]; + for (int i = 0; i < 128; ++i) { + values[i] = i; + } + values[128] = DocIdSetIterator.NO_MORE_DOCS; + for (int start = 0; start < 128; ++start) { + for (int targetIndex = start; targetIndex < 128; ++targetIndex) { + int actualIndex = binarySearch(values, values[targetIndex], start); + assertEquals(targetIndex, actualIndex); + actualIndex = inlinedBranchlessBinarySearch(values, values[targetIndex]); + assertEquals(targetIndex, actualIndex); + actualIndex = linearSearch(values, values[targetIndex], start); + assertEquals(targetIndex, actualIndex); + actualIndex = vectorUtilSearch(values, values[targetIndex], start); + assertEquals(targetIndex, actualIndex); + } + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java index 6da66e4ddf2..1a2fe05679e 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java @@ -46,6 +46,7 @@ import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SlowImpactsEnum; import org.apache.lucene.internal.vectorization.PostingDecodingUtil; +import org.apache.lucene.internal.vectorization.VectorUtilSupport; import org.apache.lucene.internal.vectorization.VectorizationProvider; import org.apache.lucene.store.ByteArrayDataInput; import org.apache.lucene.store.ChecksumIndexInput; @@ -65,6 +66,8 @@ import org.apache.lucene.util.IOUtils; public final class Lucene912PostingsReader extends PostingsReaderBase { static final VectorizationProvider VECTORIZATION_PROVIDER = VectorizationProvider.getInstance(); + private static final VectorUtilSupport VECTOR_SUPPORT = + VECTORIZATION_PROVIDER.getVectorUtilSupport(); // Dummy impacts, composed of the maximum possible term frequency and the lowest possible // (unsigned) norm value. This is typically used on tail blocks, which don't actually record // impacts as the storage overhead would not be worth any query evaluation speedup, since there's @@ -215,15 +218,6 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } } - static int findFirstGreater(long[] buffer, int target, int from) { - for (int i = from; i < BLOCK_SIZE; ++i) { - if (buffer[i] >= target) { - return i; - } - } - return BLOCK_SIZE; - } - @Override public BlockTermState newTermState() { return new IntBlockTermState(); @@ -357,6 +351,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { protected int docCountUpto; // number of docs in or before the current block protected long prevDocID; // last doc ID of the previous block + protected int docBufferSize; protected int docBufferUpto; protected IndexInput docIn; @@ -402,6 +397,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { level1DocEndFP = termState.docStartFP; } level1DocCountUpto = 0; + docBufferSize = BLOCK_SIZE; docBufferUpto = BLOCK_SIZE; return this; } @@ -487,7 +483,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { docCountUpto += BLOCK_SIZE; prevDocID = docBuffer[BLOCK_SIZE - 1]; docBufferUpto = 0; - assert docBuffer[BLOCK_SIZE] == NO_MORE_DOCS; + assert docBuffer[docBufferSize] == NO_MORE_DOCS; } private void refillRemainder() throws IOException { @@ -508,6 +504,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { docCountUpto += left; } docBufferUpto = 0; + docBufferSize = left; freqFP = -1; } @@ -604,7 +601,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } } - int next = findFirstGreater(docBuffer, target, docBufferUpto); + int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); this.doc = (int) docBuffer[next]; docBufferUpto = next + 1; return doc; @@ -782,16 +779,18 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { freqBuffer[0] = totalTermFreq; docBuffer[1] = NO_MORE_DOCS; docCountUpto++; + docBufferSize = 1; } else { // Read vInts: PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, true); prefixSum(docBuffer, left, prevDocID); docBuffer[left] = NO_MORE_DOCS; docCountUpto += left; + docBufferSize = left; } prevDocID = docBuffer[BLOCK_SIZE - 1]; docBufferUpto = 0; - assert docBuffer[BLOCK_SIZE] == NO_MORE_DOCS; + assert docBuffer[docBufferSize] == NO_MORE_DOCS; } private void skipLevel1To(int target) throws IOException { @@ -951,7 +950,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { refillDocs(); } - int next = findFirstGreater(docBuffer, target, docBufferUpto); + int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); posPendingCount += sumOverRange(freqBuffer, docBufferUpto, next + 1); this.freq = (int) freqBuffer[next]; this.docBufferUpto = next + 1; @@ -1155,6 +1154,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { protected int docCountUpto; // number of docs in or before the current block protected int doc = -1; // doc we last read protected long prevDocID = -1; // last doc ID of the previous block + protected int docBufferSize = BLOCK_SIZE; protected int docBufferUpto = BLOCK_SIZE; // true if we shallow-advanced to a new block that we have not decoded yet @@ -1306,10 +1306,11 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { docBuffer[left] = NO_MORE_DOCS; freqFP = -1; docCountUpto += left; + docBufferSize = left; } prevDocID = docBuffer[BLOCK_SIZE - 1]; docBufferUpto = 0; - assert docBuffer[BLOCK_SIZE] == NO_MORE_DOCS; + assert docBuffer[docBufferSize] == NO_MORE_DOCS; } private void skipLevel1To(int target) throws IOException { @@ -1437,7 +1438,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { needsRefilling = false; } - int next = findFirstGreater(docBuffer, target, docBufferUpto); + int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); this.doc = (int) docBuffer[next]; docBufferUpto = next + 1; return doc; @@ -1535,10 +1536,11 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { prefixSum(docBuffer, left, prevDocID); docBuffer[left] = NO_MORE_DOCS; docCountUpto += left; + docBufferSize = left; } prevDocID = docBuffer[BLOCK_SIZE - 1]; docBufferUpto = 0; - assert docBuffer[BLOCK_SIZE] == NO_MORE_DOCS; + assert docBuffer[docBufferSize] == NO_MORE_DOCS; } private void skipLevel1To(int target) throws IOException { @@ -1669,7 +1671,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { needsRefilling = false; } - int next = findFirstGreater(docBuffer, target, docBufferUpto); + int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); posPendingCount += sumOverRange(freqBuffer, docBufferUpto, next + 1); freq = (int) freqBuffer[next]; docBufferUpto = next + 1; diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java index eb5160a0f0d..c336babc9cc 100644 --- a/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java @@ -197,4 +197,14 @@ final class DefaultVectorUtilSupport implements VectorUtilSupport { } return squareSum; } + + @Override + public int findNextGEQ(long[] buffer, int length, long target, int from) { + for (int i = from; i < length; ++i) { + if (buffer[i] >= target) { + return i; + } + } + return length; + } } diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorUtilSupport.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorUtilSupport.java index 22e5e96aa25..ac5b463e6c0 100644 --- a/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorUtilSupport.java +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorUtilSupport.java @@ -44,4 +44,12 @@ public interface VectorUtilSupport { /** Returns the sum of squared differences of the two byte vectors. */ int squareDistance(byte[] a, byte[] b); + + /** + * Given an array {@code buffer} that is sorted between indexes {@code 0} inclusive and {@code + * length} exclusive, find the first array index whose value is greater than or equal to {@code + * target}. This index is guaranteed to be at least {@code from}. If there is no such array index, + * {@code length} is returned. + */ + int findNextGEQ(long[] buffer, int length, long target, int from); } diff --git a/lucene/core/src/java/org/apache/lucene/search/ImpactsDISI.java b/lucene/core/src/java/org/apache/lucene/search/ImpactsDISI.java index 5ab19747246..355aa774813 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ImpactsDISI.java +++ b/lucene/core/src/java/org/apache/lucene/search/ImpactsDISI.java @@ -106,6 +106,10 @@ public final class ImpactsDISI extends DocIdSetIterator { @Override public int nextDoc() throws IOException { + DocIdSetIterator in = this.in; + if (in.docID() < upTo) { + return in.nextDoc(); + } return advance(in.docID() + 1); } diff --git a/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java index e1c3978cff3..a7239957b32 100644 --- a/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java +++ b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java @@ -307,4 +307,14 @@ public final class VectorUtil { } return v; } + + /** + * Given an array {@code buffer} that is sorted between indexes {@code 0} inclusive and {@code + * length} exclusive, find the first array index whose value is greater than or equal to {@code + * target}. This index is guaranteed to be at least {@code from}. If there is no such array index, + * {@code length} is returned. + */ + public static int findNextGEQ(long[] buffer, int length, long target, int from) { + return IMPL.findNextGEQ(buffer, length, target, from); + } } diff --git a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java index ad2dff11cea..cc3624f1b30 100644 --- a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java +++ b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java @@ -29,8 +29,11 @@ import java.lang.foreign.MemorySegment; import jdk.incubator.vector.ByteVector; import jdk.incubator.vector.FloatVector; import jdk.incubator.vector.IntVector; +import jdk.incubator.vector.LongVector; import jdk.incubator.vector.ShortVector; import jdk.incubator.vector.Vector; +import jdk.incubator.vector.VectorMask; +import jdk.incubator.vector.VectorOperators; import jdk.incubator.vector.VectorShape; import jdk.incubator.vector.VectorSpecies; import org.apache.lucene.util.Constants; @@ -56,6 +59,7 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { PanamaVectorConstants.PRERERRED_INT_SPECIES; private static final VectorSpecies BYTE_SPECIES; private static final VectorSpecies SHORT_SPECIES; + private static final VectorSpecies LONG_SPECIES; static final int VECTOR_BITSIZE; @@ -71,6 +75,7 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { BYTE_SPECIES = null; SHORT_SPECIES = null; } + LONG_SPECIES = PanamaVectorConstants.PRERERRED_LONG_SPECIES; } // the way FMA should work! if available use it, otherwise fall back to mul/add @@ -761,4 +766,27 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { // reduce return acc1.add(acc2).reduceLanes(ADD); } + + // Experiments suggest that we need at least 4 lanes so that the overhead of going with the vector + // approach and counting trues on vector masks pays off. + private static final boolean ENABLE_FIND_NEXT_GEQ_VECTOR_OPTO = LONG_SPECIES.length() >= 4; + + @Override + public int findNextGEQ(long[] buffer, int length, long target, int from) { + if (ENABLE_FIND_NEXT_GEQ_VECTOR_OPTO) { + for (; from + LONG_SPECIES.length() < length; from += LONG_SPECIES.length() + 1) { + if (buffer[from + LONG_SPECIES.length()] >= target) { + LongVector vector = LongVector.fromArray(LONG_SPECIES, buffer, from); + VectorMask mask = vector.compare(VectorOperators.LT, target); + return from + mask.trueCount(); + } + } + } + for (int i = from; i < length; ++i) { + if (buffer[i] >= target) { + return i; + } + } + return length; + } } diff --git a/lucene/core/src/test/org/apache/lucene/util/TestVectorUtil.java b/lucene/core/src/test/org/apache/lucene/util/TestVectorUtil.java index 00577c3db52..541dbbde8d7 100644 --- a/lucene/core/src/test/org/apache/lucene/util/TestVectorUtil.java +++ b/lucene/core/src/test/org/apache/lucene/util/TestVectorUtil.java @@ -353,4 +353,35 @@ public class TestVectorUtil extends LuceneTestCase { } return res; } + + public void testFindNextGEQ() { + int padding = TestUtil.nextInt(random(), 0, 5); + long[] values = new long[128 + padding]; + long v = 0; + for (int i = 0; i < 128; ++i) { + v += TestUtil.nextInt(random(), 1, 1000); + values[i] = v; + } + + // Now duel with slowFindFirstGreater + for (int iter = 0; iter < 1_000; ++iter) { + int from = TestUtil.nextInt(random(), 0, 127); + long target = + TestUtil.nextLong(random(), values[from], Math.max(values[from], values[127])) + + random().nextInt(10) + - 5; + assertEquals( + slowFindNextGEQ(values, 128, target, from), + VectorUtil.findNextGEQ(values, 128, target, from)); + } + } + + private static int slowFindNextGEQ(long[] buffer, int length, long target, int from) { + for (int i = from; i < length; ++i) { + if (buffer[i] >= target) { + return i; + } + } + return length; + } } From 26e0737e4037074114eadc7717057b938ae93ef0 Mon Sep 17 00:00:00 2001 From: Mayya Sharipova Date: Thu, 31 Oct 2024 10:22:30 -0400 Subject: [PATCH 22/37] Account for 0 graph size when initializing HNSW graph (#13964) When initializing a joint graph from one of the segments' graphs, we always assume that a segment's graph is present. But later we want to explore an option where some segments will not have graphs (#13447). This change allows to account for missing graphs. --- .../util/hnsw/ConcurrentHnswMerger.java | 35 ++++++++----------- .../util/hnsw/IncrementalHnswGraphMerger.java | 4 +++ 2 files changed, 19 insertions(+), 20 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/ConcurrentHnswMerger.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/ConcurrentHnswMerger.java index c4e7d159b48..b4688d09730 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/ConcurrentHnswMerger.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/ConcurrentHnswMerger.java @@ -48,28 +48,23 @@ public class ConcurrentHnswMerger extends IncrementalHnswGraphMerger { @Override protected HnswBuilder createBuilder(KnnVectorValues mergedVectorValues, int maxOrd) throws IOException { + OnHeapHnswGraph graph; + BitSet initializedNodes = null; + if (initReader == null) { - return new HnswConcurrentMergeBuilder( - taskExecutor, - numWorker, - scorerSupplier, - M, - beamWidth, - new OnHeapHnswGraph(M, maxOrd), - null); + graph = new OnHeapHnswGraph(M, maxOrd); + } else { + HnswGraph initializerGraph = ((HnswGraphProvider) initReader).getGraph(fieldInfo.name); + if (initializerGraph.size() == 0) { + graph = new OnHeapHnswGraph(M, maxOrd); + } else { + initializedNodes = new FixedBitSet(maxOrd); + int[] oldToNewOrdinalMap = getNewOrdMapping(mergedVectorValues, initializedNodes); + graph = + InitializedHnswGraphBuilder.initGraph(M, initializerGraph, oldToNewOrdinalMap, maxOrd); + } } - - HnswGraph initializerGraph = ((HnswGraphProvider) initReader).getGraph(fieldInfo.name); - BitSet initializedNodes = new FixedBitSet(maxOrd); - int[] oldToNewOrdinalMap = getNewOrdMapping(mergedVectorValues, initializedNodes); - return new HnswConcurrentMergeBuilder( - taskExecutor, - numWorker, - scorerSupplier, - M, - beamWidth, - InitializedHnswGraphBuilder.initGraph(M, initializerGraph, oldToNewOrdinalMap, maxOrd), - initializedNodes); + taskExecutor, numWorker, scorerSupplier, M, beamWidth, graph, initializedNodes); } } diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java index d64961a02ee..c480d53360c 100644 --- a/lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java +++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/IncrementalHnswGraphMerger.java @@ -121,6 +121,10 @@ public class IncrementalHnswGraphMerger implements HnswGraphMerger { } HnswGraph initializerGraph = ((HnswGraphProvider) initReader).getGraph(fieldInfo.name); + if (initializerGraph.size() == 0) { + return HnswGraphBuilder.create( + scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed, maxOrd); + } BitSet initializedNodes = new FixedBitSet(maxOrd); int[] oldToNewOrdinalMap = getNewOrdMapping(mergedVectorValues, initializedNodes); From 494b16063e1d06e3018e0e0e70168e2813f86f03 Mon Sep 17 00:00:00 2001 From: panguixin Date: Thu, 31 Oct 2024 23:16:09 +0800 Subject: [PATCH 23/37] Replace Map with IntObjectHashMap for KnnVectorsReader (#13763) --- lucene/CHANGES.txt | 2 + .../lucene90/Lucene90HnswVectorsReader.java | 27 ++++---- .../lucene91/Lucene91HnswVectorsReader.java | 27 ++++---- .../lucene92/Lucene92HnswVectorsReader.java | 27 ++++---- .../lucene94/Lucene94HnswVectorsReader.java | 51 +++++++-------- .../lucene95/Lucene95HnswVectorsReader.java | 63 ++++++++----------- .../SimpleTextKnnVectorsReader.java | 13 ++-- .../lucene99/Lucene99FlatVectorsReader.java | 55 +++++++--------- .../lucene99/Lucene99HnswVectorsReader.java | 51 +++++++++------ .../Lucene99ScalarQuantizedVectorsReader.java | 49 +++++++-------- .../perfield/PerFieldKnnVectorsFormat.java | 58 ++++++++++++----- 11 files changed, 218 insertions(+), 205 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 23ee3e92a85..b512720bc9c 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -82,6 +82,8 @@ Optimizations * GITHUB#13958: Speed up advancing within a block. (Adrien Grand) +* GITHUB#13763: Replace Map with IntObjectHashMap for KnnVectorsReader (Pan Guixin) + Bug Fixes --------------------- * GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java index 3ffd4f4d75a..015fad7490c 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java @@ -20,8 +20,6 @@ package org.apache.lucene.backward_codecs.lucene90; import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; import java.io.IOException; -import java.util.HashMap; -import java.util.Map; import java.util.SplittableRandom; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.KnnVectorsReader; @@ -33,6 +31,7 @@ import org.apache.lucene.index.FloatVectorValues; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.search.KnnCollector; import org.apache.lucene.search.VectorScorer; import org.apache.lucene.store.ChecksumIndexInput; @@ -50,14 +49,16 @@ import org.apache.lucene.util.hnsw.NeighborQueue; */ public final class Lucene90HnswVectorsReader extends KnnVectorsReader { - private final Map fields = new HashMap<>(); + private final IntObjectHashMap fields = new IntObjectHashMap<>(); private final IndexInput vectorData; private final IndexInput vectorIndex; private final long checksumSeed; + private final FieldInfos fieldInfos; Lucene90HnswVectorsReader(SegmentReadState state) throws IOException { int versionMeta = readMetadata(state); long[] checksumRef = new long[1]; + this.fieldInfos = state.fieldInfos; boolean success = false; try { vectorData = @@ -158,7 +159,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader { FieldEntry fieldEntry = readField(meta, info); validateFieldEntry(info, fieldEntry); - fields.put(info.name, fieldEntry); + fields.put(info.number, fieldEntry); } } @@ -218,13 +219,18 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader { CodecUtil.checksumEntireFile(vectorIndex); } - @Override - public FloatVectorValues getFloatVectorValues(String field) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null) { + private FieldEntry getFieldEntry(String field) { + final FieldInfo info = fieldInfos.fieldInfo(field); + final FieldEntry fieldEntry; + if (info == null || (fieldEntry = fields.get(info.number)) == null) { throw new IllegalArgumentException("field=\"" + field + "\" not found"); } - return getOffHeapVectorValues(fieldEntry); + return fieldEntry; + } + + @Override + public FloatVectorValues getFloatVectorValues(String field) throws IOException { + return getOffHeapVectorValues(getFieldEntry(field)); } @Override @@ -235,8 +241,7 @@ public final class Lucene90HnswVectorsReader extends KnnVectorsReader { @Override public void search(String field, float[] target, KnnCollector knnCollector, Bits acceptDocs) throws IOException { - FieldEntry fieldEntry = fields.get(field); - + final FieldEntry fieldEntry = getFieldEntry(field); if (fieldEntry.size() == 0) { return; } diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsReader.java index a140b4fd7f3..e71fa66719f 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsReader.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene91/Lucene91HnswVectorsReader.java @@ -21,8 +21,6 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; import java.io.IOException; import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; import java.util.function.IntUnaryOperator; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.KnnVectorsReader; @@ -35,6 +33,7 @@ import org.apache.lucene.index.FloatVectorValues; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.KnnCollector; import org.apache.lucene.search.VectorScorer; @@ -55,13 +54,15 @@ import org.apache.lucene.util.hnsw.RandomVectorScorer; */ public final class Lucene91HnswVectorsReader extends KnnVectorsReader { - private final Map fields = new HashMap<>(); + private final IntObjectHashMap fields = new IntObjectHashMap<>(); private final IndexInput vectorData; private final IndexInput vectorIndex; private final DefaultFlatVectorScorer defaultFlatVectorScorer = new DefaultFlatVectorScorer(); + private final FieldInfos fieldInfos; Lucene91HnswVectorsReader(SegmentReadState state) throws IOException { int versionMeta = readMetadata(state); + this.fieldInfos = state.fieldInfos; boolean success = false; try { vectorData = @@ -154,7 +155,7 @@ public final class Lucene91HnswVectorsReader extends KnnVectorsReader { } FieldEntry fieldEntry = readField(meta, info); validateFieldEntry(info, fieldEntry); - fields.put(info.name, fieldEntry); + fields.put(info.number, fieldEntry); } } @@ -214,13 +215,18 @@ public final class Lucene91HnswVectorsReader extends KnnVectorsReader { CodecUtil.checksumEntireFile(vectorIndex); } - @Override - public FloatVectorValues getFloatVectorValues(String field) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null) { + private FieldEntry getFieldEntry(String field) { + final FieldInfo info = fieldInfos.fieldInfo(field); + final FieldEntry fieldEntry; + if (info == null || (fieldEntry = fields.get(info.number)) == null) { throw new IllegalArgumentException("field=\"" + field + "\" not found"); } - return getOffHeapVectorValues(fieldEntry); + return fieldEntry; + } + + @Override + public FloatVectorValues getFloatVectorValues(String field) throws IOException { + return getOffHeapVectorValues(getFieldEntry(field)); } @Override @@ -231,8 +237,7 @@ public final class Lucene91HnswVectorsReader extends KnnVectorsReader { @Override public void search(String field, float[] target, KnnCollector knnCollector, Bits acceptDocs) throws IOException { - FieldEntry fieldEntry = fields.get(field); - + final FieldEntry fieldEntry = getFieldEntry(field); if (fieldEntry.size() == 0) { return; } diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene92/Lucene92HnswVectorsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene92/Lucene92HnswVectorsReader.java index 39fe109a9f1..034967efbaa 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene92/Lucene92HnswVectorsReader.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene92/Lucene92HnswVectorsReader.java @@ -21,8 +21,6 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; import java.io.IOException; import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.KnnVectorsReader; import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer; @@ -34,6 +32,7 @@ import org.apache.lucene.index.FloatVectorValues; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.search.KnnCollector; import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.DataInput; @@ -53,13 +52,15 @@ import org.apache.lucene.util.packed.DirectMonotonicReader; */ public final class Lucene92HnswVectorsReader extends KnnVectorsReader { - private final Map fields = new HashMap<>(); + private final IntObjectHashMap fields = new IntObjectHashMap<>(); private final IndexInput vectorData; private final IndexInput vectorIndex; private final DefaultFlatVectorScorer defaultFlatVectorScorer = new DefaultFlatVectorScorer(); + private final FieldInfos fieldInfos; Lucene92HnswVectorsReader(SegmentReadState state) throws IOException { int versionMeta = readMetadata(state); + this.fieldInfos = state.fieldInfos; boolean success = false; try { vectorData = @@ -152,7 +153,7 @@ public final class Lucene92HnswVectorsReader extends KnnVectorsReader { } FieldEntry fieldEntry = readField(meta, info); validateFieldEntry(info, fieldEntry); - fields.put(info.name, fieldEntry); + fields.put(info.number, fieldEntry); } } @@ -212,13 +213,18 @@ public final class Lucene92HnswVectorsReader extends KnnVectorsReader { CodecUtil.checksumEntireFile(vectorIndex); } - @Override - public FloatVectorValues getFloatVectorValues(String field) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null) { + private FieldEntry getFieldEntry(String field) { + final FieldInfo info = fieldInfos.fieldInfo(field); + final FieldEntry fieldEntry; + if (info == null || (fieldEntry = fields.get(info.number)) == null) { throw new IllegalArgumentException("field=\"" + field + "\" not found"); } - return OffHeapFloatVectorValues.load(fieldEntry, vectorData); + return fieldEntry; + } + + @Override + public FloatVectorValues getFloatVectorValues(String field) throws IOException { + return OffHeapFloatVectorValues.load(getFieldEntry(field), vectorData); } @Override @@ -229,8 +235,7 @@ public final class Lucene92HnswVectorsReader extends KnnVectorsReader { @Override public void search(String field, float[] target, KnnCollector knnCollector, Bits acceptDocs) throws IOException { - FieldEntry fieldEntry = fields.get(field); - + final FieldEntry fieldEntry = getFieldEntry(field); if (fieldEntry.size() == 0) { return; } diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsReader.java index d5beae1e681..1ad2e302364 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsReader.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene94/Lucene94HnswVectorsReader.java @@ -21,8 +21,6 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; import java.io.IOException; import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.KnnVectorsReader; import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer; @@ -35,6 +33,7 @@ import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.VectorEncoding; import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.search.KnnCollector; import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.DataInput; @@ -54,13 +53,15 @@ import org.apache.lucene.util.packed.DirectMonotonicReader; */ public final class Lucene94HnswVectorsReader extends KnnVectorsReader { - private final Map fields = new HashMap<>(); + private final IntObjectHashMap fields = new IntObjectHashMap<>(); private final IndexInput vectorData; private final IndexInput vectorIndex; private final DefaultFlatVectorScorer defaultFlatVectorScorer = new DefaultFlatVectorScorer(); + private final FieldInfos fieldInfos; Lucene94HnswVectorsReader(SegmentReadState state) throws IOException { int versionMeta = readMetadata(state); + this.fieldInfos = state.fieldInfos; boolean success = false; try { vectorData = @@ -153,7 +154,7 @@ public final class Lucene94HnswVectorsReader extends KnnVectorsReader { } FieldEntry fieldEntry = readField(meta, info); validateFieldEntry(info, fieldEntry); - fields.put(info.name, fieldEntry); + fields.put(info.number, fieldEntry); } } @@ -230,48 +231,41 @@ public final class Lucene94HnswVectorsReader extends KnnVectorsReader { CodecUtil.checksumEntireFile(vectorIndex); } - @Override - public FloatVectorValues getFloatVectorValues(String field) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null) { + private FieldEntry getFieldEntry(String field, VectorEncoding expectedEncoding) { + final FieldInfo info = fieldInfos.fieldInfo(field); + final FieldEntry fieldEntry; + if (info == null || (fieldEntry = fields.get(info.number)) == null) { throw new IllegalArgumentException("field=\"" + field + "\" not found"); } - if (fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { + if (fieldEntry.vectorEncoding != expectedEncoding) { throw new IllegalArgumentException( "field=\"" + field + "\" is encoded as: " + fieldEntry.vectorEncoding + " expected: " - + VectorEncoding.FLOAT32); + + expectedEncoding); } + return fieldEntry; + } + + @Override + public FloatVectorValues getFloatVectorValues(String field) throws IOException { + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.FLOAT32); return OffHeapFloatVectorValues.load(fieldEntry, vectorData); } @Override public ByteVectorValues getByteVectorValues(String field) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null) { - throw new IllegalArgumentException("field=\"" + field + "\" not found"); - } - if (fieldEntry.vectorEncoding != VectorEncoding.BYTE) { - throw new IllegalArgumentException( - "field=\"" - + field - + "\" is encoded as: " - + fieldEntry.vectorEncoding - + " expected: " - + VectorEncoding.BYTE); - } + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.BYTE); return OffHeapByteVectorValues.load(fieldEntry, vectorData); } @Override public void search(String field, float[] target, KnnCollector knnCollector, Bits acceptDocs) throws IOException { - FieldEntry fieldEntry = fields.get(field); - - if (fieldEntry.size() == 0 || fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.FLOAT32); + if (fieldEntry.size() == 0 || knnCollector.k() == 0) { return; } @@ -289,9 +283,8 @@ public final class Lucene94HnswVectorsReader extends KnnVectorsReader { @Override public void search(String field, byte[] target, KnnCollector knnCollector, Bits acceptDocs) throws IOException { - FieldEntry fieldEntry = fields.get(field); - - if (fieldEntry.size() == 0 || fieldEntry.vectorEncoding != VectorEncoding.BYTE) { + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.BYTE); + if (fieldEntry.size() == 0 || knnCollector.k() == 0) { return; } diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene95/Lucene95HnswVectorsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene95/Lucene95HnswVectorsReader.java index 2e6714d6eb8..b5859daf9f2 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene95/Lucene95HnswVectorsReader.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene95/Lucene95HnswVectorsReader.java @@ -21,8 +21,6 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; import java.io.IOException; import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.KnnVectorsReader; import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer; @@ -39,6 +37,7 @@ import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.VectorEncoding; import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.search.KnnCollector; import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.DataInput; @@ -61,7 +60,7 @@ import org.apache.lucene.util.packed.DirectMonotonicReader; public final class Lucene95HnswVectorsReader extends KnnVectorsReader implements HnswGraphProvider { private final FieldInfos fieldInfos; - private final Map fields = new HashMap<>(); + private final IntObjectHashMap fields = new IntObjectHashMap<>(); private final IndexInput vectorData; private final IndexInput vectorIndex; private final DefaultFlatVectorScorer defaultFlatVectorScorer = new DefaultFlatVectorScorer(); @@ -161,7 +160,7 @@ public final class Lucene95HnswVectorsReader extends KnnVectorsReader implements } FieldEntry fieldEntry = readField(meta, info); validateFieldEntry(info, fieldEntry); - fields.put(info.name, fieldEntry); + fields.put(info.number, fieldEntry); } } @@ -238,21 +237,27 @@ public final class Lucene95HnswVectorsReader extends KnnVectorsReader implements CodecUtil.checksumEntireFile(vectorIndex); } - @Override - public FloatVectorValues getFloatVectorValues(String field) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null) { + private FieldEntry getFieldEntry(String field, VectorEncoding expectedEncoding) { + final FieldInfo info = fieldInfos.fieldInfo(field); + final FieldEntry fieldEntry; + if (info == null || (fieldEntry = fields.get(info.number)) == null) { throw new IllegalArgumentException("field=\"" + field + "\" not found"); } - if (fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { + if (fieldEntry.vectorEncoding != expectedEncoding) { throw new IllegalArgumentException( "field=\"" + field + "\" is encoded as: " + fieldEntry.vectorEncoding + " expected: " - + VectorEncoding.FLOAT32); + + expectedEncoding); } + return fieldEntry; + } + + @Override + public FloatVectorValues getFloatVectorValues(String field) throws IOException { + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.FLOAT32); return OffHeapFloatVectorValues.load( fieldEntry.similarityFunction, defaultFlatVectorScorer, @@ -266,19 +271,7 @@ public final class Lucene95HnswVectorsReader extends KnnVectorsReader implements @Override public ByteVectorValues getByteVectorValues(String field) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null) { - throw new IllegalArgumentException("field=\"" + field + "\" not found"); - } - if (fieldEntry.vectorEncoding != VectorEncoding.BYTE) { - throw new IllegalArgumentException( - "field=\"" - + field - + "\" is encoded as: " - + fieldEntry.vectorEncoding - + " expected: " - + VectorEncoding.BYTE); - } + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.BYTE); return OffHeapByteVectorValues.load( fieldEntry.similarityFunction, defaultFlatVectorScorer, @@ -293,11 +286,8 @@ public final class Lucene95HnswVectorsReader extends KnnVectorsReader implements @Override public void search(String field, float[] target, KnnCollector knnCollector, Bits acceptDocs) throws IOException { - FieldEntry fieldEntry = fields.get(field); - - if (fieldEntry.size() == 0 - || knnCollector.k() == 0 - || fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.FLOAT32); + if (fieldEntry.size() == 0 || knnCollector.k() == 0) { return; } @@ -324,11 +314,8 @@ public final class Lucene95HnswVectorsReader extends KnnVectorsReader implements @Override public void search(String field, byte[] target, KnnCollector knnCollector, Bits acceptDocs) throws IOException { - FieldEntry fieldEntry = fields.get(field); - - if (fieldEntry.size() == 0 - || knnCollector.k() == 0 - || fieldEntry.vectorEncoding != VectorEncoding.BYTE) { + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.BYTE); + if (fieldEntry.size() == 0 || knnCollector.k() == 0) { return; } @@ -355,12 +342,12 @@ public final class Lucene95HnswVectorsReader extends KnnVectorsReader implements /** Get knn graph values; used for testing */ @Override public HnswGraph getGraph(String field) throws IOException { - FieldInfo info = fieldInfos.fieldInfo(field); - if (info == null) { - throw new IllegalArgumentException("No such field '" + field + "'"); + final FieldInfo info = fieldInfos.fieldInfo(field); + final FieldEntry entry; + if (info == null || (entry = fields.get(info.number)) == null) { + throw new IllegalArgumentException("field=\"" + field + "\" not found"); } - FieldEntry entry = fields.get(field); - if (entry != null && entry.vectorIndexLength > 0) { + if (entry.vectorIndexLength > 0) { return getGraph(entry); } else { return HnswGraph.EMPTY; diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextKnnVectorsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextKnnVectorsReader.java index 0a8c4836321..6c7c53a38d0 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextKnnVectorsReader.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextKnnVectorsReader.java @@ -26,8 +26,6 @@ import static org.apache.lucene.codecs.simpletext.SimpleTextKnnVectorsWriter.VEC import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.HashMap; -import java.util.Map; import org.apache.lucene.codecs.KnnVectorsReader; import org.apache.lucene.index.ByteVectorValues; import org.apache.lucene.index.CorruptIndexException; @@ -36,6 +34,7 @@ import org.apache.lucene.index.FloatVectorValues; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.KnnCollector; import org.apache.lucene.search.VectorScorer; @@ -63,7 +62,7 @@ public class SimpleTextKnnVectorsReader extends KnnVectorsReader { private final SegmentReadState readState; private final IndexInput dataIn; private final BytesRefBuilder scratch = new BytesRefBuilder(); - private final Map fieldEntries = new HashMap<>(); + private final IntObjectHashMap fieldEntries = new IntObjectHashMap<>(); SimpleTextKnnVectorsReader(SegmentReadState readState) throws IOException { this.readState = readState; @@ -91,9 +90,9 @@ public class SimpleTextKnnVectorsReader extends KnnVectorsReader { for (int i = 0; i < size; i++) { docIds[i] = readInt(in, EMPTY); } - assert fieldEntries.containsKey(fieldName) == false; + assert fieldEntries.containsKey(fieldNumber) == false; fieldEntries.put( - fieldName, + fieldNumber, new FieldEntry( dimension, vectorDataOffset, @@ -126,7 +125,7 @@ public class SimpleTextKnnVectorsReader extends KnnVectorsReader { throw new IllegalStateException( "KNN vectors readers should not be called on fields that don't enable KNN vectors"); } - FieldEntry fieldEntry = fieldEntries.get(field); + FieldEntry fieldEntry = fieldEntries.get(info.number); if (fieldEntry == null) { // mirror the handling in Lucene90VectorReader#getVectorValues // needed to pass TestSimpleTextKnnVectorsFormat#testDeleteAllVectorDocs @@ -159,7 +158,7 @@ public class SimpleTextKnnVectorsReader extends KnnVectorsReader { throw new IllegalStateException( "KNN vectors readers should not be called on fields that don't enable KNN vectors"); } - FieldEntry fieldEntry = fieldEntries.get(field); + FieldEntry fieldEntry = fieldEntries.get(info.number); if (fieldEntry == null) { // mirror the handling in Lucene90VectorReader#getVectorValues // needed to pass TestSimpleTextKnnVectorsFormat#testDeleteAllVectorDocs diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99FlatVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99FlatVectorsReader.java index b334298cb8f..9b42ddd0f26 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99FlatVectorsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99FlatVectorsReader.java @@ -21,8 +21,6 @@ import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader.readSi import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader.readVectorEncoding; import java.io.IOException; -import java.util.HashMap; -import java.util.Map; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.hnsw.FlatVectorsReader; import org.apache.lucene.codecs.hnsw.FlatVectorsScorer; @@ -38,6 +36,7 @@ import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.VectorEncoding; import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; @@ -56,13 +55,15 @@ public final class Lucene99FlatVectorsReader extends FlatVectorsReader { private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(Lucene99FlatVectorsFormat.class); - private final Map fields = new HashMap<>(); + private final IntObjectHashMap fields = new IntObjectHashMap<>(); private final IndexInput vectorData; + private final FieldInfos fieldInfos; public Lucene99FlatVectorsReader(SegmentReadState state, FlatVectorsScorer scorer) throws IOException { super(scorer); int versionMeta = readMetadata(state); + this.fieldInfos = state.fieldInfos; boolean success = false; try { vectorData = @@ -155,15 +156,13 @@ public final class Lucene99FlatVectorsReader extends FlatVectorsReader { throw new CorruptIndexException("Invalid field number: " + fieldNumber, meta); } FieldEntry fieldEntry = FieldEntry.create(meta, info); - fields.put(info.name, fieldEntry); + fields.put(info.number, fieldEntry); } } @Override public long ramBytesUsed() { - return Lucene99FlatVectorsReader.SHALLOW_SIZE - + RamUsageEstimator.sizeOfMap( - fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class)); + return Lucene99FlatVectorsReader.SHALLOW_SIZE + fields.ramBytesUsed(); } @Override @@ -171,21 +170,27 @@ public final class Lucene99FlatVectorsReader extends FlatVectorsReader { CodecUtil.checksumEntireFile(vectorData); } - @Override - public FloatVectorValues getFloatVectorValues(String field) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null) { + private FieldEntry getFieldEntry(String field, VectorEncoding expectedEncoding) { + final FieldInfo info = fieldInfos.fieldInfo(field); + final FieldEntry fieldEntry; + if (info == null || (fieldEntry = fields.get(info.number)) == null) { throw new IllegalArgumentException("field=\"" + field + "\" not found"); } - if (fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { + if (fieldEntry.vectorEncoding != expectedEncoding) { throw new IllegalArgumentException( "field=\"" + field + "\" is encoded as: " + fieldEntry.vectorEncoding + " expected: " - + VectorEncoding.FLOAT32); + + expectedEncoding); } + return fieldEntry; + } + + @Override + public FloatVectorValues getFloatVectorValues(String field) throws IOException { + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.FLOAT32); return OffHeapFloatVectorValues.load( fieldEntry.similarityFunction, vectorScorer, @@ -199,19 +204,7 @@ public final class Lucene99FlatVectorsReader extends FlatVectorsReader { @Override public ByteVectorValues getByteVectorValues(String field) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null) { - throw new IllegalArgumentException("field=\"" + field + "\" not found"); - } - if (fieldEntry.vectorEncoding != VectorEncoding.BYTE) { - throw new IllegalArgumentException( - "field=\"" - + field - + "\" is encoded as: " - + fieldEntry.vectorEncoding - + " expected: " - + VectorEncoding.BYTE); - } + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.BYTE); return OffHeapByteVectorValues.load( fieldEntry.similarityFunction, vectorScorer, @@ -225,10 +218,7 @@ public final class Lucene99FlatVectorsReader extends FlatVectorsReader { @Override public RandomVectorScorer getRandomVectorScorer(String field, float[] target) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null || fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { - return null; - } + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.FLOAT32); return vectorScorer.getRandomVectorScorer( fieldEntry.similarityFunction, OffHeapFloatVectorValues.load( @@ -245,10 +235,7 @@ public final class Lucene99FlatVectorsReader extends FlatVectorsReader { @Override public RandomVectorScorer getRandomVectorScorer(String field, byte[] target) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null || fieldEntry.vectorEncoding != VectorEncoding.BYTE) { - return null; - } + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.BYTE); return vectorScorer.getRandomVectorScorer( fieldEntry.similarityFunction, OffHeapByteVectorValues.load( diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java index f27a826e9c3..2a3088527f5 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java @@ -21,9 +21,7 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; import java.io.IOException; import java.util.Arrays; -import java.util.HashMap; import java.util.List; -import java.util.Map; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.KnnVectorsReader; import org.apache.lucene.codecs.hnsw.FlatVectorsReader; @@ -37,6 +35,7 @@ import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.VectorEncoding; import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.search.KnnCollector; import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.DataInput; @@ -70,7 +69,7 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader private final FlatVectorsReader flatVectorsReader; private final FieldInfos fieldInfos; - private final Map fields = new HashMap<>(); + private final IntObjectHashMap fields = new IntObjectHashMap<>(); private final IndexInput vectorIndex; public Lucene99HnswVectorsReader(SegmentReadState state, FlatVectorsReader flatVectorsReader) @@ -162,7 +161,7 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader } FieldEntry fieldEntry = readField(meta, info); validateFieldEntry(info, fieldEntry); - fields.put(info.name, fieldEntry); + fields.put(info.number, fieldEntry); } } @@ -225,8 +224,7 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader @Override public long ramBytesUsed() { return Lucene99HnswVectorsReader.SHALLOW_SIZE - + RamUsageEstimator.sizeOfMap( - fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class)) + + fields.ramBytesUsed() + flatVectorsReader.ramBytesUsed(); } @@ -246,25 +244,43 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader return flatVectorsReader.getByteVectorValues(field); } + private FieldEntry getFieldEntry(String field, VectorEncoding expectedEncoding) { + final FieldInfo info = fieldInfos.fieldInfo(field); + final FieldEntry fieldEntry; + if (info == null || (fieldEntry = fields.get(info.number)) == null) { + throw new IllegalArgumentException("field=\"" + field + "\" not found"); + } + if (fieldEntry.vectorEncoding != expectedEncoding) { + throw new IllegalArgumentException( + "field=\"" + + field + + "\" is encoded as: " + + fieldEntry.vectorEncoding + + " expected: " + + expectedEncoding); + } + return fieldEntry; + } + @Override public void search(String field, float[] target, KnnCollector knnCollector, Bits acceptDocs) throws IOException { + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.FLOAT32); search( - fields.get(field), + fieldEntry, knnCollector, acceptDocs, - VectorEncoding.FLOAT32, () -> flatVectorsReader.getRandomVectorScorer(field, target)); } @Override public void search(String field, byte[] target, KnnCollector knnCollector, Bits acceptDocs) throws IOException { + final FieldEntry fieldEntry = getFieldEntry(field, VectorEncoding.BYTE); search( - fields.get(field), + fieldEntry, knnCollector, acceptDocs, - VectorEncoding.BYTE, () -> flatVectorsReader.getRandomVectorScorer(field, target)); } @@ -272,13 +288,10 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader FieldEntry fieldEntry, KnnCollector knnCollector, Bits acceptDocs, - VectorEncoding vectorEncoding, IOSupplier scorerSupplier) throws IOException { - if (fieldEntry.size() == 0 - || knnCollector.k() == 0 - || fieldEntry.vectorEncoding != vectorEncoding) { + if (fieldEntry.size() == 0 || knnCollector.k() == 0) { return; } final RandomVectorScorer scorer = scorerSupplier.get(); @@ -304,12 +317,12 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader @Override public HnswGraph getGraph(String field) throws IOException { - FieldInfo info = fieldInfos.fieldInfo(field); - if (info == null) { - throw new IllegalArgumentException("No such field '" + field + "'"); + final FieldInfo info = fieldInfos.fieldInfo(field); + final FieldEntry entry; + if (info == null || (entry = fields.get(info.number)) == null) { + throw new IllegalArgumentException("field=\"" + field + "\" not found"); } - FieldEntry entry = fields.get(field); - if (entry != null && entry.vectorIndexLength > 0) { + if (entry.vectorIndexLength > 0) { return getGraph(entry); } else { return HnswGraph.EMPTY; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsReader.java index 32eea942e2a..712e9b91f9d 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsReader.java @@ -21,8 +21,6 @@ import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader.readSi import static org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader.readVectorEncoding; import java.io.IOException; -import java.util.HashMap; -import java.util.Map; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.hnsw.FlatVectorsReader; import org.apache.lucene.codecs.hnsw.FlatVectorsScorer; @@ -36,6 +34,7 @@ import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.VectorEncoding; import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.internal.hppc.IntObjectHashMap; import org.apache.lucene.search.VectorScorer; import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.IOContext; @@ -59,15 +58,17 @@ public final class Lucene99ScalarQuantizedVectorsReader extends FlatVectorsReade private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(Lucene99ScalarQuantizedVectorsReader.class); - private final Map fields = new HashMap<>(); + private final IntObjectHashMap fields = new IntObjectHashMap<>(); private final IndexInput quantizedVectorData; private final FlatVectorsReader rawVectorsReader; + private final FieldInfos fieldInfos; public Lucene99ScalarQuantizedVectorsReader( SegmentReadState state, FlatVectorsReader rawVectorsReader, FlatVectorsScorer scorer) throws IOException { super(scorer); this.rawVectorsReader = rawVectorsReader; + this.fieldInfos = state.fieldInfos; int versionMeta = -1; String metaFileName = IndexFileNames.segmentFileName( @@ -118,7 +119,7 @@ public final class Lucene99ScalarQuantizedVectorsReader extends FlatVectorsReade } FieldEntry fieldEntry = readField(meta, versionMeta, info); validateFieldEntry(info, fieldEntry); - fields.put(info.name, fieldEntry); + fields.put(info.number, fieldEntry); } } @@ -163,10 +164,10 @@ public final class Lucene99ScalarQuantizedVectorsReader extends FlatVectorsReade CodecUtil.checksumEntireFile(quantizedVectorData); } - @Override - public FloatVectorValues getFloatVectorValues(String field) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null) { + private FieldEntry getFieldEntry(String field) { + final FieldInfo info = fieldInfos.fieldInfo(field); + final FieldEntry fieldEntry; + if (info == null || (fieldEntry = fields.get(info.number)) == null) { throw new IllegalArgumentException("field=\"" + field + "\" not found"); } if (fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { @@ -178,6 +179,12 @@ public final class Lucene99ScalarQuantizedVectorsReader extends FlatVectorsReade + " expected: " + VectorEncoding.FLOAT32); } + return fieldEntry; + } + + @Override + public FloatVectorValues getFloatVectorValues(String field) throws IOException { + final FieldEntry fieldEntry = getFieldEntry(field); final FloatVectorValues rawVectorValues = rawVectorsReader.getFloatVectorValues(field); OffHeapQuantizedByteVectorValues quantizedByteVectorValues = OffHeapQuantizedByteVectorValues.load( @@ -241,10 +248,7 @@ public final class Lucene99ScalarQuantizedVectorsReader extends FlatVectorsReade @Override public RandomVectorScorer getRandomVectorScorer(String field, float[] target) throws IOException { - FieldEntry fieldEntry = fields.get(field); - if (fieldEntry == null || fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { - return null; - } + final FieldEntry fieldEntry = getFieldEntry(field); if (fieldEntry.scalarQuantizer == null) { return rawVectorsReader.getRandomVectorScorer(field, target); } @@ -275,12 +279,7 @@ public final class Lucene99ScalarQuantizedVectorsReader extends FlatVectorsReade @Override public long ramBytesUsed() { - long size = SHALLOW_SIZE; - size += - RamUsageEstimator.sizeOfMap( - fields, RamUsageEstimator.shallowSizeOfInstance(FieldEntry.class)); - size += rawVectorsReader.ramBytesUsed(); - return size; + return SHALLOW_SIZE + fields.ramBytesUsed() + rawVectorsReader.ramBytesUsed(); } private FieldEntry readField(IndexInput input, int versionMeta, FieldInfo info) @@ -301,11 +300,8 @@ public final class Lucene99ScalarQuantizedVectorsReader extends FlatVectorsReade } @Override - public QuantizedByteVectorValues getQuantizedVectorValues(String fieldName) throws IOException { - FieldEntry fieldEntry = fields.get(fieldName); - if (fieldEntry == null || fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { - return null; - } + public QuantizedByteVectorValues getQuantizedVectorValues(String field) throws IOException { + final FieldEntry fieldEntry = getFieldEntry(field); return OffHeapQuantizedByteVectorValues.load( fieldEntry.ordToDoc, fieldEntry.dimension, @@ -320,11 +316,8 @@ public final class Lucene99ScalarQuantizedVectorsReader extends FlatVectorsReade } @Override - public ScalarQuantizer getQuantizationState(String fieldName) { - FieldEntry fieldEntry = fields.get(fieldName); - if (fieldEntry == null || fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) { - return null; - } + public ScalarQuantizer getQuantizationState(String field) { + final FieldEntry fieldEntry = getFieldEntry(field); return fieldEntry.scalarQuantizer; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldKnnVectorsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldKnnVectorsFormat.java index 5dc4db8db6a..63bad6d48da 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldKnnVectorsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldKnnVectorsFormat.java @@ -19,7 +19,9 @@ package org.apache.lucene.codecs.perfield; import java.io.Closeable; import java.io.IOException; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.ServiceLoader; import org.apache.lucene.codecs.KnnFieldVectorsWriter; @@ -28,11 +30,14 @@ import org.apache.lucene.codecs.KnnVectorsReader; import org.apache.lucene.codecs.KnnVectorsWriter; import org.apache.lucene.index.ByteVectorValues; import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FloatVectorValues; import org.apache.lucene.index.MergeState; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.Sorter; +import org.apache.lucene.internal.hppc.IntObjectHashMap; +import org.apache.lucene.internal.hppc.ObjectCursor; import org.apache.lucene.search.KnnCollector; import org.apache.lucene.util.Bits; import org.apache.lucene.util.IOUtils; @@ -186,7 +191,8 @@ public abstract class PerFieldKnnVectorsFormat extends KnnVectorsFormat { /** VectorReader that can wrap multiple delegate readers, selected by field. */ public static class FieldsReader extends KnnVectorsReader { - private final Map fields = new HashMap<>(); + private final IntObjectHashMap fields = new IntObjectHashMap<>(); + private final FieldInfos fieldInfos; /** * Create a FieldsReader over a segment, opening VectorReaders for each KnnVectorsFormat @@ -196,7 +202,7 @@ public abstract class PerFieldKnnVectorsFormat extends KnnVectorsFormat { * @throws IOException if one of the delegate readers throws */ public FieldsReader(final SegmentReadState readState) throws IOException { - + this.fieldInfos = readState.fieldInfos; // Init each unique format: boolean success = false; Map formats = new HashMap<>(); @@ -221,7 +227,7 @@ public abstract class PerFieldKnnVectorsFormat extends KnnVectorsFormat { segmentSuffix, format.fieldsReader(new SegmentReadState(readState, segmentSuffix))); } - fields.put(fieldName, formats.get(segmentSuffix)); + fields.put(fi.number, formats.get(segmentSuffix)); } } } @@ -239,51 +245,69 @@ public abstract class PerFieldKnnVectorsFormat extends KnnVectorsFormat { * @param field the name of a numeric vector field */ public KnnVectorsReader getFieldReader(String field) { - return fields.get(field); + final FieldInfo info = fieldInfos.fieldInfo(field); + if (info == null) { + return null; + } + return fields.get(info.number); } @Override public void checkIntegrity() throws IOException { - for (KnnVectorsReader reader : fields.values()) { - reader.checkIntegrity(); + for (ObjectCursor cursor : fields.values()) { + cursor.value.checkIntegrity(); } } @Override public FloatVectorValues getFloatVectorValues(String field) throws IOException { - KnnVectorsReader knnVectorsReader = fields.get(field); - if (knnVectorsReader == null) { + final FieldInfo info = fieldInfos.fieldInfo(field); + final KnnVectorsReader reader; + if (info == null || (reader = fields.get(info.number)) == null) { return null; - } else { - return knnVectorsReader.getFloatVectorValues(field); } + return reader.getFloatVectorValues(field); } @Override public ByteVectorValues getByteVectorValues(String field) throws IOException { - KnnVectorsReader knnVectorsReader = fields.get(field); - if (knnVectorsReader == null) { + final FieldInfo info = fieldInfos.fieldInfo(field); + final KnnVectorsReader reader; + if (info == null || (reader = fields.get(info.number)) == null) { return null; - } else { - return knnVectorsReader.getByteVectorValues(field); } + return reader.getByteVectorValues(field); } @Override public void search(String field, float[] target, KnnCollector knnCollector, Bits acceptDocs) throws IOException { - fields.get(field).search(field, target, knnCollector, acceptDocs); + final FieldInfo info = fieldInfos.fieldInfo(field); + final KnnVectorsReader reader; + if (info == null || (reader = fields.get(info.number)) == null) { + return; + } + reader.search(field, target, knnCollector, acceptDocs); } @Override public void search(String field, byte[] target, KnnCollector knnCollector, Bits acceptDocs) throws IOException { - fields.get(field).search(field, target, knnCollector, acceptDocs); + final FieldInfo info = fieldInfos.fieldInfo(field); + final KnnVectorsReader reader; + if (info == null || (reader = fields.get(info.number)) == null) { + return; + } + reader.search(field, target, knnCollector, acceptDocs); } @Override public void close() throws IOException { - IOUtils.close(fields.values()); + List readers = new ArrayList<>(fields.size()); + for (ObjectCursor cursor : fields.values()) { + readers.add(cursor.value); + } + IOUtils.close(readers); } } From cfdd20f5bc8387ba24653ca2ba15aa5be10d0ae0 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 1 Nov 2024 07:49:09 +0100 Subject: [PATCH 24/37] Move postings back to int[] to take advantage of having more lanes per vector. (#13968) In Lucene 8.4, we updated postings to work on long[] arrays internally. This allowed us to workaround the lack of explicit vectorization (auto-vectorization doesn't detect all the scenarios that we would like to handle) support in the JVM by summing up two integers in one operation for instance. With explicit vectorization now available, it looks like we can get more benefits from the ability to compare multiple intetgers in one operations than from summing up two integers in one operation. Moving back to ints helps compare 2x more integers at once vs. longs. --- gradle/generation/forUtil.gradle | 55 +- lucene/CHANGES.txt | 5 + .../checksums/generateForDeltaUtil912.json | 4 + .../checksums/generateForUtil912.json | 4 + .../backward-codecs/src/java/module-info.java | 7 +- .../lucene100/Lucene100Codec.java | 6 +- .../lucene100}/package-info.java | 4 +- .../lucene912/ForDeltaUtil.java | 109 +- .../backward_codecs}/lucene912/ForUtil.java | 180 +- .../lucene912/Lucene912Codec.java | 1 - .../lucene912/Lucene912PostingsFormat.java | 43 +- .../lucene912/Lucene912PostingsReader.java | 88 +- .../backward_codecs}/lucene912/PForUtil.java | 9 +- .../lucene912/PostingsUtil.java | 2 +- .../lucene912/gen_ForDeltaUtil.py | 35 +- .../backward_codecs}/lucene912/gen_ForUtil.py | 43 +- .../lucene912/package-info.java | 416 +--- .../lucene99/Lucene99Codec.java | 2 +- .../services/org.apache.lucene.codecs.Codec | 1 + .../org.apache.lucene.codecs.PostingsFormat | 1 + .../lucene912/Lucene912PostingsWriter.java | 20 +- .../lucene912/Lucene912RWPostingsFormat.java | 69 + .../lucene912/TestForDeltaUtil.java | 7 +- .../lucene912/TestForUtil.java | 94 + .../TestLucene912PostingsFormat.java | 6 +- .../lucene912/TestPForUtil.java | 104 + .../lucene912/TestPostingsUtil.java | 2 +- ...ene99HnswScalarQuantizedVectorsFormat.java | 3 +- .../TestInt7HnswBackwardsCompatibility.java | 4 +- .../benchmark/jmh/AdvanceBenchmark.java | 14 +- .../jmh/PostingIndexInputBenchmark.java | 8 +- .../BlockTreeOrdsPostingsFormat.java | 10 +- .../codecs/memory/DirectPostingsFormat.java | 8 +- .../codecs/memory/FSTPostingsFormat.java | 8 +- .../DeltaBaseTermStateSerializer.java | 14 +- .../UniformSplitPostingsFormat.java | 8 +- .../codecs/uniformsplit/package-info.java | 2 +- .../bitvectors/TestHnswBitVectorsFormat.java | 4 +- .../lucene90/tests/MockTermStateFactory.java | 2 +- .../checksums/generateForDeltaUtil.json | 4 +- .../generated/checksums/generateForUtil.json | 4 +- lucene/core/src/java/module-info.java | 9 +- .../java/org/apache/lucene/codecs/Codec.java | 2 +- .../lucene/codecs/lucene101/ForDeltaUtil.java | 525 +++++ .../lucene/codecs/lucene101/ForUtil.java | 841 ++++++++ .../codecs/lucene101/Lucene101Codec.java | 217 ++ .../lucene101/Lucene101PostingsFormat.java | 492 +++++ .../lucene101/Lucene101PostingsReader.java | 1864 +++++++++++++++++ .../lucene101/Lucene101PostingsWriter.java | 681 ++++++ .../lucene/codecs/lucene101/PForUtil.java | 134 ++ .../PostingIndexInput.java | 14 +- .../lucene/codecs/lucene101/PostingsUtil.java | 74 + .../codecs/lucene101/gen_ForDeltaUtil.py | 377 ++++ .../lucene/codecs/lucene101/gen_ForUtil.py | 327 +++ .../package-info.java | 22 +- .../DefaultVectorUtilSupport.java | 2 +- .../vectorization/PostingDecodingUtil.java | 8 +- .../vectorization/VectorUtilSupport.java | 2 +- .../vectorization/VectorizationProvider.java | 4 +- .../org/apache/lucene/search/PhraseQuery.java | 10 +- .../lucene/store/BufferedIndexInput.java | 2 +- .../lucene/store/ByteBuffersDataInput.java | 2 +- .../lucene/store/ByteBuffersIndexInput.java | 2 +- .../org/apache/lucene/store/DataInput.java | 4 +- .../org/apache/lucene/store/DataOutput.java | 15 + .../org/apache/lucene/util/GroupVIntUtil.java | 145 +- .../org/apache/lucene/util/VectorUtil.java | 2 +- .../MemorySegmentPostingDecodingUtil.java | 35 +- .../PanamaVectorUtilSupport.java | 17 +- .../lucene/store/MemorySegmentIndexInput.java | 2 +- .../services/org.apache.lucene.codecs.Codec | 2 +- .../org.apache.lucene.codecs.PostingsFormat | 2 +- .../codecs/lucene101/TestForDeltaUtil.java | 92 + .../{lucene912 => lucene101}/TestForUtil.java | 8 +- .../TestLucene101PostingsFormat.java | 157 ++ .../TestPForUtil.java | 8 +- .../codecs/lucene101/TestPostingsUtil.java | 49 + ...ne90StoredFieldsFormatHighCompression.java | 8 +- ...estLucene99HnswQuantizedVectorsFormat.java | 14 +- ...stLucene99ScalarQuantizedVectorScorer.java | 4 +- ...tLucene99ScalarQuantizedVectorsFormat.java | 4 +- .../TestPostingDecodingUtil.java | 24 +- .../apache/lucene/util/TestVectorUtil.java | 10 +- .../intervals/TermIntervalsSource.java | 10 +- lucene/suggest/src/java/module-info.java | 3 +- .../document/Completion101PostingsFormat.java | 45 + .../document/Completion912PostingsFormat.java | 6 +- .../org.apache.lucene.codecs.PostingsFormat | 1 + .../suggest/document/TestSuggestField.java | 2 +- .../codecs/blockterms/LuceneFixedGap.java | 12 +- .../LuceneVarGapDocFreqInterval.java | 12 +- .../blockterms/LuceneVarGapFixedInterval.java | 12 +- .../mockrandom/MockRandomPostingsFormat.java | 8 +- .../UniformSplitRot13PostingsFormat.java | 8 +- .../util/TestRuleSetupAndRestoreClassEnv.java | 4 +- .../apache/lucene/tests/util/TestUtil.java | 10 +- 96 files changed, 6831 insertions(+), 925 deletions(-) create mode 100644 lucene/backward-codecs/src/generated/checksums/generateForDeltaUtil912.json create mode 100644 lucene/backward-codecs/src/generated/checksums/generateForUtil912.json rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene100/Lucene100Codec.java (97%) rename lucene/{core/src/java/org/apache/lucene/codecs/lucene912 => backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene100}/package-info.java (90%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene912/ForDeltaUtil.java (82%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene912/ForUtil.java (84%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene912/Lucene912PostingsFormat.java (94%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene912/Lucene912PostingsReader.java (94%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene912/PForUtil.java (94%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene912/PostingsUtil.java (97%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene912/gen_ForDeltaUtil.py (90%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/java/org/apache/lucene/backward_codecs}/lucene912/gen_ForUtil.py (88%) rename lucene/{core/src/java/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene912/Lucene912PostingsWriter.java (96%) create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/Lucene912RWPostingsFormat.java rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene912/TestForDeltaUtil.java (91%) create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestForUtil.java rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene912/TestLucene912PostingsFormat.java (96%) create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestPForUtil.java rename lucene/{core/src/test/org/apache/lucene/codecs => backward-codecs/src/test/org/apache/lucene/backward_codecs}/lucene912/TestPostingsUtil.java (97%) create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene101/ForDeltaUtil.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene101/ForUtil.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101Codec.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsFormat.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsReader.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsWriter.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene101/PForUtil.java rename lucene/core/src/java/org/apache/lucene/codecs/{lucene912 => lucene101}/PostingIndexInput.java (84%) create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene101/PostingsUtil.java create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene101/gen_ForDeltaUtil.py create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene101/gen_ForUtil.py rename lucene/core/src/java/org/apache/lucene/codecs/{lucene100 => lucene101}/package-info.java (96%) create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestForDeltaUtil.java rename lucene/core/src/test/org/apache/lucene/codecs/{lucene912 => lucene101}/TestForUtil.java (94%) create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestLucene101PostingsFormat.java rename lucene/core/src/test/org/apache/lucene/codecs/{lucene912 => lucene101}/TestPForUtil.java (94%) create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestPostingsUtil.java create mode 100644 lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion101PostingsFormat.java diff --git a/gradle/generation/forUtil.gradle b/gradle/generation/forUtil.gradle index 7ef3b53fd92..b55fd0204fd 100644 --- a/gradle/generation/forUtil.gradle +++ b/gradle/generation/forUtil.gradle @@ -23,7 +23,7 @@ configure(project(":lucene:core")) { description "Regenerate gen_ForUtil.py" group "generation" - def genDir = file("src/java/org/apache/lucene/codecs/lucene912") + def genDir = file("src/java/org/apache/lucene/codecs/lucene101") def genScript = file("${genDir}/gen_ForUtil.py") def genOutput = file("${genDir}/ForUtil.java") @@ -48,7 +48,7 @@ configure(project(":lucene:core")) { description "Regenerate gen_ForDeltaUtil.py" group "generation" - def genDir = file("src/java/org/apache/lucene/codecs/lucene912") + def genDir = file("src/java/org/apache/lucene/codecs/lucene101") def genScript = file("${genDir}/gen_ForDeltaUtil.py") def genOutput = file("${genDir}/ForDeltaUtil.java") @@ -68,6 +68,7 @@ configure(project(":lucene:core")) { andThenTasks: ["spotlessJava", "spotlessJavaApply"], mustRunBefore: [ "compileJava" ] ]) + } configure(project(":lucene:backward-codecs")) { @@ -146,5 +147,55 @@ configure(project(":lucene:backward-codecs")) { andThenTasks: ["spotlessJava", "spotlessJavaApply"], mustRunBefore: [ "compileJava" ] ]) + + task generateForUtil912Internal() { + description "Regenerate gen_ForUtil.py" + group "generation" + + def genDir = file("src/java/org/apache/lucene/backward_codecs/lucene912") + def genScript = file("${genDir}/gen_ForUtil.py") + def genOutput = file("${genDir}/ForUtil.java") + + inputs.file genScript + outputs.file genOutput + + doLast { + quietExec { + workingDir genDir + executable project.externalTool("python3") + args = [ '-B', genScript ] + } + } + } + + regenerate.dependsOn wrapWithPersistentChecksums(generateForUtil912Internal, [ + andThenTasks: ["spotlessJava", "spotlessJavaApply"], + mustRunBefore: [ "compileJava" ] + ]) + + task generateForDeltaUtil912Internal() { + description "Regenerate gen_ForDeltaUtil.py" + group "generation" + + def genDir = file("src/java/org/apache/lucene/backward_codecs/lucene912") + def genScript = file("${genDir}/gen_ForDeltaUtil.py") + def genOutput = file("${genDir}/ForDeltaUtil.java") + + inputs.file genScript + outputs.file genOutput + + doLast { + quietExec { + workingDir genDir + executable project.externalTool("python3") + args = [ '-B', genScript ] + } + } + } + + regenerate.dependsOn wrapWithPersistentChecksums(generateForDeltaUtil912Internal, [ + andThenTasks: ["spotlessJava", "spotlessJavaApply"], + mustRunBefore: [ "compileJava" ] + ]) } diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index b512720bc9c..2bfde84eabe 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -84,6 +84,11 @@ Optimizations * GITHUB#13763: Replace Map with IntObjectHashMap for KnnVectorsReader (Pan Guixin) +* GITHUB#13968: Switch postings from storing doc IDs in a long[] to an int[]. + Lucene 8.4 had moved to a long[] to help speed up block decoding by using + longs that would pack two integers. We are now moving back to integers to be + able to take advantage of 2x more lanes with the vector API. (Adrien Grand) + Bug Fixes --------------------- * GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended diff --git a/lucene/backward-codecs/src/generated/checksums/generateForDeltaUtil912.json b/lucene/backward-codecs/src/generated/checksums/generateForDeltaUtil912.json new file mode 100644 index 00000000000..c8c19f1c3b5 --- /dev/null +++ b/lucene/backward-codecs/src/generated/checksums/generateForDeltaUtil912.json @@ -0,0 +1,4 @@ +{ + "lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/ForDeltaUtil.java": "b81961f0b277b1458ca259e0d23ccc4eeeb47fe7", + "lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/gen_ForDeltaUtil.py": "3191d7591309b7876c5c709fb9375af5b87c2ef8" +} \ No newline at end of file diff --git a/lucene/backward-codecs/src/generated/checksums/generateForUtil912.json b/lucene/backward-codecs/src/generated/checksums/generateForUtil912.json new file mode 100644 index 00000000000..e8c1881a925 --- /dev/null +++ b/lucene/backward-codecs/src/generated/checksums/generateForUtil912.json @@ -0,0 +1,4 @@ +{ + "lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/ForUtil.java": "e6db3c665dfebca8b93eb6b4651d2eb3af637b02", + "lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/gen_ForUtil.py": "993ecc9cf7ea821963384070669695257b16e040" +} \ No newline at end of file diff --git a/lucene/backward-codecs/src/java/module-info.java b/lucene/backward-codecs/src/java/module-info.java index 4607c21eb7e..41057c95bbf 100644 --- a/lucene/backward-codecs/src/java/module-info.java +++ b/lucene/backward-codecs/src/java/module-info.java @@ -37,6 +37,7 @@ module org.apache.lucene.backward_codecs { exports org.apache.lucene.backward_codecs.lucene95; exports org.apache.lucene.backward_codecs.lucene99; exports org.apache.lucene.backward_codecs.lucene912; + exports org.apache.lucene.backward_codecs.lucene100; exports org.apache.lucene.backward_codecs.packed; exports org.apache.lucene.backward_codecs.store; @@ -46,7 +47,8 @@ module org.apache.lucene.backward_codecs { org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat, org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat, org.apache.lucene.backward_codecs.lucene90.Lucene90PostingsFormat, - org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat; + org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat, + org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat; provides org.apache.lucene.codecs.KnnVectorsFormat with org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat, org.apache.lucene.backward_codecs.lucene91.Lucene91HnswVectorsFormat, @@ -64,5 +66,6 @@ module org.apache.lucene.backward_codecs { org.apache.lucene.backward_codecs.lucene94.Lucene94Codec, org.apache.lucene.backward_codecs.lucene95.Lucene95Codec, org.apache.lucene.backward_codecs.lucene99.Lucene99Codec, - org.apache.lucene.backward_codecs.lucene912.Lucene912Codec; + org.apache.lucene.backward_codecs.lucene912.Lucene912Codec, + org.apache.lucene.backward_codecs.lucene100.Lucene100Codec; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene100/Lucene100Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene100/Lucene100Codec.java similarity index 97% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene100/Lucene100Codec.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene100/Lucene100Codec.java index 97dc23bc07b..14ca88a98a5 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene100/Lucene100Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene100/Lucene100Codec.java @@ -14,9 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene100; +package org.apache.lucene.backward_codecs.lucene100; import java.util.Objects; +import org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.DocValuesFormat; @@ -37,7 +38,6 @@ import org.apache.lucene.codecs.lucene90.Lucene90NormsFormat; import org.apache.lucene.codecs.lucene90.Lucene90PointsFormat; import org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat; import org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; import org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat; @@ -50,7 +50,7 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; * *

If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}. * - * @see org.apache.lucene.codecs.lucene100 package documentation for file format details. + * @see org.apache.lucene.backward_codecs.lucene100 package documentation for file format details. * @lucene.experimental */ public class Lucene100Codec extends Codec { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene100/package-info.java similarity index 90% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene912/package-info.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene100/package-info.java index b9ddb1227b1..dd2af3acdbb 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/package-info.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene100/package-info.java @@ -15,5 +15,5 @@ * limitations under the License. */ -/** Lucene 9.12 file format. */ -package org.apache.lucene.codecs.lucene912; +/** Lucene 10.0 file format. */ +package org.apache.lucene.backward_codecs.lucene100; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/ForDeltaUtil.java similarity index 82% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/ForDeltaUtil.java index fe0378b388b..f87ffc13586 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/ForDeltaUtil.java @@ -16,13 +16,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.backward_codecs.lucene912; -import static org.apache.lucene.codecs.lucene912.ForUtil.*; +import static org.apache.lucene.backward_codecs.lucene912.ForUtil.*; import java.io.IOException; -import org.apache.lucene.internal.vectorization.PostingDecodingUtil; import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.packed.PackedInts; /** @@ -30,7 +30,7 @@ import org.apache.lucene.util.packed.PackedInts; * SIMD-like speedups. If bitsPerValue <= 4 then we pack 8 ints per long else if bitsPerValue * <= 11 we pack 4 ints per long else we pack 2 ints per long */ -public final class ForDeltaUtil { +final class ForDeltaUtil { private static final int ONE_BLOCK_SIZE_FOURTH = BLOCK_SIZE / 4; private static final int TWO_BLOCK_SIZE_FOURTHS = BLOCK_SIZE / 2; @@ -272,125 +272,124 @@ public final class ForDeltaUtil { } /** Decode deltas, compute the prefix sum and add {@code base} to all decoded longs. */ - void decodeAndPrefixSum(PostingDecodingUtil pdu, long base, long[] longs) throws IOException { - final int bitsPerValue = Byte.toUnsignedInt(pdu.in.readByte()); + void decodeAndPrefixSum(IndexInput in, long base, long[] longs) throws IOException { + final int bitsPerValue = Byte.toUnsignedInt(in.readByte()); if (bitsPerValue == 0) { prefixSumOfOnes(longs, base); } else { - decodeAndPrefixSum(bitsPerValue, pdu, base, longs); + decodeAndPrefixSum(bitsPerValue, in, base, longs); } } /** Delta-decode 128 integers into {@code longs}. */ - void decodeAndPrefixSum(int bitsPerValue, PostingDecodingUtil pdu, long base, long[] longs) + void decodeAndPrefixSum(int bitsPerValue, IndexInput in, long base, long[] longs) throws IOException { switch (bitsPerValue) { case 1: - decode1(pdu, longs); + decode1(in, longs); prefixSum8(longs, base); break; case 2: - decode2(pdu, longs); + decode2(in, longs); prefixSum8(longs, base); break; case 3: - decode3(pdu, tmp, longs); + decode3(in, tmp, longs); prefixSum8(longs, base); break; case 4: - decode4(pdu, longs); + decode4(in, longs); prefixSum8(longs, base); break; case 5: - decode5To16(pdu, tmp, longs); + decode5To16(in, tmp, longs); prefixSum16(longs, base); break; case 6: - decode6To16(pdu, tmp, longs); + decode6To16(in, tmp, longs); prefixSum16(longs, base); break; case 7: - decode7To16(pdu, tmp, longs); + decode7To16(in, tmp, longs); prefixSum16(longs, base); break; case 8: - decode8To16(pdu, longs); + decode8To16(in, longs); prefixSum16(longs, base); break; case 9: - decode9(pdu, tmp, longs); + decode9(in, tmp, longs); prefixSum16(longs, base); break; case 10: - decode10(pdu, tmp, longs); + decode10(in, tmp, longs); prefixSum16(longs, base); break; case 11: - decode11(pdu, tmp, longs); + decode11(in, tmp, longs); prefixSum16(longs, base); break; case 12: - decode12To32(pdu, tmp, longs); + decode12To32(in, tmp, longs); prefixSum32(longs, base); break; case 13: - decode13To32(pdu, tmp, longs); + decode13To32(in, tmp, longs); prefixSum32(longs, base); break; case 14: - decode14To32(pdu, tmp, longs); + decode14To32(in, tmp, longs); prefixSum32(longs, base); break; case 15: - decode15To32(pdu, tmp, longs); + decode15To32(in, tmp, longs); prefixSum32(longs, base); break; case 16: - decode16To32(pdu, longs); + decode16To32(in, longs); prefixSum32(longs, base); break; case 17: - decode17(pdu, tmp, longs); + decode17(in, tmp, longs); prefixSum32(longs, base); break; case 18: - decode18(pdu, tmp, longs); + decode18(in, tmp, longs); prefixSum32(longs, base); break; case 19: - decode19(pdu, tmp, longs); + decode19(in, tmp, longs); prefixSum32(longs, base); break; case 20: - decode20(pdu, tmp, longs); + decode20(in, tmp, longs); prefixSum32(longs, base); break; case 21: - decode21(pdu, tmp, longs); + decode21(in, tmp, longs); prefixSum32(longs, base); break; case 22: - decode22(pdu, tmp, longs); + decode22(in, tmp, longs); prefixSum32(longs, base); break; case 23: - decode23(pdu, tmp, longs); + decode23(in, tmp, longs); prefixSum32(longs, base); break; case 24: - decode24(pdu, tmp, longs); + decode24(in, tmp, longs); prefixSum32(longs, base); break; default: - decodeSlow(bitsPerValue, pdu, tmp, longs); + decodeSlow(bitsPerValue, in, tmp, longs); prefixSum32(longs, base); break; } } - private static void decode5To16(PostingDecodingUtil pdu, long[] tmp, long[] longs) - throws IOException { - pdu.splitLongs(10, longs, 11, 5, MASK16_5, tmp, 0, MASK16_1); + private static void decode5To16(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 10, longs, 11, 5, MASK16_5, tmp, 0, MASK16_1); for (int iter = 0, tmpIdx = 0, longsIdx = 30; iter < 2; ++iter, tmpIdx += 5, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 4; l0 |= tmp[tmpIdx + 1] << 3; @@ -401,9 +400,8 @@ public final class ForDeltaUtil { } } - private static void decode6To16(PostingDecodingUtil pdu, long[] tmp, long[] longs) - throws IOException { - pdu.splitLongs(12, longs, 10, 6, MASK16_6, tmp, 0, MASK16_4); + private static void decode6To16(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 12, longs, 10, 6, MASK16_6, tmp, 0, MASK16_4); for (int iter = 0, tmpIdx = 0, longsIdx = 24; iter < 4; ++iter, tmpIdx += 3, longsIdx += 2) { long l0 = tmp[tmpIdx + 0] << 2; l0 |= (tmp[tmpIdx + 1] >>> 2) & MASK16_2; @@ -414,9 +412,8 @@ public final class ForDeltaUtil { } } - private static void decode7To16(PostingDecodingUtil pdu, long[] tmp, long[] longs) - throws IOException { - pdu.splitLongs(14, longs, 9, 7, MASK16_7, tmp, 0, MASK16_2); + private static void decode7To16(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 14, longs, 9, 7, MASK16_7, tmp, 0, MASK16_2); for (int iter = 0, tmpIdx = 0, longsIdx = 28; iter < 2; ++iter, tmpIdx += 7, longsIdx += 2) { long l0 = tmp[tmpIdx + 0] << 5; l0 |= tmp[tmpIdx + 1] << 3; @@ -431,13 +428,12 @@ public final class ForDeltaUtil { } } - private static void decode8To16(PostingDecodingUtil pdu, long[] longs) throws IOException { - pdu.splitLongs(16, longs, 8, 8, MASK16_8, longs, 16, MASK16_8); + private static void decode8To16(IndexInput in, long[] longs) throws IOException { + splitLongs(in, 16, longs, 8, 8, MASK16_8, longs, 16, MASK16_8); } - private static void decode12To32(PostingDecodingUtil pdu, long[] tmp, long[] longs) - throws IOException { - pdu.splitLongs(24, longs, 20, 12, MASK32_12, tmp, 0, MASK32_8); + private static void decode12To32(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 24, longs, 20, 12, MASK32_12, tmp, 0, MASK32_8); for (int iter = 0, tmpIdx = 0, longsIdx = 48; iter < 8; ++iter, tmpIdx += 3, longsIdx += 2) { long l0 = tmp[tmpIdx + 0] << 4; l0 |= (tmp[tmpIdx + 1] >>> 4) & MASK32_4; @@ -448,9 +444,8 @@ public final class ForDeltaUtil { } } - private static void decode13To32(PostingDecodingUtil pdu, long[] tmp, long[] longs) - throws IOException { - pdu.splitLongs(26, longs, 19, 13, MASK32_13, tmp, 0, MASK32_6); + private static void decode13To32(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 26, longs, 19, 13, MASK32_13, tmp, 0, MASK32_6); for (int iter = 0, tmpIdx = 0, longsIdx = 52; iter < 2; ++iter, tmpIdx += 13, longsIdx += 6) { long l0 = tmp[tmpIdx + 0] << 7; l0 |= tmp[tmpIdx + 1] << 1; @@ -479,9 +474,8 @@ public final class ForDeltaUtil { } } - private static void decode14To32(PostingDecodingUtil pdu, long[] tmp, long[] longs) - throws IOException { - pdu.splitLongs(28, longs, 18, 14, MASK32_14, tmp, 0, MASK32_4); + private static void decode14To32(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 28, longs, 18, 14, MASK32_14, tmp, 0, MASK32_4); for (int iter = 0, tmpIdx = 0, longsIdx = 56; iter < 4; ++iter, tmpIdx += 7, longsIdx += 2) { long l0 = tmp[tmpIdx + 0] << 10; l0 |= tmp[tmpIdx + 1] << 6; @@ -496,9 +490,8 @@ public final class ForDeltaUtil { } } - private static void decode15To32(PostingDecodingUtil pdu, long[] tmp, long[] longs) - throws IOException { - pdu.splitLongs(30, longs, 17, 15, MASK32_15, tmp, 0, MASK32_2); + private static void decode15To32(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 30, longs, 17, 15, MASK32_15, tmp, 0, MASK32_2); for (int iter = 0, tmpIdx = 0, longsIdx = 60; iter < 2; ++iter, tmpIdx += 15, longsIdx += 2) { long l0 = tmp[tmpIdx + 0] << 13; l0 |= tmp[tmpIdx + 1] << 11; @@ -521,7 +514,7 @@ public final class ForDeltaUtil { } } - private static void decode16To32(PostingDecodingUtil pdu, long[] longs) throws IOException { - pdu.splitLongs(32, longs, 16, 16, MASK32_16, longs, 32, MASK32_16); + private static void decode16To32(IndexInput in, long[] longs) throws IOException { + splitLongs(in, 32, longs, 16, 16, MASK32_16, longs, 32, MASK32_16); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/ForUtil.java similarity index 84% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/ForUtil.java index fd5afa235f7..9f38052b2eb 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/ForUtil.java @@ -16,18 +16,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.backward_codecs.lucene912; import java.io.IOException; -import org.apache.lucene.internal.vectorization.PostingDecodingUtil; import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexInput; /** * Inspired from https://fulmicoton.com/posts/bitpacking/ Encodes multiple integers in a long to get * SIMD-like speedups. If bitsPerValue <= 8 then we pack 8 ints per long else if bitsPerValue * <= 16 we pack 4 ints per long else we pack 2 ints per long */ -public final class ForUtil { +final class ForUtil { public static final int BLOCK_SIZE = 128; static final int BLOCK_SIZE_LOG2 = 7; @@ -196,11 +196,11 @@ public final class ForUtil { return bitsPerValue << (BLOCK_SIZE_LOG2 - 3); } - static void decodeSlow(int bitsPerValue, PostingDecodingUtil pdu, long[] tmp, long[] longs) + static void decodeSlow(int bitsPerValue, IndexInput in, long[] tmp, long[] longs) throws IOException { final int numLongs = bitsPerValue << 1; final long mask = MASKS32[bitsPerValue]; - pdu.splitLongs(numLongs, longs, 32 - bitsPerValue, 32, mask, tmp, 0, -1L); + splitLongs(in, numLongs, longs, 32 - bitsPerValue, 32, mask, tmp, 0, -1L); final int remainingBitsPerLong = 32 - bitsPerValue; final long mask32RemainingBitsPerLong = MASKS32[remainingBitsPerLong]; int tmpIdx = 0; @@ -222,6 +222,28 @@ public final class ForUtil { } } + static void splitLongs( + IndexInput in, + int count, + long[] b, + int bShift, + int dec, + long bMask, + long[] c, + int cIndex, + long cMask) + throws IOException { + // takes advantage of the C2 compiler's loop unrolling and auto-vectorization. + in.readLongs(c, cIndex, count); + int maxIter = (bShift - 1) / dec; + for (int i = 0; i < count; ++i) { + for (int j = 0; j <= maxIter; ++j) { + b[count * j + i] = (c[cIndex + i] >>> (bShift - j * dec)) & bMask; + } + c[cIndex + i] &= cMask; + } + } + static final long[] MASKS8 = new long[8]; static final long[] MASKS16 = new long[16]; static final long[] MASKS32 = new long[32]; @@ -288,121 +310,121 @@ public final class ForUtil { static final long MASK32_24 = MASKS32[24]; /** Decode 128 integers into {@code longs}. */ - void decode(int bitsPerValue, PostingDecodingUtil pdu, long[] longs) throws IOException { + void decode(int bitsPerValue, IndexInput in, long[] longs) throws IOException { switch (bitsPerValue) { case 1: - decode1(pdu, longs); + decode1(in, longs); expand8(longs); break; case 2: - decode2(pdu, longs); + decode2(in, longs); expand8(longs); break; case 3: - decode3(pdu, tmp, longs); + decode3(in, tmp, longs); expand8(longs); break; case 4: - decode4(pdu, longs); + decode4(in, longs); expand8(longs); break; case 5: - decode5(pdu, tmp, longs); + decode5(in, tmp, longs); expand8(longs); break; case 6: - decode6(pdu, tmp, longs); + decode6(in, tmp, longs); expand8(longs); break; case 7: - decode7(pdu, tmp, longs); + decode7(in, tmp, longs); expand8(longs); break; case 8: - decode8(pdu, longs); + decode8(in, longs); expand8(longs); break; case 9: - decode9(pdu, tmp, longs); + decode9(in, tmp, longs); expand16(longs); break; case 10: - decode10(pdu, tmp, longs); + decode10(in, tmp, longs); expand16(longs); break; case 11: - decode11(pdu, tmp, longs); + decode11(in, tmp, longs); expand16(longs); break; case 12: - decode12(pdu, tmp, longs); + decode12(in, tmp, longs); expand16(longs); break; case 13: - decode13(pdu, tmp, longs); + decode13(in, tmp, longs); expand16(longs); break; case 14: - decode14(pdu, tmp, longs); + decode14(in, tmp, longs); expand16(longs); break; case 15: - decode15(pdu, tmp, longs); + decode15(in, tmp, longs); expand16(longs); break; case 16: - decode16(pdu, longs); + decode16(in, longs); expand16(longs); break; case 17: - decode17(pdu, tmp, longs); + decode17(in, tmp, longs); expand32(longs); break; case 18: - decode18(pdu, tmp, longs); + decode18(in, tmp, longs); expand32(longs); break; case 19: - decode19(pdu, tmp, longs); + decode19(in, tmp, longs); expand32(longs); break; case 20: - decode20(pdu, tmp, longs); + decode20(in, tmp, longs); expand32(longs); break; case 21: - decode21(pdu, tmp, longs); + decode21(in, tmp, longs); expand32(longs); break; case 22: - decode22(pdu, tmp, longs); + decode22(in, tmp, longs); expand32(longs); break; case 23: - decode23(pdu, tmp, longs); + decode23(in, tmp, longs); expand32(longs); break; case 24: - decode24(pdu, tmp, longs); + decode24(in, tmp, longs); expand32(longs); break; default: - decodeSlow(bitsPerValue, pdu, tmp, longs); + decodeSlow(bitsPerValue, in, tmp, longs); expand32(longs); break; } } - static void decode1(PostingDecodingUtil pdu, long[] longs) throws IOException { - pdu.splitLongs(2, longs, 7, 1, MASK8_1, longs, 14, MASK8_1); + static void decode1(IndexInput in, long[] longs) throws IOException { + splitLongs(in, 2, longs, 7, 1, MASK8_1, longs, 14, MASK8_1); } - static void decode2(PostingDecodingUtil pdu, long[] longs) throws IOException { - pdu.splitLongs(4, longs, 6, 2, MASK8_2, longs, 12, MASK8_2); + static void decode2(IndexInput in, long[] longs) throws IOException { + splitLongs(in, 4, longs, 6, 2, MASK8_2, longs, 12, MASK8_2); } - static void decode3(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(6, longs, 5, 3, MASK8_3, tmp, 0, MASK8_2); + static void decode3(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 6, longs, 5, 3, MASK8_3, tmp, 0, MASK8_2); for (int iter = 0, tmpIdx = 0, longsIdx = 12; iter < 2; ++iter, tmpIdx += 3, longsIdx += 2) { long l0 = tmp[tmpIdx + 0] << 1; l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK8_1; @@ -413,12 +435,12 @@ public final class ForUtil { } } - static void decode4(PostingDecodingUtil pdu, long[] longs) throws IOException { - pdu.splitLongs(8, longs, 4, 4, MASK8_4, longs, 8, MASK8_4); + static void decode4(IndexInput in, long[] longs) throws IOException { + splitLongs(in, 8, longs, 4, 4, MASK8_4, longs, 8, MASK8_4); } - static void decode5(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(10, longs, 3, 5, MASK8_5, tmp, 0, MASK8_3); + static void decode5(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 10, longs, 3, 5, MASK8_5, tmp, 0, MASK8_3); for (int iter = 0, tmpIdx = 0, longsIdx = 10; iter < 2; ++iter, tmpIdx += 5, longsIdx += 3) { long l0 = tmp[tmpIdx + 0] << 2; l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK8_2; @@ -433,8 +455,8 @@ public final class ForUtil { } } - static void decode6(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(12, longs, 2, 6, MASK8_6, tmp, 0, MASK8_2); + static void decode6(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 12, longs, 2, 6, MASK8_6, tmp, 0, MASK8_2); for (int iter = 0, tmpIdx = 0, longsIdx = 12; iter < 4; ++iter, tmpIdx += 3, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 4; l0 |= tmp[tmpIdx + 1] << 2; @@ -443,8 +465,8 @@ public final class ForUtil { } } - static void decode7(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(14, longs, 1, 7, MASK8_7, tmp, 0, MASK8_1); + static void decode7(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 14, longs, 1, 7, MASK8_7, tmp, 0, MASK8_1); for (int iter = 0, tmpIdx = 0, longsIdx = 14; iter < 2; ++iter, tmpIdx += 7, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 6; l0 |= tmp[tmpIdx + 1] << 5; @@ -457,12 +479,12 @@ public final class ForUtil { } } - static void decode8(PostingDecodingUtil pdu, long[] longs) throws IOException { - pdu.in.readLongs(longs, 0, 16); + static void decode8(IndexInput in, long[] longs) throws IOException { + in.readLongs(longs, 0, 16); } - static void decode9(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(18, longs, 7, 9, MASK16_9, tmp, 0, MASK16_7); + static void decode9(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 18, longs, 7, 9, MASK16_9, tmp, 0, MASK16_7); for (int iter = 0, tmpIdx = 0, longsIdx = 18; iter < 2; ++iter, tmpIdx += 9, longsIdx += 7) { long l0 = tmp[tmpIdx + 0] << 2; l0 |= (tmp[tmpIdx + 1] >>> 5) & MASK16_2; @@ -489,8 +511,8 @@ public final class ForUtil { } } - static void decode10(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(20, longs, 6, 10, MASK16_10, tmp, 0, MASK16_6); + static void decode10(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 20, longs, 6, 10, MASK16_10, tmp, 0, MASK16_6); for (int iter = 0, tmpIdx = 0, longsIdx = 20; iter < 4; ++iter, tmpIdx += 5, longsIdx += 3) { long l0 = tmp[tmpIdx + 0] << 4; l0 |= (tmp[tmpIdx + 1] >>> 2) & MASK16_4; @@ -505,8 +527,8 @@ public final class ForUtil { } } - static void decode11(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(22, longs, 5, 11, MASK16_11, tmp, 0, MASK16_5); + static void decode11(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 22, longs, 5, 11, MASK16_11, tmp, 0, MASK16_5); for (int iter = 0, tmpIdx = 0, longsIdx = 22; iter < 2; ++iter, tmpIdx += 11, longsIdx += 5) { long l0 = tmp[tmpIdx + 0] << 6; l0 |= tmp[tmpIdx + 1] << 1; @@ -531,8 +553,8 @@ public final class ForUtil { } } - static void decode12(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(24, longs, 4, 12, MASK16_12, tmp, 0, MASK16_4); + static void decode12(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 24, longs, 4, 12, MASK16_12, tmp, 0, MASK16_4); for (int iter = 0, tmpIdx = 0, longsIdx = 24; iter < 8; ++iter, tmpIdx += 3, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 8; l0 |= tmp[tmpIdx + 1] << 4; @@ -541,8 +563,8 @@ public final class ForUtil { } } - static void decode13(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(26, longs, 3, 13, MASK16_13, tmp, 0, MASK16_3); + static void decode13(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 26, longs, 3, 13, MASK16_13, tmp, 0, MASK16_3); for (int iter = 0, tmpIdx = 0, longsIdx = 26; iter < 2; ++iter, tmpIdx += 13, longsIdx += 3) { long l0 = tmp[tmpIdx + 0] << 10; l0 |= tmp[tmpIdx + 1] << 7; @@ -565,8 +587,8 @@ public final class ForUtil { } } - static void decode14(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(28, longs, 2, 14, MASK16_14, tmp, 0, MASK16_2); + static void decode14(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 28, longs, 2, 14, MASK16_14, tmp, 0, MASK16_2); for (int iter = 0, tmpIdx = 0, longsIdx = 28; iter < 4; ++iter, tmpIdx += 7, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 12; l0 |= tmp[tmpIdx + 1] << 10; @@ -579,8 +601,8 @@ public final class ForUtil { } } - static void decode15(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(30, longs, 1, 15, MASK16_15, tmp, 0, MASK16_1); + static void decode15(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 30, longs, 1, 15, MASK16_15, tmp, 0, MASK16_1); for (int iter = 0, tmpIdx = 0, longsIdx = 30; iter < 2; ++iter, tmpIdx += 15, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 14; l0 |= tmp[tmpIdx + 1] << 13; @@ -601,12 +623,12 @@ public final class ForUtil { } } - static void decode16(PostingDecodingUtil pdu, long[] longs) throws IOException { - pdu.in.readLongs(longs, 0, 32); + static void decode16(IndexInput in, long[] longs) throws IOException { + in.readLongs(longs, 0, 32); } - static void decode17(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(34, longs, 15, 17, MASK32_17, tmp, 0, MASK32_15); + static void decode17(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 34, longs, 15, 17, MASK32_17, tmp, 0, MASK32_15); for (int iter = 0, tmpIdx = 0, longsIdx = 34; iter < 2; ++iter, tmpIdx += 17, longsIdx += 15) { long l0 = tmp[tmpIdx + 0] << 2; l0 |= (tmp[tmpIdx + 1] >>> 13) & MASK32_2; @@ -657,8 +679,8 @@ public final class ForUtil { } } - static void decode18(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(36, longs, 14, 18, MASK32_18, tmp, 0, MASK32_14); + static void decode18(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 36, longs, 14, 18, MASK32_18, tmp, 0, MASK32_14); for (int iter = 0, tmpIdx = 0, longsIdx = 36; iter < 4; ++iter, tmpIdx += 9, longsIdx += 7) { long l0 = tmp[tmpIdx + 0] << 4; l0 |= (tmp[tmpIdx + 1] >>> 10) & MASK32_4; @@ -685,8 +707,8 @@ public final class ForUtil { } } - static void decode19(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(38, longs, 13, 19, MASK32_19, tmp, 0, MASK32_13); + static void decode19(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 38, longs, 13, 19, MASK32_19, tmp, 0, MASK32_13); for (int iter = 0, tmpIdx = 0, longsIdx = 38; iter < 2; ++iter, tmpIdx += 19, longsIdx += 13) { long l0 = tmp[tmpIdx + 0] << 6; l0 |= (tmp[tmpIdx + 1] >>> 7) & MASK32_6; @@ -735,8 +757,8 @@ public final class ForUtil { } } - static void decode20(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(40, longs, 12, 20, MASK32_20, tmp, 0, MASK32_12); + static void decode20(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 40, longs, 12, 20, MASK32_20, tmp, 0, MASK32_12); for (int iter = 0, tmpIdx = 0, longsIdx = 40; iter < 8; ++iter, tmpIdx += 5, longsIdx += 3) { long l0 = tmp[tmpIdx + 0] << 8; l0 |= (tmp[tmpIdx + 1] >>> 4) & MASK32_8; @@ -751,8 +773,8 @@ public final class ForUtil { } } - static void decode21(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(42, longs, 11, 21, MASK32_21, tmp, 0, MASK32_11); + static void decode21(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 42, longs, 11, 21, MASK32_21, tmp, 0, MASK32_11); for (int iter = 0, tmpIdx = 0, longsIdx = 42; iter < 2; ++iter, tmpIdx += 21, longsIdx += 11) { long l0 = tmp[tmpIdx + 0] << 10; l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK32_10; @@ -799,8 +821,8 @@ public final class ForUtil { } } - static void decode22(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(44, longs, 10, 22, MASK32_22, tmp, 0, MASK32_10); + static void decode22(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 44, longs, 10, 22, MASK32_22, tmp, 0, MASK32_10); for (int iter = 0, tmpIdx = 0, longsIdx = 44; iter < 4; ++iter, tmpIdx += 11, longsIdx += 5) { long l0 = tmp[tmpIdx + 0] << 12; l0 |= tmp[tmpIdx + 1] << 2; @@ -825,8 +847,8 @@ public final class ForUtil { } } - static void decode23(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(46, longs, 9, 23, MASK32_23, tmp, 0, MASK32_9); + static void decode23(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 46, longs, 9, 23, MASK32_23, tmp, 0, MASK32_9); for (int iter = 0, tmpIdx = 0, longsIdx = 46; iter < 2; ++iter, tmpIdx += 23, longsIdx += 9) { long l0 = tmp[tmpIdx + 0] << 14; l0 |= tmp[tmpIdx + 1] << 5; @@ -871,8 +893,8 @@ public final class ForUtil { } } - static void decode24(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException { - pdu.splitLongs(48, longs, 8, 24, MASK32_24, tmp, 0, MASK32_8); + static void decode24(IndexInput in, long[] tmp, long[] longs) throws IOException { + splitLongs(in, 48, longs, 8, 24, MASK32_24, tmp, 0, MASK32_8); for (int iter = 0, tmpIdx = 0, longsIdx = 48; iter < 16; ++iter, tmpIdx += 3, longsIdx += 1) { long l0 = tmp[tmpIdx + 0] << 16; l0 |= tmp[tmpIdx + 1] << 8; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912Codec.java index d7b89d31081..e455c2cc6a3 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912Codec.java @@ -37,7 +37,6 @@ import org.apache.lucene.codecs.lucene90.Lucene90NormsFormat; import org.apache.lucene.codecs.lucene90.Lucene90PointsFormat; import org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat; import org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; import org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsFormat.java similarity index 94% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsFormat.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsFormat.java index 1c452175b07..a0342635d76 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsFormat.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsFormat.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.backward_codecs.lucene912; import java.io.IOException; import org.apache.lucene.codecs.BlockTermState; @@ -23,7 +23,6 @@ import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; -import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsReader; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; import org.apache.lucene.index.IndexOptions; @@ -318,7 +317,7 @@ import org.apache.lucene.util.packed.PackedInts; * * @lucene.experimental */ -public final class Lucene912PostingsFormat extends PostingsFormat { +public class Lucene912PostingsFormat extends PostingsFormat { /** Filename extension for some small metadata about how postings are encoded. */ public static final String META_EXTENSION = "psm"; @@ -341,7 +340,7 @@ public final class Lucene912PostingsFormat extends PostingsFormat { /** Size of blocks. */ public static final int BLOCK_SIZE = ForUtil.BLOCK_SIZE; - public static final int BLOCK_MASK = BLOCK_SIZE - 1; + static final int BLOCK_MASK = BLOCK_SIZE - 1; /** We insert skip data on every block and every SKIP_FACTOR=32 blocks. */ public static final int LEVEL1_FACTOR = 32; @@ -349,7 +348,7 @@ public final class Lucene912PostingsFormat extends PostingsFormat { /** Total number of docs covered by level 1 skip data: 32 * 128 = 4,096 */ public static final int LEVEL1_NUM_DOCS = LEVEL1_FACTOR * BLOCK_SIZE; - public static final int LEVEL1_MASK = LEVEL1_NUM_DOCS - 1; + static final int LEVEL1_MASK = LEVEL1_NUM_DOCS - 1; static final String TERMS_CODEC = "Lucene90PostingsWriterTerms"; static final String META_CODEC = "Lucene912PostingsWriterMeta"; @@ -360,45 +359,15 @@ public final class Lucene912PostingsFormat extends PostingsFormat { static final int VERSION_START = 0; static final int VERSION_CURRENT = VERSION_START; - private final int minTermBlockSize; - private final int maxTermBlockSize; - /** Creates {@code Lucene912PostingsFormat} with default settings. */ public Lucene912PostingsFormat() { - this( - Lucene90BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, - Lucene90BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE); - } - - /** - * Creates {@code Lucene912PostingsFormat} with custom values for {@code minBlockSize} and {@code - * maxBlockSize} passed to block terms dictionary. - * - * @see - * Lucene90BlockTreeTermsWriter#Lucene90BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) - */ - public Lucene912PostingsFormat(int minTermBlockSize, int maxTermBlockSize) { super("Lucene912"); - Lucene90BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize); - this.minTermBlockSize = minTermBlockSize; - this.maxTermBlockSize = maxTermBlockSize; } @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); - boolean success = false; - try { - FieldsConsumer ret = - new Lucene90BlockTreeTermsWriter( - state, postingsWriter, minTermBlockSize, maxTermBlockSize); - success = true; - return ret; - } finally { - if (!success) { - IOUtils.closeWhileHandlingException(postingsWriter); - } - } + throw new UnsupportedOperationException( + "This postings format may not be used for writing, use the current postings format"); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsReader.java similarity index 94% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsReader.java index 1a2fe05679e..f15fe851006 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsReader.java @@ -14,17 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.backward_codecs.lucene912; -import static org.apache.lucene.codecs.lucene912.ForUtil.BLOCK_SIZE; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.DOC_CODEC; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.LEVEL1_NUM_DOCS; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.META_CODEC; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.PAY_CODEC; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.POS_CODEC; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.TERMS_CODEC; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.VERSION_CURRENT; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.VERSION_START; +import static org.apache.lucene.backward_codecs.lucene912.ForUtil.BLOCK_SIZE; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.DOC_CODEC; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.LEVEL1_NUM_DOCS; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.META_CODEC; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.PAY_CODEC; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.POS_CODEC; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.TERMS_CODEC; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.VERSION_CURRENT; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.VERSION_START; import java.io.IOException; import java.util.AbstractList; @@ -32,10 +32,10 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.RandomAccess; +import org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState; import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.PostingsReaderBase; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.Impact; import org.apache.lucene.index.Impacts; @@ -45,9 +45,6 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SlowImpactsEnum; -import org.apache.lucene.internal.vectorization.PostingDecodingUtil; -import org.apache.lucene.internal.vectorization.VectorUtilSupport; -import org.apache.lucene.internal.vectorization.VectorizationProvider; import org.apache.lucene.store.ByteArrayDataInput; import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.DataInput; @@ -65,9 +62,6 @@ import org.apache.lucene.util.IOUtils; */ public final class Lucene912PostingsReader extends PostingsReaderBase { - static final VectorizationProvider VECTORIZATION_PROVIDER = VectorizationProvider.getInstance(); - private static final VectorUtilSupport VECTOR_SUPPORT = - VECTORIZATION_PROVIDER.getVectorUtilSupport(); // Dummy impacts, composed of the maximum possible term frequency and the lowest possible // (unsigned) norm value. This is typically used on tail blocks, which don't actually record // impacts as the storage overhead would not be worth any query evaluation speedup, since there's @@ -355,7 +349,6 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { protected int docBufferUpto; protected IndexInput docIn; - protected PostingDecodingUtil docInUtil; protected AbstractPostingsEnum(FieldInfo fieldInfo) { indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; @@ -376,7 +369,6 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { if (docIn == null) { // lazy init docIn = Lucene912PostingsReader.this.docIn.clone(); - docInUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(docIn); } prefetchPostings(docIn, termState); } @@ -442,7 +434,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { public int freq() throws IOException { if (freqFP != -1) { docIn.seek(freqFP); - pforUtil.decode(docInUtil, freqBuffer); + pforUtil.decode(docIn, freqBuffer); freqFP = -1; } @@ -472,7 +464,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { private void refillFullBlock() throws IOException { assert docFreq - docCountUpto >= BLOCK_SIZE; - forDeltaUtil.decodeAndPrefixSum(docInUtil, prevDocID, docBuffer); + forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); if (indexHasFreq) { if (needsFreq) { @@ -601,7 +593,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } } - int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); this.doc = (int) docBuffer[next]; docBufferUpto = next + 1; return doc; @@ -633,9 +625,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { private int posBufferUpto; final IndexInput posIn; - final PostingDecodingUtil posInUtil; final IndexInput payIn; - final PostingDecodingUtil payInUtil; final BytesRef payload; final boolean indexHasOffsets; @@ -678,13 +668,10 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { indexHasOffsetsOrPayloads = indexHasOffsets || indexHasPayloads; this.posIn = Lucene912PostingsReader.this.posIn.clone(); - posInUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(posIn); if (indexHasOffsetsOrPayloads) { this.payIn = Lucene912PostingsReader.this.payIn.clone(); - payInUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(payIn); } else { this.payIn = null; - payInUtil = null; } if (indexHasOffsets) { offsetStartDeltaBuffer = new long[BLOCK_SIZE]; @@ -771,8 +758,8 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { assert left >= 0; if (left >= BLOCK_SIZE) { - forDeltaUtil.decodeAndPrefixSum(docInUtil, prevDocID, docBuffer); - pforUtil.decode(docInUtil, freqBuffer); + forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); + pforUtil.decode(docIn, freqBuffer); docCountUpto += BLOCK_SIZE; } else if (docFreq == 1) { docBuffer[0] = singletonDocID; @@ -950,7 +937,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { refillDocs(); } - int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); posPendingCount += sumOverRange(freqBuffer, docBufferUpto, next + 1); this.freq = (int) freqBuffer[next]; this.docBufferUpto = next + 1; @@ -1044,11 +1031,11 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } payloadByteUpto = 0; } else { - pforUtil.decode(posInUtil, posDeltaBuffer); + pforUtil.decode(posIn, posDeltaBuffer); if (indexHasPayloads) { if (needsPayloads) { - pforUtil.decode(payInUtil, payloadLengthBuffer); + pforUtil.decode(payIn, payloadLengthBuffer); int numBytes = payIn.readVInt(); if (numBytes > payloadBytes.length) { @@ -1067,8 +1054,8 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { if (indexHasOffsets) { if (needsOffsets) { - pforUtil.decode(payInUtil, offsetStartDeltaBuffer); - pforUtil.decode(payInUtil, offsetLengthBuffer); + pforUtil.decode(payIn, offsetStartDeltaBuffer); + pforUtil.decode(payIn, offsetLengthBuffer); } else { // this works, because when writing a vint block we always force the first length to be // written @@ -1149,7 +1136,6 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { protected final int docFreq; // number of docs in this posting list protected final IndexInput docIn; - protected final PostingDecodingUtil docInUtil; protected int docCountUpto; // number of docs in or before the current block protected int doc = -1; // doc we last read @@ -1175,7 +1161,6 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { private BlockImpactsEnum(IntBlockTermState termState) throws IOException { this.docFreq = termState.docFreq; this.docIn = Lucene912PostingsReader.this.docIn.clone(); - this.docInUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(docIn); prefetchPostings(docIn, termState); level0SerializedImpacts = new BytesRef(maxImpactNumBytesAtLevel0); level1SerializedImpacts = new BytesRef(maxImpactNumBytesAtLevel1); @@ -1279,7 +1264,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { public int freq() throws IOException { if (freqFP != -1) { docIn.seek(freqFP); - pforUtil.decode(docInUtil, freqBuffer); + pforUtil.decode(docIn, freqBuffer); freqFP = -1; } return (int) freqBuffer[docBufferUpto - 1]; @@ -1295,7 +1280,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { assert left >= 0; if (left >= BLOCK_SIZE) { - forDeltaUtil.decodeAndPrefixSum(docInUtil, prevDocID, docBuffer); + forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); freqFP = docIn.getFilePointer(); PForUtil.skip(docIn); docCountUpto += BLOCK_SIZE; @@ -1438,7 +1423,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { needsRefilling = false; } - int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); this.doc = (int) docBuffer[next]; docBufferUpto = next + 1; return doc; @@ -1450,7 +1435,6 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { private int posBufferUpto; final IndexInput posIn; - final PostingDecodingUtil posInUtil; final boolean indexHasFreq; final boolean indexHasOffsets; @@ -1491,7 +1475,6 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { indexHasOffsetsOrPayloads = indexHasOffsets || indexHasPayloads; this.posIn = Lucene912PostingsReader.this.posIn.clone(); - posInUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(posIn); // Where this term's postings start in the .pos file: final long posTermStartFP = termState.posStartFP; @@ -1522,8 +1505,8 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { assert left >= 0; if (left >= BLOCK_SIZE) { - forDeltaUtil.decodeAndPrefixSum(docInUtil, prevDocID, docBuffer); - pforUtil.decode(docInUtil, freqBuffer); + forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); + pforUtil.decode(docIn, freqBuffer); docCountUpto += BLOCK_SIZE; } else if (docFreq == 1) { docBuffer[0] = singletonDocID; @@ -1671,7 +1654,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { needsRefilling = false; } - int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); posPendingCount += sumOverRange(freqBuffer, docBufferUpto, next + 1); freq = (int) freqBuffer[next]; docBufferUpto = next + 1; @@ -1729,7 +1712,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } } } else { - pforUtil.decode(posInUtil, posDeltaBuffer); + pforUtil.decode(posIn, posDeltaBuffer); } } @@ -1754,9 +1737,6 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } } - /** - * @see Lucene912PostingsWriter#writeVInt15(org.apache.lucene.store.DataOutput, int) - */ static int readVInt15(DataInput in) throws IOException { short s = in.readShort(); if (s >= 0) { @@ -1766,9 +1746,6 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } } - /** - * @see Lucene912PostingsWriter#writeVLong15(org.apache.lucene.store.DataOutput, long) - */ static long readVLong15(DataInput in) throws IOException { short s = in.readShort(); if (s >= 0) { @@ -1778,6 +1755,15 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } } + private static int findNextGEQ(long[] buffer, int length, long target, int from) { + for (int i = from; i < length; ++i) { + if (buffer[i] >= target) { + return i; + } + } + return length; + } + private static void prefetchPostings(IndexInput docIn, IntBlockTermState state) throws IOException { assert state.docFreq > 1; // Singletons are inlined in the terms dict, nothing to prefetch diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/PForUtil.java similarity index 94% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/PForUtil.java index 063f385ba42..a075e42ec36 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/PForUtil.java @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.backward_codecs.lucene912; import java.io.IOException; import java.util.Arrays; -import org.apache.lucene.internal.vectorization.PostingDecodingUtil; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.LongHeap; import org.apache.lucene.util.packed.PackedInts; @@ -104,14 +104,13 @@ final class PForUtil { } /** Decode 128 integers into {@code ints}. */ - void decode(PostingDecodingUtil pdu, long[] longs) throws IOException { - var in = pdu.in; + void decode(IndexInput in, long[] longs) throws IOException { final int token = Byte.toUnsignedInt(in.readByte()); final int bitsPerValue = token & 0x1f; if (bitsPerValue == 0) { Arrays.fill(longs, 0, ForUtil.BLOCK_SIZE, in.readVLong()); } else { - forUtil.decode(bitsPerValue, pdu, longs); + forUtil.decode(bitsPerValue, in, longs); } final int numExceptions = token >>> 5; for (int i = 0; i < numExceptions; ++i) { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingsUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/PostingsUtil.java similarity index 97% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingsUtil.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/PostingsUtil.java index 1ae808d308f..8f526f7ef10 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingsUtil.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/PostingsUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.backward_codecs.lucene912; import java.io.IOException; import org.apache.lucene.store.DataOutput; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForDeltaUtil.py b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/gen_ForDeltaUtil.py similarity index 90% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForDeltaUtil.py rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/gen_ForDeltaUtil.py index 134c17c0b8f..90604ee75cc 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForDeltaUtil.py +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/gen_ForDeltaUtil.py @@ -40,15 +40,14 @@ HEADER = """// This file has been automatically generated, DO NOT EDIT * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.backward_codecs.lucene912; import java.io.IOException; -import org.apache.lucene.internal.vectorization.PostingDecodingUtil; import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.packed.PackedInts; -import static org.apache.lucene.codecs.lucene912.ForUtil.*; +import static org.apache.lucene.backward_codecs.lucene912.ForUtil.*; /** * Inspired from https://fulmicoton.com/posts/bitpacking/ @@ -57,7 +56,7 @@ import static org.apache.lucene.codecs.lucene912.ForUtil.*; * else if bitsPerValue <= 11 we pack 4 ints per long * else we pack 2 ints per long */ -public final class ForDeltaUtil { +final class ForDeltaUtil { private static final int ONE_BLOCK_SIZE_FOURTH = BLOCK_SIZE / 4; private static final int TWO_BLOCK_SIZE_FOURTHS = BLOCK_SIZE / 2; @@ -299,12 +298,12 @@ public final class ForDeltaUtil { } /** Decode deltas, compute the prefix sum and add {@code base} to all decoded longs. */ - void decodeAndPrefixSum(PostingDecodingUtil pdu, long base, long[] longs) throws IOException { - final int bitsPerValue = Byte.toUnsignedInt(pdu.in.readByte()); + void decodeAndPrefixSum(IndexInput in, long base, long[] longs) throws IOException { + final int bitsPerValue = Byte.toUnsignedInt(in.readByte()); if (bitsPerValue == 0) { prefixSumOfOnes(longs, base); } else { - decodeAndPrefixSum(bitsPerValue, pdu, base, longs); + decodeAndPrefixSum(bitsPerValue, in, base, longs); } } @@ -362,20 +361,20 @@ def writeRemainder(bpv, next_primitive, remaining_bits_per_long, o, num_values, def writeDecode(bpv, f): next_primitive = primitive_size_for_bpv(bpv) if next_primitive % bpv == 0: - f.write(' private static void decode%dTo%d(PostingDecodingUtil pdu, long[] longs) throws IOException {\n' %(bpv, next_primitive)) + f.write(' private static void decode%dTo%d(IndexInput in, long[] longs) throws IOException {\n' %(bpv, next_primitive)) else: - f.write(' private static void decode%dTo%d(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException {\n' %(bpv, next_primitive)) + f.write(' private static void decode%dTo%d(IndexInput in, long[] tmp, long[] longs) throws IOException {\n' %(bpv, next_primitive)) if bpv == next_primitive: - f.write(' pdu.in.readLongs(longs, 0, %d);\n' %(bpv*2)) + f.write(' in.readLongs(longs, 0, %d);\n' %(bpv*2)) else: num_values_per_long = 64 / next_primitive remaining_bits = next_primitive % bpv num_iters = (next_primitive - 1) // bpv o = 2 * bpv * num_iters if remaining_bits == 0: - f.write(' pdu.splitLongs(%d, longs, %d, %d, MASK%d_%d, longs, %d, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, bpv, next_primitive, bpv, o, next_primitive, next_primitive - num_iters * bpv)) + f.write(' splitLongs(in, %d, longs, %d, %d, MASK%d_%d, longs, %d, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, bpv, next_primitive, bpv, o, next_primitive, next_primitive - num_iters * bpv)) else: - f.write(' pdu.splitLongs(%d, longs, %d, %d, MASK%d_%d, tmp, 0, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, bpv, next_primitive, bpv, next_primitive, next_primitive - num_iters * bpv)) + f.write(' splitLongs(in, %d, longs, %d, %d, MASK%d_%d, tmp, 0, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, bpv, next_primitive, bpv, next_primitive, next_primitive - num_iters * bpv)) writeRemainder(bpv, next_primitive, remaining_bits, o, 128/num_values_per_long - o, f) f.write(' }\n') @@ -386,7 +385,7 @@ if __name__ == '__main__': /** * Delta-decode 128 integers into {@code longs}. */ - void decodeAndPrefixSum(int bitsPerValue, PostingDecodingUtil pdu, long base, long[] longs) throws IOException { + void decodeAndPrefixSum(int bitsPerValue, IndexInput in, long base, long[] longs) throws IOException { switch (bitsPerValue) { """) for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1): @@ -394,18 +393,18 @@ if __name__ == '__main__': f.write(' case %d:\n' %bpv) if next_primitive(bpv) == primitive_size: if primitive_size % bpv == 0: - f.write(' decode%d(pdu, longs);\n' %bpv) + f.write(' decode%d(in, longs);\n' %bpv) else: - f.write(' decode%d(pdu, tmp, longs);\n' %bpv) + f.write(' decode%d(in, tmp, longs);\n' %bpv) else: if primitive_size % bpv == 0: - f.write(' decode%dTo%d(pdu, longs);\n' %(bpv, primitive_size)) + f.write(' decode%dTo%d(in, longs);\n' %(bpv, primitive_size)) else: - f.write(' decode%dTo%d(pdu, tmp, longs);\n' %(bpv, primitive_size)) + f.write(' decode%dTo%d(in, tmp, longs);\n' %(bpv, primitive_size)) f.write(' prefixSum%d(longs, base);\n' %primitive_size) f.write(' break;\n') f.write(' default:\n') - f.write(' decodeSlow(bitsPerValue, pdu, tmp, longs);\n') + f.write(' decodeSlow(bitsPerValue, in, tmp, longs);\n') f.write(' prefixSum32(longs, base);\n') f.write(' break;\n') f.write(' }\n') diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/gen_ForUtil.py similarity index 88% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/gen_ForUtil.py index 9851ec99a61..29543ffe671 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/gen_ForUtil.py @@ -40,10 +40,9 @@ HEADER = """// This file has been automatically generated, DO NOT EDIT * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.backward_codecs.lucene912; import java.io.IOException; -import org.apache.lucene.internal.vectorization.PostingDecodingUtil; import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.IndexInput; @@ -54,7 +53,7 @@ import org.apache.lucene.store.IndexInput; * else if bitsPerValue <= 16 we pack 4 ints per long * else we pack 2 ints per long */ -public final class ForUtil { +final class ForUtil { public static final int BLOCK_SIZE = 128; static final int BLOCK_SIZE_LOG2 = 7; @@ -222,11 +221,11 @@ public final class ForUtil { return bitsPerValue << (BLOCK_SIZE_LOG2 - 3); } - static void decodeSlow(int bitsPerValue, PostingDecodingUtil pdu, long[] tmp, long[] longs) + static void decodeSlow(int bitsPerValue, IndexInput in, long[] tmp, long[] longs) throws IOException { final int numLongs = bitsPerValue << 1; final long mask = MASKS32[bitsPerValue]; - pdu.splitLongs(numLongs, longs, 32 - bitsPerValue, 32, mask, tmp, 0, -1L); + splitLongs(in, numLongs, longs, 32 - bitsPerValue, 32, mask, tmp, 0, -1L); final int remainingBitsPerLong = 32 - bitsPerValue; final long mask32RemainingBitsPerLong = MASKS32[remainingBitsPerLong]; int tmpIdx = 0; @@ -248,6 +247,20 @@ public final class ForUtil { } } + static void splitLongs( + IndexInput in, int count, long[] b, int bShift, int dec, long bMask, long[] c, int cIndex, long cMask) + throws IOException { + // takes advantage of the C2 compiler's loop unrolling and auto-vectorization. + in.readLongs(c, cIndex, count); + int maxIter = (bShift - 1) / dec; + for (int i = 0; i < count; ++i) { + for (int j = 0; j <= maxIter; ++j) { + b[count * j + i] = (c[cIndex + i] >>> (bShift - j * dec)) & bMask; + } + c[cIndex + i] &= cMask; + } + } + """ def writeRemainder(bpv, next_primitive, remaining_bits_per_long, o, num_values, f): @@ -288,19 +301,19 @@ def writeDecode(bpv, f): elif bpv <= 16: next_primitive = 16 if bpv == next_primitive: - f.write(' static void decode%d(PostingDecodingUtil pdu, long[] longs) throws IOException {\n' %bpv) - f.write(' pdu.in.readLongs(longs, 0, %d);\n' %(bpv*2)) + f.write(' static void decode%d(IndexInput in, long[] longs) throws IOException {\n' %bpv) + f.write(' in.readLongs(longs, 0, %d);\n' %(bpv*2)) else: num_values_per_long = 64 / next_primitive remaining_bits = next_primitive % bpv num_iters = (next_primitive - 1) // bpv o = 2 * bpv * num_iters if remaining_bits == 0: - f.write(' static void decode%d(PostingDecodingUtil pdu, long[] longs) throws IOException {\n' %bpv) - f.write(' pdu.splitLongs(%d, longs, %d, %d, MASK%d_%d, longs, %d, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, bpv, next_primitive, bpv, o, next_primitive, next_primitive - num_iters * bpv)) + f.write(' static void decode%d(IndexInput in, long[] longs) throws IOException {\n' %bpv) + f.write(' splitLongs(in, %d, longs, %d, %d, MASK%d_%d, longs, %d, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, bpv, next_primitive, bpv, o, next_primitive, next_primitive - num_iters * bpv)) else: - f.write(' static void decode%d(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IOException {\n' %bpv) - f.write(' pdu.splitLongs(%d, longs, %d, %d, MASK%d_%d, tmp, 0, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, bpv, next_primitive, bpv, next_primitive, next_primitive - num_iters * bpv)) + f.write(' static void decode%d(IndexInput in, long[] tmp, long[] longs) throws IOException {\n' %bpv) + f.write(' splitLongs(in, %d, longs, %d, %d, MASK%d_%d, tmp, 0, MASK%d_%d);\n' %(bpv*2, next_primitive - bpv, bpv, next_primitive, bpv, next_primitive, next_primitive - num_iters * bpv)) writeRemainder(bpv, next_primitive, remaining_bits, o, 128/num_values_per_long - o, f) f.write(' }\n') @@ -326,7 +339,7 @@ if __name__ == '__main__': f.write(""" /** Decode 128 integers into {@code longs}. */ - void decode(int bitsPerValue, PostingDecodingUtil pdu, long[] longs) throws IOException { + void decode(int bitsPerValue, IndexInput in, long[] longs) throws IOException { switch (bitsPerValue) { """) for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1): @@ -337,13 +350,13 @@ if __name__ == '__main__': next_primitive = 16 f.write(' case %d:\n' %bpv) if next_primitive % bpv == 0: - f.write(' decode%d(pdu, longs);\n' %bpv) + f.write(' decode%d(in, longs);\n' %bpv) else: - f.write(' decode%d(pdu, tmp, longs);\n' %bpv) + f.write(' decode%d(in, tmp, longs);\n' %bpv) f.write(' expand%d(longs);\n' %next_primitive) f.write(' break;\n') f.write(' default:\n') - f.write(' decodeSlow(bitsPerValue, pdu, tmp, longs);\n') + f.write(' decodeSlow(bitsPerValue, in, tmp, longs);\n') f.write(' expand32(longs);\n') f.write(' break;\n') f.write(' }\n') diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/package-info.java index aac717a3e6c..76666469faa 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/package-info.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/package-info.java @@ -15,419 +15,5 @@ * limitations under the License. */ -/** - * Lucene 9.12 file format. - * - *

Apache Lucene - Index File Formats

- * - * - * - *

Introduction

- * - *
- * - *

This document defines the index file formats used in this version of Lucene. If you are using - * a different version of Lucene, please consult the copy of docs/ that was distributed - * with the version you are using. - * - *

This document attempts to provide a high-level definition of the Apache Lucene file formats. - *

- * - *

Definitions

- * - *
- * - *

The fundamental concepts in Lucene are index, document, field and term. - * - *

An index contains a sequence of documents. - * - *

    - *
  • A document is a sequence of fields. - *
  • A field is a named sequence of terms. - *
  • A term is a sequence of bytes. - *
- * - *

The same sequence of bytes in two different fields is considered a different term. Thus terms - * are represented as a pair: the string naming the field, and the bytes within the field. - * - *

Inverted Indexing

- * - *

Lucene's index stores terms and statistics about those terms in order to make term-based - * search more efficient. Lucene's terms index falls into the family of indexes known as an - * inverted index. This is because it can list, for a term, the documents that contain it. - * This is the inverse of the natural relationship, in which documents list terms. - * - *

Types of Fields

- * - *

In Lucene, fields may be stored, in which case their text is stored in the index - * literally, in a non-inverted manner. Fields that are inverted are called indexed. A field - * may be both stored and indexed. - * - *

The text of a field may be tokenized into terms to be indexed, or the text of a field - * may be used literally as a term to be indexed. Most fields are tokenized, but sometimes it is - * useful for certain identifier fields to be indexed literally. - * - *

See the {@link org.apache.lucene.document.Field Field} java docs for more information on - * Fields. - * - *

Segments

- * - *

Lucene indexes may be composed of multiple sub-indexes, or segments. Each segment is a - * fully independent index, which could be searched separately. Indexes evolve by: - * - *

    - *
  1. Creating new segments for newly added documents. - *
  2. Merging existing segments. - *
- * - *

Searches may involve multiple segments and/or multiple indexes, each index potentially - * composed of a set of segments. - * - *

Document Numbers

- * - *

Internally, Lucene refers to documents by an integer document number. The first - * document added to an index is numbered zero, and each subsequent document added gets a number one - * greater than the previous. - * - *

Note that a document's number may change, so caution should be taken when storing these - * numbers outside of Lucene. In particular, numbers may change in the following situations: - * - *

    - *
  • - *

    The numbers stored in each segment are unique only within the segment, and must be - * converted before they can be used in a larger context. The standard technique is to - * allocate each segment a range of values, based on the range of numbers used in that - * segment. To convert a document number from a segment to an external value, the segment's - * base document number is added. To convert an external value back to a - * segment-specific value, the segment is identified by the range that the external value is - * in, and the segment's base value is subtracted. For example two five document segments - * might be combined, so that the first segment has a base value of zero, and the second of - * five. Document three from the second segment would have an external value of eight. - *

  • - *

    When documents are deleted, gaps are created in the numbering. These are eventually - * removed as the index evolves through merging. Deleted documents are dropped when segments - * are merged. A freshly-merged segment thus has no gaps in its numbering. - *

- * - *
- * - *

Index Structure Overview

- * - *
- * - *

Each segment index maintains the following: - * - *

    - *
  • {@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment info}. This - * contains metadata about a segment, such as the number of documents, what files it uses, and - * information about how the segment is sorted - *
  • {@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Field names}. This - * contains metadata about the set of named fields used in the index. - *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Stored Field values}. - * This contains, for each document, a list of attribute-value pairs, where the attributes are - * field names. These are used to store auxiliary information about the document, such as its - * title, url, or an identifier to access a database. The set of stored fields are what is - * returned for each hit when searching. This is keyed by document number. - *
  • {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term dictionary}. A - * dictionary containing all of the terms used in all of the indexed fields of all of the - * documents. The dictionary also contains the number of documents which contain the term, and - * pointers to the term's frequency and proximity data. - *
  • {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Frequency data}. For - * each term in the dictionary, the numbers of all the documents that contain that term, and - * the frequency of the term in that document, unless frequencies are omitted ({@link - * org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS}) - *
  • {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Proximity data}. For - * each term in the dictionary, the positions that the term occurs in each document. Note that - * this will not exist if all fields in all documents omit position data. - *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Normalization factors}. For - * each field in each document, a value is stored that is multiplied into the score for hits - * on that field. - *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vectors}. For each - * field in each document, the term vector (sometimes called document vector) may be stored. A - * term vector consists of term text and term frequency. To add Term Vectors to your index see - * the {@link org.apache.lucene.document.Field Field} constructors - *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-document values}. Like - * stored values, these are also keyed by document number, but are generally intended to be - * loaded into main memory for fast access. Whereas stored values are generally intended for - * summary results from searches, per-document values are useful for things like scoring - * factors. - *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live documents}. An - * optional file indicating which documents are live. - *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}. Optional pair - * of files, recording dimensionally indexed fields, to enable fast numeric range filtering - * and large numeric values like BigInteger and BigDecimal (1D) and geographic shape - * intersection (2D, 3D). - *
  • {@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}. The - * vector format stores numeric vectors in a format optimized for random access and - * computation, supporting high-dimensional nearest-neighbor search. - *
- * - *

Details on each of these are provided in their linked pages.

- * - *

File Naming

- * - *
- * - *

All files belonging to a segment have the same name with varying extensions. The extensions - * correspond to the different file formats described below. When using the Compound File format - * (default for small segments) these files (except for the Segment info file, the Lock file, and - * Deleted documents file) are collapsed into a single .cfs file (see below for details) - * - *

Typically, all segments in an index are stored in a single directory, although this is not - * required. - * - *

File names are never re-used. That is, when any file is saved to the Directory it is given a - * never before used filename. This is achieved using a simple generations approach. For example, - * the first segments file is segments_1, then segments_2, etc. The generation is a sequential long - * integer represented in alpha-numeric (base 36) form.

- * - *

Summary of File Extensions

- * - *
- * - *

The following table summarizes the names and extensions of the files in Lucene: - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - *
lucene filenames by extension
NameExtensionBrief Description
{@link org.apache.lucene.index.SegmentInfos Segments File}segments_NStores information about a commit point
Lock Filewrite.lockThe Write lock prevents multiple IndexWriters from writing to the same - * file.
{@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment Info}.siStores metadata about a segment
{@link org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat Compound File}.cfs, .cfeAn optional "virtual" file consisting of all the other index files for - * systems that frequently run out of file handles.
{@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Fields}.fnmStores information about the fields
{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}.fdxContains pointers to field data
{@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}.fdtThe stored fields for documents
{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Dictionary}.timThe term dictionary, stores term info
{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Index}.tipThe index into the Term Dictionary
{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Frequencies}.docContains the list of docs which contain each term along with frequency
{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Positions}.posStores position information about where a term occurs in the index
{@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Payloads}.payStores additional per-position metadata information such as character offsets and user payloads
{@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Norms}.nvd, .nvmEncodes length and boost factors for docs and fields
{@link org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat Per-Document Values}.dvd, .dvmEncodes additional scoring factors or other per-document information.
{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Index}.tvxStores offset into the document data file
{@link org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat Term Vector Data}.tvdContains term vector data.
{@link org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat Live Documents}.livInfo about what documents are live
{@link org.apache.lucene.codecs.lucene90.Lucene90PointsFormat Point values}.kdd, .kdi, .kdmHolds indexed points
{@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}.vec, .vem, .veq, vexHolds indexed vectors; .vec files contain the raw vector data, - * .vem the vector metadata, .veq the quantized vector data, and .vex the - * hnsw graph data.
- * - *

- * - *

Lock File

- * - * The write lock, which is stored in the index directory by default, is named "write.lock". If the - * lock directory is different from the index directory then the write lock will be named - * "XXXX-write.lock" where XXXX is a unique prefix derived from the full path to the index - * directory. When this file is present, a writer is currently modifying the index (adding or - * removing documents). This lock file ensures that only one writer is modifying the index at a - * time. - * - *

History

- * - *

Compatibility notes are provided in this document, describing how file formats have changed - * from prior versions: - * - *

    - *
  • In version 2.1, the file format was changed to allow lock-less commits (ie, no more commit - * lock). The change is fully backwards compatible: you can open a pre-2.1 index for searching - * or adding/deleting of docs. When the new segments file is saved (committed), it will be - * written in the new file format (meaning no specific "upgrade" process is needed). But note - * that once a commit has occurred, pre-2.1 Lucene will not be able to read the index. - *
  • In version 2.3, the file format was changed to allow segments to share a single set of doc - * store (vectors & stored fields) files. This allows for faster indexing in certain - * cases. The change is fully backwards compatible (in the same way as the lock-less commits - * change in 2.1). - *
  • In version 2.4, Strings are now written as true UTF-8 byte sequence, not Java's modified - * UTF-8. See LUCENE-510 for - * details. - *
  • In version 2.9, an optional opaque Map<String,String> CommitUserData may be passed to - * IndexWriter's commit methods (and later retrieved), which is recorded in the segments_N - * file. See LUCENE-1382 for - * details. Also, diagnostics were added to each segment written recording details about why - * it was written (due to flush, merge; which OS/JRE was used; etc.). See issue LUCENE-1654 for details. - *
  • In version 3.0, compressed fields are no longer written to the index (they can still be - * read, but on merge the new segment will write them, uncompressed). See issue LUCENE-1960 for details. - *
  • In version 3.1, segments records the code version that created them. See LUCENE-2720 for details. - * Additionally segments track explicitly whether or not they have term vectors. See LUCENE-2811 for details. - *
  • In version 3.2, numeric fields are written as natively to stored fields file, previously - * they were stored in text format only. - *
  • In version 3.4, fields can omit position data while still indexing term frequencies. - *
  • In version 4.0, the format of the inverted index became extensible via the {@link - * org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage ({@code DocValues}) - * was introduced. Normalization factors need no longer be a single byte, they can be any - * {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. Terms need not be - * unicode strings, they can be any byte sequence. Term offsets can optionally be indexed into - * the postings lists. Payloads can be stored in the term vectors. - *
  • In version 4.1, the format of the postings list changed to use either of FOR compression or - * variable-byte encoding, depending upon the frequency of the term. Terms appearing only once - * were changed to inline directly into the term dictionary. Stored fields are compressed by - * default. - *
  • In version 4.2, term vectors are compressed by default. DocValues has a new multi-valued - * type (SortedSet), that can be used for faceting/grouping/joining on multi-valued fields. - *
  • In version 4.5, DocValues were extended to explicitly represent missing values. - *
  • In version 4.6, FieldInfos were extended to support per-field DocValues generation, to - * allow updating NumericDocValues fields. - *
  • In version 4.8, checksum footers were added to the end of each index file for improved data - * integrity. Specifically, the last 8 bytes of every index file contain the zlib-crc32 - * checksum of the file. - *
  • In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric) that is - * suitable for faceting/sorting/analytics. - *
  • In version 5.4, DocValues have been improved to store more information on disk: addresses - * for binary fields and ord indexes for multi-valued fields. - *
  • In version 6.0, Points were added, for multi-dimensional range/distance search. - *
  • In version 6.2, new Segment info format that reads/writes the index sort, to support index - * sorting. - *
  • In version 7.0, DocValues have been improved to better support sparse doc values thanks to - * an iterator API. - *
  • In version 8.0, postings have been enhanced to record, for each block of doc ids, the (term - * freq, normalization factor) pairs that may trigger the maximum score of the block. This - * information is recorded alongside skip data in order to be able to skip blocks of doc ids - * if they may not produce high enough scores. Additionally doc values and norms has been - * extended with jump-tables to make access O(1) instead of O(n), where n is the number of - * elements to skip when advancing in the data. - *
  • In version 8.4, postings, positions, offsets and payload lengths have move to a more - * performant encoding that is vectorized. - *
  • In version 8.6, index sort serialization is delegated to the sorts themselves, to allow - * user-defined sorts to be used - *
  • In version 8.6, points fields split the index tree and leaf data into separate files, to - * allow for different access patterns to the different data structures - *
  • In version 8.7, stored fields compression became adaptive to better handle documents with - * smaller stored fields. - *
  • In version 9.0, vector-valued fields were added. - *
  • In version 9.1, vector-valued fields were modified to add a graph hierarchy. - *
  • In version 9.2, docs of vector-valued fields were moved from .vem to .vec and encoded by - * IndexDISI. ordToDoc mappings was added to .vem. - *
  • In version 9.5, HNSW graph connections were changed to be delta-encoded with vints. - * Additionally, metadata file size improvements were made by delta-encoding nodes by graph - * layer and not writing the node ids for the zeroth layer. - *
  • In version 9.9, Vector scalar quantization support was added. Allowing the HNSW vector - * format to utilize int8 quantized vectors for float32 vector search. - *
  • In version 9.12, skip data was refactored to have only two levels: every 128 docs and every - * 4,06 docs, and to be inlined in postings lists. This resulted in a speedup for queries that - * need skipping, especially conjunctions. - *
- * - * - * - *

Limitations

- * - *
- * - *

Lucene uses a Java int to refer to document numbers, and the index file format - * uses an Int32 on-disk to store document numbers. This is a limitation of both the - * index file format and the current implementation. Eventually these should be replaced with either - * UInt64 values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt - * VInt} values which have no limit.

- */ +/** Lucene 9.12 file format. */ package org.apache.lucene.backward_codecs.lucene912; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99Codec.java index d540abc85c2..bb7764da8e8 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99Codec.java @@ -17,6 +17,7 @@ package org.apache.lucene.backward_codecs.lucene99; import java.util.Objects; +import org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.CompoundFormat; import org.apache.lucene.codecs.DocValuesFormat; @@ -37,7 +38,6 @@ import org.apache.lucene.codecs.lucene90.Lucene90NormsFormat; import org.apache.lucene.codecs.lucene90.Lucene90PointsFormat; import org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat; import org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; import org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat; diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec index a4638b5fcc7..ff4d7eeda4e 100644 --- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec +++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.Codec @@ -24,3 +24,4 @@ org.apache.lucene.backward_codecs.lucene94.Lucene94Codec org.apache.lucene.backward_codecs.lucene95.Lucene95Codec org.apache.lucene.backward_codecs.lucene99.Lucene99Codec org.apache.lucene.backward_codecs.lucene912.Lucene912Codec +org.apache.lucene.backward_codecs.lucene100.Lucene100Codec diff --git a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat index 9733362abe7..49d917dc427 100644 --- a/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat +++ b/lucene/backward-codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat @@ -17,3 +17,4 @@ org.apache.lucene.backward_codecs.lucene50.Lucene50PostingsFormat org.apache.lucene.backward_codecs.lucene84.Lucene84PostingsFormat org.apache.lucene.backward_codecs.lucene90.Lucene90PostingsFormat org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat +org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsWriter.java similarity index 96% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsWriter.java index b0da9347700..a3660b2ac0f 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsWriter.java @@ -14,25 +14,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.backward_codecs.lucene912; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.BLOCK_SIZE; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.DOC_CODEC; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.LEVEL1_MASK; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.META_CODEC; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.PAY_CODEC; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.POS_CODEC; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.TERMS_CODEC; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.VERSION_CURRENT; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.BLOCK_SIZE; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.DOC_CODEC; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.LEVEL1_MASK; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.META_CODEC; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.PAY_CODEC; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.POS_CODEC; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.TERMS_CODEC; +import static org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.VERSION_CURRENT; import java.io.IOException; import java.util.Collection; import java.util.List; +import org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState; import org.apache.lucene.codecs.BlockTermState; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CompetitiveImpactAccumulator; import org.apache.lucene.codecs.PushPostingsWriterBase; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.Impact; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/Lucene912RWPostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/Lucene912RWPostingsFormat.java new file mode 100644 index 00000000000..af1037432af --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/Lucene912RWPostingsFormat.java @@ -0,0 +1,69 @@ +/* + * 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.backward_codecs.lucene912; + +import java.io.IOException; +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.util.IOUtils; + +/** Read-write impersonation of {@link Lucene912PostingsFormat}. */ +public final class Lucene912RWPostingsFormat extends Lucene912PostingsFormat { + + private final int minTermBlockSize; + private final int maxTermBlockSize; + + /** Creates {@code Lucene912PostingsFormat} with default settings. */ + public Lucene912RWPostingsFormat() { + this( + Lucene90BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, + Lucene90BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE); + } + + /** + * Creates {@code Lucene912PostingsFormat} with custom values for {@code minBlockSize} and {@code + * maxBlockSize} passed to block terms dictionary. + * + * @see + * Lucene90BlockTreeTermsWriter#Lucene90BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) + */ + public Lucene912RWPostingsFormat(int minTermBlockSize, int maxTermBlockSize) { + super(); + Lucene90BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize); + this.minTermBlockSize = minTermBlockSize; + this.maxTermBlockSize = maxTermBlockSize; + } + + @Override + public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { + PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); + boolean success = false; + try { + FieldsConsumer ret = + new Lucene90BlockTreeTermsWriter( + state, postingsWriter, minTermBlockSize, maxTermBlockSize); + success = true; + return ret; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(postingsWriter); + } + } + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForDeltaUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestForDeltaUtil.java similarity index 91% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForDeltaUtil.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestForDeltaUtil.java index 3363b8a2202..471333b20f4 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForDeltaUtil.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestForDeltaUtil.java @@ -14,12 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.backward_codecs.lucene912; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; import java.util.Arrays; -import org.apache.lucene.internal.vectorization.PostingDecodingUtil; import org.apache.lucene.store.ByteBuffersDirectory; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; @@ -65,13 +64,11 @@ public class TestForDeltaUtil extends LuceneTestCase { { // decode IndexInput in = d.openInput("test.bin", IOContext.READONCE); - PostingDecodingUtil pdu = - Lucene912PostingsReader.VECTORIZATION_PROVIDER.newPostingDecodingUtil(in); ForDeltaUtil forDeltaUtil = new ForDeltaUtil(); for (int i = 0; i < iterations; ++i) { long base = 0; final long[] restored = new long[ForUtil.BLOCK_SIZE]; - forDeltaUtil.decodeAndPrefixSum(pdu, base, restored); + forDeltaUtil.decodeAndPrefixSum(in, base, restored); final long[] expected = new long[ForUtil.BLOCK_SIZE]; for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { expected[j] = values[i * ForUtil.BLOCK_SIZE + j]; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestForUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestForUtil.java new file mode 100644 index 00000000000..e728cb9e50d --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestForUtil.java @@ -0,0 +1,94 @@ +/* + * 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.backward_codecs.lucene912; + +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; +import java.io.IOException; +import java.util.Arrays; +import org.apache.lucene.store.ByteBuffersDirectory; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.tests.util.TestUtil; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.packed.PackedInts; + +public class TestForUtil extends LuceneTestCase { + + public void testEncodeDecode() throws IOException { + final int iterations = RandomNumbers.randomIntBetween(random(), 50, 1000); + final int[] values = new int[iterations * ForUtil.BLOCK_SIZE]; + + for (int i = 0; i < iterations; ++i) { + final int bpv = TestUtil.nextInt(random(), 1, 31); + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + values[i * ForUtil.BLOCK_SIZE + j] = + RandomNumbers.randomIntBetween(random(), 0, (int) PackedInts.maxValue(bpv)); + } + } + + final Directory d = new ByteBuffersDirectory(); + final long endPointer; + + { + // encode + IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT); + final ForUtil forUtil = new ForUtil(); + + for (int i = 0; i < iterations; ++i) { + long[] source = new long[ForUtil.BLOCK_SIZE]; + long or = 0; + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + source[j] = values[i * ForUtil.BLOCK_SIZE + j]; + or |= source[j]; + } + final int bpv = PackedInts.bitsRequired(or); + out.writeByte((byte) bpv); + forUtil.encode(source, bpv, out); + } + endPointer = out.getFilePointer(); + out.close(); + } + + { + // decode + IndexInput in = d.openInput("test.bin", IOContext.READONCE); + ForUtil forUtil = new ForUtil(); + for (int i = 0; i < iterations; ++i) { + final int bitsPerValue = in.readByte(); + final long currentFilePointer = in.getFilePointer(); + final long[] restored = new long[ForUtil.BLOCK_SIZE]; + forUtil.decode(bitsPerValue, in, restored); + int[] ints = new int[ForUtil.BLOCK_SIZE]; + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + ints[j] = Math.toIntExact(restored[j]); + } + assertArrayEquals( + Arrays.toString(ints), + ArrayUtil.copyOfSubArray(values, i * ForUtil.BLOCK_SIZE, (i + 1) * ForUtil.BLOCK_SIZE), + ints); + assertEquals(ForUtil.numBytes(bitsPerValue), in.getFilePointer() - currentFilePointer); + } + assertEquals(endPointer, in.getFilePointer()); + in.close(); + } + + d.close(); + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestLucene912PostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestLucene912PostingsFormat.java similarity index 96% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestLucene912PostingsFormat.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestLucene912PostingsFormat.java index 1b8d0618c60..adf8aaf9ec7 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestLucene912PostingsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestLucene912PostingsFormat.java @@ -14,17 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.backward_codecs.lucene912; import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; +import org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsReader.MutableImpactList; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.CompetitiveImpactAccumulator; import org.apache.lucene.codecs.lucene90.blocktree.FieldReader; import org.apache.lucene.codecs.lucene90.blocktree.Stats; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader.MutableImpactList; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.index.DirectoryReader; @@ -45,7 +45,7 @@ public class TestLucene912PostingsFormat extends BasePostingsFormatTestCase { @Override protected Codec getCodec() { - return TestUtil.alwaysPostingsFormat(new Lucene912PostingsFormat()); + return TestUtil.alwaysPostingsFormat(new Lucene912RWPostingsFormat()); } public void testVInt15() throws IOException { diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestPForUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestPForUtil.java new file mode 100644 index 00000000000..f3d550dff4d --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestPForUtil.java @@ -0,0 +1,104 @@ +/* + * 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.backward_codecs.lucene912; + +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; +import java.io.IOException; +import java.util.Arrays; +import org.apache.lucene.store.ByteBuffersDirectory; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.tests.util.TestUtil; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.packed.PackedInts; + +public class TestPForUtil extends LuceneTestCase { + + public void testEncodeDecode() throws IOException { + final int iterations = RandomNumbers.randomIntBetween(random(), 50, 1000); + final int[] values = createTestData(iterations, 31); + + final Directory d = new ByteBuffersDirectory(); + final long endPointer = encodeTestData(iterations, values, d); + + IndexInput in = d.openInput("test.bin", IOContext.READONCE); + final PForUtil pforUtil = new PForUtil(); + for (int i = 0; i < iterations; ++i) { + if (random().nextInt(5) == 0) { + PForUtil.skip(in); + continue; + } + final long[] restored = new long[ForUtil.BLOCK_SIZE]; + pforUtil.decode(in, restored); + int[] ints = new int[ForUtil.BLOCK_SIZE]; + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + ints[j] = Math.toIntExact(restored[j]); + } + assertArrayEquals( + Arrays.toString(ints), + ArrayUtil.copyOfSubArray(values, i * ForUtil.BLOCK_SIZE, (i + 1) * ForUtil.BLOCK_SIZE), + ints); + } + assertEquals(endPointer, in.getFilePointer()); + in.close(); + + d.close(); + } + + private int[] createTestData(int iterations, int maxBpv) { + final int[] values = new int[iterations * ForUtil.BLOCK_SIZE]; + + for (int i = 0; i < iterations; ++i) { + final int bpv = TestUtil.nextInt(random(), 0, maxBpv); + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + values[i * ForUtil.BLOCK_SIZE + j] = + RandomNumbers.randomIntBetween(random(), 0, (int) PackedInts.maxValue(bpv)); + if (random().nextInt(100) == 0) { + final int exceptionBpv; + if (random().nextInt(10) == 0) { + exceptionBpv = Math.min(bpv + TestUtil.nextInt(random(), 9, 16), maxBpv); + } else { + exceptionBpv = Math.min(bpv + TestUtil.nextInt(random(), 1, 8), maxBpv); + } + values[i * ForUtil.BLOCK_SIZE + j] |= random().nextInt(1 << (exceptionBpv - bpv)) << bpv; + } + } + } + + return values; + } + + private long encodeTestData(int iterations, int[] values, Directory d) throws IOException { + IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT); + final PForUtil pforUtil = new PForUtil(); + + for (int i = 0; i < iterations; ++i) { + long[] source = new long[ForUtil.BLOCK_SIZE]; + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + source[j] = values[i * ForUtil.BLOCK_SIZE + j]; + } + pforUtil.encode(source, out); + } + final long endPointer = out.getFilePointer(); + out.close(); + + return endPointer; + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPostingsUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestPostingsUtil.java similarity index 97% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPostingsUtil.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestPostingsUtil.java index 870f6e94e98..869345ed4f8 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPostingsUtil.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene912/TestPostingsUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.backward_codecs.lucene912; import java.io.IOException; import org.apache.lucene.store.Directory; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java index 01d5b4ab17f..6b89cb6b8a4 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java @@ -19,12 +19,13 @@ package org.apache.lucene.backward_codecs.lucene99; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.KnnVectorsFormat; +import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.tests.index.BaseKnnVectorsFormatTestCase; public class TestLucene99HnswScalarQuantizedVectorsFormat extends BaseKnnVectorsFormatTestCase { @Override protected Codec getCodec() { - return new Lucene99Codec() { + return new Lucene101Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return new Lucene99RWHnswScalarQuantizationVectorsFormat(); diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt7HnswBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt7HnswBackwardsCompatibility.java index 7e4b59542fa..402747a3138 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt7HnswBackwardsCompatibility.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt7HnswBackwardsCompatibility.java @@ -20,10 +20,10 @@ import static org.apache.lucene.backward_index.TestBasicBackwardsCompatibility.a import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import java.io.IOException; -import org.apache.lucene.backward_codecs.lucene99.Lucene99Codec; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader; @@ -69,7 +69,7 @@ public class TestInt7HnswBackwardsCompatibility extends BackwardsCompatibilityTe } protected Codec getCodec() { - return new Lucene99Codec() { + return new Lucene101Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return new Lucene99HnswScalarQuantizedVectorsFormat( diff --git a/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/AdvanceBenchmark.java b/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/AdvanceBenchmark.java index 7046e10c06e..a47242c6e57 100644 --- a/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/AdvanceBenchmark.java +++ b/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/AdvanceBenchmark.java @@ -50,9 +50,9 @@ import org.openjdk.jmh.annotations.Warmup; }) public class AdvanceBenchmark { - private final long[] values = new long[129]; + private final int[] values = new int[129]; private final int[] startIndexes = new int[1_000]; - private final long[] targets = new long[startIndexes.length]; + private final int[] targets = new int[startIndexes.length]; @Setup(Level.Trial) public void setup() throws Exception { @@ -75,7 +75,7 @@ public class AdvanceBenchmark { } @CompilerControl(CompilerControl.Mode.DONT_INLINE) - private static int binarySearch(long[] values, long target, int startIndex) { + private static int binarySearch(int[] values, int target, int startIndex) { // Standard binary search int i = Arrays.binarySearch(values, startIndex, values.length, target); if (i < 0) { @@ -92,7 +92,7 @@ public class AdvanceBenchmark { } @CompilerControl(CompilerControl.Mode.DONT_INLINE) - private static int inlinedBranchlessBinarySearch(long[] values, long target) { + private static int inlinedBranchlessBinarySearch(int[] values, int target) { // This compiles to cmov instructions. int start = 0; @@ -129,7 +129,7 @@ public class AdvanceBenchmark { } @CompilerControl(CompilerControl.Mode.DONT_INLINE) - private static int linearSearch(long[] values, long target, int startIndex) { + private static int linearSearch(int[] values, long target, int startIndex) { // Naive linear search. for (int i = startIndex; i < values.length; ++i) { if (values[i] >= target) { @@ -147,7 +147,7 @@ public class AdvanceBenchmark { } @CompilerControl(CompilerControl.Mode.DONT_INLINE) - private static int vectorUtilSearch(long[] values, long target, int startIndex) { + private static int vectorUtilSearch(int[] values, int target, int startIndex) { return VectorUtil.findNextGEQ(values, 128, target, startIndex); } @@ -159,7 +159,7 @@ public class AdvanceBenchmark { public static void main(String[] args) { // For testing purposes - long[] values = new long[129]; + int[] values = new int[129]; for (int i = 0; i < 128; ++i) { values[i] = i; } diff --git a/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/PostingIndexInputBenchmark.java b/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/PostingIndexInputBenchmark.java index 4ebab2f4024..241b289c5f6 100644 --- a/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/PostingIndexInputBenchmark.java +++ b/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/PostingIndexInputBenchmark.java @@ -21,9 +21,9 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.Random; import java.util.concurrent.TimeUnit; -import org.apache.lucene.codecs.lucene912.ForDeltaUtil; -import org.apache.lucene.codecs.lucene912.ForUtil; -import org.apache.lucene.codecs.lucene912.PostingIndexInput; +import org.apache.lucene.codecs.lucene101.ForDeltaUtil; +import org.apache.lucene.codecs.lucene101.ForUtil; +import org.apache.lucene.codecs.lucene101.PostingIndexInput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; @@ -61,7 +61,7 @@ public class PostingIndexInputBenchmark { private PostingIndexInput postingIn; private final ForUtil forUtil = new ForUtil(); private final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(); - private final long[] values = new long[128]; + private final int[] values = new int[ForUtil.BLOCK_SIZE]; @Param({"2", "3", "4", "5", "6", "7", "8", "9", "10"}) public int bpv; diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java index db704148573..29854aa500f 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java @@ -22,14 +22,14 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsReader; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsWriter; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.util.IOUtils; -/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene912PostingsWriter}. */ +/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene101PostingsWriter}. */ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { private final int minTermBlockSize; @@ -67,7 +67,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene101PostingsWriter(state); boolean success = false; try { @@ -84,7 +84,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postingsReader = new Lucene912PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene101PostingsReader(state); boolean success = false; try { FieldsProducer ret = new OrdsBlockTreeTermsReader(postingsReader, state); diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java index 8e17edb9e2c..5af23fb4945 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java @@ -24,7 +24,7 @@ import java.util.TreeMap; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat; import org.apache.lucene.index.BaseTermsEnum; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.Fields; @@ -54,7 +54,7 @@ import org.apache.lucene.util.automaton.TransitionAccessor; // - or: longer dense skip lists than just next byte? /** - * Wraps {@link Lucene912PostingsFormat} format for on-disk storage, but then at read time loads and + * Wraps {@link Lucene101PostingsFormat} format for on-disk storage, but then at read time loads and * stores all terms and postings directly in RAM as byte[], int[]. * *

WARNING: This is exceptionally RAM intensive: it makes no effort to compress the @@ -97,12 +97,12 @@ public final class DirectPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - return PostingsFormat.forName("Lucene912").fieldsConsumer(state); + return PostingsFormat.forName("Lucene101").fieldsConsumer(state); } @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - FieldsProducer postings = PostingsFormat.forName("Lucene912").fieldsProducer(state); + FieldsProducer postings = PostingsFormat.forName("Lucene101").fieldsProducer(state); if (state.context.context() != IOContext.Context.MERGE) { FieldsProducer loadedPostings; try { diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java index b184f85b176..4893ee8ad26 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java @@ -22,8 +22,8 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsReader; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.util.IOUtils; @@ -41,7 +41,7 @@ public final class FSTPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene101PostingsWriter(state); boolean success = false; try { @@ -57,7 +57,7 @@ public final class FSTPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postingsReader = new Lucene912PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene101PostingsReader(state); boolean success = false; try { FieldsProducer ret = new FSTTermsReader(state, postingsReader); diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/DeltaBaseTermStateSerializer.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/DeltaBaseTermStateSerializer.java index 1f876e5e9d1..140d62b4e96 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/DeltaBaseTermStateSerializer.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/DeltaBaseTermStateSerializer.java @@ -17,13 +17,13 @@ package org.apache.lucene.codecs.uniformsplit; -import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.BLOCK_SIZE; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.BLOCK_SIZE; import java.io.IOException; import org.apache.lucene.codecs.BlockTermState; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.IntBlockTermState; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsReader; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsWriter; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.TermState; @@ -34,7 +34,7 @@ import org.apache.lucene.util.RamUsageEstimator; /** * {@link TermState} serializer which encodes each file pointer as a delta relative to a base file - * pointer. It differs from {@link Lucene912PostingsWriter#encodeTerm} which encodes each file + * pointer. It differs from {@link Lucene101PostingsWriter#encodeTerm} which encodes each file * pointer as a delta relative to the previous file pointer. * *

It automatically sets the base file pointer to the first valid file pointer for doc start FP, @@ -95,7 +95,7 @@ public class DeltaBaseTermStateSerializer implements Accountable { /** * Writes a {@link BlockTermState} to the provided {@link DataOutput}. * - *

Simpler variant of {@link Lucene912PostingsWriter#encodeTerm(DataOutput, FieldInfo, + *

Simpler variant of {@link Lucene101PostingsWriter#encodeTerm(DataOutput, FieldInfo, * BlockTermState, boolean)}. */ public void writeTermState( @@ -145,7 +145,7 @@ public class DeltaBaseTermStateSerializer implements Accountable { /** * Reads a {@link BlockTermState} from the provided {@link DataInput}. * - *

Simpler variant of {@link Lucene912PostingsReader#decodeTerm(DataInput, FieldInfo, + *

Simpler variant of {@link Lucene101PostingsReader#decodeTerm(DataInput, FieldInfo, * BlockTermState, boolean)}. * * @param reuse {@link BlockTermState} to reuse; or null to create a new one. diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java index c8a19bf9da9..690eab21400 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java @@ -23,8 +23,8 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsReader; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.util.IOUtils; @@ -113,7 +113,7 @@ public class UniformSplitPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene101PostingsWriter(state); boolean success = false; try { FieldsConsumer termsWriter = @@ -130,7 +130,7 @@ public class UniformSplitPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postingsReader = new Lucene912PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene101PostingsReader(state); boolean success = false; try { FieldsProducer termsReader = diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/package-info.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/package-info.java index dc77bc710a1..d31b28704ef 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/package-info.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/package-info.java @@ -28,7 +28,7 @@ * org.apache.lucene.search.PhraseQuery}) *

  • Quite efficient for {@link org.apache.lucene.search.PrefixQuery} *
  • Not efficient for spell-check and {@link org.apache.lucene.search.FuzzyQuery}, in this case - * prefer {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat} + * prefer {@link org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat} * */ package org.apache.lucene.codecs.uniformsplit; diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java index a0ea5833e2e..22e0eff62c1 100644 --- a/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java +++ b/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java @@ -22,7 +22,7 @@ import java.io.IOException; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.KnnVectorsFormat; -import org.apache.lucene.codecs.lucene100.Lucene100Codec; +import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.KnnByteVectorField; @@ -42,7 +42,7 @@ import org.apache.lucene.tests.index.BaseIndexFileFormatTestCase; public class TestHnswBitVectorsFormat extends BaseIndexFileFormatTestCase { @Override protected Codec getCodec() { - return new Lucene100Codec() { + return new Lucene101Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return new HnswBitVectorsFormat(); diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/tests/MockTermStateFactory.java b/lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/tests/MockTermStateFactory.java index 51891b9d0ad..0708f3b8050 100644 --- a/lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/tests/MockTermStateFactory.java +++ b/lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/tests/MockTermStateFactory.java @@ -17,7 +17,7 @@ package org.apache.lucene.codecs.lucene90.tests; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.IntBlockTermState; /** Test utility class to create mock {@link IntBlockTermState}. */ public class MockTermStateFactory { diff --git a/lucene/core/src/generated/checksums/generateForDeltaUtil.json b/lucene/core/src/generated/checksums/generateForDeltaUtil.json index bb88ff10324..5cc8a3fe5a0 100644 --- a/lucene/core/src/generated/checksums/generateForDeltaUtil.json +++ b/lucene/core/src/generated/checksums/generateForDeltaUtil.json @@ -1,4 +1,4 @@ { - "lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java": "b662da5848b0decc8bceb4225f433875ae9e3c11", - "lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForDeltaUtil.py": "01787b97bbe79edb7703498cef8ddb85901a6b1e" + "lucene/core/src/java/org/apache/lucene/codecs/lucene101/ForDeltaUtil.java": "0ff7fb9159693055d9e4b9468b004166156f6550", + "lucene/core/src/java/org/apache/lucene/codecs/lucene101/gen_ForDeltaUtil.py": "8c55b7aaced028388408c5eb968b1f1197e11142" } \ No newline at end of file diff --git a/lucene/core/src/generated/checksums/generateForUtil.json b/lucene/core/src/generated/checksums/generateForUtil.json index e862c737e5d..6f61f8fc2c7 100644 --- a/lucene/core/src/generated/checksums/generateForUtil.json +++ b/lucene/core/src/generated/checksums/generateForUtil.json @@ -1,4 +1,4 @@ { - "lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java": "02e0c8c290e65d0314664fde24c9331bdec44925", - "lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py": "d7850f37e52a16c6592322950d0f6219cad23a33" + "lucene/core/src/java/org/apache/lucene/codecs/lucene101/ForUtil.java": "10ceb79f031232bc1e4564db7e3ebb16eedd2e0a", + "lucene/core/src/java/org/apache/lucene/codecs/lucene101/gen_ForUtil.py": "d69e734bce30375952046a3776bbb7a5c1edbd51" } \ No newline at end of file diff --git a/lucene/core/src/java/module-info.java b/lucene/core/src/java/module-info.java index 6fd1767aa34..85aff572249 100644 --- a/lucene/core/src/java/module-info.java +++ b/lucene/core/src/java/module-info.java @@ -15,8 +15,6 @@ * limitations under the License. */ -import org.apache.lucene.codecs.lucene100.Lucene100Codec; - /** Lucene Core. */ @SuppressWarnings("module") // the test framework is compiled after the core... module org.apache.lucene.core { @@ -33,8 +31,7 @@ module org.apache.lucene.core { exports org.apache.lucene.codecs.lucene94; exports org.apache.lucene.codecs.lucene95; exports org.apache.lucene.codecs.lucene99; - exports org.apache.lucene.codecs.lucene912; - exports org.apache.lucene.codecs.lucene100; + exports org.apache.lucene.codecs.lucene101; exports org.apache.lucene.codecs.perfield; exports org.apache.lucene.codecs; exports org.apache.lucene.document; @@ -73,7 +70,7 @@ module org.apache.lucene.core { provides org.apache.lucene.analysis.TokenizerFactory with org.apache.lucene.analysis.standard.StandardTokenizerFactory; provides org.apache.lucene.codecs.Codec with - Lucene100Codec; + org.apache.lucene.codecs.lucene101.Lucene101Codec; provides org.apache.lucene.codecs.DocValuesFormat with org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; provides org.apache.lucene.codecs.KnnVectorsFormat with @@ -81,7 +78,7 @@ module org.apache.lucene.core { org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat, org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat; provides org.apache.lucene.codecs.PostingsFormat with - org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; + org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat; provides org.apache.lucene.index.SortFieldProvider with org.apache.lucene.search.SortField.Provider, org.apache.lucene.search.SortedNumericSortField.Provider, diff --git a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java index ff5a5bb21c0..50974d13ff3 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java @@ -55,7 +55,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI { return LOADER; } - static Codec defaultCodec = LOADER.lookup("Lucene100"); + static Codec defaultCodec = LOADER.lookup("Lucene101"); } private final String name; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene101/ForDeltaUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/ForDeltaUtil.java new file mode 100644 index 00000000000..2fe9a1cce6f --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/ForDeltaUtil.java @@ -0,0 +1,525 @@ +// This file has been automatically generated, DO NOT EDIT + +/* + * 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.lucene101; + +import static org.apache.lucene.codecs.lucene101.ForUtil.*; + +import java.io.IOException; +import org.apache.lucene.internal.vectorization.PostingDecodingUtil; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.packed.PackedInts; + +/** + * Inspired from https://fulmicoton.com/posts/bitpacking/ Encodes multiple integers in a Java int to + * get SIMD-like speedups. If bitsPerValue <= 4 then we pack 4 ints per Java int else if + * bitsPerValue <= 11 we pack 2 ints per Java int else we use scalar operations. + */ +public final class ForDeltaUtil { + + private static final int HALF_BLOCK_SIZE = BLOCK_SIZE / 2; + private static final int ONE_BLOCK_SIZE_FOURTH = BLOCK_SIZE / 4; + private static final int TWO_BLOCK_SIZE_FOURTHS = BLOCK_SIZE / 2; + private static final int THREE_BLOCK_SIZE_FOURTHS = 3 * BLOCK_SIZE / 4; + + // IDENTITY_PLUS_ONE[i] == i+1 + private static final int[] IDENTITY_PLUS_ONE = new int[ForUtil.BLOCK_SIZE]; + + static { + for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { + IDENTITY_PLUS_ONE[i] = i + 1; + } + } + + private static void prefixSumOfOnes(int[] arr, int base) { + System.arraycopy(IDENTITY_PLUS_ONE, 0, arr, 0, ForUtil.BLOCK_SIZE); + // This loop gets auto-vectorized + for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { + arr[i] += base; + } + } + + private static void prefixSum8(int[] arr, int base) { + // When the number of bits per value is 4 or less, we can sum up all values in a block without + // risking overflowing an 8-bits integer. This allows computing the prefix sum by summing up 4 + // values at once. + innerPrefixSum8(arr); + expand8(arr); + final int l0 = base; + final int l1 = l0 + arr[ONE_BLOCK_SIZE_FOURTH - 1]; + final int l2 = l1 + arr[TWO_BLOCK_SIZE_FOURTHS - 1]; + final int l3 = l2 + arr[THREE_BLOCK_SIZE_FOURTHS - 1]; + + for (int i = 0; i < ONE_BLOCK_SIZE_FOURTH; ++i) { + arr[i] += l0; + arr[ONE_BLOCK_SIZE_FOURTH + i] += l1; + arr[TWO_BLOCK_SIZE_FOURTHS + i] += l2; + arr[THREE_BLOCK_SIZE_FOURTHS + i] += l3; + } + } + + private static void prefixSum16(int[] arr, int base) { + // When the number of bits per value is 11 or less, we can sum up all values in a block without + // risking overflowing an 16-bits integer. This allows computing the prefix sum by summing up 2 + // values at once. + innerPrefixSum16(arr); + expand16(arr); + final int l0 = base; + final int l1 = base + arr[HALF_BLOCK_SIZE - 1]; + for (int i = 0; i < HALF_BLOCK_SIZE; ++i) { + arr[i] += l0; + arr[HALF_BLOCK_SIZE + i] += l1; + } + } + + private static void prefixSum32(int[] arr, int base) { + arr[0] += base; + for (int i = 1; i < BLOCK_SIZE; ++i) { + arr[i] += arr[i - 1]; + } + } + + // For some reason unrolling seems to help + private static void innerPrefixSum8(int[] arr) { + arr[1] += arr[0]; + arr[2] += arr[1]; + arr[3] += arr[2]; + arr[4] += arr[3]; + arr[5] += arr[4]; + arr[6] += arr[5]; + arr[7] += arr[6]; + arr[8] += arr[7]; + arr[9] += arr[8]; + arr[10] += arr[9]; + arr[11] += arr[10]; + arr[12] += arr[11]; + arr[13] += arr[12]; + arr[14] += arr[13]; + arr[15] += arr[14]; + arr[16] += arr[15]; + arr[17] += arr[16]; + arr[18] += arr[17]; + arr[19] += arr[18]; + arr[20] += arr[19]; + arr[21] += arr[20]; + arr[22] += arr[21]; + arr[23] += arr[22]; + arr[24] += arr[23]; + arr[25] += arr[24]; + arr[26] += arr[25]; + arr[27] += arr[26]; + arr[28] += arr[27]; + arr[29] += arr[28]; + arr[30] += arr[29]; + arr[31] += arr[30]; + } + + // For some reason unrolling seems to help + private static void innerPrefixSum16(int[] arr) { + arr[1] += arr[0]; + arr[2] += arr[1]; + arr[3] += arr[2]; + arr[4] += arr[3]; + arr[5] += arr[4]; + arr[6] += arr[5]; + arr[7] += arr[6]; + arr[8] += arr[7]; + arr[9] += arr[8]; + arr[10] += arr[9]; + arr[11] += arr[10]; + arr[12] += arr[11]; + arr[13] += arr[12]; + arr[14] += arr[13]; + arr[15] += arr[14]; + arr[16] += arr[15]; + arr[17] += arr[16]; + arr[18] += arr[17]; + arr[19] += arr[18]; + arr[20] += arr[19]; + arr[21] += arr[20]; + arr[22] += arr[21]; + arr[23] += arr[22]; + arr[24] += arr[23]; + arr[25] += arr[24]; + arr[26] += arr[25]; + arr[27] += arr[26]; + arr[28] += arr[27]; + arr[29] += arr[28]; + arr[30] += arr[29]; + arr[31] += arr[30]; + arr[32] += arr[31]; + arr[33] += arr[32]; + arr[34] += arr[33]; + arr[35] += arr[34]; + arr[36] += arr[35]; + arr[37] += arr[36]; + arr[38] += arr[37]; + arr[39] += arr[38]; + arr[40] += arr[39]; + arr[41] += arr[40]; + arr[42] += arr[41]; + arr[43] += arr[42]; + arr[44] += arr[43]; + arr[45] += arr[44]; + arr[46] += arr[45]; + arr[47] += arr[46]; + arr[48] += arr[47]; + arr[49] += arr[48]; + arr[50] += arr[49]; + arr[51] += arr[50]; + arr[52] += arr[51]; + arr[53] += arr[52]; + arr[54] += arr[53]; + arr[55] += arr[54]; + arr[56] += arr[55]; + arr[57] += arr[56]; + arr[58] += arr[57]; + arr[59] += arr[58]; + arr[60] += arr[59]; + arr[61] += arr[60]; + arr[62] += arr[61]; + arr[63] += arr[62]; + } + + private final int[] tmp = new int[BLOCK_SIZE]; + + /** + * Encode deltas of a strictly monotonically increasing sequence of integers. The provided {@code + * ints} are expected to be deltas between consecutive values. + */ + void encodeDeltas(int[] ints, DataOutput out) throws IOException { + if (ints[0] == 1 && PForUtil.allEqual(ints)) { // happens with very dense postings + out.writeByte((byte) 0); + } else { + int or = 0; + for (int l : ints) { + or |= l; + } + assert or != 0; + final int bitsPerValue = PackedInts.bitsRequired(or); + out.writeByte((byte) bitsPerValue); + + final int primitiveSize; + if (bitsPerValue <= 3) { + primitiveSize = 8; + collapse8(ints); + } else if (bitsPerValue <= 10) { + primitiveSize = 16; + collapse16(ints); + } else { + primitiveSize = 32; + } + encode(ints, bitsPerValue, primitiveSize, out, tmp); + } + } + + /** Decode deltas, compute the prefix sum and add {@code base} to all decoded ints. */ + void decodeAndPrefixSum(PostingDecodingUtil pdu, int base, int[] ints) throws IOException { + final int bitsPerValue = Byte.toUnsignedInt(pdu.in.readByte()); + if (bitsPerValue == 0) { + prefixSumOfOnes(ints, base); + } else { + decodeAndPrefixSum(bitsPerValue, pdu, base, ints); + } + } + + /** Delta-decode 128 integers into {@code ints}. */ + void decodeAndPrefixSum(int bitsPerValue, PostingDecodingUtil pdu, int base, int[] ints) + throws IOException { + switch (bitsPerValue) { + case 1: + decode1(pdu, ints); + prefixSum8(ints, base); + break; + case 2: + decode2(pdu, ints); + prefixSum8(ints, base); + break; + case 3: + decode3(pdu, tmp, ints); + prefixSum8(ints, base); + break; + case 4: + decode4To16(pdu, ints); + prefixSum16(ints, base); + break; + case 5: + decode5To16(pdu, tmp, ints); + prefixSum16(ints, base); + break; + case 6: + decode6To16(pdu, tmp, ints); + prefixSum16(ints, base); + break; + case 7: + decode7To16(pdu, tmp, ints); + prefixSum16(ints, base); + break; + case 8: + decode8To16(pdu, ints); + prefixSum16(ints, base); + break; + case 9: + decode9(pdu, tmp, ints); + prefixSum16(ints, base); + break; + case 10: + decode10(pdu, tmp, ints); + prefixSum16(ints, base); + break; + case 11: + decode11To32(pdu, tmp, ints); + prefixSum32(ints, base); + break; + case 12: + decode12To32(pdu, tmp, ints); + prefixSum32(ints, base); + break; + case 13: + decode13To32(pdu, tmp, ints); + prefixSum32(ints, base); + break; + case 14: + decode14To32(pdu, tmp, ints); + prefixSum32(ints, base); + break; + case 15: + decode15To32(pdu, tmp, ints); + prefixSum32(ints, base); + break; + case 16: + decode16To32(pdu, ints); + prefixSum32(ints, base); + break; + case 17: + decode17(pdu, tmp, ints); + prefixSum32(ints, base); + break; + case 18: + decode18(pdu, tmp, ints); + prefixSum32(ints, base); + break; + case 19: + decode19(pdu, tmp, ints); + prefixSum32(ints, base); + break; + case 20: + decode20(pdu, tmp, ints); + prefixSum32(ints, base); + break; + case 21: + decode21(pdu, tmp, ints); + prefixSum32(ints, base); + break; + case 22: + decode22(pdu, tmp, ints); + prefixSum32(ints, base); + break; + case 23: + decode23(pdu, tmp, ints); + prefixSum32(ints, base); + break; + case 24: + decode24(pdu, tmp, ints); + prefixSum32(ints, base); + break; + default: + decodeSlow(bitsPerValue, pdu, tmp, ints); + prefixSum32(ints, base); + break; + } + } + + private static void decode4To16(PostingDecodingUtil pdu, int[] ints) throws IOException { + pdu.splitInts(16, ints, 12, 4, MASK16_4, ints, 48, MASK16_4); + } + + private static void decode5To16(PostingDecodingUtil pdu, int[] tmp, int[] ints) + throws IOException { + pdu.splitInts(20, ints, 11, 5, MASK16_5, tmp, 0, MASK16_1); + for (int iter = 0, tmpIdx = 0, intsIdx = 60; iter < 4; ++iter, tmpIdx += 5, intsIdx += 1) { + int l0 = tmp[tmpIdx + 0] << 4; + l0 |= tmp[tmpIdx + 1] << 3; + l0 |= tmp[tmpIdx + 2] << 2; + l0 |= tmp[tmpIdx + 3] << 1; + l0 |= tmp[tmpIdx + 4] << 0; + ints[intsIdx + 0] = l0; + } + } + + private static void decode6To16(PostingDecodingUtil pdu, int[] tmp, int[] ints) + throws IOException { + pdu.splitInts(24, ints, 10, 6, MASK16_6, tmp, 0, MASK16_4); + for (int iter = 0, tmpIdx = 0, intsIdx = 48; iter < 8; ++iter, tmpIdx += 3, intsIdx += 2) { + int l0 = tmp[tmpIdx + 0] << 2; + l0 |= (tmp[tmpIdx + 1] >>> 2) & MASK16_2; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 1] & MASK16_2) << 4; + l1 |= tmp[tmpIdx + 2] << 0; + ints[intsIdx + 1] = l1; + } + } + + private static void decode7To16(PostingDecodingUtil pdu, int[] tmp, int[] ints) + throws IOException { + pdu.splitInts(28, ints, 9, 7, MASK16_7, tmp, 0, MASK16_2); + for (int iter = 0, tmpIdx = 0, intsIdx = 56; iter < 4; ++iter, tmpIdx += 7, intsIdx += 2) { + int l0 = tmp[tmpIdx + 0] << 5; + l0 |= tmp[tmpIdx + 1] << 3; + l0 |= tmp[tmpIdx + 2] << 1; + l0 |= (tmp[tmpIdx + 3] >>> 1) & MASK16_1; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 3] & MASK16_1) << 6; + l1 |= tmp[tmpIdx + 4] << 4; + l1 |= tmp[tmpIdx + 5] << 2; + l1 |= tmp[tmpIdx + 6] << 0; + ints[intsIdx + 1] = l1; + } + } + + private static void decode8To16(PostingDecodingUtil pdu, int[] ints) throws IOException { + pdu.splitInts(32, ints, 8, 8, MASK16_8, ints, 32, MASK16_8); + } + + private static void decode11To32(PostingDecodingUtil pdu, int[] tmp, int[] ints) + throws IOException { + pdu.splitInts(44, ints, 21, 11, MASK32_11, tmp, 0, MASK32_10); + for (int iter = 0, tmpIdx = 0, intsIdx = 88; iter < 4; ++iter, tmpIdx += 11, intsIdx += 10) { + int l0 = tmp[tmpIdx + 0] << 1; + l0 |= (tmp[tmpIdx + 1] >>> 9) & MASK32_1; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 1] & MASK32_9) << 2; + l1 |= (tmp[tmpIdx + 2] >>> 8) & MASK32_2; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 2] & MASK32_8) << 3; + l2 |= (tmp[tmpIdx + 3] >>> 7) & MASK32_3; + ints[intsIdx + 2] = l2; + int l3 = (tmp[tmpIdx + 3] & MASK32_7) << 4; + l3 |= (tmp[tmpIdx + 4] >>> 6) & MASK32_4; + ints[intsIdx + 3] = l3; + int l4 = (tmp[tmpIdx + 4] & MASK32_6) << 5; + l4 |= (tmp[tmpIdx + 5] >>> 5) & MASK32_5; + ints[intsIdx + 4] = l4; + int l5 = (tmp[tmpIdx + 5] & MASK32_5) << 6; + l5 |= (tmp[tmpIdx + 6] >>> 4) & MASK32_6; + ints[intsIdx + 5] = l5; + int l6 = (tmp[tmpIdx + 6] & MASK32_4) << 7; + l6 |= (tmp[tmpIdx + 7] >>> 3) & MASK32_7; + ints[intsIdx + 6] = l6; + int l7 = (tmp[tmpIdx + 7] & MASK32_3) << 8; + l7 |= (tmp[tmpIdx + 8] >>> 2) & MASK32_8; + ints[intsIdx + 7] = l7; + int l8 = (tmp[tmpIdx + 8] & MASK32_2) << 9; + l8 |= (tmp[tmpIdx + 9] >>> 1) & MASK32_9; + ints[intsIdx + 8] = l8; + int l9 = (tmp[tmpIdx + 9] & MASK32_1) << 10; + l9 |= tmp[tmpIdx + 10] << 0; + ints[intsIdx + 9] = l9; + } + } + + private static void decode12To32(PostingDecodingUtil pdu, int[] tmp, int[] ints) + throws IOException { + pdu.splitInts(48, ints, 20, 12, MASK32_12, tmp, 0, MASK32_8); + for (int iter = 0, tmpIdx = 0, intsIdx = 96; iter < 16; ++iter, tmpIdx += 3, intsIdx += 2) { + int l0 = tmp[tmpIdx + 0] << 4; + l0 |= (tmp[tmpIdx + 1] >>> 4) & MASK32_4; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 1] & MASK32_4) << 8; + l1 |= tmp[tmpIdx + 2] << 0; + ints[intsIdx + 1] = l1; + } + } + + private static void decode13To32(PostingDecodingUtil pdu, int[] tmp, int[] ints) + throws IOException { + pdu.splitInts(52, ints, 19, 13, MASK32_13, tmp, 0, MASK32_6); + for (int iter = 0, tmpIdx = 0, intsIdx = 104; iter < 4; ++iter, tmpIdx += 13, intsIdx += 6) { + int l0 = tmp[tmpIdx + 0] << 7; + l0 |= tmp[tmpIdx + 1] << 1; + l0 |= (tmp[tmpIdx + 2] >>> 5) & MASK32_1; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 2] & MASK32_5) << 8; + l1 |= tmp[tmpIdx + 3] << 2; + l1 |= (tmp[tmpIdx + 4] >>> 4) & MASK32_2; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 4] & MASK32_4) << 9; + l2 |= tmp[tmpIdx + 5] << 3; + l2 |= (tmp[tmpIdx + 6] >>> 3) & MASK32_3; + ints[intsIdx + 2] = l2; + int l3 = (tmp[tmpIdx + 6] & MASK32_3) << 10; + l3 |= tmp[tmpIdx + 7] << 4; + l3 |= (tmp[tmpIdx + 8] >>> 2) & MASK32_4; + ints[intsIdx + 3] = l3; + int l4 = (tmp[tmpIdx + 8] & MASK32_2) << 11; + l4 |= tmp[tmpIdx + 9] << 5; + l4 |= (tmp[tmpIdx + 10] >>> 1) & MASK32_5; + ints[intsIdx + 4] = l4; + int l5 = (tmp[tmpIdx + 10] & MASK32_1) << 12; + l5 |= tmp[tmpIdx + 11] << 6; + l5 |= tmp[tmpIdx + 12] << 0; + ints[intsIdx + 5] = l5; + } + } + + private static void decode14To32(PostingDecodingUtil pdu, int[] tmp, int[] ints) + throws IOException { + pdu.splitInts(56, ints, 18, 14, MASK32_14, tmp, 0, MASK32_4); + for (int iter = 0, tmpIdx = 0, intsIdx = 112; iter < 8; ++iter, tmpIdx += 7, intsIdx += 2) { + int l0 = tmp[tmpIdx + 0] << 10; + l0 |= tmp[tmpIdx + 1] << 6; + l0 |= tmp[tmpIdx + 2] << 2; + l0 |= (tmp[tmpIdx + 3] >>> 2) & MASK32_2; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 3] & MASK32_2) << 12; + l1 |= tmp[tmpIdx + 4] << 8; + l1 |= tmp[tmpIdx + 5] << 4; + l1 |= tmp[tmpIdx + 6] << 0; + ints[intsIdx + 1] = l1; + } + } + + private static void decode15To32(PostingDecodingUtil pdu, int[] tmp, int[] ints) + throws IOException { + pdu.splitInts(60, ints, 17, 15, MASK32_15, tmp, 0, MASK32_2); + for (int iter = 0, tmpIdx = 0, intsIdx = 120; iter < 4; ++iter, tmpIdx += 15, intsIdx += 2) { + int l0 = tmp[tmpIdx + 0] << 13; + l0 |= tmp[tmpIdx + 1] << 11; + l0 |= tmp[tmpIdx + 2] << 9; + l0 |= tmp[tmpIdx + 3] << 7; + l0 |= tmp[tmpIdx + 4] << 5; + l0 |= tmp[tmpIdx + 5] << 3; + l0 |= tmp[tmpIdx + 6] << 1; + l0 |= (tmp[tmpIdx + 7] >>> 1) & MASK32_1; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 7] & MASK32_1) << 14; + l1 |= tmp[tmpIdx + 8] << 12; + l1 |= tmp[tmpIdx + 9] << 10; + l1 |= tmp[tmpIdx + 10] << 8; + l1 |= tmp[tmpIdx + 11] << 6; + l1 |= tmp[tmpIdx + 12] << 4; + l1 |= tmp[tmpIdx + 13] << 2; + l1 |= tmp[tmpIdx + 14] << 0; + ints[intsIdx + 1] = l1; + } + } + + private static void decode16To32(PostingDecodingUtil pdu, int[] ints) throws IOException { + pdu.splitInts(64, ints, 16, 16, MASK32_16, ints, 64, MASK32_16); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene101/ForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/ForUtil.java new file mode 100644 index 00000000000..1fe54b56fd5 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/ForUtil.java @@ -0,0 +1,841 @@ +// This file has been automatically generated, DO NOT EDIT + +/* + * 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.lucene101; + +import java.io.IOException; +import org.apache.lucene.internal.vectorization.PostingDecodingUtil; +import org.apache.lucene.store.DataOutput; + +/** + * Inspired from https://fulmicoton.com/posts/bitpacking/ Encodes multiple integers in one to get + * SIMD-like speedups. If bitsPerValue <= 8 then we pack 4 ints per Java int else if bitsPerValue + * <= 16 we pack 2 ints per Java int else we do scalar operations. + */ +public final class ForUtil { + + public static final int BLOCK_SIZE = 128; + static final int BLOCK_SIZE_LOG2 = 7; + + static int expandMask16(int mask16) { + return mask16 | (mask16 << 16); + } + + static int expandMask8(int mask8) { + return expandMask16(mask8 | (mask8 << 8)); + } + + static int mask32(int bitsPerValue) { + return (1 << bitsPerValue) - 1; + } + + static int mask16(int bitsPerValue) { + return expandMask16((1 << bitsPerValue) - 1); + } + + static int mask8(int bitsPerValue) { + return expandMask8((1 << bitsPerValue) - 1); + } + + static void expand8(int[] arr) { + for (int i = 0; i < 32; ++i) { + int l = arr[i]; + arr[i] = (l >>> 24) & 0xFF; + arr[32 + i] = (l >>> 16) & 0xFF; + arr[64 + i] = (l >>> 8) & 0xFF; + arr[96 + i] = l & 0xFF; + } + } + + static void collapse8(int[] arr) { + for (int i = 0; i < 32; ++i) { + arr[i] = (arr[i] << 24) | (arr[32 + i] << 16) | (arr[64 + i] << 8) | arr[96 + i]; + } + } + + static void expand16(int[] arr) { + for (int i = 0; i < 64; ++i) { + int l = arr[i]; + arr[i] = (l >>> 16) & 0xFFFF; + arr[64 + i] = l & 0xFFFF; + } + } + + static void collapse16(int[] arr) { + for (int i = 0; i < 64; ++i) { + arr[i] = (arr[i] << 16) | arr[64 + i]; + } + } + + private final int[] tmp = new int[BLOCK_SIZE]; + + /** Encode 128 integers from {@code ints} into {@code out}. */ + void encode(int[] ints, int bitsPerValue, DataOutput out) throws IOException { + final int nextPrimitive; + if (bitsPerValue <= 8) { + nextPrimitive = 8; + collapse8(ints); + } else if (bitsPerValue <= 16) { + nextPrimitive = 16; + collapse16(ints); + } else { + nextPrimitive = 32; + } + encode(ints, bitsPerValue, nextPrimitive, out, tmp); + } + + static void encode(int[] ints, int bitsPerValue, int primitiveSize, DataOutput out, int[] tmp) + throws IOException { + final int numInts = BLOCK_SIZE * primitiveSize / Integer.SIZE; + + final int numIntsPerShift = bitsPerValue * 4; + int idx = 0; + int shift = primitiveSize - bitsPerValue; + for (int i = 0; i < numIntsPerShift; ++i) { + tmp[i] = ints[idx++] << shift; + } + for (shift = shift - bitsPerValue; shift >= 0; shift -= bitsPerValue) { + for (int i = 0; i < numIntsPerShift; ++i) { + tmp[i] |= ints[idx++] << shift; + } + } + + final int remainingBitsPerInt = shift + bitsPerValue; + final int maskRemainingBitsPerInt; + if (primitiveSize == 8) { + maskRemainingBitsPerInt = MASKS8[remainingBitsPerInt]; + } else if (primitiveSize == 16) { + maskRemainingBitsPerInt = MASKS16[remainingBitsPerInt]; + } else { + maskRemainingBitsPerInt = MASKS32[remainingBitsPerInt]; + } + + int tmpIdx = 0; + int remainingBitsPerValue = bitsPerValue; + while (idx < numInts) { + if (remainingBitsPerValue >= remainingBitsPerInt) { + remainingBitsPerValue -= remainingBitsPerInt; + tmp[tmpIdx++] |= (ints[idx] >>> remainingBitsPerValue) & maskRemainingBitsPerInt; + if (remainingBitsPerValue == 0) { + idx++; + remainingBitsPerValue = bitsPerValue; + } + } else { + final int mask1, mask2; + if (primitiveSize == 8) { + mask1 = MASKS8[remainingBitsPerValue]; + mask2 = MASKS8[remainingBitsPerInt - remainingBitsPerValue]; + } else if (primitiveSize == 16) { + mask1 = MASKS16[remainingBitsPerValue]; + mask2 = MASKS16[remainingBitsPerInt - remainingBitsPerValue]; + } else { + mask1 = MASKS32[remainingBitsPerValue]; + mask2 = MASKS32[remainingBitsPerInt - remainingBitsPerValue]; + } + tmp[tmpIdx] |= (ints[idx++] & mask1) << (remainingBitsPerInt - remainingBitsPerValue); + remainingBitsPerValue = bitsPerValue - remainingBitsPerInt + remainingBitsPerValue; + tmp[tmpIdx++] |= (ints[idx] >>> remainingBitsPerValue) & mask2; + } + } + + for (int i = 0; i < numIntsPerShift; ++i) { + out.writeInt(tmp[i]); + } + } + + /** Number of bytes required to encode 128 integers of {@code bitsPerValue} bits per value. */ + static int numBytes(int bitsPerValue) { + return bitsPerValue << (BLOCK_SIZE_LOG2 - 3); + } + + static void decodeSlow(int bitsPerValue, PostingDecodingUtil pdu, int[] tmp, int[] ints) + throws IOException { + final int numInts = bitsPerValue << 2; + final int mask = MASKS32[bitsPerValue]; + pdu.splitInts(numInts, ints, 32 - bitsPerValue, 32, mask, tmp, 0, -1); + final int remainingBitsPerInt = 32 - bitsPerValue; + final int mask32RemainingBitsPerInt = MASKS32[remainingBitsPerInt]; + int tmpIdx = 0; + int remainingBits = remainingBitsPerInt; + for (int intsIdx = numInts; intsIdx < BLOCK_SIZE; ++intsIdx) { + int b = bitsPerValue - remainingBits; + int l = (tmp[tmpIdx++] & MASKS32[remainingBits]) << b; + while (b >= remainingBitsPerInt) { + b -= remainingBitsPerInt; + l |= (tmp[tmpIdx++] & mask32RemainingBitsPerInt) << b; + } + if (b > 0) { + l |= (tmp[tmpIdx] >>> (remainingBitsPerInt - b)) & MASKS32[b]; + remainingBits = remainingBitsPerInt - b; + } else { + remainingBits = remainingBitsPerInt; + } + ints[intsIdx] = l; + } + } + + static final int[] MASKS8 = new int[8]; + static final int[] MASKS16 = new int[16]; + static final int[] MASKS32 = new int[32]; + + static { + for (int i = 0; i < 8; ++i) { + MASKS8[i] = mask8(i); + } + for (int i = 0; i < 16; ++i) { + MASKS16[i] = mask16(i); + } + for (int i = 0; i < 32; ++i) { + MASKS32[i] = mask32(i); + } + } + + // mark values in array as final ints to avoid the cost of reading array, arrays should only be + // used when the idx is a variable + static final int MASK8_1 = MASKS8[1]; + static final int MASK8_2 = MASKS8[2]; + static final int MASK8_3 = MASKS8[3]; + static final int MASK8_4 = MASKS8[4]; + static final int MASK8_5 = MASKS8[5]; + static final int MASK8_6 = MASKS8[6]; + static final int MASK8_7 = MASKS8[7]; + static final int MASK16_1 = MASKS16[1]; + static final int MASK16_2 = MASKS16[2]; + static final int MASK16_3 = MASKS16[3]; + static final int MASK16_4 = MASKS16[4]; + static final int MASK16_5 = MASKS16[5]; + static final int MASK16_6 = MASKS16[6]; + static final int MASK16_7 = MASKS16[7]; + static final int MASK16_8 = MASKS16[8]; + static final int MASK16_9 = MASKS16[9]; + static final int MASK16_10 = MASKS16[10]; + static final int MASK16_11 = MASKS16[11]; + static final int MASK16_12 = MASKS16[12]; + static final int MASK16_13 = MASKS16[13]; + static final int MASK16_14 = MASKS16[14]; + static final int MASK16_15 = MASKS16[15]; + static final int MASK32_1 = MASKS32[1]; + static final int MASK32_2 = MASKS32[2]; + static final int MASK32_3 = MASKS32[3]; + static final int MASK32_4 = MASKS32[4]; + static final int MASK32_5 = MASKS32[5]; + static final int MASK32_6 = MASKS32[6]; + static final int MASK32_7 = MASKS32[7]; + static final int MASK32_8 = MASKS32[8]; + static final int MASK32_9 = MASKS32[9]; + static final int MASK32_10 = MASKS32[10]; + static final int MASK32_11 = MASKS32[11]; + static final int MASK32_12 = MASKS32[12]; + static final int MASK32_13 = MASKS32[13]; + static final int MASK32_14 = MASKS32[14]; + static final int MASK32_15 = MASKS32[15]; + static final int MASK32_16 = MASKS32[16]; + static final int MASK32_17 = MASKS32[17]; + static final int MASK32_18 = MASKS32[18]; + static final int MASK32_19 = MASKS32[19]; + static final int MASK32_20 = MASKS32[20]; + static final int MASK32_21 = MASKS32[21]; + static final int MASK32_22 = MASKS32[22]; + static final int MASK32_23 = MASKS32[23]; + static final int MASK32_24 = MASKS32[24]; + + /** Decode 128 integers into {@code ints}. */ + void decode(int bitsPerValue, PostingDecodingUtil pdu, int[] ints) throws IOException { + switch (bitsPerValue) { + case 1: + decode1(pdu, ints); + expand8(ints); + break; + case 2: + decode2(pdu, ints); + expand8(ints); + break; + case 3: + decode3(pdu, tmp, ints); + expand8(ints); + break; + case 4: + decode4(pdu, ints); + expand8(ints); + break; + case 5: + decode5(pdu, tmp, ints); + expand8(ints); + break; + case 6: + decode6(pdu, tmp, ints); + expand8(ints); + break; + case 7: + decode7(pdu, tmp, ints); + expand8(ints); + break; + case 8: + decode8(pdu, ints); + expand8(ints); + break; + case 9: + decode9(pdu, tmp, ints); + expand16(ints); + break; + case 10: + decode10(pdu, tmp, ints); + expand16(ints); + break; + case 11: + decode11(pdu, tmp, ints); + expand16(ints); + break; + case 12: + decode12(pdu, tmp, ints); + expand16(ints); + break; + case 13: + decode13(pdu, tmp, ints); + expand16(ints); + break; + case 14: + decode14(pdu, tmp, ints); + expand16(ints); + break; + case 15: + decode15(pdu, tmp, ints); + expand16(ints); + break; + case 16: + decode16(pdu, ints); + expand16(ints); + break; + case 17: + decode17(pdu, tmp, ints); + break; + case 18: + decode18(pdu, tmp, ints); + break; + case 19: + decode19(pdu, tmp, ints); + break; + case 20: + decode20(pdu, tmp, ints); + break; + case 21: + decode21(pdu, tmp, ints); + break; + case 22: + decode22(pdu, tmp, ints); + break; + case 23: + decode23(pdu, tmp, ints); + break; + case 24: + decode24(pdu, tmp, ints); + break; + default: + decodeSlow(bitsPerValue, pdu, tmp, ints); + break; + } + } + + static void decode1(PostingDecodingUtil pdu, int[] ints) throws IOException { + pdu.splitInts(4, ints, 7, 1, MASK8_1, ints, 28, MASK8_1); + } + + static void decode2(PostingDecodingUtil pdu, int[] ints) throws IOException { + pdu.splitInts(8, ints, 6, 2, MASK8_2, ints, 24, MASK8_2); + } + + static void decode3(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(12, ints, 5, 3, MASK8_3, tmp, 0, MASK8_2); + for (int iter = 0, tmpIdx = 0, intsIdx = 24; iter < 4; ++iter, tmpIdx += 3, intsIdx += 2) { + int l0 = tmp[tmpIdx + 0] << 1; + l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK8_1; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 1] & MASK8_1) << 2; + l1 |= tmp[tmpIdx + 2] << 0; + ints[intsIdx + 1] = l1; + } + } + + static void decode4(PostingDecodingUtil pdu, int[] ints) throws IOException { + pdu.splitInts(16, ints, 4, 4, MASK8_4, ints, 16, MASK8_4); + } + + static void decode5(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(20, ints, 3, 5, MASK8_5, tmp, 0, MASK8_3); + for (int iter = 0, tmpIdx = 0, intsIdx = 20; iter < 4; ++iter, tmpIdx += 5, intsIdx += 3) { + int l0 = tmp[tmpIdx + 0] << 2; + l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK8_2; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 1] & MASK8_1) << 4; + l1 |= tmp[tmpIdx + 2] << 1; + l1 |= (tmp[tmpIdx + 3] >>> 2) & MASK8_1; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 3] & MASK8_2) << 3; + l2 |= tmp[tmpIdx + 4] << 0; + ints[intsIdx + 2] = l2; + } + } + + static void decode6(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(24, ints, 2, 6, MASK8_6, tmp, 0, MASK8_2); + for (int iter = 0, tmpIdx = 0, intsIdx = 24; iter < 8; ++iter, tmpIdx += 3, intsIdx += 1) { + int l0 = tmp[tmpIdx + 0] << 4; + l0 |= tmp[tmpIdx + 1] << 2; + l0 |= tmp[tmpIdx + 2] << 0; + ints[intsIdx + 0] = l0; + } + } + + static void decode7(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(28, ints, 1, 7, MASK8_7, tmp, 0, MASK8_1); + for (int iter = 0, tmpIdx = 0, intsIdx = 28; iter < 4; ++iter, tmpIdx += 7, intsIdx += 1) { + int l0 = tmp[tmpIdx + 0] << 6; + l0 |= tmp[tmpIdx + 1] << 5; + l0 |= tmp[tmpIdx + 2] << 4; + l0 |= tmp[tmpIdx + 3] << 3; + l0 |= tmp[tmpIdx + 4] << 2; + l0 |= tmp[tmpIdx + 5] << 1; + l0 |= tmp[tmpIdx + 6] << 0; + ints[intsIdx + 0] = l0; + } + } + + static void decode8(PostingDecodingUtil pdu, int[] ints) throws IOException { + pdu.in.readInts(ints, 0, 32); + } + + static void decode9(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(36, ints, 7, 9, MASK16_9, tmp, 0, MASK16_7); + for (int iter = 0, tmpIdx = 0, intsIdx = 36; iter < 4; ++iter, tmpIdx += 9, intsIdx += 7) { + int l0 = tmp[tmpIdx + 0] << 2; + l0 |= (tmp[tmpIdx + 1] >>> 5) & MASK16_2; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 1] & MASK16_5) << 4; + l1 |= (tmp[tmpIdx + 2] >>> 3) & MASK16_4; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 2] & MASK16_3) << 6; + l2 |= (tmp[tmpIdx + 3] >>> 1) & MASK16_6; + ints[intsIdx + 2] = l2; + int l3 = (tmp[tmpIdx + 3] & MASK16_1) << 8; + l3 |= tmp[tmpIdx + 4] << 1; + l3 |= (tmp[tmpIdx + 5] >>> 6) & MASK16_1; + ints[intsIdx + 3] = l3; + int l4 = (tmp[tmpIdx + 5] & MASK16_6) << 3; + l4 |= (tmp[tmpIdx + 6] >>> 4) & MASK16_3; + ints[intsIdx + 4] = l4; + int l5 = (tmp[tmpIdx + 6] & MASK16_4) << 5; + l5 |= (tmp[tmpIdx + 7] >>> 2) & MASK16_5; + ints[intsIdx + 5] = l5; + int l6 = (tmp[tmpIdx + 7] & MASK16_2) << 7; + l6 |= tmp[tmpIdx + 8] << 0; + ints[intsIdx + 6] = l6; + } + } + + static void decode10(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(40, ints, 6, 10, MASK16_10, tmp, 0, MASK16_6); + for (int iter = 0, tmpIdx = 0, intsIdx = 40; iter < 8; ++iter, tmpIdx += 5, intsIdx += 3) { + int l0 = tmp[tmpIdx + 0] << 4; + l0 |= (tmp[tmpIdx + 1] >>> 2) & MASK16_4; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 1] & MASK16_2) << 8; + l1 |= tmp[tmpIdx + 2] << 2; + l1 |= (tmp[tmpIdx + 3] >>> 4) & MASK16_2; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 3] & MASK16_4) << 6; + l2 |= tmp[tmpIdx + 4] << 0; + ints[intsIdx + 2] = l2; + } + } + + static void decode11(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(44, ints, 5, 11, MASK16_11, tmp, 0, MASK16_5); + for (int iter = 0, tmpIdx = 0, intsIdx = 44; iter < 4; ++iter, tmpIdx += 11, intsIdx += 5) { + int l0 = tmp[tmpIdx + 0] << 6; + l0 |= tmp[tmpIdx + 1] << 1; + l0 |= (tmp[tmpIdx + 2] >>> 4) & MASK16_1; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 2] & MASK16_4) << 7; + l1 |= tmp[tmpIdx + 3] << 2; + l1 |= (tmp[tmpIdx + 4] >>> 3) & MASK16_2; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 4] & MASK16_3) << 8; + l2 |= tmp[tmpIdx + 5] << 3; + l2 |= (tmp[tmpIdx + 6] >>> 2) & MASK16_3; + ints[intsIdx + 2] = l2; + int l3 = (tmp[tmpIdx + 6] & MASK16_2) << 9; + l3 |= tmp[tmpIdx + 7] << 4; + l3 |= (tmp[tmpIdx + 8] >>> 1) & MASK16_4; + ints[intsIdx + 3] = l3; + int l4 = (tmp[tmpIdx + 8] & MASK16_1) << 10; + l4 |= tmp[tmpIdx + 9] << 5; + l4 |= tmp[tmpIdx + 10] << 0; + ints[intsIdx + 4] = l4; + } + } + + static void decode12(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(48, ints, 4, 12, MASK16_12, tmp, 0, MASK16_4); + for (int iter = 0, tmpIdx = 0, intsIdx = 48; iter < 16; ++iter, tmpIdx += 3, intsIdx += 1) { + int l0 = tmp[tmpIdx + 0] << 8; + l0 |= tmp[tmpIdx + 1] << 4; + l0 |= tmp[tmpIdx + 2] << 0; + ints[intsIdx + 0] = l0; + } + } + + static void decode13(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(52, ints, 3, 13, MASK16_13, tmp, 0, MASK16_3); + for (int iter = 0, tmpIdx = 0, intsIdx = 52; iter < 4; ++iter, tmpIdx += 13, intsIdx += 3) { + int l0 = tmp[tmpIdx + 0] << 10; + l0 |= tmp[tmpIdx + 1] << 7; + l0 |= tmp[tmpIdx + 2] << 4; + l0 |= tmp[tmpIdx + 3] << 1; + l0 |= (tmp[tmpIdx + 4] >>> 2) & MASK16_1; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 4] & MASK16_2) << 11; + l1 |= tmp[tmpIdx + 5] << 8; + l1 |= tmp[tmpIdx + 6] << 5; + l1 |= tmp[tmpIdx + 7] << 2; + l1 |= (tmp[tmpIdx + 8] >>> 1) & MASK16_2; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 8] & MASK16_1) << 12; + l2 |= tmp[tmpIdx + 9] << 9; + l2 |= tmp[tmpIdx + 10] << 6; + l2 |= tmp[tmpIdx + 11] << 3; + l2 |= tmp[tmpIdx + 12] << 0; + ints[intsIdx + 2] = l2; + } + } + + static void decode14(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(56, ints, 2, 14, MASK16_14, tmp, 0, MASK16_2); + for (int iter = 0, tmpIdx = 0, intsIdx = 56; iter < 8; ++iter, tmpIdx += 7, intsIdx += 1) { + int l0 = tmp[tmpIdx + 0] << 12; + l0 |= tmp[tmpIdx + 1] << 10; + l0 |= tmp[tmpIdx + 2] << 8; + l0 |= tmp[tmpIdx + 3] << 6; + l0 |= tmp[tmpIdx + 4] << 4; + l0 |= tmp[tmpIdx + 5] << 2; + l0 |= tmp[tmpIdx + 6] << 0; + ints[intsIdx + 0] = l0; + } + } + + static void decode15(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(60, ints, 1, 15, MASK16_15, tmp, 0, MASK16_1); + for (int iter = 0, tmpIdx = 0, intsIdx = 60; iter < 4; ++iter, tmpIdx += 15, intsIdx += 1) { + int l0 = tmp[tmpIdx + 0] << 14; + l0 |= tmp[tmpIdx + 1] << 13; + l0 |= tmp[tmpIdx + 2] << 12; + l0 |= tmp[tmpIdx + 3] << 11; + l0 |= tmp[tmpIdx + 4] << 10; + l0 |= tmp[tmpIdx + 5] << 9; + l0 |= tmp[tmpIdx + 6] << 8; + l0 |= tmp[tmpIdx + 7] << 7; + l0 |= tmp[tmpIdx + 8] << 6; + l0 |= tmp[tmpIdx + 9] << 5; + l0 |= tmp[tmpIdx + 10] << 4; + l0 |= tmp[tmpIdx + 11] << 3; + l0 |= tmp[tmpIdx + 12] << 2; + l0 |= tmp[tmpIdx + 13] << 1; + l0 |= tmp[tmpIdx + 14] << 0; + ints[intsIdx + 0] = l0; + } + } + + static void decode16(PostingDecodingUtil pdu, int[] ints) throws IOException { + pdu.in.readInts(ints, 0, 64); + } + + static void decode17(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(68, ints, 15, 17, MASK32_17, tmp, 0, MASK32_15); + for (int iter = 0, tmpIdx = 0, intsIdx = 68; iter < 4; ++iter, tmpIdx += 17, intsIdx += 15) { + int l0 = tmp[tmpIdx + 0] << 2; + l0 |= (tmp[tmpIdx + 1] >>> 13) & MASK32_2; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 1] & MASK32_13) << 4; + l1 |= (tmp[tmpIdx + 2] >>> 11) & MASK32_4; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 2] & MASK32_11) << 6; + l2 |= (tmp[tmpIdx + 3] >>> 9) & MASK32_6; + ints[intsIdx + 2] = l2; + int l3 = (tmp[tmpIdx + 3] & MASK32_9) << 8; + l3 |= (tmp[tmpIdx + 4] >>> 7) & MASK32_8; + ints[intsIdx + 3] = l3; + int l4 = (tmp[tmpIdx + 4] & MASK32_7) << 10; + l4 |= (tmp[tmpIdx + 5] >>> 5) & MASK32_10; + ints[intsIdx + 4] = l4; + int l5 = (tmp[tmpIdx + 5] & MASK32_5) << 12; + l5 |= (tmp[tmpIdx + 6] >>> 3) & MASK32_12; + ints[intsIdx + 5] = l5; + int l6 = (tmp[tmpIdx + 6] & MASK32_3) << 14; + l6 |= (tmp[tmpIdx + 7] >>> 1) & MASK32_14; + ints[intsIdx + 6] = l6; + int l7 = (tmp[tmpIdx + 7] & MASK32_1) << 16; + l7 |= tmp[tmpIdx + 8] << 1; + l7 |= (tmp[tmpIdx + 9] >>> 14) & MASK32_1; + ints[intsIdx + 7] = l7; + int l8 = (tmp[tmpIdx + 9] & MASK32_14) << 3; + l8 |= (tmp[tmpIdx + 10] >>> 12) & MASK32_3; + ints[intsIdx + 8] = l8; + int l9 = (tmp[tmpIdx + 10] & MASK32_12) << 5; + l9 |= (tmp[tmpIdx + 11] >>> 10) & MASK32_5; + ints[intsIdx + 9] = l9; + int l10 = (tmp[tmpIdx + 11] & MASK32_10) << 7; + l10 |= (tmp[tmpIdx + 12] >>> 8) & MASK32_7; + ints[intsIdx + 10] = l10; + int l11 = (tmp[tmpIdx + 12] & MASK32_8) << 9; + l11 |= (tmp[tmpIdx + 13] >>> 6) & MASK32_9; + ints[intsIdx + 11] = l11; + int l12 = (tmp[tmpIdx + 13] & MASK32_6) << 11; + l12 |= (tmp[tmpIdx + 14] >>> 4) & MASK32_11; + ints[intsIdx + 12] = l12; + int l13 = (tmp[tmpIdx + 14] & MASK32_4) << 13; + l13 |= (tmp[tmpIdx + 15] >>> 2) & MASK32_13; + ints[intsIdx + 13] = l13; + int l14 = (tmp[tmpIdx + 15] & MASK32_2) << 15; + l14 |= tmp[tmpIdx + 16] << 0; + ints[intsIdx + 14] = l14; + } + } + + static void decode18(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(72, ints, 14, 18, MASK32_18, tmp, 0, MASK32_14); + for (int iter = 0, tmpIdx = 0, intsIdx = 72; iter < 8; ++iter, tmpIdx += 9, intsIdx += 7) { + int l0 = tmp[tmpIdx + 0] << 4; + l0 |= (tmp[tmpIdx + 1] >>> 10) & MASK32_4; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 1] & MASK32_10) << 8; + l1 |= (tmp[tmpIdx + 2] >>> 6) & MASK32_8; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 2] & MASK32_6) << 12; + l2 |= (tmp[tmpIdx + 3] >>> 2) & MASK32_12; + ints[intsIdx + 2] = l2; + int l3 = (tmp[tmpIdx + 3] & MASK32_2) << 16; + l3 |= tmp[tmpIdx + 4] << 2; + l3 |= (tmp[tmpIdx + 5] >>> 12) & MASK32_2; + ints[intsIdx + 3] = l3; + int l4 = (tmp[tmpIdx + 5] & MASK32_12) << 6; + l4 |= (tmp[tmpIdx + 6] >>> 8) & MASK32_6; + ints[intsIdx + 4] = l4; + int l5 = (tmp[tmpIdx + 6] & MASK32_8) << 10; + l5 |= (tmp[tmpIdx + 7] >>> 4) & MASK32_10; + ints[intsIdx + 5] = l5; + int l6 = (tmp[tmpIdx + 7] & MASK32_4) << 14; + l6 |= tmp[tmpIdx + 8] << 0; + ints[intsIdx + 6] = l6; + } + } + + static void decode19(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(76, ints, 13, 19, MASK32_19, tmp, 0, MASK32_13); + for (int iter = 0, tmpIdx = 0, intsIdx = 76; iter < 4; ++iter, tmpIdx += 19, intsIdx += 13) { + int l0 = tmp[tmpIdx + 0] << 6; + l0 |= (tmp[tmpIdx + 1] >>> 7) & MASK32_6; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 1] & MASK32_7) << 12; + l1 |= (tmp[tmpIdx + 2] >>> 1) & MASK32_12; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 2] & MASK32_1) << 18; + l2 |= tmp[tmpIdx + 3] << 5; + l2 |= (tmp[tmpIdx + 4] >>> 8) & MASK32_5; + ints[intsIdx + 2] = l2; + int l3 = (tmp[tmpIdx + 4] & MASK32_8) << 11; + l3 |= (tmp[tmpIdx + 5] >>> 2) & MASK32_11; + ints[intsIdx + 3] = l3; + int l4 = (tmp[tmpIdx + 5] & MASK32_2) << 17; + l4 |= tmp[tmpIdx + 6] << 4; + l4 |= (tmp[tmpIdx + 7] >>> 9) & MASK32_4; + ints[intsIdx + 4] = l4; + int l5 = (tmp[tmpIdx + 7] & MASK32_9) << 10; + l5 |= (tmp[tmpIdx + 8] >>> 3) & MASK32_10; + ints[intsIdx + 5] = l5; + int l6 = (tmp[tmpIdx + 8] & MASK32_3) << 16; + l6 |= tmp[tmpIdx + 9] << 3; + l6 |= (tmp[tmpIdx + 10] >>> 10) & MASK32_3; + ints[intsIdx + 6] = l6; + int l7 = (tmp[tmpIdx + 10] & MASK32_10) << 9; + l7 |= (tmp[tmpIdx + 11] >>> 4) & MASK32_9; + ints[intsIdx + 7] = l7; + int l8 = (tmp[tmpIdx + 11] & MASK32_4) << 15; + l8 |= tmp[tmpIdx + 12] << 2; + l8 |= (tmp[tmpIdx + 13] >>> 11) & MASK32_2; + ints[intsIdx + 8] = l8; + int l9 = (tmp[tmpIdx + 13] & MASK32_11) << 8; + l9 |= (tmp[tmpIdx + 14] >>> 5) & MASK32_8; + ints[intsIdx + 9] = l9; + int l10 = (tmp[tmpIdx + 14] & MASK32_5) << 14; + l10 |= tmp[tmpIdx + 15] << 1; + l10 |= (tmp[tmpIdx + 16] >>> 12) & MASK32_1; + ints[intsIdx + 10] = l10; + int l11 = (tmp[tmpIdx + 16] & MASK32_12) << 7; + l11 |= (tmp[tmpIdx + 17] >>> 6) & MASK32_7; + ints[intsIdx + 11] = l11; + int l12 = (tmp[tmpIdx + 17] & MASK32_6) << 13; + l12 |= tmp[tmpIdx + 18] << 0; + ints[intsIdx + 12] = l12; + } + } + + static void decode20(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(80, ints, 12, 20, MASK32_20, tmp, 0, MASK32_12); + for (int iter = 0, tmpIdx = 0, intsIdx = 80; iter < 16; ++iter, tmpIdx += 5, intsIdx += 3) { + int l0 = tmp[tmpIdx + 0] << 8; + l0 |= (tmp[tmpIdx + 1] >>> 4) & MASK32_8; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 1] & MASK32_4) << 16; + l1 |= tmp[tmpIdx + 2] << 4; + l1 |= (tmp[tmpIdx + 3] >>> 8) & MASK32_4; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 3] & MASK32_8) << 12; + l2 |= tmp[tmpIdx + 4] << 0; + ints[intsIdx + 2] = l2; + } + } + + static void decode21(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(84, ints, 11, 21, MASK32_21, tmp, 0, MASK32_11); + for (int iter = 0, tmpIdx = 0, intsIdx = 84; iter < 4; ++iter, tmpIdx += 21, intsIdx += 11) { + int l0 = tmp[tmpIdx + 0] << 10; + l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK32_10; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 1] & MASK32_1) << 20; + l1 |= tmp[tmpIdx + 2] << 9; + l1 |= (tmp[tmpIdx + 3] >>> 2) & MASK32_9; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 3] & MASK32_2) << 19; + l2 |= tmp[tmpIdx + 4] << 8; + l2 |= (tmp[tmpIdx + 5] >>> 3) & MASK32_8; + ints[intsIdx + 2] = l2; + int l3 = (tmp[tmpIdx + 5] & MASK32_3) << 18; + l3 |= tmp[tmpIdx + 6] << 7; + l3 |= (tmp[tmpIdx + 7] >>> 4) & MASK32_7; + ints[intsIdx + 3] = l3; + int l4 = (tmp[tmpIdx + 7] & MASK32_4) << 17; + l4 |= tmp[tmpIdx + 8] << 6; + l4 |= (tmp[tmpIdx + 9] >>> 5) & MASK32_6; + ints[intsIdx + 4] = l4; + int l5 = (tmp[tmpIdx + 9] & MASK32_5) << 16; + l5 |= tmp[tmpIdx + 10] << 5; + l5 |= (tmp[tmpIdx + 11] >>> 6) & MASK32_5; + ints[intsIdx + 5] = l5; + int l6 = (tmp[tmpIdx + 11] & MASK32_6) << 15; + l6 |= tmp[tmpIdx + 12] << 4; + l6 |= (tmp[tmpIdx + 13] >>> 7) & MASK32_4; + ints[intsIdx + 6] = l6; + int l7 = (tmp[tmpIdx + 13] & MASK32_7) << 14; + l7 |= tmp[tmpIdx + 14] << 3; + l7 |= (tmp[tmpIdx + 15] >>> 8) & MASK32_3; + ints[intsIdx + 7] = l7; + int l8 = (tmp[tmpIdx + 15] & MASK32_8) << 13; + l8 |= tmp[tmpIdx + 16] << 2; + l8 |= (tmp[tmpIdx + 17] >>> 9) & MASK32_2; + ints[intsIdx + 8] = l8; + int l9 = (tmp[tmpIdx + 17] & MASK32_9) << 12; + l9 |= tmp[tmpIdx + 18] << 1; + l9 |= (tmp[tmpIdx + 19] >>> 10) & MASK32_1; + ints[intsIdx + 9] = l9; + int l10 = (tmp[tmpIdx + 19] & MASK32_10) << 11; + l10 |= tmp[tmpIdx + 20] << 0; + ints[intsIdx + 10] = l10; + } + } + + static void decode22(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(88, ints, 10, 22, MASK32_22, tmp, 0, MASK32_10); + for (int iter = 0, tmpIdx = 0, intsIdx = 88; iter < 8; ++iter, tmpIdx += 11, intsIdx += 5) { + int l0 = tmp[tmpIdx + 0] << 12; + l0 |= tmp[tmpIdx + 1] << 2; + l0 |= (tmp[tmpIdx + 2] >>> 8) & MASK32_2; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 2] & MASK32_8) << 14; + l1 |= tmp[tmpIdx + 3] << 4; + l1 |= (tmp[tmpIdx + 4] >>> 6) & MASK32_4; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 4] & MASK32_6) << 16; + l2 |= tmp[tmpIdx + 5] << 6; + l2 |= (tmp[tmpIdx + 6] >>> 4) & MASK32_6; + ints[intsIdx + 2] = l2; + int l3 = (tmp[tmpIdx + 6] & MASK32_4) << 18; + l3 |= tmp[tmpIdx + 7] << 8; + l3 |= (tmp[tmpIdx + 8] >>> 2) & MASK32_8; + ints[intsIdx + 3] = l3; + int l4 = (tmp[tmpIdx + 8] & MASK32_2) << 20; + l4 |= tmp[tmpIdx + 9] << 10; + l4 |= tmp[tmpIdx + 10] << 0; + ints[intsIdx + 4] = l4; + } + } + + static void decode23(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(92, ints, 9, 23, MASK32_23, tmp, 0, MASK32_9); + for (int iter = 0, tmpIdx = 0, intsIdx = 92; iter < 4; ++iter, tmpIdx += 23, intsIdx += 9) { + int l0 = tmp[tmpIdx + 0] << 14; + l0 |= tmp[tmpIdx + 1] << 5; + l0 |= (tmp[tmpIdx + 2] >>> 4) & MASK32_5; + ints[intsIdx + 0] = l0; + int l1 = (tmp[tmpIdx + 2] & MASK32_4) << 19; + l1 |= tmp[tmpIdx + 3] << 10; + l1 |= tmp[tmpIdx + 4] << 1; + l1 |= (tmp[tmpIdx + 5] >>> 8) & MASK32_1; + ints[intsIdx + 1] = l1; + int l2 = (tmp[tmpIdx + 5] & MASK32_8) << 15; + l2 |= tmp[tmpIdx + 6] << 6; + l2 |= (tmp[tmpIdx + 7] >>> 3) & MASK32_6; + ints[intsIdx + 2] = l2; + int l3 = (tmp[tmpIdx + 7] & MASK32_3) << 20; + l3 |= tmp[tmpIdx + 8] << 11; + l3 |= tmp[tmpIdx + 9] << 2; + l3 |= (tmp[tmpIdx + 10] >>> 7) & MASK32_2; + ints[intsIdx + 3] = l3; + int l4 = (tmp[tmpIdx + 10] & MASK32_7) << 16; + l4 |= tmp[tmpIdx + 11] << 7; + l4 |= (tmp[tmpIdx + 12] >>> 2) & MASK32_7; + ints[intsIdx + 4] = l4; + int l5 = (tmp[tmpIdx + 12] & MASK32_2) << 21; + l5 |= tmp[tmpIdx + 13] << 12; + l5 |= tmp[tmpIdx + 14] << 3; + l5 |= (tmp[tmpIdx + 15] >>> 6) & MASK32_3; + ints[intsIdx + 5] = l5; + int l6 = (tmp[tmpIdx + 15] & MASK32_6) << 17; + l6 |= tmp[tmpIdx + 16] << 8; + l6 |= (tmp[tmpIdx + 17] >>> 1) & MASK32_8; + ints[intsIdx + 6] = l6; + int l7 = (tmp[tmpIdx + 17] & MASK32_1) << 22; + l7 |= tmp[tmpIdx + 18] << 13; + l7 |= tmp[tmpIdx + 19] << 4; + l7 |= (tmp[tmpIdx + 20] >>> 5) & MASK32_4; + ints[intsIdx + 7] = l7; + int l8 = (tmp[tmpIdx + 20] & MASK32_5) << 18; + l8 |= tmp[tmpIdx + 21] << 9; + l8 |= tmp[tmpIdx + 22] << 0; + ints[intsIdx + 8] = l8; + } + } + + static void decode24(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException { + pdu.splitInts(96, ints, 8, 24, MASK32_24, tmp, 0, MASK32_8); + for (int iter = 0, tmpIdx = 0, intsIdx = 96; iter < 32; ++iter, tmpIdx += 3, intsIdx += 1) { + int l0 = tmp[tmpIdx + 0] << 16; + l0 |= tmp[tmpIdx + 1] << 8; + l0 |= tmp[tmpIdx + 2] << 0; + ints[intsIdx + 0] = l0; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101Codec.java new file mode 100644 index 00000000000..2b764b87685 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101Codec.java @@ -0,0 +1,217 @@ +/* + * 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.lucene101; + +import java.util.Objects; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.CompoundFormat; +import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.FieldInfosFormat; +import org.apache.lucene.codecs.FilterCodec; +import org.apache.lucene.codecs.KnnVectorsFormat; +import org.apache.lucene.codecs.LiveDocsFormat; +import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.codecs.PointsFormat; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.SegmentInfoFormat; +import org.apache.lucene.codecs.StoredFieldsFormat; +import org.apache.lucene.codecs.TermVectorsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat; +import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; +import org.apache.lucene.codecs.lucene90.Lucene90LiveDocsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90NormsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90PointsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat; +import org.apache.lucene.codecs.lucene90.Lucene90TermVectorsFormat; +import org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat; +import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; +import org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat; +import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; + +/** + * Implements the Lucene 10.1 index format + * + *

    If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}. + * + * @see org.apache.lucene.codecs.lucene101 package documentation for file format details. + * @lucene.experimental + */ +public class Lucene101Codec extends Codec { + + /** Configuration option for the codec. */ + public enum Mode { + /** Trade compression ratio for retrieval speed. */ + BEST_SPEED(Lucene90StoredFieldsFormat.Mode.BEST_SPEED), + /** Trade retrieval speed for compression ratio. */ + BEST_COMPRESSION(Lucene90StoredFieldsFormat.Mode.BEST_COMPRESSION); + + private final Lucene90StoredFieldsFormat.Mode storedMode; + + private Mode(Lucene90StoredFieldsFormat.Mode storedMode) { + this.storedMode = Objects.requireNonNull(storedMode); + } + } + + private final TermVectorsFormat vectorsFormat = new Lucene90TermVectorsFormat(); + private final FieldInfosFormat fieldInfosFormat = new Lucene94FieldInfosFormat(); + private final SegmentInfoFormat segmentInfosFormat = new Lucene99SegmentInfoFormat(); + private final LiveDocsFormat liveDocsFormat = new Lucene90LiveDocsFormat(); + private final CompoundFormat compoundFormat = new Lucene90CompoundFormat(); + private final NormsFormat normsFormat = new Lucene90NormsFormat(); + + private final PostingsFormat defaultPostingsFormat; + private final PostingsFormat postingsFormat = + new PerFieldPostingsFormat() { + @Override + public PostingsFormat getPostingsFormatForField(String field) { + return Lucene101Codec.this.getPostingsFormatForField(field); + } + }; + + private final DocValuesFormat defaultDVFormat; + private final DocValuesFormat docValuesFormat = + new PerFieldDocValuesFormat() { + @Override + public DocValuesFormat getDocValuesFormatForField(String field) { + return Lucene101Codec.this.getDocValuesFormatForField(field); + } + }; + + private final KnnVectorsFormat defaultKnnVectorsFormat; + private final KnnVectorsFormat knnVectorsFormat = + new PerFieldKnnVectorsFormat() { + @Override + public KnnVectorsFormat getKnnVectorsFormatForField(String field) { + return Lucene101Codec.this.getKnnVectorsFormatForField(field); + } + }; + + private final StoredFieldsFormat storedFieldsFormat; + + /** Instantiates a new codec. */ + public Lucene101Codec() { + this(Mode.BEST_SPEED); + } + + /** + * Instantiates a new codec, specifying the stored fields compression mode to use. + * + * @param mode stored fields compression mode to use for newly flushed/merged segments. + */ + public Lucene101Codec(Mode mode) { + super("Lucene101"); + this.storedFieldsFormat = + new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode); + this.defaultPostingsFormat = new Lucene101PostingsFormat(); + this.defaultDVFormat = new Lucene90DocValuesFormat(); + this.defaultKnnVectorsFormat = new Lucene99HnswVectorsFormat(); + } + + @Override + public final StoredFieldsFormat storedFieldsFormat() { + return storedFieldsFormat; + } + + @Override + public final TermVectorsFormat termVectorsFormat() { + return vectorsFormat; + } + + @Override + public final PostingsFormat postingsFormat() { + return postingsFormat; + } + + @Override + public final FieldInfosFormat fieldInfosFormat() { + return fieldInfosFormat; + } + + @Override + public final SegmentInfoFormat segmentInfoFormat() { + return segmentInfosFormat; + } + + @Override + public final LiveDocsFormat liveDocsFormat() { + return liveDocsFormat; + } + + @Override + public final CompoundFormat compoundFormat() { + return compoundFormat; + } + + @Override + public final PointsFormat pointsFormat() { + return new Lucene90PointsFormat(); + } + + @Override + public final KnnVectorsFormat knnVectorsFormat() { + return knnVectorsFormat; + } + + /** + * Returns the postings format that should be used for writing new segments of field. + * + *

    The default implementation always returns "Lucene101". + * + *

    WARNING: if you subclass, you are responsible for index backwards compatibility: + * future version of Lucene are only guaranteed to be able to read the default implementation, + */ + public PostingsFormat getPostingsFormatForField(String field) { + return defaultPostingsFormat; + } + + /** + * Returns the docvalues format that should be used for writing new segments of field + * . + * + *

    The default implementation always returns "Lucene90". + * + *

    WARNING: if you subclass, you are responsible for index backwards compatibility: + * future version of Lucene are only guaranteed to be able to read the default implementation. + */ + public DocValuesFormat getDocValuesFormatForField(String field) { + return defaultDVFormat; + } + + /** + * Returns the vectors format that should be used for writing new segments of field + * + *

    The default implementation always returns "Lucene99HnswVectorsFormat". + * + *

    WARNING: if you subclass, you are responsible for index backwards compatibility: + * future version of Lucene are only guaranteed to be able to read the default implementation. + */ + public KnnVectorsFormat getKnnVectorsFormatForField(String field) { + return defaultKnnVectorsFormat; + } + + @Override + public final DocValuesFormat docValuesFormat() { + return docValuesFormat; + } + + @Override + public final NormsFormat normsFormat() { + return normsFormat; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsFormat.java new file mode 100644 index 00000000000..e228f1090ab --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsFormat.java @@ -0,0 +1,492 @@ +/* + * 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.lucene101; + +import java.io.IOException; +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.FieldsConsumer; +import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.codecs.PostingsFormat; +import org.apache.lucene.codecs.PostingsReaderBase; +import org.apache.lucene.codecs.PostingsWriterBase; +import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsReader; +import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.TermState; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.packed.PackedInts; + +/** + * Lucene 10.1 postings format, which encodes postings in packed integer blocks for fast decode. + * + *

    Basic idea: + * + *

      + *
    • Packed Blocks and VInt Blocks: + *

      In packed blocks, integers are encoded with the same bit width ({@link PackedInts packed + * format}): the block size (i.e. number of integers inside block) is fixed (currently 128). + * Additionally blocks that are all the same value are encoded in an optimized way. + *

      In VInt blocks, integers are encoded as {@link DataOutput#writeVInt VInt}: the block + * size is variable. + *

    • Block structure: + *

      When the postings are long enough, Lucene101PostingsFormat will try to encode most + * integer data as a packed block. + *

      Take a term with 259 documents as an example, the first 256 document ids are encoded as + * two packed blocks, while the remaining 3 are encoded as one VInt block. + *

      Different kinds of data are always encoded separately into different packed blocks, but + * may possibly be interleaved into the same VInt block. + *

      This strategy is applied to pairs: <document number, frequency>, <position, + * payload length>, <position, offset start, offset length>, and <position, + * payload length, offsetstart, offset length>. + *

    • Skipdata: + *

      Skipdata is interleaved with blocks on 2 levels. Level 0 skip data is interleaved + * between every packed block. Level 1 skip data is interleaved between every 32 packed + * blocks. + *

    • Positions, Payloads, and Offsets: + *

      A position is an integer indicating where the term occurs within one document. A payload + * is a blob of metadata associated with current position. An offset is a pair of integers + * indicating the tokenized start/end offsets for given term in current position: it is + * essentially a specialized payload. + *

      When payloads and offsets are not omitted, numPositions==numPayloads==numOffsets + * (assuming a null payload contributes one count). As mentioned in block structure, it is + * possible to encode these three either combined or separately. + *

      In all cases, payloads and offsets are stored together. When encoded as a packed block, + * position data is separated out as .pos, while payloads and offsets are encoded in .pay + * (payload metadata will also be stored directly in .pay). When encoded as VInt blocks, all + * these three are stored interleaved into the .pos (so is payload metadata). + *

      With this strategy, the majority of payload and offset data will be outside .pos file. + * So for queries that require only position data, running on a full index with payloads and + * offsets, this reduces disk pre-fetches. + *

    + * + *

    Files and detailed format: + * + *

    + * + * + * + *
    + *
    Term Dictionary + *

    The .tim file contains the list of terms in each field along with per-term statistics + * (such as docfreq) and pointers to the frequencies, positions, payload and skip data in the + * .doc, .pos, and .pay files. See {@link Lucene90BlockTreeTermsWriter} for more details on + * the format. + *

    NOTE: The term dictionary can plug into different postings implementations: the postings + * writer/reader are actually responsible for encoding and decoding the PostingsHeader and + * TermMetadata sections described here: + *

      + *
    • PostingsHeader --> Header, PackedBlockSize + *
    • TermMetadata --> (DocFPDelta|SingletonDocID), PosFPDelta?, PosVIntBlockFPDelta?, + * PayFPDelta? + *
    • Header, --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • PackedBlockSize, SingletonDocID --> {@link DataOutput#writeVInt VInt} + *
    • DocFPDelta, PosFPDelta, PayFPDelta, PosVIntBlockFPDelta --> {@link + * DataOutput#writeVLong VLong} + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    Notes: + *

      + *
    • Header is a {@link CodecUtil#writeIndexHeader IndexHeader} storing the version + * information for the postings. + *
    • PackedBlockSize is the fixed block size for packed blocks. In packed block, bit width + * is determined by the largest integer. Smaller block size result in smaller variance + * among width of integers hence smaller indexes. Larger block size result in more + * efficient bulk i/o hence better acceleration. This value should always be a multiple + * of 64, currently fixed as 128 as a tradeoff. It is also the skip interval used to + * accelerate {@link org.apache.lucene.index.PostingsEnum#advance(int)}. + *
    • DocFPDelta determines the position of this term's TermFreqs within the .doc file. In + * particular, it is the difference of file offset between this term's data and previous + * term's data (or zero, for the first term in the block).On disk it is stored as the + * difference from previous value in sequence. + *
    • PosFPDelta determines the position of this term's TermPositions within the .pos file. + * While PayFPDelta determines the position of this term's <TermPayloads, + * TermOffsets?> within the .pay file. Similar to DocFPDelta, it is the difference + * between two file positions (or neglected, for fields that omit payloads and offsets). + *
    • PosVIntBlockFPDelta determines the position of this term's last TermPosition in last + * pos packed block within the .pos file. It is synonym for PayVIntBlockFPDelta or + * OffsetVIntBlockFPDelta. This is actually used to indicate whether it is necessary to + * load following payloads and offsets from .pos instead of .pay. Every time a new block + * of positions are to be loaded, the PostingsReader will use this value to check + * whether current block is packed format or VInt. When packed format, payloads and + * offsets are fetched from .pay, otherwise from .pos. (this value is neglected when + * total number of positions i.e. totalTermFreq is less or equal to PackedBlockSize). + *
    • SingletonDocID is an optimization when a term only appears in one document. In this + * case, instead of writing a file pointer to the .doc file (DocFPDelta), and then a + * VIntBlock at that location, the single document ID is written to the term dictionary. + *
    + *
    + * + * + * + *
    + *
    Term Index + *

    The .tip file contains an index into the term dictionary, so that it can be accessed + * randomly. See {@link Lucene90BlockTreeTermsWriter} for more details on the format. + *

    + * + * + * + *
    + *
    Frequencies and Skip Data + *

    The .doc file contains the lists of documents which contain each term, along with the + * frequency of the term in that document (except when frequencies are omitted: {@link + * IndexOptions#DOCS}). Skip data is saved at the end of each term's postings. The skip data + * is saved once for the entire postings list. + *

      + *
    • docFile(.doc) --> Header, <TermFreqs>TermCount, Footer + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • TermFreqs --> <PackedBlock32> PackedDocBlockNum/32, VIntBlock? + *
    • PackedBlock32 --> Level1SkipData, <PackedBlock> 32 + *
    • PackedBlock --> Level0SkipData, PackedDocDeltaBlock, PackedFreqBlock? + *
    • VIntBlock --> + * <DocDelta[,Freq?]>DocFreq-PackedBlockSize*PackedDocBlockNum + *
    • Level1SkipData --> DocDelta, DocFPDelta, Skip1NumBytes?, ImpactLength?, Impacts?, + * PosFPDelta?, NextPosUpto?, PayFPDelta?, NextPayByteUpto? + *
    • Level0SkipData --> Skip0NumBytes, DocDelta, DocFPDelta, PackedBlockLength, + * ImpactLength?, Impacts?, PosFPDelta?, NextPosUpto?, PayFPDelta?, NextPayByteUpto? + *
    • PackedFreqBlock --> {@link PackedInts PackedInts}, uses patching + *
    • PackedDocDeltaBlock --> {@link PackedInts PackedInts}, does not use patching + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    Notes: + *

      + *
    • PackedDocDeltaBlock is theoretically generated from two steps: + *
        + *
      1. Calculate the difference between each document number and previous one, and get + * a d-gaps list (for the first document, use absolute value); + *
      2. For those d-gaps from first one to + * PackedDocBlockNum*PackedBlockSizeth, separately encode as packed + * blocks. + *
      + * If frequencies are not omitted, PackedFreqBlock will be generated without d-gap step. + *
    • VIntBlock stores remaining d-gaps (along with frequencies when possible) with a + * format that encodes DocDelta and Freq: + *

      DocDelta: if frequencies are indexed, this determines both the document number and + * the frequency. In particular, DocDelta/2 is the difference between this document + * number and the previous document number (or zero when this is the first document in a + * TermFreqs). When DocDelta is odd, the frequency is one. When DocDelta is even, the + * frequency is read as another VInt. If frequencies are omitted, DocDelta contains the + * gap (not multiplied by 2) between document numbers and no frequency information is + * stored. + *

      For example, the TermFreqs for a term which occurs once in document seven and + * three times in document eleven, with frequencies indexed, would be the following + * sequence of VInts: + *

      15, 8, 3 + *

      If frequencies were omitted ({@link IndexOptions#DOCS}) it would be this sequence + * of VInts instead: + *

      7,4 + *

    • PackedDocBlockNum is the number of packed blocks for current term's docids or + * frequencies. In particular, PackedDocBlockNum = floor(DocFreq/PackedBlockSize) + *
    • On skip data, DocDelta is the delta between the last doc of the previous block - or + * -1 if there is no previous block - and the last doc of this block. This helps know by + * how much the doc ID should be incremented in case the block gets skipped. + *
    • Skip0Length is the length of skip data at level 0. Encoding it is useful when skip + * data is never needed to quickly skip over skip data, e.g. if only using nextDoc(). It + * is also used when only the first fields of skip data are needed, in order to skip + * over remaining fields without reading them. + *
    • ImpactLength and Impacts are only stored if frequencies are indexed. + *
    • Since positions and payloads are also block encoded, the skip should skip to related + * block first, then fetch the values according to in-block offset. PosFPSkip and + * PayFPSkip record the file offsets of related block in .pos and .pay, respectively. + * While PosBlockOffset indicates which value to fetch inside the related block + * (PayBlockOffset is unnecessary since it is always equal to PosBlockOffset). Same as + * DocFPSkip, the file offsets are relative to the start of current term's TermFreqs, + * and stored as a difference sequence. + *
    • PayByteUpto indicates the start offset of the current payload. It is equivalent to + * the sum of the payload lengths in the current block up to PosBlockOffset + *
    • ImpactLength is the total length of CompetitiveFreqDelta and CompetitiveNormDelta + * pairs. CompetitiveFreqDelta and CompetitiveNormDelta are used to safely skip score + * calculation for uncompetitive documents; See {@link + * org.apache.lucene.codecs.CompetitiveImpactAccumulator} for more details. + *
    + *
    + * + * + * + *
    + *
    Positions + *

    The .pos file contains the lists of positions that each term occurs at within documents. + * It also sometimes stores part of payloads and offsets for speedup. + *

      + *
    • PosFile(.pos) --> Header, <TermPositions> TermCount, Footer + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • TermPositions --> <PackedPosDeltaBlock> PackedPosBlockNum, + * VIntBlock? + *
    • VIntBlock --> <PositionDelta[, PayloadLength?], PayloadData?, OffsetDelta?, + * OffsetLength?>PosVIntCount + *
    • PackedPosDeltaBlock --> {@link PackedInts PackedInts} + *
    • PositionDelta, OffsetDelta, OffsetLength --> {@link DataOutput#writeVInt VInt} + *
    • PayloadData --> {@link DataOutput#writeByte byte}PayLength + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    Notes: + *

      + *
    • TermPositions are order by term (terms are implicit, from the term dictionary), and + * position values for each term document pair are incremental, and ordered by document + * number. + *
    • PackedPosBlockNum is the number of packed blocks for current term's positions, + * payloads or offsets. In particular, PackedPosBlockNum = + * floor(totalTermFreq/PackedBlockSize) + *
    • PosVIntCount is the number of positions encoded as VInt format. In particular, + * PosVIntCount = totalTermFreq - PackedPosBlockNum*PackedBlockSize + *
    • The procedure how PackedPosDeltaBlock is generated is the same as PackedDocDeltaBlock + * in chapter Frequencies and Skip Data. + *
    • PositionDelta is, if payloads are disabled for the term's field, the difference + * between the position of the current occurrence in the document and the previous + * occurrence (or zero, if this is the first occurrence in this document). If payloads + * are enabled for the term's field, then PositionDelta/2 is the difference between the + * current and the previous position. If payloads are enabled and PositionDelta is odd, + * then PayloadLength is stored, indicating the length of the payload at the current + * term position. + *
    • For example, the TermPositions for a term which occurs as the fourth term in one + * document, and as the fifth and ninth term in a subsequent document, would be the + * following sequence of VInts (payloads disabled): + *

      4, 5, 4 + *

    • PayloadData is metadata associated with the current term position. If PayloadLength + * is stored at the current position, then it indicates the length of this payload. If + * PayloadLength is not stored, then this payload has the same length as the payload at + * the previous position. + *
    • OffsetDelta/2 is the difference between this position's startOffset from the previous + * occurrence (or zero, if this is the first occurrence in this document). If + * OffsetDelta is odd, then the length (endOffset-startOffset) differs from the previous + * occurrence and an OffsetLength follows. Offset data is only written for {@link + * IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}. + *
    + *
    + * + * + * + *
    + *
    Payloads and Offsets + *

    The .pay file will store payloads and offsets associated with certain term-document + * positions. Some payloads and offsets will be separated out into .pos file, for performance + * reasons. + *

      + *
    • PayFile(.pay): --> Header, <TermPayloads?, TermOffsets?> + * TermCount, Footer + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader} + *
    • TermPayloads --> <PackedPayLengthBlock, SumPayLength, PayData> + * PackedPayBlockNum + *
    • TermOffsets --> <PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock> + * PackedPayBlockNum + *
    • PackedPayLengthBlock, PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock --> + * {@link PackedInts PackedInts} + *
    • SumPayLength --> {@link DataOutput#writeVInt VInt} + *
    • PayData --> {@link DataOutput#writeByte byte}SumPayLength + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter} + *
    + *

    Notes: + *

      + *
    • The order of TermPayloads/TermOffsets will be the same as TermPositions, note that + * part of payload/offsets are stored in .pos. + *
    • The procedure how PackedPayLengthBlock and PackedOffsetLengthBlock are generated is + * the same as PackedFreqBlock in chapter Frequencies and Skip + * Data. While PackedStartDeltaBlock follows a same procedure as + * PackedDocDeltaBlock. + *
    • PackedPayBlockNum is always equal to PackedPosBlockNum, for the same term. It is also + * synonym for PackedOffsetBlockNum. + *
    • SumPayLength is the total length of payloads written within one block, should be the + * sum of PayLengths in one packed block. + *
    • PayLength in PackedPayLengthBlock is the length of each payload associated with the + * current position. + *
    + *
    + * + * @lucene.experimental + */ +public final class Lucene101PostingsFormat extends PostingsFormat { + + /** Filename extension for some small metadata about how postings are encoded. */ + public static final String META_EXTENSION = "psm"; + + /** + * Filename extension for document number, frequencies, and skip data. See chapter: Frequencies and Skip Data + */ + public static final String DOC_EXTENSION = "doc"; + + /** Filename extension for positions. See chapter: Positions */ + public static final String POS_EXTENSION = "pos"; + + /** + * Filename extension for payloads and offsets. See chapter: Payloads and + * Offsets + */ + public static final String PAY_EXTENSION = "pay"; + + /** Size of blocks. */ + public static final int BLOCK_SIZE = ForUtil.BLOCK_SIZE; + + public static final int BLOCK_MASK = BLOCK_SIZE - 1; + + /** We insert skip data on every block and every SKIP_FACTOR=32 blocks. */ + public static final int LEVEL1_FACTOR = 32; + + /** Total number of docs covered by level 1 skip data: 32 * 128 = 4,096 */ + public static final int LEVEL1_NUM_DOCS = LEVEL1_FACTOR * BLOCK_SIZE; + + public static final int LEVEL1_MASK = LEVEL1_NUM_DOCS - 1; + + static final String TERMS_CODEC = "Lucene90PostingsWriterTerms"; + static final String META_CODEC = "Lucene101PostingsWriterMeta"; + static final String DOC_CODEC = "Lucene101PostingsWriterDoc"; + static final String POS_CODEC = "Lucene101PostingsWriterPos"; + static final String PAY_CODEC = "Lucene101PostingsWriterPay"; + + static final int VERSION_START = 0; + static final int VERSION_CURRENT = VERSION_START; + + private final int minTermBlockSize; + private final int maxTermBlockSize; + + /** Creates {@code Lucene101PostingsFormat} with default settings. */ + public Lucene101PostingsFormat() { + this( + Lucene90BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, + Lucene90BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE); + } + + /** + * Creates {@code Lucene101PostingsFormat} with custom values for {@code minBlockSize} and {@code + * maxBlockSize} passed to block terms dictionary. + * + * @see + * Lucene90BlockTreeTermsWriter#Lucene90BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) + */ + public Lucene101PostingsFormat(int minTermBlockSize, int maxTermBlockSize) { + super("Lucene101"); + Lucene90BlockTreeTermsWriter.validateSettings(minTermBlockSize, maxTermBlockSize); + this.minTermBlockSize = minTermBlockSize; + this.maxTermBlockSize = maxTermBlockSize; + } + + @Override + public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { + PostingsWriterBase postingsWriter = new Lucene101PostingsWriter(state); + boolean success = false; + try { + FieldsConsumer ret = + new Lucene90BlockTreeTermsWriter( + state, postingsWriter, minTermBlockSize, maxTermBlockSize); + success = true; + return ret; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(postingsWriter); + } + } + } + + @Override + public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { + PostingsReaderBase postingsReader = new Lucene101PostingsReader(state); + boolean success = false; + try { + FieldsProducer ret = new Lucene90BlockTreeTermsReader(postingsReader, state); + success = true; + return ret; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(postingsReader); + } + } + } + + /** + * Holds all state required for {@link Lucene101PostingsReader} to produce a {@link + * org.apache.lucene.index.PostingsEnum} without re-seeking the terms dict. + * + * @lucene.internal + */ + public static final class IntBlockTermState extends BlockTermState { + /** file pointer to the start of the doc ids enumeration, in {@link #DOC_EXTENSION} file */ + public long docStartFP; + + /** file pointer to the start of the positions enumeration, in {@link #POS_EXTENSION} file */ + public long posStartFP; + + /** file pointer to the start of the payloads enumeration, in {@link #PAY_EXTENSION} file */ + public long payStartFP; + + /** + * file offset for the last position in the last block, if there are more than {@link + * ForUtil#BLOCK_SIZE} positions; otherwise -1 + * + *

    One might think to use total term frequency to track how many positions are left to read + * as we decode the blocks, and decode the last block differently when num_left_positions < + * BLOCK_SIZE. Unfortunately this won't work since the tracking will be messed up when we skip + * blocks as the skipper will only tell us new position offset (start of block) and number of + * positions to skip for that block, without telling us how many positions it has skipped. + */ + public long lastPosBlockOffset; + + /** + * docid when there is a single pulsed posting, otherwise -1. freq is always implicitly + * totalTermFreq in this case. + */ + public int singletonDocID; + + /** Sole constructor. */ + public IntBlockTermState() { + lastPosBlockOffset = -1; + singletonDocID = -1; + } + + @Override + public IntBlockTermState clone() { + IntBlockTermState other = new IntBlockTermState(); + other.copyFrom(this); + return other; + } + + @Override + public void copyFrom(TermState _other) { + super.copyFrom(_other); + IntBlockTermState other = (IntBlockTermState) _other; + docStartFP = other.docStartFP; + posStartFP = other.posStartFP; + payStartFP = other.payStartFP; + lastPosBlockOffset = other.lastPosBlockOffset; + singletonDocID = other.singletonDocID; + } + + @Override + public String toString() { + return super.toString() + + " docStartFP=" + + docStartFP + + " posStartFP=" + + posStartFP + + " payStartFP=" + + payStartFP + + " lastPosBlockOffset=" + + lastPosBlockOffset + + " singletonDocID=" + + singletonDocID; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsReader.java new file mode 100644 index 00000000000..4ae03dcdd9d --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsReader.java @@ -0,0 +1,1864 @@ +/* + * 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.lucene101; + +import static org.apache.lucene.codecs.lucene101.ForUtil.BLOCK_SIZE; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.DOC_CODEC; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.LEVEL1_NUM_DOCS; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.META_CODEC; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.PAY_CODEC; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.POS_CODEC; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.TERMS_CODEC; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.VERSION_CURRENT; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.VERSION_START; + +import java.io.IOException; +import java.util.AbstractList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.RandomAccess; +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.PostingsReaderBase; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.IntBlockTermState; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.Impact; +import org.apache.lucene.index.Impacts; +import org.apache.lucene.index.ImpactsEnum; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.PostingsEnum; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SlowImpactsEnum; +import org.apache.lucene.internal.vectorization.PostingDecodingUtil; +import org.apache.lucene.internal.vectorization.VectorUtilSupport; +import org.apache.lucene.internal.vectorization.VectorizationProvider; +import org.apache.lucene.store.ByteArrayDataInput; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.ReadAdvice; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BitUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; + +/** + * Concrete class that reads docId(maybe frq,pos,offset,payloads) list with postings format. + * + * @lucene.experimental + */ +public final class Lucene101PostingsReader extends PostingsReaderBase { + + static final VectorizationProvider VECTORIZATION_PROVIDER = VectorizationProvider.getInstance(); + private static final VectorUtilSupport VECTOR_SUPPORT = + VECTORIZATION_PROVIDER.getVectorUtilSupport(); + // Dummy impacts, composed of the maximum possible term frequency and the lowest possible + // (unsigned) norm value. This is typically used on tail blocks, which don't actually record + // impacts as the storage overhead would not be worth any query evaluation speedup, since there's + // less than 128 docs left to evaluate anyway. + private static final List DUMMY_IMPACTS = + Collections.singletonList(new Impact(Integer.MAX_VALUE, 1L)); + + private final IndexInput docIn; + private final IndexInput posIn; + private final IndexInput payIn; + + private final int maxNumImpactsAtLevel0; + private final int maxImpactNumBytesAtLevel0; + private final int maxNumImpactsAtLevel1; + private final int maxImpactNumBytesAtLevel1; + + /** Sole constructor. */ + public Lucene101PostingsReader(SegmentReadState state) throws IOException { + String metaName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene101PostingsFormat.META_EXTENSION); + final long expectedDocFileLength, expectedPosFileLength, expectedPayFileLength; + ChecksumIndexInput metaIn = null; + boolean success = false; + int version; + try { + metaIn = state.directory.openChecksumInput(metaName); + version = + CodecUtil.checkIndexHeader( + metaIn, + META_CODEC, + VERSION_START, + VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + maxNumImpactsAtLevel0 = metaIn.readInt(); + maxImpactNumBytesAtLevel0 = metaIn.readInt(); + maxNumImpactsAtLevel1 = metaIn.readInt(); + maxImpactNumBytesAtLevel1 = metaIn.readInt(); + expectedDocFileLength = metaIn.readLong(); + if (state.fieldInfos.hasProx()) { + expectedPosFileLength = metaIn.readLong(); + if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) { + expectedPayFileLength = metaIn.readLong(); + } else { + expectedPayFileLength = -1; + } + } else { + expectedPosFileLength = -1; + expectedPayFileLength = -1; + } + CodecUtil.checkFooter(metaIn, null); + success = true; + } catch (Throwable t) { + if (metaIn != null) { + CodecUtil.checkFooter(metaIn, t); + throw new AssertionError("unreachable"); + } else { + throw t; + } + } finally { + if (success) { + metaIn.close(); + } else { + IOUtils.closeWhileHandlingException(metaIn); + } + } + + success = false; + IndexInput docIn = null; + IndexInput posIn = null; + IndexInput payIn = null; + + // NOTE: these data files are too costly to verify checksum against all the bytes on open, + // but for now we at least verify proper structure of the checksum footer: which looks + // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption + // such as file truncation. + + String docName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene101PostingsFormat.DOC_EXTENSION); + try { + // Postings have a forward-only access pattern, so pass ReadAdvice.NORMAL to perform + // readahead. + docIn = state.directory.openInput(docName, state.context.withReadAdvice(ReadAdvice.NORMAL)); + CodecUtil.checkIndexHeader( + docIn, DOC_CODEC, version, version, state.segmentInfo.getId(), state.segmentSuffix); + CodecUtil.retrieveChecksum(docIn, expectedDocFileLength); + + if (state.fieldInfos.hasProx()) { + String proxName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene101PostingsFormat.POS_EXTENSION); + posIn = state.directory.openInput(proxName, state.context); + CodecUtil.checkIndexHeader( + posIn, POS_CODEC, version, version, state.segmentInfo.getId(), state.segmentSuffix); + CodecUtil.retrieveChecksum(posIn, expectedPosFileLength); + + if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) { + String payName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene101PostingsFormat.PAY_EXTENSION); + payIn = state.directory.openInput(payName, state.context); + CodecUtil.checkIndexHeader( + payIn, PAY_CODEC, version, version, state.segmentInfo.getId(), state.segmentSuffix); + CodecUtil.retrieveChecksum(payIn, expectedPayFileLength); + } + } + + this.docIn = docIn; + this.posIn = posIn; + this.payIn = payIn; + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(docIn, posIn, payIn); + } + } + } + + @Override + public void init(IndexInput termsIn, SegmentReadState state) throws IOException { + // Make sure we are talking to the matching postings writer + CodecUtil.checkIndexHeader( + termsIn, + TERMS_CODEC, + VERSION_START, + VERSION_CURRENT, + state.segmentInfo.getId(), + state.segmentSuffix); + final int indexBlockSize = termsIn.readVInt(); + if (indexBlockSize != BLOCK_SIZE) { + throw new IllegalStateException( + "index-time BLOCK_SIZE (" + + indexBlockSize + + ") != read-time BLOCK_SIZE (" + + BLOCK_SIZE + + ")"); + } + } + + static void prefixSum(int[] buffer, int count, long base) { + buffer[0] += base; + for (int i = 1; i < count; ++i) { + buffer[i] += buffer[i - 1]; + } + } + + @Override + public BlockTermState newTermState() { + return new IntBlockTermState(); + } + + @Override + public void close() throws IOException { + IOUtils.close(docIn, posIn, payIn); + } + + @Override + public void decodeTerm( + DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute) + throws IOException { + final IntBlockTermState termState = (IntBlockTermState) _termState; + if (absolute) { + termState.docStartFP = 0; + termState.posStartFP = 0; + termState.payStartFP = 0; + } + + final long l = in.readVLong(); + if ((l & 0x01) == 0) { + termState.docStartFP += l >>> 1; + if (termState.docFreq == 1) { + termState.singletonDocID = in.readVInt(); + } else { + termState.singletonDocID = -1; + } + } else { + assert absolute == false; + assert termState.singletonDocID != -1; + termState.singletonDocID += BitUtil.zigZagDecode(l >>> 1); + } + + if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) { + termState.posStartFP += in.readVLong(); + if (fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0 + || fieldInfo.hasPayloads()) { + termState.payStartFP += in.readVLong(); + } + if (termState.totalTermFreq > BLOCK_SIZE) { + termState.lastPosBlockOffset = in.readVLong(); + } else { + termState.lastPosBlockOffset = -1; + } + } + } + + @Override + public PostingsEnum postings( + FieldInfo fieldInfo, BlockTermState termState, PostingsEnum reuse, int flags) + throws IOException { + if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0 + || PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS) == false) { + return (reuse instanceof BlockDocsEnum blockDocsEnum + && blockDocsEnum.canReuse(docIn, fieldInfo) + ? blockDocsEnum + : new BlockDocsEnum(fieldInfo)) + .reset((IntBlockTermState) termState, flags); + } else { + return (reuse instanceof EverythingEnum everythingEnum + && everythingEnum.canReuse(docIn, fieldInfo) + ? everythingEnum + : new EverythingEnum(fieldInfo)) + .reset((IntBlockTermState) termState, flags); + } + } + + @Override + public ImpactsEnum impacts(FieldInfo fieldInfo, BlockTermState state, int flags) + throws IOException { + final IndexOptions options = fieldInfo.getIndexOptions(); + final boolean indexHasPositions = + options.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + + if (state.docFreq >= BLOCK_SIZE) { + if (options.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0 + && (indexHasPositions == false + || PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS) == false)) { + return new BlockImpactsDocsEnum(indexHasPositions, (IntBlockTermState) state); + } + + if (indexHasPositions + && (options.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0 + || PostingsEnum.featureRequested(flags, PostingsEnum.OFFSETS) == false) + && (fieldInfo.hasPayloads() == false + || PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS) == false)) { + return new BlockImpactsPostingsEnum(fieldInfo, (IntBlockTermState) state); + } + } + + return new SlowImpactsEnum(postings(fieldInfo, state, null, flags)); + } + + private static long sumOverRange(int[] arr, int start, int end) { + long res = 0L; + for (int i = start; i < end; i++) { + res += arr[i]; + } + return res; + } + + private abstract class AbstractPostingsEnum extends PostingsEnum { + + protected ForDeltaUtil forDeltaUtil; + protected PForUtil pforUtil; + + protected final int[] docBuffer = new int[BLOCK_SIZE + 1]; + protected final boolean indexHasFreq; + + protected int doc; // doc we last read + + // level 0 skip data + protected int level0LastDocID; + + // level 1 skip data + protected int level1LastDocID; + protected long level1DocEndFP; + protected int level1DocCountUpto; + + protected int docFreq; // number of docs in this posting list + protected long + totalTermFreq; // sum of freqBuffer in this posting list (or docFreq when omitted) + + protected int singletonDocID; // docid when there is a single pulsed posting, otherwise -1 + + protected int docCountUpto; // number of docs in or before the current block + protected int prevDocID; // last doc ID of the previous block + + protected int docBufferSize; + protected int docBufferUpto; + + protected IndexInput docIn; + protected PostingDecodingUtil docInUtil; + + protected AbstractPostingsEnum(FieldInfo fieldInfo) { + indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + } + + @Override + public int docID() { + return doc; + } + + protected void resetIndexInput(IntBlockTermState termState) throws IOException { + docFreq = termState.docFreq; + singletonDocID = termState.singletonDocID; + if (docFreq > 1) { + if (docIn == null) { + // lazy init + docIn = Lucene101PostingsReader.this.docIn.clone(); + docInUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(docIn); + } + prefetchPostings(docIn, termState); + } + } + + protected PostingsEnum resetIdsAndLevelParams(IntBlockTermState termState) throws IOException { + doc = -1; + prevDocID = -1; + docCountUpto = 0; + level0LastDocID = -1; + if (docFreq < LEVEL1_NUM_DOCS) { + level1LastDocID = NO_MORE_DOCS; + if (docFreq > 1) { + docIn.seek(termState.docStartFP); + } + } else { + level1LastDocID = -1; + level1DocEndFP = termState.docStartFP; + } + level1DocCountUpto = 0; + docBufferSize = BLOCK_SIZE; + docBufferUpto = BLOCK_SIZE; + return this; + } + } + + final class BlockDocsEnum extends AbstractPostingsEnum { + + private final int[] freqBuffer = new int[BLOCK_SIZE]; + + private boolean needsFreq; // true if the caller actually needs frequencies + private long freqFP; + + public BlockDocsEnum(FieldInfo fieldInfo) { + super(fieldInfo); + } + + public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) { + final IndexOptions options = fieldInfo.getIndexOptions(); + return docIn == Lucene101PostingsReader.this.docIn + && indexHasFreq == (options.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0); + } + + public PostingsEnum reset(IntBlockTermState termState, int flags) throws IOException { + resetIndexInput(termState); + if (pforUtil == null && docFreq >= BLOCK_SIZE) { + pforUtil = new PForUtil(); + forDeltaUtil = new ForDeltaUtil(); + } + totalTermFreq = indexHasFreq ? termState.totalTermFreq : docFreq; + + this.needsFreq = PostingsEnum.featureRequested(flags, PostingsEnum.FREQS); + if (indexHasFreq == false || needsFreq == false) { + // Filling this buffer may not be cheap when doing primary key lookups, so we make sure to + // not fill more than `docFreq` entries. + Arrays.fill(freqBuffer, 0, Math.min(ForUtil.BLOCK_SIZE, docFreq), 1); + } + freqFP = -1; + return resetIdsAndLevelParams(termState); + } + + @Override + public int freq() throws IOException { + if (freqFP != -1) { + docIn.seek(freqFP); + pforUtil.decode(docInUtil, freqBuffer); + freqFP = -1; + } + + return freqBuffer[docBufferUpto - 1]; + } + + @Override + public int nextPosition() { + return -1; + } + + @Override + public int startOffset() { + return -1; + } + + @Override + public int endOffset() { + return -1; + } + + @Override + public BytesRef getPayload() { + return null; + } + + private void refillFullBlock() throws IOException { + assert docFreq - docCountUpto >= BLOCK_SIZE; + + forDeltaUtil.decodeAndPrefixSum(docInUtil, prevDocID, docBuffer); + + if (indexHasFreq) { + if (needsFreq) { + freqFP = docIn.getFilePointer(); + } + PForUtil.skip(docIn); + } + docCountUpto += BLOCK_SIZE; + prevDocID = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[docBufferSize] == NO_MORE_DOCS; + } + + private void refillRemainder() throws IOException { + final int left = docFreq - docCountUpto; + assert left >= 0; + assert left < BLOCK_SIZE; + + if (docFreq == 1) { + docBuffer[0] = singletonDocID; + freqBuffer[0] = (int) totalTermFreq; + docBuffer[1] = NO_MORE_DOCS; + docCountUpto++; + } else { + // Read vInts: + PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, needsFreq); + prefixSum(docBuffer, left, prevDocID); + docBuffer[left] = NO_MORE_DOCS; + docCountUpto += left; + } + docBufferUpto = 0; + docBufferSize = left; + freqFP = -1; + } + + private void skipLevel1To(int target) throws IOException { + while (true) { + prevDocID = level1LastDocID; + level0LastDocID = level1LastDocID; + docIn.seek(level1DocEndFP); + docCountUpto = level1DocCountUpto; + level1DocCountUpto += LEVEL1_NUM_DOCS; + + if (docFreq - docCountUpto < LEVEL1_NUM_DOCS) { + level1LastDocID = NO_MORE_DOCS; + break; + } + + level1LastDocID += docIn.readVInt(); + level1DocEndFP = docIn.readVLong() + docIn.getFilePointer(); + + if (level1LastDocID >= target) { + if (indexHasFreq) { + // skip impacts and pos skip data + docIn.skipBytes(docIn.readShort()); + } + break; + } + } + } + + private void skipLevel0To(int target) throws IOException { + while (true) { + prevDocID = level0LastDocID; + if (docFreq - docCountUpto >= BLOCK_SIZE) { + long skip0NumBytes = docIn.readVLong(); + // end offset of skip data (before the actual data starts) + long skip0EndFP = docIn.getFilePointer() + skip0NumBytes; + int docDelta = readVInt15(docIn); + level0LastDocID += docDelta; + + if (target <= level0LastDocID) { + docIn.seek(skip0EndFP); + break; + } + + // skip block + docIn.skipBytes(readVLong15(docIn)); + docCountUpto += BLOCK_SIZE; + } else { + level0LastDocID = NO_MORE_DOCS; + break; + } + } + } + + private void moveToNextLevel0Block() throws IOException { + if (doc == level1LastDocID) { // advance skip data on level 1 + skipLevel1To(doc + 1); + } + + prevDocID = level0LastDocID; + if (docFreq - docCountUpto >= BLOCK_SIZE) { + docIn.skipBytes(docIn.readVLong()); + refillFullBlock(); + level0LastDocID = docBuffer[BLOCK_SIZE - 1]; + } else { + level0LastDocID = NO_MORE_DOCS; + refillRemainder(); + } + } + + @Override + public int nextDoc() throws IOException { + if (docBufferUpto == BLOCK_SIZE) { // advance skip data on level 0 + moveToNextLevel0Block(); + } + + return this.doc = docBuffer[docBufferUpto++]; + } + + @Override + public int advance(int target) throws IOException { + if (target > level0LastDocID) { // advance skip data on level 0 + + if (target > level1LastDocID) { // advance skip data on level 1 + skipLevel1To(target); + } + + skipLevel0To(target); + + if (docFreq - docCountUpto >= BLOCK_SIZE) { + refillFullBlock(); + } else { + refillRemainder(); + } + } + + int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + this.doc = docBuffer[next]; + docBufferUpto = next + 1; + return doc; + } + + @Override + public long cost() { + return docFreq; + } + } + + final class EverythingEnum extends AbstractPostingsEnum { + + private final int[] freqBuffer = new int[BLOCK_SIZE + 1]; + private final int[] posDeltaBuffer = new int[BLOCK_SIZE]; + + private final int[] payloadLengthBuffer; + private final int[] offsetStartDeltaBuffer; + private final int[] offsetLengthBuffer; + + private byte[] payloadBytes; + private int payloadByteUpto; + private int payloadLength; + + private int lastStartOffset; + private int startOffset; + private int endOffset; + + private int posBufferUpto; + + final IndexInput posIn; + final PostingDecodingUtil posInUtil; + final IndexInput payIn; + final PostingDecodingUtil payInUtil; + final BytesRef payload; + + final boolean indexHasOffsets; + final boolean indexHasPayloads; + final boolean indexHasOffsetsOrPayloads; + + private int freq; // freq we last read + private int position; // current position + + // how many positions "behind" we are; nextPosition must + // skip these to "catch up": + private int posPendingCount; + + // File pointer where the last (vInt encoded) pos delta + // block is. We need this to know whether to bulk + // decode vs vInt decode the block: + private long lastPosBlockFP; + + private long level0PosEndFP; + private int level0BlockPosUpto; + private long level0PayEndFP; + private int level0BlockPayUpto; + + private long level1PosEndFP; + private int level1BlockPosUpto; + private long level1PayEndFP; + private int level1BlockPayUpto; + + private boolean needsOffsets; // true if we actually need offsets + private boolean needsPayloads; // true if we actually need payloads + + public EverythingEnum(FieldInfo fieldInfo) throws IOException { + super(fieldInfo); + indexHasOffsets = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + indexHasPayloads = fieldInfo.hasPayloads(); + indexHasOffsetsOrPayloads = indexHasOffsets || indexHasPayloads; + + this.posIn = Lucene101PostingsReader.this.posIn.clone(); + posInUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(posIn); + if (indexHasOffsetsOrPayloads) { + this.payIn = Lucene101PostingsReader.this.payIn.clone(); + payInUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(payIn); + } else { + this.payIn = null; + payInUtil = null; + } + if (indexHasOffsets) { + offsetStartDeltaBuffer = new int[BLOCK_SIZE]; + offsetLengthBuffer = new int[BLOCK_SIZE]; + } else { + offsetStartDeltaBuffer = null; + offsetLengthBuffer = null; + startOffset = -1; + endOffset = -1; + } + + if (indexHasPayloads) { + payloadLengthBuffer = new int[BLOCK_SIZE]; + payloadBytes = new byte[128]; + payload = new BytesRef(); + } else { + payloadLengthBuffer = null; + payloadBytes = null; + payload = null; + } + } + + public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) { + return docIn == Lucene101PostingsReader.this.docIn + && indexHasOffsets + == (fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0) + && indexHasPayloads == fieldInfo.hasPayloads(); + } + + public PostingsEnum reset(IntBlockTermState termState, int flags) throws IOException { + resetIndexInput(termState); + if (forDeltaUtil == null && docFreq >= BLOCK_SIZE) { + forDeltaUtil = new ForDeltaUtil(); + } + totalTermFreq = termState.totalTermFreq; + if (pforUtil == null && totalTermFreq >= BLOCK_SIZE) { + pforUtil = new PForUtil(); + } + // Where this term's postings start in the .pos file: + final long posTermStartFP = termState.posStartFP; + // Where this term's payloads/offsets start in the .pay + // file: + final long payTermStartFP = termState.payStartFP; + posIn.seek(posTermStartFP); + if (indexHasOffsetsOrPayloads) { + payIn.seek(payTermStartFP); + } + level1PosEndFP = posTermStartFP; + level1PayEndFP = payTermStartFP; + level0PosEndFP = posTermStartFP; + level0PayEndFP = payTermStartFP; + posPendingCount = 0; + payloadByteUpto = 0; + if (termState.totalTermFreq < BLOCK_SIZE) { + lastPosBlockFP = posTermStartFP; + } else if (termState.totalTermFreq == BLOCK_SIZE) { + lastPosBlockFP = -1; + } else { + lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset; + } + + this.needsOffsets = PostingsEnum.featureRequested(flags, PostingsEnum.OFFSETS); + this.needsPayloads = PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS); + + level1BlockPosUpto = 0; + level1BlockPayUpto = 0; + level0BlockPosUpto = 0; + level0BlockPayUpto = 0; + posBufferUpto = BLOCK_SIZE; + + return resetIdsAndLevelParams(termState); + } + + @Override + public int freq() { + return freq; + } + + private void refillDocs() throws IOException { + final int left = docFreq - docCountUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docInUtil, prevDocID, docBuffer); + pforUtil.decode(docInUtil, freqBuffer); + docCountUpto += BLOCK_SIZE; + } else if (docFreq == 1) { + docBuffer[0] = singletonDocID; + freqBuffer[0] = (int) totalTermFreq; + docBuffer[1] = NO_MORE_DOCS; + docCountUpto++; + docBufferSize = 1; + } else { + // Read vInts: + PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, true); + prefixSum(docBuffer, left, prevDocID); + docBuffer[left] = NO_MORE_DOCS; + docCountUpto += left; + docBufferSize = left; + } + prevDocID = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[docBufferSize] == NO_MORE_DOCS; + } + + private void skipLevel1To(int target) throws IOException { + while (true) { + prevDocID = level1LastDocID; + level0LastDocID = level1LastDocID; + docIn.seek(level1DocEndFP); + level0PosEndFP = level1PosEndFP; + level0BlockPosUpto = level1BlockPosUpto; + if (indexHasOffsetsOrPayloads) { + level0PayEndFP = level1PayEndFP; + level0BlockPayUpto = level1BlockPayUpto; + } + docCountUpto = level1DocCountUpto; + level1DocCountUpto += LEVEL1_NUM_DOCS; + + if (docFreq - docCountUpto < LEVEL1_NUM_DOCS) { + level1LastDocID = NO_MORE_DOCS; + break; + } + + level1LastDocID += docIn.readVInt(); + long delta = docIn.readVLong(); + level1DocEndFP = delta + docIn.getFilePointer(); + + long skip1EndFP = docIn.readShort() + docIn.getFilePointer(); + docIn.skipBytes(docIn.readShort()); // impacts + level1PosEndFP += docIn.readVLong(); + level1BlockPosUpto = docIn.readByte(); + if (indexHasOffsetsOrPayloads) { + level1PayEndFP += docIn.readVLong(); + level1BlockPayUpto = docIn.readVInt(); + } + assert docIn.getFilePointer() == skip1EndFP; + + if (level1LastDocID >= target) { + break; + } + } + } + + private void moveToNextLevel0Block() throws IOException { + if (doc == level1LastDocID) { // advance level 1 skip data + skipLevel1To(doc + 1); + } + + // Now advance level 0 skip data + prevDocID = level0LastDocID; + + assert docBufferUpto == BLOCK_SIZE; + if (level0PosEndFP >= posIn.getFilePointer()) { + posIn.seek(level0PosEndFP); + posPendingCount = level0BlockPosUpto; + if (indexHasOffsetsOrPayloads) { + assert level0PayEndFP >= payIn.getFilePointer(); + payIn.seek(level0PayEndFP); + payloadByteUpto = level0BlockPayUpto; + } + posBufferUpto = BLOCK_SIZE; + } + + if (docFreq - docCountUpto >= BLOCK_SIZE) { + docIn.readVLong(); // skip0 num bytes + int docDelta = readVInt15(docIn); + level0LastDocID += docDelta; + readVLong15(docIn); // block length + docIn.skipBytes(docIn.readVLong()); // impacts + + level0PosEndFP += docIn.readVLong(); + level0BlockPosUpto = docIn.readByte(); + if (indexHasOffsetsOrPayloads) { + level0PayEndFP += docIn.readVLong(); + level0BlockPayUpto = docIn.readVInt(); + } + } else { + level0LastDocID = NO_MORE_DOCS; + } + + refillDocs(); + } + + @Override + public int nextDoc() throws IOException { + if (docBufferUpto == BLOCK_SIZE) { // advance level 0 skip data + moveToNextLevel0Block(); + } + + this.doc = docBuffer[docBufferUpto]; + this.freq = freqBuffer[docBufferUpto]; + docBufferUpto++; + posPendingCount += freq; + position = 0; + lastStartOffset = 0; + return doc; + } + + private void skipLevel0To(int target) throws IOException { + while (true) { + prevDocID = level0LastDocID; + + // If nextBlockPosFP is less than the current FP, it means that the block of positions for + // the first docs of the next block are already decoded. In this case we just accumulate + // frequencies into posPendingCount instead of seeking backwards and decoding the same pos + // block again. + if (level0PosEndFP >= posIn.getFilePointer()) { + posIn.seek(level0PosEndFP); + posPendingCount = level0BlockPosUpto; + if (indexHasOffsetsOrPayloads) { + assert level0PayEndFP >= payIn.getFilePointer(); + payIn.seek(level0PayEndFP); + payloadByteUpto = level0BlockPayUpto; + } + posBufferUpto = BLOCK_SIZE; + } else { + posPendingCount += sumOverRange(freqBuffer, docBufferUpto, BLOCK_SIZE); + } + + if (docFreq - docCountUpto >= BLOCK_SIZE) { + docIn.readVLong(); // skip0 num bytes + int docDelta = readVInt15(docIn); + level0LastDocID += docDelta; + + long blockLength = readVLong15(docIn); + long blockEndFP = docIn.getFilePointer() + blockLength; + docIn.skipBytes(docIn.readVLong()); // impacts + + level0PosEndFP += docIn.readVLong(); + level0BlockPosUpto = docIn.readByte(); + if (indexHasOffsetsOrPayloads) { + level0PayEndFP += docIn.readVLong(); + level0BlockPayUpto = docIn.readVInt(); + } + + if (target <= level0LastDocID) { + break; + } + + docIn.seek(blockEndFP); + docCountUpto += BLOCK_SIZE; + } else { + level0LastDocID = NO_MORE_DOCS; + break; + } + } + } + + @Override + public int advance(int target) throws IOException { + if (target > level0LastDocID) { // advance level 0 skip data + + if (target > level1LastDocID) { // advance level 1 skip data + skipLevel1To(target); + } + + skipLevel0To(target); + + refillDocs(); + } + + int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + posPendingCount += sumOverRange(freqBuffer, docBufferUpto, next + 1); + this.freq = freqBuffer[next]; + this.docBufferUpto = next + 1; + position = 0; + lastStartOffset = 0; + + return this.doc = docBuffer[next]; + } + + private void skipPositions() throws IOException { + // Skip positions now: + int toSkip = posPendingCount - freq; + // if (DEBUG) { + // System.out.println(" FPR.skipPositions: toSkip=" + toSkip); + // } + + final int leftInBlock = BLOCK_SIZE - posBufferUpto; + if (toSkip < leftInBlock) { + int end = posBufferUpto + toSkip; + if (indexHasPayloads) { + payloadByteUpto += sumOverRange(payloadLengthBuffer, posBufferUpto, end); + } + posBufferUpto = end; + } else { + toSkip -= leftInBlock; + while (toSkip >= BLOCK_SIZE) { + assert posIn.getFilePointer() != lastPosBlockFP; + PForUtil.skip(posIn); + + if (indexHasPayloads) { + // Skip payloadLength block: + PForUtil.skip(payIn); + + // Skip payloadBytes block: + int numBytes = payIn.readVInt(); + payIn.seek(payIn.getFilePointer() + numBytes); + } + + if (indexHasOffsets) { + PForUtil.skip(payIn); + PForUtil.skip(payIn); + } + toSkip -= BLOCK_SIZE; + } + refillPositions(); + payloadByteUpto = 0; + if (indexHasPayloads) { + payloadByteUpto += sumOverRange(payloadLengthBuffer, 0, toSkip); + } + posBufferUpto = toSkip; + } + + position = 0; + lastStartOffset = 0; + } + + private void refillPositions() throws IOException { + if (posIn.getFilePointer() == lastPosBlockFP) { + final int count = (int) (totalTermFreq % BLOCK_SIZE); + int payloadLength = 0; + int offsetLength = 0; + payloadByteUpto = 0; + for (int i = 0; i < count; i++) { + int code = posIn.readVInt(); + if (indexHasPayloads) { + if ((code & 1) != 0) { + payloadLength = posIn.readVInt(); + } + payloadLengthBuffer[i] = payloadLength; + posDeltaBuffer[i] = code >>> 1; + if (payloadLength != 0) { + if (payloadByteUpto + payloadLength > payloadBytes.length) { + payloadBytes = ArrayUtil.grow(payloadBytes, payloadByteUpto + payloadLength); + } + posIn.readBytes(payloadBytes, payloadByteUpto, payloadLength); + payloadByteUpto += payloadLength; + } + } else { + posDeltaBuffer[i] = code; + } + + if (indexHasOffsets) { + int deltaCode = posIn.readVInt(); + if ((deltaCode & 1) != 0) { + offsetLength = posIn.readVInt(); + } + offsetStartDeltaBuffer[i] = deltaCode >>> 1; + offsetLengthBuffer[i] = offsetLength; + } + } + payloadByteUpto = 0; + } else { + pforUtil.decode(posInUtil, posDeltaBuffer); + + if (indexHasPayloads) { + if (needsPayloads) { + pforUtil.decode(payInUtil, payloadLengthBuffer); + int numBytes = payIn.readVInt(); + + if (numBytes > payloadBytes.length) { + payloadBytes = ArrayUtil.growNoCopy(payloadBytes, numBytes); + } + payIn.readBytes(payloadBytes, 0, numBytes); + } else { + // this works, because when writing a vint block we always force the first length to be + // written + PForUtil.skip(payIn); // skip over lengths + int numBytes = payIn.readVInt(); // read length of payloadBytes + payIn.seek(payIn.getFilePointer() + numBytes); // skip over payloadBytes + } + payloadByteUpto = 0; + } + + if (indexHasOffsets) { + if (needsOffsets) { + pforUtil.decode(payInUtil, offsetStartDeltaBuffer); + pforUtil.decode(payInUtil, offsetLengthBuffer); + } else { + // this works, because when writing a vint block we always force the first length to be + // written + PForUtil.skip(payIn); // skip over starts + PForUtil.skip(payIn); // skip over lengths + } + } + } + } + + @Override + public int nextPosition() throws IOException { + assert posPendingCount > 0; + + if (posPendingCount > freq) { + skipPositions(); + posPendingCount = freq; + } + + if (posBufferUpto == BLOCK_SIZE) { + refillPositions(); + posBufferUpto = 0; + } + position += posDeltaBuffer[posBufferUpto]; + + if (indexHasPayloads) { + payloadLength = payloadLengthBuffer[posBufferUpto]; + payload.bytes = payloadBytes; + payload.offset = payloadByteUpto; + payload.length = payloadLength; + payloadByteUpto += payloadLength; + } + + if (indexHasOffsets) { + startOffset = lastStartOffset + offsetStartDeltaBuffer[posBufferUpto]; + endOffset = startOffset + offsetLengthBuffer[posBufferUpto]; + lastStartOffset = startOffset; + } + + posBufferUpto++; + posPendingCount--; + return position; + } + + @Override + public int startOffset() { + return startOffset; + } + + @Override + public int endOffset() { + return endOffset; + } + + @Override + public BytesRef getPayload() { + if (payloadLength == 0) { + return null; + } else { + return payload; + } + } + + @Override + public long cost() { + return docFreq; + } + } + + private abstract class BlockImpactsEnum extends ImpactsEnum { + + protected final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(); + protected final PForUtil pforUtil = new PForUtil(); + + protected final int[] docBuffer = new int[BLOCK_SIZE + 1]; + protected final int[] freqBuffer = new int[BLOCK_SIZE]; + + protected final int docFreq; // number of docs in this posting list + + protected final IndexInput docIn; + protected final PostingDecodingUtil docInUtil; + + protected int docCountUpto; // number of docs in or before the current block + protected int doc = -1; // doc we last read + protected int prevDocID = -1; // last doc ID of the previous block + protected int docBufferSize = BLOCK_SIZE; + protected int docBufferUpto = BLOCK_SIZE; + + // true if we shallow-advanced to a new block that we have not decoded yet + protected boolean needsRefilling; + + // level 0 skip data + protected int level0LastDocID = -1; + protected long level0DocEndFP; + protected final BytesRef level0SerializedImpacts; + protected final MutableImpactList level0Impacts; + // level 1 skip data + protected int level1LastDocID; + protected long level1DocEndFP; + protected int level1DocCountUpto = 0; + protected final BytesRef level1SerializedImpacts; + protected final MutableImpactList level1Impacts; + + private BlockImpactsEnum(IntBlockTermState termState) throws IOException { + this.docFreq = termState.docFreq; + this.docIn = Lucene101PostingsReader.this.docIn.clone(); + this.docInUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(docIn); + prefetchPostings(docIn, termState); + level0SerializedImpacts = new BytesRef(maxImpactNumBytesAtLevel0); + level1SerializedImpacts = new BytesRef(maxImpactNumBytesAtLevel1); + level0Impacts = new MutableImpactList(maxNumImpactsAtLevel0); + level1Impacts = new MutableImpactList(maxNumImpactsAtLevel1); + if (docFreq < LEVEL1_NUM_DOCS) { + level1LastDocID = NO_MORE_DOCS; + if (docFreq > 1) { + docIn.seek(termState.docStartFP); + } + } else { + level1LastDocID = -1; + level1DocEndFP = termState.docStartFP; + } + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + } + + @Override + public int docID() { + return doc; + } + + @Override + public int startOffset() { + return -1; + } + + @Override + public int endOffset() { + return -1; + } + + @Override + public BytesRef getPayload() { + return null; + } + + @Override + public long cost() { + return docFreq; + } + + private final Impacts impacts = + new Impacts() { + + private final ByteArrayDataInput scratch = new ByteArrayDataInput(); + + @Override + public int numLevels() { + return level1LastDocID == NO_MORE_DOCS ? 1 : 2; + } + + @Override + public int getDocIdUpTo(int level) { + if (level == 0) { + return level0LastDocID; + } + return level == 1 ? level1LastDocID : NO_MORE_DOCS; + } + + @Override + public List getImpacts(int level) { + if (level == 0 && level0LastDocID != NO_MORE_DOCS) { + return readImpacts(level0SerializedImpacts, level0Impacts); + } + if (level == 1) { + return readImpacts(level1SerializedImpacts, level1Impacts); + } + return DUMMY_IMPACTS; + } + + private List readImpacts(BytesRef serialized, MutableImpactList impactsList) { + var scratch = this.scratch; + scratch.reset(serialized.bytes, 0, serialized.length); + Lucene101PostingsReader.readImpacts(scratch, impactsList); + return impactsList; + } + }; + + @Override + public Impacts getImpacts() { + return impacts; + } + } + + final class BlockImpactsDocsEnum extends BlockImpactsEnum { + final boolean indexHasPos; + + private long freqFP; + + public BlockImpactsDocsEnum(boolean indexHasPos, IntBlockTermState termState) + throws IOException { + super(termState); + this.indexHasPos = indexHasPos; + freqFP = -1; + } + + @Override + public int freq() throws IOException { + if (freqFP != -1) { + docIn.seek(freqFP); + pforUtil.decode(docInUtil, freqBuffer); + freqFP = -1; + } + return freqBuffer[docBufferUpto - 1]; + } + + @Override + public int nextPosition() { + return -1; + } + + private void refillDocs() throws IOException { + final int left = docFreq - docCountUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docInUtil, prevDocID, docBuffer); + freqFP = docIn.getFilePointer(); + PForUtil.skip(docIn); + docCountUpto += BLOCK_SIZE; + } else { + // Read vInts: + PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, true, true); + prefixSum(docBuffer, left, prevDocID); + docBuffer[left] = NO_MORE_DOCS; + freqFP = -1; + docCountUpto += left; + docBufferSize = left; + } + prevDocID = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[docBufferSize] == NO_MORE_DOCS; + } + + private void skipLevel1To(int target) throws IOException { + while (true) { + prevDocID = level1LastDocID; + level0LastDocID = level1LastDocID; + docIn.seek(level1DocEndFP); + docCountUpto = level1DocCountUpto; + level1DocCountUpto += LEVEL1_NUM_DOCS; + + if (docFreq - docCountUpto < LEVEL1_NUM_DOCS) { + level1LastDocID = NO_MORE_DOCS; + break; + } + + level1LastDocID += docIn.readVInt(); + level1DocEndFP = docIn.readVLong() + docIn.getFilePointer(); + + if (level1LastDocID >= target) { + long skip1EndFP = docIn.readShort() + docIn.getFilePointer(); + int numImpactBytes = docIn.readShort(); + docIn.readBytes(level1SerializedImpacts.bytes, 0, numImpactBytes); + level1SerializedImpacts.length = numImpactBytes; + assert indexHasPos || docIn.getFilePointer() == skip1EndFP; + docIn.seek(skip1EndFP); + break; + } + } + } + + private void skipLevel0To(int target) throws IOException { + while (true) { + prevDocID = level0LastDocID; + if (docFreq - docCountUpto >= BLOCK_SIZE) { + long skip0NumBytes = docIn.readVLong(); + // end offset of skip data (before the actual data starts) + long skip0End = docIn.getFilePointer() + skip0NumBytes; + int docDelta = readVInt15(docIn); + long blockLength = readVLong15(docIn); + + level0LastDocID += docDelta; + + if (target <= level0LastDocID) { + level0DocEndFP = docIn.getFilePointer() + blockLength; + int numImpactBytes = docIn.readVInt(); + docIn.readBytes(level0SerializedImpacts.bytes, 0, numImpactBytes); + level0SerializedImpacts.length = numImpactBytes; + docIn.seek(skip0End); + break; + } + + // skip block + docIn.skipBytes(blockLength); + docCountUpto += BLOCK_SIZE; + } else { + level0LastDocID = NO_MORE_DOCS; + break; + } + } + } + + @Override + public void advanceShallow(int target) throws IOException { + if (target > level0LastDocID) { // advance skip data on level 0 + if (target > level1LastDocID) { // advance skip data on level 1 + skipLevel1To(target); + } else if (needsRefilling) { + docIn.seek(level0DocEndFP); + docCountUpto += BLOCK_SIZE; + } + + skipLevel0To(target); + + needsRefilling = true; + } + } + + private void moveToNextLevel0Block() throws IOException { + if (doc == level1LastDocID) { + skipLevel1To(doc + 1); + } else if (needsRefilling) { + docIn.seek(level0DocEndFP); + docCountUpto += BLOCK_SIZE; + } + + prevDocID = level0LastDocID; + if (docFreq - docCountUpto >= BLOCK_SIZE) { + final long skip0Len = docIn.readVLong(); // skip len + final long skip0End = docIn.getFilePointer() + skip0Len; + final int docDelta = readVInt15(docIn); + final long blockLength = readVLong15(docIn); + level0LastDocID += docDelta; + level0DocEndFP = docIn.getFilePointer() + blockLength; + final int numImpactBytes = docIn.readVInt(); + docIn.readBytes(level0SerializedImpacts.bytes, 0, numImpactBytes); + level0SerializedImpacts.length = numImpactBytes; + docIn.seek(skip0End); + } else { + level0LastDocID = NO_MORE_DOCS; + } + + refillDocs(); + needsRefilling = false; + } + + @Override + public int nextDoc() throws IOException { + if (docBufferUpto == BLOCK_SIZE) { + if (needsRefilling) { + refillDocs(); + needsRefilling = false; + } else { + moveToNextLevel0Block(); + } + } + + return this.doc = docBuffer[docBufferUpto++]; + } + + @Override + public int advance(int target) throws IOException { + if (target > level0LastDocID || needsRefilling) { + advanceShallow(target); + refillDocs(); + needsRefilling = false; + } + + int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + this.doc = docBuffer[next]; + docBufferUpto = next + 1; + return doc; + } + } + + final class BlockImpactsPostingsEnum extends BlockImpactsEnum { + private final int[] posDeltaBuffer = new int[BLOCK_SIZE]; + + private int posBufferUpto; + final IndexInput posIn; + final PostingDecodingUtil posInUtil; + + final boolean indexHasFreq; + final boolean indexHasOffsets; + final boolean indexHasPayloads; + final boolean indexHasOffsetsOrPayloads; + + private final long + totalTermFreq; // sum of freqBuffer in this posting list (or docFreq when omitted) + private int freq; // freq we last read + private int position; // current position + + // how many positions "behind" we are; nextPosition must + // skip these to "catch up": + private int posPendingCount; + + // File pointer where the last (vInt encoded) pos delta + // block is. We need this to know whether to bulk + // decode vs vInt decode the block: + private final long lastPosBlockFP; + + // level 0 skip data + private long level0PosEndFP; + private int level0BlockPosUpto; + // level 1 skip data + private long level1PosEndFP; + private int level1BlockPosUpto; + + private final int singletonDocID; // docid when there is a single pulsed posting, otherwise -1 + + public BlockImpactsPostingsEnum(FieldInfo fieldInfo, IntBlockTermState termState) + throws IOException { + super(termState); + final IndexOptions options = fieldInfo.getIndexOptions(); + indexHasFreq = options.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + indexHasOffsets = + options.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0; + indexHasPayloads = fieldInfo.hasPayloads(); + indexHasOffsetsOrPayloads = indexHasOffsets || indexHasPayloads; + + this.posIn = Lucene101PostingsReader.this.posIn.clone(); + posInUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(posIn); + + // Where this term's postings start in the .pos file: + final long posTermStartFP = termState.posStartFP; + totalTermFreq = termState.totalTermFreq; + singletonDocID = termState.singletonDocID; + posIn.seek(posTermStartFP); + level1PosEndFP = posTermStartFP; + level0PosEndFP = posTermStartFP; + posPendingCount = 0; + if (termState.totalTermFreq < BLOCK_SIZE) { + lastPosBlockFP = posTermStartFP; + } else if (termState.totalTermFreq == BLOCK_SIZE) { + lastPosBlockFP = -1; + } else { + lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset; + } + level1BlockPosUpto = 0; + posBufferUpto = BLOCK_SIZE; + } + + @Override + public int freq() { + return freq; + } + + private void refillDocs() throws IOException { + final int left = docFreq - docCountUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docInUtil, prevDocID, docBuffer); + pforUtil.decode(docInUtil, freqBuffer); + docCountUpto += BLOCK_SIZE; + } else if (docFreq == 1) { + docBuffer[0] = singletonDocID; + freqBuffer[0] = (int) totalTermFreq; + docBuffer[1] = NO_MORE_DOCS; + docCountUpto++; + } else { + // Read vInts: + PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, true); + prefixSum(docBuffer, left, prevDocID); + docBuffer[left] = NO_MORE_DOCS; + docCountUpto += left; + docBufferSize = left; + } + prevDocID = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[docBufferSize] == NO_MORE_DOCS; + } + + private void skipLevel1To(int target) throws IOException { + while (true) { + prevDocID = level1LastDocID; + level0LastDocID = level1LastDocID; + docIn.seek(level1DocEndFP); + level0PosEndFP = level1PosEndFP; + level0BlockPosUpto = level1BlockPosUpto; + docCountUpto = level1DocCountUpto; + level1DocCountUpto += LEVEL1_NUM_DOCS; + + if (docFreq - docCountUpto < LEVEL1_NUM_DOCS) { + level1LastDocID = NO_MORE_DOCS; + break; + } + + level1LastDocID += docIn.readVInt(); + level1DocEndFP = docIn.readVLong() + docIn.getFilePointer(); + + long skip1EndFP = docIn.readShort() + docIn.getFilePointer(); + int numImpactBytes = docIn.readShort(); + if (level1LastDocID >= target) { + docIn.readBytes(level1SerializedImpacts.bytes, 0, numImpactBytes); + level1SerializedImpacts.length = numImpactBytes; + } else { + docIn.skipBytes(numImpactBytes); + } + level1PosEndFP += docIn.readVLong(); + level1BlockPosUpto = docIn.readByte(); + assert indexHasOffsetsOrPayloads || docIn.getFilePointer() == skip1EndFP; + + if (level1LastDocID >= target) { + docIn.seek(skip1EndFP); + break; + } + } + } + + private void skipLevel0To(int target) throws IOException { + while (true) { + prevDocID = level0LastDocID; + + // If nextBlockPosFP is less than the current FP, it means that the block of positions for + // the first docs of the next block are already decoded. In this case we just accumulate + // frequencies into posPendingCount instead of seeking backwards and decoding the same pos + // block again. + if (level0PosEndFP >= posIn.getFilePointer()) { + posIn.seek(level0PosEndFP); + posPendingCount = level0BlockPosUpto; + posBufferUpto = BLOCK_SIZE; + } else { + posPendingCount += sumOverRange(freqBuffer, docBufferUpto, BLOCK_SIZE); + } + + if (docFreq - docCountUpto >= BLOCK_SIZE) { + docIn.readVLong(); // skip0 num bytes + int docDelta = readVInt15(docIn); + long blockLength = readVLong15(docIn); + level0DocEndFP = docIn.getFilePointer() + blockLength; + + level0LastDocID += docDelta; + + if (target <= level0LastDocID) { + int numImpactBytes = docIn.readVInt(); + docIn.readBytes(level0SerializedImpacts.bytes, 0, numImpactBytes); + level0SerializedImpacts.length = numImpactBytes; + level0PosEndFP += docIn.readVLong(); + level0BlockPosUpto = docIn.readByte(); + if (indexHasOffsetsOrPayloads) { + docIn.readVLong(); // pay fp delta + docIn.readVInt(); // pay upto + } + break; + } + // skip block + docIn.skipBytes(docIn.readVLong()); // impacts + level0PosEndFP += docIn.readVLong(); + level0BlockPosUpto = docIn.readVInt(); + docIn.seek(level0DocEndFP); + docCountUpto += BLOCK_SIZE; + } else { + level0LastDocID = NO_MORE_DOCS; + break; + } + } + } + + @Override + public void advanceShallow(int target) throws IOException { + if (target > level0LastDocID) { // advance level 0 skip data + + if (target > level1LastDocID) { // advance skip data on level 1 + skipLevel1To(target); + } else if (needsRefilling) { + docIn.seek(level0DocEndFP); + docCountUpto += BLOCK_SIZE; + } + + skipLevel0To(target); + + needsRefilling = true; + } + } + + @Override + public int nextDoc() throws IOException { + if (docBufferUpto == BLOCK_SIZE) { + advanceShallow(doc + 1); + assert needsRefilling; + refillDocs(); + needsRefilling = false; + } + + doc = docBuffer[docBufferUpto]; + freq = freqBuffer[docBufferUpto]; + posPendingCount += freq; + docBufferUpto++; + position = 0; + return this.doc; + } + + @Override + public int advance(int target) throws IOException { + advanceShallow(target); + if (needsRefilling) { + refillDocs(); + needsRefilling = false; + } + + int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + posPendingCount += sumOverRange(freqBuffer, docBufferUpto, next + 1); + freq = freqBuffer[next]; + docBufferUpto = next + 1; + position = 0; + return this.doc = docBuffer[next]; + } + + private void skipPositions() throws IOException { + // Skip positions now: + int toSkip = posPendingCount - freq; + // if (DEBUG) { + // System.out.println(" FPR.skipPositions: toSkip=" + toSkip); + // } + + final int leftInBlock = BLOCK_SIZE - posBufferUpto; + if (toSkip < leftInBlock) { + posBufferUpto += toSkip; + } else { + toSkip -= leftInBlock; + while (toSkip >= BLOCK_SIZE) { + assert posIn.getFilePointer() != lastPosBlockFP; + PForUtil.skip(posIn); + toSkip -= BLOCK_SIZE; + } + refillPositions(); + posBufferUpto = toSkip; + } + + position = 0; + } + + private void refillPositions() throws IOException { + if (posIn.getFilePointer() == lastPosBlockFP) { + final int count = (int) (totalTermFreq % BLOCK_SIZE); + int payloadLength = 0; + for (int i = 0; i < count; i++) { + int code = posIn.readVInt(); + if (indexHasPayloads) { + if ((code & 1) != 0) { + payloadLength = posIn.readVInt(); + } + posDeltaBuffer[i] = code >>> 1; + if (payloadLength != 0) { + posIn.skipBytes(payloadLength); + } + } else { + posDeltaBuffer[i] = code; + } + + if (indexHasOffsets) { + int deltaCode = posIn.readVInt(); + if ((deltaCode & 1) != 0) { + posIn.readVInt(); // offset length + } + } + } + } else { + pforUtil.decode(posInUtil, posDeltaBuffer); + } + } + + @Override + public int nextPosition() throws IOException { + assert posPendingCount > 0; + + if (posPendingCount > freq) { + skipPositions(); + posPendingCount = freq; + } + + if (posBufferUpto == BLOCK_SIZE) { + refillPositions(); + posBufferUpto = 0; + } + position += posDeltaBuffer[posBufferUpto]; + + posBufferUpto++; + posPendingCount--; + return position; + } + } + + /** + * @see Lucene101PostingsWriter#writeVInt15(org.apache.lucene.store.DataOutput, int) + */ + static int readVInt15(DataInput in) throws IOException { + short s = in.readShort(); + if (s >= 0) { + return s; + } else { + return (s & 0x7FFF) | (in.readVInt() << 15); + } + } + + /** + * @see Lucene101PostingsWriter#writeVLong15(org.apache.lucene.store.DataOutput, long) + */ + static long readVLong15(DataInput in) throws IOException { + short s = in.readShort(); + if (s >= 0) { + return s; + } else { + return (s & 0x7FFFL) | (in.readVLong() << 15); + } + } + + private static void prefetchPostings(IndexInput docIn, IntBlockTermState state) + throws IOException { + assert state.docFreq > 1; // Singletons are inlined in the terms dict, nothing to prefetch + if (docIn.getFilePointer() != state.docStartFP) { + // Don't prefetch if the input is already positioned at the right offset, which suggests that + // the caller is streaming the entire inverted index (e.g. for merging), let the read-ahead + // logic do its work instead. Note that this heuristic doesn't work for terms that have skip + // data, since skip data is stored after the last term, but handling all terms that have <128 + // docs is a good start already. + docIn.prefetch(state.docStartFP, 1); + } + // Note: we don't prefetch positions or offsets, which are less likely to be needed. + } + + static class MutableImpactList extends AbstractList implements RandomAccess { + int length; + final Impact[] impacts; + + MutableImpactList(int capacity) { + impacts = new Impact[capacity]; + for (int i = 0; i < capacity; ++i) { + impacts[i] = new Impact(Integer.MAX_VALUE, 1L); + } + } + + @Override + public Impact get(int index) { + return impacts[index]; + } + + @Override + public int size() { + return length; + } + } + + static MutableImpactList readImpacts(ByteArrayDataInput in, MutableImpactList reuse) { + int freq = 0; + long norm = 0; + int length = 0; + while (in.getPosition() < in.length()) { + int freqDelta = in.readVInt(); + if ((freqDelta & 0x01) != 0) { + freq += 1 + (freqDelta >>> 1); + try { + norm += 1 + in.readZLong(); + } catch (IOException e) { + throw new RuntimeException(e); // cannot happen on a BADI + } + } else { + freq += 1 + (freqDelta >>> 1); + norm++; + } + Impact impact = reuse.impacts[length]; + impact.freq = freq; + impact.norm = norm; + length++; + } + reuse.length = length; + return reuse; + } + + @Override + public void checkIntegrity() throws IOException { + if (docIn != null) { + CodecUtil.checksumEntireFile(docIn); + } + if (posIn != null) { + CodecUtil.checksumEntireFile(posIn); + } + if (payIn != null) { + CodecUtil.checksumEntireFile(payIn); + } + } + + @Override + public String toString() { + return getClass().getSimpleName() + + "(positions=" + + (posIn != null) + + ",payloads=" + + (payIn != null) + + ")"; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsWriter.java new file mode 100644 index 00000000000..788a5515f2d --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsWriter.java @@ -0,0 +1,681 @@ +/* + * 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.lucene101; + +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.BLOCK_SIZE; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.DOC_CODEC; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.LEVEL1_MASK; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.META_CODEC; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.PAY_CODEC; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.POS_CODEC; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.TERMS_CODEC; +import static org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.VERSION_CURRENT; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import org.apache.lucene.codecs.BlockTermState; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.CompetitiveImpactAccumulator; +import org.apache.lucene.codecs.PushPostingsWriterBase; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat.IntBlockTermState; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.Impact; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.ByteBuffersDataOutput; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BitUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; + +/** Writer for {@link Lucene101PostingsFormat}. */ +public class Lucene101PostingsWriter extends PushPostingsWriterBase { + + static final IntBlockTermState EMPTY_STATE = new IntBlockTermState(); + + IndexOutput metaOut; + IndexOutput docOut; + IndexOutput posOut; + IndexOutput payOut; + + IntBlockTermState lastState; + + // Holds starting file pointers for current term: + private long docStartFP; + private long posStartFP; + private long payStartFP; + + final int[] docDeltaBuffer; + final int[] freqBuffer; + private int docBufferUpto; + + final int[] posDeltaBuffer; + final int[] payloadLengthBuffer; + final int[] offsetStartDeltaBuffer; + final int[] offsetLengthBuffer; + private int posBufferUpto; + + private byte[] payloadBytes; + private int payloadByteUpto; + + private int level0LastDocID; + private long level0LastPosFP; + private long level0LastPayFP; + + private int level1LastDocID; + private long level1LastPosFP; + private long level1LastPayFP; + + private int docID; + private int lastDocID; + private int lastPosition; + private int lastStartOffset; + private int docCount; + + private final PForUtil pforUtil; + private final ForDeltaUtil forDeltaUtil; + + private boolean fieldHasNorms; + private NumericDocValues norms; + private final CompetitiveImpactAccumulator level0FreqNormAccumulator = + new CompetitiveImpactAccumulator(); + private final CompetitiveImpactAccumulator level1CompetitiveFreqNormAccumulator = + new CompetitiveImpactAccumulator(); + + private int maxNumImpactsAtLevel0; + private int maxImpactNumBytesAtLevel0; + private int maxNumImpactsAtLevel1; + private int maxImpactNumBytesAtLevel1; + + /** Scratch output that we use to be able to prepend the encoded length, e.g. impacts. */ + private final ByteBuffersDataOutput scratchOutput = ByteBuffersDataOutput.newResettableInstance(); + + /** + * Output for a single block. This is useful to be able to prepend skip data before each block, + * which can only be computed once the block is encoded. The content is then typically copied to + * {@link #level1Output}. + */ + private final ByteBuffersDataOutput level0Output = ByteBuffersDataOutput.newResettableInstance(); + + /** + * Output for groups of 32 blocks. This is useful to prepend skip data for these 32 blocks, which + * can only be done once we have encoded these 32 blocks. The content is then typically copied to + * {@link #docCount}. + */ + private final ByteBuffersDataOutput level1Output = ByteBuffersDataOutput.newResettableInstance(); + + /** Sole constructor. */ + public Lucene101PostingsWriter(SegmentWriteState state) throws IOException { + String metaFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene101PostingsFormat.META_EXTENSION); + String docFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene101PostingsFormat.DOC_EXTENSION); + metaOut = state.directory.createOutput(metaFileName, state.context); + IndexOutput posOut = null; + IndexOutput payOut = null; + boolean success = false; + try { + docOut = state.directory.createOutput(docFileName, state.context); + CodecUtil.writeIndexHeader( + metaOut, META_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + CodecUtil.writeIndexHeader( + docOut, DOC_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + forDeltaUtil = new ForDeltaUtil(); + pforUtil = new PForUtil(); + if (state.fieldInfos.hasProx()) { + posDeltaBuffer = new int[BLOCK_SIZE]; + String posFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene101PostingsFormat.POS_EXTENSION); + posOut = state.directory.createOutput(posFileName, state.context); + CodecUtil.writeIndexHeader( + posOut, POS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + + if (state.fieldInfos.hasPayloads()) { + payloadBytes = new byte[128]; + payloadLengthBuffer = new int[BLOCK_SIZE]; + } else { + payloadBytes = null; + payloadLengthBuffer = null; + } + + if (state.fieldInfos.hasOffsets()) { + offsetStartDeltaBuffer = new int[BLOCK_SIZE]; + offsetLengthBuffer = new int[BLOCK_SIZE]; + } else { + offsetStartDeltaBuffer = null; + offsetLengthBuffer = null; + } + + if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) { + String payFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene101PostingsFormat.PAY_EXTENSION); + payOut = state.directory.createOutput(payFileName, state.context); + CodecUtil.writeIndexHeader( + payOut, PAY_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + } + } else { + posDeltaBuffer = null; + payloadLengthBuffer = null; + offsetStartDeltaBuffer = null; + offsetLengthBuffer = null; + payloadBytes = null; + } + this.payOut = payOut; + this.posOut = posOut; + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(metaOut, docOut, posOut, payOut); + } + } + + docDeltaBuffer = new int[BLOCK_SIZE]; + freqBuffer = new int[BLOCK_SIZE]; + } + + @Override + public IntBlockTermState newTermState() { + return new IntBlockTermState(); + } + + @Override + public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException { + CodecUtil.writeIndexHeader( + termsOut, TERMS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + termsOut.writeVInt(BLOCK_SIZE); + } + + @Override + public void setField(FieldInfo fieldInfo) { + super.setField(fieldInfo); + lastState = EMPTY_STATE; + fieldHasNorms = fieldInfo.hasNorms(); + } + + @Override + public void startTerm(NumericDocValues norms) { + docStartFP = docOut.getFilePointer(); + if (writePositions) { + posStartFP = posOut.getFilePointer(); + level1LastPosFP = level0LastPosFP = posStartFP; + if (writePayloads || writeOffsets) { + payStartFP = payOut.getFilePointer(); + level1LastPayFP = level0LastPayFP = payStartFP; + } + } + lastDocID = -1; + level0LastDocID = -1; + level1LastDocID = -1; + this.norms = norms; + if (writeFreqs) { + level0FreqNormAccumulator.clear(); + } + } + + @Override + public void startDoc(int docID, int termDocFreq) throws IOException { + if (docBufferUpto == BLOCK_SIZE) { + flushDocBlock(false); + docBufferUpto = 0; + } + + final int docDelta = docID - lastDocID; + + if (docID < 0 || docDelta <= 0) { + throw new CorruptIndexException( + "docs out of order (" + docID + " <= " + lastDocID + " )", docOut); + } + + docDeltaBuffer[docBufferUpto] = docDelta; + if (writeFreqs) { + freqBuffer[docBufferUpto] = termDocFreq; + } + + this.docID = docID; + lastPosition = 0; + lastStartOffset = 0; + + if (writeFreqs) { + long norm; + if (fieldHasNorms) { + boolean found = norms.advanceExact(docID); + if (found == false) { + // This can happen if indexing hits a problem after adding a doc to the + // postings but before buffering the norm. Such documents are written + // deleted and will go away on the first merge. + norm = 1L; + } else { + norm = norms.longValue(); + assert norm != 0 : docID; + } + } else { + norm = 1L; + } + + level0FreqNormAccumulator.add(termDocFreq, norm); + } + } + + @Override + public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) + throws IOException { + if (position > IndexWriter.MAX_POSITION) { + throw new CorruptIndexException( + "position=" + + position + + " is too large (> IndexWriter.MAX_POSITION=" + + IndexWriter.MAX_POSITION + + ")", + docOut); + } + if (position < 0) { + throw new CorruptIndexException("position=" + position + " is < 0", docOut); + } + posDeltaBuffer[posBufferUpto] = position - lastPosition; + if (writePayloads) { + if (payload == null || payload.length == 0) { + // no payload + payloadLengthBuffer[posBufferUpto] = 0; + } else { + payloadLengthBuffer[posBufferUpto] = payload.length; + if (payloadByteUpto + payload.length > payloadBytes.length) { + payloadBytes = ArrayUtil.grow(payloadBytes, payloadByteUpto + payload.length); + } + System.arraycopy( + payload.bytes, payload.offset, payloadBytes, payloadByteUpto, payload.length); + payloadByteUpto += payload.length; + } + } + + if (writeOffsets) { + assert startOffset >= lastStartOffset; + assert endOffset >= startOffset; + offsetStartDeltaBuffer[posBufferUpto] = startOffset - lastStartOffset; + offsetLengthBuffer[posBufferUpto] = endOffset - startOffset; + lastStartOffset = startOffset; + } + + posBufferUpto++; + lastPosition = position; + if (posBufferUpto == BLOCK_SIZE) { + pforUtil.encode(posDeltaBuffer, posOut); + + if (writePayloads) { + pforUtil.encode(payloadLengthBuffer, payOut); + payOut.writeVInt(payloadByteUpto); + payOut.writeBytes(payloadBytes, 0, payloadByteUpto); + payloadByteUpto = 0; + } + if (writeOffsets) { + pforUtil.encode(offsetStartDeltaBuffer, payOut); + pforUtil.encode(offsetLengthBuffer, payOut); + } + posBufferUpto = 0; + } + } + + @Override + public void finishDoc() { + docBufferUpto++; + docCount++; + + lastDocID = docID; + } + + /** + * Special vints that are encoded on 2 bytes if they require 15 bits or less. VInt becomes + * especially slow when the number of bytes is variable, so this special layout helps in the case + * when the number likely requires 15 bits or less + */ + static void writeVInt15(DataOutput out, int v) throws IOException { + assert v >= 0; + writeVLong15(out, v); + } + + /** + * @see #writeVInt15(DataOutput, int) + */ + static void writeVLong15(DataOutput out, long v) throws IOException { + assert v >= 0; + if ((v & ~0x7FFFL) == 0) { + out.writeShort((short) v); + } else { + out.writeShort((short) (0x8000 | (v & 0x7FFF))); + out.writeVLong(v >> 15); + } + } + + private void flushDocBlock(boolean finishTerm) throws IOException { + assert docBufferUpto != 0; + + if (docBufferUpto < BLOCK_SIZE) { + assert finishTerm; + PostingsUtil.writeVIntBlock( + level0Output, docDeltaBuffer, freqBuffer, docBufferUpto, writeFreqs); + } else { + if (writeFreqs) { + List impacts = level0FreqNormAccumulator.getCompetitiveFreqNormPairs(); + if (impacts.size() > maxNumImpactsAtLevel0) { + maxNumImpactsAtLevel0 = impacts.size(); + } + writeImpacts(impacts, scratchOutput); + assert level0Output.size() == 0; + if (scratchOutput.size() > maxImpactNumBytesAtLevel0) { + maxImpactNumBytesAtLevel0 = Math.toIntExact(scratchOutput.size()); + } + level0Output.writeVLong(scratchOutput.size()); + scratchOutput.copyTo(level0Output); + scratchOutput.reset(); + if (writePositions) { + level0Output.writeVLong(posOut.getFilePointer() - level0LastPosFP); + level0Output.writeByte((byte) posBufferUpto); + level0LastPosFP = posOut.getFilePointer(); + + if (writeOffsets || writePayloads) { + level0Output.writeVLong(payOut.getFilePointer() - level0LastPayFP); + level0Output.writeVInt(payloadByteUpto); + level0LastPayFP = payOut.getFilePointer(); + } + } + } + long numSkipBytes = level0Output.size(); + forDeltaUtil.encodeDeltas(docDeltaBuffer, level0Output); + if (writeFreqs) { + pforUtil.encode(freqBuffer, level0Output); + } + + // docID - lastBlockDocID is at least 128, so it can never fit a single byte with a vint + // Even if we subtracted 128, only extremely dense blocks would be eligible to a single byte + // so let's go with 2 bytes right away + writeVInt15(scratchOutput, docID - level0LastDocID); + writeVLong15(scratchOutput, level0Output.size()); + numSkipBytes += scratchOutput.size(); + level1Output.writeVLong(numSkipBytes); + scratchOutput.copyTo(level1Output); + scratchOutput.reset(); + } + + level0Output.copyTo(level1Output); + level0Output.reset(); + level0LastDocID = docID; + if (writeFreqs) { + level1CompetitiveFreqNormAccumulator.addAll(level0FreqNormAccumulator); + level0FreqNormAccumulator.clear(); + } + + if ((docCount & LEVEL1_MASK) == 0) { // true every 32 blocks (4,096 docs) + writeLevel1SkipData(); + level1LastDocID = docID; + level1CompetitiveFreqNormAccumulator.clear(); + } else if (finishTerm) { + level1Output.copyTo(docOut); + level1Output.reset(); + level1CompetitiveFreqNormAccumulator.clear(); + } + } + + private void writeLevel1SkipData() throws IOException { + docOut.writeVInt(docID - level1LastDocID); + final long level1End; + if (writeFreqs) { + List impacts = level1CompetitiveFreqNormAccumulator.getCompetitiveFreqNormPairs(); + if (impacts.size() > maxNumImpactsAtLevel1) { + maxNumImpactsAtLevel1 = impacts.size(); + } + writeImpacts(impacts, scratchOutput); + long numImpactBytes = scratchOutput.size(); + if (numImpactBytes > maxImpactNumBytesAtLevel1) { + maxImpactNumBytesAtLevel1 = Math.toIntExact(numImpactBytes); + } + if (writePositions) { + scratchOutput.writeVLong(posOut.getFilePointer() - level1LastPosFP); + scratchOutput.writeByte((byte) posBufferUpto); + level1LastPosFP = posOut.getFilePointer(); + if (writeOffsets || writePayloads) { + scratchOutput.writeVLong(payOut.getFilePointer() - level1LastPayFP); + scratchOutput.writeVInt(payloadByteUpto); + level1LastPayFP = payOut.getFilePointer(); + } + } + final long level1Len = 2 * Short.BYTES + scratchOutput.size() + level1Output.size(); + docOut.writeVLong(level1Len); + level1End = docOut.getFilePointer() + level1Len; + // There are at most 128 impacts, that require at most 2 bytes each + assert numImpactBytes <= Short.MAX_VALUE; + // Like impacts plus a few vlongs, still way under the max short value + assert scratchOutput.size() + Short.BYTES <= Short.MAX_VALUE; + docOut.writeShort((short) (scratchOutput.size() + Short.BYTES)); + docOut.writeShort((short) numImpactBytes); + scratchOutput.copyTo(docOut); + scratchOutput.reset(); + } else { + docOut.writeVLong(level1Output.size()); + level1End = docOut.getFilePointer() + level1Output.size(); + } + level1Output.copyTo(docOut); + level1Output.reset(); + assert docOut.getFilePointer() == level1End : docOut.getFilePointer() + " " + level1End; + } + + static void writeImpacts(Collection impacts, DataOutput out) throws IOException { + Impact previous = new Impact(0, 0); + for (Impact impact : impacts) { + assert impact.freq > previous.freq; + assert Long.compareUnsigned(impact.norm, previous.norm) > 0; + int freqDelta = impact.freq - previous.freq - 1; + long normDelta = impact.norm - previous.norm - 1; + if (normDelta == 0) { + // most of time, norm only increases by 1, so we can fold everything in a single byte + out.writeVInt(freqDelta << 1); + } else { + out.writeVInt((freqDelta << 1) | 1); + out.writeZLong(normDelta); + } + previous = impact; + } + } + + /** Called when we are done adding docs to this term */ + @Override + public void finishTerm(BlockTermState _state) throws IOException { + IntBlockTermState state = (IntBlockTermState) _state; + assert state.docFreq > 0; + + // TODO: wasteful we are counting this (counting # docs + // for this term) in two places? + assert state.docFreq == docCount : state.docFreq + " vs " + docCount; + + // docFreq == 1, don't write the single docid/freq to a separate file along with a pointer to + // it. + final int singletonDocID; + if (state.docFreq == 1) { + // pulse the singleton docid into the term dictionary, freq is implicitly totalTermFreq + singletonDocID = docDeltaBuffer[0] - 1; + } else { + singletonDocID = -1; + flushDocBlock(true); + } + + final long lastPosBlockOffset; + + if (writePositions) { + // totalTermFreq is just total number of positions(or payloads, or offsets) + // associated with current term. + assert state.totalTermFreq != -1; + if (state.totalTermFreq > BLOCK_SIZE) { + // record file offset for last pos in last block + lastPosBlockOffset = posOut.getFilePointer() - posStartFP; + } else { + lastPosBlockOffset = -1; + } + if (posBufferUpto > 0) { + assert posBufferUpto < BLOCK_SIZE; + // TODO: should we send offsets/payloads to + // .pay...? seems wasteful (have to store extra + // vLong for low (< BLOCK_SIZE) DF terms = vast vast + // majority) + + // vInt encode the remaining positions/payloads/offsets: + int lastPayloadLength = -1; // force first payload length to be written + int lastOffsetLength = -1; // force first offset length to be written + int payloadBytesReadUpto = 0; + for (int i = 0; i < posBufferUpto; i++) { + final int posDelta = posDeltaBuffer[i]; + if (writePayloads) { + final int payloadLength = payloadLengthBuffer[i]; + if (payloadLength != lastPayloadLength) { + lastPayloadLength = payloadLength; + posOut.writeVInt((posDelta << 1) | 1); + posOut.writeVInt(payloadLength); + } else { + posOut.writeVInt(posDelta << 1); + } + + if (payloadLength != 0) { + posOut.writeBytes(payloadBytes, payloadBytesReadUpto, payloadLength); + payloadBytesReadUpto += payloadLength; + } + } else { + posOut.writeVInt(posDelta); + } + + if (writeOffsets) { + int delta = offsetStartDeltaBuffer[i]; + int length = offsetLengthBuffer[i]; + if (length == lastOffsetLength) { + posOut.writeVInt(delta << 1); + } else { + posOut.writeVInt(delta << 1 | 1); + posOut.writeVInt(length); + lastOffsetLength = length; + } + } + } + + if (writePayloads) { + assert payloadBytesReadUpto == payloadByteUpto; + payloadByteUpto = 0; + } + } + } else { + lastPosBlockOffset = -1; + } + + state.docStartFP = docStartFP; + state.posStartFP = posStartFP; + state.payStartFP = payStartFP; + state.singletonDocID = singletonDocID; + + state.lastPosBlockOffset = lastPosBlockOffset; + docBufferUpto = 0; + posBufferUpto = 0; + lastDocID = -1; + docCount = 0; + } + + @Override + public void encodeTerm( + DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) + throws IOException { + IntBlockTermState state = (IntBlockTermState) _state; + if (absolute) { + lastState = EMPTY_STATE; + assert lastState.docStartFP == 0; + } + + if (lastState.singletonDocID != -1 + && state.singletonDocID != -1 + && state.docStartFP == lastState.docStartFP) { + // With runs of rare values such as ID fields, the increment of pointers in the docs file is + // often 0. + // Furthermore some ID schemes like auto-increment IDs or Flake IDs are monotonic, so we + // encode the delta + // between consecutive doc IDs to save space. + final long delta = (long) state.singletonDocID - lastState.singletonDocID; + out.writeVLong((BitUtil.zigZagEncode(delta) << 1) | 0x01); + } else { + out.writeVLong((state.docStartFP - lastState.docStartFP) << 1); + if (state.singletonDocID != -1) { + out.writeVInt(state.singletonDocID); + } + } + + if (writePositions) { + out.writeVLong(state.posStartFP - lastState.posStartFP); + if (writePayloads || writeOffsets) { + out.writeVLong(state.payStartFP - lastState.payStartFP); + } + } + if (writePositions) { + if (state.lastPosBlockOffset != -1) { + out.writeVLong(state.lastPosBlockOffset); + } + } + lastState = state; + } + + @Override + public void close() throws IOException { + // TODO: add a finish() at least to PushBase? DV too...? + boolean success = false; + try { + if (docOut != null) { + CodecUtil.writeFooter(docOut); + } + if (posOut != null) { + CodecUtil.writeFooter(posOut); + } + if (payOut != null) { + CodecUtil.writeFooter(payOut); + } + if (metaOut != null) { + metaOut.writeInt(maxNumImpactsAtLevel0); + metaOut.writeInt(maxImpactNumBytesAtLevel0); + metaOut.writeInt(maxNumImpactsAtLevel1); + metaOut.writeInt(maxImpactNumBytesAtLevel1); + metaOut.writeLong(docOut.getFilePointer()); + if (posOut != null) { + metaOut.writeLong(posOut.getFilePointer()); + if (payOut != null) { + metaOut.writeLong(payOut.getFilePointer()); + } + } + CodecUtil.writeFooter(metaOut); + } + success = true; + } finally { + if (success) { + IOUtils.close(metaOut, docOut, posOut, payOut); + } else { + IOUtils.closeWhileHandlingException(metaOut, docOut, posOut, payOut); + } + metaOut = docOut = posOut = payOut = null; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene101/PForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/PForUtil.java new file mode 100644 index 00000000000..fd8ecd056b4 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/PForUtil.java @@ -0,0 +1,134 @@ +/* + * 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.lucene101; + +import java.io.IOException; +import java.util.Arrays; +import org.apache.lucene.internal.vectorization.PostingDecodingUtil; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.LongHeap; +import org.apache.lucene.util.packed.PackedInts; + +/** Utility class to encode sequences of 128 small positive integers. */ +final class PForUtil { + + private static final int MAX_EXCEPTIONS = 7; + + static boolean allEqual(int[] l) { + for (int i = 1; i < ForUtil.BLOCK_SIZE; ++i) { + if (l[i] != l[0]) { + return false; + } + } + return true; + } + + private final ForUtil forUtil = new ForUtil(); + + static { + assert ForUtil.BLOCK_SIZE <= 256 : "blocksize must fit in one byte. got " + ForUtil.BLOCK_SIZE; + } + + /** Encode 128 integers from {@code ints} into {@code out}. */ + void encode(int[] ints, DataOutput out) throws IOException { + // Determine the top MAX_EXCEPTIONS + 1 values + final LongHeap top = new LongHeap(MAX_EXCEPTIONS + 1); + for (int i = 0; i <= MAX_EXCEPTIONS; ++i) { + top.push(ints[i]); + } + long topValue = top.top(); + for (int i = MAX_EXCEPTIONS + 1; i < ForUtil.BLOCK_SIZE; ++i) { + if (ints[i] > topValue) { + topValue = top.updateTop(ints[i]); + } + } + + long max = 0L; + for (int i = 1; i <= top.size(); ++i) { + max = Math.max(max, top.get(i)); + } + + final int maxBitsRequired = PackedInts.bitsRequired(max); + // We store the patch on a byte, so we can't decrease the number of bits required by more than 8 + final int patchedBitsRequired = + Math.max(PackedInts.bitsRequired(topValue), maxBitsRequired - 8); + int numExceptions = 0; + final long maxUnpatchedValue = (1L << patchedBitsRequired) - 1; + for (int i = 2; i <= top.size(); ++i) { + if (top.get(i) > maxUnpatchedValue) { + numExceptions++; + } + } + final byte[] exceptions = new byte[numExceptions * 2]; + if (numExceptions > 0) { + int exceptionCount = 0; + for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { + if (ints[i] > maxUnpatchedValue) { + exceptions[exceptionCount * 2] = (byte) i; + exceptions[exceptionCount * 2 + 1] = (byte) (ints[i] >>> patchedBitsRequired); + ints[i] &= maxUnpatchedValue; + exceptionCount++; + } + } + assert exceptionCount == numExceptions : exceptionCount + " " + numExceptions; + } + + if (allEqual(ints) && maxBitsRequired <= 8) { + for (int i = 0; i < numExceptions; ++i) { + exceptions[2 * i + 1] = + (byte) (Byte.toUnsignedLong(exceptions[2 * i + 1]) << patchedBitsRequired); + } + out.writeByte((byte) (numExceptions << 5)); + out.writeVInt(ints[0]); + } else { + final int token = (numExceptions << 5) | patchedBitsRequired; + out.writeByte((byte) token); + forUtil.encode(ints, patchedBitsRequired, out); + } + out.writeBytes(exceptions, exceptions.length); + } + + /** Decode 128 integers into {@code ints}. */ + void decode(PostingDecodingUtil pdu, int[] ints) throws IOException { + var in = pdu.in; + final int token = Byte.toUnsignedInt(in.readByte()); + final int bitsPerValue = token & 0x1f; + if (bitsPerValue == 0) { + Arrays.fill(ints, 0, ForUtil.BLOCK_SIZE, in.readVInt()); + } else { + forUtil.decode(bitsPerValue, pdu, ints); + } + final int numExceptions = token >>> 5; + for (int i = 0; i < numExceptions; ++i) { + ints[Byte.toUnsignedInt(in.readByte())] |= Byte.toUnsignedLong(in.readByte()) << bitsPerValue; + } + } + + /** Skip 128 integers. */ + static void skip(DataInput in) throws IOException { + final int token = Byte.toUnsignedInt(in.readByte()); + final int bitsPerValue = token & 0x1f; + final int numExceptions = token >>> 5; + if (bitsPerValue == 0) { + in.readVLong(); + in.skipBytes((numExceptions << 1)); + } else { + in.skipBytes(ForUtil.numBytes(bitsPerValue) + (numExceptions << 1)); + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingIndexInput.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/PostingIndexInput.java similarity index 84% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingIndexInput.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene101/PostingIndexInput.java index 19bae4770c5..59388e3446b 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingIndexInput.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/PostingIndexInput.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.codecs.lucene101; import java.io.IOException; import org.apache.lucene.internal.vectorization.PostingDecodingUtil; @@ -42,16 +42,16 @@ public final class PostingIndexInput { this.postingDecodingUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(in); } - /** Decode 128 integers stored on {@code bitsPerValues} bits per value into {@code longs}. */ - public void decode(int bitsPerValue, long[] longs) throws IOException { - forUtil.decode(bitsPerValue, postingDecodingUtil, longs); + /** Decode 128 integers stored on {@code bitsPerValues} bits per value into {@code ints}. */ + public void decode(int bitsPerValue, int[] ints) throws IOException { + forUtil.decode(bitsPerValue, postingDecodingUtil, ints); } /** * Decode 128 integers stored on {@code bitsPerValues} bits per value, compute their prefix sum, - * and store results into {@code longs}. + * and store results into {@code ints}. */ - public void decodeAndPrefixSum(int bitsPerValue, long base, long[] longs) throws IOException { - forDeltaUtil.decodeAndPrefixSum(bitsPerValue, postingDecodingUtil, base, longs); + public void decodeAndPrefixSum(int bitsPerValue, int base, int[] ints) throws IOException { + forDeltaUtil.decodeAndPrefixSum(bitsPerValue, postingDecodingUtil, base, ints); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene101/PostingsUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/PostingsUtil.java new file mode 100644 index 00000000000..34431a3689f --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/PostingsUtil.java @@ -0,0 +1,74 @@ +/* + * 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.lucene101; + +import java.io.IOException; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.GroupVIntUtil; + +/** Utility class to encode/decode postings block. */ +final class PostingsUtil { + + /** + * Read values that have been written using variable-length encoding and group-varint encoding + * instead of bit-packing. + */ + static void readVIntBlock( + IndexInput docIn, + int[] docBuffer, + int[] freqBuffer, + int num, + boolean indexHasFreq, + boolean decodeFreq) + throws IOException { + GroupVIntUtil.readGroupVInts(docIn, docBuffer, num); + if (indexHasFreq && decodeFreq) { + for (int i = 0; i < num; ++i) { + freqBuffer[i] = docBuffer[i] & 0x01; + docBuffer[i] >>>= 1; + if (freqBuffer[i] == 0) { + freqBuffer[i] = docIn.readVInt(); + } + } + } else if (indexHasFreq) { + for (int i = 0; i < num; ++i) { + docBuffer[i] >>>= 1; + } + } + } + + /** Write freq buffer with variable-length encoding and doc buffer with group-varint encoding. */ + static void writeVIntBlock( + DataOutput docOut, int[] docBuffer, int[] freqBuffer, int num, boolean writeFreqs) + throws IOException { + if (writeFreqs) { + for (int i = 0; i < num; i++) { + docBuffer[i] = (docBuffer[i] << 1) | (freqBuffer[i] == 1 ? 1 : 0); + } + } + docOut.writeGroupVInts(docBuffer, num); + if (writeFreqs) { + for (int i = 0; i < num; i++) { + final int freq = freqBuffer[i]; + if (freq != 1) { + docOut.writeVInt(freq); + } + } + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene101/gen_ForDeltaUtil.py b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/gen_ForDeltaUtil.py new file mode 100644 index 00000000000..240c66530ae --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/gen_ForDeltaUtil.py @@ -0,0 +1,377 @@ +#! /usr/bin/env python + +# 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. + +from math import gcd + +"""Code generation for ForDeltaUtil.java""" + +MAX_SPECIALIZED_BITS_PER_VALUE = 24 +OUTPUT_FILE = "ForDeltaUtil.java" +PRIMITIVE_SIZE = [8, 16, 32] +HEADER = """// This file has been automatically generated, DO NOT EDIT + +/* + * 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.lucene101; + +import java.io.IOException; +import org.apache.lucene.internal.vectorization.PostingDecodingUtil; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.packed.PackedInts; + +import static org.apache.lucene.codecs.lucene101.ForUtil.*; + +/** + * Inspired from https://fulmicoton.com/posts/bitpacking/ + * Encodes multiple integers in a Java int to get SIMD-like speedups. + * If bitsPerValue <= 4 then we pack 4 ints per Java int + * else if bitsPerValue <= 11 we pack 2 ints per Java int + * else we use scalar operations. + */ +public final class ForDeltaUtil { + + private static final int HALF_BLOCK_SIZE = BLOCK_SIZE / 2; + private static final int ONE_BLOCK_SIZE_FOURTH = BLOCK_SIZE / 4; + private static final int TWO_BLOCK_SIZE_FOURTHS = BLOCK_SIZE / 2; + private static final int THREE_BLOCK_SIZE_FOURTHS = 3 * BLOCK_SIZE / 4; + + // IDENTITY_PLUS_ONE[i] == i+1 + private static final int[] IDENTITY_PLUS_ONE = new int[ForUtil.BLOCK_SIZE]; + + static { + for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { + IDENTITY_PLUS_ONE[i] = i + 1; + } + } + + private static void prefixSumOfOnes(int[] arr, int base) { + System.arraycopy(IDENTITY_PLUS_ONE, 0, arr, 0, ForUtil.BLOCK_SIZE); + // This loop gets auto-vectorized + for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { + arr[i] += base; + } + } + + private static void prefixSum8(int[] arr, int base) { + // When the number of bits per value is 4 or less, we can sum up all values in a block without + // risking overflowing an 8-bits integer. This allows computing the prefix sum by summing up 4 + // values at once. + innerPrefixSum8(arr); + expand8(arr); + final int l0 = base; + final int l1 = l0 + arr[ONE_BLOCK_SIZE_FOURTH - 1]; + final int l2 = l1 + arr[TWO_BLOCK_SIZE_FOURTHS - 1]; + final int l3 = l2 + arr[THREE_BLOCK_SIZE_FOURTHS - 1]; + + for (int i = 0; i < ONE_BLOCK_SIZE_FOURTH; ++i) { + arr[i] += l0; + arr[ONE_BLOCK_SIZE_FOURTH + i] += l1; + arr[TWO_BLOCK_SIZE_FOURTHS + i] += l2; + arr[THREE_BLOCK_SIZE_FOURTHS + i] += l3; + } + } + + private static void prefixSum16(int[] arr, int base) { + // When the number of bits per value is 11 or less, we can sum up all values in a block without + // risking overflowing an 16-bits integer. This allows computing the prefix sum by summing up 2 + // values at once. + innerPrefixSum16(arr); + expand16(arr); + final int l0 = base; + final int l1 = base + arr[HALF_BLOCK_SIZE - 1]; + for (int i = 0; i < HALF_BLOCK_SIZE; ++i) { + arr[i] += l0; + arr[HALF_BLOCK_SIZE + i] += l1; + } + } + + private static void prefixSum32(int[] arr, int base) { + arr[0] += base; + for (int i = 1; i < BLOCK_SIZE; ++i) { + arr[i] += arr[i-1]; + } + } + + // For some reason unrolling seems to help + private static void innerPrefixSum8(int[] arr) { + arr[1] += arr[0]; + arr[2] += arr[1]; + arr[3] += arr[2]; + arr[4] += arr[3]; + arr[5] += arr[4]; + arr[6] += arr[5]; + arr[7] += arr[6]; + arr[8] += arr[7]; + arr[9] += arr[8]; + arr[10] += arr[9]; + arr[11] += arr[10]; + arr[12] += arr[11]; + arr[13] += arr[12]; + arr[14] += arr[13]; + arr[15] += arr[14]; + arr[16] += arr[15]; + arr[17] += arr[16]; + arr[18] += arr[17]; + arr[19] += arr[18]; + arr[20] += arr[19]; + arr[21] += arr[20]; + arr[22] += arr[21]; + arr[23] += arr[22]; + arr[24] += arr[23]; + arr[25] += arr[24]; + arr[26] += arr[25]; + arr[27] += arr[26]; + arr[28] += arr[27]; + arr[29] += arr[28]; + arr[30] += arr[29]; + arr[31] += arr[30]; + } + + // For some reason unrolling seems to help + private static void innerPrefixSum16(int[] arr) { + arr[1] += arr[0]; + arr[2] += arr[1]; + arr[3] += arr[2]; + arr[4] += arr[3]; + arr[5] += arr[4]; + arr[6] += arr[5]; + arr[7] += arr[6]; + arr[8] += arr[7]; + arr[9] += arr[8]; + arr[10] += arr[9]; + arr[11] += arr[10]; + arr[12] += arr[11]; + arr[13] += arr[12]; + arr[14] += arr[13]; + arr[15] += arr[14]; + arr[16] += arr[15]; + arr[17] += arr[16]; + arr[18] += arr[17]; + arr[19] += arr[18]; + arr[20] += arr[19]; + arr[21] += arr[20]; + arr[22] += arr[21]; + arr[23] += arr[22]; + arr[24] += arr[23]; + arr[25] += arr[24]; + arr[26] += arr[25]; + arr[27] += arr[26]; + arr[28] += arr[27]; + arr[29] += arr[28]; + arr[30] += arr[29]; + arr[31] += arr[30]; + arr[32] += arr[31]; + arr[33] += arr[32]; + arr[34] += arr[33]; + arr[35] += arr[34]; + arr[36] += arr[35]; + arr[37] += arr[36]; + arr[38] += arr[37]; + arr[39] += arr[38]; + arr[40] += arr[39]; + arr[41] += arr[40]; + arr[42] += arr[41]; + arr[43] += arr[42]; + arr[44] += arr[43]; + arr[45] += arr[44]; + arr[46] += arr[45]; + arr[47] += arr[46]; + arr[48] += arr[47]; + arr[49] += arr[48]; + arr[50] += arr[49]; + arr[51] += arr[50]; + arr[52] += arr[51]; + arr[53] += arr[52]; + arr[54] += arr[53]; + arr[55] += arr[54]; + arr[56] += arr[55]; + arr[57] += arr[56]; + arr[58] += arr[57]; + arr[59] += arr[58]; + arr[60] += arr[59]; + arr[61] += arr[60]; + arr[62] += arr[61]; + arr[63] += arr[62]; + } + + private final int[] tmp = new int[BLOCK_SIZE]; + + /** + * Encode deltas of a strictly monotonically increasing sequence of integers. The provided {@code + * ints} are expected to be deltas between consecutive values. + */ + void encodeDeltas(int[] ints, DataOutput out) throws IOException { + if (ints[0] == 1 && PForUtil.allEqual(ints)) { // happens with very dense postings + out.writeByte((byte) 0); + } else { + int or = 0; + for (int l : ints) { + or |= l; + } + assert or != 0; + final int bitsPerValue = PackedInts.bitsRequired(or); + out.writeByte((byte) bitsPerValue); + + final int primitiveSize; + if (bitsPerValue <= 3) { + primitiveSize = 8; + collapse8(ints); + } else if (bitsPerValue <= 10) { + primitiveSize = 16; + collapse16(ints); + } else { + primitiveSize = 32; + } + encode(ints, bitsPerValue, primitiveSize, out, tmp); + } + } + + /** Decode deltas, compute the prefix sum and add {@code base} to all decoded ints. */ + void decodeAndPrefixSum(PostingDecodingUtil pdu, int base, int[] ints) throws IOException { + final int bitsPerValue = Byte.toUnsignedInt(pdu.in.readByte()); + if (bitsPerValue == 0) { + prefixSumOfOnes(ints, base); + } else { + decodeAndPrefixSum(bitsPerValue, pdu, base, ints); + } + } + +""" + +def primitive_size_for_bpv(bpv): + if bpv <= 3: + # If we have 4 bits per value or less then we can compute the prefix sum of 32 ints that store 4 8-bit values each without overflowing. + return 8 + elif bpv <= 10: + # If we have 10 bits per value or less then we can compute the prefix sum of 64 ints that store 2 16-bit values each without overflowing. + return 16 + else: + # No risk of overflow with 32 bits per value + return 32 + +def next_primitive(bpv): + if bpv <= 8: + return 8 + elif bpv <= 16: + return 16 + else: + return 32 + +def writeRemainder(bpv, next_primitive, remaining_bits_per_int, o, num_values, f): + iteration = 1 + num_ints = bpv * num_values / remaining_bits_per_int + while num_ints % 2 == 0 and num_values % 2 == 0: + num_ints /= 2 + num_values /= 2 + iteration *= 2 + f.write(' for (int iter = 0, tmpIdx = 0, intsIdx = %d; iter < %d; ++iter, tmpIdx += %d, intsIdx += %d) {\n' %(o, iteration, num_ints, num_values)) + i = 0 + remaining_bits = 0 + tmp_idx = 0 + for i in range(int(num_values)): + b = bpv + if remaining_bits == 0: + b -= remaining_bits_per_int + f.write(' int l%d = tmp[tmpIdx + %d] << %d;\n' %(i, tmp_idx, b)) + else: + b -= remaining_bits + f.write(' int l%d = (tmp[tmpIdx + %d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits, b)) + tmp_idx += 1 + while b >= remaining_bits_per_int: + b -= remaining_bits_per_int + f.write(' l%d |= tmp[tmpIdx + %d] << %d;\n' %(i, tmp_idx, b)) + tmp_idx += 1 + if b > 0: + f.write(' l%d |= (tmp[tmpIdx + %d] >>> %d) & MASK%d_%d;\n' %(i, tmp_idx, remaining_bits_per_int-b, next_primitive, b)) + remaining_bits = remaining_bits_per_int-b + f.write(' ints[intsIdx + %d] = l%d;\n' %(i, i)) + f.write(' }\n') + +def writeDecode(bpv, f): + next_primitive = primitive_size_for_bpv(bpv) + if next_primitive % bpv == 0: + f.write(' private static void decode%dTo%d(PostingDecodingUtil pdu, int[] ints) throws IOException {\n' %(bpv, next_primitive)) + else: + f.write(' private static void decode%dTo%d(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException {\n' %(bpv, next_primitive)) + if bpv == next_primitive: + f.write(' pdu.in.readInts(ints, 0, %d);\n' %(bpv*4)) + else: + num_values_per_int = 32 / next_primitive + remaining_bits = next_primitive % bpv + num_iters = (next_primitive - 1) // bpv + o = 4 * bpv * num_iters + if remaining_bits == 0: + f.write(' pdu.splitInts(%d, ints, %d, %d, MASK%d_%d, ints, %d, MASK%d_%d);\n' %(bpv*4, next_primitive - bpv, bpv, next_primitive, bpv, o, next_primitive, next_primitive - num_iters * bpv)) + else: + f.write(' pdu.splitInts(%d, ints, %d, %d, MASK%d_%d, tmp, 0, MASK%d_%d);\n' %(bpv*4, next_primitive - bpv, bpv, next_primitive, bpv, next_primitive, next_primitive - num_iters * bpv)) + writeRemainder(bpv, next_primitive, remaining_bits, o, 128/num_values_per_int - o, f) + f.write(' }\n') + +if __name__ == '__main__': + f = open(OUTPUT_FILE, 'w') + f.write(HEADER) + f.write(""" + /** + * Delta-decode 128 integers into {@code ints}. + */ + void decodeAndPrefixSum(int bitsPerValue, PostingDecodingUtil pdu, int base, int[] ints) throws IOException { + switch (bitsPerValue) { +""") + for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1): + primitive_size = primitive_size_for_bpv(bpv) + f.write(' case %d:\n' %bpv) + if next_primitive(bpv) == primitive_size: + if primitive_size % bpv == 0: + f.write(' decode%d(pdu, ints);\n' %bpv) + else: + f.write(' decode%d(pdu, tmp, ints);\n' %bpv) + else: + if primitive_size % bpv == 0: + f.write(' decode%dTo%d(pdu, ints);\n' %(bpv, primitive_size)) + else: + f.write(' decode%dTo%d(pdu, tmp, ints);\n' %(bpv, primitive_size)) + f.write(' prefixSum%d(ints, base);\n' %primitive_size) + f.write(' break;\n') + f.write(' default:\n') + f.write(' decodeSlow(bitsPerValue, pdu, tmp, ints);\n') + f.write(' prefixSum32(ints, base);\n') + f.write(' break;\n') + f.write(' }\n') + f.write(' }\n') + + f.write('\n') + for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1): + if next_primitive(bpv) != primitive_size_for_bpv(bpv): + writeDecode(bpv, f) + if bpv < MAX_SPECIALIZED_BITS_PER_VALUE: + f.write('\n') + + f.write('}\n') diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene101/gen_ForUtil.py b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/gen_ForUtil.py new file mode 100644 index 00000000000..0af17974532 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/gen_ForUtil.py @@ -0,0 +1,327 @@ +#! /usr/bin/env python + +# 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. + +from math import gcd + +"""Code generation for ForUtil.java""" + +MAX_SPECIALIZED_BITS_PER_VALUE = 24 +OUTPUT_FILE = "ForUtil.java" +PRIMITIVE_SIZE = [8, 16, 32] +HEADER = """// This file has been automatically generated, DO NOT EDIT + +/* + * 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.lucene101; + +import java.io.IOException; +import org.apache.lucene.internal.vectorization.PostingDecodingUtil; +import org.apache.lucene.store.DataOutput; + +/** + * Inspired from https://fulmicoton.com/posts/bitpacking/ + * Encodes multiple integers in one to get SIMD-like speedups. + * If bitsPerValue <= 8 then we pack 4 ints per Java int + * else if bitsPerValue <= 16 we pack 2 ints per Java int + * else we do scalar operations. + */ +public final class ForUtil { + + public static final int BLOCK_SIZE = 128; + static final int BLOCK_SIZE_LOG2 = 7; + + static int expandMask16(int mask16) { + return mask16 | (mask16 << 16); + } + + static int expandMask8(int mask8) { + return expandMask16(mask8 | (mask8 << 8)); + } + + static int mask32(int bitsPerValue) { + return (1 << bitsPerValue) - 1; + } + + static int mask16(int bitsPerValue) { + return expandMask16((1 << bitsPerValue) - 1); + } + + static int mask8(int bitsPerValue) { + return expandMask8((1 << bitsPerValue) - 1); + } + + static void expand8(int[] arr) { + for (int i = 0; i < 32; ++i) { + int l = arr[i]; + arr[i] = (l >>> 24) & 0xFF; + arr[32 + i] = (l >>> 16) & 0xFF; + arr[64 + i] = (l >>> 8) & 0xFF; + arr[96 + i] = l & 0xFF; + } + } + + static void collapse8(int[] arr) { + for (int i = 0; i < 32; ++i) { + arr[i] = + (arr[i] << 24) + | (arr[32 + i] << 16) + | (arr[64 + i] << 8) + | arr[96 + i]; + } + } + + static void expand16(int[] arr) { + for (int i = 0; i < 64; ++i) { + int l = arr[i]; + arr[i] = (l >>> 16) & 0xFFFF; + arr[64 + i] = l & 0xFFFF; + } + } + + static void collapse16(int[] arr) { + for (int i = 0; i < 64; ++i) { + arr[i] = (arr[i] << 16) | arr[64 + i]; + } + } + + private final int[] tmp = new int[BLOCK_SIZE]; + + /** Encode 128 integers from {@code ints} into {@code out}. */ + void encode(int[] ints, int bitsPerValue, DataOutput out) throws IOException { + final int nextPrimitive; + if (bitsPerValue <= 8) { + nextPrimitive = 8; + collapse8(ints); + } else if (bitsPerValue <= 16) { + nextPrimitive = 16; + collapse16(ints); + } else { + nextPrimitive = 32; + } + encode(ints, bitsPerValue, nextPrimitive, out, tmp); + } + + static void encode(int[] ints, int bitsPerValue, int primitiveSize, DataOutput out, int[] tmp) throws IOException { + final int numInts = BLOCK_SIZE * primitiveSize / Integer.SIZE; + + final int numIntsPerShift = bitsPerValue * 4; + int idx = 0; + int shift = primitiveSize - bitsPerValue; + for (int i = 0; i < numIntsPerShift; ++i) { + tmp[i] = ints[idx++] << shift; + } + for (shift = shift - bitsPerValue; shift >= 0; shift -= bitsPerValue) { + for (int i = 0; i < numIntsPerShift; ++i) { + tmp[i] |= ints[idx++] << shift; + } + } + + final int remainingBitsPerInt = shift + bitsPerValue; + final int maskRemainingBitsPerInt; + if (primitiveSize == 8) { + maskRemainingBitsPerInt = MASKS8[remainingBitsPerInt]; + } else if (primitiveSize == 16) { + maskRemainingBitsPerInt = MASKS16[remainingBitsPerInt]; + } else { + maskRemainingBitsPerInt = MASKS32[remainingBitsPerInt]; + } + + int tmpIdx = 0; + int remainingBitsPerValue = bitsPerValue; + while (idx < numInts) { + if (remainingBitsPerValue >= remainingBitsPerInt) { + remainingBitsPerValue -= remainingBitsPerInt; + tmp[tmpIdx++] |= (ints[idx] >>> remainingBitsPerValue) & maskRemainingBitsPerInt; + if (remainingBitsPerValue == 0) { + idx++; + remainingBitsPerValue = bitsPerValue; + } + } else { + final int mask1, mask2; + if (primitiveSize == 8) { + mask1 = MASKS8[remainingBitsPerValue]; + mask2 = MASKS8[remainingBitsPerInt - remainingBitsPerValue]; + } else if (primitiveSize == 16) { + mask1 = MASKS16[remainingBitsPerValue]; + mask2 = MASKS16[remainingBitsPerInt - remainingBitsPerValue]; + } else { + mask1 = MASKS32[remainingBitsPerValue]; + mask2 = MASKS32[remainingBitsPerInt - remainingBitsPerValue]; + } + tmp[tmpIdx] |= (ints[idx++] & mask1) << (remainingBitsPerInt - remainingBitsPerValue); + remainingBitsPerValue = bitsPerValue - remainingBitsPerInt + remainingBitsPerValue; + tmp[tmpIdx++] |= (ints[idx] >>> remainingBitsPerValue) & mask2; + } + } + + for (int i = 0; i < numIntsPerShift; ++i) { + out.writeInt(tmp[i]); + } + } + + /** Number of bytes required to encode 128 integers of {@code bitsPerValue} bits per value. */ + static int numBytes(int bitsPerValue) { + return bitsPerValue << (BLOCK_SIZE_LOG2 - 3); + } + + static void decodeSlow(int bitsPerValue, PostingDecodingUtil pdu, int[] tmp, int[] ints) + throws IOException { + final int numInts = bitsPerValue << 2; + final int mask = MASKS32[bitsPerValue]; + pdu.splitInts(numInts, ints, 32 - bitsPerValue, 32, mask, tmp, 0, -1); + final int remainingBitsPerInt = 32 - bitsPerValue; + final int mask32RemainingBitsPerInt = MASKS32[remainingBitsPerInt]; + int tmpIdx = 0; + int remainingBits = remainingBitsPerInt; + for (int intsIdx = numInts; intsIdx < BLOCK_SIZE; ++intsIdx) { + int b = bitsPerValue - remainingBits; + int l = (tmp[tmpIdx++] & MASKS32[remainingBits]) << b; + while (b >= remainingBitsPerInt) { + b -= remainingBitsPerInt; + l |= (tmp[tmpIdx++] & mask32RemainingBitsPerInt) << b; + } + if (b > 0) { + l |= (tmp[tmpIdx] >>> (remainingBitsPerInt - b)) & MASKS32[b]; + remainingBits = remainingBitsPerInt - b; + } else { + remainingBits = remainingBitsPerInt; + } + ints[intsIdx] = l; + } + } + +""" + +def writeRemainder(bpv, next_primitive, remaining_bits_per_int, o, num_values, f): + iteration = 1 + num_ints = bpv * num_values / remaining_bits_per_int + while num_ints % 2 == 0 and num_values % 2 == 0: + num_ints /= 2 + num_values /= 2 + iteration *= 2 + f.write(' for (int iter = 0, tmpIdx = 0, intsIdx = %d; iter < %d; ++iter, tmpIdx += %d, intsIdx += %d) {\n' %(o, iteration, num_ints, num_values)) + i = 0 + remaining_bits = 0 + tmp_idx = 0 + for i in range(int(num_values)): + b = bpv + if remaining_bits == 0: + b -= remaining_bits_per_int + f.write(' int l%d = tmp[tmpIdx + %d] << %d;\n' %(i, tmp_idx, b)) + else: + b -= remaining_bits + f.write(' int l%d = (tmp[tmpIdx + %d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits, b)) + tmp_idx += 1 + while b >= remaining_bits_per_int: + b -= remaining_bits_per_int + f.write(' l%d |= tmp[tmpIdx + %d] << %d;\n' %(i, tmp_idx, b)) + tmp_idx += 1 + if b > 0: + f.write(' l%d |= (tmp[tmpIdx + %d] >>> %d) & MASK%d_%d;\n' %(i, tmp_idx, remaining_bits_per_int-b, next_primitive, b)) + remaining_bits = remaining_bits_per_int-b + f.write(' ints[intsIdx + %d] = l%d;\n' %(i, i)) + f.write(' }\n') + + +def writeDecode(bpv, f): + next_primitive = 32 + if bpv <= 8: + next_primitive = 8 + elif bpv <= 16: + next_primitive = 16 + if bpv == next_primitive: + f.write(' static void decode%d(PostingDecodingUtil pdu, int[] ints) throws IOException {\n' %bpv) + f.write(' pdu.in.readInts(ints, 0, %d);\n' %(bpv*4)) + else: + num_values_per_int = 32 / next_primitive + remaining_bits = next_primitive % bpv + num_iters = (next_primitive - 1) // bpv + o = 4 * bpv * num_iters + if remaining_bits == 0: + f.write(' static void decode%d(PostingDecodingUtil pdu, int[] ints) throws IOException {\n' %bpv) + f.write(' pdu.splitInts(%d, ints, %d, %d, MASK%d_%d, ints, %d, MASK%d_%d);\n' %(bpv*4, next_primitive - bpv, bpv, next_primitive, bpv, o, next_primitive, next_primitive - num_iters * bpv)) + else: + f.write(' static void decode%d(PostingDecodingUtil pdu, int[] tmp, int[] ints) throws IOException {\n' %bpv) + f.write(' pdu.splitInts(%d, ints, %d, %d, MASK%d_%d, tmp, 0, MASK%d_%d);\n' %(bpv*4, next_primitive - bpv, bpv, next_primitive, bpv, next_primitive, next_primitive - num_iters * bpv)) + writeRemainder(bpv, next_primitive, remaining_bits, o, 128/num_values_per_int - o, f) + f.write(' }\n') + +if __name__ == '__main__': + f = open(OUTPUT_FILE, 'w') + f.write(HEADER) + for primitive_size in PRIMITIVE_SIZE: + f.write(' static final int[] MASKS%d = new int[%d];\n' %(primitive_size, primitive_size)) + f.write('\n') + f.write(' static {\n') + for primitive_size in PRIMITIVE_SIZE: + f.write(' for (int i = 0; i < %d; ++i) {\n' %primitive_size) + f.write(' MASKS%d[i] = mask%d(i);\n' %(primitive_size, primitive_size)) + f.write(' }\n') + f.write(' }') + f.write(""" + // mark values in array as final ints to avoid the cost of reading array, arrays should only be + // used when the idx is a variable +""") + for primitive_size in PRIMITIVE_SIZE: + for bpv in range(1, min(MAX_SPECIALIZED_BITS_PER_VALUE + 1, primitive_size)): + f.write(' static final int MASK%d_%d = MASKS%d[%d];\n' %(primitive_size, bpv, primitive_size, bpv)) + + f.write(""" + /** Decode 128 integers into {@code ints}. */ + void decode(int bitsPerValue, PostingDecodingUtil pdu, int[] ints) throws IOException { + switch (bitsPerValue) { +""") + for bpv in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1): + next_primitive = 32 + if bpv <= 8: + next_primitive = 8 + elif bpv <= 16: + next_primitive = 16 + f.write(' case %d:\n' %bpv) + if next_primitive % bpv == 0: + f.write(' decode%d(pdu, ints);\n' %bpv) + else: + f.write(' decode%d(pdu, tmp, ints);\n' %bpv) + if next_primitive != 32: + f.write(' expand%d(ints);\n' %next_primitive) + f.write(' break;\n') + f.write(' default:\n') + f.write(' decodeSlow(bitsPerValue, pdu, tmp, ints);\n') + f.write(' break;\n') + f.write(' }\n') + f.write(' }\n') + + for i in range(1, MAX_SPECIALIZED_BITS_PER_VALUE+1): + writeDecode(i, f) + if i < MAX_SPECIALIZED_BITS_PER_VALUE: + f.write('\n') + + f.write('}\n') diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene100/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/package-info.java similarity index 96% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene100/package-info.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene101/package-info.java index 64189bfa9d1..e582f12c318 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene100/package-info.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/package-info.java @@ -16,7 +16,7 @@ */ /** - * Lucene 10.0 file format. + * Lucene 10.1 file format. * *

    Apache Lucene - Index File Formats

    * @@ -151,15 +151,15 @@ * field names. These are used to store auxiliary information about the document, such as its * title, url, or an identifier to access a database. The set of stored fields are what is * returned for each hit when searching. This is keyed by document number. - *
  • {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term dictionary}. A + *
  • {@link org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat Term dictionary}. A * dictionary containing all of the terms used in all of the indexed fields of all of the * documents. The dictionary also contains the number of documents which contain the term, and * pointers to the term's frequency and proximity data. - *
  • {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Frequency data}. For + *
  • {@link org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat Term Frequency data}. For * each term in the dictionary, the numbers of all the documents that contain that term, and * the frequency of the term in that document, unless frequencies are omitted ({@link * org.apache.lucene.index.IndexOptions#DOCS IndexOptions.DOCS}) - *
  • {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Proximity data}. For + *
  • {@link org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat Term Proximity data}. For * each term in the dictionary, the positions that the term occurs in each document. Note that * this will not exist if all fields in all documents omit position data. *
  • {@link org.apache.lucene.codecs.lucene90.Lucene90NormsFormat Normalization factors}. For @@ -255,27 +255,27 @@ * The stored fields for documents * * - * {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Dictionary} + * {@link org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat Term Dictionary} * .tim * The term dictionary, stores term info * * - * {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Term Index} + * {@link org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat Term Index} * .tip * The index into the Term Dictionary * * - * {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Frequencies} + * {@link org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat Frequencies} * .doc * Contains the list of docs which contain each term along with frequency * * - * {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Positions} + * {@link org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat Positions} * .pos * Stores position information about where a term occurs in the index * * - * {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat Payloads} + * {@link org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat Payloads} * .pay * Stores additional per-position metadata information such as character offsets and user payloads * @@ -416,6 +416,8 @@ *
  • In version 9.12, skip data was refactored to have only two levels: every 128 docs and every * 4,06 docs, and to be inlined in postings lists. This resulted in a speedup for queries that * need skipping, especially conjunctions. + *
  • In version 10.1, block encoding changed to be optimized for int[] storage instead of + * long[]. * * * @@ -430,4 +432,4 @@ * UInt64 values, or better yet, {@link org.apache.lucene.store.DataOutput#writeVInt * VInt} values which have no limit. */ -package org.apache.lucene.codecs.lucene100; +package org.apache.lucene.codecs.lucene101; diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java index c336babc9cc..b3e1bed4fd0 100644 --- a/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java @@ -199,7 +199,7 @@ final class DefaultVectorUtilSupport implements VectorUtilSupport { } @Override - public int findNextGEQ(long[] buffer, int length, long target, int from) { + public int findNextGEQ(int[] buffer, int length, int target, int from) { for (int i = from; i < length; ++i) { if (buffer[i] >= target) { return i; diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/PostingDecodingUtil.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/PostingDecodingUtil.java index 4dfa89422d0..e45ce55bbc5 100644 --- a/lucene/core/src/java/org/apache/lucene/internal/vectorization/PostingDecodingUtil.java +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/PostingDecodingUtil.java @@ -34,19 +34,19 @@ public class PostingDecodingUtil { * Core methods for decoding blocks of docs / freqs / positions / offsets. * *
      - *
    • Read {@code count} longs. + *
    • Read {@code count} ints. *
    • For all {@code i} >= 0 so that {@code bShift - i * dec} > 0, apply shift {@code * bShift - i * dec} and store the result in {@code b} at offset {@code count * i}. *
    • Apply mask {@code cMask} and store the result in {@code c} starting at offset {@code * cIndex}. *
    */ - public void splitLongs( - int count, long[] b, int bShift, int dec, long bMask, long[] c, int cIndex, long cMask) + public void splitInts( + int count, int[] b, int bShift, int dec, int bMask, int[] c, int cIndex, int cMask) throws IOException { // Default implementation, which takes advantage of the C2 compiler's loop unrolling and // auto-vectorization. - in.readLongs(c, cIndex, count); + in.readInts(c, cIndex, count); int maxIter = (bShift - 1) / dec; for (int i = 0; i < count; ++i) { for (int j = 0; j <= maxIter; ++j) { diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorUtilSupport.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorUtilSupport.java index ac5b463e6c0..04bd7df52a3 100644 --- a/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorUtilSupport.java +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorUtilSupport.java @@ -51,5 +51,5 @@ public interface VectorUtilSupport { * target}. This index is guaranteed to be at least {@code from}. If there is no such array index, * {@code length} is returned. */ - int findNextGEQ(long[] buffer, int length, long target, int from); + int findNextGEQ(int[] buffer, int length, int target, int from); } diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorizationProvider.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorizationProvider.java index 2a6365297c8..e6441310dcc 100644 --- a/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorizationProvider.java +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorizationProvider.java @@ -190,8 +190,8 @@ public abstract class VectorizationProvider { Set.of( "org.apache.lucene.codecs.hnsw.FlatVectorScorerUtil", "org.apache.lucene.util.VectorUtil", - "org.apache.lucene.codecs.lucene912.Lucene912PostingsReader", - "org.apache.lucene.codecs.lucene912.PostingIndexInput"); + "org.apache.lucene.codecs.lucene101.Lucene101PostingsReader", + "org.apache.lucene.codecs.lucene101.PostingIndexInput"); private static void ensureCaller() { final boolean validCaller = diff --git a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java index 48ac987bee3..1c5436dcf0a 100644 --- a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java @@ -21,8 +21,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Objects; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsReader; import org.apache.lucene.index.ImpactsEnum; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; @@ -399,10 +399,10 @@ public class PhraseQuery extends Query { /** * A guess of the average number of simple operations for the initial seek and buffer refill per * document for the positions of a term. See also {@link - * Lucene912PostingsReader.BlockImpactsPostingsEnum#nextPosition()}. + * Lucene101PostingsReader.BlockImpactsPostingsEnum#nextPosition()}. * *

    Aside: Instead of being constant this could depend among others on {@link - * Lucene912PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link + * Lucene101PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link * TermsEnum#totalTermFreq()}, {@link DocIdSetIterator#cost()} (expected number of matching docs), * {@link LeafReader#maxDoc()} (total number of docs in the segment), and the seek time and block * size of the device storing the index. @@ -411,7 +411,7 @@ public class PhraseQuery extends Query { /** * Number of simple operations in {@link - * Lucene912PostingsReader.BlockImpactsPostingsEnum#nextPosition()} when no seek or buffer refill + * Lucene101PostingsReader.BlockImpactsPostingsEnum#nextPosition()} when no seek or buffer refill * is done. */ private static final int TERM_OPS_PER_POS = 7; diff --git a/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java b/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java index 7f2aadf54a5..1738259fa2f 100644 --- a/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java +++ b/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java @@ -151,7 +151,7 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc } @Override - public void readGroupVInt(long[] dst, int offset) throws IOException { + public void readGroupVInt(int[] dst, int offset) throws IOException { final int len = GroupVIntUtil.readGroupVInt( this, buffer.remaining(), p -> buffer.getInt((int) p), buffer.position(), dst, offset); diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java index a09f78e5f3a..39e92061620 100644 --- a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java +++ b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java @@ -204,7 +204,7 @@ public final class ByteBuffersDataInput extends DataInput } @Override - public void readGroupVInt(long[] dst, int offset) throws IOException { + public void readGroupVInt(int[] dst, int offset) throws IOException { final ByteBuffer block = blocks[blockIndex(pos)]; final int blockOffset = blockOffset(pos); // We MUST save the return value to local variable, could not use pos += readGroupVInt(...). diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersIndexInput.java b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersIndexInput.java index 6aebb771b68..959f429ecfc 100644 --- a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersIndexInput.java +++ b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersIndexInput.java @@ -206,7 +206,7 @@ public final class ByteBuffersIndexInput extends IndexInput implements RandomAcc } @Override - public void readGroupVInt(long[] dst, int offset) throws IOException { + public void readGroupVInt(int[] dst, int offset) throws IOException { ensureOpen(); in.readGroupVInt(dst, offset); } diff --git a/lucene/core/src/java/org/apache/lucene/store/DataInput.java b/lucene/core/src/java/org/apache/lucene/store/DataInput.java index 70f9a96db9c..369b631426b 100644 --- a/lucene/core/src/java/org/apache/lucene/store/DataInput.java +++ b/lucene/core/src/java/org/apache/lucene/store/DataInput.java @@ -102,8 +102,10 @@ public abstract class DataInput implements Cloneable { /** * Override if you have an efficient implementation. In general this is when the input supports * random access. + * + * @lucene.experimental */ - public void readGroupVInt(long[] dst, int offset) throws IOException { + public void readGroupVInt(int[] dst, int offset) throws IOException { GroupVIntUtil.readGroupVInt(this, dst, offset); } diff --git a/lucene/core/src/java/org/apache/lucene/store/DataOutput.java b/lucene/core/src/java/org/apache/lucene/store/DataOutput.java index 7b97dcd8a7b..b312a693eba 100644 --- a/lucene/core/src/java/org/apache/lucene/store/DataOutput.java +++ b/lucene/core/src/java/org/apache/lucene/store/DataOutput.java @@ -340,4 +340,19 @@ public abstract class DataOutput { } GroupVIntUtil.writeGroupVInts(this, groupVIntBytes, values, limit); } + + /** + * Encode integers using group-varint. It uses {@link DataOutput#writeVInt VInt} to encode tail + * values that are not enough for a group. + * + * @param values the values to write + * @param limit the number of values to write. + * @lucene.experimental + */ + public void writeGroupVInts(int[] values, int limit) throws IOException { + if (groupVIntBytes == null) { + groupVIntBytes = new byte[GroupVIntUtil.MAX_LENGTH_PER_GROUP]; + } + GroupVIntUtil.writeGroupVInts(this, groupVIntBytes, values, limit); + } } diff --git a/lucene/core/src/java/org/apache/lucene/util/GroupVIntUtil.java b/lucene/core/src/java/org/apache/lucene/util/GroupVIntUtil.java index 1c5033172db..949292fd9d2 100644 --- a/lucene/core/src/java/org/apache/lucene/util/GroupVIntUtil.java +++ b/lucene/core/src/java/org/apache/lucene/util/GroupVIntUtil.java @@ -30,7 +30,8 @@ public final class GroupVIntUtil { public static final int MAX_LENGTH_PER_GROUP = 17; // we use long array instead of int array to make negative integer to be read as positive long. - private static final long[] MASKS = new long[] {0xFFL, 0xFFFFL, 0xFFFFFFL, 0xFFFFFFFFL}; + private static final long[] LONG_MASKS = new long[] {0xFFL, 0xFFFFL, 0xFFFFFFL, 0xFFFFFFFFL}; + private static final int[] INT_MASKS = new int[] {0xFF, 0xFFFF, 0xFFFFFF, ~0}; /** * Read all the group varints, including the tail vints. we need a long[] because this is what @@ -43,13 +44,30 @@ public final class GroupVIntUtil { public static void readGroupVInts(DataInput in, long[] dst, int limit) throws IOException { int i; for (i = 0; i <= limit - 4; i += 4) { - in.readGroupVInt(dst, i); + readGroupVInt(in, dst, i); } for (; i < limit; ++i) { dst[i] = in.readVInt() & 0xFFFFFFFFL; } } + /** + * Read all the group varints, including the tail vints. + * + * @param dst the array to read ints into. + * @param limit the number of int values to read. + * @lucene.experimental + */ + public static void readGroupVInts(DataInput in, int[] dst, int limit) throws IOException { + int i; + for (i = 0; i <= limit - 4; i += 4) { + in.readGroupVInt(dst, i); + } + for (; i < limit; ++i) { + dst[i] = in.readVInt(); + } + } + /** * Default implementation of read single group, for optimal performance, you should use {@link * GroupVIntUtil#readGroupVInts(DataInput, long[], int)} instead. @@ -66,22 +84,44 @@ public final class GroupVIntUtil { final int n3Minus1 = (flag >> 2) & 0x03; final int n4Minus1 = flag & 0x03; - dst[offset] = readLongInGroup(in, n1Minus1); - dst[offset + 1] = readLongInGroup(in, n2Minus1); - dst[offset + 2] = readLongInGroup(in, n3Minus1); - dst[offset + 3] = readLongInGroup(in, n4Minus1); + dst[offset] = readIntInGroup(in, n1Minus1) & 0xFFFFFFFFL; + dst[offset + 1] = readIntInGroup(in, n2Minus1) & 0xFFFFFFFFL; + dst[offset + 2] = readIntInGroup(in, n3Minus1) & 0xFFFFFFFFL; + dst[offset + 3] = readIntInGroup(in, n4Minus1) & 0xFFFFFFFFL; } - private static long readLongInGroup(DataInput in, int numBytesMinus1) throws IOException { + /** + * Default implementation of read single group, for optimal performance, you should use {@link + * GroupVIntUtil#readGroupVInts(DataInput, int[], int)} instead. + * + * @param in the input to use to read data. + * @param dst the array to read ints into. + * @param offset the offset in the array to start storing ints. + */ + public static void readGroupVInt(DataInput in, int[] dst, int offset) throws IOException { + final int flag = in.readByte() & 0xFF; + + final int n1Minus1 = flag >> 6; + final int n2Minus1 = (flag >> 4) & 0x03; + final int n3Minus1 = (flag >> 2) & 0x03; + final int n4Minus1 = flag & 0x03; + + dst[offset] = readIntInGroup(in, n1Minus1); + dst[offset + 1] = readIntInGroup(in, n2Minus1); + dst[offset + 2] = readIntInGroup(in, n3Minus1); + dst[offset + 3] = readIntInGroup(in, n4Minus1); + } + + private static int readIntInGroup(DataInput in, int numBytesMinus1) throws IOException { switch (numBytesMinus1) { case 0: - return in.readByte() & 0xFFL; + return in.readByte() & 0xFF; case 1: - return in.readShort() & 0xFFFFL; + return in.readShort() & 0xFFFF; case 2: - return (in.readShort() & 0xFFFFL) | ((in.readByte() & 0xFFL) << 16); + return (in.readShort() & 0xFFFF) | ((in.readByte() & 0xFF) << 16); default: - return in.readInt() & 0xFFFFFFFFL; + return in.readInt(); } } @@ -123,13 +163,53 @@ public final class GroupVIntUtil { final int n4Minus1 = flag & 0x03; // This code path has fewer conditionals and tends to be significantly faster in benchmarks - dst[offset] = reader.read(pos) & MASKS[n1Minus1]; + dst[offset] = reader.read(pos) & LONG_MASKS[n1Minus1]; pos += 1 + n1Minus1; - dst[offset + 1] = reader.read(pos) & MASKS[n2Minus1]; + dst[offset + 1] = reader.read(pos) & LONG_MASKS[n2Minus1]; pos += 1 + n2Minus1; - dst[offset + 2] = reader.read(pos) & MASKS[n3Minus1]; + dst[offset + 2] = reader.read(pos) & LONG_MASKS[n3Minus1]; pos += 1 + n3Minus1; - dst[offset + 3] = reader.read(pos) & MASKS[n4Minus1]; + dst[offset + 3] = reader.read(pos) & LONG_MASKS[n4Minus1]; + pos += 1 + n4Minus1; + return (int) (pos - posStart); + } + + /** + * Faster implementation of read single group, It read values from the buffer that would not cross + * boundaries. + * + * @param in the input to use to read data. + * @param remaining the number of remaining bytes allowed to read for current block/segment. + * @param reader the supplier of read int. + * @param pos the start pos to read from the reader. + * @param dst the array to read ints into. + * @param offset the offset in the array to start storing ints. + * @return the number of bytes read excluding the flag. this indicates the number of positions + * should to be increased for caller, it is 0 or positive number and less than {@link + * #MAX_LENGTH_PER_GROUP} + */ + public static int readGroupVInt( + DataInput in, long remaining, IntReader reader, long pos, int[] dst, int offset) + throws IOException { + if (remaining < MAX_LENGTH_PER_GROUP) { + readGroupVInt(in, dst, offset); + return 0; + } + final int flag = in.readByte() & 0xFF; + final long posStart = ++pos; // exclude the flag bytes, the position has updated via readByte(). + final int n1Minus1 = flag >> 6; + final int n2Minus1 = (flag >> 4) & 0x03; + final int n3Minus1 = (flag >> 2) & 0x03; + final int n4Minus1 = flag & 0x03; + + // This code path has fewer conditionals and tends to be significantly faster in benchmarks + dst[offset] = reader.read(pos) & INT_MASKS[n1Minus1]; + pos += 1 + n1Minus1; + dst[offset + 1] = reader.read(pos) & INT_MASKS[n2Minus1]; + pos += 1 + n2Minus1; + dst[offset + 2] = reader.read(pos) & INT_MASKS[n3Minus1]; + pos += 1 + n3Minus1; + dst[offset + 3] = reader.read(pos) & INT_MASKS[n4Minus1]; pos += 1 + n4Minus1; return (int) (pos - posStart); } @@ -180,4 +260,39 @@ public final class GroupVIntUtil { out.writeVInt(toInt(values[readPos])); } } + + /** + * The implementation for group-varint encoding, It uses a maximum of {@link + * #MAX_LENGTH_PER_GROUP} bytes scratch buffer. + */ + public static void writeGroupVInts(DataOutput out, byte[] scratch, int[] values, int limit) + throws IOException { + int readPos = 0; + + // encode each group + while ((limit - readPos) >= 4) { + int writePos = 0; + final int n1Minus1 = numBytes(values[readPos]) - 1; + final int n2Minus1 = numBytes(values[readPos + 1]) - 1; + final int n3Minus1 = numBytes(values[readPos + 2]) - 1; + final int n4Minus1 = numBytes(values[readPos + 3]) - 1; + int flag = (n1Minus1 << 6) | (n2Minus1 << 4) | (n3Minus1 << 2) | (n4Minus1); + scratch[writePos++] = (byte) flag; + BitUtil.VH_LE_INT.set(scratch, writePos, values[readPos++]); + writePos += n1Minus1 + 1; + BitUtil.VH_LE_INT.set(scratch, writePos, values[readPos++]); + writePos += n2Minus1 + 1; + BitUtil.VH_LE_INT.set(scratch, writePos, values[readPos++]); + writePos += n3Minus1 + 1; + BitUtil.VH_LE_INT.set(scratch, writePos, values[readPos++]); + writePos += n4Minus1 + 1; + + out.writeBytes(scratch, writePos); + } + + // tail vints + for (; readPos < limit; readPos++) { + out.writeVInt(toInt(values[readPos])); + } + } } diff --git a/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java index a7239957b32..7d83ca20c00 100644 --- a/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java +++ b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java @@ -314,7 +314,7 @@ public final class VectorUtil { * target}. This index is guaranteed to be at least {@code from}. If there is no such array index, * {@code length} is returned. */ - public static int findNextGEQ(long[] buffer, int length, long target, int from) { + public static int findNextGEQ(int[] buffer, int length, int target, int from) { return IMPL.findNextGEQ(buffer, length, target, from); } } diff --git a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/MemorySegmentPostingDecodingUtil.java b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/MemorySegmentPostingDecodingUtil.java index 582fab392c7..c4a3f823270 100644 --- a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/MemorySegmentPostingDecodingUtil.java +++ b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/MemorySegmentPostingDecodingUtil.java @@ -19,15 +19,15 @@ package org.apache.lucene.internal.vectorization; import java.io.IOException; import java.lang.foreign.MemorySegment; import java.nio.ByteOrder; -import jdk.incubator.vector.LongVector; +import jdk.incubator.vector.IntVector; import jdk.incubator.vector.VectorOperators; import jdk.incubator.vector.VectorSpecies; import org.apache.lucene.store.IndexInput; final class MemorySegmentPostingDecodingUtil extends PostingDecodingUtil { - private static final VectorSpecies LONG_SPECIES = - PanamaVectorConstants.PRERERRED_LONG_SPECIES; + private static final VectorSpecies INT_SPECIES = + PanamaVectorConstants.PRERERRED_INT_SPECIES; private final MemorySegment memorySegment; @@ -37,7 +37,7 @@ final class MemorySegmentPostingDecodingUtil extends PostingDecodingUtil { } private static void shift( - LongVector vector, int bShift, int dec, int maxIter, long bMask, long[] b, int count, int i) { + IntVector vector, int bShift, int dec, int maxIter, int bMask, int[] b, int count, int i) { for (int j = 0; j <= maxIter; ++j) { vector .lanewise(VectorOperators.LSHR, bShift - j * dec) @@ -47,36 +47,35 @@ final class MemorySegmentPostingDecodingUtil extends PostingDecodingUtil { } @Override - public void splitLongs( - int count, long[] b, int bShift, int dec, long bMask, long[] c, int cIndex, long cMask) + public void splitInts( + int count, int[] b, int bShift, int dec, int bMask, int[] c, int cIndex, int cMask) throws IOException { - if (count < LONG_SPECIES.length()) { + if (count < INT_SPECIES.length()) { // Not enough data to vectorize without going out-of-bounds. In practice, this branch is never // used if the bit width is 256, and is used for 2 and 3 bits per value if the bit width is // 512. - super.splitLongs(count, b, bShift, dec, bMask, c, cIndex, cMask); + super.splitInts(count, b, bShift, dec, bMask, c, cIndex, cMask); return; } int maxIter = (bShift - 1) / dec; long offset = in.getFilePointer(); - long endOffset = offset + count * Long.BYTES; - int loopBound = LONG_SPECIES.loopBound(count - 1); + long endOffset = offset + count * Integer.BYTES; + int loopBound = INT_SPECIES.loopBound(count - 1); for (int i = 0; i < loopBound; - i += LONG_SPECIES.length(), offset += LONG_SPECIES.length() * Long.BYTES) { - LongVector vector = - LongVector.fromMemorySegment( - LONG_SPECIES, memorySegment, offset, ByteOrder.LITTLE_ENDIAN); + i += INT_SPECIES.length(), offset += INT_SPECIES.length() * Integer.BYTES) { + IntVector vector = + IntVector.fromMemorySegment(INT_SPECIES, memorySegment, offset, ByteOrder.LITTLE_ENDIAN); shift(vector, bShift, dec, maxIter, bMask, b, count, i); vector.lanewise(VectorOperators.AND, cMask).intoArray(c, cIndex + i); } // Handle the tail by reading a vector that is aligned with `count` on the right side. - int i = count - LONG_SPECIES.length(); - offset = endOffset - LONG_SPECIES.length() * Long.BYTES; - LongVector vector = - LongVector.fromMemorySegment(LONG_SPECIES, memorySegment, offset, ByteOrder.LITTLE_ENDIAN); + int i = count - INT_SPECIES.length(); + offset = endOffset - INT_SPECIES.length() * Integer.BYTES; + IntVector vector = + IntVector.fromMemorySegment(INT_SPECIES, memorySegment, offset, ByteOrder.LITTLE_ENDIAN); shift(vector, bShift, dec, maxIter, bMask, b, count, i); vector.lanewise(VectorOperators.AND, cMask).intoArray(c, cIndex + i); diff --git a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java index cc3624f1b30..68bac1b0b95 100644 --- a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java +++ b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java @@ -29,7 +29,6 @@ import java.lang.foreign.MemorySegment; import jdk.incubator.vector.ByteVector; import jdk.incubator.vector.FloatVector; import jdk.incubator.vector.IntVector; -import jdk.incubator.vector.LongVector; import jdk.incubator.vector.ShortVector; import jdk.incubator.vector.Vector; import jdk.incubator.vector.VectorMask; @@ -59,7 +58,6 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { PanamaVectorConstants.PRERERRED_INT_SPECIES; private static final VectorSpecies BYTE_SPECIES; private static final VectorSpecies SHORT_SPECIES; - private static final VectorSpecies LONG_SPECIES; static final int VECTOR_BITSIZE; @@ -75,7 +73,6 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { BYTE_SPECIES = null; SHORT_SPECIES = null; } - LONG_SPECIES = PanamaVectorConstants.PRERERRED_LONG_SPECIES; } // the way FMA should work! if available use it, otherwise fall back to mul/add @@ -767,17 +764,17 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { return acc1.add(acc2).reduceLanes(ADD); } - // Experiments suggest that we need at least 4 lanes so that the overhead of going with the vector + // Experiments suggest that we need at least 8 lanes so that the overhead of going with the vector // approach and counting trues on vector masks pays off. - private static final boolean ENABLE_FIND_NEXT_GEQ_VECTOR_OPTO = LONG_SPECIES.length() >= 4; + private static final boolean ENABLE_FIND_NEXT_GEQ_VECTOR_OPTO = INT_SPECIES.length() >= 8; @Override - public int findNextGEQ(long[] buffer, int length, long target, int from) { + public int findNextGEQ(int[] buffer, int length, int target, int from) { if (ENABLE_FIND_NEXT_GEQ_VECTOR_OPTO) { - for (; from + LONG_SPECIES.length() < length; from += LONG_SPECIES.length() + 1) { - if (buffer[from + LONG_SPECIES.length()] >= target) { - LongVector vector = LongVector.fromArray(LONG_SPECIES, buffer, from); - VectorMask mask = vector.compare(VectorOperators.LT, target); + for (; from + INT_SPECIES.length() < length; from += INT_SPECIES.length() + 1) { + if (buffer[from + INT_SPECIES.length()] >= target) { + IntVector vector = IntVector.fromArray(INT_SPECIES, buffer, from); + VectorMask mask = vector.compare(VectorOperators.LT, target); return from + mask.trueCount(); } } diff --git a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java index 832fa5f98e6..417511b6f5d 100644 --- a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java +++ b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java @@ -419,7 +419,7 @@ abstract class MemorySegmentIndexInput extends IndexInput } @Override - public void readGroupVInt(long[] dst, int offset) throws IOException { + public void readGroupVInt(int[] dst, int offset) throws IOException { try { final int len = GroupVIntUtil.readGroupVInt( diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec index bd950aeaebd..f7a246c7684 100644 --- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec +++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.lucene.codecs.lucene100.Lucene100Codec +org.apache.lucene.codecs.lucene101.Lucene101Codec diff --git a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat index cd63926a287..874ebafd971 100644 --- a/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat +++ b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat +org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestForDeltaUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestForDeltaUtil.java new file mode 100644 index 00000000000..3e346f3eb20 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestForDeltaUtil.java @@ -0,0 +1,92 @@ +/* + * 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.lucene101; + +import com.carrotsearch.randomizedtesting.generators.RandomNumbers; +import java.io.IOException; +import java.util.Arrays; +import org.apache.lucene.internal.vectorization.PostingDecodingUtil; +import org.apache.lucene.store.ByteBuffersDirectory; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.tests.util.TestUtil; +import org.apache.lucene.util.packed.PackedInts; + +public class TestForDeltaUtil extends LuceneTestCase { + + public void testEncodeDecode() throws IOException { + final int iterations = RandomNumbers.randomIntBetween(random(), 50, 1000); + final int[] values = new int[iterations * ForUtil.BLOCK_SIZE]; + + for (int i = 0; i < iterations; ++i) { + final int bpv = TestUtil.nextInt(random(), 1, 31 - 7); + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + values[i * ForUtil.BLOCK_SIZE + j] = + RandomNumbers.randomIntBetween(random(), 1, (int) PackedInts.maxValue(bpv)); + } + } + + final Directory d = new ByteBuffersDirectory(); + final long endPointer; + + { + // encode + IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(); + + for (int i = 0; i < iterations; ++i) { + int[] source = new int[ForUtil.BLOCK_SIZE]; + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + source[j] = values[i * ForUtil.BLOCK_SIZE + j]; + } + forDeltaUtil.encodeDeltas(source, out); + } + endPointer = out.getFilePointer(); + out.close(); + } + + { + // decode + IndexInput in = d.openInput("test.bin", IOContext.READONCE); + PostingDecodingUtil pdu = + Lucene101PostingsReader.VECTORIZATION_PROVIDER.newPostingDecodingUtil(in); + ForDeltaUtil forDeltaUtil = new ForDeltaUtil(); + for (int i = 0; i < iterations; ++i) { + int base = 0; + final int[] restored = new int[ForUtil.BLOCK_SIZE]; + forDeltaUtil.decodeAndPrefixSum(pdu, base, restored); + final int[] expected = new int[ForUtil.BLOCK_SIZE]; + for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { + expected[j] = values[i * ForUtil.BLOCK_SIZE + j]; + if (j > 0) { + expected[j] += expected[j - 1]; + } else { + expected[j] += base; + } + } + assertArrayEquals(Arrays.toString(restored), expected, restored); + } + assertEquals(endPointer, in.getFilePointer()); + in.close(); + } + + d.close(); + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestForUtil.java similarity index 94% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForUtil.java rename to lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestForUtil.java index df78e045fbb..d93cb0ab3ec 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestForUtil.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestForUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.codecs.lucene101; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; @@ -53,7 +53,7 @@ public class TestForUtil extends LuceneTestCase { final ForUtil forUtil = new ForUtil(); for (int i = 0; i < iterations; ++i) { - long[] source = new long[ForUtil.BLOCK_SIZE]; + int[] source = new int[ForUtil.BLOCK_SIZE]; long or = 0; for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { source[j] = values[i * ForUtil.BLOCK_SIZE + j]; @@ -71,12 +71,12 @@ public class TestForUtil extends LuceneTestCase { // decode IndexInput in = d.openInput("test.bin", IOContext.READONCE); PostingDecodingUtil pdu = - Lucene912PostingsReader.VECTORIZATION_PROVIDER.newPostingDecodingUtil(in); + Lucene101PostingsReader.VECTORIZATION_PROVIDER.newPostingDecodingUtil(in); ForUtil forUtil = new ForUtil(); for (int i = 0; i < iterations; ++i) { final int bitsPerValue = in.readByte(); final long currentFilePointer = in.getFilePointer(); - final long[] restored = new long[ForUtil.BLOCK_SIZE]; + final int[] restored = new int[ForUtil.BLOCK_SIZE]; forUtil.decode(bitsPerValue, pdu, restored); int[] ints = new int[ForUtil.BLOCK_SIZE]; for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestLucene101PostingsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestLucene101PostingsFormat.java new file mode 100644 index 00000000000..6b0ff1fe5e3 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestLucene101PostingsFormat.java @@ -0,0 +1,157 @@ +/* + * 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.lucene101; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.CompetitiveImpactAccumulator; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsReader.MutableImpactList; +import org.apache.lucene.codecs.lucene90.blocktree.FieldReader; +import org.apache.lucene.codecs.lucene90.blocktree.Stats; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.Impact; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.store.ByteArrayDataInput; +import org.apache.lucene.store.ByteArrayDataOutput; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.tests.analysis.MockAnalyzer; +import org.apache.lucene.tests.index.BasePostingsFormatTestCase; +import org.apache.lucene.tests.util.TestUtil; + +public class TestLucene101PostingsFormat extends BasePostingsFormatTestCase { + + @Override + protected Codec getCodec() { + return TestUtil.alwaysPostingsFormat(new Lucene101PostingsFormat()); + } + + public void testVInt15() throws IOException { + byte[] bytes = new byte[5]; + ByteArrayDataOutput out = new ByteArrayDataOutput(bytes); + ByteArrayDataInput in = new ByteArrayDataInput(); + for (int i : new int[] {0, 1, 127, 128, 32767, 32768, Integer.MAX_VALUE}) { + out.reset(bytes); + Lucene101PostingsWriter.writeVInt15(out, i); + in.reset(bytes, 0, out.getPosition()); + assertEquals(i, Lucene101PostingsReader.readVInt15(in)); + assertEquals(out.getPosition(), in.getPosition()); + } + } + + public void testVLong15() throws IOException { + byte[] bytes = new byte[9]; + ByteArrayDataOutput out = new ByteArrayDataOutput(bytes); + ByteArrayDataInput in = new ByteArrayDataInput(); + for (long i : new long[] {0, 1, 127, 128, 32767, 32768, Integer.MAX_VALUE, Long.MAX_VALUE}) { + out.reset(bytes); + Lucene101PostingsWriter.writeVLong15(out, i); + in.reset(bytes, 0, out.getPosition()); + assertEquals(i, Lucene101PostingsReader.readVLong15(in)); + assertEquals(out.getPosition(), in.getPosition()); + } + } + + /** Make sure the final sub-block(s) are not skipped. */ + public void testFinalBlock() throws Exception { + Directory d = newDirectory(); + IndexWriter w = new IndexWriter(d, new IndexWriterConfig(new MockAnalyzer(random()))); + for (int i = 0; i < 25; i++) { + Document doc = new Document(); + doc.add(newStringField("field", Character.toString((char) (97 + i)), Field.Store.NO)); + doc.add(newStringField("field", "z" + Character.toString((char) (97 + i)), Field.Store.NO)); + w.addDocument(doc); + } + w.forceMerge(1); + + DirectoryReader r = DirectoryReader.open(w); + assertEquals(1, r.leaves().size()); + FieldReader field = (FieldReader) r.leaves().get(0).reader().terms("field"); + // We should see exactly two blocks: one root block (prefix empty string) and one block for z* + // terms (prefix z): + Stats stats = field.getStats(); + assertEquals(0, stats.floorBlockCount); + assertEquals(2, stats.nonFloorBlockCount); + r.close(); + w.close(); + d.close(); + } + + public void testImpactSerialization() throws IOException { + // omit norms and omit freqs + doTestImpactSerialization(Collections.singletonList(new Impact(1, 1L))); + + // omit freqs + doTestImpactSerialization(Collections.singletonList(new Impact(1, 42L))); + // omit freqs with very large norms + doTestImpactSerialization(Collections.singletonList(new Impact(1, -100L))); + + // omit norms + doTestImpactSerialization(Collections.singletonList(new Impact(30, 1L))); + // omit norms with large freq + doTestImpactSerialization(Collections.singletonList(new Impact(500, 1L))); + + // freqs and norms, basic + doTestImpactSerialization( + Arrays.asList( + new Impact(1, 7L), + new Impact(3, 9L), + new Impact(7, 10L), + new Impact(15, 11L), + new Impact(20, 13L), + new Impact(28, 14L))); + + // freqs and norms, high values + doTestImpactSerialization( + Arrays.asList( + new Impact(2, 2L), + new Impact(10, 10L), + new Impact(12, 50L), + new Impact(50, -100L), + new Impact(1000, -80L), + new Impact(1005, -3L))); + } + + private void doTestImpactSerialization(List impacts) throws IOException { + CompetitiveImpactAccumulator acc = new CompetitiveImpactAccumulator(); + for (Impact impact : impacts) { + acc.add(impact.freq, impact.norm); + } + try (Directory dir = newDirectory()) { + try (IndexOutput out = dir.createOutput("foo", IOContext.DEFAULT)) { + Lucene101PostingsWriter.writeImpacts(acc.getCompetitiveFreqNormPairs(), out); + } + try (IndexInput in = dir.openInput("foo", IOContext.DEFAULT)) { + byte[] b = new byte[Math.toIntExact(in.length())]; + in.readBytes(b, 0, b.length); + List impacts2 = + Lucene101PostingsReader.readImpacts( + new ByteArrayDataInput(b), + new MutableImpactList(impacts.size() + random().nextInt(3))); + assertEquals(impacts, impacts2); + } + } + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestPForUtil.java similarity index 94% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java rename to lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestPForUtil.java index 2dd3f5ece75..4e9ab4b55ee 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestPForUtil.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestPForUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene912; +package org.apache.lucene.codecs.lucene101; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; @@ -41,14 +41,14 @@ public class TestPForUtil extends LuceneTestCase { IndexInput in = d.openInput("test.bin", IOContext.READONCE); PostingDecodingUtil pdu = - Lucene912PostingsReader.VECTORIZATION_PROVIDER.newPostingDecodingUtil(in); + Lucene101PostingsReader.VECTORIZATION_PROVIDER.newPostingDecodingUtil(in); final PForUtil pforUtil = new PForUtil(); for (int i = 0; i < iterations; ++i) { if (random().nextInt(5) == 0) { PForUtil.skip(in); continue; } - final long[] restored = new long[ForUtil.BLOCK_SIZE]; + final int[] restored = new int[ForUtil.BLOCK_SIZE]; pforUtil.decode(pdu, restored); int[] ints = new int[ForUtil.BLOCK_SIZE]; for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { @@ -93,7 +93,7 @@ public class TestPForUtil extends LuceneTestCase { final PForUtil pforUtil = new PForUtil(); for (int i = 0; i < iterations; ++i) { - long[] source = new long[ForUtil.BLOCK_SIZE]; + int[] source = new int[ForUtil.BLOCK_SIZE]; for (int j = 0; j < ForUtil.BLOCK_SIZE; ++j) { source[j] = values[i * ForUtil.BLOCK_SIZE + j]; } diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestPostingsUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestPostingsUtil.java new file mode 100644 index 00000000000..41bb5c01f07 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestPostingsUtil.java @@ -0,0 +1,49 @@ +/* + * 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.lucene101; + +import java.io.IOException; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.tests.util.LuceneTestCase; + +public class TestPostingsUtil extends LuceneTestCase { + + // checks for bug described in https://github.com/apache/lucene/issues/13373 + public void testIntegerOverflow() throws IOException { + final int size = random().nextInt(1, ForUtil.BLOCK_SIZE); + final int[] docDeltaBuffer = new int[size]; + final int[] freqBuffer = new int[size]; + + final int delta = 1 << 30; + docDeltaBuffer[0] = delta; + try (Directory dir = newDirectory()) { + try (IndexOutput out = dir.createOutput("test", IOContext.DEFAULT)) { + // In old implementation, this would cause integer overflow exception. + PostingsUtil.writeVIntBlock(out, docDeltaBuffer, freqBuffer, size, true); + } + int[] restoredDocs = new int[size]; + int[] restoredFreqs = new int[size]; + try (IndexInput in = dir.openInput("test", IOContext.DEFAULT)) { + PostingsUtil.readVIntBlock(in, restoredDocs, restoredFreqs, size, true, true); + } + assertEquals(delta, restoredDocs[0]); + } + } +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java index fe6c82e73bb..69fbf96f6da 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java @@ -18,7 +18,7 @@ package org.apache.lucene.codecs.lucene90; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.lucene100.Lucene100Codec; +import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.document.Document; import org.apache.lucene.document.StoredField; import org.apache.lucene.index.DirectoryReader; @@ -31,7 +31,7 @@ import org.apache.lucene.tests.index.BaseStoredFieldsFormatTestCase; public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase { @Override protected Codec getCodec() { - return new Lucene100Codec(Lucene100Codec.Mode.BEST_COMPRESSION); + return new Lucene101Codec(Lucene101Codec.Mode.BEST_COMPRESSION); } /** @@ -42,7 +42,7 @@ public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFie for (int i = 0; i < 10; i++) { IndexWriterConfig iwc = newIndexWriterConfig(); iwc.setCodec( - new Lucene100Codec(RandomPicks.randomFrom(random(), Lucene100Codec.Mode.values()))); + new Lucene101Codec(RandomPicks.randomFrom(random(), Lucene101Codec.Mode.values()))); IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig()); Document doc = new Document(); doc.add(new StoredField("field1", "value1")); @@ -72,7 +72,7 @@ public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFie expectThrows( NullPointerException.class, () -> { - new Lucene100Codec(null); + new Lucene101Codec(null); }); expectThrows( diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java index ed70b2df002..8b6efa0e41f 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java @@ -28,7 +28,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; -import org.apache.lucene.codecs.lucene100.Lucene100Codec; +import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.KnnFloatVectorField; @@ -74,7 +74,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat @Override protected Codec getCodec() { - return new Lucene100Codec() { + return new Lucene101Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return format; @@ -106,7 +106,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat dir, newIndexWriterConfig() .setCodec( - new Lucene100Codec() { + new Lucene101Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return getKnnFormat(4); @@ -126,7 +126,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat dir, newIndexWriterConfig() .setCodec( - new Lucene100Codec() { + new Lucene101Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return getKnnFormat(7); @@ -163,7 +163,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat dir, newIndexWriterConfig() .setCodec( - new Lucene100Codec() { + new Lucene101Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return new Lucene99HnswVectorsFormat(); @@ -183,7 +183,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat dir, newIndexWriterConfig() .setCodec( - new Lucene100Codec() { + new Lucene101Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return getKnnFormat(7); @@ -216,7 +216,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat dir, newIndexWriterConfig() .setCodec( - new Lucene100Codec() { + new Lucene101Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return new Lucene99HnswScalarQuantizedVectorsFormat( diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java index 3b758de6ce6..6cd249240c0 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java @@ -27,7 +27,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer; -import org.apache.lucene.codecs.lucene100.Lucene100Codec; +import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; @@ -52,7 +52,7 @@ import org.apache.lucene.util.quantization.ScalarQuantizer; public class TestLucene99ScalarQuantizedVectorScorer extends LuceneTestCase { private static Codec getCodec(int bits, boolean compress) { - return new Lucene100Codec() { + return new Lucene101Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return new Lucene99HnswScalarQuantizedVectorsFormat( diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java index c7f4515c25c..86a5ff5068d 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java @@ -28,7 +28,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; -import org.apache.lucene.codecs.lucene100.Lucene100Codec; +import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.KnnFloatVectorField; @@ -70,7 +70,7 @@ public class TestLucene99ScalarQuantizedVectorsFormat extends BaseKnnVectorsForm @Override protected Codec getCodec() { - return new Lucene100Codec() { + return new Lucene101Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return format; diff --git a/lucene/core/src/test/org/apache/lucene/internal/vectorization/TestPostingDecodingUtil.java b/lucene/core/src/test/org/apache/lucene/internal/vectorization/TestPostingDecodingUtil.java index 517b3ec32aa..6c914dfcc03 100644 --- a/lucene/core/src/test/org/apache/lucene/internal/vectorization/TestPostingDecodingUtil.java +++ b/lucene/core/src/test/org/apache/lucene/internal/vectorization/TestPostingDecodingUtil.java @@ -16,7 +16,7 @@ */ package org.apache.lucene.internal.vectorization; -import org.apache.lucene.codecs.lucene912.ForUtil; +import org.apache.lucene.codecs.lucene101.ForUtil; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; @@ -27,7 +27,7 @@ import org.apache.lucene.tests.util.TestUtil; public class TestPostingDecodingUtil extends LuceneTestCase { - public void testDuelSplitLongs() throws Exception { + public void testDuelSplitInts() throws Exception { final int iterations = atLeast(100); try (Directory dir = new MMapDirectory(createTempDir())) { @@ -39,25 +39,25 @@ public class TestPostingDecodingUtil extends LuceneTestCase { } VectorizationProvider vectorizationProvider = VectorizationProvider.lookup(true); try (IndexInput in = dir.openInput("tests.bin", IOContext.DEFAULT)) { - long[] expectedB = new long[ForUtil.BLOCK_SIZE]; - long[] expectedC = new long[ForUtil.BLOCK_SIZE]; - long[] actualB = new long[ForUtil.BLOCK_SIZE]; - long[] actualC = new long[ForUtil.BLOCK_SIZE]; + int[] expectedB = new int[ForUtil.BLOCK_SIZE]; + int[] expectedC = new int[ForUtil.BLOCK_SIZE]; + int[] actualB = new int[ForUtil.BLOCK_SIZE]; + int[] actualC = new int[ForUtil.BLOCK_SIZE]; for (int iter = 0; iter < iterations; ++iter) { // Initialize arrays with random content. for (int i = 0; i < expectedB.length; ++i) { - expectedB[i] = random().nextLong(); + expectedB[i] = random().nextInt(); actualB[i] = expectedB[i]; - expectedC[i] = random().nextLong(); + expectedC[i] = random().nextInt(); actualC[i] = expectedC[i]; } int bShift = TestUtil.nextInt(random(), 1, 31); int dec = TestUtil.nextInt(random(), 1, bShift); int numIters = (bShift + dec - 1) / dec; int count = TestUtil.nextInt(random(), 1, 64 / numIters); - long bMask = random().nextLong(); + int bMask = random().nextInt(); int cIndex = random().nextInt(64); - long cMask = random().nextLong(); + int cMask = random().nextInt(); long startFP = random().nextInt(4); // Work on a slice that has just the right number of bytes to make the test fail with an @@ -69,10 +69,10 @@ public class TestPostingDecodingUtil extends LuceneTestCase { PostingDecodingUtil optimizedUtil = vectorizationProvider.newPostingDecodingUtil(slice); slice.seek(startFP); - defaultUtil.splitLongs(count, expectedB, bShift, dec, bMask, expectedC, cIndex, cMask); + defaultUtil.splitInts(count, expectedB, bShift, dec, bMask, expectedC, cIndex, cMask); long expectedEndFP = slice.getFilePointer(); slice.seek(startFP); - optimizedUtil.splitLongs(count, actualB, bShift, dec, bMask, actualC, cIndex, cMask); + optimizedUtil.splitInts(count, actualB, bShift, dec, bMask, actualC, cIndex, cMask); assertEquals(expectedEndFP, slice.getFilePointer()); assertArrayEquals(expectedB, actualB); assertArrayEquals(expectedC, actualC); diff --git a/lucene/core/src/test/org/apache/lucene/util/TestVectorUtil.java b/lucene/core/src/test/org/apache/lucene/util/TestVectorUtil.java index 541dbbde8d7..05450861024 100644 --- a/lucene/core/src/test/org/apache/lucene/util/TestVectorUtil.java +++ b/lucene/core/src/test/org/apache/lucene/util/TestVectorUtil.java @@ -356,8 +356,8 @@ public class TestVectorUtil extends LuceneTestCase { public void testFindNextGEQ() { int padding = TestUtil.nextInt(random(), 0, 5); - long[] values = new long[128 + padding]; - long v = 0; + int[] values = new int[128 + padding]; + int v = 0; for (int i = 0; i < 128; ++i) { v += TestUtil.nextInt(random(), 1, 1000); values[i] = v; @@ -366,8 +366,8 @@ public class TestVectorUtil extends LuceneTestCase { // Now duel with slowFindFirstGreater for (int iter = 0; iter < 1_000; ++iter) { int from = TestUtil.nextInt(random(), 0, 127); - long target = - TestUtil.nextLong(random(), values[from], Math.max(values[from], values[127])) + int target = + TestUtil.nextInt(random(), values[from], Math.max(values[from], values[127])) + random().nextInt(10) - 5; assertEquals( @@ -376,7 +376,7 @@ public class TestVectorUtil extends LuceneTestCase { } } - private static int slowFindNextGEQ(long[] buffer, int length, long target, int from) { + private static int slowFindNextGEQ(int[] buffer, int length, int target, int from) { for (int i = from; i < length; ++i) { if (buffer[i] >= target) { return i; diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java index e5ca29b9fae..c0f2f61d7cc 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java @@ -21,8 +21,8 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.Objects; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsReader; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.PostingsEnum; @@ -261,10 +261,10 @@ class TermIntervalsSource extends IntervalsSource { /** * A guess of the average number of simple operations for the initial seek and buffer refill per * document for the positions of a term. See also {@link - * Lucene912PostingsReader.EverythingEnum#nextPosition()}. + * Lucene101PostingsReader.EverythingEnum#nextPosition()}. * *

    Aside: Instead of being constant this could depend among others on {@link - * Lucene912PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link + * Lucene101PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link * TermsEnum#totalTermFreq()}, {@link DocIdSetIterator#cost()} (expected number of matching docs), * {@link LeafReader#maxDoc()} (total number of docs in the segment), and the seek time and block * size of the device storing the index. @@ -272,7 +272,7 @@ class TermIntervalsSource extends IntervalsSource { private static final int TERM_POSNS_SEEK_OPS_PER_DOC = 128; /** - * Number of simple operations in {@link Lucene912PostingsReader.EverythingEnum#nextPosition()} + * Number of simple operations in {@link Lucene101PostingsReader.EverythingEnum#nextPosition()} * when no seek or buffer refill is done. */ private static final int TERM_OPS_PER_POS = 7; diff --git a/lucene/suggest/src/java/module-info.java b/lucene/suggest/src/java/module-info.java index c3f5b7c178f..20542ac5281 100644 --- a/lucene/suggest/src/java/module-info.java +++ b/lucene/suggest/src/java/module-info.java @@ -32,7 +32,8 @@ module org.apache.lucene.suggest { org.apache.lucene.search.suggest.document.Completion84PostingsFormat, org.apache.lucene.search.suggest.document.Completion90PostingsFormat, org.apache.lucene.search.suggest.document.Completion99PostingsFormat, - org.apache.lucene.search.suggest.document.Completion912PostingsFormat; + org.apache.lucene.search.suggest.document.Completion912PostingsFormat, + org.apache.lucene.search.suggest.document.Completion101PostingsFormat; provides org.apache.lucene.analysis.TokenFilterFactory with org.apache.lucene.search.suggest.analyzing.SuggestStopFilterFactory; } diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion101PostingsFormat.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion101PostingsFormat.java new file mode 100644 index 00000000000..2a51f01fd7a --- /dev/null +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion101PostingsFormat.java @@ -0,0 +1,45 @@ +/* + * 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.search.suggest.document; + +import org.apache.lucene.codecs.PostingsFormat; + +/** + * {@link CompletionPostingsFormat} for {@link + * org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat} + * + * @lucene.experimental + */ +public class Completion101PostingsFormat extends CompletionPostingsFormat { + /** Creates a {@link Completion101PostingsFormat} that will load the completion FST on-heap. */ + public Completion101PostingsFormat() { + this(FSTLoadMode.ON_HEAP); + } + + /** + * Creates a {@link Completion101PostingsFormat} that will use the provided fstLoadMode + * to determine if the completion FST should be loaded on or off heap. + */ + public Completion101PostingsFormat(FSTLoadMode fstLoadMode) { + super("Completion101", fstLoadMode); + } + + @Override + protected PostingsFormat delegatePostingsFormat() { + return PostingsFormat.forName("Lucene101"); + } +} diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion912PostingsFormat.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion912PostingsFormat.java index 341b034e8e8..77a62443f31 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion912PostingsFormat.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion912PostingsFormat.java @@ -19,8 +19,10 @@ package org.apache.lucene.search.suggest.document; import org.apache.lucene.codecs.PostingsFormat; /** - * {@link CompletionPostingsFormat} for {@link - * org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat} + * {@link org.apache.lucene.search.suggest.document.CompletionPostingsFormat} for {@code + * org.apache.lucene.backward_codecs.lucene912.Lucene912PostingsFormat}. This format is only used + * for backward-compatibility of the index format and cannot be used to write data, use {@link + * Completion101PostingsFormat} on new indices. * * @lucene.experimental */ diff --git a/lucene/suggest/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/suggest/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat index 8544a97b88f..5c0365616cd 100644 --- a/lucene/suggest/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat +++ b/lucene/suggest/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat @@ -35,3 +35,4 @@ org.apache.lucene.search.suggest.document.Completion84PostingsFormat org.apache.lucene.search.suggest.document.Completion90PostingsFormat org.apache.lucene.search.suggest.document.Completion99PostingsFormat org.apache.lucene.search.suggest.document.Completion912PostingsFormat +org.apache.lucene.search.suggest.document.Completion101PostingsFormat diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java index 410c6e76b4e..4a59d09d35c 100644 --- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java +++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java @@ -951,7 +951,7 @@ public class TestSuggestField extends LuceneTestCase { new FilterCodec(TestUtil.getDefaultCodec().getName(), TestUtil.getDefaultCodec()) { final CompletionPostingsFormat.FSTLoadMode fstLoadMode = RandomPicks.randomFrom(random(), CompletionPostingsFormat.FSTLoadMode.values()); - final PostingsFormat postingsFormat = new Completion912PostingsFormat(fstLoadMode); + final PostingsFormat postingsFormat = new Completion101PostingsFormat(fstLoadMode); @Override public PostingsFormat postingsFormat() { diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneFixedGap.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneFixedGap.java index 9688c6c0e35..7c5a0bcf254 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneFixedGap.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneFixedGap.java @@ -28,9 +28,9 @@ import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader; import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter; import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase; import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsReader; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -38,7 +38,7 @@ import org.apache.lucene.index.SegmentWriteState; // any PostingsFormat and make it ord-able... /** - * Customized version of {@link Lucene912PostingsFormat} that uses {@link FixedGapTermsIndexWriter}. + * Customized version of {@link Lucene101PostingsFormat} that uses {@link FixedGapTermsIndexWriter}. */ public final class LuceneFixedGap extends PostingsFormat { final int termIndexInterval; @@ -54,7 +54,7 @@ public final class LuceneFixedGap extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase docs = new Lucene912PostingsWriter(state); + PostingsWriterBase docs = new Lucene101PostingsWriter(state); // TODO: should we make the terms index more easily // pluggable? Ie so that this codec would record which @@ -91,7 +91,7 @@ public final class LuceneFixedGap extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene912PostingsReader(state); + PostingsReaderBase postings = new Lucene101PostingsReader(state); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapDocFreqInterval.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapDocFreqInterval.java index 1fafc8f448c..ff5e0fc07dd 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapDocFreqInterval.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapDocFreqInterval.java @@ -29,9 +29,9 @@ import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase; import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsReader; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -39,7 +39,7 @@ import org.apache.lucene.index.SegmentWriteState; // any PostingsFormat and make it ord-able... /** - * Customized version of {@link Lucene912PostingsFormat} that uses {@link + * Customized version of {@link Lucene101PostingsFormat} that uses {@link * VariableGapTermsIndexWriter} with a fixed interval, but forcing high docfreq terms to be indexed * terms. */ @@ -59,7 +59,7 @@ public final class LuceneVarGapDocFreqInterval extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase docs = new Lucene912PostingsWriter(state); + PostingsWriterBase docs = new Lucene101PostingsWriter(state); // TODO: should we make the terms index more easily // pluggable? Ie so that this codec would record which @@ -100,7 +100,7 @@ public final class LuceneVarGapDocFreqInterval extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene912PostingsReader(state); + PostingsReaderBase postings = new Lucene101PostingsReader(state); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapFixedInterval.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapFixedInterval.java index bfaf0914651..7899f3d5436 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapFixedInterval.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapFixedInterval.java @@ -29,9 +29,9 @@ import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase; import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsReader; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -39,7 +39,7 @@ import org.apache.lucene.index.SegmentWriteState; // any PostingsFormat and make it ord-able... /** - * Customized version of {@link Lucene912PostingsFormat} that uses {@link + * Customized version of {@link Lucene101PostingsFormat} that uses {@link * VariableGapTermsIndexWriter} with a fixed interval. */ public final class LuceneVarGapFixedInterval extends PostingsFormat { @@ -56,7 +56,7 @@ public final class LuceneVarGapFixedInterval extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase docs = new Lucene912PostingsWriter(state); + PostingsWriterBase docs = new Lucene101PostingsWriter(state); // TODO: should we make the terms index more easily // pluggable? Ie so that this codec would record which @@ -95,7 +95,7 @@ public final class LuceneVarGapFixedInterval extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene912PostingsReader(state); + PostingsReaderBase postings = new Lucene101PostingsReader(state); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/mockrandom/MockRandomPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/mockrandom/MockRandomPostingsFormat.java index 6b10e7b19f4..8770e0d81fa 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/mockrandom/MockRandomPostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/mockrandom/MockRandomPostingsFormat.java @@ -35,10 +35,10 @@ import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter; import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsReader; import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsWriter; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsReader; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsWriter; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsReader; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.codecs.memory.FSTTermsReader; import org.apache.lucene.codecs.memory.FSTTermsWriter; import org.apache.lucene.index.FieldInfo; @@ -121,7 +121,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat { random.nextInt(); // consume a random for buffersize - PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene101PostingsWriter(state); final FieldsConsumer fields; final int t1 = random.nextInt(4); @@ -289,7 +289,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat { System.out.println("MockRandomCodec: readBufferSize=" + readBufferSize); } - PostingsReaderBase postingsReader = new Lucene912PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene101PostingsReader(state); final FieldsProducer fields; final int t1 = random.nextInt(4); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java index 4b13531327b..1961ecc746e 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java @@ -23,8 +23,8 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsReader; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsWriter; import org.apache.lucene.codecs.uniformsplit.BlockDecoder; import org.apache.lucene.codecs.uniformsplit.BlockEncoder; import org.apache.lucene.codecs.uniformsplit.IndexDictionary; @@ -67,7 +67,7 @@ public class UniformSplitRot13PostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState segmentWriteState) throws IOException { - PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(segmentWriteState); + PostingsWriterBase postingsWriter = new Lucene101PostingsWriter(segmentWriteState); boolean success = false; try { FieldsConsumer fieldsConsumer = createFieldsConsumer(segmentWriteState, postingsWriter); @@ -145,7 +145,7 @@ public class UniformSplitRot13PostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState segmentReadState) throws IOException { - PostingsReaderBase postingsReader = new Lucene912PostingsReader(segmentReadState); + PostingsReaderBase postingsReader = new Lucene101PostingsReader(segmentReadState); boolean success = false; try { FieldsProducer fieldsProducer = createFieldsProducer(segmentReadState, postingsReader); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java index 44f28b817ad..1ace55a98a0 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java @@ -38,7 +38,7 @@ import java.util.TimeZone; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.lucene100.Lucene100Codec; +import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.codecs.simpletext.SimpleTextCodec; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.tests.codecs.asserting.AssertingCodec; @@ -190,7 +190,7 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule { codec = CompressingCodec.randomInstance(random); } else if ("Lucene100".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene100"))) { - codec = new Lucene100Codec(RandomPicks.randomFrom(random, Lucene100Codec.Mode.values())); + codec = new Lucene101Codec(RandomPicks.randomFrom(random, Lucene101Codec.Mode.values())); } else if (!"random".equals(TEST_CODEC)) { codec = Codec.forName(TEST_CODEC); } else if ("random".equals(TEST_POSTINGSFORMAT)) { diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java index 95f06ea5570..66ae7a03141 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java @@ -55,9 +55,9 @@ import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat; -import org.apache.lucene.codecs.lucene100.Lucene100Codec; +import org.apache.lucene.codecs.lucene101.Lucene101Codec; +import org.apache.lucene.codecs.lucene101.Lucene101PostingsFormat; import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; -import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; @@ -1315,7 +1315,7 @@ public final class TestUtil { * different from {@link Codec#getDefault()} because that is randomized. */ public static Codec getDefaultCodec() { - return new Lucene100Codec(); + return new Lucene101Codec(); } /** @@ -1323,7 +1323,7 @@ public final class TestUtil { * Lucene. */ public static PostingsFormat getDefaultPostingsFormat() { - return new Lucene912PostingsFormat(); + return new Lucene101PostingsFormat(); } /** @@ -1334,7 +1334,7 @@ public final class TestUtil { */ public static PostingsFormat getDefaultPostingsFormat( int minItemsPerBlock, int maxItemsPerBlock) { - return new Lucene912PostingsFormat(minItemsPerBlock, maxItemsPerBlock); + return new Lucene101PostingsFormat(minItemsPerBlock, maxItemsPerBlock); } /** Returns a random postings format that supports term ordinals */ From 13285279c2d193fe6ad3f323046dd53bbdc8dd4a Mon Sep 17 00:00:00 2001 From: Dzung Bui Date: Sat, 2 Nov 2024 07:14:17 +0900 Subject: [PATCH 25/37] Rename NodeHash to FSTSuffixNodeCache (#13259) --- .../apache/lucene/util/fst/FSTCompiler.java | 12 +++++----- ...{NodeHash.java => FSTSuffixNodeCache.java} | 22 ++++++++++++++++--- ...eHash.java => TestFSTSuffixNodeCache.java} | 10 +++++---- 3 files changed, 31 insertions(+), 13 deletions(-) rename lucene/core/src/java/org/apache/lucene/util/fst/{NodeHash.java => FSTSuffixNodeCache.java} (93%) rename lucene/core/src/test/org/apache/lucene/util/fst/{TestNodeHash.java => TestFSTSuffixNodeCache.java} (84%) diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java b/lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java index d51c256d48e..b83024930ee 100644 --- a/lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java +++ b/lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java @@ -98,8 +98,8 @@ public class FSTCompiler { // it will throw exceptions if attempt to call getReverseBytesReader() or writeTo(DataOutput) private static final FSTReader NULL_FST_READER = new NullFSTReader(); - private final NodeHash dedupHash; - // a temporary FST used during building for NodeHash cache + private final FSTSuffixNodeCache suffixDedupCache; + // a temporary FST used during building for FSTSuffixNodeCache cache final FST fst; private final T NO_OUTPUT; @@ -178,9 +178,9 @@ public class FSTCompiler { if (suffixRAMLimitMB < 0) { throw new IllegalArgumentException("ramLimitMB must be >= 0; got: " + suffixRAMLimitMB); } else if (suffixRAMLimitMB > 0) { - dedupHash = new NodeHash<>(this, suffixRAMLimitMB); + suffixDedupCache = new FSTSuffixNodeCache<>(this, suffixRAMLimitMB); } else { - dedupHash = null; + suffixDedupCache = null; } NO_OUTPUT = outputs.getNoOutput(); @@ -379,12 +379,12 @@ public class FSTCompiler { private CompiledNode compileNode(UnCompiledNode nodeIn) throws IOException { final long node; long bytesPosStart = numBytesWritten; - if (dedupHash != null) { + if (suffixDedupCache != null) { if (nodeIn.numArcs == 0) { node = addNode(nodeIn); lastFrozenNode = node; } else { - node = dedupHash.add(nodeIn); + node = suffixDedupCache.add(nodeIn); } } else { node = addNode(nodeIn); diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java b/lucene/core/src/java/org/apache/lucene/util/fst/FSTSuffixNodeCache.java similarity index 93% rename from lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java rename to lucene/core/src/java/org/apache/lucene/util/fst/FSTSuffixNodeCache.java index 7326fd77f73..f33f09e9072 100644 --- a/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java +++ b/lucene/core/src/java/org/apache/lucene/util/fst/FSTSuffixNodeCache.java @@ -31,8 +31,24 @@ import org.apache.lucene.util.packed.PagedGrowableWriter; // TODO: couldn't we prune naturally back until we see a transition with an output? it's highly // unlikely (mostly impossible) such suffixes can be shared? -// Used to dedup states (lookup already-frozen states) -final class NodeHash { +/** + * This is essentially a LRU cache to maintain and lookup node suffix. Un-compiled node can be added + * into the cache and if a similar node exists we will return its address in the FST. A node is + * defined as similar if it has the same label, arcs, outputs & other properties that identify a + * node. + * + *

    The total size of the cache is controlled through the constructor parameter ramLimitMB + * Implementation-wise, we maintain two lookup tables, a primary table where node can be + * looked up from, and a fallback lookup table in case the lookup in the primary table fails. Nodes + * from the fallback table can also be promoted to the primary table when that happens. When the + * primary table is full, we swap it with the fallback table and clear out the primary table. + * + *

    To lookup the node address, we build a special hash table which maps from the Node hash value + * to the Node address in the FST, called PagedGrowableHash. Internally it uses {@link + * PagedGrowableWriter} to store the mapping, which allows efficient packing the hash & address long + * values, and uses {@link ByteBlockPool} to store the actual node content (arcs & outputs). + */ +final class FSTSuffixNodeCache { // primary table -- we add nodes into this until it reaches the requested tableSizeLimit/2, then // we move it to fallback @@ -60,7 +76,7 @@ final class NodeHash { * recently used suffixes are discarded, and the FST is no longer minimalI. Still, larger * ramLimitMB will make the FST smaller (closer to minimal). */ - public NodeHash(FSTCompiler fstCompiler, double ramLimitMB) { + public FSTSuffixNodeCache(FSTCompiler fstCompiler, double ramLimitMB) { if (ramLimitMB <= 0) { throw new IllegalArgumentException("ramLimitMB must be > 0; got: " + ramLimitMB); } diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/TestNodeHash.java b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTSuffixNodeCache.java similarity index 84% rename from lucene/core/src/test/org/apache/lucene/util/fst/TestNodeHash.java rename to lucene/core/src/test/org/apache/lucene/util/fst/TestFSTSuffixNodeCache.java index 8319f20efea..2fb93c7f5a4 100644 --- a/lucene/core/src/test/org/apache/lucene/util/fst/TestNodeHash.java +++ b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTSuffixNodeCache.java @@ -19,14 +19,16 @@ package org.apache.lucene.util.fst; import com.carrotsearch.randomizedtesting.generators.RandomBytes; import org.apache.lucene.tests.util.LuceneTestCase; -public class TestNodeHash extends LuceneTestCase { +public class TestFSTSuffixNodeCache extends LuceneTestCase { public void testCopyFallbackNodeBytes() { // we don't need the FSTCompiler in this test - NodeHash nodeHash = new NodeHash<>(null, 1); + FSTSuffixNodeCache suffixCache = new FSTSuffixNodeCache<>(null, 1); - NodeHash.PagedGrowableHash primaryHashTable = nodeHash.new PagedGrowableHash(); - NodeHash.PagedGrowableHash fallbackHashTable = nodeHash.new PagedGrowableHash(); + FSTSuffixNodeCache.PagedGrowableHash primaryHashTable = + suffixCache.new PagedGrowableHash(); + FSTSuffixNodeCache.PagedGrowableHash fallbackHashTable = + suffixCache.new PagedGrowableHash(); int nodeLength = atLeast(500); long fallbackHashSlot = 1; byte[] fallbackBytes = RandomBytes.randomBytesOfLength(random(), nodeLength); From a3d56ea2eea9ecad21e5f0a7b3ab22ff865185a2 Mon Sep 17 00:00:00 2001 From: Greg Miller Date: Tue, 5 Nov 2024 01:09:27 +0100 Subject: [PATCH 26/37] A few small tweaks to VectorUtil#findNextGEQ: (#13972) 1. Rearrange/rename the parameters to be more idiomatic (e.g., follow conventions of Arrays#... methods) 2. Add assert to ensure expected sortedness we may rely on in the future (so we're not trappy) 3. Migrate PostingsReader to call VectorUtil instead of VectorUtilSupport (so it benefits from the common assert) --- .../lucene912/Lucene912PostingsReader.java | 14 +++++++------- .../lucene/benchmark/jmh/AdvanceBenchmark.java | 4 ++-- .../codecs/lucene101/Lucene101PostingsReader.java | 12 +++++------- .../vectorization/DefaultVectorUtilSupport.java | 6 +++--- .../internal/vectorization/VectorUtilSupport.java | 10 +++++----- .../java/org/apache/lucene/util/VectorUtil.java | 14 ++++++++------ .../vectorization/PanamaVectorUtilSupport.java | 8 ++++---- .../org/apache/lucene/util/TestVectorUtil.java | 2 +- 8 files changed, 35 insertions(+), 35 deletions(-) diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsReader.java index f15fe851006..a51c848c4cc 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsReader.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene912/Lucene912PostingsReader.java @@ -593,7 +593,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } } - int next = findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); this.doc = (int) docBuffer[next]; docBufferUpto = next + 1; return doc; @@ -937,7 +937,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { refillDocs(); } - int next = findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); posPendingCount += sumOverRange(freqBuffer, docBufferUpto, next + 1); this.freq = (int) freqBuffer[next]; this.docBufferUpto = next + 1; @@ -1423,7 +1423,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { needsRefilling = false; } - int next = findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); this.doc = (int) docBuffer[next]; docBufferUpto = next + 1; return doc; @@ -1654,7 +1654,7 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { needsRefilling = false; } - int next = findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); posPendingCount += sumOverRange(freqBuffer, docBufferUpto, next + 1); freq = (int) freqBuffer[next]; docBufferUpto = next + 1; @@ -1755,13 +1755,13 @@ public final class Lucene912PostingsReader extends PostingsReaderBase { } } - private static int findNextGEQ(long[] buffer, int length, long target, int from) { - for (int i = from; i < length; ++i) { + private static int findNextGEQ(long[] buffer, long target, int from, int to) { + for (int i = from; i < to; ++i) { if (buffer[i] >= target) { return i; } } - return length; + return to; } private static void prefetchPostings(IndexInput docIn, IntBlockTermState state) diff --git a/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/AdvanceBenchmark.java b/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/AdvanceBenchmark.java index a47242c6e57..784ace1ae32 100644 --- a/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/AdvanceBenchmark.java +++ b/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/AdvanceBenchmark.java @@ -142,13 +142,13 @@ public class AdvanceBenchmark { @Benchmark public void vectorUtilSearch() { for (int i = 0; i < startIndexes.length; ++i) { - VectorUtil.findNextGEQ(values, 128, targets[i], startIndexes[i]); + VectorUtil.findNextGEQ(values, targets[i], startIndexes[i], 128); } } @CompilerControl(CompilerControl.Mode.DONT_INLINE) private static int vectorUtilSearch(int[] values, int target, int startIndex) { - return VectorUtil.findNextGEQ(values, 128, target, startIndex); + return VectorUtil.findNextGEQ(values, target, startIndex, 128); } private static void assertEquals(int expected, int actual) { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsReader.java index 4ae03dcdd9d..ce32e853424 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsReader.java @@ -46,7 +46,6 @@ import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SlowImpactsEnum; import org.apache.lucene.internal.vectorization.PostingDecodingUtil; -import org.apache.lucene.internal.vectorization.VectorUtilSupport; import org.apache.lucene.internal.vectorization.VectorizationProvider; import org.apache.lucene.store.ByteArrayDataInput; import org.apache.lucene.store.ChecksumIndexInput; @@ -57,6 +56,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.VectorUtil; /** * Concrete class that reads docId(maybe frq,pos,offset,payloads) list with postings format. @@ -66,8 +66,6 @@ import org.apache.lucene.util.IOUtils; public final class Lucene101PostingsReader extends PostingsReaderBase { static final VectorizationProvider VECTORIZATION_PROVIDER = VectorizationProvider.getInstance(); - private static final VectorUtilSupport VECTOR_SUPPORT = - VECTORIZATION_PROVIDER.getVectorUtilSupport(); // Dummy impacts, composed of the maximum possible term frequency and the lowest possible // (unsigned) norm value. This is typically used on tail blocks, which don't actually record // impacts as the storage overhead would not be worth any query evaluation speedup, since there's @@ -601,7 +599,7 @@ public final class Lucene101PostingsReader extends PostingsReaderBase { } } - int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = VectorUtil.findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); this.doc = docBuffer[next]; docBufferUpto = next + 1; return doc; @@ -950,7 +948,7 @@ public final class Lucene101PostingsReader extends PostingsReaderBase { refillDocs(); } - int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = VectorUtil.findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); posPendingCount += sumOverRange(freqBuffer, docBufferUpto, next + 1); this.freq = freqBuffer[next]; this.docBufferUpto = next + 1; @@ -1437,7 +1435,7 @@ public final class Lucene101PostingsReader extends PostingsReaderBase { needsRefilling = false; } - int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = VectorUtil.findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); this.doc = docBuffer[next]; docBufferUpto = next + 1; return doc; @@ -1670,7 +1668,7 @@ public final class Lucene101PostingsReader extends PostingsReaderBase { needsRefilling = false; } - int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = VectorUtil.findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); posPendingCount += sumOverRange(freqBuffer, docBufferUpto, next + 1); freq = freqBuffer[next]; docBufferUpto = next + 1; diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java index b3e1bed4fd0..184403cf48b 100644 --- a/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java @@ -199,12 +199,12 @@ final class DefaultVectorUtilSupport implements VectorUtilSupport { } @Override - public int findNextGEQ(int[] buffer, int length, int target, int from) { - for (int i = from; i < length; ++i) { + public int findNextGEQ(int[] buffer, int target, int from, int to) { + for (int i = from; i < to; ++i) { if (buffer[i] >= target) { return i; } } - return length; + return to; } } diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorUtilSupport.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorUtilSupport.java index 04bd7df52a3..fb94b0e3173 100644 --- a/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorUtilSupport.java +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorUtilSupport.java @@ -46,10 +46,10 @@ public interface VectorUtilSupport { int squareDistance(byte[] a, byte[] b); /** - * Given an array {@code buffer} that is sorted between indexes {@code 0} inclusive and {@code - * length} exclusive, find the first array index whose value is greater than or equal to {@code - * target}. This index is guaranteed to be at least {@code from}. If there is no such array index, - * {@code length} is returned. + * Given an array {@code buffer} that is sorted between indexes {@code 0} inclusive and {@code to} + * exclusive, find the first array index whose value is greater than or equal to {@code target}. + * This index is guaranteed to be at least {@code from}. If there is no such array index, {@code + * to} is returned. */ - int findNextGEQ(int[] buffer, int length, int target, int from); + int findNextGEQ(int[] buffer, int target, int from, int to); } diff --git a/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java index 7d83ca20c00..250c6544870 100644 --- a/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java +++ b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java @@ -17,6 +17,7 @@ package org.apache.lucene.util; +import java.util.stream.IntStream; import org.apache.lucene.internal.vectorization.VectorUtilSupport; import org.apache.lucene.internal.vectorization.VectorizationProvider; @@ -309,12 +310,13 @@ public final class VectorUtil { } /** - * Given an array {@code buffer} that is sorted between indexes {@code 0} inclusive and {@code - * length} exclusive, find the first array index whose value is greater than or equal to {@code - * target}. This index is guaranteed to be at least {@code from}. If there is no such array index, - * {@code length} is returned. + * Given an array {@code buffer} that is sorted between indexes {@code 0} inclusive and {@code to} + * exclusive, find the first array index whose value is greater than or equal to {@code target}. + * This index is guaranteed to be at least {@code from}. If there is no such array index, {@code + * to} is returned. */ - public static int findNextGEQ(int[] buffer, int length, int target, int from) { - return IMPL.findNextGEQ(buffer, length, target, from); + public static int findNextGEQ(int[] buffer, int target, int from, int to) { + assert IntStream.range(0, to - 1).noneMatch(i -> buffer[i] > buffer[i + 1]); + return IMPL.findNextGEQ(buffer, target, from, to); } } diff --git a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java index 68bac1b0b95..9273f7c5a81 100644 --- a/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java +++ b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/PanamaVectorUtilSupport.java @@ -769,9 +769,9 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { private static final boolean ENABLE_FIND_NEXT_GEQ_VECTOR_OPTO = INT_SPECIES.length() >= 8; @Override - public int findNextGEQ(int[] buffer, int length, int target, int from) { + public int findNextGEQ(int[] buffer, int target, int from, int to) { if (ENABLE_FIND_NEXT_GEQ_VECTOR_OPTO) { - for (; from + INT_SPECIES.length() < length; from += INT_SPECIES.length() + 1) { + for (; from + INT_SPECIES.length() < to; from += INT_SPECIES.length() + 1) { if (buffer[from + INT_SPECIES.length()] >= target) { IntVector vector = IntVector.fromArray(INT_SPECIES, buffer, from); VectorMask mask = vector.compare(VectorOperators.LT, target); @@ -779,11 +779,11 @@ final class PanamaVectorUtilSupport implements VectorUtilSupport { } } } - for (int i = from; i < length; ++i) { + for (int i = from; i < to; ++i) { if (buffer[i] >= target) { return i; } } - return length; + return to; } } diff --git a/lucene/core/src/test/org/apache/lucene/util/TestVectorUtil.java b/lucene/core/src/test/org/apache/lucene/util/TestVectorUtil.java index 05450861024..6e449a55002 100644 --- a/lucene/core/src/test/org/apache/lucene/util/TestVectorUtil.java +++ b/lucene/core/src/test/org/apache/lucene/util/TestVectorUtil.java @@ -372,7 +372,7 @@ public class TestVectorUtil extends LuceneTestCase { - 5; assertEquals( slowFindNextGEQ(values, 128, target, from), - VectorUtil.findNextGEQ(values, 128, target, from)); + VectorUtil.findNextGEQ(values, target, from, 128)); } } From e37eaea11a523fd31ed6fa320b8b3479c18a7841 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 5 Nov 2024 12:00:45 +0100 Subject: [PATCH 27/37] Revert "Disjunction as CompetitiveIterator for numeric dynamic pruning (#13221)" (#13857) (#13971) This reverts commit 1ee4f8a1115d1de623f242014681032d87ed2c1e. We have observed performance regressions that can be linked to #13221. We will need to revise the logic that such change introduced in main and branch_10x. While we do so, I propose that we bake it out of branch_10_0 and we release Lucene 10 without it. Closes #13856 --- .../search/comparators/NumericComparator.java | 460 ++++-------------- .../apache/lucene/util/IntArrayDocIdSet.java | 27 +- .../lucene/search/IntArrayDocIdSet.java | 88 ++++ .../apache/lucene/search/TestBooleanOr.java | 1 - 4 files changed, 195 insertions(+), 381 deletions(-) create mode 100644 lucene/core/src/test/org/apache/lucene/search/IntArrayDocIdSet.java diff --git a/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java b/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java index f967e0cfd88..31022f6bb05 100644 --- a/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java +++ b/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java @@ -18,9 +18,6 @@ package org.apache.lucene.search.comparators; import java.io.IOException; -import java.util.ArrayDeque; -import java.util.Deque; -import java.util.function.Consumer; import org.apache.lucene.index.DocValues; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.LeafReaderContext; @@ -32,12 +29,7 @@ import org.apache.lucene.search.LeafFieldComparator; import org.apache.lucene.search.Pruning; import org.apache.lucene.search.Scorable; import org.apache.lucene.search.Scorer; -import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.IntArrayDocIdSet; -import org.apache.lucene.util.IntsRef; -import org.apache.lucene.util.LSBRadixSorter; -import org.apache.lucene.util.PriorityQueue; -import org.apache.lucene.util.packed.PackedInts; +import org.apache.lucene.util.DocIdSetBuilder; /** * Abstract numeric comparator for comparing numeric values. This comparator provides a skipping @@ -50,6 +42,9 @@ import org.apache.lucene.util.packed.PackedInts; */ public abstract class NumericComparator extends FieldComparator { + // MIN_SKIP_INTERVAL and MAX_SKIP_INTERVAL both should be powers of 2 + private static final int MIN_SKIP_INTERVAL = 32; + private static final int MAX_SKIP_INTERVAL = 8192; protected final T missingValue; private final long missingValueAsLong; protected final String field; @@ -97,10 +92,10 @@ public abstract class NumericComparator extends FieldComparato /** Leaf comparator for {@link NumericComparator} that provides skipping functionality */ public abstract class NumericLeafComparator implements LeafFieldComparator { - private static final long MAX_DISJUNCTION_CLAUSE = 128; private final LeafReaderContext context; protected final NumericDocValues docValues; private final PointValues pointValues; + private final PointValues.PointTree pointTree; // if skipping functionality should be enabled on this segment private final boolean enableSkipping; private final int maxDoc; @@ -110,11 +105,14 @@ public abstract class NumericComparator extends FieldComparato private long minValueAsLong = Long.MIN_VALUE; private long maxValueAsLong = Long.MAX_VALUE; - private Long thresholdAsLong; private DocIdSetIterator competitiveIterator; - private long leadCost = -1; + private long iteratorCost = -1; private int maxDocVisited = -1; + private int updateCounter = 0; + private int currentSkipInterval = MIN_SKIP_INTERVAL; + // helps to be conservative about increasing the sampling interval + private int tryUpdateFailCount = 0; public NumericLeafComparator(LeafReaderContext context) throws IOException { this.context = context; @@ -141,6 +139,7 @@ public abstract class NumericComparator extends FieldComparato + " expected " + bytesCount); } + this.pointTree = pointValues.getPointTree(); this.enableSkipping = true; // skipping is enabled when points are available this.maxDoc = context.reader().maxDoc(); this.competitiveIterator = DocIdSetIterator.all(maxDoc); @@ -148,6 +147,7 @@ public abstract class NumericComparator extends FieldComparato encodeTop(); } } else { + this.pointTree = null; this.enableSkipping = false; this.maxDoc = 0; } @@ -183,12 +183,12 @@ public abstract class NumericComparator extends FieldComparato @Override public void setScorer(Scorable scorer) throws IOException { - if (leadCost == -1) { + if (iteratorCost == -1) { if (scorer instanceof Scorer) { - leadCost = + iteratorCost = ((Scorer) scorer).iterator().cost(); // starting iterator cost is the scorer's cost } else { - leadCost = maxDoc; + iteratorCost = maxDoc; } updateCompetitiveIterator(); // update an iterator when we have a new segment } @@ -207,91 +207,102 @@ public abstract class NumericComparator extends FieldComparato || hitsThresholdReached == false || (leafTopSet == false && queueFull == false)) return; // if some documents have missing points, check that missing values prohibits optimization - boolean dense = pointValues.getDocCount() == maxDoc; - if (dense == false && isMissingValueCompetitive()) { + if ((pointValues.getDocCount() < maxDoc) && isMissingValueCompetitive()) { return; // we can't filter out documents, as documents with missing values are competitive } - if (competitiveIterator instanceof CompetitiveIterator iter) { - if (queueFull) { - encodeBottom(); - } - // CompetitiveIterator already built, try to reduce clause. - tryReduceDisjunctionClause(iter); + updateCounter++; + // Start sampling if we get called too much + if (updateCounter > 256 + && (updateCounter & (currentSkipInterval - 1)) != currentSkipInterval - 1) { return; } - if (thresholdAsLong == null) { - if (dense == false) { + if (queueFull) { + encodeBottom(); + } + + DocIdSetBuilder result = new DocIdSetBuilder(maxDoc); + PointValues.IntersectVisitor visitor = + new PointValues.IntersectVisitor() { + DocIdSetBuilder.BulkAdder adder; + + @Override + public void grow(int count) { + adder = result.grow(count); + } + + @Override + public void visit(int docID) { + if (docID <= maxDocVisited) { + return; // Already visited or skipped + } + adder.add(docID); + } + + @Override + public void visit(int docID, byte[] packedValue) { + if (docID <= maxDocVisited) { + return; // already visited or skipped + } + long l = sortableBytesToLong(packedValue); + if (l >= minValueAsLong && l <= maxValueAsLong) { + adder.add(docID); // doc is competitive + } + } + + @Override + public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) { + long min = sortableBytesToLong(minPackedValue); + long max = sortableBytesToLong(maxPackedValue); + + if (min > maxValueAsLong || max < minValueAsLong) { + // 1. cmp ==0 and pruning==Pruning.GREATER_THAN_OR_EQUAL_TO : if the sort is + // ascending then maxValueAsLong is bottom's next less value, so it is competitive + // 2. cmp ==0 and pruning==Pruning.GREATER_THAN: maxValueAsLong equals to + // bottom, but there are multiple comparators, so it could be competitive + return PointValues.Relation.CELL_OUTSIDE_QUERY; + } + + if (min < minValueAsLong || max > maxValueAsLong) { + return PointValues.Relation.CELL_CROSSES_QUERY; + } + return PointValues.Relation.CELL_INSIDE_QUERY; + } + }; + + final long threshold = iteratorCost >>> 3; + + if (PointValues.isEstimatedPointCountGreaterThanOrEqualTo(visitor, pointTree, threshold)) { + // the new range is not selective enough to be worth materializing, it doesn't reduce number + // of docs at least 8x + updateSkipInterval(false); + if (pointValues.getDocCount() < iteratorCost) { + // Use the set of doc with values to help drive iteration competitiveIterator = getNumericDocValues(context, field); - leadCost = Math.min(leadCost, competitiveIterator.cost()); + iteratorCost = pointValues.getDocCount(); } - long threshold = Math.min(leadCost >> 3, maxDoc >> 5); - thresholdAsLong = intersectThresholdValue(threshold); + return; } + pointValues.intersect(visitor); + competitiveIterator = result.build().iterator(); + iteratorCost = competitiveIterator.cost(); + updateSkipInterval(true); + } - if ((reverse == false && bottomAsComparableLong() <= thresholdAsLong) - || (reverse && bottomAsComparableLong() >= thresholdAsLong)) { - if (queueFull) { - encodeBottom(); + private void updateSkipInterval(boolean success) { + if (updateCounter > 256) { + if (success) { + currentSkipInterval = Math.max(currentSkipInterval / 2, MIN_SKIP_INTERVAL); + tryUpdateFailCount = 0; + } else { + if (tryUpdateFailCount >= 3) { + currentSkipInterval = Math.min(currentSkipInterval * 2, MAX_SKIP_INTERVAL); + tryUpdateFailCount = 0; + } else { + tryUpdateFailCount++; + } } - DisjunctionBuildVisitor visitor = new DisjunctionBuildVisitor(); - competitiveIterator = visitor.generateCompetitiveIterator(); - } - } - - private void tryReduceDisjunctionClause(CompetitiveIterator iter) { - int originalSize = iter.disis.size(); - - while (iter.disis.isEmpty() == false - && (iter.disis.getFirst().mostCompetitiveValue > maxValueAsLong - || iter.disis.getFirst().mostCompetitiveValue < minValueAsLong)) { - iter.disis.removeFirst(); - } - - if (originalSize != iter.disis.size()) { - iter.disjunction.clear(); - iter.disjunction.addAll(iter.disis); - } - } - - /** Find out the value that threshold docs away from topValue/infinite. */ - private long intersectThresholdValue(long threshold) throws IOException { - long thresholdValuePos; - if (leafTopSet) { - long topValue = topAsComparableLong(); - PointValues.IntersectVisitor visitor = new RangeVisitor(Long.MIN_VALUE, topValue, -1); - long topValuePos = pointValues.estimatePointCount(visitor); - thresholdValuePos = reverse == false ? topValuePos + threshold : topValuePos - threshold; - } else { - thresholdValuePos = reverse == false ? threshold : pointValues.size() - threshold; - } - if (thresholdValuePos <= 0) { - return sortableBytesToLong(pointValues.getMinPackedValue()); - } else if (thresholdValuePos >= pointValues.size()) { - return sortableBytesToLong(pointValues.getMaxPackedValue()); - } else { - return intersectValueByPos(pointValues.getPointTree(), thresholdValuePos); - } - } - - /** Get the point value by a left-to-right position. */ - private long intersectValueByPos(PointValues.PointTree pointTree, long pos) throws IOException { - assert pos > 0 : pos; - while (pointTree.size() < pos) { - pos -= pointTree.size(); - pointTree.moveToSibling(); - } - if (pointTree.size() == pos) { - return sortableBytesToLong(pointTree.getMaxPackedValue()); - } else if (pos == 0) { - return sortableBytesToLong(pointTree.getMinPackedValue()); - } else if (pointTree.moveToChild()) { - return intersectValueByPos(pointTree, pos); - } else { - return reverse == false - ? sortableBytesToLong(pointTree.getMaxPackedValue()) - : sortableBytesToLong(pointTree.getMinPackedValue()); } } @@ -400,276 +411,5 @@ public abstract class NumericComparator extends FieldComparato protected abstract long bottomAsComparableLong(); protected abstract long topAsComparableLong(); - - class DisjunctionBuildVisitor extends RangeVisitor { - - final Deque disis = new ArrayDeque<>(); - // most competitive entry stored last. - final Consumer adder = - reverse == false ? disis::addFirst : disis::addLast; - - final int minBlockLength = minBlockLength(); - - final LSBRadixSorter sorter = new LSBRadixSorter(); - int[] docs = IntsRef.EMPTY_INTS; - int index = 0; - int blockMaxDoc = -1; - boolean docsInOrder = true; - long blockMinValue = Long.MAX_VALUE; - long blockMaxValue = Long.MIN_VALUE; - - private DisjunctionBuildVisitor() { - super(minValueAsLong, maxValueAsLong, maxDocVisited); - } - - @Override - public void grow(int count) { - docs = ArrayUtil.grow(docs, index + count + 1); - } - - @Override - protected void consumeDoc(int doc) { - docs[index++] = doc; - if (doc >= blockMaxDoc) { - blockMaxDoc = doc; - } else { - docsInOrder = false; - } - } - - void intersectLeaves(PointValues.PointTree pointTree) throws IOException { - PointValues.Relation r = - compare(pointTree.getMinPackedValue(), pointTree.getMaxPackedValue()); - switch (r) { - case CELL_INSIDE_QUERY, CELL_CROSSES_QUERY -> { - if (pointTree.moveToChild()) { - do { - intersectLeaves(pointTree); - } while (pointTree.moveToSibling()); - pointTree.moveToParent(); - } else { - if (r == PointValues.Relation.CELL_CROSSES_QUERY) { - pointTree.visitDocValues(this); - } else { - pointTree.visitDocIDs(this); - } - updateMinMax( - sortableBytesToLong(pointTree.getMinPackedValue()), - sortableBytesToLong(pointTree.getMaxPackedValue())); - } - } - case CELL_OUTSIDE_QUERY -> {} - default -> throw new IllegalStateException("unreachable code"); - } - } - - void updateMinMax(long leafMinValue, long leafMaxValue) throws IOException { - this.blockMinValue = Math.min(blockMinValue, leafMinValue); - this.blockMaxValue = Math.max(blockMaxValue, leafMaxValue); - if (index >= minBlockLength) { - update(); - this.blockMinValue = Long.MAX_VALUE; - this.blockMaxValue = Long.MIN_VALUE; - } - } - - void update() throws IOException { - if (blockMinValue > blockMaxValue) { - return; - } - long mostCompetitiveValue = - reverse == false - ? Math.max(blockMinValue, minValueAsLong) - : Math.min(blockMaxValue, maxValueAsLong); - - if (docsInOrder == false) { - sorter.sort(PackedInts.bitsRequired(blockMaxDoc), docs, index); - } - docs[index] = DocIdSetIterator.NO_MORE_DOCS; - DocIdSetIterator iter = new IntArrayDocIdSet(docs, index).iterator(); - adder.accept(new DisiAndMostCompetitiveValue(iter, mostCompetitiveValue)); - docs = IntsRef.EMPTY_INTS; - index = 0; - blockMaxDoc = -1; - docsInOrder = true; - } - - DocIdSetIterator generateCompetitiveIterator() throws IOException { - intersectLeaves(pointValues.getPointTree()); - update(); - - if (disis.isEmpty()) { - return DocIdSetIterator.empty(); - } - assert assertMostCompetitiveValuesSorted(disis); - - PriorityQueue disjunction = - new PriorityQueue<>(disis.size()) { - @Override - protected boolean lessThan( - DisiAndMostCompetitiveValue a, DisiAndMostCompetitiveValue b) { - return a.disi.docID() < b.disi.docID(); - } - }; - disjunction.addAll(disis); - - return new CompetitiveIterator(maxDoc, disis, disjunction); - } - - /** - * Used for assert. When reverse is false, smaller values are more competitive, so - * mostCompetitiveValues should be in desc order. - */ - private boolean assertMostCompetitiveValuesSorted(Deque deque) { - long lastValue = reverse == false ? Long.MAX_VALUE : Long.MIN_VALUE; - for (DisiAndMostCompetitiveValue value : deque) { - if (reverse == false) { - assert value.mostCompetitiveValue <= lastValue - : deque.stream().map(d -> d.mostCompetitiveValue).toList().toString(); - } else { - assert value.mostCompetitiveValue >= lastValue - : deque.stream().map(d -> d.mostCompetitiveValue).toList().toString(); - } - lastValue = value.mostCompetitiveValue; - } - return true; - } - - private int minBlockLength() { - // bottom value can be much more competitive than thresholdAsLong, recompute the cost. - long cost = - pointValues.estimatePointCount(new RangeVisitor(minValueAsLong, maxValueAsLong, -1)); - long disjunctionClause = Math.min(MAX_DISJUNCTION_CLAUSE, cost / 512 + 1); - return Math.toIntExact(cost / disjunctionClause); - } - } - } - - private class RangeVisitor implements PointValues.IntersectVisitor { - - private final long minInclusive; - private final long maxInclusive; - private final int docLowerBound; - - private RangeVisitor(long minInclusive, long maxInclusive, int docLowerBound) { - this.minInclusive = minInclusive; - this.maxInclusive = maxInclusive; - this.docLowerBound = docLowerBound; - } - - @Override - public void visit(int docID) throws IOException { - if (docID <= docLowerBound) { - return; // Already visited or skipped - } - consumeDoc(docID); - } - - @Override - public void visit(int docID, byte[] packedValue) throws IOException { - if (docID <= docLowerBound) { - return; // already visited or skipped - } - long l = sortableBytesToLong(packedValue); - if (l >= minInclusive && l <= maxInclusive) { - consumeDoc(docID); - } - } - - @Override - public void visit(DocIdSetIterator iterator, byte[] packedValue) throws IOException { - long l = sortableBytesToLong(packedValue); - if (l >= minInclusive && l <= maxInclusive) { - int doc = docLowerBound >= 0 ? iterator.advance(docLowerBound) : iterator.nextDoc(); - while (doc != DocIdSetIterator.NO_MORE_DOCS) { - consumeDoc(doc); - doc = iterator.nextDoc(); - } - } - } - - @Override - public void visit(DocIdSetIterator iterator) throws IOException { - int doc = docLowerBound >= 0 ? iterator.advance(docLowerBound) : iterator.nextDoc(); - while (doc != DocIdSetIterator.NO_MORE_DOCS) { - consumeDoc(doc); - doc = iterator.nextDoc(); - } - } - - @Override - public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) { - long min = sortableBytesToLong(minPackedValue); - long max = sortableBytesToLong(maxPackedValue); - - if (min > maxInclusive || max < minInclusive) { - // 1. cmp ==0 and pruning==Pruning.GREATER_THAN_OR_EQUAL_TO : if the sort is - // ascending then maxValueAsLong is bottom's next less value, so it is competitive - // 2. cmp ==0 and pruning==Pruning.GREATER_THAN: maxValueAsLong equals to - // bottom, but there are multiple comparators, so it could be competitive - return PointValues.Relation.CELL_OUTSIDE_QUERY; - } - - if (min < minInclusive || max > maxInclusive) { - return PointValues.Relation.CELL_CROSSES_QUERY; - } - return PointValues.Relation.CELL_INSIDE_QUERY; - } - - void consumeDoc(int doc) { - throw new UnsupportedOperationException(); - } - } - - private record DisiAndMostCompetitiveValue(DocIdSetIterator disi, long mostCompetitiveValue) {} - - private static class CompetitiveIterator extends DocIdSetIterator { - - private final int maxDoc; - private int doc = -1; - private final Deque disis; - private final PriorityQueue disjunction; - - CompetitiveIterator( - int maxDoc, - Deque disis, - PriorityQueue disjunction) { - this.maxDoc = maxDoc; - this.disis = disis; - this.disjunction = disjunction; - } - - @Override - public int docID() { - return doc; - } - - @Override - public int nextDoc() throws IOException { - return advance(docID() + 1); - } - - @Override - public int advance(int target) throws IOException { - if (target >= maxDoc) { - return doc = NO_MORE_DOCS; - } else { - DisiAndMostCompetitiveValue top = disjunction.top(); - if (top == null) { - // priority queue is empty, none of the remaining documents are competitive - return doc = NO_MORE_DOCS; - } - while (top.disi.docID() < target) { - top.disi.advance(target); - top = disjunction.updateTop(); - } - return doc = top.disi.docID(); - } - } - - @Override - public long cost() { - return maxDoc; - } } } diff --git a/lucene/core/src/java/org/apache/lucene/util/IntArrayDocIdSet.java b/lucene/core/src/java/org/apache/lucene/util/IntArrayDocIdSet.java index eb4b93f499e..d44cc783923 100644 --- a/lucene/core/src/java/org/apache/lucene/util/IntArrayDocIdSet.java +++ b/lucene/core/src/java/org/apache/lucene/util/IntArrayDocIdSet.java @@ -21,12 +21,7 @@ import java.util.Arrays; import org.apache.lucene.search.DocIdSet; import org.apache.lucene.search.DocIdSetIterator; -/** - * A doc id set based on sorted int array. - * - * @lucene.internal - */ -public final class IntArrayDocIdSet extends DocIdSet { +final class IntArrayDocIdSet extends DocIdSet { private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(IntArrayDocIdSet.class); @@ -34,23 +29,15 @@ public final class IntArrayDocIdSet extends DocIdSet { private final int[] docs; private final int length; - /** - * Build an IntArrayDocIdSet by an int array and len. - * - * @param docs A docs array whose length need to be greater than the param len. It needs to be - * sorted from 0(inclusive) to the len(exclusive), and the len-th doc in docs need to be - * {@link DocIdSetIterator#NO_MORE_DOCS}. - * @param len The valid docs length in array. - */ - public IntArrayDocIdSet(int[] docs, int len) { - if (docs[len] != DocIdSetIterator.NO_MORE_DOCS) { + IntArrayDocIdSet(int[] docs, int length) { + if (docs[length] != DocIdSetIterator.NO_MORE_DOCS) { throw new IllegalArgumentException(); } - assert assertArraySorted(docs, len) - : "IntArrayDocIdSet need docs to be sorted" - + Arrays.toString(ArrayUtil.copyOfSubArray(docs, 0, len)); this.docs = docs; - this.length = len; + assert assertArraySorted(docs, length) + : "IntArrayDocIdSet need docs to be sorted" + + Arrays.toString(ArrayUtil.copyOfSubArray(docs, 0, length)); + this.length = length; } private static boolean assertArraySorted(int[] docs, int length) { diff --git a/lucene/core/src/test/org/apache/lucene/search/IntArrayDocIdSet.java b/lucene/core/src/test/org/apache/lucene/search/IntArrayDocIdSet.java new file mode 100644 index 00000000000..fa4ddece1e4 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/search/IntArrayDocIdSet.java @@ -0,0 +1,88 @@ +/* + * 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.search; + +import java.io.IOException; +import java.util.Arrays; +import org.apache.lucene.util.ArrayUtil; + +class IntArrayDocIdSet extends DocIdSet { + private final int[] docs; + private final int length; + + IntArrayDocIdSet(int[] docs, int length) { + if (docs[length] != DocIdSetIterator.NO_MORE_DOCS) { + throw new IllegalArgumentException(); + } + this.docs = docs; + assert assertArraySorted(docs, length) + : "IntArrayDocIdSet need docs to be sorted" + + Arrays.toString(ArrayUtil.copyOfSubArray(docs, 0, length)); + this.length = length; + } + + private static boolean assertArraySorted(int[] docs, int length) { + for (int i = 1; i < length; i++) { + if (docs[i] < docs[i - 1]) { + return false; + } + } + return true; + } + + @Override + public DocIdSetIterator iterator() throws IOException { + return new DocIdSetIterator() { + int i = 0; + int doc = -1; + + @Override + public int docID() { + return doc; + } + + @Override + public int nextDoc() { + return doc = docs[i++]; + } + + @Override + public int advance(int target) { + int bound = 1; + // given that we use this for small arrays only, this is very unlikely to overflow + while (i + bound < length && docs[i + bound] < target) { + bound *= 2; + } + i = Arrays.binarySearch(docs, i + bound / 2, Math.min(i + bound + 1, length), target); + if (i < 0) { + i = -1 - i; + } + return doc = docs[i++]; + } + + @Override + public long cost() { + return length; + } + }; + } + + @Override + public long ramBytesUsed() { + throw new UnsupportedOperationException(); + } +} diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java index 6fe375c086d..6e84993902d 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java @@ -34,7 +34,6 @@ import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.FixedBitSet; -import org.apache.lucene.util.IntArrayDocIdSet; public class TestBooleanOr extends LuceneTestCase { From 07955ff88d6ef900ee471426408ad02e42312309 Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Wed, 9 Oct 2024 12:33:34 +0200 Subject: [PATCH 28/37] Avoid performance regression by constructing lazily the PointTree in NumericComparator (#13498) (#13877) --- lucene/CHANGES.txt | 2 ++ .../search/comparators/NumericComparator.java | 15 +++++++++++---- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 2bfde84eabe..d547d7bdc65 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -333,6 +333,8 @@ Bug Fixes * GITHUB#12878: Fix the declared Exceptions of Expression#evaluate() to match those of DoubleValues#doubleValue(). (Uwe Schindler) +* GITHUB#13498: Avoid performance regression by constructing lazily the PointTree in NumericComparator, (Ignacio Vera) + Changes in Runtime Behavior --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java b/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java index 31022f6bb05..2bd594cc731 100644 --- a/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java +++ b/lucene/core/src/java/org/apache/lucene/search/comparators/NumericComparator.java @@ -95,7 +95,8 @@ public abstract class NumericComparator extends FieldComparato private final LeafReaderContext context; protected final NumericDocValues docValues; private final PointValues pointValues; - private final PointValues.PointTree pointTree; + // lazily constructed to avoid performance overhead when this is not used + private PointValues.PointTree pointTree; // if skipping functionality should be enabled on this segment private final boolean enableSkipping; private final int maxDoc; @@ -139,7 +140,6 @@ public abstract class NumericComparator extends FieldComparato + " expected " + bytesCount); } - this.pointTree = pointValues.getPointTree(); this.enableSkipping = true; // skipping is enabled when points are available this.maxDoc = context.reader().maxDoc(); this.competitiveIterator = DocIdSetIterator.all(maxDoc); @@ -147,7 +147,6 @@ public abstract class NumericComparator extends FieldComparato encodeTop(); } } else { - this.pointTree = null; this.enableSkipping = false; this.maxDoc = 0; } @@ -273,7 +272,8 @@ public abstract class NumericComparator extends FieldComparato final long threshold = iteratorCost >>> 3; - if (PointValues.isEstimatedPointCountGreaterThanOrEqualTo(visitor, pointTree, threshold)) { + if (PointValues.isEstimatedPointCountGreaterThanOrEqualTo( + visitor, getPointTree(), threshold)) { // the new range is not selective enough to be worth materializing, it doesn't reduce number // of docs at least 8x updateSkipInterval(false); @@ -290,6 +290,13 @@ public abstract class NumericComparator extends FieldComparato updateSkipInterval(true); } + private PointValues.PointTree getPointTree() throws IOException { + if (pointTree == null) { + pointTree = pointValues.getPointTree(); + } + return pointTree; + } + private void updateSkipInterval(boolean success) { if (updateCounter > 256) { if (success) { From bc18828fcccb8a941d2c84066654d14ff3a6cfd7 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 5 Nov 2024 15:36:40 +0100 Subject: [PATCH 29/37] Remove `supportsConcurrency` flag from top-docs collector managers. (#13977) Our collector managers have a `supportsConcurrency` flag to optimize the case when they are used in a single thread. This PR proposes to remove this flag now that the optimization doesn't do much as a result of #13943. --- .../benchmark/byTask/tasks/ReadTask.java | 3 +- .../apache/lucene/search/IndexSearcher.java | 9 +-- .../apache/lucene/search/SortRescorer.java | 4 +- .../search/TopFieldCollectorManager.java | 58 ++++++++----------- .../search/TopScoreDocCollectorManager.java | 47 ++++++--------- .../lucene/index/TestIndexWriterMaxDocs.java | 2 +- .../TestBooleanQueryVisitSubscorers.java | 2 +- .../search/TestElevationComparator.java | 2 +- .../apache/lucene/search/TestNeedsScores.java | 3 +- .../TestPositiveScoresOnlyCollector.java | 3 +- .../apache/lucene/search/TestSearchAfter.java | 10 ++-- .../lucene/search/TestSortOptimization.java | 22 +++---- .../lucene/search/TestTopDocsCollector.java | 6 +- .../lucene/search/TestTopDocsMerge.java | 5 +- .../lucene/search/TestTopFieldCollector.java | 17 +++--- .../apache/lucene/facet/DrillSideways.java | 6 +- .../lucene/facet/FacetsCollectorManager.java | 4 +- .../grouping/BlockGroupingCollector.java | 5 +- .../search/grouping/TopGroupsCollector.java | 4 +- .../lucene/search/join/TestJoinUtil.java | 4 +- .../lucene/luke/models/search/SearchImpl.java | 3 +- .../analyzing/AnalyzingInfixSuggester.java | 3 +- .../apache/lucene/tests/search/CheckHits.java | 6 +- 23 files changed, 91 insertions(+), 137 deletions(-) diff --git a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java index a9465e5d12f..9940196ce47 100644 --- a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java +++ b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ReadTask.java @@ -112,8 +112,7 @@ public abstract class ReadTask extends PerfTask { // pulling the Weight ourselves: int totalHitsThreshold = withTotalHits() ? Integer.MAX_VALUE : 1; TopFieldCollectorManager collectorManager = - new TopFieldCollectorManager( - sort, numHits, null, totalHitsThreshold, searcher.getSlices().length > 1); + new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold); hits = searcher.search(q, collectorManager); } else { hits = searcher.search(q, numHits); diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index 9c7274da3eb..44120064483 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -564,10 +564,8 @@ public class IndexSearcher { } final int cappedNumHits = Math.min(numHits, limit); - final boolean supportsConcurrency = getSlices().length > 1; CollectorManager manager = - new TopScoreDocCollectorManager( - cappedNumHits, after, TOTAL_HITS_THRESHOLD, supportsConcurrency); + new TopScoreDocCollectorManager(cappedNumHits, after, TOTAL_HITS_THRESHOLD); return search(query, manager); } @@ -699,12 +697,9 @@ public class IndexSearcher { } final int cappedNumHits = Math.min(numHits, limit); final Sort rewrittenSort = sort.rewrite(this); - final LeafSlice[] leafSlices = getSlices(); - final boolean supportsConcurrency = leafSlices.length > 1; final CollectorManager manager = - new TopFieldCollectorManager( - rewrittenSort, cappedNumHits, after, TOTAL_HITS_THRESHOLD, supportsConcurrency); + new TopFieldCollectorManager(rewrittenSort, cappedNumHits, after, TOTAL_HITS_THRESHOLD); TopFieldDocs topDocs = search(query, manager); if (doDocScores) { diff --git a/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java b/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java index 3139da98a84..e4a4891c570 100644 --- a/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java @@ -46,9 +46,7 @@ public class SortRescorer extends Rescorer { List leaves = searcher.getIndexReader().leaves(); TopFieldCollector collector = - new TopFieldCollectorManager( - sort, topN, null, Integer.MAX_VALUE, searcher.getSlices().length > 1) - .newCollector(); + new TopFieldCollectorManager(sort, topN, null, Integer.MAX_VALUE).newCollector(); // Now merge sort docIDs from hits, with reader's leaves: int hitUpto = 0; diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java index 48c1ca98091..1f37a82864e 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollectorManager.java @@ -35,8 +35,6 @@ public class TopFieldCollectorManager implements CollectorManager collectors; - private final boolean supportsConcurrency; - private boolean collectorCreated; /** * Creates a new {@link TopFieldCollectorManager} from the given arguments. @@ -53,9 +51,32 @@ public class TopFieldCollectorManager implements CollectorManagerNOTE: The instances returned by this method pre-allocate a full array of length + * numHits. + * + * @param sort the sort criteria (SortFields). + * @param numHits the number of results to collect. + * @param after the previous doc after which matching docs will be collected. + * @param totalHitsThreshold the number of docs to count accurately. If the query matches more + * than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other + * hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit + * count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit + * count accurate, but this will also make query processing slower. + */ + public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) { if (totalHitsThreshold < 0) { throw new IllegalArgumentException( "totalHitsThreshold must be >= 0, got " + totalHitsThreshold); @@ -88,35 +109,11 @@ public class TopFieldCollectorManager implements CollectorManager(); } - /** - * Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe - * internal states. - * - *

    NOTE: The instances returned by this method pre-allocate a full array of length - * numHits. - * - * @param sort the sort criteria (SortFields). - * @param numHits the number of results to collect. - * @param after the previous doc after which matching docs will be collected. - * @param totalHitsThreshold the number of docs to count accurately. If the query matches more - * than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other - * hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit - * count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit - * count accurate, but this will also make query processing slower. - */ - public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) { - this(sort, numHits, after, totalHitsThreshold, true); - } - /** * Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe * internal states. @@ -138,13 +135,6 @@ public class TopFieldCollectorManager implements CollectorManager queue = FieldValueHitQueue.create(sort.getSort(), numHits); diff --git a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java index 7933313febe..6a206088013 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollectorManager.java @@ -31,8 +31,6 @@ public class TopScoreDocCollectorManager private final ScoreDoc after; private final int totalHitsThreshold; private final MaxScoreAccumulator minScoreAcc; - private final boolean supportsConcurrency; - private boolean collectorCreated; /** * Creates a new {@link TopScoreDocCollectorManager} given the number of hits to collect and the @@ -54,28 +52,13 @@ public class TopScoreDocCollectorManager * hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit * count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit * count accurate, but this will also make query processing slower. - * @param supportsConcurrency to use thread-safe and slower internal states for count tracking. + * @deprecated Use {@link #TopScoreDocCollectorManager(int, ScoreDoc, int)}, the + * supportsConcurrency parameter is now a no-op. */ + @Deprecated public TopScoreDocCollectorManager( int numHits, ScoreDoc after, int totalHitsThreshold, boolean supportsConcurrency) { - if (totalHitsThreshold < 0) { - throw new IllegalArgumentException( - "totalHitsThreshold must be >= 0, got " + totalHitsThreshold); - } - - if (numHits <= 0) { - throw new IllegalArgumentException( - "numHits must be > 0; please use TotalHitCountCollectorManager if you just need the total hit count"); - } - - this.numHits = numHits; - this.after = after; - this.supportsConcurrency = supportsConcurrency; - this.totalHitsThreshold = Math.max(totalHitsThreshold, numHits); - this.minScoreAcc = - supportsConcurrency && totalHitsThreshold != Integer.MAX_VALUE - ? new MaxScoreAccumulator() - : null; + this(numHits, after, totalHitsThreshold); } /** @@ -100,7 +83,20 @@ public class TopScoreDocCollectorManager * count accurate, but this will also make query processing slower. */ public TopScoreDocCollectorManager(int numHits, ScoreDoc after, int totalHitsThreshold) { - this(numHits, after, totalHitsThreshold, true); + if (totalHitsThreshold < 0) { + throw new IllegalArgumentException( + "totalHitsThreshold must be >= 0, got " + totalHitsThreshold); + } + + if (numHits <= 0) { + throw new IllegalArgumentException( + "numHits must be > 0; please use TotalHitCountCollectorManager if you just need the total hit count"); + } + + this.numHits = numHits; + this.after = after; + this.totalHitsThreshold = Math.max(totalHitsThreshold, numHits); + this.minScoreAcc = totalHitsThreshold != Integer.MAX_VALUE ? new MaxScoreAccumulator() : null; } /** @@ -129,13 +125,6 @@ public class TopScoreDocCollectorManager @Override public TopScoreDocCollector newCollector() { - if (collectorCreated && supportsConcurrency == false) { - throw new IllegalStateException( - "This TopScoreDocCollectorManager was created without concurrency (supportsConcurrency=false), but multiple collectors are being created"); - } else { - collectorCreated = true; - } - if (after == null) { return new TopScoreDocCollector.SimpleTopScoreDocCollector( numHits, totalHitsThreshold, minScoreAcc); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java index 48aac56a22d..d9b8a055f6c 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java @@ -67,7 +67,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase { assertEquals(IndexWriter.MAX_DOCS, ir.numDocs()); IndexSearcher searcher = new IndexSearcher(ir); TopScoreDocCollectorManager collectorManager = - new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE, true); + new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE); TopDocs hits = searcher.search(new TermQuery(new Term("field", "text")), collectorManager); assertEquals(IndexWriter.MAX_DOCS, hits.totalHits.value()); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java index dc48dc1699d..9a2c177c62d 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java @@ -155,7 +155,7 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase { private final Set tqsSet = new HashSet<>(); MyCollector() { - super(new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE, false).newCollector()); + super(new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE).newCollector()); } @Override diff --git a/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java b/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java index 46ab7ad420b..dab1e165113 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestElevationComparator.java @@ -98,7 +98,7 @@ public class TestElevationComparator extends LuceneTestCase { TopDocs topDocs = searcher.search( - newq.build(), new TopFieldCollectorManager(sort, 50, null, Integer.MAX_VALUE, true)); + newq.build(), new TopFieldCollectorManager(sort, 50, null, Integer.MAX_VALUE)); int nDocsReturned = topDocs.scoreDocs.length; assertEquals(4, nDocsReturned); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java b/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java index 78bebc115ea..d58fa829ed6 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java @@ -81,8 +81,7 @@ public class TestNeedsScores extends LuceneTestCase { assertEquals(5, searcher.count(constantScore)); TopDocs hits = - searcher.search( - constantScore, new TopScoreDocCollectorManager(5, null, Integer.MAX_VALUE, true)); + searcher.search(constantScore, new TopScoreDocCollectorManager(5, null, Integer.MAX_VALUE)); assertEquals(5, hits.totalHits.value()); // Queries that support dynamic pruning like top-score or top-doc queries that do not compute diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java index 784cc885733..2a53f49759b 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java @@ -110,8 +110,7 @@ public class TestPositiveScoresOnlyCollector extends LuceneTestCase { writer.close(); Scorer s = new SimpleScorer(); TopDocsCollector tdc = - new TopScoreDocCollectorManager(scores.length, null, Integer.MAX_VALUE, false) - .newCollector(); + new TopScoreDocCollectorManager(scores.length, null, Integer.MAX_VALUE).newCollector(); Collector c = new PositiveScoresOnlyCollector(tdc); LeafCollector ac = c.getLeafCollector(ir.leaves().get(0)); ac.setScorer(s); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java b/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java index bf33e7732e0..ecc5727625c 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSearchAfter.java @@ -234,10 +234,10 @@ public class TestSearchAfter extends LuceneTestCase { allManager = new TopScoreDocCollectorManager(maxDoc, null, Integer.MAX_VALUE); doScores = false; } else if (sort == Sort.RELEVANCE) { - allManager = new TopFieldCollectorManager(sort, maxDoc, null, Integer.MAX_VALUE, true); + allManager = new TopFieldCollectorManager(sort, maxDoc, null, Integer.MAX_VALUE); doScores = true; } else { - allManager = new TopFieldCollectorManager(sort, maxDoc, null, Integer.MAX_VALUE, true); + allManager = new TopFieldCollectorManager(sort, maxDoc, null, Integer.MAX_VALUE); doScores = random().nextBoolean(); } all = searcher.search(query, allManager); @@ -268,15 +268,13 @@ public class TestSearchAfter extends LuceneTestCase { if (VERBOSE) { System.out.println(" iter lastBottom=" + lastBottom); } - pagedManager = - new TopScoreDocCollectorManager(pageSize, lastBottom, Integer.MAX_VALUE, true); + pagedManager = new TopScoreDocCollectorManager(pageSize, lastBottom, Integer.MAX_VALUE); } else { if (VERBOSE) { System.out.println(" iter lastBottom=" + lastBottom); } pagedManager = - new TopFieldCollectorManager( - sort, pageSize, (FieldDoc) lastBottom, Integer.MAX_VALUE, true); + new TopFieldCollectorManager(sort, pageSize, (FieldDoc) lastBottom, Integer.MAX_VALUE); } paged = searcher.search(query, pagedManager); if (doScores) { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java b/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java index 38dbe3399ee..98496ae3210 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java @@ -237,7 +237,7 @@ public class TestSortOptimization extends LuceneTestCase { sortField2.setMissingValue(0L); // set a competitive missing value final Sort sort = new Sort(sortField1, sortField2); CollectorManager manager = - new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, true); + new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold); TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager); assertEquals(topDocs.scoreDocs.length, numHits); assertEquals( @@ -264,7 +264,7 @@ public class TestSortOptimization extends LuceneTestCase { sortField.setMissingValue(Long.MAX_VALUE); // set a competitive missing value final Sort sort = new Sort(sortField); CollectorManager manager = - new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold, true); + new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold); TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager); assertEquals(topDocs.scoreDocs.length, numHits); assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value(), numDocs); @@ -279,7 +279,7 @@ public class TestSortOptimization extends LuceneTestCase { sortField.setMissingValue(Long.MAX_VALUE); // set a competitive missing value final Sort sort = new Sort(sortField); CollectorManager manager = - new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold, true); + new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold); TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager); assertEquals(topDocs.scoreDocs.length, numHits); assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value(), numDocs); @@ -347,7 +347,7 @@ public class TestSortOptimization extends LuceneTestCase { sortField.setMissingValue(0L); // missing value is not competitive final Sort sort = new Sort(sortField); CollectorManager manager = - new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, true); + new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold); topDocs1 = searcher.search(new MatchAllDocsQuery(), manager); assertNonCompetitiveHitsAreSkipped(topDocs1.totalHits.value(), numDocs); } @@ -358,7 +358,7 @@ public class TestSortOptimization extends LuceneTestCase { final Sort sort = new Sort(sortField); sortField.setOptimizeSortWithPoints(false); CollectorManager manager = - new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, true); + new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold); topDocs2 = searcher.search(new MatchAllDocsQuery(), manager); // assert that the resulting hits are the same assertEquals(topDocs1.scoreDocs.length, topDocs2.scoreDocs.length); @@ -381,7 +381,7 @@ public class TestSortOptimization extends LuceneTestCase { sortField2.setMissingValue(0L); // missing value is not competitive final Sort multiSorts = new Sort(new SortField[] {sortField1, sortField2}); CollectorManager manager = - new TopFieldCollectorManager(multiSorts, numHits, null, totalHitsThreshold, true); + new TopFieldCollectorManager(multiSorts, numHits, null, totalHitsThreshold); TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager); // can't optimization with NumericDocValues when there are multiple comparators assertEquals(topDocs.totalHits.value(), numDocs); @@ -959,7 +959,7 @@ public class TestSortOptimization extends LuceneTestCase { // test search int numHits = 1 + random().nextInt(100); CollectorManager manager = - new TopFieldCollectorManager(new Sort(sortField), numHits, null, numHits, true); + new TopFieldCollectorManager(new Sort(sortField), numHits, null, numHits); TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager); for (int i = 0; i < topDocs.scoreDocs.length; i++) { long expectedSeqNo = seqNos.get(i); @@ -1010,12 +1010,12 @@ public class TestSortOptimization extends LuceneTestCase { int expectedHits = Math.min(numDocs - visitedHits, batch); CollectorManager manager = - new TopFieldCollectorManager(sort, batch, (FieldDoc) after, totalHitsThreshold, true); + new TopFieldCollectorManager(sort, batch, (FieldDoc) after, totalHitsThreshold); TopDocs topDocs = searcher.search(query, manager); ScoreDoc[] scoreDocs = topDocs.scoreDocs; CollectorManager manager2 = - new TopFieldCollectorManager(sort2, batch, (FieldDoc) after, totalHitsThreshold, true); + new TopFieldCollectorManager(sort2, batch, (FieldDoc) after, totalHitsThreshold); TopDocs topDocs2 = searcher.search(query, manager2); ScoreDoc[] scoreDocs2 = topDocs2.scoreDocs; @@ -1210,7 +1210,7 @@ public class TestSortOptimization extends LuceneTestCase { final int totalHitsThreshold = 5; CollectorManager manager = - new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, true); + new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold); IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean(), false); TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager); @@ -1240,7 +1240,7 @@ public class TestSortOptimization extends LuceneTestCase { IndexSearcher searcher = newSearcher(reader, true, true, false); Query query = new MatchAllDocsQuery(); CollectorManager manager = - new TopFieldCollectorManager(sort, n, after, n, true); + new TopFieldCollectorManager(sort, n, after, n); TopDocs topDocs = searcher.search(query, manager); IndexSearcher unoptimizedSearcher = newSearcher(new NoIndexDirectoryReader(reader), true, true, false); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java index 44bd66148cd..cc7405f6772 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java @@ -160,8 +160,7 @@ public class TestTopDocsCollector extends LuceneTestCase { int numResults, int thresHold, Query q, IndexReader indexReader) throws IOException { IndexSearcher searcher = newSearcher(indexReader, true, true, false); TopScoreDocCollectorManager collectorManager = - new TopScoreDocCollectorManager( - numResults, null, thresHold, searcher.getSlices().length > 1); + new TopScoreDocCollectorManager(numResults, null, thresHold); return searcher.search(q, collectorManager); } @@ -169,8 +168,7 @@ public class TestTopDocsCollector extends LuceneTestCase { int numResults, int threshold, Query q, IndexReader indexReader) throws IOException { IndexSearcher searcher = newSearcher(indexReader, true, true, true); TopScoreDocCollectorManager collectorManager = - new TopScoreDocCollectorManager( - numResults, null, threshold, searcher.getSlices().length > 1); + new TopScoreDocCollectorManager(numResults, null, threshold); return searcher.search(q, collectorManager); } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java index 4215b755b36..1d5b346c29d 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopDocsMerge.java @@ -56,7 +56,7 @@ public class TestTopDocsMerge extends LuceneTestCase { public TopDocs search(Weight weight, int topN) throws IOException { TopScoreDocCollector collector = - new TopScoreDocCollectorManager(topN, null, Integer.MAX_VALUE, false).newCollector(); + new TopScoreDocCollectorManager(topN, null, Integer.MAX_VALUE).newCollector(); searchLeaf(ctx, 0, DocIdSetIterator.NO_MORE_DOCS, weight, collector); return collector.topDocs(); } @@ -348,8 +348,7 @@ public class TestTopDocsMerge extends LuceneTestCase { subHits = subSearcher.search(w, numHits); } else { final TopFieldCollector c = - new TopFieldCollectorManager(sort, numHits, null, Integer.MAX_VALUE, false) - .newCollector(); + new TopFieldCollectorManager(sort, numHits, null, Integer.MAX_VALUE).newCollector(); subSearcher.search(w, c); subHits = c.topDocs(0, numHits); } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java index bd0d408e0b2..4393ace2c26 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollector.java @@ -76,8 +76,7 @@ public class TestTopFieldCollector extends LuceneTestCase { throws IOException { IndexSearcher searcher = newSearcher(indexReader); TopFieldCollectorManager manager = - new TopFieldCollectorManager( - sort, numResults, null, thresHold, searcher.getSlices().length > 1); + new TopFieldCollectorManager(sort, numResults, null, thresHold); return searcher.search(q, manager); } @@ -87,8 +86,7 @@ public class TestTopFieldCollector extends LuceneTestCase { IndexSearcher searcher = newSearcher(indexReader, true, true, true); TopFieldCollectorManager collectorManager = - new TopFieldCollectorManager( - sort, numResults, null, threshold, searcher.getSlices().length > 1); + new TopFieldCollectorManager(sort, numResults, null, threshold); TopDocs topDoc = searcher.search(q, collectorManager); @@ -122,7 +120,7 @@ public class TestTopFieldCollector extends LuceneTestCase { for (int i = 0; i < sort.length; i++) { Query q = new MatchAllDocsQuery(); TopFieldCollectorManager tdc = - new TopFieldCollectorManager(sort[i], 10, null, Integer.MAX_VALUE, false); + new TopFieldCollectorManager(sort[i], 10, null, Integer.MAX_VALUE); TopDocs td = is.search(q, tdc); ScoreDoc[] sd = td.scoreDocs; for (int j = 0; j < sd.length; j++) { @@ -384,7 +382,7 @@ public class TestTopFieldCollector extends LuceneTestCase { Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort()}; for (int i = 0; i < sort.length; i++) { TopDocsCollector tdc = - new TopFieldCollectorManager(sort[i], 10, null, Integer.MAX_VALUE, false).newCollector(); + new TopFieldCollectorManager(sort[i], 10, null, Integer.MAX_VALUE).newCollector(); TopDocs td = tdc.topDocs(); assertEquals(0, td.totalHits.value()); } @@ -716,18 +714,17 @@ public class TestTopFieldCollector extends LuceneTestCase { try (IndexReader reader = DirectoryReader.open(w)) { IndexSearcher searcher = new IndexSearcher(reader); - TopFieldCollectorManager collectorManager = - new TopFieldCollectorManager(sort, 2, null, 10, true); + TopFieldCollectorManager collectorManager = new TopFieldCollectorManager(sort, 2, null, 10); TopDocs topDocs = searcher.search(new TermQuery(new Term("f", "foo")), collectorManager); assertEquals(10, topDocs.totalHits.value()); assertEquals(TotalHits.Relation.EQUAL_TO, topDocs.totalHits.relation()); - collectorManager = new TopFieldCollectorManager(sort, 2, null, 2, true); + collectorManager = new TopFieldCollectorManager(sort, 2, null, 2); topDocs = searcher.search(new TermQuery(new Term("f", "foo")), collectorManager); assertTrue(10 >= topDocs.totalHits.value()); assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, topDocs.totalHits.relation()); - collectorManager = new TopFieldCollectorManager(sort, 10, null, 2, true); + collectorManager = new TopFieldCollectorManager(sort, 10, null, 2); topDocs = searcher.search(new TermQuery(new Term("f", "foo")), collectorManager); assertEquals(10, topDocs.totalHits.value()); assertEquals(TotalHits.Relation.EQUAL_TO, topDocs.totalHits.relation()); diff --git a/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java index c3e09dbb072..576aa84d51f 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java @@ -197,9 +197,8 @@ public class DrillSideways { limit = 1; // the collector does not alow numHits = 0 } final int fTopN = Math.min(topN, limit); - final boolean supportsConcurrency = searcher.getSlices().length > 1; final TopFieldCollectorManager collectorManager = - new TopFieldCollectorManager(sort, fTopN, after, Integer.MAX_VALUE, supportsConcurrency); + new TopFieldCollectorManager(sort, fTopN, after, Integer.MAX_VALUE); final ConcurrentDrillSidewaysResult r = search(query, collectorManager); TopFieldDocs topDocs = r.collectorResult; @@ -230,9 +229,8 @@ public class DrillSideways { limit = 1; // the collector does not alow numHits = 0 } final int fTopN = Math.min(topN, limit); - final boolean supportsConcurrency = searcher.getSlices().length > 1; final TopScoreDocCollectorManager collectorManager = - new TopScoreDocCollectorManager(fTopN, after, Integer.MAX_VALUE, supportsConcurrency); + new TopScoreDocCollectorManager(fTopN, after, Integer.MAX_VALUE); final ConcurrentDrillSidewaysResult r = search(query, collectorManager); return new DrillSidewaysResult( r.facets, diff --git a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java index 34ce3ab6697..bc547407ee9 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java @@ -285,11 +285,11 @@ public class FacetsCollectorManager implements CollectorManager extends SecondPassGroupingCollector { () -> new TopDocsAndMaxScoreCollector( true, - new TopScoreDocCollectorManager(maxDocsPerGroup, null, Integer.MAX_VALUE, false) + new TopScoreDocCollectorManager(maxDocsPerGroup, null, Integer.MAX_VALUE) .newCollector(), null); } else { @@ -137,7 +137,7 @@ public class TopGroupsCollector extends SecondPassGroupingCollector { () -> { TopFieldCollector topDocsCollector = new TopFieldCollectorManager( - withinGroupSort, maxDocsPerGroup, null, Integer.MAX_VALUE, false) + withinGroupSort, maxDocsPerGroup, null, Integer.MAX_VALUE) .newCollector(); // TODO: disable exact counts? MaxScoreCollector maxScoreCollector = getMaxScores ? new MaxScoreCollector() : null; return new TopDocsAndMaxScoreCollector(false, topDocsCollector, maxScoreCollector); diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java index 227a9550a88..d959d9e2630 100644 --- a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java +++ b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java @@ -468,7 +468,7 @@ public class TestJoinUtil extends LuceneTestCase { final BitSet actualResult = new FixedBitSet(indexSearcher.getIndexReader().maxDoc()); final TopScoreDocCollector topScoreDocCollector = - new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE, false).newCollector(); + new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE).newCollector(); indexSearcher.search( joinQuery, MultiCollector.wrap(new BitSetCollector(actualResult), topScoreDocCollector)); assertBitSet(expectedResult, actualResult, indexSearcher); @@ -1546,7 +1546,7 @@ public class TestJoinUtil extends LuceneTestCase { // be also testing TopDocsCollector... final BitSet actualResult = new FixedBitSet(indexSearcher.getIndexReader().maxDoc()); final TopScoreDocCollector topScoreDocCollector = - new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE, false).newCollector(); + new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE).newCollector(); indexSearcher.search( joinQuery, MultiCollector.wrap(new BitSetCollector(actualResult), topScoreDocCollector)); diff --git a/lucene/luke/src/java/org/apache/lucene/luke/models/search/SearchImpl.java b/lucene/luke/src/java/org/apache/lucene/luke/models/search/SearchImpl.java index 4d5706be0f2..e95ca09060d 100644 --- a/lucene/luke/src/java/org/apache/lucene/luke/models/search/SearchImpl.java +++ b/lucene/luke/src/java/org/apache/lucene/luke/models/search/SearchImpl.java @@ -315,8 +315,7 @@ public final class SearchImpl extends LukeModel implements Search { } else { int hitsThreshold = exactHitsCount ? Integer.MAX_VALUE : DEFAULT_TOTAL_HITS_THRESHOLD; TopScoreDocCollectorManager collectorManager = - new TopScoreDocCollectorManager( - pageSize, after, hitsThreshold, searcher.getSlices().length > 1); + new TopScoreDocCollectorManager(pageSize, after, hitsThreshold); topDocs = searcher.search(query, collectorManager); } diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java index 426c02999f7..c5f1fdc899a 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java @@ -739,8 +739,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable { searcherMgrReadLock.unlock(); } try { - TopFieldCollectorManager c = - new TopFieldCollectorManager(SORT, num, null, 1, searcher.getSlices().length > 1); + TopFieldCollectorManager c = new TopFieldCollectorManager(SORT, num, null, 1); // System.out.println("got searcher=" + searcher); TopFieldDocs hits = searcher.search(finalQuery, c); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/CheckHits.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/CheckHits.java index 5a2611da2e2..f5d2c7ea174 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/CheckHits.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/CheckHits.java @@ -705,12 +705,10 @@ public class CheckHits { private static void doCheckTopScores(Query query, IndexSearcher searcher, int numHits) throws IOException { - boolean supportsConcurrency = searcher.getSlices().length > 1; TopScoreDocCollectorManager complete = - new TopScoreDocCollectorManager( - numHits, null, Integer.MAX_VALUE, supportsConcurrency); // COMPLETE + new TopScoreDocCollectorManager(numHits, null, Integer.MAX_VALUE); // COMPLETE TopScoreDocCollectorManager topScores = - new TopScoreDocCollectorManager(numHits, null, 1, supportsConcurrency); // TOP_SCORES + new TopScoreDocCollectorManager(numHits, null, 1); // TOP_SCORES TopDocs completeTopDocs = searcher.search(query, complete); TopDocs topScoresTopDocs = searcher.search(query, topScores); checkEqual(query, completeTopDocs.scoreDocs, topScoresTopDocs.scoreDocs); From 539cf3c9a335bccb50a0bddbf8cabd2738727528 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 5 Nov 2024 23:52:56 +0100 Subject: [PATCH 30/37] Adjust expectations of TestTopFieldCollectorEarlyTermination after #13943. --- ...TestTopFieldCollectorEarlyTermination.java | 24 ++++++++++++++----- 1 file changed, 18 insertions(+), 6 deletions(-) diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollectorEarlyTermination.java b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollectorEarlyTermination.java index 1540d77270c..52f313b3dfd 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollectorEarlyTermination.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollectorEarlyTermination.java @@ -29,9 +29,10 @@ import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.StringField; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.SerialMergeScheduler; import org.apache.lucene.index.Term; +import org.apache.lucene.search.IndexSearcher.LeafReaderContextPartition; +import org.apache.lucene.search.IndexSearcher.LeafSlice; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.analysis.MockAnalyzer; import org.apache.lucene.tests.index.MockRandomMergePolicy; @@ -39,6 +40,7 @@ import org.apache.lucene.tests.index.RandomIndexWriter; import org.apache.lucene.tests.search.CheckHits; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.tests.util.TestUtil; +import org.apache.lucene.util.Bits; public class TestTopFieldCollectorEarlyTermination extends LuceneTestCase { @@ -119,12 +121,22 @@ public class TestTopFieldCollectorEarlyTermination extends LuceneTestCase { final int iters = atLeast(1); for (int i = 0; i < iters; ++i) { createRandomIndex(false); - int maxSegmentSize = 0; - for (LeafReaderContext ctx : reader.leaves()) { - maxSegmentSize = Math.max(ctx.reader().numDocs(), maxSegmentSize); - } for (int j = 0; j < iters; ++j) { final IndexSearcher searcher = newSearcher(reader); + int maxSliceSize = 0; + for (LeafSlice slice : searcher.getSlices()) { + int numDocs = 0; // number of live docs in the slice + for (LeafReaderContextPartition partition : slice.partitions) { + Bits liveDocs = partition.ctx.reader().getLiveDocs(); + int maxDoc = Math.min(partition.maxDocId, partition.ctx.reader().maxDoc()); + for (int doc = partition.minDocId; doc < maxDoc; ++doc) { + if (liveDocs == null || liveDocs.get(doc)) { + numDocs++; + } + } + } + maxSliceSize = Math.max(maxSliceSize, numDocs); + } final int numHits = TestUtil.nextInt(random(), 1, numDocs); FieldDoc after; if (paging) { @@ -149,7 +161,7 @@ public class TestTopFieldCollectorEarlyTermination extends LuceneTestCase { TopDocs td2 = searcher.search(query, manager2); assertNotEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, td1.totalHits.relation()); - if (paging == false && maxSegmentSize > numHits && query instanceof MatchAllDocsQuery) { + if (paging == false && maxSliceSize > numHits && query instanceof MatchAllDocsQuery) { // Make sure that we sometimes early terminate assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, td2.totalHits.relation()); } From a888af76b2f4d120c716497602980e9b783d9881 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 6 Nov 2024 10:36:06 +0100 Subject: [PATCH 31/37] Simplify leaf slice calculation (#13893) No need to go through the indirection of 2 wrapped functions, just put the logic in plain methods. Also, we can just outright set the field if there's no executor. --- .../apache/lucene/search/IndexSearcher.java | 129 +++++++----------- 1 file changed, 50 insertions(+), 79 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index 44120064483..6e8bbf81966 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -27,7 +27,6 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.Executor; -import java.util.function.Function; import java.util.function.Supplier; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; @@ -115,13 +114,7 @@ public class IndexSearcher { protected final IndexReaderContext readerContext; protected final List leafContexts; - /** - * Used with executor - LeafSlice supplier where each slice holds a set of leafs executed within - * one thread. We are caching it instead of creating it eagerly to avoid calling a protected - * method from constructor, which is a bad practice. Always non-null, regardless of whether an - * executor is provided or not. - */ - private final Supplier leafSlicesSupplier; + private volatile LeafSlice[] leafSlices; // Used internally for load balancing threads executing for the query private final TaskExecutor taskExecutor; @@ -230,20 +223,18 @@ public class IndexSearcher { executor == null ? new TaskExecutor(Runnable::run) : new TaskExecutor(executor); this.readerContext = context; leafContexts = context.leaves(); - Function, LeafSlice[]> slicesProvider = - executor == null - ? leaves -> - leaves.isEmpty() - ? new LeafSlice[0] - : new LeafSlice[] { - new LeafSlice( - new ArrayList<>( - leaves.stream() - .map(LeafReaderContextPartition::createForEntireSegment) - .toList())) - } - : this::slices; - leafSlicesSupplier = new CachingLeafSlicesSupplier(slicesProvider, leafContexts); + if (executor == null) { + leafSlices = + leafContexts.isEmpty() + ? new LeafSlice[0] + : new LeafSlice[] { + new LeafSlice( + new ArrayList<>( + leafContexts.stream() + .map(LeafReaderContextPartition::createForEntireSegment) + .toList())) + }; + } } /** @@ -540,7 +531,43 @@ public class IndexSearcher { * @lucene.experimental */ public final LeafSlice[] getSlices() { - return leafSlicesSupplier.get(); + LeafSlice[] res = leafSlices; + if (res == null) { + res = computeAndCacheSlices(); + } + return res; + } + + private synchronized LeafSlice[] computeAndCacheSlices() { + LeafSlice[] res = leafSlices; + if (res == null) { + res = slices(leafContexts); + /* + * Enforce that there aren't multiple leaf partitions within the same leaf slice pointing to the + * same leaf context. It is a requirement that {@link Collector#getLeafCollector(LeafReaderContext)} + * gets called once per leaf context. Also, it does not make sense to partition a segment to then search + * those partitions as part of the same slice, because the goal of partitioning is parallel searching + * which happens at the slice level. + */ + for (LeafSlice leafSlice : res) { + if (leafSlice.partitions.length <= 1) { + continue; + } + enforceDistinctLeaves(leafSlice); + } + leafSlices = res; + } + return res; + } + + private static void enforceDistinctLeaves(LeafSlice leafSlice) { + Set distinctLeaves = new HashSet<>(); + for (LeafReaderContextPartition leafPartition : leafSlice.partitions) { + if (distinctLeaves.add(leafPartition.ctx) == false) { + throw new IllegalStateException( + "The same slice targets multiple leaf partitions of the same leaf reader context. A physical segment should rather get partitioned to be searched concurrently from as many slices as the number of leaf partitions it is split into."); + } + } } /** @@ -1164,60 +1191,4 @@ public class IndexSearcher { + IndexSearcher.getMaxClauseCount()); } } - - /** - * Supplier for {@link LeafSlice} slices which computes and caches the value on first invocation - * and returns cached value on subsequent invocation. If the passed in provider for slice - * computation throws exception then same will be passed to the caller of this supplier on each - * invocation. If the provider returns null then {@link NullPointerException} will be thrown to - * the caller. - * - *

    NOTE: To provide thread safe caching mechanism this class is implementing the (subtle) double-checked locking - * idiom - */ - private static class CachingLeafSlicesSupplier implements Supplier { - private volatile LeafSlice[] leafSlices; - - private final Function, LeafSlice[]> sliceProvider; - - private final List leaves; - - private CachingLeafSlicesSupplier( - Function, LeafSlice[]> provider, List leaves) { - this.sliceProvider = Objects.requireNonNull(provider, "leaf slice provider cannot be null"); - this.leaves = Objects.requireNonNull(leaves, "list of LeafReaderContext cannot be null"); - } - - @Override - public LeafSlice[] get() { - if (leafSlices == null) { - synchronized (this) { - if (leafSlices == null) { - leafSlices = - Objects.requireNonNull( - sliceProvider.apply(leaves), "slices computed by the provider is null"); - /* - * Enforce that there aren't multiple leaf partitions within the same leaf slice pointing to the - * same leaf context. It is a requirement that {@link Collector#getLeafCollector(LeafReaderContext)} - * gets called once per leaf context. Also, it does not make sense to partition a segment to then search - * those partitions as part of the same slice, because the goal of partitioning is parallel searching - * which happens at the slice level. - */ - for (LeafSlice leafSlice : leafSlices) { - Set distinctLeaves = new HashSet<>(); - for (LeafReaderContextPartition leafPartition : leafSlice.partitions) { - distinctLeaves.add(leafPartition.ctx); - } - if (leafSlice.partitions.length != distinctLeaves.size()) { - throw new IllegalStateException( - "The same slice targets multiple leaf partitions of the same leaf reader context. A physical segment should rather get partitioned to be searched concurrently from as many slices as the number of leaf partitions it is split into."); - } - } - } - } - } - return leafSlices; - } - } } From 12ca4779b962c96367f3e6a8b06523837e5e6434 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 6 Nov 2024 11:34:27 +0100 Subject: [PATCH 32/37] Fix TestPostingsUtil#testIntegerOverflow failure. (#13979) The group vint logic is mistakenly using the long->int conversion logic for the case when integers are being written rather than longs. Closes #13978 --- .../src/java/org/apache/lucene/util/GroupVIntUtil.java | 2 +- .../lucene/codecs/lucene101/TestPostingsUtil.java | 10 +++++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/util/GroupVIntUtil.java b/lucene/core/src/java/org/apache/lucene/util/GroupVIntUtil.java index 949292fd9d2..e95e2eee4db 100644 --- a/lucene/core/src/java/org/apache/lucene/util/GroupVIntUtil.java +++ b/lucene/core/src/java/org/apache/lucene/util/GroupVIntUtil.java @@ -292,7 +292,7 @@ public final class GroupVIntUtil { // tail vints for (; readPos < limit; readPos++) { - out.writeVInt(toInt(values[readPos])); + out.writeVInt(values[readPos]); } } } diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestPostingsUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestPostingsUtil.java index 41bb5c01f07..5d02d0561e3 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestPostingsUtil.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestPostingsUtil.java @@ -27,7 +27,15 @@ public class TestPostingsUtil extends LuceneTestCase { // checks for bug described in https://github.com/apache/lucene/issues/13373 public void testIntegerOverflow() throws IOException { - final int size = random().nextInt(1, ForUtil.BLOCK_SIZE); + // Size that writes the first value as a regular vint + int randomSize1 = random().nextInt(1, 3); + // Size that writes the first value as a group vint + int randomSize2 = random().nextInt(4, ForUtil.BLOCK_SIZE); + doTestIntegerOverflow(randomSize1); + doTestIntegerOverflow(randomSize2); + } + + private void doTestIntegerOverflow(int size) throws IOException { final int[] docDeltaBuffer = new int[size]; final int[] freqBuffer = new int[size]; From 65457224fb2eac4a9b2dea43c60d1a2d2bdf6068 Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Mon, 11 Nov 2024 13:28:09 +0100 Subject: [PATCH 33/37] Tessellator: Improve logic when two holes share the same vertex with the polygon (#13980) This commit tries to improve the algorithm by: 1.- If there is only one vertex, then there is not further checks and that's the one used. 2.- if there is a common vertex, it first compute the signed area of the join, if they have different sign, it chooses the negative one as that's the convex union. If they have the same sign, it computes the angle of the join and chooses the smallest angle. --- lucene/CHANGES.txt | 2 + .../org/apache/lucene/geo/Tessellator.java | 252 ++++++++++-------- .../apache/lucene/geo/TestTessellator.java | 158 +++++------ .../tests/geo/github-13841-1.geojson.gz | Bin 0 -> 2374 bytes .../lucene/tests/geo/github-13841-2.wkt.gz | Bin 0 -> 6501 bytes .../lucene/tests/geo/github-13841-3.wkt.gz | Bin 0 -> 2270 bytes 6 files changed, 205 insertions(+), 207 deletions(-) create mode 100644 lucene/test-framework/src/resources/org/apache/lucene/tests/geo/github-13841-1.geojson.gz create mode 100644 lucene/test-framework/src/resources/org/apache/lucene/tests/geo/github-13841-2.wkt.gz create mode 100644 lucene/test-framework/src/resources/org/apache/lucene/tests/geo/github-13841-3.wkt.gz diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index d547d7bdc65..1c4b5cbb249 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -97,6 +97,8 @@ Bug Fixes * GITHUB#12686: Added support for highlighting IndexOrDocValuesQuery. (Prudhvi Godithi) * GITHUB#13927: Fix StoredFieldsConsumer finish. (linfn) * GITHUB#13944: Ensure deterministic order of clauses for `DisjunctionMaxQuery#toString`. (Laurent Jakubina) +* GITHUB#13841: Improve Tessellatorlogic when two holes share the same vertex with the polygon which was failing + in valid polygons. (Ignacio Vera) Build --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/geo/Tessellator.java b/lucene/core/src/java/org/apache/lucene/geo/Tessellator.java index f1093c3e8b0..a2b8cad84ff 100644 --- a/lucene/core/src/java/org/apache/lucene/geo/Tessellator.java +++ b/lucene/core/src/java/org/apache/lucene/geo/Tessellator.java @@ -20,7 +20,6 @@ import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitude; import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitude; import static org.apache.lucene.geo.GeoUtils.lineCrossesLine; import static org.apache.lucene.geo.GeoUtils.lineOverlapLine; -import static org.apache.lucene.geo.GeoUtils.orient; import java.util.ArrayList; import java.util.HashMap; @@ -215,7 +214,7 @@ public final class Tessellator { * Creates a circular doubly linked list using polygon points. The order is governed by the * specified winding order */ - private static final Node createDoublyLinkedList( + private static Node createDoublyLinkedList( final double[] x, final double[] y, final WindingOrder polyWindingOrder, @@ -243,7 +242,7 @@ public final class Tessellator { return filterPoints(lastNode, null); } - private static final Node eliminateHoles(final XYPolygon polygon, Node outerNode) { + private static Node eliminateHoles(final XYPolygon polygon, Node outerNode) { // Define a list to hole a reference to each filtered hole list. final List holeList = new ArrayList<>(); // keep a reference to the hole @@ -273,8 +272,8 @@ public final class Tessellator { return eliminateHoles(holeList, holeListPolygons, outerNode); } - /** Links every hole into the outer loop, producing a single-ring polygon without holes. * */ - private static final Node eliminateHoles(final Polygon polygon, Node outerNode) { + /** Links every hole into the outer loop, producing a single-ring polygon without holes. */ + private static Node eliminateHoles(final Polygon polygon, Node outerNode) { // Define a list to hole a reference to each filtered hole list. final List holeList = new ArrayList<>(); // keep a reference to the hole @@ -304,7 +303,7 @@ public final class Tessellator { return eliminateHoles(holeList, holeListPolygons, outerNode); } - private static final Node eliminateHoles( + private static Node eliminateHoles( List holeList, final Map holeListPolygons, Node outerNode) { // Sort the hole vertices by x coordinate holeList.sort( @@ -350,30 +349,19 @@ public final class Tessellator { } /** Finds a bridge between vertices that connects a hole with an outer ring, and links it */ - private static final void eliminateHole( + private static void eliminateHole( final Node holeNode, Node outerNode, double holeMinX, double holeMaxX, double holeMinY, double holeMaxY) { - // Attempt to find a common point between the HoleNode and OuterNode. - Node next = outerNode; - do { - if (Rectangle.containsPoint( - next.getY(), next.getX(), holeMinY, holeMaxY, holeMinX, holeMaxX)) { - Node sharedVertex = getSharedVertex(holeNode, next); - if (sharedVertex != null) { - // Split the resulting polygon. - Node node = splitPolygon(next, sharedVertex, true); - // Filter the split nodes. - filterPoints(node, node.next); - return; - } - } - next = next.next; - } while (next != outerNode); + // Attempt to merge the hole using a common point between if it exists. + if (maybeMergeHoleWithSharedVertices( + holeNode, outerNode, holeMinX, holeMaxX, holeMinY, holeMaxY)) { + return; + } // Attempt to find a logical bridge between the HoleNode and OuterNode. outerNode = fetchHoleBridge(holeNode, outerNode); @@ -390,12 +378,112 @@ public final class Tessellator { } } + /** + * Choose a common vertex between the polygon and the hole if it exists and return true, otherwise + * return false + */ + private static boolean maybeMergeHoleWithSharedVertices( + final Node holeNode, + Node outerNode, + double holeMinX, + double holeMaxX, + double holeMinY, + double holeMaxY) { + // Attempt to find a common point between the HoleNode and OuterNode. + Node sharedVertex = null; + Node sharedVertexConnection = null; + Node next = outerNode; + do { + if (Rectangle.containsPoint( + next.getY(), next.getX(), holeMinY, holeMaxY, holeMinX, holeMaxX)) { + Node newSharedVertex = getSharedVertex(holeNode, next); + if (newSharedVertex != null) { + if (sharedVertex == null) { + sharedVertex = newSharedVertex; + sharedVertexConnection = next; + } else if (newSharedVertex.equals(sharedVertex)) { + // This can only happen if this vertex has been already used for a bridge. We need to + // choose the right one. + sharedVertexConnection = + getSharedInsideVertex(sharedVertex, sharedVertexConnection, next); + } + } + } + next = next.next; + } while (next != outerNode); + if (sharedVertex != null) { + // Split the resulting polygon. + Node node = splitPolygon(sharedVertexConnection, sharedVertex, true); + // Filter the split nodes. + filterPoints(node, node.next); + return true; + } + return false; + } + + /** Check if the provided vertex is in the polygon and return it */ + private static Node getSharedVertex(final Node polygon, final Node vertex) { + Node next = polygon; + do { + if (isVertexEquals(next, vertex)) { + return next; + } + next = next.next; + } while (next != polygon); + return null; + } + + /** Choose the vertex that has a smaller angle with the hole vertex */ + static Node getSharedInsideVertex(Node holeVertex, Node candidateA, Node candidateB) { + assert isVertexEquals(holeVertex, candidateA) && isVertexEquals(holeVertex, candidateB); + // we are joining candidate.prevNode -> holeVertex.node -> holeVertex.nextNode. + // A negative area means a convex angle. if both are convex/reflex choose the point of + // minimum angle + final double a1 = + area( + candidateA.previous.getX(), + candidateA.previous.getY(), + holeVertex.getX(), + holeVertex.getY(), + holeVertex.next.getX(), + holeVertex.next.getY()); + final double a2 = + area( + candidateB.previous.getX(), + candidateB.previous.getY(), + holeVertex.getX(), + holeVertex.getY(), + holeVertex.next.getX(), + holeVertex.next.getY()); + + if (a1 < 0 != a2 < 0) { + // one is convex, the other reflex, get the convex one + return a1 < a2 ? candidateA : candidateB; + } else { + // both are convex / reflex, choose the smallest angle + final double angle1 = angle(candidateA.previous, candidateA, holeVertex.next); + final double angle2 = angle(candidateB.previous, candidateB, holeVertex.next); + return angle1 < angle2 ? candidateA : candidateB; + } + } + + private static double angle(Node a, Node b, Node c) { + final double ax = a.getX() - b.getX(); + final double ay = a.getY() - b.getY(); + final double cx = c.getX() - b.getX(); + final double cy = c.getY() - b.getY(); + final double dotProduct = ax * cx + ay * cy; + final double aLength = Math.sqrt(ax * ax + ay * ay); + final double bLength = Math.sqrt(cx * cx + cy * cy); + return Math.acos(dotProduct / (aLength * bLength)); + } + /** * David Eberly's algorithm for finding a bridge between a hole and outer polygon * *

    see: http://www.geometrictools.com/Documentation/TriangulationByEarClipping.pdf */ - private static final Node fetchHoleBridge(final Node holeNode, final Node outerNode) { + private static Node fetchHoleBridge(final Node holeNode, final Node outerNode) { Node p = outerNode; double qx = Double.NEGATIVE_INFINITY; final double hx = holeNode.getX(); @@ -453,34 +541,8 @@ public final class Tessellator { return connection; } - /** Check if the provided vertex is in the polygon and return it * */ - private static Node getSharedVertex(final Node polygon, final Node vertex) { - Node next = polygon; - do { - if (isVertexEquals(next, vertex)) { - // make sure we are not crossing the polygon. This might happen when several holes share the - // same polygon vertex. - boolean crosses = - GeoUtils.lineCrossesLine( - next.previous.getX(), - next.previous.getY(), - vertex.next.getX(), - vertex.next.getY(), - next.next.getX(), - next.next.getY(), - vertex.previous.getX(), - vertex.previous.getY()); - if (crosses == false) { - return next; - } - } - next = next.next; - } while (next != polygon); - return null; - } - /** Finds the left-most hole of a polygon ring. * */ - private static final Node fetchLeftmost(final Node start) { + private static Node fetchLeftmost(final Node start) { Node node = start; Node leftMost = start; do { @@ -502,7 +564,7 @@ public final class Tessellator { * Main ear slicing loop which triangulates the vertices of a polygon, provided as a doubly-linked * list. * */ - private static final List earcutLinkedList( + private static List earcutLinkedList( Object polygon, Node currEar, final List tessellation, @@ -587,7 +649,7 @@ public final class Tessellator { } /** Determines whether a polygon node forms a valid ear with adjacent nodes. * */ - private static final boolean isEar(final Node ear, final boolean mortonOptimized) { + private static boolean isEar(final Node ear, final boolean mortonOptimized) { if (mortonOptimized == true) { return mortonIsEar(ear); } @@ -623,7 +685,7 @@ public final class Tessellator { * Uses morton code for speed to determine whether or a polygon node forms a valid ear w/ adjacent * nodes */ - private static final boolean mortonIsEar(final Node ear) { + private static boolean mortonIsEar(final Node ear) { // triangle bbox (flip the bits so negative encoded values are < positive encoded values) int minTX = StrictMath.min(StrictMath.min(ear.previous.x, ear.x), ear.next.x) ^ 0x80000000; int minTY = StrictMath.min(StrictMath.min(ear.previous.y, ear.y), ear.next.y) ^ 0x80000000; @@ -740,7 +802,7 @@ public final class Tessellator { } /** Iterate through all polygon nodes and remove small local self-intersections * */ - private static final Node cureLocalIntersections( + private static Node cureLocalIntersections( Node startNode, final List tessellation, final boolean mortonOptimized) { Node node = startNode; Node nextNode; @@ -794,7 +856,7 @@ public final class Tessellator { * Attempt to split a polygon and independently triangulate each side. Return true if the polygon * was splitted * */ - private static final boolean splitEarcut( + private static boolean splitEarcut( final Object polygon, final Node start, final List tessellation, @@ -858,7 +920,7 @@ public final class Tessellator { * Uses morton code for speed to determine whether or not and edge defined by a and b overlaps * with a polygon edge */ - private static final void mortonCheckIntersection(final Node a, final Node b) { + private static void mortonCheckIntersection(final Node a, final Node b) { // edge bbox (flip the bits so negative encoded values are < positive encoded values) int minTX = StrictMath.min(a.x, a.next.x) ^ 0x80000000; int minTY = StrictMath.min(a.y, a.next.y) ^ 0x80000000; @@ -974,7 +1036,7 @@ public final class Tessellator { * Uses morton code for speed to determine whether or not and edge defined by a and b overlaps * with a polygon edge */ - private static final boolean isMortonEdgeFromPolygon(final Node a, final Node b) { + private static boolean isMortonEdgeFromPolygon(final Node a, final Node b) { // edge bbox (flip the bits so negative encoded values are < positive encoded values) final int minTX = StrictMath.min(a.x, b.x) ^ 0x80000000; final int minTY = StrictMath.min(a.y, b.y) ^ 0x80000000; @@ -1060,7 +1122,7 @@ public final class Tessellator { } /** Links two polygon vertices using a bridge. * */ - private static final Node splitPolygon(final Node a, final Node b, boolean edgeFromPolygon) { + private static Node splitPolygon(final Node a, final Node b, boolean edgeFromPolygon) { final Node a2 = new Node(a); final Node b2 = new Node(b); final Node an = a.next; @@ -1136,7 +1198,7 @@ public final class Tessellator { return windingSum; } - private static final boolean isLocallyInside(final Node a, final Node b) { + private static boolean isLocallyInside(final Node a, final Node b) { double area = area( a.previous.getX(), a.previous.getY(), a.getX(), a.getY(), a.next.getX(), a.next.getY()); @@ -1156,7 +1218,7 @@ public final class Tessellator { } /** Determine whether the middle point of a polygon diagonal is contained within the polygon */ - private static final boolean middleInsert( + private static boolean middleInsert( final Node start, final double x0, final double y0, final double x1, final double y1) { Node node = start; Node nextNode; @@ -1179,7 +1241,7 @@ public final class Tessellator { } /** Determines if the diagonal of a polygon is intersecting with any polygon elements. * */ - private static final boolean isIntersectingPolygon( + private static boolean isIntersectingPolygon( final Node start, final double x0, final double y0, final double x1, final double y1) { Node node = start; Node nextNode; @@ -1198,7 +1260,7 @@ public final class Tessellator { } /** Determines whether two line segments intersect. * */ - public static final boolean linesIntersect( + public static boolean linesIntersect( final double aX0, final double aY0, final double aX1, @@ -1212,7 +1274,7 @@ public final class Tessellator { } /** Interlinks polygon nodes in Z-Order. It reset the values on the z values* */ - private static final void sortByMortonWithReset(Node start) { + private static void sortByMortonWithReset(Node start) { Node next = start; do { next.previousZ = next.previous; @@ -1223,7 +1285,7 @@ public final class Tessellator { } /** Interlinks polygon nodes in Z-Order. * */ - private static final void sortByMorton(Node start) { + private static void sortByMorton(Node start) { start.previousZ.nextZ = null; start.previousZ = null; // Sort the generated ring using Z ordering. @@ -1234,7 +1296,7 @@ public final class Tessellator { * Simon Tatham's doubly-linked list O(n log n) mergesort see: * http://www.chiark.greenend.org.uk/~sgtatham/algorithms/listsort.html */ - private static final void tathamSort(Node list) { + private static void tathamSort(Node list) { Node p, q, e, tail; int i, numMerges, pSize, qSize; int inSize = 1; @@ -1290,7 +1352,7 @@ public final class Tessellator { } /** Eliminate colinear/duplicate points from the doubly linked list */ - private static final Node filterPoints(final Node start, Node end) { + private static Node filterPoints(final Node start, Node end) { if (start == null) { return start; } @@ -1343,7 +1405,7 @@ public final class Tessellator { /** * Creates a node and optionally links it with a previous node in a circular doubly-linked list */ - private static final Node insertNode( + private static Node insertNode( final double[] x, final double[] y, int index, @@ -1370,7 +1432,7 @@ public final class Tessellator { } /** Removes a node from the doubly linked list */ - private static final void removeNode(Node node, boolean edgeFromPolygon) { + private static void removeNode(Node node, boolean edgeFromPolygon) { node.next.previous = node.previous; node.previous.next = node.next; node.previous.isNextEdgeFromPolygon = edgeFromPolygon; @@ -1384,16 +1446,16 @@ public final class Tessellator { } /** Determines if two point vertices are equal. * */ - private static final boolean isVertexEquals(final Node a, final Node b) { + private static boolean isVertexEquals(final Node a, final Node b) { return isVertexEquals(a, b.getX(), b.getY()); } /** Determines if two point vertices are equal. * */ - private static final boolean isVertexEquals(final Node a, final double x, final double y) { + private static boolean isVertexEquals(final Node a, final double x, final double y) { return a.getX() == x && a.getY() == y; } - /** Compute signed area of triangle */ + /** Compute signed area of triangle, negative means convex angle and positive reflex angle. */ private static double area( final double aX, final double aY, @@ -1419,29 +1481,6 @@ public final class Tessellator { && (bx - x) * (cy - y) - (cx - x) * (by - y) >= 0; } - /** compute whether the given x, y point is in a triangle; uses the winding order method */ - public static boolean pointInTriangle( - double x, double y, double ax, double ay, double bx, double by, double cx, double cy) { - double minX = StrictMath.min(ax, StrictMath.min(bx, cx)); - double minY = StrictMath.min(ay, StrictMath.min(by, cy)); - double maxX = StrictMath.max(ax, StrictMath.max(bx, cx)); - double maxY = StrictMath.max(ay, StrictMath.max(by, cy)); - // check the bounding box because if the triangle is degenerated, e.g points and lines, we need - // to filter out - // coplanar points that are not part of the triangle. - if (x >= minX && x <= maxX && y >= minY && y <= maxY) { - int a = orient(x, y, ax, ay, bx, by); - int b = orient(x, y, bx, by, cx, cy); - if (a == 0 || b == 0 || a < 0 == b < 0) { - int c = orient(x, y, cx, cy, ax, ay); - return c == 0 || (c < 0 == (b < 0 || a < 0)); - } - return false; - } else { - return false; - } - } - /** * Implementation of this interface will receive calls with internal data at each step of the * triangulation algorithm. This is of use for debugging complex cases, as well as gaining insight @@ -1508,7 +1547,7 @@ public final class Tessellator { } /** Circular Doubly-linked list used for polygon coordinates */ - protected static class Node { + static class Node { // node index in the linked list private final int idx; // vertex index in the polygon @@ -1524,9 +1563,9 @@ public final class Tessellator { private final long morton; // previous node - private Node previous; + Node previous; // next node - private Node next; + Node next; // previous z node private Node previousZ; // next z node @@ -1534,7 +1573,7 @@ public final class Tessellator { // if the edge from this node to the next node is part of the polygon edges private boolean isNextEdgeFromPolygon; - protected Node( + Node( final double[] x, final double[] y, final int index, @@ -1600,7 +1639,7 @@ public final class Tessellator { Node[] vertex; boolean[] edgeFromPolygon; - protected Triangle( + private Triangle( Node a, boolean isABfromPolygon, Node b, @@ -1636,19 +1675,6 @@ public final class Tessellator { return edgeFromPolygon[startVertex]; } - /** utility method to compute whether the point is in the triangle */ - protected boolean containsPoint(double lat, double lon) { - return pointInTriangle( - lon, - lat, - vertex[0].getX(), - vertex[0].getY(), - vertex[1].getX(), - vertex[1].getY(), - vertex[2].getX(), - vertex[2].getY()); - } - /** pretty print the triangle vertices */ @Override public String toString() { diff --git a/lucene/core/src/test/org/apache/lucene/geo/TestTessellator.java b/lucene/core/src/test/org/apache/lucene/geo/TestTessellator.java index 8002717a9ce..e2e964526e8 100644 --- a/lucene/core/src/test/org/apache/lucene/geo/TestTessellator.java +++ b/lucene/core/src/test/org/apache/lucene/geo/TestTessellator.java @@ -430,11 +430,7 @@ public class TestTessellator extends LuceneTestCase { + "(6.9735097 51.6245538,6.9736199 51.624605,6.9736853 51.6246203,6.9737516 51.6246231,6.9738024 51.6246107,6.9738324 51.6245878,6.9738425 51.6245509,6.9738332 51.6245122,6.9738039 51.6244869,6.9737616 51.6244687,6.9737061 51.6244625,6.9736445 51.6244749,6.9735736 51.6245046,6.9735097 51.6245538))," + "((6.9731576 51.6249947,6.9731361 51.6250664,6.9731161 51.6251037,6.9731022 51.6250803,6.9731277 51.62502,6.9731576 51.6249947)))"; Polygon[] polygons = (Polygon[]) SimpleWKTShapeParser.parse(wkt); - for (Polygon polygon : polygons) { - List tessellation = - Tessellator.tessellate(polygon, random().nextBoolean()); - assertTrue(tessellation.size() > 0); - } + checkMultiPolygon(polygons, 0.0); } public void testComplexPolygon27() throws Exception { @@ -684,13 +680,7 @@ public class TestTessellator extends LuceneTestCase { public void testComplexPolygon40() throws Exception { String wkt = GeoTestUtil.readShape("lucene-9251.wkt.gz"); Polygon polygon = (Polygon) SimpleWKTShapeParser.parse(wkt); - List tessellation = - Tessellator.tessellate(polygon, random().nextBoolean()); - // calculate the area of big polygons have numerical error - assertEquals(area(polygon), area(tessellation), 1e-12); - for (Tessellator.Triangle t : tessellation) { - checkTriangleEdgesFromPolygon(polygon, t); - } + checkPolygon(polygon, 1e-12); } public void testComplexPolygon41() throws Exception { @@ -706,15 +696,7 @@ public class TestTessellator extends LuceneTestCase { public void testComplexPolygon42() throws Exception { String geoJson = GeoTestUtil.readShape("lucene-9417.geojson.gz"); Polygon[] polygons = Polygon.fromGeoJSON(geoJson); - for (Polygon polygon : polygons) { - List tessellation = - Tessellator.tessellate(polygon, random().nextBoolean()); - // calculate the area of big polygons have numerical error - assertEquals(area(polygon), area(tessellation), 1e-11); - for (Tessellator.Triangle t : tessellation) { - checkTriangleEdgesFromPolygon(polygon, t); - } - } + checkMultiPolygon(polygons, 1e-11); } public void testComplexPolygon43() throws Exception { @@ -727,12 +709,7 @@ public class TestTessellator extends LuceneTestCase { + "(-88.3245325358123 41.9306419084828,-88.3245478066552 41.9305086556331,-88.3245658060855 41.930351580587,-88.3242368660096 41.9303327977821,-88.3242200926128 41.9304905242189,-88.324206161464 41.9306215207536,-88.3245325358123 41.9306419084828)," + "(-88.3236767661893 41.9307089429871,-88.3237008716322 41.930748885445,-88.323876104365 41.9306891087739,-88.324063438129 41.9306252050871,-88.3239244290607 41.930399373909,-88.3237349076233 41.9304653056436,-88.3235653339759 41.9305242981369,-88.3236767661893 41.9307089429871))"; Polygon polygon = (Polygon) SimpleWKTShapeParser.parse(wkt); - List tessellation = - Tessellator.tessellate(polygon, random().nextBoolean()); - assertEquals(area(polygon), area(tessellation), 1e-11); - for (Tessellator.Triangle t : tessellation) { - checkTriangleEdgesFromPolygon(polygon, t); - } + checkPolygon(polygon, 1e-11); } public void testComplexPolygon44() throws Exception { @@ -748,12 +725,7 @@ public class TestTessellator extends LuceneTestCase { "Polygon self-intersection at lat=34.21165542666664 lon=-83.88787058666672", ex.getMessage()); } else { - List tessellation = - Tessellator.tessellate(polygons[i], random().nextBoolean()); - assertEquals(area(polygons[i]), area(tessellation), 0.0); - for (Tessellator.Triangle t : tessellation) { - checkTriangleEdgesFromPolygon(polygons[i], t); - } + checkPolygon(polygons[i], 0.0); } } } @@ -761,55 +733,26 @@ public class TestTessellator extends LuceneTestCase { public void testComplexPolygon45() throws Exception { String geoJson = GeoTestUtil.readShape("lucene-10470.geojson.gz"); Polygon[] polygons = Polygon.fromGeoJSON(geoJson); - for (Polygon polygon : polygons) { - List tessellation = - Tessellator.tessellate(polygon, random().nextBoolean()); - // calculate the area of big polygons have numerical error - assertEquals(area(polygon), area(tessellation), 1e-11); - for (Tessellator.Triangle t : tessellation) { - checkTriangleEdgesFromPolygon(polygon, t); - } - } + checkMultiPolygon(polygons, 1e-11); } public void testComplexPolygon46() throws Exception { String wkt = GeoTestUtil.readShape("lucene-10470.wkt.gz"); Polygon polygon = (Polygon) SimpleWKTShapeParser.parse(wkt); - List tessellation = - Tessellator.tessellate(polygon, random().nextBoolean()); - // calculate the area of big polygons have numerical error - assertEquals(area(polygon), area(tessellation), 1e-11); - for (Tessellator.Triangle t : tessellation) { - checkTriangleEdgesFromPolygon(polygon, t); - } + checkPolygon(polygon, 1e-11); } public void testComplexPolygon47() throws Exception { String geoJson = GeoTestUtil.readShape("lucene-10470-2.geojson.gz"); Polygon[] polygons = Polygon.fromGeoJSON(geoJson); - for (Polygon polygon : polygons) { - List tessellation = - Tessellator.tessellate(polygon, random().nextBoolean()); - // calculate the area of big polygons have numerical error - assertEquals(area(polygon), area(tessellation), 1e-11); - for (Tessellator.Triangle t : tessellation) { - checkTriangleEdgesFromPolygon(polygon, t); - } - } + checkMultiPolygon(polygons, 1e-11); } @Nightly public void testComplexPolygon48() throws Exception { String geoJson = GeoTestUtil.readShape("lucene-10470-3.geojson.gz"); Polygon[] polygons = Polygon.fromGeoJSON(geoJson); - for (Polygon polygon : polygons) { - List tessellation = Tessellator.tessellate(polygon, true); - // calculate the area of big polygons have numerical error - assertEquals(area(polygon), area(tessellation), 1e-11); - for (Tessellator.Triangle t : tessellation) { - checkTriangleEdgesFromPolygon(polygon, t); - } - } + checkMultiPolygon(polygons, 1e-11); } public void testComplexPolygon49() throws Exception { @@ -817,25 +760,14 @@ public class TestTessellator extends LuceneTestCase { "POLYGON((77.500 13.500, 77.550 13.500, 77.530 13.470, 77.570 13.470," + "77.550 13.500, 77.600 13.500, 77.600 13.400, 77.500 13.400, 77.500 13.500))"; Polygon polygon = (Polygon) SimpleWKTShapeParser.parse(wkt); - List tessellation = - Tessellator.tessellate(polygon, random().nextBoolean()); - assertEquals(area(polygon), area(tessellation), 1e-11); - for (Tessellator.Triangle t : tessellation) { - checkTriangleEdgesFromPolygon(polygon, t); - } + checkPolygon(polygon, 1e-11); } public void testComplexPolygon50() throws Exception { String geoJson = GeoTestUtil.readShape("lucene-10563-1.geojson.gz"); Polygon[] polygons = Polygon.fromGeoJSON(geoJson); assertEquals("Only one polygon", 1, polygons.length); - Polygon polygon = polygons[0]; - List tessellation = Tessellator.tessellate(polygon, true); - // calculate the area of big polygons have numerical error - assertEquals(area(polygon), area(tessellation), 1e-11); - for (Tessellator.Triangle t : tessellation) { - checkTriangleEdgesFromPolygon(polygon, t); - } + checkPolygon(polygons[0], 1e-11); } public void testComplexPolygon50_WithMonitor() throws Exception { @@ -893,25 +825,13 @@ public class TestTessellator extends LuceneTestCase { public void testComplexPolygon53() throws Exception { String geoJson = GeoTestUtil.readShape("github-11986-1.geojson.gz"); Polygon[] polygons = Polygon.fromGeoJSON(geoJson); - for (Polygon polygon : polygons) { - List tessellation = Tessellator.tessellate(polygon, true); - assertEquals(area(polygon), area(tessellation), 0.0); - for (Tessellator.Triangle t : tessellation) { - checkTriangleEdgesFromPolygon(polygon, t); - } - } + checkMultiPolygon(polygons, 0.0); } public void testComplexPolygon54() throws Exception { String geoJson = GeoTestUtil.readShape("github-11986-2.geojson.gz"); Polygon[] polygons = Polygon.fromGeoJSON(geoJson); - for (Polygon polygon : polygons) { - List tessellation = Tessellator.tessellate(polygon, true); - assertEquals(area(polygon), area(tessellation), 0.0); - for (Tessellator.Triangle t : tessellation) { - checkTriangleEdgesFromPolygon(polygon, t); - } - } + checkMultiPolygon(polygons, 0.0); } public void testComplexPolygon55() throws Exception { @@ -936,6 +856,41 @@ public class TestTessellator extends LuceneTestCase { } } + public void testComplexPolygon57() throws Exception { + String geoJson = GeoTestUtil.readShape("github-13841-1.geojson.gz"); + Polygon[] polygons = Polygon.fromGeoJSON(geoJson); + checkMultiPolygon(polygons, 3e-11); + } + + @Nightly + public void testComplexPolygon58() throws Exception { + String wkt = GeoTestUtil.readShape("github-13841-2.wkt.gz"); + checkMultiPolygon(wkt); + } + + @Nightly + public void testComplexPolygon59() throws Exception { + String wkt = GeoTestUtil.readShape("github-13841-3.wkt.gz"); + Polygon[] polygons = (Polygon[]) SimpleWKTShapeParser.parse(wkt); + checkMultiPolygon(polygons, 1e-11); + } + + public void testComplexPolygon60() throws Exception { + String wkt = + "POLYGON((0 0, 5 1, 10 0, 11 5, 10 10,5 11, 0 10, 1 5, 0 0)," + + "(1 5, 1 7, 2 7, 1 5), (1 5, 4 8, 5 8, 1 5)," + + "(1 5, 3 6, 7 7, 1 5), (1 5, 2 3, 1 3, 1 5)," + + "(1 5, 3 4, 4 4, 1 5), (1 5, 5 6, 6 6, 1 5)," + + "(11 5, 10 3, 10 4, 11 5), (11 5,8 3, 8 4, 11 5)," + + "(11 5,5 4, 5 5, 11 5), (11 5, 4.5 3, 4 3, 11 5)," + + "(11 5, 8 6, 9 7, 11 5), (11 5, 10 8, 10 7, 11 5)," + + "(5 11, 2 10, 3 10, 5 11), (5 11, 3 9, 4 9, 5 11)," + + "(5 11, 5.5 8, 6 7, 5 11), (5 11, 8 8, 9 8, 5 11)," + + "(5 1, 2 0.5, 3 1, 5 1), (5 1, 8 0.5, 7 2, 5 1)," + + "(5 1, 3 2, 3 3, 5 1), (5 1, 5 2, 6 2, 5 1))"; + checkPolygon(wkt); + } + private static class TestCountingMonitor implements Tessellator.Monitor { private int count = 0; private int splitsStarted = 0; @@ -958,11 +913,26 @@ public class TestTessellator extends LuceneTestCase { } } + private void checkMultiPolygon(String wkt) throws Exception { + Polygon[] polygons = (Polygon[]) SimpleWKTShapeParser.parse(wkt); + checkMultiPolygon(polygons, 0.0); + } + + private void checkMultiPolygon(Polygon[] polygons, double delta) { + for (Polygon polygon : polygons) { + checkPolygon(polygon, delta); + } + } + private void checkPolygon(String wkt) throws Exception { Polygon polygon = (Polygon) SimpleWKTShapeParser.parse(wkt); + checkPolygon(polygon, 0.0); + } + + private void checkPolygon(Polygon polygon, double delta) { List tessellation = Tessellator.tessellate(polygon, random().nextBoolean()); - assertEquals(area(polygon), area(tessellation), 0.0); + assertEquals(area(polygon), area(tessellation), delta); for (Tessellator.Triangle t : tessellation) { checkTriangleEdgesFromPolygon(polygon, t); } diff --git a/lucene/test-framework/src/resources/org/apache/lucene/tests/geo/github-13841-1.geojson.gz b/lucene/test-framework/src/resources/org/apache/lucene/tests/geo/github-13841-1.geojson.gz new file mode 100644 index 0000000000000000000000000000000000000000..4b933e78534237e3a0cf1f4d526e7c860e6def3f GIT binary patch literal 2374 zcmV-M3Ay$kiwFn@N+V|g17~S;Xmw&OF*7(cF)n9iZ)$UIZUC*D%Zet(4Tbl9iot8g z)J2z4o+5jbMMz8t1W!o3F(L1sPi-(JlpGa|!98aF?$cEz>F69)`QKlC;-9a7{OKRx zfBpMUU;p9r-+%h+=O4cQ=I6iu^XJe1{L9~d_|uQy|BFw*`}oUGzx?^@|N8W4ihLV( z*4#0dADQyauRi_V_HWNTE_1CZbNKw_!#{rWfADP0vGyw9Lc?^2_VM;UB?oi1UEAkH#wCR-CDGL?6xwt&wF{yc2vTA`f`#BUr>8PF?mqdp*FB z%TCVT@u;;ikL(A$7_LS3k(RoB9(~9b8^fr#`|-KNmU*mS9Lwh5hxRr5Sled5LBJl; zWr|Ib;tFW)xAxyTN`pC8I=a3l}shqNPI?$VEJp$<8~(T}esmC|nR>l+?Hi(CD8r0}HO z=smtevl}gwhXuw50p7`FEFUWya2w9oFe+db>l-03-$O4#8Bm}qSAIcS?TOgXhxPH5 zqAK7_UB}SA7&_TE)bC7uEt2v!>|2K50uUz3k6DMDMLxAXyF zOPR%P5C57YzWQ*-&}kv}{T_?MceJ;Q1`_w-cjK{+lF4H|z|mIe5XELUaFg6@!(jm& z+2{AvJ5-ud-F;yS*Au;a|QI+x$9_G z$Ksnc&E9>1{Hm;wHkIPy-KpO9RfaD&@^A0H(BSYUoiNrtU)%!f zj?=HUl{TIVJ@;yJ8)4zn#Y-6!f_uDEeGX7aPs~RfkIjxW(gl_7WFPk~TzcriX2+fU zd?}de+}ePWjbq)F9}YDHnA zZ&xsC8>NllxX-`3|5r1=x!V9K~Jh2gneLP!%{w%h;&q&T~H}sv+82c_>fSZPXK~nbd@ZJ-- zTda@Twl(@=PD~a)(EMH=$f1ps(Ee0oqlaRg`;+mfSs~Umatb zms4FqtkE$a#&mcU7_KmGkQI8!p@eogR@!9r$yFXC`33YmAKH1?l)*CZ)rNdU0It4S zfXjbwz<0P*uaHZh?y)O6b9h?hA66Pc*EgywCO)*XTHwvxg7?R!>3=wKFK&5)--J)> z>wCCYiOGHLEC^G68S(1lp}9rJeewcc4#S|cJ|1*wdLQZ{)LFd3xDRk+Z{8k9}NL0JHHkIR-6uu z&`&_L4jg^wm%M?RV*P8L9Pq0~;TB|yVn3rl$XVo9P^1OF(5X0G^wk64*2t1C@Mh|V z^RQo14-u-xB`x2O7cFf#4o$kY314I!RXuMhFfVG;`J6bmF; zw$ZwMHqc@Cty$fg=b#Ke>*q5C8k;BPDeX^R^YR{_66{Ltu`IbXoaMU_Z?M*6^Ua6~Ser zAIkq>mZ;NFLj@dE5ieXib8wdSvA{#PVq_oK4`(E9mGd1FBAfo8t%lmn(OSUb zBd1NPkQyuxc59n>U$k?j)WE~INjyA%!z$;4ha$W??{6P&R1wVdw(2$A$jf|$@WX>4 zk8HeBXsZE~p81e|ek`B7-0v8{v*EZ)dsv{cI0&->JV)S{hXtZ@LyI>Xbn?jx5p;b# zTz-x&3O?YstrMr8fK+>|vh|7~``5$rEViABn+1_%7|%YMpUkkTlcTz@(hql_X#;X7 zUg7P$<%xdt$l&GGWHE)sAbvc8>$3~glB3-s8mJ(f+0pxWdh*rb^%@1dkC#pl%|n&m zkni&d2{W>VwLs=7{#;IVaVw z;>GjEG^z-ZY5lp&`(x~{{s;HF-^Bm&;*#U^-;m}jA2mon!8rlgrkWFtj|R5wMNG%e zwpP@0zDT1R_qFf)`Mm|`%V(5igBCCPGp9p$>_1BYZVog_QQt5LCG-X-2l-kq53#u( zjP}pl)vDXG-L=O<`+E<^YrMbgdUY9mU0Jhy0hKRb9{61H(TEoRH%IztllAaCU)RSY ziu-G376||Ou!sH90{s5y2DFnm5l`9p7L3POfAPZrYU0Q#Nc6aWAK literal 0 HcmV?d00001 diff --git a/lucene/test-framework/src/resources/org/apache/lucene/tests/geo/github-13841-2.wkt.gz b/lucene/test-framework/src/resources/org/apache/lucene/tests/geo/github-13841-2.wkt.gz new file mode 100644 index 0000000000000000000000000000000000000000..18089bef04ea40aa7f30acd3ac0fdcae4fac0b18 GIT binary patch literal 6501 zcmV-r8JgxFiwFqNk|SpT17~S;Xmw&OF*7(cF)cDKXJv0{b8l_{m0ZoPF1c-dl`Llf zdmxFFC_O7J2^{NwJ$%>N_V05ZH@5b)*(W@n6*Upa1o@fBnZl|M9o~{`Y_U z{kPwK`|EChwASWa|7PQlGy55TA?ER7HoFV7`GPa%7C8C~?r|>I{RQ)X7qkMy?>GA> zaGwj3qO;d2uv_q(95>qF7u(>tu?`7ZPJ-;L8X0Hw0)3BKCVX(DuU*)%Ma*`AYcI|= z8?AUWeTgP(QOnQ1MCggyxY+^$+?Rg69`s8e55NuXpv|Yz$ zD~h7tw$(_GQQz#N*C)s7HM{$)wMXxpSa|Jm_}~}k7+l7D5VTD>T7Co`fs1QT9|WUo zBsfoyo9xMLf_I=MA5sn>B8OVn#L@Z zwpxLP0R@64ayV8H42s-XL2w*%@;$o1!Rcdvz#?gwUy$1z9$ei(YJt;S;G!Uai3EG& zLX!eNY|N?YQy?W9lFPlDYh?HbBuSy!lPWa!5Js(pWQri)lsNofAjQz|9|UbvI@&%0 zxpV70^)EQ4*a1sXpr1SP3h~7xg==1rXTJ6&XgFol?=obD?I8wx8w59ZuDL^Vw{c}y zH4bZDEO!_Vf>LM*0at;Ha^w*`-lsG`2M)ENDRa{n4>dXs9Kp4V&SfGA2N|@QeorE5 z^{{(pgZdXP4S;#AZ^pc8(YYfC78M4!|(ZENrH$D}88 zynD3^UPq~o$A}W)y*u{^B8u`eOr{XFkp9>;l9n9~k6ho4R_g@aWQWwhvlrZ(Mp^|J zt4U4!_Bsj})J6{N#o0bS*bpe&yla!IDQ@V89uYSFg2Q)#u}IOO>I0M<96y}CX%9ML z8csi5-=wMnG)+IH)Bt67;C%T_y8VHur8Wd-E^wDiTpdoF;3>ZXaLBKWa*6=COK@G& z;f(~YI?F_56cE2??7rTr@vGJ}@|3`??mQ#tsuK&EY{UrM(i{S(R$uh!V?YK94Du78 z3ltW{wRhJicOyf#T=F=LBVirn|H_^%FigH`@SFJiHAq|F$BC5wuHi>+jXr3NyUr}* zu|~8DRsajow9cFYk0|2O`)KmAV#bg)12}6tP!^3m<>?lv6_JNZfN)HHfJ{$+B8*z; zy~{22#gVQ^#u3E5Abf+sRc3Fcoguin_;MXNLYuJ^{Bn(M)h6t z4v4}?{GwliTvIml)H#q5H7MetTc$b1q^`v?_mD{%#Q{DR!M^beqM@yWO$Ke-BABYu(@cawKNTl*g5(4H4GQn^D6 z-oY=PE^h42kw+rD=oB$s`-$cPna-NhSx_m#jLnS?vdCK_-yoMpPzO0B2LNlfqEA2+ z4Iid)rx(}Y<%U2W8FJ4f@DN16%b{@N^-+^~1xPYs?k`wrNWdzRre5YTGS48!{0nEt za2#L-J%PYLBaKJi{0MYfS17E&McrLfYX>b9A|~##)40MoYVxMZgb&fcL%)r@nf5W! zAlgLxDeYZTHJt>@#`qEKqcwqygaFblZHDNckq|vyzo0W*L|#TSgTtO&9}yw-A8@5D z2t1CkO?Q`Zizu+es7a+WAFOUc){0Zmhx>R6(_8Zufxm zsxZLD3fsJ(F^juAR-SqkP5@Mj&@MBWsqVPE>`E}Sm%(Bb;(Bps6eHnK`yI&-4k;>P zO3rR^Pm)Q;=MXy6Vc2GEhJFrqaQYPv&R#>5_r~e)PZBNW-B8MT<&F7UUq2)>lh1xZ zWvQ}Joz(_tl{AP3Wo9TMuybT=*^{8rh5k9qFMwHsL=dR22FofCVS$s0=8OxvzH9_~F!ePZNk3_iUSuK%n)1_8 zS-UTlFcATkqjjXJa>*=)0aOeK-(3sM8$vD?!+c6=Y_jBzKho@4Ruo?lK^2`cJZOQ} zW!R{*up@_KNL1bpf!T^Rk^8IyCu_fk#J@&D^vI;X%DlplD|Ucz)kuW!iiuwiV-Kol zWhncCHdLiL%w4k}&lvr*GQtLtQRq$go0WTx+%)0{n3a)kgeC*xVpef6hW^li^z+2f%B7{oWfkC*L2wh_M6)< z8V9KSZ?R^Fcv`iIWn~b?jjjOJz0X*R&5*NKEQ3F*j4&%g$$u#SixL<(qFbLXaT{Z{ zTkISJ#a|U}(}!!IupUZ=`*eci78X`*vE2&9s`?nrxk{Dr^rfbK--0(1u$Q17O*NTQ zk^yYoLQM%P!YNga5q|apV+)MpP+vF8Y_p=UzQ(;&m<9D)4{{^eRFBR4`~@d9Z)C-; zu&F{v>&4!@e^J%y6gLpSuto%s%z$ZCd2Cmh3$E!Fi3MU{l)$VH4T3z!td4P{>fP|= zi=PaYP7$pe!SSiMb%O0y@oJz&sH}KOqq>;yJW}s8pKTe@w9QIpsT>ORo|t&WZh2nUEcVl0anF&6*mwL-rDqwY(>3TxH4ualj@;) zTlj)Y>D;}0YniSSWJK4NYS2g^i^gk{0hT*zOL~HT)EDO#C!G~1xH7GcxT2_>i{?Ae zuGY!xNrJ3mHDx~t=(J8(Q5=4)K-WDAi~?tST&(it2A?Z}c#Oz1C00s*QnLiMvP~Hq zl0`6!wGO;EPb~jg;9^t*xg+gEc`#U6p>VtVQPC6;B>h$zv^OGzL&#`G@Oq8)1FP1b zn!E|$YYs^2E8KvyBwH|;Ne;G@qZfv#kIL-;^J*NfvP>F=E*ek3rQ0oieu02nu`%ta zU~8qxS@cR_i}MEvGG>kV#7xKY!tIbldG+w3d&0=_*fQ*S;WXA1IgnW>Vu#uW9LOnb z42m`^&=4q1+i9U5uv^8BsprCqRA7FwGCS>wZ#kJpVyruxhOj*~&H|mwtdzq7gVSS7 zf=)x&_R#7xWTLe3teTfCYfaNnqk(%Y`@8cN0)wk(@xQ>K)kg)m{c%4p#6nVsL~uWY z6<#7*(rf`IXPLs{$*gr(7={oaMG8yyLP!QfMTYL=jQPP16PH~J9{?~u6^5ypP>5^ zLHgv{+KqVe+%CKzJwShN_Un_Ai$4bK&)Zr3=XRFN0}?%Q=($HM&>6tAyU8G#NPCyS z@}}@P^MW)v@HARn5u#h%k;ZS%OA%Ex|N29Se8=567l*ov!@Vx9(urtfFieAkmmDa% zojM<+O7+RrjI);yM!Zki;{}7Ut&~0*m3}=9F9jZfTO)^-8PdIAH4mV88H)o=DiOOA zbcuhk64?#^6qwx=yMr}4}z|M+6IGmEUkkNp{FYd-u z7ka^h=+%0XPyD0t&ySp%9I_aK(2fur2OcoDimgV{o14KK2E@ zC;}*@*z7OC;!+>jc!AR0tRWQWox42B$X{S()u6T=;RF|T^exH0V9;-!$&<)Ngp$kS zN7GrH$ntv#JOqsa$ZG6QME*qihq`v2nJs}z7>}<-kj)(Ly?S94-QsX*o{+ZEiXPhK z)!74dYrwb+Ru|U^I@DQe@u}zq!SB91vGO zI&*eSjPYD^zSvf=g>lJ33d(^Y4j0qLb|I~78$#+XkcA_oicXDN^Yel)ias;*AL(qMvf5Q1>B>N8u)lX4h;Po|vP&vmvWm0?W8& zR}LwfR=KlH5rnL^)=>#2vD!{XuNSOkR=XeEy2BAOCYcWR86u}VspSp--A!T3v`VX> z+^%3SV$6Jt5+vVh(LZOyQw+uaI9lExg-Zt5dkjAzjyqc?EfmpYBS`Ifx6eS{hS?PX zHjUbiw6Qinxs~*QN_}!MCxu+-n>#Lp3Ob>~W=X_O7sJIMxETomVgF1QuH^7-)haIj62C_I@uwSspoNHM@Q>J0bat_Bu|d;^!d)cET1eq8Kz-9BNa_WH z*l#Vwu*h#D32>2hOOmt>7guB(s^vP|slZ<4R2-m@N$GqFRJsQ+beGK}9m!%D1UJcC zuuB+nHU(}9jfwKj21|cm4ZR?h*S$;4b%g{_5yxim1>eF_SZv-(xvRvL^=f`om8)!s zD`rATD4a|+f5BpN@wtl3-FOL_J-Gs-{3iBr1Rl}8S)1OSS4(~))Q#YwD4MEO?ymla zX*6cLvmMf>Q;K)yZw2bqviDygwGGC%()k35+5a+`?*Vd|LvZqVr>qCSN_Bn8`0OFU zcDWUZ0gpnN?jAZwMPFMls30&Quu}nS>nM!TzMG0dNZqzXcx&XQHOyyI1$sN2Zd87x z_?*qszwCx79LqifX_OAkrH)UJ;ft~S2$1>A7o2{fSb16>}ud+vtpQSyhv4nI@UY{cCY^ zc`xHjuj~YpaXpkIr0`^tv!f&PwrhKs`wU&%z;rQh%N)5t=Kt43O`5384k;XZrfg$N z5a!azYUK!TF*c8_g-;i+p1fw@i*M_3s zk%~mZ?m8}vo3y9D=1&8s_87xSxy!;Ki&o)6fzcPdvp0(wihHrll1hrf!LE0J@=Dr# zlZgtpu6(DjL)UHliqWPUfbYH_iH9Y8{basAC*J4^^HxyZ_9J(%% z0kVMRx((gS&QF!TO1We@;0sC6lCPWl8X1{fm^glNOOPht8pGa84Cka5n7#cpM^o`I z!fFn5$d`qzGm+5x;5l zAg9|=ycEP(`Y_!S=6(GQp@dQVR%qOA*CXkJH`MA3QgapB0^LP{97>BB(|>?urS;yoR+l{j+va-XvDp4pa-Z(X@K! z>abVhB_Gy}b9UTgoCK#tWAnx=)q@}#Pez881Zk`j{79BY%GMtW(KuUq$&oW{H*Vdm zhS=NA`+{3sYF8uqn9{Oak|2YNaSrRfdz)52iUg3FI6=iGP<;_EvmQqy_mJ;nt4MN5 zK8a}rzIk05+aRn-$&vXBQ*fWXnFQB0P)A?XGLu)ddA})t$Sb$w3hwj5u0YkGcjzXq zDa3_);0uJ11z)<9+}9B5w=eQqP}6nKiG)$pC+|*S5fzltjv{8i&X{4$h-)$ze;j4R zCnRjT{X8a@bSRwf`{;5%lb5GNCV>} z9rxCG*o|#8vPlVozc_@*WnMYrT{29rq?m7o@?E5qz+JcX@o`I?ppIzs_dv$Au+TPM zpngUkwQRtU%d@Z@?fI<(g6sgeN%P^4FaSK)eep)ut`zI1J4RO6bv<*FCkv2CkGNfqeZm)-8NyWxGH%CG4|x zhd*WD8z1X^MrJU;g*qdQrwNYW^}Y<&M*dKSd>}hKmNM6De&EQDpWf`3THfRX4_jr{ z$5Hd*8KC?aKE-!QMu9&Fe`u#Vllwg2A671ZHHv0i?PpJGF5MstY58QbJYGI-B&dL` z54n226GkF0e#m^}vop?Ktm$WJLyBOMR7VJnyt$2Ps%NR#;RFWh#uuwYokHg<{WzFjZ>12|qtSl&xZ~GfF#P>5|Htnd4z||JxBuY*jlWPb@9*3+ zCGE24}{2u)j15SYVK*#mKXz;Og0@$=TLxer5!6CqAF#rK zELChxXL~}{dnCcUE6%=Jf{eJ3qwzsz{m( literal 0 HcmV?d00001 diff --git a/lucene/test-framework/src/resources/org/apache/lucene/tests/geo/github-13841-3.wkt.gz b/lucene/test-framework/src/resources/org/apache/lucene/tests/geo/github-13841-3.wkt.gz new file mode 100644 index 0000000000000000000000000000000000000000..34f3f6dde22581dcaa23342ad303a2b732bdb497 GIT binary patch literal 2270 zcmV<42qE_$iwFohup?&x17~S;Xmw&OF*7(cF)cGLXJv0{b8l_{m6%Je-?t5gx8k7< zq=7__j)QSdDV_GvPYTY{Kfu|0U{fe^kxpXa>Wp}8+zO!6Z+S&CiR@po6!^7bd0BDr#56H$yZv7W%5Caok|kE zWCt8x`8DC=rgt~rHXV%QeJ0XtcA0pO=sKt>Qz0iOGykZeE^6*Y-|3mX$EOIlN$Q&? zq4-F?!riq}rc>5L;<2*b59+OF%p;OV_hduDLIRi)u{@g$@o;6Yz2255uT_O1KQg7! zvd<)pMI}_O(6MixB!y;yzV^GwoNPI={3utT2wf*n@}DF&S?*b7A}!`jw^L-6lqWq+ zb<~^GY!D_3E^Jm#LN{eV@24d>D7;PXu#OXzKt@vLTts50QxvMT3vN-ej?pwlZzBsH+deCoU9v z?i8>fa-lt1^g{{FwK5I5<&+S$vzS37>S!MoGU!kOxWF?KMzzsEv7)LaN=3aINvJ!O z7tw`HiXHqQbTXu-gtNM6413{hfYDgm(ot0CPb}T~_MCsXEDA@VI@%{4&lAL*b@FKZ z;?J}4h$^*RD9J9v93~`fdQ78Pq~k2p;6s^6R6Gw68%dDvBc}Rvqpcwu2wCrPF0>eu z5G6@*VAL5$=q0F1okMdHdKAb7DfX(P?Q_|rGz;CL(qbN!XGm66YtV-Og;H)Lwng1a zlx|)UIGJ-DsI^mPS2UnR=PJ34(ZhvOla>CAgH4KIcohZQ=OuS&8^DN&w$QaTJz(0A z6${OW60NgQ2&7z;!4$n)M?dN!HCGQ%I#lU3J_6RgUTB+jG#0EvDFKO6S!VfhFhL2m zyX&$+9~VlQE4if65*owkk8;nfoQ%muI`k+UWv|cl*rML=Tj*($U`B{P-fhx>60=XC zcdSCsD!H_TBNSn!FS?ZkI9(FhuS6olN%5qJ*;wy5xtJGAN0yJl>jEp^GQ`Im%z|l zV7CuS*2{|wY|7t7Q!bCsS$vL5ued)$vmepR5+XFVVvVUZ0hvo_QgvMHQYO?ZV$KMl zyL_06D-FQwRGB!7%Cr$+8rigP?d($csPd>WsLrKXVJ`PR(juCfwCy9Pg2 z$EFG&>+IBUGb_9#N!Jq!jbp3xFa+JE3Gc?US)%m%n)_IyO9Jz@=2To;RG{f@(}*`1 z+(f(^_9SX(yo#hQ3Gpzv+mQn!aadVc)d zyXv=}{^xTmwE&Xp-3pRww7*d+^`yrVB$CDBnYhbj?&{Xp?J8_6B3+fAI4q6p_}8XP z+tKwT0Y={cSKk|(Yp(9(;wLTHaB@RJ+pU+7c=E8GuVkiohnNdT?$3=u+3LX Date: Tue, 12 Nov 2024 10:38:54 +0100 Subject: [PATCH 34/37] Simplify codec setup in vector-related tests. (#13970) Many of vector-related tests set up a codec manually by extending the current codec. This makes bumping the current codec a bit painful as all these files need to be touched. This commit migrates to `TestUtil#alwaysKnnVectorsFormat`, similarly to what we do for postings and doc values. --- ...ene99HnswScalarQuantizedVectorsFormat.java | 10 +--- .../TestInt7HnswBackwardsCompatibility.java | 12 +--- .../bitvectors/TestHnswBitVectorsFormat.java | 9 +-- ...estLucene99HnswQuantizedVectorsFormat.java | 54 +++--------------- ...stLucene99ScalarQuantizedVectorScorer.java | 13 ++--- ...tLucene99ScalarQuantizedVectorsFormat.java | 9 +-- .../org/apache/lucene/index/TestKnnGraph.java | 34 +++--------- .../search/BaseKnnVectorQueryTestCase.java | 17 +----- .../lucene/util/hnsw/HnswGraphTestCase.java | 55 ++----------------- .../apache/lucene/tests/util/TestUtil.java | 19 +++++++ 10 files changed, 54 insertions(+), 178 deletions(-) diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java index 6b89cb6b8a4..268d231c4f4 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java @@ -18,18 +18,12 @@ package org.apache.lucene.backward_codecs.lucene99; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.KnnVectorsFormat; -import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.tests.index.BaseKnnVectorsFormatTestCase; +import org.apache.lucene.tests.util.TestUtil; public class TestLucene99HnswScalarQuantizedVectorsFormat extends BaseKnnVectorsFormatTestCase { @Override protected Codec getCodec() { - return new Lucene101Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene99RWHnswScalarQuantizationVectorsFormat(); - } - }; + return TestUtil.alwaysKnnVectorsFormat(new Lucene99RWHnswScalarQuantizationVectorsFormat()); } } diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt7HnswBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt7HnswBackwardsCompatibility.java index 402747a3138..384ce889c5a 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt7HnswBackwardsCompatibility.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt7HnswBackwardsCompatibility.java @@ -21,9 +21,7 @@ import static org.apache.lucene.backward_index.TestBasicBackwardsCompatibility.a import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import java.io.IOException; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; -import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader; @@ -69,14 +67,10 @@ public class TestInt7HnswBackwardsCompatibility extends BackwardsCompatibilityTe } protected Codec getCodec() { - return new Lucene101Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene99HnswScalarQuantizedVectorsFormat( + return TestUtil.alwaysKnnVectorsFormat( + new Lucene99HnswScalarQuantizedVectorsFormat( Lucene99HnswVectorsFormat.DEFAULT_MAX_CONN, - Lucene99HnswVectorsFormat.DEFAULT_BEAM_WIDTH); - } - }; + Lucene99HnswVectorsFormat.DEFAULT_BEAM_WIDTH)); } @Override diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java index 22e0eff62c1..388f0879256 100644 --- a/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java +++ b/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java @@ -22,7 +22,6 @@ import java.io.IOException; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.KnnVectorsFormat; -import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.KnnByteVectorField; @@ -38,16 +37,12 @@ import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopKnnCollector; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.BaseIndexFileFormatTestCase; +import org.apache.lucene.tests.util.TestUtil; public class TestHnswBitVectorsFormat extends BaseIndexFileFormatTestCase { @Override protected Codec getCodec() { - return new Lucene101Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new HnswBitVectorsFormat(); - } - }; + return TestUtil.alwaysKnnVectorsFormat(new HnswBitVectorsFormat()); } @Override diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java index 8b6efa0e41f..d2aa9b8d054 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java @@ -28,7 +28,6 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; -import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.KnnFloatVectorField; @@ -48,6 +47,7 @@ import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopKnnCollector; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.BaseKnnVectorsFormatTestCase; +import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.SameThreadExecutorService; import org.apache.lucene.util.VectorUtil; import org.apache.lucene.util.quantization.QuantizedByteVectorValues; @@ -74,12 +74,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat @Override protected Codec getCodec() { - return new Lucene101Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return format; - } - }; + return TestUtil.alwaysKnnVectorsFormat(format); } private final KnnVectorsFormat getKnnFormat(int bits) { @@ -104,14 +99,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat try (IndexWriter w = new IndexWriter( dir, - newIndexWriterConfig() - .setCodec( - new Lucene101Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return getKnnFormat(4); - } - }))) { + newIndexWriterConfig().setCodec(TestUtil.alwaysKnnVectorsFormat(getKnnFormat(4))))) { Document doc = new Document(); doc.add( @@ -124,14 +112,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat try (IndexWriter w = new IndexWriter( dir, - newIndexWriterConfig() - .setCodec( - new Lucene101Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return getKnnFormat(7); - } - }))) { + newIndexWriterConfig().setCodec(TestUtil.alwaysKnnVectorsFormat(getKnnFormat(7))))) { Document doc = new Document(); doc.add( @@ -162,13 +143,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat new IndexWriter( dir, newIndexWriterConfig() - .setCodec( - new Lucene101Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene99HnswVectorsFormat(); - } - }))) { + .setCodec(TestUtil.alwaysKnnVectorsFormat(new Lucene99HnswVectorsFormat())))) { Document doc = new Document(); doc.add( @@ -181,14 +156,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat try (IndexWriter w = new IndexWriter( dir, - newIndexWriterConfig() - .setCodec( - new Lucene101Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return getKnnFormat(7); - } - }))) { + newIndexWriterConfig().setCodec(TestUtil.alwaysKnnVectorsFormat(getKnnFormat(7))))) { Document doc = new Document(); doc.add( @@ -216,13 +184,9 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat dir, newIndexWriterConfig() .setCodec( - new Lucene101Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene99HnswScalarQuantizedVectorsFormat( - 16, 100, 1, (byte) 7, false, 0.9f, null); - } - }))) { + TestUtil.alwaysKnnVectorsFormat( + new Lucene99HnswScalarQuantizedVectorsFormat( + 16, 100, 1, (byte) 7, false, 0.9f, null))))) { for (float[] vector : vectors) { Document doc = new Document(); doc.add(new KnnFloatVectorField("f", vector, VectorSimilarityFunction.DOT_PRODUCT)); diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java index 6cd249240c0..3ffeef501e8 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java @@ -24,10 +24,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer; -import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; @@ -44,6 +42,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.VectorUtil; import org.apache.lucene.util.hnsw.RandomVectorScorer; import org.apache.lucene.util.quantization.QuantizedByteVectorValues; @@ -52,19 +51,15 @@ import org.apache.lucene.util.quantization.ScalarQuantizer; public class TestLucene99ScalarQuantizedVectorScorer extends LuceneTestCase { private static Codec getCodec(int bits, boolean compress) { - return new Lucene101Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene99HnswScalarQuantizedVectorsFormat( + return TestUtil.alwaysKnnVectorsFormat( + new Lucene99HnswScalarQuantizedVectorsFormat( Lucene99HnswVectorsFormat.DEFAULT_MAX_CONN, Lucene99HnswVectorsFormat.DEFAULT_BEAM_WIDTH, 1, bits, compress, 0f, - null); - } - }; + null)); } public void testNonZeroScores() throws IOException { diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java index 86a5ff5068d..cf1436f2109 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java @@ -28,7 +28,6 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; -import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.KnnFloatVectorField; @@ -43,6 +42,7 @@ import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.VectorSimilarityFunction; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.BaseKnnVectorsFormatTestCase; +import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.VectorUtil; import org.apache.lucene.util.quantization.QuantizedByteVectorValues; import org.apache.lucene.util.quantization.ScalarQuantizer; @@ -70,12 +70,7 @@ public class TestLucene99ScalarQuantizedVectorsFormat extends BaseKnnVectorsForm @Override protected Codec getCodec() { - return new Lucene101Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return format; - } - }; + return TestUtil.alwaysKnnVectorsFormat(format); } public void testSearch() throws Exception { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java b/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java index 5def0a26d84..a3d655ebe3b 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java @@ -30,8 +30,6 @@ import java.util.List; import java.util.Set; import java.util.concurrent.CountDownLatch; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.FilterCodec; -import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader; @@ -85,33 +83,15 @@ public class TestKnnGraph extends LuceneTestCase { vectorEncoding = randomVectorEncoding(); boolean quantized = randomBoolean(); codec = - new FilterCodec(TestUtil.getDefaultCodec().getName(), TestUtil.getDefaultCodec()) { - @Override - public KnnVectorsFormat knnVectorsFormat() { - return new PerFieldKnnVectorsFormat() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return quantized - ? new Lucene99HnswScalarQuantizedVectorsFormat( - M, HnswGraphBuilder.DEFAULT_BEAM_WIDTH) - : new Lucene99HnswVectorsFormat(M, HnswGraphBuilder.DEFAULT_BEAM_WIDTH); - } - }; - } - }; + TestUtil.alwaysKnnVectorsFormat( + quantized + ? new Lucene99HnswScalarQuantizedVectorsFormat( + M, HnswGraphBuilder.DEFAULT_BEAM_WIDTH) + : new Lucene99HnswVectorsFormat(M, HnswGraphBuilder.DEFAULT_BEAM_WIDTH)); float32Codec = - new FilterCodec(TestUtil.getDefaultCodec().getName(), TestUtil.getDefaultCodec()) { - @Override - public KnnVectorsFormat knnVectorsFormat() { - return new PerFieldKnnVectorsFormat() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene99HnswVectorsFormat(M, HnswGraphBuilder.DEFAULT_BEAM_WIDTH); - } - }; - } - }; + TestUtil.alwaysKnnVectorsFormat( + new Lucene99HnswVectorsFormat(M, HnswGraphBuilder.DEFAULT_BEAM_WIDTH)); } private VectorEncoding randomVectorEncoding() { diff --git a/lucene/core/src/test/org/apache/lucene/search/BaseKnnVectorQueryTestCase.java b/lucene/core/src/test/org/apache/lucene/search/BaseKnnVectorQueryTestCase.java index afa150e387f..4abd955eeb6 100644 --- a/lucene/core/src/test/org/apache/lucene/search/BaseKnnVectorQueryTestCase.java +++ b/lucene/core/src/test/org/apache/lucene/search/BaseKnnVectorQueryTestCase.java @@ -50,7 +50,6 @@ import org.apache.lucene.search.knn.KnnCollectorManager; import org.apache.lucene.search.knn.TopKnnCollectorManager; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.analysis.MockAnalyzer; -import org.apache.lucene.tests.codecs.asserting.AssertingCodec; import org.apache.lucene.tests.index.RandomIndexWriter; import org.apache.lucene.tests.store.BaseDirectoryWrapper; import org.apache.lucene.tests.util.LuceneTestCase; @@ -1084,13 +1083,7 @@ abstract class BaseKnnVectorQueryTestCase extends LuceneTestCase { IndexWriterConfig iwc = newIndexWriterConfig(mockAnalyzer); KnnVectorsFormat format1 = randomVectorFormat(VectorEncoding.FLOAT32); KnnVectorsFormat format2 = randomVectorFormat(VectorEncoding.FLOAT32); - iwc.setCodec( - new AssertingCodec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return format1; - } - }); + iwc.setCodec(TestUtil.alwaysKnnVectorsFormat(format1)); try (IndexWriter iwriter = new IndexWriter(directory, iwc)) { Document doc = new Document(); @@ -1104,13 +1097,7 @@ abstract class BaseKnnVectorQueryTestCase extends LuceneTestCase { } iwc = newIndexWriterConfig(mockAnalyzer); - iwc.setCodec( - new AssertingCodec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return format2; - } - }); + iwc.setCodec(TestUtil.alwaysKnnVectorsFormat(format2)); try (IndexWriter iwriter = new IndexWriter(directory, iwc)) { Document doc = new Document(); diff --git a/lucene/core/src/test/org/apache/lucene/util/hnsw/HnswGraphTestCase.java b/lucene/core/src/test/org/apache/lucene/util/hnsw/HnswGraphTestCase.java index 41aeef2e5c8..1da8c8169a9 100644 --- a/lucene/core/src/test/org/apache/lucene/util/hnsw/HnswGraphTestCase.java +++ b/lucene/core/src/test/org/apache/lucene/util/hnsw/HnswGraphTestCase.java @@ -38,8 +38,6 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; -import org.apache.lucene.codecs.FilterCodec; -import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsReader; @@ -152,19 +150,7 @@ abstract class HnswGraphTestCase extends LuceneTestCase { IndexWriterConfig iwc = new IndexWriterConfig() .setCodec( - new FilterCodec( - TestUtil.getDefaultCodec().getName(), TestUtil.getDefaultCodec()) { - - @Override - public KnnVectorsFormat knnVectorsFormat() { - return new PerFieldKnnVectorsFormat() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene99HnswVectorsFormat(M, beamWidth); - } - }; - } - }) + TestUtil.alwaysKnnVectorsFormat(new Lucene99HnswVectorsFormat(M, beamWidth))) // set a random merge policy .setMergePolicy(newMergePolicy(random())); try (IndexWriter iw = new IndexWriter(dir, iwc)) { @@ -255,18 +241,7 @@ abstract class HnswGraphTestCase extends LuceneTestCase { IndexWriterConfig iwc = new IndexWriterConfig() .setCodec( - new FilterCodec( - TestUtil.getDefaultCodec().getName(), TestUtil.getDefaultCodec()) { - @Override - public KnnVectorsFormat knnVectorsFormat() { - return new PerFieldKnnVectorsFormat() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene99HnswVectorsFormat(M, beamWidth); - } - }; - } - }); + TestUtil.alwaysKnnVectorsFormat(new Lucene99HnswVectorsFormat(M, beamWidth))); try (IndexWriter iw = new IndexWriter(dir, iwc)) { KnnVectorValues.DocIndexIterator it2 = v2.iterator(); while (it2.nextDoc() != NO_MORE_DOCS) { @@ -317,32 +292,10 @@ abstract class HnswGraphTestCase extends LuceneTestCase { HnswGraphBuilder.randSeed = seed; IndexWriterConfig iwc = new IndexWriterConfig() - .setCodec( - new FilterCodec(TestUtil.getDefaultCodec().getName(), TestUtil.getDefaultCodec()) { - @Override - public KnnVectorsFormat knnVectorsFormat() { - return new PerFieldKnnVectorsFormat() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene99HnswVectorsFormat(M, beamWidth); - } - }; - } - }); + .setCodec(TestUtil.alwaysKnnVectorsFormat(new Lucene99HnswVectorsFormat(M, beamWidth))); IndexWriterConfig iwc2 = new IndexWriterConfig() - .setCodec( - new FilterCodec(TestUtil.getDefaultCodec().getName(), TestUtil.getDefaultCodec()) { - @Override - public KnnVectorsFormat knnVectorsFormat() { - return new PerFieldKnnVectorsFormat() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new Lucene99HnswVectorsFormat(M, beamWidth); - } - }; - } - }) + .setCodec(TestUtil.alwaysKnnVectorsFormat(new Lucene99HnswVectorsFormat(M, beamWidth))) .setIndexSort(new Sort(new SortField("sortkey", SortField.Type.LONG))); try (Directory dir = newDirectory(); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java index 66ae7a03141..6715edecc16 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java @@ -1310,6 +1310,25 @@ public final class TestUtil { }; } + /** + * Return a Codec that can read any of the default codecs and formats, but always writes in the + * specified format. + */ + public static Codec alwaysKnnVectorsFormat(final KnnVectorsFormat format) { + // TODO: we really need for knn vectors impls etc to announce themselves + // (and maybe their params, too) to infostream on flush and merge. + // otherwise in a real debugging situation we won't know whats going on! + if (LuceneTestCase.VERBOSE) { + System.out.println("TestUtil: forcing knn vectors format to:" + format); + } + return new AssertingCodec() { + @Override + public KnnVectorsFormat getKnnVectorsFormatForField(String field) { + return format; + } + }; + } + /** * Returns the actual default codec (e.g. LuceneMNCodec) for this version of Lucene. This may be * different from {@link Codec#getDefault()} because that is randomized. From 8698dd85d867737d9ab721f54fa98a8f22f59968 Mon Sep 17 00:00:00 2001 From: Chris Hegarty <62058229+ChrisHegarty@users.noreply.github.com> Date: Thu, 14 Nov 2024 09:02:15 +0000 Subject: [PATCH 35/37] Allow easier verification of the Panama Vectorization provider with newer Java versions (#13986) This commit allows easier verification of the Panama Vectorization provider with newer Java versions. The upper bound Java version of the Vectorization provider is hardcoded to the version that has been tested and is known to work. This is a bit inflexible when experimenting with and verifying newer JDK versions. This change proposes to add a new system property that allows to set the upper bound of the range of Java versions supported. With this change, and the accompanying small gradle change, then one can verify newer JDKs as follows: CI=true; RUNTIME_JAVA_HOME=/Users/chegar/binaries/jdk-24.jdk-ea-b23/Contents/Home ./gradlew :lucene:core:test -Dorg.apache.lucene.vectorization.upperJavaFeatureVersion=24 This change helps both testing and verifying with Early Access JDK builds, as well as allowing to override the upper bound when the JDK is known to work fine. --- gradle/testing/defaults-tests.gradle | 8 +++++- lucene/CHANGES.txt | 6 ++++- .../vectorization/VectorizationProvider.java | 27 ++++++++++++++++++- 3 files changed, 38 insertions(+), 3 deletions(-) diff --git a/gradle/testing/defaults-tests.gradle b/gradle/testing/defaults-tests.gradle index 1f3a7d8b1a0..14e64647d66 100644 --- a/gradle/testing/defaults-tests.gradle +++ b/gradle/testing/defaults-tests.gradle @@ -128,8 +128,14 @@ allprojects { jvmArgs '--add-modules', 'jdk.management' // Enable the vector incubator module on supported Java versions: - if (rootProject.vectorIncubatorJavaVersions.contains(rootProject.runtimeJavaVersion)) { + def prop = propertyOrDefault("org.apache.lucene.vectorization.upperJavaFeatureVersion", "1") as String + def v = JavaVersion.toVersion(Integer.parseInt(prop)).majorVersion + if (rootProject.vectorIncubatorJavaVersions.contains(rootProject.runtimeJavaVersion) || + rootProject.runtimeJavaVersion.majorVersion <= v) { jvmArgs '--add-modules', 'jdk.incubator.vector' + if (rootProject.runtimeJavaVersion.majorVersion <= v) { + systemProperty 'org.apache.lucene.vectorization.upperJavaFeatureVersion', v + } } jvmArgs '--enable-native-access=' + (project.path in [ diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 1c4b5cbb249..062c107158b 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -47,7 +47,11 @@ New Features Improvements --------------------- -(No changes) + +* GITHUB#13986: Allow easier configuration of the Panama Vectorization provider with + newer Java versions. Set the `org.apache.lucene.vectorization.upperJavaFeatureVersion` + system property to increase the set of Java versions that Panama Vectorization will + provide optimized implementations for. (Chris Hegarty) Optimizations --------------------- diff --git a/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorizationProvider.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorizationProvider.java index e6441310dcc..c0ed905353b 100644 --- a/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorizationProvider.java +++ b/lucene/core/src/java/org/apache/lucene/internal/vectorization/VectorizationProvider.java @@ -38,12 +38,16 @@ import org.apache.lucene.util.VectorUtil; * vectorization modules in the Java runtime this class provides optimized implementations (using * SIMD) of several algorithms used throughout Apache Lucene. * + *

    Expert: set the {@value #UPPER_JAVA_FEATURE_VERSION_SYSPROP} system property to increase the + * set of Java versions this class will provide optimized implementations for. + * * @lucene.internal */ public abstract class VectorizationProvider { static final OptionalInt TESTS_VECTOR_SIZE; static final boolean TESTS_FORCE_INTEGER_VECTORS; + static final int UPPER_JAVA_FEATURE_VERSION = getUpperJavaFeatureVersion(); static { var vs = OptionalInt.empty(); @@ -71,6 +75,27 @@ public abstract class VectorizationProvider { TESTS_FORCE_INTEGER_VECTORS = enforce; } + private static final String UPPER_JAVA_FEATURE_VERSION_SYSPROP = + "org.apache.lucene.vectorization.upperJavaFeatureVersion"; + private static final int DEFAULT_UPPER_JAVA_FEATURE_VERSION = 23; + + private static int getUpperJavaFeatureVersion() { + int runtimeVersion = DEFAULT_UPPER_JAVA_FEATURE_VERSION; + try { + String str = System.getProperty(UPPER_JAVA_FEATURE_VERSION_SYSPROP); + if (str != null) { + runtimeVersion = Math.max(Integer.parseInt(str), runtimeVersion); + } + } catch (@SuppressWarnings("unused") NumberFormatException | SecurityException ignored) { + Logger.getLogger(VectorizationProvider.class.getName()) + .warning( + "Cannot read sysprop " + + UPPER_JAVA_FEATURE_VERSION_SYSPROP + + ", so the default value will be used."); + } + return runtimeVersion; + } + /** * Returns the default instance of the provider matching vectorization possibilities of actual * runtime. @@ -108,7 +133,7 @@ public abstract class VectorizationProvider { static VectorizationProvider lookup(boolean testMode) { final int runtimeVersion = Runtime.version().feature(); assert runtimeVersion >= 21; - if (runtimeVersion <= 23) { + if (runtimeVersion <= UPPER_JAVA_FEATURE_VERSION) { // only use vector module with Hotspot VM if (!Constants.IS_HOTSPOT_VM) { LOG.warning( From 300862bd20427a683e0dc6671a6e97815e5a5540 Mon Sep 17 00:00:00 2001 From: Lu Xugang Date: Sat, 16 Nov 2024 02:09:41 +0800 Subject: [PATCH 36/37] Remove duplicate test code (#13982) The only diff between doIterate1 and doIterate2 is bb < b.length() in doIterate1 , but this condition is always true, so we should remove it --- lucene/CHANGES.txt | 2 +- .../apache/lucene/util/TestFixedBitSet.java | 36 +++++-------------- 2 files changed, 9 insertions(+), 29 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 062c107158b..66f13d264e5 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -111,7 +111,7 @@ Build Other --------------------- -(No changes) +* GITHUB#13982: Remove duplicate test code. (Lu Xugang) ======================== Lucene 10.0.1 ======================= diff --git a/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java b/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java index e26aed408b7..67f0918f46b 100644 --- a/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java +++ b/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java @@ -98,26 +98,7 @@ public class TestFixedBitSet extends BaseBitSetTestCase { } // test interleaving different FixedBitSetIterator.next()/skipTo() - void doIterate(java.util.BitSet a, FixedBitSet b, int mode) throws IOException { - if (mode == 1) doIterate1(a, b); - if (mode == 2) doIterate2(a, b); - } - - void doIterate1(java.util.BitSet a, FixedBitSet b) throws IOException { - assertEquals(a.cardinality(), b.cardinality()); - int aa = -1, bb = -1; - DocIdSetIterator iterator = new BitSetIterator(b, 0); - do { - aa = a.nextSetBit(aa + 1); - bb = - (bb < b.length() && random().nextBoolean()) - ? iterator.nextDoc() - : iterator.advance(bb + 1); - assertEquals(aa == -1 ? DocIdSetIterator.NO_MORE_DOCS : aa, bb); - } while (aa >= 0); - } - - void doIterate2(java.util.BitSet a, FixedBitSet b) throws IOException { + void doIterate(java.util.BitSet a, FixedBitSet b) throws IOException { assertEquals(a.cardinality(), b.cardinality()); int aa = -1, bb = -1; DocIdSetIterator iterator = new BitSetIterator(b, 0); @@ -128,7 +109,7 @@ public class TestFixedBitSet extends BaseBitSetTestCase { } while (aa >= 0); } - void doRandomSets(int maxSize, int iter, int mode) throws IOException { + void doRandomSets(int maxSize, int iter) throws IOException { java.util.BitSet a0 = null; FixedBitSet b0 = null; @@ -181,7 +162,7 @@ public class TestFixedBitSet extends BaseBitSetTestCase { FixedBitSet bb = b.clone(); bb.flip(fromIndex, toIndex); - doIterate(aa, bb, mode); // a problem here is from flip or doIterate + doIterate(aa, bb); // a problem here is from flip or doIterate fromIndex = random().nextInt(sz / 2); toIndex = fromIndex + random().nextInt(sz - fromIndex); @@ -230,10 +211,10 @@ public class TestFixedBitSet extends BaseBitSetTestCase { assertEquals(a0.cardinality(), b0.cardinality()); assertEquals(a_or.cardinality(), b_or.cardinality()); - doIterate(a_and, b_and, mode); - doIterate(a_or, b_or, mode); - doIterate(a_andn, b_andn, mode); - doIterate(a_xor, b_xor, mode); + doIterate(a_and, b_and); + doIterate(a_or, b_or); + doIterate(a_andn, b_andn); + doIterate(a_xor, b_xor); assertEquals(a_and.cardinality(), b_and.cardinality()); assertEquals(a_or.cardinality(), b_or.cardinality()); @@ -250,8 +231,7 @@ public class TestFixedBitSet extends BaseBitSetTestCase { // larger testsuite. public void testSmall() throws IOException { final int iters = TEST_NIGHTLY ? atLeast(1000) : 100; - doRandomSets(atLeast(1200), iters, 1); - doRandomSets(atLeast(1200), iters, 2); + doRandomSets(atLeast(1200), iters); } // uncomment to run a bigger test (~2 minutes). From a0e1eeefebf1cf7df078c3478433d0a546d4c513 Mon Sep 17 00:00:00 2001 From: Viswanath Kuchibhotla Date: Sun, 17 Nov 2024 18:11:27 +0530 Subject: [PATCH 37/37] Suppressing "automatic module" warning for benchmark-jmh (#13997) via Viswanath Kuchibhotla --- lucene/benchmark-jmh/src/java/module-info.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/lucene/benchmark-jmh/src/java/module-info.java b/lucene/benchmark-jmh/src/java/module-info.java index d92164cfae1..019e08abce1 100644 --- a/lucene/benchmark-jmh/src/java/module-info.java +++ b/lucene/benchmark-jmh/src/java/module-info.java @@ -16,6 +16,9 @@ */ /** Lucene JMH benchmarks. */ + +// jmh.core is not modularized and causes a warning. Suppressing it until it is modularized. +@SuppressWarnings("requires-automatic") module org.apache.lucene.benchmark.jmh { requires jmh.core; requires jdk.unsupported;