diff --git a/gradle/generation/forUtil.gradle b/gradle/generation/forUtil.gradle index 7ef3b53fd928..b55fd0204fd3 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/gradle/testing/defaults-tests.gradle b/gradle/testing/defaults-tests.gradle index 1f3a7d8b1a07..14e64647d667 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 b512720bc9ca..8b7afae0ec6b 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 --------------------- @@ -84,6 +88,14 @@ 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) + +* GITHUB#13994: Speed up top-k retrieval of filtered conjunctions. + (Adrien Grand) + Bug Fixes --------------------- * GITHUB#13832: Fixed an issue where the DefaultPassageFormatter.format method did not format passages as intended @@ -92,6 +104,10 @@ 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) +* GITHUB#13990: Added filter to the toString() method of Knn[Float|Byte]VectorQuery + and DiversifyingChildren[Float|Byte]KnnVectorQuery. (Viswanath Kuchibhotla) Build --------------------- @@ -100,7 +116,7 @@ Build Other --------------------- -(No changes) +* GITHUB#13982: Remove duplicate test code. (Lu Xugang) ======================== Lucene 10.0.1 ======================= @@ -328,6 +344,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/backward-codecs/src/generated/checksums/generateForDeltaUtil912.json b/lucene/backward-codecs/src/generated/checksums/generateForDeltaUtil912.json new file mode 100644 index 000000000000..c8c19f1c3b50 --- /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 000000000000..e8c1881a925d --- /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 4607c21eb7ee..41057c95bbf3 100644 --- a/lucene/backward-codecs/src/java/module-info.java +++ b/lucene/backward-codecs/src/java/module-info.java @@ -37,6 +37,7 @@ 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 @@ 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 @@ 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 97dc23bc07bd..14ca88a98a51 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.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 @@ * *

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 b9ddb1227b14..dd2af3acdbb7 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 fe0378b388b1..f87ffc135860 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 @@ * 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 @@ void encodeDeltas(long[] longs, DataOutput out) throws IOException { } /** 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 @@ private static void decode5To16(PostingDecodingUtil pdu, long[] tmp, long[] long } } - 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 @@ private static void decode6To16(PostingDecodingUtil pdu, long[] tmp, long[] long } } - 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 @@ private static void decode7To16(PostingDecodingUtil pdu, long[] tmp, long[] long } } - 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 @@ private static void decode12To32(PostingDecodingUtil pdu, long[] tmp, long[] lon } } - 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 @@ private static void decode13To32(PostingDecodingUtil pdu, long[] tmp, long[] lon } } - 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 @@ private static void decode14To32(PostingDecodingUtil pdu, long[] tmp, long[] lon } } - 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 @@ private static void decode15To32(PostingDecodingUtil pdu, long[] tmp, long[] lon } } - 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 fd5afa235f73..9f38052b2eb1 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 @@ static int numBytes(int bitsPerValue) { 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 @@ static void decodeSlow(int bitsPerValue, PostingDecodingUtil pdu, long[] tmp, lo } } + 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 @@ static void decodeSlow(int bitsPerValue, PostingDecodingUtil pdu, long[] tmp, lo 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 @@ static void decode3(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IO } } - 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 @@ static void decode5(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IO } } - 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 @@ static void decode6(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IO } } - 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 @@ static void decode7(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IO } } - 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 @@ static void decode9(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws IO } } - 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 @@ static void decode10(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 @@ static void decode11(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 @@ static void decode12(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 @@ static void decode13(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 @@ static void decode14(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 @@ static void decode15(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 @@ static void decode17(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 @@ static void decode18(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 @@ static void decode19(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 @@ static void decode20(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 @@ static void decode21(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 @@ static void decode22(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 @@ static void decode23(PostingDecodingUtil pdu, long[] tmp, long[] longs) throws I } } - 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 d7b89d31081c..e455c2cc6a30 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.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 1c452175b070..a0342635d766 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.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 @@ * * @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 1a2fe05679e9..a51c848c4cce 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; - -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; +package org.apache.lucene.backward_codecs.lucene912; + +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.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.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 @@ */ 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 @@ private abstract class AbstractPostingsEnum extends PostingsEnum { 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 @@ protected void resetIndexInput(IntBlockTermState termState) throws IOException { if (docIn == null) { // lazy init docIn = Lucene912PostingsReader.this.docIn.clone(); - docInUtil = VECTORIZATION_PROVIDER.newPostingDecodingUtil(docIn); } prefetchPostings(docIn, termState); } @@ -442,7 +434,7 @@ public PostingsEnum reset(IntBlockTermState termState, int flags) throws IOExcep 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 BytesRef getPayload() { 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 int advance(int target) throws IOException { } } - int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); this.doc = (int) docBuffer[next]; docBufferUpto = next + 1; return doc; @@ -633,9 +625,7 @@ final class EverythingEnum extends AbstractPostingsEnum { 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 EverythingEnum(FieldInfo fieldInfo) throws IOException { 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 @@ private void refillDocs() throws IOException { 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 int advance(int target) throws IOException { refillDocs(); } - int next = VECTOR_SUPPORT.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; @@ -1044,11 +1031,11 @@ private void refillPositions() throws IOException { } 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 @@ private void refillPositions() throws IOException { 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 @@ private abstract class BlockImpactsEnum extends ImpactsEnum { 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 @@ private abstract class BlockImpactsEnum extends ImpactsEnum { 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 BlockImpactsDocsEnum(boolean indexHasPos, IntBlockTermState termState) 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 @@ private void refillDocs() throws IOException { 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 int advance(int target) throws IOException { needsRefilling = false; } - int next = VECTOR_SUPPORT.findNextGEQ(docBuffer, docBufferSize, target, docBufferUpto); + int next = findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); this.doc = (int) docBuffer[next]; docBufferUpto = next + 1; return doc; @@ -1450,7 +1435,6 @@ final class BlockImpactsPostingsEnum extends BlockImpactsEnum { private int posBufferUpto; final IndexInput posIn; - final PostingDecodingUtil posInUtil; final boolean indexHasFreq; final boolean indexHasOffsets; @@ -1491,7 +1475,6 @@ public BlockImpactsPostingsEnum(FieldInfo fieldInfo, IntBlockTermState termState 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 @@ private void refillDocs() throws IOException { 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 int advance(int target) throws IOException { needsRefilling = false; } - int next = VECTOR_SUPPORT.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; @@ -1729,7 +1712,7 @@ private void refillPositions() throws IOException { } } } else { - pforUtil.decode(posInUtil, posDeltaBuffer); + pforUtil.decode(posIn, posDeltaBuffer); } } @@ -1754,9 +1737,6 @@ public int nextPosition() throws IOException { } } - /** - * @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 @@ static int readVInt15(DataInput in) throws IOException { } } - /** - * @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 @@ static long readVLong15(DataInput in) throws IOException { } } + 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 to; + } + 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 063f385ba42b..a075e42ec361 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 @@ void encode(long[] longs, DataOutput out) throws IOException { } /** 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 1ae808d308fb..8f526f7ef104 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 134c17c0b8f2..90604ee75cca 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 @@ * 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 @@ * 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 @@ } /** 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 @@ def writeDecode(bpv, f): /** * 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 @@ def writeDecode(bpv, f): 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 9851ec99a610..29543ffe671b 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 @@ * 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 @@ * 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 @@ 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 @@ } } + 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 @@ def writeDecode(bpv, f): 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 @@ def writeDecode(bpv, f): 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 aac717a3e6c1..76666469faaa 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. - * - *

- * - *

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: - * - *

- * - *
- * - *

Index Structure Overview

- * - *
- * - *

Each segment index maintains the following: - * - *

- * - *

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: - * - *

- * - * - * - *

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 d540abc85c28..bb7764da8e86 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.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 a4638b5fcc74..ff4d7eeda4e9 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 9733362abe79..49d917dc4273 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 b0da9347700e..a3660b2ac0ff 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 000000000000..af1037432afa --- /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 3363b8a22028..471333b20f46 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 void testEncodeDecode() throws IOException { { // 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 000000000000..e728cb9e50d3 --- /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 1b8d0618c601..adf8aaf9ec76 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 000000000000..f3d550dff4d3 --- /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 870f6e94e98c..869345ed4f8e 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 01d5b4ab17fc..268d231c4f49 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,17 +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.tests.index.BaseKnnVectorsFormatTestCase; +import org.apache.lucene.tests.util.TestUtil; public class TestLucene99HnswScalarQuantizedVectorsFormat extends BaseKnnVectorsFormatTestCase { @Override protected Codec getCodec() { - return new Lucene99Codec() { - @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 7e4b59542fa0..384ce889c5a1 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,9 +20,7 @@ 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.lucene99.Lucene99HnswScalarQuantizedVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; @@ -69,14 +67,10 @@ public static Iterable testVersionsFactory() throws IllegalAccessExcep } protected Codec getCodec() { - return new Lucene99Codec() { - @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/benchmark-jmh/src/java/module-info.java b/lucene/benchmark-jmh/src/java/module-info.java index d92164cfae1d..019e08abce1b 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; 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 7046e10c06e5..784ace1ae321 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 @@ }) 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 void binarySearch() { } @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 void inlinedBranchlessBinarySearch() { } @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 void linearSearch() { } @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) { @@ -142,13 +142,13 @@ private static int linearSearch(long[] values, long target, int startIndex) { @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(long[] values, long target, int startIndex) { - return VectorUtil.findNextGEQ(values, 128, target, startIndex); + private static int vectorUtilSearch(int[] values, int target, int startIndex) { + return VectorUtil.findNextGEQ(values, target, startIndex, 128); } private static void assertEquals(int expected, int actual) { @@ -159,7 +159,7 @@ private static void assertEquals(int expected, int actual) { 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 4ebab2f40241..241b289c5f61 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.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/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 a9465e5d12fb..9940196ce479 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 int doLogic() throws Exception { // 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/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/BlockTreeOrdsPostingsFormat.java index db7041485739..29854aa500f9 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.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 String toString() { @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 FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException @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 8e17edb9e2cc..5af23fb49455 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 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 @@ // - 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 DirectPostingsFormat(int minSkipCount, int lowFreqCutoff) { @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 b184f85b176d..4893ee8ad265 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.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 String toString() { @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 FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException @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 1f876e5e9d14..140d62b4e967 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 @@ /** * {@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 long getBasePayStartFP() { /** * 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 void writeTermState( /** * 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 c8a19bf9da94..690eab214003 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.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 @@ protected UniformSplitPostingsFormat( @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 FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException @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 dc77bc710a1c..d31b28704ef7 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 a0ea5833e2ee..388f08792565 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 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.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.KnnByteVectorField; @@ -38,16 +37,12 @@ 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 Lucene100Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return new HnswBitVectorsFormat(); - } - }; + return TestUtil.alwaysKnnVectorsFormat(new HnswBitVectorsFormat()); } @Override 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 51891b9d0adc..0708f3b8050b 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 bb88ff103244..5cc8a3fe5a0e 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 e862c737e5d2..6f61f8fc2c75 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 6fd1767aa34e..85aff5722498 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 @@ 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 @@ 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 @@ 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 ff5a5bb21c02..50974d13ff34 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 @@ static NamedSPILoader getLoader() { 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 000000000000..2fe9a1cce6fd --- /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 000000000000..1fe54b56fd50 --- /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 000000000000..2b764b876856 --- /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 000000000000..e228f1090ab8 --- /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 000000000000..ce32e8534249 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene101/Lucene101PostingsReader.java @@ -0,0 +1,1862 @@ +/* + * 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.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; +import org.apache.lucene.util.VectorUtil; + +/** + * 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(); + // 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 = VectorUtil.findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); + 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 = VectorUtil.findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); + 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 = VectorUtil.findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); + 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 = VectorUtil.findNextGEQ(docBuffer, target, docBufferUpto, docBufferSize); + 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 000000000000..788a5515f2d1 --- /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 000000000000..fd8ecd056b49 --- /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 19bae4770c5b..59388e3446b9 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 PostingIndexInput(IndexInput in, ForUtil forUtil, ForDeltaUtil forDeltaUt 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 000000000000..34431a3689fb --- /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 000000000000..240c66530ae7 --- /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 000000000000..0af17974532f --- /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 64189bfa9d11..e582f12c3185 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/geo/Tessellator.java b/lucene/core/src/java/org/apache/lucene/geo/Tessellator.java index f1093c3e8b04..a2b8cad84ff9 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.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 static List tessellate( * 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 @@ private static final Node createDoublyLinkedList( 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 @@ private static final Node eliminateHoles(final XYPolygon polygon, Node outerNode 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 @@ private static final Node eliminateHoles(final Polygon polygon, Node outerNode) 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 @@ private static final Node eliminateHoles( } /** 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 @@ private static final void eliminateHole( } } + /** + * 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 @@ && isLocallyInside(p, holeNode)) { 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 @@ private static final Node fetchLeftmost(final Node start) { * 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 @@ private static final List earcutLinkedList( } /** 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 @@ && area( * 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 @@ && area( } /** 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 @@ && isIntersectingPolygon(a, a.getX(), a.getY(), b.getX(), b.getY()) == false) { * 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 @@ private static void checkIntersection(Node a, boolean isMorton) { * 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 @@ private static boolean isEdgeFromPolygon(final Node a, final Node b, final boole * 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 @@ private static boolean isPointInLine( } /** 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 @@ private static double signedArea(final Node start, final Node end) { 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 @@ && area(a.getX(), a.getY(), a.previous.getX(), a.previous.getY(), b.getX(), b.ge } /** 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 @@ private static final boolean middleInsert( } /** 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 @@ private static final boolean isIntersectingPolygon( } /** 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 static final boolean linesIntersect( } /** 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 @@ private static final void sortByMortonWithReset(Node start) { } /** 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 @@ private static final void sortByMorton(Node start) { * 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 @@ private static final void tathamSort(Node list) { } /** 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 @@ && area( /** * 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 @@ private static final Node insertNode( } /** 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 @@ private static final void removeNode(Node node, boolean edgeFromPolygon) { } /** 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 @@ private static boolean pointInEar( && (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 @@ private static void notifyMonitor( } /** 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 @@ protected static class Node { 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 @@ protected static class Node { // 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 static final class Triangle { Node[] vertex; boolean[] edgeFromPolygon; - protected Triangle( + private Triangle( Node a, boolean isABfromPolygon, Node b, @@ -1636,19 +1675,6 @@ public boolean isEdgefromPolygon(int startVertex) { 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/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/DefaultVectorUtilSupport.java index c336babc9cc8..184403cf48b7 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 @@ public int squareDistance(byte[] a, byte[] b) { } @Override - public int findNextGEQ(long[] buffer, int length, long 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/PostingDecodingUtil.java b/lucene/core/src/java/org/apache/lucene/internal/vectorization/PostingDecodingUtil.java index 4dfa89422d0a..e45ce55bbc59 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 @@ protected PostingDecodingUtil(IndexInput in) { * 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 ac5b463e6c08..fb94b0e31736 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(long[] buffer, int length, long target, int from); + int findNextGEQ(int[] buffer, int target, int from, int to); } 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 2a6365297c85..c0ed905353b2 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 @@ * 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 static VectorizationProvider getInstance() { 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( @@ -190,8 +215,8 @@ private static Optional lookupVectorModule() { 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/AbstractKnnVectorQuery.java b/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java index ce7c1ea41012..4c47a6e75b62 100644 --- a/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java @@ -55,7 +55,7 @@ abstract class AbstractKnnVectorQuery extends Query { protected final String field; protected final int k; - private final Query filter; + protected final Query filter; public AbstractKnnVectorQuery(String field, int k, Query filter) { this.field = Objects.requireNonNull(field, "field"); 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 0d6daa07ba1e..b50b0530a2d1 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java @@ -268,6 +268,11 @@ public Query rewrite(IndexSearcher indexSearcher) throws IOException { return new MatchNoDocsQuery("empty BooleanQuery"); } + // Queries with no positive clauses have no matches + if (clauses.size() == clauseSets.get(Occur.MUST_NOT).size()) { + return new MatchNoDocsQuery("pure negative BooleanQuery"); + } + // optimize 1-clause queries if (clauses.size() == 1) { BooleanClause c = clauses.get(0); @@ -283,8 +288,6 @@ public Query rewrite(IndexSearcher indexSearcher) throws IOException { // no scoring clauses, so return a score of 0 return new BoostQuery(new ConstantScoreQuery(query), 0); case MUST_NOT: - // no positive clauses - return new MatchNoDocsQuery("pure negative BooleanQuery"); default: throw new AssertionError(); } @@ -539,8 +542,7 @@ public Query rewrite(IndexSearcher indexSearcher) throws IOException { builder.setMinimumNumberShouldMatch(minimumNumberShouldMatch); boolean actuallyRewritten = false; for (BooleanClause clause : clauses) { - if (clause.occur() == Occur.SHOULD && clause.query() instanceof BooleanQuery) { - BooleanQuery innerQuery = (BooleanQuery) clause.query(); + if (clause.occur() == Occur.SHOULD && clause.query() instanceof BooleanQuery innerQuery) { if (innerQuery.isPureDisjunction()) { actuallyRewritten = true; for (BooleanClause innerClause : innerQuery.clauses()) { @@ -558,6 +560,46 @@ public Query rewrite(IndexSearcher indexSearcher) throws IOException { } } + // Inline required / prohibited clauses. This helps run filtered conjunctive queries more + // efficiently by providing all clauses to the block-max AND scorer. + { + BooleanQuery.Builder builder = new BooleanQuery.Builder(); + builder.setMinimumNumberShouldMatch(minimumNumberShouldMatch); + boolean actuallyRewritten = false; + for (BooleanClause outerClause : clauses) { + if (outerClause.isRequired() && outerClause.query() instanceof BooleanQuery innerQuery) { + // Inlining prohibited clauses is not legal if the query is a pure negation, since pure + // negations have no matches. It works because the inner BooleanQuery would have first + // rewritten to a MatchNoDocsQuery if it only had prohibited clauses. + assert innerQuery.getClauses(Occur.MUST_NOT).size() != innerQuery.clauses().size(); + if (innerQuery.getMinimumNumberShouldMatch() == 0 + && innerQuery.getClauses(Occur.SHOULD).isEmpty()) { + + actuallyRewritten = true; + for (BooleanClause innerClause : innerQuery) { + Occur innerOccur = innerClause.occur(); + if (innerOccur == Occur.FILTER + || innerOccur == Occur.MUST_NOT + || outerClause.occur() == Occur.MUST) { + builder.add(innerClause); + } else { + assert outerClause.occur() == Occur.FILTER && innerOccur == Occur.MUST; + // In this case we need to change the occur of the inner query from MUST to FILTER. + builder.add(innerClause.query(), Occur.FILTER); + } + } + } else { + builder.add(outerClause); + } + } else { + builder.add(outerClause); + } + } + if (actuallyRewritten) { + return builder.build(); + } + } + // SHOULD clause count less than or equal to minimumNumberShouldMatch // Important(this can only be processed after nested clauses have been flattened) { 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 a8169ad227f1..7a53bc9a4852 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java @@ -333,10 +333,15 @@ private BulkScorer requiredBulkScorer() throws IOException { requiredScoring.add(ss.get(leadCost)); } if (scoreMode == ScoreMode.TOP_SCORES - && requiredNoScoring.isEmpty() && requiredScoring.size() > 1 // Only specialize top-level conjunctions for clauses that don't have a two-phase iterator. + && requiredNoScoring.stream().map(Scorer::twoPhaseIterator).allMatch(Objects::isNull) && requiredScoring.stream().map(Scorer::twoPhaseIterator).allMatch(Objects::isNull)) { + // Turn all filters into scoring clauses with a score of zero, so that + // BlockMaxConjunctionBulkScorer is applicable. + for (Scorer filter : requiredNoScoring) { + requiredScoring.add(new ConstantScoreScorer(0f, ScoreMode.COMPLETE, filter.iterator())); + } return new BlockMaxConjunctionBulkScorer(maxDoc, requiredScoring); } if (scoreMode != ScoreMode.TOP_SCORES 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 9c7274da3eb3..6e8bbf81966e 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.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 IndexSearcher(IndexReaderContext context, Executor executor) { 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 int count(Query query) throws IOException { * @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."); + } + } } /** @@ -564,10 +591,8 @@ public TopDocs searchAfter(ScoreDoc after, Query query, int numHits) throws IOEx } 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 +724,9 @@ private TopFieldDocs searchAfter( } 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) { @@ -1169,60 +1191,4 @@ public TooManyNestedClauses() { + 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; - } - } } diff --git a/lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java b/lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java index 9d6d71bc7a70..35144055830c 100644 --- a/lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java @@ -111,7 +111,14 @@ VectorScorer createVectorScorer(LeafReaderContext context, FieldInfo fi) throws @Override public String toString(String field) { - return getClass().getSimpleName() + ":" + this.field + "[" + target[0] + ",...][" + k + "]"; + StringBuilder buffer = new StringBuilder(); + buffer.append(getClass().getSimpleName() + ":"); + buffer.append(this.field + "[" + target[0] + ",...]"); + buffer.append("[" + k + "]"); + if (this.filter != null) { + buffer.append("[" + this.filter + "]"); + } + return buffer.toString(); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/search/KnnFloatVectorQuery.java b/lucene/core/src/java/org/apache/lucene/search/KnnFloatVectorQuery.java index 585893fa3c2a..d2aaf4296eda 100644 --- a/lucene/core/src/java/org/apache/lucene/search/KnnFloatVectorQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/KnnFloatVectorQuery.java @@ -112,7 +112,14 @@ VectorScorer createVectorScorer(LeafReaderContext context, FieldInfo fi) throws @Override public String toString(String field) { - return getClass().getSimpleName() + ":" + this.field + "[" + target[0] + ",...][" + k + "]"; + StringBuilder buffer = new StringBuilder(); + buffer.append(getClass().getSimpleName() + ":"); + buffer.append(this.field + "[" + target[0] + ",...]"); + buffer.append("[" + k + "]"); + if (this.filter != null) { + buffer.append("[" + this.filter + "]"); + } + return buffer.toString(); } @Override 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 48ac987bee30..1c5436dcf0a5 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.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 boolean equals(Object obj) { /** * 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 boolean equals(Object obj) { /** * 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/search/SortRescorer.java b/lucene/core/src/java/org/apache/lucene/search/SortRescorer.java index 3139da98a84c..e4a4891c570d 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 TopDocs rescore(IndexSearcher searcher, TopDocs firstPassTopDocs, int top 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 48c1ca980913..1f37a82864eb 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 TopFieldCollectorManager( this.sort = sort; this.numHits = numHits; this.after = after; - this.supportsConcurrency = supportsConcurrency; this.totalHitsThreshold = totalHitsThreshold; - this.minScoreAcc = - supportsConcurrency && totalHitsThreshold != Integer.MAX_VALUE - ? new MaxScoreAccumulator() - : null; + this.minScoreAcc = totalHitsThreshold != Integer.MAX_VALUE ? new MaxScoreAccumulator() : null; this.collectors = new ArrayList<>(); } - /** - * 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 TopFieldCollectorManager(Sort sort, int numHits, int totalHitsThreshold) @Override public TopFieldCollector newCollector() { - if (collectorCreated && supportsConcurrency == false) { - throw new IllegalStateException( - "This TopFieldCollectorManager was created without concurrency (supportsConcurrency=false), but multiple collectors are being created"); - } else { - collectorCreated = true; - } - FieldValueHitQueue 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 7933313febec..6a206088013b 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 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 TopScoreDocCollectorManager(int numHits, int totalHitsThreshold) { @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/java/org/apache/lucene/search/WANDScorer.java b/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java index fe5e8803b4cb..f910db30a26b 100644 --- a/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java @@ -229,7 +229,11 @@ private boolean ensureConsistent() { } for (DisiWrapper w : head) { - assert w.doc > doc; + if (lead == null) { // After calling advance() but before matches() + assert w.doc >= doc; + } else { + assert w.doc > doc; + } } return true; @@ -284,20 +288,21 @@ public int advance(int target) throws IOException { // Move 'lead' iterators back to the tail pushBackLeads(target); - // Advance 'head' as well - advanceHead(target); - - // Pop the new 'lead' from 'head' - moveToNextCandidate(target); + // Make sure `head` is also on or beyond `target` + DisiWrapper headTop = advanceHead(target); - if (doc == DocIdSetIterator.NO_MORE_DOCS) { - return DocIdSetIterator.NO_MORE_DOCS; + if (scoreMode == ScoreMode.TOP_SCORES && (headTop == null || headTop.doc > upTo)) { + // Update score bounds if necessary + moveToNextBlock(target); + assert upTo >= target; + headTop = head.top(); } - assert ensureConsistent(); - - // Advance to the next possible match - return doNextCompetitiveCandidate(); + if (headTop == null) { + return doc = DocIdSetIterator.NO_MORE_DOCS; + } else { + return doc = headTop.doc; + } } @Override @@ -309,6 +314,9 @@ public long cost() { @Override public boolean matches() throws IOException { + assert lead == null; + moveToNextCandidate(); + while (leadMaxScore < minCompetitiveScore || freq < minShouldMatch) { if (leadMaxScore + tailMaxScore < minCompetitiveScore || freq + tailSize < minShouldMatch) { @@ -353,7 +361,7 @@ private void pushBackLeads(int target) throws IOException { } /** Make sure all disis in 'head' are on or after 'target'. */ - private void advanceHead(int target) throws IOException { + private DisiWrapper advanceHead(int target) throws IOException { DisiWrapper headTop = head.top(); while (headTop != null && headTop.doc < target) { final DisiWrapper evicted = insertTailWithOverFlow(headTop); @@ -365,6 +373,7 @@ private void advanceHead(int target) throws IOException { headTop = head.top(); } } + return headTop; } private void advanceTail(DisiWrapper disi) throws IOException { @@ -429,7 +438,7 @@ private void updateMaxScores(int target) throws IOException { * Update {@code upTo} and maximum scores of sub scorers so that {@code upTo} is greater than or * equal to the next candidate after {@code target}, i.e. the top of `head`. */ - private void updateMaxScoresIfNecessary(int target) throws IOException { + private void moveToNextBlock(int target) throws IOException { assert lead == null; while (upTo < DocIdSetIterator.NO_MORE_DOCS) { @@ -460,48 +469,19 @@ private void updateMaxScoresIfNecessary(int target) throws IOException { * Set 'doc' to the next potential match, and move all disis of 'head' that are on this doc into * 'lead'. */ - private void moveToNextCandidate(int target) throws IOException { - if (scoreMode == ScoreMode.TOP_SCORES) { - // Update score bounds if necessary so - updateMaxScoresIfNecessary(target); - assert upTo >= target; - - // updateMaxScores tries to move forward until a block with matches is found - // so if the head is empty it means there are no matches at all anymore - if (head.size() == 0) { - assert upTo == DocIdSetIterator.NO_MORE_DOCS; - doc = DocIdSetIterator.NO_MORE_DOCS; - return; - } - } - + private void moveToNextCandidate() throws IOException { // The top of `head` defines the next potential match // pop all documents which are on this doc lead = head.pop(); + assert doc == lead.doc; lead.next = null; leadMaxScore = lead.scaledMaxScore; freq = 1; - doc = lead.doc; while (head.size() > 0 && head.top().doc == doc) { addLead(head.pop()); } } - /** Move iterators to the tail until there is a potential match. */ - private int doNextCompetitiveCandidate() throws IOException { - while (leadMaxScore + tailMaxScore < minCompetitiveScore || freq + tailSize < minShouldMatch) { - // no match on doc is possible, move to the next potential match - pushBackLeads(doc + 1); - moveToNextCandidate(doc + 1); - assert ensureConsistent(); - if (doc == DocIdSetIterator.NO_MORE_DOCS) { - break; - } - } - - return doc; - } - /** Advance all entries from the tail to know about all matches on the current doc. */ private void advanceAllTail() throws IOException { // we return the next doc when the sum of the scores of the potential 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 f967e0cfd881..2bd594cc731c 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.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 @@ */ 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,11 @@ public void disableSkipping() { /** 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; + // 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; @@ -110,11 +106,14 @@ public abstract class NumericLeafComparator implements LeafFieldComparator { 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; @@ -183,12 +182,12 @@ public void copy(int slot, int doc) throws IOException { @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 +206,110 @@ private void updateCompetitiveIterator() throws IOException { || 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) { - competitiveIterator = getNumericDocValues(context, field); - leadCost = Math.min(leadCost, competitiveIterator.cost()); - } - long threshold = Math.min(leadCost >> 3, maxDoc >> 5); - thresholdAsLong = intersectThresholdValue(threshold); + if (queueFull) { + encodeBottom(); } - if ((reverse == false && bottomAsComparableLong() <= thresholdAsLong) - || (reverse && bottomAsComparableLong() >= thresholdAsLong)) { - if (queueFull) { - encodeBottom(); - } - DisjunctionBuildVisitor visitor = new DisjunctionBuildVisitor(); - competitiveIterator = visitor.generateCompetitiveIterator(); - } - } + DocIdSetBuilder result = new DocIdSetBuilder(maxDoc); + PointValues.IntersectVisitor visitor = + new PointValues.IntersectVisitor() { + DocIdSetBuilder.BulkAdder adder; + + @Override + public void grow(int count) { + adder = result.grow(count); + } - private void tryReduceDisjunctionClause(CompetitiveIterator iter) { - int originalSize = iter.disis.size(); + @Override + public void visit(int docID) { + if (docID <= maxDocVisited) { + return; // Already visited or skipped + } + adder.add(docID); + } - while (iter.disis.isEmpty() == false - && (iter.disis.getFirst().mostCompetitiveValue > maxValueAsLong - || iter.disis.getFirst().mostCompetitiveValue < minValueAsLong)) { - iter.disis.removeFirst(); - } + @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; + } + }; - if (originalSize != iter.disis.size()) { - iter.disjunction.clear(); - iter.disjunction.addAll(iter.disis); + final long threshold = iteratorCost >>> 3; + + 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); + if (pointValues.getDocCount() < iteratorCost) { + // Use the set of doc with values to help drive iteration + competitiveIterator = getNumericDocValues(context, field); + iteratorCost = pointValues.getDocCount(); + } + return; } + pointValues.intersect(visitor); + competitiveIterator = result.build().iterator(); + iteratorCost = competitiveIterator.cost(); + updateSkipInterval(true); } - /** 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); + private PointValues.PointTree getPointTree() throws IOException { + if (pointTree == null) { + pointTree = pointValues.getPointTree(); } + return pointTree; } - /** 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()); + 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++; + } + } } } @@ -400,276 +418,5 @@ public int advance(int target) throws IOException { 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/store/BufferedIndexInput.java b/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java index 7f2aadf54a5b..1738259fa2fb 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 final int readInt() throws IOException { } @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 a09f78e5f3a6..39e920616209 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 long readLong() throws IOException { } @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 6aebb771b686..959f429ecfce 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 void readLongs(long[] dst, int offset, int length) throws IOException { } @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 70f9a96db9c4..369b631426bd 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 int readInt() throws IOException { /** * 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 7b97dcd8a7b0..b312a693eba5 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 void writeGroupVInts(long[] values, int limit) throws IOException { } 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 1c5033172db1..e95e2eee4db0 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 static void readGroupVInt(DataInput in, long[] dst, int offset) throws IO 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; + } + + /** + * 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 long readLongInGroup(DataInput in, int numBytesMinus1) throws IOException { + 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 static int readGroupVInt( 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 static void writeGroupVInts(DataOutput out, byte[] scratch, long[] values 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(values[readPos]); + } + } } 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 eb4b93f499ef..d44cc7839233 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 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/java/org/apache/lucene/util/VectorUtil.java b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java index a7239957b326..250c65448703 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 static float[] checkFinite(float[] 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. + * 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(long[] buffer, int length, long 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/java/org/apache/lucene/util/fst/FSTCompiler.java b/lucene/core/src/java/org/apache/lucene/util/fst/FSTCompiler.java index d51c256d48e8..b83024930eea 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 @@ private 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 long getArcCount() { 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 7326fd77f73b..f33f09e90723 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 @@ // 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/java21/org/apache/lucene/internal/vectorization/MemorySegmentPostingDecodingUtil.java b/lucene/core/src/java21/org/apache/lucene/internal/vectorization/MemorySegmentPostingDecodingUtil.java index 582fab392c78..c4a3f8232704 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 @@ 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 @@ private static void shift( } @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 cc3624f1b306..9273f7c5a813 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 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,26 +764,26 @@ private static int squareDistanceBody128(MemorySegment a, MemorySegment b, int l 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 target, int from, int to) { 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() < 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); return from + mask.trueCount(); } } } - 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/java21/org/apache/lucene/store/MemorySegmentIndexInput.java b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java index 832fa5f98e6b..417511b6f5d5 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 @@ public byte readByte(long pos) throws IOException { } @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 bd950aeaebd6..f7a246c76849 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 cd63926a287d..874ebafd971b 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 000000000000..3e346f3eb206 --- /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 df78e045fbb6..d93cb0ab3ecd 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 void testEncodeDecode() throws IOException { 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 void testEncodeDecode() throws IOException { // 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 000000000000..6b0ff1fe5e3b --- /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 2dd3f5ece750..4e9ab4b55ee2 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 void testEncodeDecode() throws IOException { 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 @@ private long encodeTestData(int iterations, int[] values, Directory d) throws IO 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 000000000000..5d02d0561e33 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene101/TestPostingsUtil.java @@ -0,0 +1,57 @@ +/* + * 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 { + // 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]; + + 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 fe6c82e73bb4..69fbf96f6da8 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 @@ 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 @@ 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 void testMixedCompressions() throws Exception { 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 void testInvalidOptions() { 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 ed70b2df002a..d2aa9b8d0542 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.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.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.KnnFloatVectorField; @@ -48,6 +47,7 @@ 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 void setUp() throws Exception { @Override protected Codec getCodec() { - return new Lucene100Codec() { - @Override - public KnnVectorsFormat getKnnVectorsFormatForField(String field) { - return format; - } - }; + return TestUtil.alwaysKnnVectorsFormat(format); } private final KnnVectorsFormat getKnnFormat(int bits) { @@ -104,14 +99,7 @@ public void testMixedQuantizedBits() throws Exception { try (IndexWriter w = new IndexWriter( dir, - newIndexWriterConfig() - .setCodec( - new Lucene100Codec() { - @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 KnnVectorsFormat getKnnVectorsFormatForField(String field) { try (IndexWriter w = new IndexWriter( dir, - newIndexWriterConfig() - .setCodec( - new Lucene100Codec() { - @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 void testMixedQuantizedUnQuantized() throws Exception { new IndexWriter( dir, newIndexWriterConfig() - .setCodec( - new Lucene100Codec() { - @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 KnnVectorsFormat getKnnVectorsFormatForField(String field) { try (IndexWriter w = new IndexWriter( dir, - newIndexWriterConfig() - .setCodec( - new Lucene100Codec() { - @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 void testQuantizationScoringEdgeCase() throws Exception { dir, newIndexWriterConfig() .setCodec( - new Lucene100Codec() { - @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 3b758de6ce67..3ffeef501e87 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.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.lucene100.Lucene100Codec; 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.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 @@ public class TestLucene99ScalarQuantizedVectorScorer extends LuceneTestCase { private static Codec getCodec(int bits, boolean compress) { - return new Lucene100Codec() { - @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 c7f4515c25cf..cf1436f21092 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.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.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.KnnFloatVectorField; @@ -43,6 +42,7 @@ 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 void setUp() throws Exception { @Override protected Codec getCodec() { - return new Lucene100Codec() { - @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/geo/TestTessellator.java b/lucene/core/src/test/org/apache/lucene/geo/TestTessellator.java index 8002717a9ced..e2e964526e88 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 void testComplexPolygon26() throws Exception { + "(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 void testComplexPolygon39() throws Exception { 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 void testComplexPolygon41() throws Exception { 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 void testComplexPolygon43() throws Exception { + "(-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 void testComplexPolygon44() throws Exception { "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 void testComplexPolygon44() throws Exception { 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 void testComplexPolygon49() throws Exception { "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 void testComplexPolygon52() throws Exception { 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 void testComplexPolygon56() throws Exception { } } + 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 void endSplit(String status) { } } + 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/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMaxDocs.java index 48aac56a22d0..d9b8a055f6c0 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 void testExactlyAtTrueLimit() throws Exception { 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/index/TestKnnGraph.java b/lucene/core/src/test/org/apache/lucene/index/TestKnnGraph.java index 5def0a26d84a..a3d655ebe3b0 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.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 void setup() { 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/internal/vectorization/TestPostingDecodingUtil.java b/lucene/core/src/test/org/apache/lucene/internal/vectorization/TestPostingDecodingUtil.java index 517b3ec32aa9..6c914dfcc032 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 @@ 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 void testDuelSplitLongs() throws Exception { } 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 void testDuelSplitLongs() throws Exception { 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/search/BaseKnnVectorQueryTestCase.java b/lucene/core/src/test/org/apache/lucene/search/BaseKnnVectorQueryTestCase.java index afa150e387fe..4abd955eeb66 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.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 @@ public void testSameFieldDifferentFormats() throws IOException { 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 @@ public KnnVectorsFormat getKnnVectorsFormatForField(String field) { } 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/search/IntArrayDocIdSet.java b/lucene/core/src/test/org/apache/lucene/search/IntArrayDocIdSet.java new file mode 100644 index 000000000000..fa4ddece1e4d --- /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 6fe375c086dd..6e84993902d0 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.TestUtil; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.FixedBitSet; -import org.apache.lucene.util.IntArrayDocIdSet; public class TestBooleanOr extends LuceneTestCase { 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 dc48dc1699db..9a2c177c62d5 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 @@ static class MyCollector extends FilterCollector { 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/TestBooleanRewrites.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java index 9083a62e37a1..f36c7539c7dc 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java @@ -330,8 +330,8 @@ public void testDeeplyNestedBooleanRewriteShouldClauses() throws IOException { int depth = TestUtil.nextInt(random(), 10, 30); TestRewriteQuery rewriteQueryExpected = new TestRewriteQuery(); TestRewriteQuery rewriteQuery = new TestRewriteQuery(); - Query expectedQuery = - new BooleanQuery.Builder().add(rewriteQueryExpected, Occur.FILTER).build(); + BooleanQuery.Builder expectedQueryBuilder = + new BooleanQuery.Builder().add(rewriteQueryExpected, Occur.FILTER); Query deepBuilder = new BooleanQuery.Builder() .add(rewriteQuery, Occur.SHOULD) @@ -345,21 +345,19 @@ public void testDeeplyNestedBooleanRewriteShouldClauses() throws IOException { .add(tq, Occur.SHOULD) .add(deepBuilder, Occur.SHOULD); deepBuilder = bq.build(); - BooleanQuery.Builder expectedBq = new BooleanQuery.Builder().add(tq, Occur.FILTER); + expectedQueryBuilder.add(tq, Occur.FILTER); if (i == depth) { - expectedBq.add(rewriteQuery, Occur.FILTER); - } else { - expectedBq.add(expectedQuery, Occur.FILTER); + expectedQueryBuilder.add(rewriteQuery, Occur.FILTER); } - expectedQuery = expectedBq.build(); } BooleanQuery bq = new BooleanQuery.Builder().add(deepBuilder, Occur.FILTER).build(); - expectedQuery = new BoostQuery(new ConstantScoreQuery(expectedQuery), 0.0f); + Query expectedQuery = + new BoostQuery(new ConstantScoreQuery(expectedQueryBuilder.build()), 0.0f); Query rewritten = searcher.rewrite(bq); assertEquals(expectedQuery, rewritten); // the SHOULD clauses cause more rewrites because they incrementally change to `MUST` and then - // `FILTER` - assertEquals("Depth=" + depth, depth + 1, rewriteQuery.numRewrites); + // `FILTER`, plus the flattening of required clauses + assertEquals("Depth=" + depth, depth * 2, rewriteQuery.numRewrites); } public void testDeeplyNestedBooleanRewrite() throws IOException { @@ -369,27 +367,26 @@ public void testDeeplyNestedBooleanRewrite() throws IOException { int depth = TestUtil.nextInt(random(), 10, 30); TestRewriteQuery rewriteQueryExpected = new TestRewriteQuery(); TestRewriteQuery rewriteQuery = new TestRewriteQuery(); - Query expectedQuery = - new BooleanQuery.Builder().add(rewriteQueryExpected, Occur.FILTER).build(); + BooleanQuery.Builder expectedQueryBuilder = + new BooleanQuery.Builder().add(rewriteQueryExpected, Occur.FILTER); Query deepBuilder = new BooleanQuery.Builder().add(rewriteQuery, Occur.MUST).build(); for (int i = depth; i > 0; i--) { TermQuery tq = termQueryFunction.apply(i); BooleanQuery.Builder bq = new BooleanQuery.Builder().add(tq, Occur.MUST).add(deepBuilder, Occur.MUST); deepBuilder = bq.build(); - BooleanQuery.Builder expectedBq = new BooleanQuery.Builder().add(tq, Occur.FILTER); + expectedQueryBuilder.add(tq, Occur.FILTER); if (i == depth) { - expectedBq.add(rewriteQuery, Occur.FILTER); - } else { - expectedBq.add(expectedQuery, Occur.FILTER); + expectedQueryBuilder.add(rewriteQuery, Occur.FILTER); } - expectedQuery = expectedBq.build(); } BooleanQuery bq = new BooleanQuery.Builder().add(deepBuilder, Occur.FILTER).build(); - expectedQuery = new BoostQuery(new ConstantScoreQuery(expectedQuery), 0.0f); + Query expectedQuery = + new BoostQuery(new ConstantScoreQuery(expectedQueryBuilder.build()), 0.0f); Query rewritten = searcher.rewrite(bq); assertEquals(expectedQuery, rewritten); - assertEquals("Depth=" + depth, 1, rewriteQuery.numRewrites); + // `depth` rewrites because of the flattening + assertEquals("Depth=" + depth, depth, rewriteQuery.numRewrites); } public void testRemoveMatchAllFilter() throws IOException { @@ -691,6 +688,110 @@ public void testFlattenInnerDisjunctions() throws IOException { assertSame(query, searcher.rewrite(query)); } + public void testFlattenInnerConjunctions() throws IOException { + IndexSearcher searcher = newSearcher(new MultiReader()); + + Query inner = + new BooleanQuery.Builder() + .add(new TermQuery(new Term("foo", "bar")), Occur.MUST) + .add(new TermQuery(new Term("foo", "quux")), Occur.MUST) + .build(); + Query query = + new BooleanQuery.Builder() + .add(inner, Occur.MUST) + .add(new TermQuery(new Term("foo", "baz")), Occur.FILTER) + .build(); + Query expectedRewritten = + new BooleanQuery.Builder() + .add(new TermQuery(new Term("foo", "bar")), Occur.MUST) + .add(new TermQuery(new Term("foo", "quux")), Occur.MUST) + .add(new TermQuery(new Term("foo", "baz")), Occur.FILTER) + .build(); + assertEquals(expectedRewritten, searcher.rewrite(query)); + + query = + new BooleanQuery.Builder() + .setMinimumNumberShouldMatch(0) + .add(inner, Occur.MUST) + .add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD) + .build(); + expectedRewritten = + new BooleanQuery.Builder() + .setMinimumNumberShouldMatch(0) + .add(new TermQuery(new Term("foo", "bar")), Occur.MUST) + .add(new TermQuery(new Term("foo", "quux")), Occur.MUST) + .add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD) + .build(); + assertEquals(expectedRewritten, searcher.rewrite(query)); + + query = + new BooleanQuery.Builder() + .add(inner, Occur.MUST) + .add(new TermQuery(new Term("foo", "baz")), Occur.MUST_NOT) + .build(); + expectedRewritten = + new BooleanQuery.Builder() + .add(new TermQuery(new Term("foo", "bar")), Occur.MUST) + .add(new TermQuery(new Term("foo", "quux")), Occur.MUST) + .add(new TermQuery(new Term("foo", "baz")), Occur.MUST_NOT) + .build(); + assertEquals(expectedRewritten, searcher.rewrite(query)); + + inner = + new BooleanQuery.Builder() + .add(new TermQuery(new Term("foo", "bar")), Occur.MUST) + .add(new TermQuery(new Term("foo", "quux")), Occur.FILTER) + .build(); + query = + new BooleanQuery.Builder() + .add(inner, Occur.MUST) + .add(new TermQuery(new Term("foo", "baz")), Occur.MUST) + .build(); + expectedRewritten = + new BooleanQuery.Builder() + .add(new TermQuery(new Term("foo", "bar")), Occur.MUST) + .add(new TermQuery(new Term("foo", "quux")), Occur.FILTER) + .add(new TermQuery(new Term("foo", "baz")), Occur.MUST) + .build(); + assertEquals(expectedRewritten, searcher.rewrite(query)); + + inner = + new BooleanQuery.Builder() + .add(new TermQuery(new Term("foo", "bar")), Occur.MUST) + .add(new TermQuery(new Term("foo", "quux")), Occur.FILTER) + .build(); + query = + new BooleanQuery.Builder() + .add(inner, Occur.FILTER) + .add(new TermQuery(new Term("foo", "baz")), Occur.MUST) + .build(); + expectedRewritten = + new BooleanQuery.Builder() + .add(new TermQuery(new Term("foo", "bar")), Occur.FILTER) + .add(new TermQuery(new Term("foo", "quux")), Occur.FILTER) + .add(new TermQuery(new Term("foo", "baz")), Occur.MUST) + .build(); + assertEquals(expectedRewritten, searcher.rewrite(query)); + + inner = + new BooleanQuery.Builder() + .add(new TermQuery(new Term("foo", "bar")), Occur.MUST) + .add(new TermQuery(new Term("foo", "quux")), Occur.MUST_NOT) + .build(); + query = + new BooleanQuery.Builder() + .add(inner, Occur.FILTER) + .add(new TermQuery(new Term("foo", "baz")), Occur.MUST) + .build(); + expectedRewritten = + new BooleanQuery.Builder() + .add(new TermQuery(new Term("foo", "bar")), Occur.FILTER) + .add(new TermQuery(new Term("foo", "quux")), Occur.MUST_NOT) + .add(new TermQuery(new Term("foo", "baz")), Occur.MUST) + .build(); + assertEquals(expectedRewritten, searcher.rewrite(query)); + } + public void testDiscardShouldClauses() throws IOException { IndexSearcher searcher = newSearcher(new MultiReader()); 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 46ab7ad420b5..dab1e1651136 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 @@ private void runTest(boolean reversed) throws Throwable { 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/TestKnnByteVectorQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestKnnByteVectorQuery.java index 4dc3d385b087..b45d6e8fb641 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestKnnByteVectorQuery.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestKnnByteVectorQuery.java @@ -23,6 +23,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.QueryTimeout; +import org.apache.lucene.index.Term; import org.apache.lucene.index.VectorSimilarityFunction; import org.apache.lucene.store.Directory; import org.apache.lucene.util.TestVectorUtil; @@ -78,6 +79,11 @@ public void testToString() throws IOException { assertEquals("KnnByteVectorQuery:field[0,...][10]", query.toString("ignored")); assertDocScoreQueryToString(query.rewrite(newSearcher(reader))); + + // test with filter + Query filter = new TermQuery(new Term("id", "text")); + query = getKnnVectorQuery("field", new float[] {0, 1}, 10, filter); + assertEquals("KnnByteVectorQuery:field[0,...][10][id:text]", query.toString("ignored")); } } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestKnnFloatVectorQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestKnnFloatVectorQuery.java index cd66d6a27cd9..feebe858c099 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestKnnFloatVectorQuery.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestKnnFloatVectorQuery.java @@ -35,6 +35,7 @@ import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.QueryTimeout; +import org.apache.lucene.index.Term; import org.apache.lucene.index.VectorSimilarityFunction; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; @@ -77,6 +78,11 @@ public void testToString() throws IOException { assertEquals("KnnFloatVectorQuery:field[0.0,...][10]", query.toString("ignored")); assertDocScoreQueryToString(query.rewrite(newSearcher(reader))); + + // test with filter + Query filter = new TermQuery(new Term("id", "text")); + query = getKnnVectorQuery("field", new float[] {0.0f, 1.0f}, 10, filter); + assertEquals("KnnFloatVectorQuery:field[0.0,...][10][id:text]", query.toString("ignored")); } } 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 78bebc115ea6..d58fa829ed62 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 void testConstantScoreQuery() throws Exception { 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 784cc8857334..2a53f49759bb 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 void testNegativeScores() throws Exception { 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 bf33e7732e0f..ecc5727625cf 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 @@ void assertQuery(Query query, Sort sort) throws Exception { 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 @@ void assertQuery(Query query, Sort sort) throws Exception { 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 38dbe3399ee8..98496ae32106 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 void testSortOptimizationWithMissingValues() throws IOException { 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 void testSortOptimizationWithMissingValues() throws IOException { 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 void testSortOptimizationWithMissingValues() throws IOException { 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 void testNumericDocValuesOptimizationWithMissingValues() throws IOExcepti 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 void testNumericDocValuesOptimizationWithMissingValues() throws IOExcepti 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 void testNumericDocValuesOptimizationWithMissingValues() throws IOExcepti 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 void testRandomLong() throws IOException { // 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 void testSortOptimizationOnSortedNumericField() throws IOException { 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 void doTestStringSortOptimizationDisabled(DirectoryReader reader) throws 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 @@ private TopDocs assertSearchHits(DirectoryReader reader, Sort sort, int n, Field 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 44bd66148cd6..cc7405f67724 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 @@ private TopDocs doSearchWithThreshold( 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 @@ private static TopDocs doConcurrentSearchWithThreshold( 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 4215b755b36a..1d5b346c29db 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 void search(Weight weight, Collector collector) throws IOException { 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 @@ void testSort(boolean useFrom) throws Exception { 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 bd0d408e0b2f..4393ace2c265 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 @@ private static TopDocs doSearchWithThreshold( 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 @@ private static TopDocs doConcurrentSearchWithThreshold( 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 void testSort() throws Exception { 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 void testSortNoResults() throws Exception { 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 void testRelationVsTopDocsCount() throws Exception { 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/core/src/test/org/apache/lucene/search/TestTopFieldCollectorEarlyTermination.java b/lucene/core/src/test/org/apache/lucene/search/TestTopFieldCollectorEarlyTermination.java index 1540d77270c5..52f313b3dfd9 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.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.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 @@ private void doTestEarlyTermination(boolean paging) throws IOException { 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 @@ private void doTestEarlyTermination(boolean paging) throws IOException { 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()); } 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 e26aed408b74..67f0918f46b7 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 @@ void doPrevSetBit(java.util.BitSet a, FixedBitSet b) { } // 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 @@ void doIterate2(java.util.BitSet a, FixedBitSet b) throws IOException { } 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 @@ void doRandomSets(int maxSize, int iter, int mode) throws IOException { 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 @@ void doRandomSets(int maxSize, int iter, int mode) throws IOException { 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 @@ void doRandomSets(int maxSize, int iter, int mode) throws IOException { // 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). 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 541dbbde8d7d..6e449a550028 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 @@ private static int xorBitCount(byte[] a, byte[] b) { 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,17 +366,17 @@ public void testFindNextGEQ() { // 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( slowFindNextGEQ(values, 128, target, from), - VectorUtil.findNextGEQ(values, 128, target, from)); + VectorUtil.findNextGEQ(values, target, from, 128)); } } - 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/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 8319f20efea3..2fb93c7f5a46 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 @@ 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); 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 41aeef2e5c8d..1da8c8169a98 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.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 @@ public void testRandomReadWriteAndMerge() throws IOException { 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 @@ public void testReadWrite() throws IOException { 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 @@ public void testSortedAndUnsortedIndicesReturnSameResults() throws IOException { 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/facet/src/java/org/apache/lucene/facet/DrillSideways.java b/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java index c3e09dbb072a..576aa84d51fc 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 DrillSidewaysResult search( 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 DrillSidewaysResult search(ScoreDoc after, DrillDownQuery query, int topN 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 34ce3ab6697b..bc547407ee92 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 @@ private static FacetsResult doSearch( throw new IllegalArgumentException("after must be a FieldDoc; got " + after); } TopFieldCollectorManager topFieldCollectorManager = - new TopFieldCollectorManager(sort, n, (FieldDoc) after, Integer.MAX_VALUE, true); + new TopFieldCollectorManager(sort, n, (FieldDoc) after, Integer.MAX_VALUE); multiCollectorManager = new MultiCollectorManager(topFieldCollectorManager, fcm); } else { TopScoreDocCollectorManager topScoreDocCollectorManager = - new TopScoreDocCollectorManager(n, after, Integer.MAX_VALUE, true); + new TopScoreDocCollectorManager(n, after, Integer.MAX_VALUE); multiCollectorManager = new MultiCollectorManager(topScoreDocCollectorManager, fcm); } Object[] result = searcher.search(q, multiCollectorManager); diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java index 92c420c58372..779b62291a69 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java @@ -295,13 +295,12 @@ public TopGroups getTopGroups( "cannot sort by relevance within group: needsScores=false"); } collector = - new TopScoreDocCollectorManager(maxDocsPerGroup, null, Integer.MAX_VALUE, false) + new TopScoreDocCollectorManager(maxDocsPerGroup, null, Integer.MAX_VALUE) .newCollector(); } else { // Sort by fields collector = - new TopFieldCollectorManager( - withinGroupSort, maxDocsPerGroup, null, Integer.MAX_VALUE, false) + new TopFieldCollectorManager(withinGroupSort, maxDocsPerGroup, null, Integer.MAX_VALUE) .newCollector(); // TODO: disable exact counts? } diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java index 00940be58ecc..6a54f7b59af0 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java @@ -129,7 +129,7 @@ private static class TopDocsReducer extends GroupReducer 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 @@ private static class TopDocsReducer extends GroupReducer { 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/java/org/apache/lucene/search/join/DiversifyingChildrenByteKnnVectorQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/DiversifyingChildrenByteKnnVectorQuery.java index 456a885b49a0..45cb8b9c88fa 100644 --- a/lucene/join/src/java/org/apache/lucene/search/join/DiversifyingChildrenByteKnnVectorQuery.java +++ b/lucene/join/src/java/org/apache/lucene/search/join/DiversifyingChildrenByteKnnVectorQuery.java @@ -154,7 +154,14 @@ protected TopDocs approximateSearch( @Override public String toString(String field) { - return getClass().getSimpleName() + ":" + this.field + "[" + query[0] + ",...][" + k + "]"; + StringBuilder buffer = new StringBuilder(); + buffer.append(getClass().getSimpleName() + ":"); + buffer.append(this.field + "[" + query[0] + ",...]"); + buffer.append("[" + k + "]"); + if (this.filter != null) { + buffer.append("[" + this.filter + "]"); + } + return buffer.toString(); } @Override diff --git a/lucene/join/src/java/org/apache/lucene/search/join/DiversifyingChildrenFloatKnnVectorQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/DiversifyingChildrenFloatKnnVectorQuery.java index 7b5a656d1414..9c44a2f78566 100644 --- a/lucene/join/src/java/org/apache/lucene/search/join/DiversifyingChildrenFloatKnnVectorQuery.java +++ b/lucene/join/src/java/org/apache/lucene/search/join/DiversifyingChildrenFloatKnnVectorQuery.java @@ -153,7 +153,14 @@ protected TopDocs approximateSearch( @Override public String toString(String field) { - return getClass().getSimpleName() + ":" + this.field + "[" + query[0] + ",...][" + k + "]"; + StringBuilder buffer = new StringBuilder(); + buffer.append(getClass().getSimpleName() + ":"); + buffer.append(this.field + "[" + query[0] + ",...]"); + buffer.append("[" + k + "]"); + if (this.filter != null) { + buffer.append("[" + this.filter + "]"); + } + return buffer.toString(); } @Override 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 227a9550a88b..d959d9e26308 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 void testRandomOrdinalsJoin() throws Exception { 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 @@ private void executeRandomJoin( // 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/join/src/test/org/apache/lucene/search/join/TestParentBlockJoinByteKnnVectorQuery.java b/lucene/join/src/test/org/apache/lucene/search/join/TestParentBlockJoinByteKnnVectorQuery.java index 6f773300a6dc..6c1d461d4bf8 100644 --- a/lucene/join/src/test/org/apache/lucene/search/join/TestParentBlockJoinByteKnnVectorQuery.java +++ b/lucene/join/src/test/org/apache/lucene/search/join/TestParentBlockJoinByteKnnVectorQuery.java @@ -29,9 +29,11 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.Term; import org.apache.lucene.index.VectorSimilarityFunction; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.Directory; public class TestParentBlockJoinByteKnnVectorQuery extends ParentBlockJoinKnnVectorQueryTestCase { @@ -81,6 +83,20 @@ d, new IndexWriterConfig().setMergePolicy(newMergePolicy(random(), false)))) { } } + public void testToString() { + // test without filter + Query query = getParentJoinKnnQuery("field", new float[] {0, 1}, null, 10, null); + assertEquals( + "DiversifyingChildrenByteKnnVectorQuery:field[0,...][10]", query.toString("ignored")); + + // test with filter + Query filter = new TermQuery(new Term("id", "text")); + query = getParentJoinKnnQuery("field", new float[] {0, 1}, filter, 10, null); + assertEquals( + "DiversifyingChildrenByteKnnVectorQuery:field[0,...][10][id:text]", + query.toString("ignored")); + } + private static byte[] fromFloat(float[] queryVector) { byte[] query = new byte[queryVector.length]; for (int i = 0; i < queryVector.length; i++) { diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestParentBlockJoinFloatKnnVectorQuery.java b/lucene/join/src/test/org/apache/lucene/search/join/TestParentBlockJoinFloatKnnVectorQuery.java index 616c8fdb3706..f15de3b57eea 100644 --- a/lucene/join/src/test/org/apache/lucene/search/join/TestParentBlockJoinFloatKnnVectorQuery.java +++ b/lucene/join/src/test/org/apache/lucene/search/join/TestParentBlockJoinFloatKnnVectorQuery.java @@ -29,9 +29,11 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.Term; import org.apache.lucene.index.VectorSimilarityFunction; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.Directory; public class TestParentBlockJoinFloatKnnVectorQuery extends ParentBlockJoinKnnVectorQueryTestCase { @@ -110,6 +112,20 @@ d, new IndexWriterConfig().setMergePolicy(newMergePolicy(random(), false)))) { } } + public void testToString() { + // test without filter + Query query = getParentJoinKnnQuery("field", new float[] {0, 1}, null, 10, null); + assertEquals( + "DiversifyingChildrenFloatKnnVectorQuery:field[0.0,...][10]", query.toString("ignored")); + + // test with filter + Query filter = new TermQuery(new Term("id", "text")); + query = getParentJoinKnnQuery("field", new float[] {0.0f, 1.0f}, filter, 10, null); + assertEquals( + "DiversifyingChildrenFloatKnnVectorQuery:field[0.0,...][10][id:text]", + query.toString("ignored")); + } + @Override Field getKnnVectorField(String name, float[] vector) { return new KnnFloatVectorField(name, vector); 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 4d5706be0f2a..e95ca09060d8 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 @@ private SearchResults search() throws IOException { } 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/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java index e5ca29b9fae5..c0f2f61d7cc1 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.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 @@ public void visit(String field, QueryVisitor visitor) { /** * 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 @@ public void visit(String field, QueryVisitor visitor) { 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/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java index 72a709bd4e05..18476f50c2b4 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/complexPhrase/ComplexPhraseQueryParser.java @@ -294,9 +294,9 @@ public Query rewrite(IndexSearcher indexSearcher) throws IOException { if (qc instanceof BooleanQuery || qc instanceof SynonymQuery) { ArrayList sc = new ArrayList<>(); - BooleanQuery booleanCaluse = + BooleanQuery booleanClause = qc instanceof BooleanQuery ? (BooleanQuery) qc : convert((SynonymQuery) qc); - addComplexPhraseClause(sc, booleanCaluse); + addComplexPhraseClause(sc, booleanClause); if (sc.size() > 0) { allSpanClauses[i] = sc.get(0); } else { diff --git a/lucene/suggest/src/java/module-info.java b/lucene/suggest/src/java/module-info.java index c3f5b7c178f3..20542ac5281d 100644 --- a/lucene/suggest/src/java/module-info.java +++ b/lucene/suggest/src/java/module-info.java @@ -32,7 +32,8 @@ 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/analyzing/AnalyzingInfixSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java index 426c02999f7d..c5f1fdc899af 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 List lookup( 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/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 000000000000..2a51f01fd7ae --- /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 341b034e8e8b..77a62443f312 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 @@ 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 8544a97b88fa..5c0365616cde 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 410c6e76b4ec..4a59d09d35ce 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 @@ static IndexWriterConfig iwcWithSuggestField(Analyzer analyzer, final Set