diff --git a/gradle/generation/forUtil.gradle b/gradle/generation/forUtil.gradle index e1428234875..5de1d850a02 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/lucene99") + def genDir = file("src/java/org/apache/lucene/codecs/lucene912") def genScript = file("${genDir}/gen_ForUtil.py") def genOutput = file("${genDir}/ForUtil.java") @@ -96,5 +96,30 @@ configure(project(":lucene:backward-codecs")) { andThenTasks: ["spotlessJava", "spotlessJavaApply"], mustRunBefore: [ "compileJava" ] ]) + + task generateForUtil99Internal() { + description "Regenerate gen_ForUtil.py" + group "generation" + + def genDir = file("src/java/org/apache/lucene/backward_codecs/lucene99") + 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(generateForUtil99Internal, [ + andThenTasks: ["spotlessJava", "spotlessJavaApply"], + mustRunBefore: [ "compileJava" ] + ]) } diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index dfe8a5e5442..d23c491fe32 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -317,6 +317,12 @@ Optimizations by 1) using a confined Arena where appropriate, and 2) grouping files from the same segment to a single shared Arena. (Chris Hegarty, Michael Gibney, Uwe Schindler) +* GITHUB#13585: Lucene912PostingsFormat, the new default postings format, now + only has 2 levels of skip data, which are inlined into postings instead of + being stored at the end of postings lists. This translates into better + performance for queries that need skipping such as conjunctions. + (Adrien Grand) + Changes in runtime behavior --------------------- diff --git a/lucene/backward-codecs/src/generated/checksums/generateForUtil99.json b/lucene/backward-codecs/src/generated/checksums/generateForUtil99.json new file mode 100644 index 00000000000..872e9c7bd6d --- /dev/null +++ b/lucene/backward-codecs/src/generated/checksums/generateForUtil99.json @@ -0,0 +1,4 @@ +{ + "lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForUtil.java": "f31797842f047626df6a1a6b97167bec60269fec", + "lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/gen_ForUtil.py": "325f2610974b0e76e278b6445405a098a3763feb" +} \ 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 992ad22a773..fbc2cdba98e 100644 --- a/lucene/backward-codecs/src/java/module-info.java +++ b/lucene/backward-codecs/src/java/module-info.java @@ -35,6 +35,7 @@ module org.apache.lucene.backward_codecs { exports org.apache.lucene.backward_codecs.lucene92; exports org.apache.lucene.backward_codecs.lucene94; exports org.apache.lucene.backward_codecs.lucene95; + exports org.apache.lucene.backward_codecs.lucene99; exports org.apache.lucene.backward_codecs.packed; exports org.apache.lucene.backward_codecs.store; @@ -43,7 +44,8 @@ module org.apache.lucene.backward_codecs { provides org.apache.lucene.codecs.PostingsFormat with 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.lucene90.Lucene90PostingsFormat, + org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat; provides org.apache.lucene.codecs.KnnVectorsFormat with org.apache.lucene.backward_codecs.lucene90.Lucene90HnswVectorsFormat, org.apache.lucene.backward_codecs.lucene91.Lucene91HnswVectorsFormat, @@ -59,5 +61,6 @@ module org.apache.lucene.backward_codecs { org.apache.lucene.backward_codecs.lucene91.Lucene91Codec, org.apache.lucene.backward_codecs.lucene92.Lucene92Codec, org.apache.lucene.backward_codecs.lucene94.Lucene94Codec, - org.apache.lucene.backward_codecs.lucene95.Lucene95Codec; + org.apache.lucene.backward_codecs.lucene95.Lucene95Codec, + org.apache.lucene.backward_codecs.lucene99.Lucene99Codec; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForDeltaUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForDeltaUtil.java similarity index 98% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForDeltaUtil.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForDeltaUtil.java index 2403c997435..2fa67ebf40e 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForDeltaUtil.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForDeltaUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import java.io.IOException; import org.apache.lucene.store.DataInput; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForUtil.java new file mode 100644 index 00000000000..cc089751784 --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/ForUtil.java @@ -0,0 +1,1148 @@ +// 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.backward_codecs.lucene99; + +import java.io.IOException; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.DataOutput; + +// 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 +final class ForUtil { + + static final int BLOCK_SIZE = 128; + private static final int BLOCK_SIZE_LOG2 = 7; + + private static long expandMask32(long mask32) { + return mask32 | (mask32 << 32); + } + + private static long expandMask16(long mask16) { + return expandMask32(mask16 | (mask16 << 16)); + } + + private static long expandMask8(long mask8) { + return expandMask16(mask8 | (mask8 << 8)); + } + + private static long mask32(int bitsPerValue) { + return expandMask32((1L << bitsPerValue) - 1); + } + + private static long mask16(int bitsPerValue) { + return expandMask16((1L << bitsPerValue) - 1); + } + + private static long mask8(int bitsPerValue) { + return expandMask8((1L << bitsPerValue) - 1); + } + + private static void expand8(long[] arr) { + for (int i = 0; i < 16; ++i) { + long l = arr[i]; + arr[i] = (l >>> 56) & 0xFFL; + arr[16 + i] = (l >>> 48) & 0xFFL; + arr[32 + i] = (l >>> 40) & 0xFFL; + arr[48 + i] = (l >>> 32) & 0xFFL; + arr[64 + i] = (l >>> 24) & 0xFFL; + arr[80 + i] = (l >>> 16) & 0xFFL; + arr[96 + i] = (l >>> 8) & 0xFFL; + arr[112 + i] = l & 0xFFL; + } + } + + private static void expand8To32(long[] arr) { + for (int i = 0; i < 16; ++i) { + long l = arr[i]; + arr[i] = (l >>> 24) & 0x000000FF000000FFL; + arr[16 + i] = (l >>> 16) & 0x000000FF000000FFL; + arr[32 + i] = (l >>> 8) & 0x000000FF000000FFL; + arr[48 + i] = l & 0x000000FF000000FFL; + } + } + + private static void collapse8(long[] arr) { + for (int i = 0; i < 16; ++i) { + arr[i] = + (arr[i] << 56) + | (arr[16 + i] << 48) + | (arr[32 + i] << 40) + | (arr[48 + i] << 32) + | (arr[64 + i] << 24) + | (arr[80 + i] << 16) + | (arr[96 + i] << 8) + | arr[112 + i]; + } + } + + private static void expand16(long[] arr) { + for (int i = 0; i < 32; ++i) { + long l = arr[i]; + arr[i] = (l >>> 48) & 0xFFFFL; + arr[32 + i] = (l >>> 32) & 0xFFFFL; + arr[64 + i] = (l >>> 16) & 0xFFFFL; + arr[96 + i] = l & 0xFFFFL; + } + } + + private static void expand16To32(long[] arr) { + for (int i = 0; i < 32; ++i) { + long l = arr[i]; + arr[i] = (l >>> 16) & 0x0000FFFF0000FFFFL; + arr[32 + i] = l & 0x0000FFFF0000FFFFL; + } + } + + private static void collapse16(long[] arr) { + for (int i = 0; i < 32; ++i) { + arr[i] = (arr[i] << 48) | (arr[32 + i] << 32) | (arr[64 + i] << 16) | arr[96 + i]; + } + } + + private static void expand32(long[] arr) { + for (int i = 0; i < 64; ++i) { + long l = arr[i]; + arr[i] = l >>> 32; + arr[64 + i] = l & 0xFFFFFFFFL; + } + } + + private static void collapse32(long[] arr) { + for (int i = 0; i < 64; ++i) { + arr[i] = (arr[i] << 32) | arr[64 + i]; + } + } + + private static void prefixSum8(long[] arr, long base) { + expand8To32(arr); + prefixSum32(arr, base); + } + + private static void prefixSum16(long[] arr, long base) { + // We need to move to the next primitive size to avoid overflows + expand16To32(arr); + prefixSum32(arr, base); + } + + private static void prefixSum32(long[] arr, long base) { + arr[0] += base << 32; + innerPrefixSum32(arr); + expand32(arr); + final long l = arr[BLOCK_SIZE / 2 - 1]; + for (int i = BLOCK_SIZE / 2; i < BLOCK_SIZE; ++i) { + arr[i] += l; + } + } + + // For some reason unrolling seems to help + private static void innerPrefixSum32(long[] 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 long[] tmp = new long[BLOCK_SIZE / 2]; + + /** Encode 128 integers from {@code longs} into {@code out}. */ + void encode(long[] longs, int bitsPerValue, DataOutput out) throws IOException { + final int nextPrimitive; + final int numLongs; + if (bitsPerValue <= 8) { + nextPrimitive = 8; + numLongs = BLOCK_SIZE / 8; + collapse8(longs); + } else if (bitsPerValue <= 16) { + nextPrimitive = 16; + numLongs = BLOCK_SIZE / 4; + collapse16(longs); + } else { + nextPrimitive = 32; + numLongs = BLOCK_SIZE / 2; + collapse32(longs); + } + + final int numLongsPerShift = bitsPerValue * 2; + int idx = 0; + int shift = nextPrimitive - bitsPerValue; + for (int i = 0; i < numLongsPerShift; ++i) { + tmp[i] = longs[idx++] << shift; + } + for (shift = shift - bitsPerValue; shift >= 0; shift -= bitsPerValue) { + for (int i = 0; i < numLongsPerShift; ++i) { + tmp[i] |= longs[idx++] << shift; + } + } + + final int remainingBitsPerLong = shift + bitsPerValue; + final long maskRemainingBitsPerLong; + if (nextPrimitive == 8) { + maskRemainingBitsPerLong = MASKS8[remainingBitsPerLong]; + } else if (nextPrimitive == 16) { + maskRemainingBitsPerLong = MASKS16[remainingBitsPerLong]; + } else { + maskRemainingBitsPerLong = MASKS32[remainingBitsPerLong]; + } + + int tmpIdx = 0; + int remainingBitsPerValue = bitsPerValue; + while (idx < numLongs) { + if (remainingBitsPerValue >= remainingBitsPerLong) { + remainingBitsPerValue -= remainingBitsPerLong; + tmp[tmpIdx++] |= (longs[idx] >>> remainingBitsPerValue) & maskRemainingBitsPerLong; + if (remainingBitsPerValue == 0) { + idx++; + remainingBitsPerValue = bitsPerValue; + } + } else { + final long mask1, mask2; + if (nextPrimitive == 8) { + mask1 = MASKS8[remainingBitsPerValue]; + mask2 = MASKS8[remainingBitsPerLong - remainingBitsPerValue]; + } else if (nextPrimitive == 16) { + mask1 = MASKS16[remainingBitsPerValue]; + mask2 = MASKS16[remainingBitsPerLong - remainingBitsPerValue]; + } else { + mask1 = MASKS32[remainingBitsPerValue]; + mask2 = MASKS32[remainingBitsPerLong - remainingBitsPerValue]; + } + tmp[tmpIdx] |= (longs[idx++] & mask1) << (remainingBitsPerLong - remainingBitsPerValue); + remainingBitsPerValue = bitsPerValue - remainingBitsPerLong + remainingBitsPerValue; + tmp[tmpIdx++] |= (longs[idx] >>> remainingBitsPerValue) & mask2; + } + } + + for (int i = 0; i < numLongsPerShift; ++i) { + out.writeLong(tmp[i]); + } + } + + /** Number of bytes required to encode 128 integers of {@code bitsPerValue} bits per value. */ + int numBytes(int bitsPerValue) { + return bitsPerValue << (BLOCK_SIZE_LOG2 - 3); + } + + private static void decodeSlow(int bitsPerValue, DataInput in, long[] tmp, long[] longs) + throws IOException { + final int numLongs = bitsPerValue << 1; + in.readLongs(tmp, 0, numLongs); + final long mask = MASKS32[bitsPerValue]; + int longsIdx = 0; + int shift = 32 - bitsPerValue; + for (; shift >= 0; shift -= bitsPerValue) { + shiftLongs(tmp, numLongs, longs, longsIdx, shift, mask); + longsIdx += numLongs; + } + final int remainingBitsPerLong = shift + bitsPerValue; + final long mask32RemainingBitsPerLong = MASKS32[remainingBitsPerLong]; + int tmpIdx = 0; + int remainingBits = remainingBitsPerLong; + for (; longsIdx < BLOCK_SIZE / 2; ++longsIdx) { + int b = bitsPerValue - remainingBits; + long l = (tmp[tmpIdx++] & MASKS32[remainingBits]) << b; + while (b >= remainingBitsPerLong) { + b -= remainingBitsPerLong; + l |= (tmp[tmpIdx++] & mask32RemainingBitsPerLong) << b; + } + if (b > 0) { + l |= (tmp[tmpIdx] >>> (remainingBitsPerLong - b)) & MASKS32[b]; + remainingBits = remainingBitsPerLong - b; + } else { + remainingBits = remainingBitsPerLong; + } + longs[longsIdx] = l; + } + } + + /** + * The pattern that this shiftLongs method applies is recognized by the C2 compiler, which + * generates SIMD instructions for it in order to shift multiple longs at once. + */ + private static void shiftLongs(long[] a, int count, long[] b, int bi, int shift, long mask) { + for (int i = 0; i < count; ++i) { + b[bi + i] = (a[i] >>> shift) & mask; + } + } + + private static final long[] MASKS8 = new long[8]; + private static final long[] MASKS16 = new long[16]; + private static final long[] MASKS32 = new long[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 longs to avoid the cost of reading array, arrays should only be + // used when the idx is a variable + private static final long MASK8_1 = MASKS8[1]; + private static final long MASK8_2 = MASKS8[2]; + private static final long MASK8_3 = MASKS8[3]; + private static final long MASK8_4 = MASKS8[4]; + private static final long MASK8_5 = MASKS8[5]; + private static final long MASK8_6 = MASKS8[6]; + private static final long MASK8_7 = MASKS8[7]; + private static final long MASK16_1 = MASKS16[1]; + private static final long MASK16_2 = MASKS16[2]; + private static final long MASK16_3 = MASKS16[3]; + private static final long MASK16_4 = MASKS16[4]; + private static final long MASK16_5 = MASKS16[5]; + private static final long MASK16_6 = MASKS16[6]; + private static final long MASK16_7 = MASKS16[7]; + private static final long MASK16_9 = MASKS16[9]; + private static final long MASK16_10 = MASKS16[10]; + private static final long MASK16_11 = MASKS16[11]; + private static final long MASK16_12 = MASKS16[12]; + private static final long MASK16_13 = MASKS16[13]; + private static final long MASK16_14 = MASKS16[14]; + private static final long MASK16_15 = MASKS16[15]; + private static final long MASK32_1 = MASKS32[1]; + private static final long MASK32_2 = MASKS32[2]; + private static final long MASK32_3 = MASKS32[3]; + private static final long MASK32_4 = MASKS32[4]; + private static final long MASK32_5 = MASKS32[5]; + private static final long MASK32_6 = MASKS32[6]; + private static final long MASK32_7 = MASKS32[7]; + private static final long MASK32_8 = MASKS32[8]; + private static final long MASK32_9 = MASKS32[9]; + private static final long MASK32_10 = MASKS32[10]; + private static final long MASK32_11 = MASKS32[11]; + private static final long MASK32_12 = MASKS32[12]; + private static final long MASK32_13 = MASKS32[13]; + private static final long MASK32_14 = MASKS32[14]; + private static final long MASK32_15 = MASKS32[15]; + private static final long MASK32_17 = MASKS32[17]; + private static final long MASK32_18 = MASKS32[18]; + private static final long MASK32_19 = MASKS32[19]; + private static final long MASK32_20 = MASKS32[20]; + private static final long MASK32_21 = MASKS32[21]; + private static final long MASK32_22 = MASKS32[22]; + private static final long MASK32_23 = MASKS32[23]; + private static final long MASK32_24 = MASKS32[24]; + + /** Decode 128 integers into {@code longs}. */ + void decode(int bitsPerValue, DataInput in, long[] longs) throws IOException { + switch (bitsPerValue) { + case 1: + decode1(in, tmp, longs); + expand8(longs); + break; + case 2: + decode2(in, tmp, longs); + expand8(longs); + break; + case 3: + decode3(in, tmp, longs); + expand8(longs); + break; + case 4: + decode4(in, tmp, longs); + expand8(longs); + break; + case 5: + decode5(in, tmp, longs); + expand8(longs); + break; + case 6: + decode6(in, tmp, longs); + expand8(longs); + break; + case 7: + decode7(in, tmp, longs); + expand8(longs); + break; + case 8: + decode8(in, tmp, longs); + expand8(longs); + break; + case 9: + decode9(in, tmp, longs); + expand16(longs); + break; + case 10: + decode10(in, tmp, longs); + expand16(longs); + break; + case 11: + decode11(in, tmp, longs); + expand16(longs); + break; + case 12: + decode12(in, tmp, longs); + expand16(longs); + break; + case 13: + decode13(in, tmp, longs); + expand16(longs); + break; + case 14: + decode14(in, tmp, longs); + expand16(longs); + break; + case 15: + decode15(in, tmp, longs); + expand16(longs); + break; + case 16: + decode16(in, tmp, longs); + expand16(longs); + break; + case 17: + decode17(in, tmp, longs); + expand32(longs); + break; + case 18: + decode18(in, tmp, longs); + expand32(longs); + break; + case 19: + decode19(in, tmp, longs); + expand32(longs); + break; + case 20: + decode20(in, tmp, longs); + expand32(longs); + break; + case 21: + decode21(in, tmp, longs); + expand32(longs); + break; + case 22: + decode22(in, tmp, longs); + expand32(longs); + break; + case 23: + decode23(in, tmp, longs); + expand32(longs); + break; + case 24: + decode24(in, tmp, longs); + expand32(longs); + break; + default: + decodeSlow(bitsPerValue, in, tmp, longs); + expand32(longs); + break; + } + } + + /** Delta-decode 128 integers into {@code longs}. */ + void decodeAndPrefixSum(int bitsPerValue, DataInput in, long base, long[] longs) + throws IOException { + switch (bitsPerValue) { + case 1: + decode1(in, tmp, longs); + prefixSum8(longs, base); + break; + case 2: + decode2(in, tmp, longs); + prefixSum8(longs, base); + break; + case 3: + decode3(in, tmp, longs); + prefixSum8(longs, base); + break; + case 4: + decode4(in, tmp, longs); + prefixSum8(longs, base); + break; + case 5: + decode5(in, tmp, longs); + prefixSum8(longs, base); + break; + case 6: + decode6(in, tmp, longs); + prefixSum8(longs, base); + break; + case 7: + decode7(in, tmp, longs); + prefixSum8(longs, base); + break; + case 8: + decode8(in, tmp, longs); + prefixSum8(longs, base); + break; + case 9: + decode9(in, tmp, longs); + prefixSum16(longs, base); + break; + case 10: + decode10(in, tmp, longs); + prefixSum16(longs, base); + break; + case 11: + decode11(in, tmp, longs); + prefixSum16(longs, base); + break; + case 12: + decode12(in, tmp, longs); + prefixSum16(longs, base); + break; + case 13: + decode13(in, tmp, longs); + prefixSum16(longs, base); + break; + case 14: + decode14(in, tmp, longs); + prefixSum16(longs, base); + break; + case 15: + decode15(in, tmp, longs); + prefixSum16(longs, base); + break; + case 16: + decode16(in, tmp, longs); + prefixSum16(longs, base); + break; + case 17: + decode17(in, tmp, longs); + prefixSum32(longs, base); + break; + case 18: + decode18(in, tmp, longs); + prefixSum32(longs, base); + break; + case 19: + decode19(in, tmp, longs); + prefixSum32(longs, base); + break; + case 20: + decode20(in, tmp, longs); + prefixSum32(longs, base); + break; + case 21: + decode21(in, tmp, longs); + prefixSum32(longs, base); + break; + case 22: + decode22(in, tmp, longs); + prefixSum32(longs, base); + break; + case 23: + decode23(in, tmp, longs); + prefixSum32(longs, base); + break; + case 24: + decode24(in, tmp, longs); + prefixSum32(longs, base); + break; + default: + decodeSlow(bitsPerValue, in, tmp, longs); + prefixSum32(longs, base); + break; + } + } + + private static void decode1(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 2); + shiftLongs(tmp, 2, longs, 0, 7, MASK8_1); + shiftLongs(tmp, 2, longs, 2, 6, MASK8_1); + shiftLongs(tmp, 2, longs, 4, 5, MASK8_1); + shiftLongs(tmp, 2, longs, 6, 4, MASK8_1); + shiftLongs(tmp, 2, longs, 8, 3, MASK8_1); + shiftLongs(tmp, 2, longs, 10, 2, MASK8_1); + shiftLongs(tmp, 2, longs, 12, 1, MASK8_1); + shiftLongs(tmp, 2, longs, 14, 0, MASK8_1); + } + + private static void decode2(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 4); + shiftLongs(tmp, 4, longs, 0, 6, MASK8_2); + shiftLongs(tmp, 4, longs, 4, 4, MASK8_2); + shiftLongs(tmp, 4, longs, 8, 2, MASK8_2); + shiftLongs(tmp, 4, longs, 12, 0, MASK8_2); + } + + private static void decode3(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 6); + shiftLongs(tmp, 6, longs, 0, 5, MASK8_3); + shiftLongs(tmp, 6, longs, 6, 2, MASK8_3); + for (int iter = 0, tmpIdx = 0, longsIdx = 12; iter < 2; ++iter, tmpIdx += 3, longsIdx += 2) { + long l0 = (tmp[tmpIdx + 0] & MASK8_2) << 1; + l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK8_1; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK8_1) << 2; + l1 |= (tmp[tmpIdx + 2] & MASK8_2) << 0; + longs[longsIdx + 1] = l1; + } + } + + private static void decode4(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 8); + shiftLongs(tmp, 8, longs, 0, 4, MASK8_4); + shiftLongs(tmp, 8, longs, 8, 0, MASK8_4); + } + + private static void decode5(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 10); + shiftLongs(tmp, 10, longs, 0, 3, MASK8_5); + for (int iter = 0, tmpIdx = 0, longsIdx = 10; iter < 2; ++iter, tmpIdx += 5, longsIdx += 3) { + long l0 = (tmp[tmpIdx + 0] & MASK8_3) << 2; + l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK8_2; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK8_1) << 4; + l1 |= (tmp[tmpIdx + 2] & MASK8_3) << 1; + l1 |= (tmp[tmpIdx + 3] >>> 2) & MASK8_1; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 3] & MASK8_2) << 3; + l2 |= (tmp[tmpIdx + 4] & MASK8_3) << 0; + longs[longsIdx + 2] = l2; + } + } + + private static void decode6(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 12); + shiftLongs(tmp, 12, longs, 0, 2, MASK8_6); + shiftLongs(tmp, 12, tmp, 0, 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; + l0 |= tmp[tmpIdx + 2] << 0; + longs[longsIdx + 0] = l0; + } + } + + private static void decode7(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 14); + shiftLongs(tmp, 14, longs, 0, 1, MASK8_7); + shiftLongs(tmp, 14, tmp, 0, 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; + 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; + longs[longsIdx + 0] = l0; + } + } + + private static void decode8(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(longs, 0, 16); + } + + private static void decode9(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 18); + shiftLongs(tmp, 18, longs, 0, 7, MASK16_9); + for (int iter = 0, tmpIdx = 0, longsIdx = 18; iter < 2; ++iter, tmpIdx += 9, longsIdx += 7) { + long l0 = (tmp[tmpIdx + 0] & MASK16_7) << 2; + l0 |= (tmp[tmpIdx + 1] >>> 5) & MASK16_2; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK16_5) << 4; + l1 |= (tmp[tmpIdx + 2] >>> 3) & MASK16_4; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 2] & MASK16_3) << 6; + l2 |= (tmp[tmpIdx + 3] >>> 1) & MASK16_6; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 3] & MASK16_1) << 8; + l3 |= (tmp[tmpIdx + 4] & MASK16_7) << 1; + l3 |= (tmp[tmpIdx + 5] >>> 6) & MASK16_1; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 5] & MASK16_6) << 3; + l4 |= (tmp[tmpIdx + 6] >>> 4) & MASK16_3; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 6] & MASK16_4) << 5; + l5 |= (tmp[tmpIdx + 7] >>> 2) & MASK16_5; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 7] & MASK16_2) << 7; + l6 |= (tmp[tmpIdx + 8] & MASK16_7) << 0; + longs[longsIdx + 6] = l6; + } + } + + private static void decode10(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 20); + shiftLongs(tmp, 20, longs, 0, 6, MASK16_10); + for (int iter = 0, tmpIdx = 0, longsIdx = 20; iter < 4; ++iter, tmpIdx += 5, longsIdx += 3) { + long l0 = (tmp[tmpIdx + 0] & MASK16_6) << 4; + l0 |= (tmp[tmpIdx + 1] >>> 2) & MASK16_4; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK16_2) << 8; + l1 |= (tmp[tmpIdx + 2] & MASK16_6) << 2; + l1 |= (tmp[tmpIdx + 3] >>> 4) & MASK16_2; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 3] & MASK16_4) << 6; + l2 |= (tmp[tmpIdx + 4] & MASK16_6) << 0; + longs[longsIdx + 2] = l2; + } + } + + private static void decode11(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 22); + shiftLongs(tmp, 22, longs, 0, 5, MASK16_11); + for (int iter = 0, tmpIdx = 0, longsIdx = 22; iter < 2; ++iter, tmpIdx += 11, longsIdx += 5) { + long l0 = (tmp[tmpIdx + 0] & MASK16_5) << 6; + l0 |= (tmp[tmpIdx + 1] & MASK16_5) << 1; + l0 |= (tmp[tmpIdx + 2] >>> 4) & MASK16_1; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 2] & MASK16_4) << 7; + l1 |= (tmp[tmpIdx + 3] & MASK16_5) << 2; + l1 |= (tmp[tmpIdx + 4] >>> 3) & MASK16_2; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 4] & MASK16_3) << 8; + l2 |= (tmp[tmpIdx + 5] & MASK16_5) << 3; + l2 |= (tmp[tmpIdx + 6] >>> 2) & MASK16_3; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 6] & MASK16_2) << 9; + l3 |= (tmp[tmpIdx + 7] & MASK16_5) << 4; + l3 |= (tmp[tmpIdx + 8] >>> 1) & MASK16_4; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 8] & MASK16_1) << 10; + l4 |= (tmp[tmpIdx + 9] & MASK16_5) << 5; + l4 |= (tmp[tmpIdx + 10] & MASK16_5) << 0; + longs[longsIdx + 4] = l4; + } + } + + private static void decode12(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 24); + shiftLongs(tmp, 24, longs, 0, 4, MASK16_12); + shiftLongs(tmp, 24, tmp, 0, 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; + l0 |= tmp[tmpIdx + 2] << 0; + longs[longsIdx + 0] = l0; + } + } + + private static void decode13(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 26); + shiftLongs(tmp, 26, longs, 0, 3, MASK16_13); + for (int iter = 0, tmpIdx = 0, longsIdx = 26; iter < 2; ++iter, tmpIdx += 13, longsIdx += 3) { + long l0 = (tmp[tmpIdx + 0] & MASK16_3) << 10; + l0 |= (tmp[tmpIdx + 1] & MASK16_3) << 7; + l0 |= (tmp[tmpIdx + 2] & MASK16_3) << 4; + l0 |= (tmp[tmpIdx + 3] & MASK16_3) << 1; + l0 |= (tmp[tmpIdx + 4] >>> 2) & MASK16_1; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 4] & MASK16_2) << 11; + l1 |= (tmp[tmpIdx + 5] & MASK16_3) << 8; + l1 |= (tmp[tmpIdx + 6] & MASK16_3) << 5; + l1 |= (tmp[tmpIdx + 7] & MASK16_3) << 2; + l1 |= (tmp[tmpIdx + 8] >>> 1) & MASK16_2; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 8] & MASK16_1) << 12; + l2 |= (tmp[tmpIdx + 9] & MASK16_3) << 9; + l2 |= (tmp[tmpIdx + 10] & MASK16_3) << 6; + l2 |= (tmp[tmpIdx + 11] & MASK16_3) << 3; + l2 |= (tmp[tmpIdx + 12] & MASK16_3) << 0; + longs[longsIdx + 2] = l2; + } + } + + private static void decode14(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 28); + shiftLongs(tmp, 28, longs, 0, 2, MASK16_14); + shiftLongs(tmp, 28, tmp, 0, 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; + 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; + longs[longsIdx + 0] = l0; + } + } + + private static void decode15(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 30); + shiftLongs(tmp, 30, longs, 0, 1, MASK16_15); + shiftLongs(tmp, 30, tmp, 0, 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; + 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; + longs[longsIdx + 0] = l0; + } + } + + private static void decode16(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(longs, 0, 32); + } + + private static void decode17(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 34); + shiftLongs(tmp, 34, longs, 0, 15, MASK32_17); + for (int iter = 0, tmpIdx = 0, longsIdx = 34; iter < 2; ++iter, tmpIdx += 17, longsIdx += 15) { + long l0 = (tmp[tmpIdx + 0] & MASK32_15) << 2; + l0 |= (tmp[tmpIdx + 1] >>> 13) & MASK32_2; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_13) << 4; + l1 |= (tmp[tmpIdx + 2] >>> 11) & MASK32_4; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 2] & MASK32_11) << 6; + l2 |= (tmp[tmpIdx + 3] >>> 9) & MASK32_6; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 3] & MASK32_9) << 8; + l3 |= (tmp[tmpIdx + 4] >>> 7) & MASK32_8; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 4] & MASK32_7) << 10; + l4 |= (tmp[tmpIdx + 5] >>> 5) & MASK32_10; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 5] & MASK32_5) << 12; + l5 |= (tmp[tmpIdx + 6] >>> 3) & MASK32_12; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 6] & MASK32_3) << 14; + l6 |= (tmp[tmpIdx + 7] >>> 1) & MASK32_14; + longs[longsIdx + 6] = l6; + long l7 = (tmp[tmpIdx + 7] & MASK32_1) << 16; + l7 |= (tmp[tmpIdx + 8] & MASK32_15) << 1; + l7 |= (tmp[tmpIdx + 9] >>> 14) & MASK32_1; + longs[longsIdx + 7] = l7; + long l8 = (tmp[tmpIdx + 9] & MASK32_14) << 3; + l8 |= (tmp[tmpIdx + 10] >>> 12) & MASK32_3; + longs[longsIdx + 8] = l8; + long l9 = (tmp[tmpIdx + 10] & MASK32_12) << 5; + l9 |= (tmp[tmpIdx + 11] >>> 10) & MASK32_5; + longs[longsIdx + 9] = l9; + long l10 = (tmp[tmpIdx + 11] & MASK32_10) << 7; + l10 |= (tmp[tmpIdx + 12] >>> 8) & MASK32_7; + longs[longsIdx + 10] = l10; + long l11 = (tmp[tmpIdx + 12] & MASK32_8) << 9; + l11 |= (tmp[tmpIdx + 13] >>> 6) & MASK32_9; + longs[longsIdx + 11] = l11; + long l12 = (tmp[tmpIdx + 13] & MASK32_6) << 11; + l12 |= (tmp[tmpIdx + 14] >>> 4) & MASK32_11; + longs[longsIdx + 12] = l12; + long l13 = (tmp[tmpIdx + 14] & MASK32_4) << 13; + l13 |= (tmp[tmpIdx + 15] >>> 2) & MASK32_13; + longs[longsIdx + 13] = l13; + long l14 = (tmp[tmpIdx + 15] & MASK32_2) << 15; + l14 |= (tmp[tmpIdx + 16] & MASK32_15) << 0; + longs[longsIdx + 14] = l14; + } + } + + private static void decode18(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 36); + shiftLongs(tmp, 36, longs, 0, 14, MASK32_18); + for (int iter = 0, tmpIdx = 0, longsIdx = 36; iter < 4; ++iter, tmpIdx += 9, longsIdx += 7) { + long l0 = (tmp[tmpIdx + 0] & MASK32_14) << 4; + l0 |= (tmp[tmpIdx + 1] >>> 10) & MASK32_4; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_10) << 8; + l1 |= (tmp[tmpIdx + 2] >>> 6) & MASK32_8; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 2] & MASK32_6) << 12; + l2 |= (tmp[tmpIdx + 3] >>> 2) & MASK32_12; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 3] & MASK32_2) << 16; + l3 |= (tmp[tmpIdx + 4] & MASK32_14) << 2; + l3 |= (tmp[tmpIdx + 5] >>> 12) & MASK32_2; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 5] & MASK32_12) << 6; + l4 |= (tmp[tmpIdx + 6] >>> 8) & MASK32_6; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 6] & MASK32_8) << 10; + l5 |= (tmp[tmpIdx + 7] >>> 4) & MASK32_10; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 7] & MASK32_4) << 14; + l6 |= (tmp[tmpIdx + 8] & MASK32_14) << 0; + longs[longsIdx + 6] = l6; + } + } + + private static void decode19(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 38); + shiftLongs(tmp, 38, longs, 0, 13, MASK32_19); + for (int iter = 0, tmpIdx = 0, longsIdx = 38; iter < 2; ++iter, tmpIdx += 19, longsIdx += 13) { + long l0 = (tmp[tmpIdx + 0] & MASK32_13) << 6; + l0 |= (tmp[tmpIdx + 1] >>> 7) & MASK32_6; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_7) << 12; + l1 |= (tmp[tmpIdx + 2] >>> 1) & MASK32_12; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 2] & MASK32_1) << 18; + l2 |= (tmp[tmpIdx + 3] & MASK32_13) << 5; + l2 |= (tmp[tmpIdx + 4] >>> 8) & MASK32_5; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 4] & MASK32_8) << 11; + l3 |= (tmp[tmpIdx + 5] >>> 2) & MASK32_11; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 5] & MASK32_2) << 17; + l4 |= (tmp[tmpIdx + 6] & MASK32_13) << 4; + l4 |= (tmp[tmpIdx + 7] >>> 9) & MASK32_4; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 7] & MASK32_9) << 10; + l5 |= (tmp[tmpIdx + 8] >>> 3) & MASK32_10; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 8] & MASK32_3) << 16; + l6 |= (tmp[tmpIdx + 9] & MASK32_13) << 3; + l6 |= (tmp[tmpIdx + 10] >>> 10) & MASK32_3; + longs[longsIdx + 6] = l6; + long l7 = (tmp[tmpIdx + 10] & MASK32_10) << 9; + l7 |= (tmp[tmpIdx + 11] >>> 4) & MASK32_9; + longs[longsIdx + 7] = l7; + long l8 = (tmp[tmpIdx + 11] & MASK32_4) << 15; + l8 |= (tmp[tmpIdx + 12] & MASK32_13) << 2; + l8 |= (tmp[tmpIdx + 13] >>> 11) & MASK32_2; + longs[longsIdx + 8] = l8; + long l9 = (tmp[tmpIdx + 13] & MASK32_11) << 8; + l9 |= (tmp[tmpIdx + 14] >>> 5) & MASK32_8; + longs[longsIdx + 9] = l9; + long l10 = (tmp[tmpIdx + 14] & MASK32_5) << 14; + l10 |= (tmp[tmpIdx + 15] & MASK32_13) << 1; + l10 |= (tmp[tmpIdx + 16] >>> 12) & MASK32_1; + longs[longsIdx + 10] = l10; + long l11 = (tmp[tmpIdx + 16] & MASK32_12) << 7; + l11 |= (tmp[tmpIdx + 17] >>> 6) & MASK32_7; + longs[longsIdx + 11] = l11; + long l12 = (tmp[tmpIdx + 17] & MASK32_6) << 13; + l12 |= (tmp[tmpIdx + 18] & MASK32_13) << 0; + longs[longsIdx + 12] = l12; + } + } + + private static void decode20(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 40); + shiftLongs(tmp, 40, longs, 0, 12, MASK32_20); + for (int iter = 0, tmpIdx = 0, longsIdx = 40; iter < 8; ++iter, tmpIdx += 5, longsIdx += 3) { + long l0 = (tmp[tmpIdx + 0] & MASK32_12) << 8; + l0 |= (tmp[tmpIdx + 1] >>> 4) & MASK32_8; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_4) << 16; + l1 |= (tmp[tmpIdx + 2] & MASK32_12) << 4; + l1 |= (tmp[tmpIdx + 3] >>> 8) & MASK32_4; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 3] & MASK32_8) << 12; + l2 |= (tmp[tmpIdx + 4] & MASK32_12) << 0; + longs[longsIdx + 2] = l2; + } + } + + private static void decode21(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 42); + shiftLongs(tmp, 42, longs, 0, 11, MASK32_21); + for (int iter = 0, tmpIdx = 0, longsIdx = 42; iter < 2; ++iter, tmpIdx += 21, longsIdx += 11) { + long l0 = (tmp[tmpIdx + 0] & MASK32_11) << 10; + l0 |= (tmp[tmpIdx + 1] >>> 1) & MASK32_10; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 1] & MASK32_1) << 20; + l1 |= (tmp[tmpIdx + 2] & MASK32_11) << 9; + l1 |= (tmp[tmpIdx + 3] >>> 2) & MASK32_9; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 3] & MASK32_2) << 19; + l2 |= (tmp[tmpIdx + 4] & MASK32_11) << 8; + l2 |= (tmp[tmpIdx + 5] >>> 3) & MASK32_8; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 5] & MASK32_3) << 18; + l3 |= (tmp[tmpIdx + 6] & MASK32_11) << 7; + l3 |= (tmp[tmpIdx + 7] >>> 4) & MASK32_7; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 7] & MASK32_4) << 17; + l4 |= (tmp[tmpIdx + 8] & MASK32_11) << 6; + l4 |= (tmp[tmpIdx + 9] >>> 5) & MASK32_6; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 9] & MASK32_5) << 16; + l5 |= (tmp[tmpIdx + 10] & MASK32_11) << 5; + l5 |= (tmp[tmpIdx + 11] >>> 6) & MASK32_5; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 11] & MASK32_6) << 15; + l6 |= (tmp[tmpIdx + 12] & MASK32_11) << 4; + l6 |= (tmp[tmpIdx + 13] >>> 7) & MASK32_4; + longs[longsIdx + 6] = l6; + long l7 = (tmp[tmpIdx + 13] & MASK32_7) << 14; + l7 |= (tmp[tmpIdx + 14] & MASK32_11) << 3; + l7 |= (tmp[tmpIdx + 15] >>> 8) & MASK32_3; + longs[longsIdx + 7] = l7; + long l8 = (tmp[tmpIdx + 15] & MASK32_8) << 13; + l8 |= (tmp[tmpIdx + 16] & MASK32_11) << 2; + l8 |= (tmp[tmpIdx + 17] >>> 9) & MASK32_2; + longs[longsIdx + 8] = l8; + long l9 = (tmp[tmpIdx + 17] & MASK32_9) << 12; + l9 |= (tmp[tmpIdx + 18] & MASK32_11) << 1; + l9 |= (tmp[tmpIdx + 19] >>> 10) & MASK32_1; + longs[longsIdx + 9] = l9; + long l10 = (tmp[tmpIdx + 19] & MASK32_10) << 11; + l10 |= (tmp[tmpIdx + 20] & MASK32_11) << 0; + longs[longsIdx + 10] = l10; + } + } + + private static void decode22(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 44); + shiftLongs(tmp, 44, longs, 0, 10, MASK32_22); + for (int iter = 0, tmpIdx = 0, longsIdx = 44; iter < 4; ++iter, tmpIdx += 11, longsIdx += 5) { + long l0 = (tmp[tmpIdx + 0] & MASK32_10) << 12; + l0 |= (tmp[tmpIdx + 1] & MASK32_10) << 2; + l0 |= (tmp[tmpIdx + 2] >>> 8) & MASK32_2; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 2] & MASK32_8) << 14; + l1 |= (tmp[tmpIdx + 3] & MASK32_10) << 4; + l1 |= (tmp[tmpIdx + 4] >>> 6) & MASK32_4; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 4] & MASK32_6) << 16; + l2 |= (tmp[tmpIdx + 5] & MASK32_10) << 6; + l2 |= (tmp[tmpIdx + 6] >>> 4) & MASK32_6; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 6] & MASK32_4) << 18; + l3 |= (tmp[tmpIdx + 7] & MASK32_10) << 8; + l3 |= (tmp[tmpIdx + 8] >>> 2) & MASK32_8; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 8] & MASK32_2) << 20; + l4 |= (tmp[tmpIdx + 9] & MASK32_10) << 10; + l4 |= (tmp[tmpIdx + 10] & MASK32_10) << 0; + longs[longsIdx + 4] = l4; + } + } + + private static void decode23(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 46); + shiftLongs(tmp, 46, longs, 0, 9, MASK32_23); + for (int iter = 0, tmpIdx = 0, longsIdx = 46; iter < 2; ++iter, tmpIdx += 23, longsIdx += 9) { + long l0 = (tmp[tmpIdx + 0] & MASK32_9) << 14; + l0 |= (tmp[tmpIdx + 1] & MASK32_9) << 5; + l0 |= (tmp[tmpIdx + 2] >>> 4) & MASK32_5; + longs[longsIdx + 0] = l0; + long l1 = (tmp[tmpIdx + 2] & MASK32_4) << 19; + l1 |= (tmp[tmpIdx + 3] & MASK32_9) << 10; + l1 |= (tmp[tmpIdx + 4] & MASK32_9) << 1; + l1 |= (tmp[tmpIdx + 5] >>> 8) & MASK32_1; + longs[longsIdx + 1] = l1; + long l2 = (tmp[tmpIdx + 5] & MASK32_8) << 15; + l2 |= (tmp[tmpIdx + 6] & MASK32_9) << 6; + l2 |= (tmp[tmpIdx + 7] >>> 3) & MASK32_6; + longs[longsIdx + 2] = l2; + long l3 = (tmp[tmpIdx + 7] & MASK32_3) << 20; + l3 |= (tmp[tmpIdx + 8] & MASK32_9) << 11; + l3 |= (tmp[tmpIdx + 9] & MASK32_9) << 2; + l3 |= (tmp[tmpIdx + 10] >>> 7) & MASK32_2; + longs[longsIdx + 3] = l3; + long l4 = (tmp[tmpIdx + 10] & MASK32_7) << 16; + l4 |= (tmp[tmpIdx + 11] & MASK32_9) << 7; + l4 |= (tmp[tmpIdx + 12] >>> 2) & MASK32_7; + longs[longsIdx + 4] = l4; + long l5 = (tmp[tmpIdx + 12] & MASK32_2) << 21; + l5 |= (tmp[tmpIdx + 13] & MASK32_9) << 12; + l5 |= (tmp[tmpIdx + 14] & MASK32_9) << 3; + l5 |= (tmp[tmpIdx + 15] >>> 6) & MASK32_3; + longs[longsIdx + 5] = l5; + long l6 = (tmp[tmpIdx + 15] & MASK32_6) << 17; + l6 |= (tmp[tmpIdx + 16] & MASK32_9) << 8; + l6 |= (tmp[tmpIdx + 17] >>> 1) & MASK32_8; + longs[longsIdx + 6] = l6; + long l7 = (tmp[tmpIdx + 17] & MASK32_1) << 22; + l7 |= (tmp[tmpIdx + 18] & MASK32_9) << 13; + l7 |= (tmp[tmpIdx + 19] & MASK32_9) << 4; + l7 |= (tmp[tmpIdx + 20] >>> 5) & MASK32_4; + longs[longsIdx + 7] = l7; + long l8 = (tmp[tmpIdx + 20] & MASK32_5) << 18; + l8 |= (tmp[tmpIdx + 21] & MASK32_9) << 9; + l8 |= (tmp[tmpIdx + 22] & MASK32_9) << 0; + longs[longsIdx + 8] = l8; + } + } + + private static void decode24(DataInput in, long[] tmp, long[] longs) throws IOException { + in.readLongs(tmp, 0, 48); + shiftLongs(tmp, 48, longs, 0, 8, MASK32_24); + shiftLongs(tmp, 48, tmp, 0, 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; + l0 |= tmp[tmpIdx + 2] << 0; + longs[longsIdx + 0] = l0; + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99Codec.java similarity index 82% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99Codec.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99Codec.java index ce6c40f9402..d540abc85c2 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99Codec.java @@ -14,12 +14,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import java.util.Objects; -import org.apache.lucene.codecs.*; -import org.apache.lucene.codecs.lucene90.*; +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.lucene912.Lucene912PostingsFormat; 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; @@ -98,7 +119,7 @@ public class Lucene99Codec extends Codec { super("Lucene99"); this.storedFieldsFormat = new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode); - this.defaultPostingsFormat = new Lucene99PostingsFormat(); + this.defaultPostingsFormat = new Lucene912PostingsFormat(); this.defaultDVFormat = new Lucene90DocValuesFormat(); this.defaultKnnVectorsFormat = new Lucene99HnswVectorsFormat(); } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99PostingsFormat.java similarity index 95% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsFormat.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99PostingsFormat.java index 877746641b4..7ff614d684e 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsFormat.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/Lucene99PostingsFormat.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import java.io.IOException; import org.apache.lucene.codecs.BlockTermState; @@ -24,7 +24,6 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.MultiLevelSkipListWriter; 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; @@ -339,7 +338,7 @@ import org.apache.lucene.util.packed.PackedInts; * * @lucene.experimental */ -public final class Lucene99PostingsFormat extends PostingsFormat { +public class Lucene99PostingsFormat extends PostingsFormat { /** * Filename extension for document number, frequencies, and skip data. See chapter: impacts = acc.getCompetitiveFreqNormPairs(); diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/PForUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/PForUtil.java new file mode 100644 index 00000000000..9eddd3a0c55 --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/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.backward_codecs.lucene99; + +import java.io.IOException; +import java.util.Arrays; +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(long[] l) { + for (int i = 1; i < ForUtil.BLOCK_SIZE; ++i) { + if (l[i] != l[0]) { + return false; + } + } + return true; + } + + private final ForUtil forUtil; + + PForUtil(ForUtil forUtil) { + assert ForUtil.BLOCK_SIZE <= 256 : "blocksize must fit in one byte. got " + ForUtil.BLOCK_SIZE; + this.forUtil = forUtil; + } + + /** Encode 128 integers from {@code longs} into {@code out}. */ + void encode(long[] longs, 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(longs[i]); + } + long topValue = top.top(); + for (int i = MAX_EXCEPTIONS + 1; i < ForUtil.BLOCK_SIZE; ++i) { + if (longs[i] > topValue) { + topValue = top.updateTop(longs[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 (longs[i] > maxUnpatchedValue) { + exceptions[exceptionCount * 2] = (byte) i; + exceptions[exceptionCount * 2 + 1] = (byte) (longs[i] >>> patchedBitsRequired); + longs[i] &= maxUnpatchedValue; + exceptionCount++; + } + } + assert exceptionCount == numExceptions : exceptionCount + " " + numExceptions; + } + + if (allEqual(longs) && 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.writeVLong(longs[0]); + } else { + final int token = (numExceptions << 5) | patchedBitsRequired; + out.writeByte((byte) token); + forUtil.encode(longs, patchedBitsRequired, out); + } + out.writeBytes(exceptions, exceptions.length); + } + + /** Decode 128 integers into {@code ints}. */ + void decode(DataInput in, long[] longs) throws IOException { + final int token = Byte.toUnsignedInt(in.readByte()); + final int bitsPerValue = token & 0x1f; + final int numExceptions = token >>> 5; + if (bitsPerValue == 0) { + Arrays.fill(longs, 0, ForUtil.BLOCK_SIZE, in.readVLong()); + } else { + forUtil.decode(bitsPerValue, in, longs); + } + for (int i = 0; i < numExceptions; ++i) { + longs[Byte.toUnsignedInt(in.readByte())] |= + Byte.toUnsignedLong(in.readByte()) << bitsPerValue; + } + } + + /** Skip 128 integers. */ + 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/lucene99/PostingsUtil.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/PostingsUtil.java similarity index 97% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/PostingsUtil.java rename to lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/PostingsUtil.java index 678754047b6..7b95bada5bc 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/PostingsUtil.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/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.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import java.io.IOException; import org.apache.lucene.store.IndexInput; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/gen_ForUtil.py b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/gen_ForUtil.py new file mode 100644 index 00000000000..97a3a49a4ff --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/gen_ForUtil.py @@ -0,0 +1,524 @@ +#! /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.backward_codecs.lucene99; + +import java.io.IOException; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.DataOutput; + +// 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 +final class ForUtil { + + static final int BLOCK_SIZE = 128; + private static final int BLOCK_SIZE_LOG2 = 7; + + private static long expandMask32(long mask32) { + return mask32 | (mask32 << 32); + } + + private static long expandMask16(long mask16) { + return expandMask32(mask16 | (mask16 << 16)); + } + + private static long expandMask8(long mask8) { + return expandMask16(mask8 | (mask8 << 8)); + } + + private static long mask32(int bitsPerValue) { + return expandMask32((1L << bitsPerValue) - 1); + } + + private static long mask16(int bitsPerValue) { + return expandMask16((1L << bitsPerValue) - 1); + } + + private static long mask8(int bitsPerValue) { + return expandMask8((1L << bitsPerValue) - 1); + } + + private static void expand8(long[] arr) { + for (int i = 0; i < 16; ++i) { + long l = arr[i]; + arr[i] = (l >>> 56) & 0xFFL; + arr[16 + i] = (l >>> 48) & 0xFFL; + arr[32 + i] = (l >>> 40) & 0xFFL; + arr[48 + i] = (l >>> 32) & 0xFFL; + arr[64 + i] = (l >>> 24) & 0xFFL; + arr[80 + i] = (l >>> 16) & 0xFFL; + arr[96 + i] = (l >>> 8) & 0xFFL; + arr[112 + i] = l & 0xFFL; + } + } + + private static void expand8To32(long[] arr) { + for (int i = 0; i < 16; ++i) { + long l = arr[i]; + arr[i] = (l >>> 24) & 0x000000FF000000FFL; + arr[16 + i] = (l >>> 16) & 0x000000FF000000FFL; + arr[32 + i] = (l >>> 8) & 0x000000FF000000FFL; + arr[48 + i] = l & 0x000000FF000000FFL; + } + } + + private static void collapse8(long[] arr) { + for (int i = 0; i < 16; ++i) { + arr[i] = + (arr[i] << 56) + | (arr[16 + i] << 48) + | (arr[32 + i] << 40) + | (arr[48 + i] << 32) + | (arr[64 + i] << 24) + | (arr[80 + i] << 16) + | (arr[96 + i] << 8) + | arr[112 + i]; + } + } + + private static void expand16(long[] arr) { + for (int i = 0; i < 32; ++i) { + long l = arr[i]; + arr[i] = (l >>> 48) & 0xFFFFL; + arr[32 + i] = (l >>> 32) & 0xFFFFL; + arr[64 + i] = (l >>> 16) & 0xFFFFL; + arr[96 + i] = l & 0xFFFFL; + } + } + + private static void expand16To32(long[] arr) { + for (int i = 0; i < 32; ++i) { + long l = arr[i]; + arr[i] = (l >>> 16) & 0x0000FFFF0000FFFFL; + arr[32 + i] = l & 0x0000FFFF0000FFFFL; + } + } + + private static void collapse16(long[] arr) { + for (int i = 0; i < 32; ++i) { + arr[i] = (arr[i] << 48) | (arr[32 + i] << 32) | (arr[64 + i] << 16) | arr[96 + i]; + } + } + + private static void expand32(long[] arr) { + for (int i = 0; i < 64; ++i) { + long l = arr[i]; + arr[i] = l >>> 32; + arr[64 + i] = l & 0xFFFFFFFFL; + } + } + + private static void collapse32(long[] arr) { + for (int i = 0; i < 64; ++i) { + arr[i] = (arr[i] << 32) | arr[64 + i]; + } + } + + private static void prefixSum8(long[] arr, long base) { + expand8To32(arr); + prefixSum32(arr, base); + } + + private static void prefixSum16(long[] arr, long base) { + // We need to move to the next primitive size to avoid overflows + expand16To32(arr); + prefixSum32(arr, base); + } + + private static void prefixSum32(long[] arr, long base) { + arr[0] += base << 32; + innerPrefixSum32(arr); + expand32(arr); + final long l = arr[BLOCK_SIZE/2-1]; + for (int i = BLOCK_SIZE/2; i < BLOCK_SIZE; ++i) { + arr[i] += l; + } + } + + // For some reason unrolling seems to help + private static void innerPrefixSum32(long[] 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 long[] tmp = new long[BLOCK_SIZE / 2]; + + /** Encode 128 integers from {@code longs} into {@code out}. */ + void encode(long[] longs, int bitsPerValue, DataOutput out) throws IOException { + final int nextPrimitive; + final int numLongs; + if (bitsPerValue <= 8) { + nextPrimitive = 8; + numLongs = BLOCK_SIZE / 8; + collapse8(longs); + } else if (bitsPerValue <= 16) { + nextPrimitive = 16; + numLongs = BLOCK_SIZE / 4; + collapse16(longs); + } else { + nextPrimitive = 32; + numLongs = BLOCK_SIZE / 2; + collapse32(longs); + } + + final int numLongsPerShift = bitsPerValue * 2; + int idx = 0; + int shift = nextPrimitive - bitsPerValue; + for (int i = 0; i < numLongsPerShift; ++i) { + tmp[i] = longs[idx++] << shift; + } + for (shift = shift - bitsPerValue; shift >= 0; shift -= bitsPerValue) { + for (int i = 0; i < numLongsPerShift; ++i) { + tmp[i] |= longs[idx++] << shift; + } + } + + final int remainingBitsPerLong = shift + bitsPerValue; + final long maskRemainingBitsPerLong; + if (nextPrimitive == 8) { + maskRemainingBitsPerLong = MASKS8[remainingBitsPerLong]; + } else if (nextPrimitive == 16) { + maskRemainingBitsPerLong = MASKS16[remainingBitsPerLong]; + } else { + maskRemainingBitsPerLong = MASKS32[remainingBitsPerLong]; + } + + int tmpIdx = 0; + int remainingBitsPerValue = bitsPerValue; + while (idx < numLongs) { + if (remainingBitsPerValue >= remainingBitsPerLong) { + remainingBitsPerValue -= remainingBitsPerLong; + tmp[tmpIdx++] |= (longs[idx] >>> remainingBitsPerValue) & maskRemainingBitsPerLong; + if (remainingBitsPerValue == 0) { + idx++; + remainingBitsPerValue = bitsPerValue; + } + } else { + final long mask1, mask2; + if (nextPrimitive == 8) { + mask1 = MASKS8[remainingBitsPerValue]; + mask2 = MASKS8[remainingBitsPerLong - remainingBitsPerValue]; + } else if (nextPrimitive == 16) { + mask1 = MASKS16[remainingBitsPerValue]; + mask2 = MASKS16[remainingBitsPerLong - remainingBitsPerValue]; + } else { + mask1 = MASKS32[remainingBitsPerValue]; + mask2 = MASKS32[remainingBitsPerLong - remainingBitsPerValue]; + } + tmp[tmpIdx] |= (longs[idx++] & mask1) << (remainingBitsPerLong - remainingBitsPerValue); + remainingBitsPerValue = bitsPerValue - remainingBitsPerLong + remainingBitsPerValue; + tmp[tmpIdx++] |= (longs[idx] >>> remainingBitsPerValue) & mask2; + } + } + + for (int i = 0; i < numLongsPerShift; ++i) { + out.writeLong(tmp[i]); + } + } + + /** Number of bytes required to encode 128 integers of {@code bitsPerValue} bits per value. */ + int numBytes(int bitsPerValue) { + return bitsPerValue << (BLOCK_SIZE_LOG2 - 3); + } + + private static void decodeSlow(int bitsPerValue, DataInput in, long[] tmp, long[] longs) + throws IOException { + final int numLongs = bitsPerValue << 1; + in.readLongs(tmp, 0, numLongs); + final long mask = MASKS32[bitsPerValue]; + int longsIdx = 0; + int shift = 32 - bitsPerValue; + for (; shift >= 0; shift -= bitsPerValue) { + shiftLongs(tmp, numLongs, longs, longsIdx, shift, mask); + longsIdx += numLongs; + } + final int remainingBitsPerLong = shift + bitsPerValue; + final long mask32RemainingBitsPerLong = MASKS32[remainingBitsPerLong]; + int tmpIdx = 0; + int remainingBits = remainingBitsPerLong; + for (; longsIdx < BLOCK_SIZE / 2; ++longsIdx) { + int b = bitsPerValue - remainingBits; + long l = (tmp[tmpIdx++] & MASKS32[remainingBits]) << b; + while (b >= remainingBitsPerLong) { + b -= remainingBitsPerLong; + l |= (tmp[tmpIdx++] & mask32RemainingBitsPerLong) << b; + } + if (b > 0) { + l |= (tmp[tmpIdx] >>> (remainingBitsPerLong - b)) & MASKS32[b]; + remainingBits = remainingBitsPerLong - b; + } else { + remainingBits = remainingBitsPerLong; + } + longs[longsIdx] = l; + } + } + + /** + * The pattern that this shiftLongs method applies is recognized by the C2 compiler, which + * generates SIMD instructions for it in order to shift multiple longs at once. + */ + private static void shiftLongs(long[] a, int count, long[] b, int bi, int shift, long mask) { + for (int i = 0; i < count; ++i) { + b[bi + i] = (a[i] >>> shift) & mask; + } + } + +""" + +def writeRemainderWithSIMDOptimize(bpv, next_primitive, remaining_bits_per_long, o, num_values, f): + iteration = 1 + num_longs = bpv * num_values / remaining_bits_per_long + while num_longs % 2 == 0 and num_values % 2 == 0: + num_longs /= 2 + num_values /= 2 + iteration *= 2 + + f.write(' shiftLongs(tmp, %d, tmp, 0, 0, MASK%d_%d);\n' % (iteration * num_longs, next_primitive, remaining_bits_per_long)) + f.write(' for (int iter = 0, tmpIdx = 0, longsIdx = %d; iter < %d; ++iter, tmpIdx += %d, longsIdx += %d) {\n' %(o, iteration, num_longs, num_values)) + tmp_idx = 0 + b = bpv + b -= remaining_bits_per_long + f.write(' long l0 = tmp[tmpIdx + %d] << %d;\n' %(tmp_idx, b)) + tmp_idx += 1 + while b >= remaining_bits_per_long: + b -= remaining_bits_per_long + f.write(' l0 |= tmp[tmpIdx + %d] << %d;\n' %(tmp_idx, b)) + tmp_idx += 1 + f.write(' longs[longsIdx + 0] = l0;\n') + f.write(' }\n') + + +def writeRemainder(bpv, next_primitive, remaining_bits_per_long, o, num_values, f): + iteration = 1 + num_longs = bpv * num_values / remaining_bits_per_long + while num_longs % 2 == 0 and num_values % 2 == 0: + num_longs /= 2 + num_values /= 2 + iteration *= 2 + f.write(' for (int iter = 0, tmpIdx = 0, longsIdx = %d; iter < %d; ++iter, tmpIdx += %d, longsIdx += %d) {\n' %(o, iteration, num_longs, 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_long + f.write(' long l%d = (tmp[tmpIdx + %d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits_per_long, b)) + else: + b -= remaining_bits + f.write(' long 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_long: + b -= remaining_bits_per_long + f.write(' l%d |= (tmp[tmpIdx + %d] & MASK%d_%d) << %d;\n' %(i, tmp_idx, next_primitive, remaining_bits_per_long, 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_long-b, next_primitive, b)) + remaining_bits = remaining_bits_per_long-b + f.write(' longs[longsIdx + %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 + f.write(' private static void decode%d(DataInput in, long[] tmp, long[] longs) throws IOException {\n' %bpv) + num_values_per_long = 64 / next_primitive + if bpv == next_primitive: + f.write(' in.readLongs(longs, 0, %d);\n' %(bpv*2)) + else: + f.write(' in.readLongs(tmp, 0, %d);\n' %(bpv*2)) + shift = next_primitive - bpv + o = 0 + while shift >= 0: + f.write(' shiftLongs(tmp, %d, longs, %d, %d, MASK%d_%d);\n' %(bpv*2, o, shift, next_primitive, bpv)) + o += bpv*2 + shift -= bpv + if shift + bpv > 0: + if bpv % (next_primitive % bpv) == 0: + writeRemainderWithSIMDOptimize(bpv, next_primitive, shift + bpv, o, 128/num_values_per_long - o, f) + else: + writeRemainder(bpv, next_primitive, shift + bpv, o, 128/num_values_per_long - o, f) + f.write(' }\n') + + +if __name__ == '__main__': + f = open(OUTPUT_FILE, 'w') + f.write(HEADER) + for primitive_size in PRIMITIVE_SIZE: + f.write(' private static final long[] MASKS%d = new long[%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 longs 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)): + if bpv * 2 != primitive_size or primitive_size == 8: + f.write(' private static final long MASK%d_%d = MASKS%d[%d];\n' %(primitive_size, bpv, primitive_size, bpv)) + + f.write(""" + /** Decode 128 integers into {@code longs}. */ + void decode(int bitsPerValue, DataInput in, long[] longs) 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) + 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, in, tmp, longs);\n') + f.write(' expand32(longs);\n') + f.write(' break;\n') + f.write(' }\n') + f.write(' }\n') + + f.write(""" + /** + * Delta-decode 128 integers into {@code longs}. + */ + void decodeAndPrefixSum(int bitsPerValue, DataInput in, long base, long[] longs) 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) + f.write(' decode%d(in, tmp, longs);\n' %bpv) + f.write(' prefixSum%d(longs, base);\n' %next_primitive) + f.write(' break;\n') + f.write(' default:\n') + f.write(' decodeSlow(bitsPerValue, in, tmp, longs);\n') + f.write(' prefixSum32(longs, base);\n') + f.write(' break;\n') + f.write(' }\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/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/package-info.java new file mode 100644 index 00000000000..6c1b0b44a07 --- /dev/null +++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene99/package-info.java @@ -0,0 +1,428 @@ +/* + * 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. + */ + +/** + * Lucene 9.9 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.backward_codecs.lucene99.Lucene99PostingsFormat Term Dictionary}.timThe term dictionary, stores term info
{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Term Index}.tipThe index into the Term Dictionary
{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Frequencies}.docContains the list of docs which contain each term along with frequency
{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat Positions}.posStores position information about where a term occurs in the index
{@link org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat 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}.dii, .dimHolds 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.

+ */ +package org.apache.lucene.backward_codecs.lucene99; 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 36cb0bbbd24..df14387fc68 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 @@ -22,3 +22,4 @@ org.apache.lucene.backward_codecs.lucene91.Lucene91Codec org.apache.lucene.backward_codecs.lucene92.Lucene92Codec org.apache.lucene.backward_codecs.lucene94.Lucene94Codec org.apache.lucene.backward_codecs.lucene95.Lucene95Codec +org.apache.lucene.backward_codecs.lucene99.Lucene99Codec 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 18ac359d2ae..9733362abe7 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 @@ -16,3 +16,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 diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90PostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90PostingsFormat.java index 7965bc9c778..814285a8895 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90PostingsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene90/TestLucene90PostingsFormat.java @@ -23,12 +23,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; import org.apache.lucene.backward_codecs.lucene90.Lucene90ScoreSkipReader.MutableImpactList; +import org.apache.lucene.backward_codecs.lucene99.Lucene99SkipWriter; 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.lucene99.Lucene99PostingsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99SkipWriter; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.index.DirectoryReader; @@ -77,22 +76,6 @@ public class TestLucene90PostingsFormat extends BasePostingsFormatTestCase { d.close(); } - private void shouldFail(int minItemsInBlock, int maxItemsInBlock) { - expectThrows( - IllegalArgumentException.class, - () -> { - new Lucene99PostingsFormat(minItemsInBlock, maxItemsInBlock); - }); - } - - public void testInvalidBlockSizes() throws Exception { - shouldFail(0, 0); - shouldFail(10, 8); - shouldFail(-1, 10); - shouldFail(10, -1); - shouldFail(10, 12); - } - public void testImpactSerialization() throws IOException { // omit norms and omit freqs doTestImpactSerialization(Collections.singletonList(new Impact(1, 1L))); diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99PostingsWriter.java similarity index 95% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsWriter.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99PostingsWriter.java index 61949cfe227..44ff826e159 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99PostingsWriter.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99PostingsWriter.java @@ -14,22 +14,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; -import static org.apache.lucene.codecs.lucene99.ForUtil.BLOCK_SIZE; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.DOC_CODEC; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.MAX_SKIP_LEVELS; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.PAY_CODEC; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.POS_CODEC; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.TERMS_CODEC; -import static org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.VERSION_CURRENT; +import static org.apache.lucene.backward_codecs.lucene99.ForUtil.BLOCK_SIZE; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.DOC_CODEC; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.MAX_SKIP_LEVELS; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.PAY_CODEC; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.POS_CODEC; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.TERMS_CODEC; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.VERSION_CURRENT; import java.io.IOException; +import org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat.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.lucene99.Lucene99PostingsFormat.IntBlockTermState; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.IndexFileNames; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99RWPostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99RWPostingsFormat.java new file mode 100644 index 00000000000..f513562cb35 --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/Lucene99RWPostingsFormat.java @@ -0,0 +1,68 @@ +/* + * 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.lucene99; + +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; + +public class Lucene99RWPostingsFormat extends Lucene99PostingsFormat { + + private final int minTermBlockSize; + private final int maxTermBlockSize; + + /** Creates {@code Lucene99PostingsFormat} with default settings. */ + public Lucene99RWPostingsFormat() { + this( + Lucene90BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, + Lucene90BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE); + } + + /** + * Creates {@code Lucene99PostingsFormat} with custom values for {@code minBlockSize} and {@code + * maxBlockSize} passed to block terms dictionary. + * + * @see + * Lucene90BlockTreeTermsWriter#Lucene90BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) + */ + public Lucene99RWPostingsFormat(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 Lucene99PostingsWriter(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/lucene99/TestForDeltaUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestForDeltaUtil.java similarity index 98% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestForDeltaUtil.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestForDeltaUtil.java index 29448ff1e41..e21eaab461c 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestForDeltaUtil.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestForDeltaUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestForUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestForUtil.java similarity index 98% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestForUtil.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestForUtil.java index 2f179c743af..85e0dcaebc4 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestForUtil.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/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.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; 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 0e0d63963cc..01d5b4ab17f 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99HnswScalarQuantizedVectorsFormat.java @@ -19,7 +19,6 @@ package org.apache.lucene.backward_codecs.lucene99; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.KnnVectorsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; import org.apache.lucene.tests.index.BaseKnnVectorsFormatTestCase; public class TestLucene99HnswScalarQuantizedVectorsFormat extends BaseKnnVectorsFormatTestCase { diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99PostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99PostingsFormat.java similarity index 90% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99PostingsFormat.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99PostingsFormat.java index 341805e8a3e..e954af88c3b 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99PostingsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestLucene99PostingsFormat.java @@ -14,22 +14,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.backward_codecs.lucene99; -import static org.apache.lucene.codecs.lucene99.Lucene99ScoreSkipReader.readImpacts; +import static org.apache.lucene.backward_codecs.lucene99.Lucene99ScoreSkipReader.readImpacts; import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; +import org.apache.lucene.backward_codecs.lucene99.Lucene99ScoreSkipReader.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.lucene99.Lucene99ScoreSkipReader.MutableImpactList; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; -import org.apache.lucene.index.*; +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.index.TermsEnum; import org.apache.lucene.store.ByteArrayDataInput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; @@ -41,7 +45,7 @@ import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.BytesRef; public class TestLucene99PostingsFormat extends BasePostingsFormatTestCase { - private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene99PostingsFormat()); + private final Codec codec = TestUtil.alwaysPostingsFormat(new Lucene99RWPostingsFormat()); @Override protected Codec getCodec() { @@ -77,7 +81,7 @@ public class TestLucene99PostingsFormat extends BasePostingsFormatTestCase { expectThrows( IllegalArgumentException.class, () -> { - new Lucene99PostingsFormat(minItemsInBlock, maxItemsInBlock); + new Lucene99RWPostingsFormat(minItemsInBlock, maxItemsInBlock); }); } diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestPForUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestPForUtil.java similarity index 98% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestPForUtil.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestPForUtil.java index 16f719e93b2..528d5d2b960 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestPForUtil.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/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.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import java.io.IOException; diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestPostingsUtil.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestPostingsUtil.java similarity index 97% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestPostingsUtil.java rename to lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/TestPostingsUtil.java index b9cb3f20dba..736bdca51be 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestPostingsUtil.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene99/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.lucene99; +package org.apache.lucene.backward_codecs.lucene99; import java.io.IOException; import org.apache.lucene.store.Directory; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt8HnswBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt8HnswBackwardsCompatibility.java index d840a00ca5b..8db406df992 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt8HnswBackwardsCompatibility.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestInt8HnswBackwardsCompatibility.java @@ -20,9 +20,9 @@ import static org.apache.lucene.backward_index.TestBasicBackwardsCompatibility.a import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import java.io.IOException; +import org.apache.lucene.backward_codecs.lucene99.Lucene99Codec; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.KnnVectorsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; import org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; import org.apache.lucene.document.Document; 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 d389372d8f8..db704148573 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 @@ -23,13 +23,13 @@ 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.Lucene90BlockTreeTermsWriter; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +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 Lucene99PostingsWriter}. */ +/** Uses {@link OrdsBlockTreeTermsWriter} with {@link Lucene912PostingsWriter}. */ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { private final int minTermBlockSize; @@ -67,7 +67,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); boolean success = false; try { @@ -84,7 +84,7 @@ public class BlockTreeOrdsPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postingsReader = new Lucene99PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene912PostingsReader(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 89494297c2a..8e17edb9e2c 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java @@ -24,7 +24,7 @@ import java.util.TreeMap; import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; import org.apache.lucene.index.BaseTermsEnum; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.Fields; @@ -54,7 +54,7 @@ import org.apache.lucene.util.automaton.TransitionAccessor; // - or: longer dense skip lists than just next byte? /** - * Wraps {@link Lucene99PostingsFormat} format for on-disk storage, but then at read time loads and + * Wraps {@link Lucene912PostingsFormat} format for on-disk storage, but then at read time loads and * stores all terms and postings directly in RAM as byte[], int[]. * *

WARNING: This is exceptionally RAM intensive: it makes no effort to compress the @@ -97,12 +97,12 @@ public final class DirectPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - return PostingsFormat.forName("Lucene99").fieldsConsumer(state); + return PostingsFormat.forName("Lucene912").fieldsConsumer(state); } @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - FieldsProducer postings = PostingsFormat.forName("Lucene99").fieldsProducer(state); + FieldsProducer postings = PostingsFormat.forName("Lucene912").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 0a18b75ac00..b184f85b176 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java @@ -22,8 +22,8 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +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; @@ -41,7 +41,7 @@ public final class FSTPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); boolean success = false; try { @@ -57,7 +57,7 @@ public final class FSTPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postingsReader = new Lucene99PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene912PostingsReader(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 ae01216ccf6..1f876e5e9d1 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.lucene99.Lucene99PostingsFormat.BLOCK_SIZE; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.BLOCK_SIZE; import java.io.IOException; import org.apache.lucene.codecs.BlockTermState; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat.IntBlockTermState; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +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.index.FieldInfo; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.TermState; @@ -34,7 +34,7 @@ import org.apache.lucene.util.RamUsageEstimator; /** * {@link TermState} serializer which encodes each file pointer as a delta relative to a base file - * pointer. It differs from {@link Lucene99PostingsWriter#encodeTerm} which encodes each file + * pointer. It differs from {@link Lucene912PostingsWriter#encodeTerm} which encodes each file * pointer as a delta relative to the previous file pointer. * *

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

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

Simpler variant of {@link Lucene912PostingsWriter#encodeTerm(DataOutput, FieldInfo, * BlockTermState, boolean)}. */ public void writeTermState( @@ -140,15 +140,12 @@ public class DeltaBaseTermStateSerializer implements Accountable { termStatesOutput.writeVLong(intTermState.lastPosBlockOffset); } } - if (intTermState.skipOffset != -1) { - termStatesOutput.writeVLong(intTermState.skipOffset); - } } /** * Reads a {@link BlockTermState} from the provided {@link DataInput}. * - *

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

Simpler variant of {@link Lucene912PostingsReader#decodeTerm(DataInput, FieldInfo, * BlockTermState, boolean)}. * * @param reuse {@link BlockTermState} to reuse; or null to create a new one. @@ -190,9 +187,6 @@ public class DeltaBaseTermStateSerializer implements Accountable { intTermState.lastPosBlockOffset = termStatesInput.readVLong(); } } - if (intTermState.docFreq > BLOCK_SIZE) { - intTermState.skipOffset = termStatesInput.readVLong(); - } return intTermState; } @@ -210,7 +204,6 @@ public class DeltaBaseTermStateSerializer implements Accountable { termState.docStartFP = 0; termState.posStartFP = 0; termState.payStartFP = 0; - termState.skipOffset = -1; termState.lastPosBlockOffset = -1; termState.singletonDocID = -1; 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 8e2b7bb7d54..c8a19bf9da9 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/UniformSplitPostingsFormat.java @@ -23,8 +23,8 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +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; @@ -113,7 +113,7 @@ public class UniformSplitPostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); boolean success = false; try { FieldsConsumer termsWriter = @@ -130,7 +130,7 @@ public class UniformSplitPostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postingsReader = new Lucene99PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene912PostingsReader(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 a0fa67508f3..dc77bc710a1 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.lucene99.Lucene99PostingsFormat} + * prefer {@link org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat} * */ 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 8ce34b0e566..ab20ee67c8c 100644 --- a/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java +++ b/lucene/codecs/src/test/org/apache/lucene/codecs/bitvectors/TestHnswBitVectorsFormat.java @@ -22,7 +22,7 @@ import java.io.IOException; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.KnnVectorsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.KnnByteVectorField; @@ -42,7 +42,7 @@ import org.apache.lucene.tests.index.BaseIndexFileFormatTestCase; public class TestHnswBitVectorsFormat extends BaseIndexFileFormatTestCase { @Override protected Codec getCodec() { - return new Lucene99Codec() { + return new Lucene912Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return new HnswBitVectorsFormat(); diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/tests/MockTermStateFactory.java b/lucene/codecs/src/test/org/apache/lucene/codecs/lucene90/tests/MockTermStateFactory.java index dde59993f6e..51891b9d0ad 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.lucene99.Lucene99PostingsFormat.IntBlockTermState; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.IntBlockTermState; /** Test utility class to create mock {@link IntBlockTermState}. */ public class MockTermStateFactory { diff --git a/lucene/core/src/generated/checksums/generateForUtil.json b/lucene/core/src/generated/checksums/generateForUtil.json index 527070d7bf0..752285f4d7f 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/lucene99/ForUtil.java": "1292ad354d255b1272ffd3db684aa2ddb2bc49ec", - "lucene/core/src/java/org/apache/lucene/codecs/lucene99/gen_ForUtil.py": "ab7b63a1b73986cc04e43de1c8f474b97aef5116" + "lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java": "5ff856e80cab30f9e5704aa89f3197f017d07624", + "lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py": "3ccf92b3ddbff6340a13e8a55090bfb900dc7be2" } \ 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 9df3adf7aef..a0f0bad01eb 100644 --- a/lucene/core/src/java/module-info.java +++ b/lucene/core/src/java/module-info.java @@ -15,7 +15,7 @@ * limitations under the License. */ -import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; /** Lucene Core. */ @SuppressWarnings("module") // the test framework is compiled after the core... @@ -33,6 +33,7 @@ module org.apache.lucene.core { exports org.apache.lucene.codecs.lucene94; exports org.apache.lucene.codecs.lucene95; exports org.apache.lucene.codecs.lucene99; + exports org.apache.lucene.codecs.lucene912; exports org.apache.lucene.codecs.perfield; exports org.apache.lucene.codecs; exports org.apache.lucene.document; @@ -71,7 +72,7 @@ module org.apache.lucene.core { provides org.apache.lucene.analysis.TokenizerFactory with org.apache.lucene.analysis.standard.StandardTokenizerFactory; provides org.apache.lucene.codecs.Codec with - Lucene99Codec; + Lucene912Codec; provides org.apache.lucene.codecs.DocValuesFormat with org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; provides org.apache.lucene.codecs.KnnVectorsFormat with @@ -79,7 +80,7 @@ module org.apache.lucene.core { org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat, org.apache.lucene.codecs.lucene99.Lucene99ScalarQuantizedVectorsFormat; provides org.apache.lucene.codecs.PostingsFormat with - org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; + org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; 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 0b654a134a6..e5a5dac8ff5 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java @@ -55,7 +55,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI { return LOADER; } - static Codec defaultCodec = LOADER.lookup("Lucene99"); + static Codec defaultCodec = LOADER.lookup("Lucene912"); } private final String name; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/CompetitiveImpactAccumulator.java b/lucene/core/src/java/org/apache/lucene/codecs/CompetitiveImpactAccumulator.java index 37b9fc8bd20..77d38d290f8 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/CompetitiveImpactAccumulator.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/CompetitiveImpactAccumulator.java @@ -18,8 +18,6 @@ package org.apache.lucene.codecs; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.Iterator; import java.util.List; @@ -106,7 +104,7 @@ public final class CompetitiveImpactAccumulator { } /** Get the set of competitive freq and norm pairs, ordered by increasing freq and norm. */ - public Collection getCompetitiveFreqNormPairs() { + public List getCompetitiveFreqNormPairs() { List impacts = new ArrayList<>(); int maxFreqForLowerNorms = 0; for (int i = 0; i < maxFreqs.length; ++i) { @@ -126,7 +124,7 @@ public final class CompetitiveImpactAccumulator { for (Impact impact : impacts) { add(impact, freqNormPairs); } - return Collections.unmodifiableSet(freqNormPairs); + return List.copyOf(freqNormPairs); } private void add(Impact newEntry, TreeSet freqNormPairs) { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90StoredFieldsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90StoredFieldsFormat.java index 5646724546a..ce0310d6396 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90StoredFieldsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90StoredFieldsFormat.java @@ -49,9 +49,9 @@ import org.apache.lucene.util.packed.DirectMonotonicWriter; * *
      *   // the default: for high performance
    - *   indexWriterConfig.setCodec(new Lucene99Codec(Mode.BEST_SPEED));
    + *   indexWriterConfig.setCodec(new Lucene912Codec(Mode.BEST_SPEED));
      *   // instead for higher performance (but slower):
    - *   // indexWriterConfig.setCodec(new Lucene99Codec(Mode.BEST_COMPRESSION));
    + *   // indexWriterConfig.setCodec(new Lucene912Codec(Mode.BEST_COMPRESSION));
      * 
    * *

    File formats diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java new file mode 100644 index 00000000000..8b9aedcfb2b --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForDeltaUtil.java @@ -0,0 +1,83 @@ +/* + * 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.lucene912; + +import java.io.IOException; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.packed.PackedInts; + +/** Utility class to encode/decode increasing sequences of 128 integers. */ +public class ForDeltaUtil { + + // IDENTITY_PLUS_ONE[i] == i+1 + private static final long[] IDENTITY_PLUS_ONE = new long[ForUtil.BLOCK_SIZE]; + + static { + for (int i = 0; i < ForUtil.BLOCK_SIZE; ++i) { + IDENTITY_PLUS_ONE[i] = i + 1; + } + } + + private static void prefixSumOfOnes(long[] arr, long 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 final ForUtil forUtil; + + ForDeltaUtil(ForUtil forUtil) { + this.forUtil = forUtil; + } + + /** + * Encode deltas of a strictly monotonically increasing sequence of integers. The provided {@code + * longs} are expected to be deltas between consecutive values. + */ + void encodeDeltas(long[] longs, DataOutput out) throws IOException { + if (longs[0] == 1 && PForUtil.allEqual(longs)) { // happens with very dense postings + out.writeByte((byte) 0); + } else { + long or = 0; + for (long l : longs) { + or |= l; + } + assert or != 0; + final int bitsPerValue = PackedInts.bitsRequired(or); + out.writeByte((byte) bitsPerValue); + forUtil.encode(longs, bitsPerValue, out); + } + } + + /** Decode deltas, compute the prefix sum and add {@code base} to all decoded longs. */ + void decodeAndPrefixSum(DataInput in, long base, long[] longs) throws IOException { + final int bitsPerValue = Byte.toUnsignedInt(in.readByte()); + if (bitsPerValue == 0) { + prefixSumOfOnes(longs, base); + } else { + forUtil.decodeAndPrefixSum(bitsPerValue, in, base, longs); + } + } + + void skip(DataInput in) throws IOException { + final int bitsPerValue = Byte.toUnsignedInt(in.readByte()); + in.skipBytes(forUtil.numBytes(bitsPerValue)); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForUtil.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java index 3126cdb02c0..63ee7baaf10 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/ForUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/ForUtil.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.codecs.lucene912; import java.io.IOException; import org.apache.lucene.store.DataInput; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912Codec.java new file mode 100644 index 00000000000..cb4ef755a6b --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912Codec.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.lucene912; + +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 9.12 index format + * + *

    If you want to reuse functionality of this codec in another codec, extend {@link FilterCodec}. + * + * @see org.apache.lucene.codecs.lucene99 package documentation for file format details. + * @lucene.experimental + */ +public class Lucene912Codec 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 Lucene912Codec.this.getPostingsFormatForField(field); + } + }; + + private final DocValuesFormat defaultDVFormat; + private final DocValuesFormat docValuesFormat = + new PerFieldDocValuesFormat() { + @Override + public DocValuesFormat getDocValuesFormatForField(String field) { + return Lucene912Codec.this.getDocValuesFormatForField(field); + } + }; + + private final KnnVectorsFormat defaultKnnVectorsFormat; + private final KnnVectorsFormat knnVectorsFormat = + new PerFieldKnnVectorsFormat() { + @Override + public KnnVectorsFormat getKnnVectorsFormatForField(String field) { + return Lucene912Codec.this.getKnnVectorsFormatForField(field); + } + }; + + private final StoredFieldsFormat storedFieldsFormat; + + /** Instantiates a new codec. */ + public Lucene912Codec() { + 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 Lucene912Codec(Mode mode) { + super("Lucene912"); + this.storedFieldsFormat = + new Lucene90StoredFieldsFormat(Objects.requireNonNull(mode).storedMode); + this.defaultPostingsFormat = new Lucene912PostingsFormat(); + 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 "Lucene912". + * + *

    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/lucene912/Lucene912PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsFormat.java new file mode 100644 index 00000000000..1c452175b07 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsFormat.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.lucene912; + +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 9.12 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, Lucene912PostingsFormat 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 Lucene912PostingsFormat 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 = "Lucene912PostingsWriterMeta"; + static final String DOC_CODEC = "Lucene912PostingsWriterDoc"; + static final String POS_CODEC = "Lucene912PostingsWriterPos"; + static final String PAY_CODEC = "Lucene912PostingsWriterPay"; + + 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); + } + } + } + + @Override + public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { + PostingsReaderBase postingsReader = new Lucene912PostingsReader(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 Lucene912PostingsReader} 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/lucene912/Lucene912PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java new file mode 100644 index 00000000000..f0f22f42663 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsReader.java @@ -0,0 +1,2104 @@ +/* + * 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.lucene912; + +import static org.apache.lucene.codecs.lucene912.ForUtil.BLOCK_SIZE; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.DOC_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.LEVEL1_NUM_DOCS; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.META_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.PAY_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.POS_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.TERMS_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.VERSION_CURRENT; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.VERSION_START; + +import 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.lucene912.Lucene912PostingsFormat.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.search.DocIdSetIterator; +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.BytesRefBuilder; +import org.apache.lucene.util.IOUtils; + +/** + * Concrete class that reads docId(maybe frq,pos,offset,payloads) list with postings format. + * + * @lucene.experimental + */ +public final class Lucene912PostingsReader extends PostingsReaderBase { + + 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; + + private final int version; + + /** Sole constructor. */ + public Lucene912PostingsReader(SegmentReadState state) throws IOException { + String metaName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.META_EXTENSION); + final long expectedDocFileLength, expectedPosFileLength, expectedPayFileLength; + ChecksumIndexInput metaIn = null; + boolean success = false; + 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, Lucene912PostingsFormat.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, Lucene912PostingsFormat.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, + Lucene912PostingsFormat.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(long[] buffer, int count, long base) { + buffer[0] += base; + for (int i = 1; i < count; ++i) { + buffer[i] += buffer[i - 1]; + } + } + + static int findFirstGreater(long[] buffer, int target, int from) { + for (int i = from; i < BLOCK_SIZE; ++i) { + if (buffer[i] >= target) { + return i; + } + } + return BLOCK_SIZE; + } + + @Override + public BlockTermState newTermState() { + return new IntBlockTermState(); + } + + @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; + final boolean fieldHasPositions = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + final boolean fieldHasOffsets = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + final boolean fieldHasPayloads = fieldInfo.hasPayloads(); + + 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 (fieldHasPositions) { + termState.posStartFP += in.readVLong(); + if (fieldHasOffsets || fieldHasPayloads) { + 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 { + + boolean indexHasPositions = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + + if (indexHasPositions == false + || PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS) == false) { + BlockDocsEnum docsEnum; + if (reuse instanceof BlockDocsEnum) { + docsEnum = (BlockDocsEnum) reuse; + if (!docsEnum.canReuse(docIn, fieldInfo)) { + docsEnum = new BlockDocsEnum(fieldInfo); + } + } else { + docsEnum = new BlockDocsEnum(fieldInfo); + } + return docsEnum.reset((IntBlockTermState) termState, flags); + } else { + EverythingEnum everythingEnum; + if (reuse instanceof EverythingEnum) { + everythingEnum = (EverythingEnum) reuse; + if (!everythingEnum.canReuse(docIn, fieldInfo)) { + everythingEnum = new EverythingEnum(fieldInfo); + } + } else { + everythingEnum = new EverythingEnum(fieldInfo); + } + return everythingEnum.reset((IntBlockTermState) termState, flags); + } + } + + @Override + public ImpactsEnum impacts(FieldInfo fieldInfo, BlockTermState state, int flags) + throws IOException { + final boolean indexHasFreqs = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + final boolean indexHasPositions = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + + if (state.docFreq >= BLOCK_SIZE + && indexHasFreqs + && (indexHasPositions == false + || PostingsEnum.featureRequested(flags, PostingsEnum.POSITIONS) == false)) { + return new BlockImpactsDocsEnum(fieldInfo, (IntBlockTermState) state); + } + + final boolean indexHasOffsets = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + final boolean indexHasPayloads = fieldInfo.hasPayloads(); + + if (state.docFreq >= BLOCK_SIZE + && indexHasPositions + && (indexHasOffsets == false + || PostingsEnum.featureRequested(flags, PostingsEnum.OFFSETS) == false) + && (indexHasPayloads == false + || PostingsEnum.featureRequested(flags, PostingsEnum.PAYLOADS) == false)) { + return new BlockImpactsPostingsEnum(fieldInfo, (IntBlockTermState) state); + } + + return new SlowImpactsEnum(postings(fieldInfo, state, null, flags)); + } + + final class BlockDocsEnum extends PostingsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE + 1]; + private final long[] freqBuffer = new long[BLOCK_SIZE]; + + private int docBufferUpto; + + final IndexInput startDocIn; + + IndexInput docIn; + final boolean indexHasFreq; + final boolean indexHasPos; + final boolean indexHasOffsetsOrPayloads; + + private int docFreq; // number of docs in this posting list + private long totalTermFreq; // sum of freqBuffer in this posting list (or docFreq when omitted) + private int docCountUpto; // number of docs in or before the current block + private int doc; // doc we last read + private long prevDocID; // last doc ID of the previous block + + // level 0 skip data + private int level0LastDocID; + // level 1 skip data + private int level1LastDocID; + private long level1DocEndOffset; + private int level1DocCountUpto; + + private boolean needsFreq; // true if the caller actually needs frequencies + private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1 + private long freqFP; + + public BlockDocsEnum(FieldInfo fieldInfo) throws IOException { + this.startDocIn = Lucene912PostingsReader.this.docIn; + this.docIn = null; + indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + indexHasPos = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + indexHasOffsetsOrPayloads = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0 + || fieldInfo.hasPayloads(); + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + } + + public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) { + return docIn == startDocIn + && indexHasFreq + == (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0) + && indexHasPos + == (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) + >= 0) + && indexHasOffsetsOrPayloads + == (fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0 + || fieldInfo.hasPayloads()); + } + + public PostingsEnum reset(IntBlockTermState termState, int flags) throws IOException { + docFreq = termState.docFreq; + totalTermFreq = indexHasFreq ? termState.totalTermFreq : docFreq; + singletonDocID = termState.singletonDocID; + if (docIn == null) { + // lazy init + docIn = startDocIn.clone(); + } + prefetchPostings(docIn, termState); + + doc = -1; + 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); + } + prevDocID = -1; + docCountUpto = 0; + level0LastDocID = -1; + level1LastDocID = -1; + level1DocEndOffset = termState.docStartFP; + level1DocCountUpto = 0; + docBufferUpto = BLOCK_SIZE; + freqFP = -1; + return this; + } + + @Override + public int freq() throws IOException { + if (freqFP != -1) { + docIn.seek(freqFP); + pforUtil.decode(docIn, freqBuffer); + } + + return (int) freqBuffer[docBufferUpto - 1]; + } + + @Override + public int nextPosition() throws IOException { + return -1; + } + + @Override + public int startOffset() throws IOException { + return -1; + } + + @Override + public int endOffset() throws IOException { + return -1; + } + + @Override + public BytesRef getPayload() throws IOException { + return null; + } + + @Override + public int docID() { + return doc; + } + + private void refillFullBlock() throws IOException { + assert docFreq - docCountUpto >= BLOCK_SIZE; + + forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); + + if (indexHasFreq) { + if (needsFreq) { + freqFP = docIn.getFilePointer(); + } + pforUtil.skip(docIn); + } + docCountUpto += BLOCK_SIZE; + prevDocID = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[BLOCK_SIZE] == 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] = 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; + freqFP = -1; + } + + private void skipLevel1To(int target) throws IOException { + while (true) { + prevDocID = level1LastDocID; + level0LastDocID = level1LastDocID; + docIn.seek(level1DocEndOffset); + docCountUpto = level1DocCountUpto; + level1DocCountUpto += LEVEL1_NUM_DOCS; + + if (docFreq - docCountUpto < LEVEL1_NUM_DOCS) { + level1LastDocID = DocIdSetIterator.NO_MORE_DOCS; + break; + } + + level1LastDocID += docIn.readVInt(); + level1DocEndOffset = 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 = DocIdSetIterator.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 = (int) docBuffer[BLOCK_SIZE - 1]; + } else { + level0LastDocID = DocIdSetIterator.NO_MORE_DOCS; + refillRemainder(); + } + } + + @Override + public int nextDoc() throws IOException { + if (doc == level0LastDocID) { // advance skip data on level 0 + moveToNextLevel0Block(); + } + + return this.doc = (int) 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 = findFirstGreater(docBuffer, target, docBufferUpto); + this.doc = (int) docBuffer[next]; + docBufferUpto = next + 1; + return doc; + } + + @Override + public long cost() { + return docFreq; + } + } + + final class EverythingEnum extends PostingsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE + 1]; + private final long[] freqBuffer = new long[BLOCK_SIZE + 1]; + private final long[] posDeltaBuffer = new long[BLOCK_SIZE]; + + private final long[] payloadLengthBuffer; + private final long[] offsetStartDeltaBuffer; + private final long[] offsetLengthBuffer; + + private byte[] payloadBytes; + private int payloadByteUpto; + private int payloadLength; + + private int lastStartOffset; + private int startOffset; + private int endOffset; + + private int docBufferUpto; + private int posBufferUpto; + + final IndexInput startDocIn; + + IndexInput docIn; + final IndexInput posIn; + final IndexInput payIn; + final BytesRef payload; + + final boolean indexHasFreq; + final boolean indexHasPos; + final boolean indexHasOffsets; + final boolean indexHasPayloads; + final boolean indexHasOffsetsOrPayloads; + + private int docFreq; // number of docs in this posting list + private long totalTermFreq; // sum of freqBuffer in this posting list (or docFreq when omitted) + private int docCountUpto; // number of docs in or before the current block + private int doc; // doc we last read + private long prevDocID; // last doc ID of the previous block + 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 long posPendingCount; + + // Where this term's postings start in the .pos file: + private long posTermStartFP; + + // Where this term's payloads/offsets start in the .pay + // file: + private long payTermStartFP; + + // 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; + + // level 0 skip data + private int level0LastDocID; + private long level0PosEndFP; + private int level0BlockPosUpto; + private long levelPayEndFP; + private int level0BlockPayUpto; + // level 1 skip data + private int level1LastDocID; + private long level1DocEndFP; + private int level1DocCountUpto; + 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 + + private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1 + + public EverythingEnum(FieldInfo fieldInfo) throws IOException { + this.startDocIn = Lucene912PostingsReader.this.docIn; + this.docIn = null; + indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + indexHasPos = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + indexHasOffsets = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + indexHasPayloads = fieldInfo.hasPayloads(); + indexHasOffsetsOrPayloads = indexHasOffsets || indexHasPayloads; + + this.posIn = Lucene912PostingsReader.this.posIn.clone(); + if (indexHasOffsetsOrPayloads) { + this.payIn = Lucene912PostingsReader.this.payIn.clone(); + } else { + this.payIn = null; + } + if (indexHasOffsets) { + offsetStartDeltaBuffer = new long[BLOCK_SIZE]; + offsetLengthBuffer = new long[BLOCK_SIZE]; + } else { + offsetStartDeltaBuffer = null; + offsetLengthBuffer = null; + startOffset = -1; + endOffset = -1; + } + + if (indexHasPayloads) { + payloadLengthBuffer = new long[BLOCK_SIZE]; + payloadBytes = new byte[128]; + payload = new BytesRef(); + } else { + payloadLengthBuffer = null; + payloadBytes = null; + payload = null; + } + + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + } + + public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) { + return docIn == startDocIn + && 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 { + docFreq = termState.docFreq; + posTermStartFP = termState.posStartFP; + payTermStartFP = termState.payStartFP; + totalTermFreq = termState.totalTermFreq; + singletonDocID = termState.singletonDocID; + if (docFreq > 1 || true) { + if (docIn == null) { + // lazy init + docIn = startDocIn.clone(); + } + prefetchPostings(docIn, termState); + } + posIn.seek(posTermStartFP); + if (indexHasOffsetsOrPayloads) { + payIn.seek(payTermStartFP); + } + level1PosEndFP = posTermStartFP; + level1PayEndFP = payTermStartFP; + level0PosEndFP = posTermStartFP; + levelPayEndFP = 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); + + doc = -1; + prevDocID = -1; + docCountUpto = 0; + level0LastDocID = -1; + level1LastDocID = -1; + level1DocEndFP = termState.docStartFP; + level1DocCountUpto = 0; + level1BlockPosUpto = 0; + level1BlockPayUpto = 0; + level0BlockPosUpto = 0; + level0BlockPayUpto = 0; + docBufferUpto = BLOCK_SIZE; + posBufferUpto = BLOCK_SIZE; + return this; + } + + @Override + public int freq() throws IOException { + return freq; + } + + @Override + public int docID() { + return doc; + } + + private void refillDocs() throws IOException { + final int left = docFreq - docCountUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); + pforUtil.decode(docIn, freqBuffer); + docCountUpto += BLOCK_SIZE; + } else if (docFreq == 1) { + docBuffer[0] = singletonDocID; + freqBuffer[0] = 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; + } + prevDocID = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[BLOCK_SIZE] == 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) { + levelPayEndFP = level1PayEndFP; + level0BlockPayUpto = level1BlockPayUpto; + } + docCountUpto = level1DocCountUpto; + level1DocCountUpto += LEVEL1_NUM_DOCS; + + if (docFreq - docCountUpto < LEVEL1_NUM_DOCS) { + level1LastDocID = DocIdSetIterator.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 levelPayEndFP >= payIn.getFilePointer(); + payIn.seek(levelPayEndFP); + 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) { + levelPayEndFP += docIn.readVLong(); + level0BlockPayUpto = docIn.readVInt(); + } + } else { + level0LastDocID = DocIdSetIterator.NO_MORE_DOCS; + } + + refillDocs(); + } + + @Override + public int nextDoc() throws IOException { + if (doc == level0LastDocID) { // advance level 0 skip data + moveToNextLevel0Block(); + } + + this.doc = (int) docBuffer[docBufferUpto]; + this.freq = (int) 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 levelPayEndFP >= payIn.getFilePointer(); + payIn.seek(levelPayEndFP); + payloadByteUpto = level0BlockPayUpto; + } + posBufferUpto = BLOCK_SIZE; + } else { + for (int i = docBufferUpto; i < BLOCK_SIZE; ++i) { + posPendingCount += freqBuffer[i]; + } + } + + 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) { + levelPayEndFP += docIn.readVLong(); + level0BlockPayUpto = docIn.readVInt(); + } + + if (target <= level0LastDocID) { + break; + } + + docIn.seek(blockEndFP); + docCountUpto += BLOCK_SIZE; + } else { + level0LastDocID = DocIdSetIterator.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 = findFirstGreater(docBuffer, target, docBufferUpto); + for (int i = docBufferUpto; i <= next; ++i) { + posPendingCount += freqBuffer[i]; + } + this.freq = (int) freqBuffer[next]; + this.docBufferUpto = next + 1; + position = 0; + lastStartOffset = 0; + + return this.doc = (int) docBuffer[next]; + } + + private void skipPositions() throws IOException { + // Skip positions now: + long toSkip = posPendingCount - freq; + // if (DEBUG) { + // System.out.println(" FPR.skipPositions: toSkip=" + toSkip); + // } + + final int leftInBlock = BLOCK_SIZE - posBufferUpto; + if (toSkip < leftInBlock) { + int end = (int) (posBufferUpto + toSkip); + if (indexHasPayloads) { + for (int i = posBufferUpto; i < end; ++i) { + payloadByteUpto += payloadLengthBuffer[i]; + } + } + 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; + posBufferUpto = 0; + final int toSkipInt = (int) toSkip; + if (indexHasPayloads) { + for (int i = 0; i < toSkipInt; ++i) { + payloadByteUpto += payloadLengthBuffer[i]; + } + } + posBufferUpto = toSkipInt; + } + + 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(posIn, posDeltaBuffer); + + if (indexHasPayloads) { + if (needsPayloads) { + pforUtil.decode(payIn, 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(payIn, offsetStartDeltaBuffer); + pforUtil.decode(payIn, 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 = (int) payloadLengthBuffer[posBufferUpto]; + payload.bytes = payloadBytes; + payload.offset = payloadByteUpto; + payload.length = payloadLength; + payloadByteUpto += payloadLength; + } + + if (indexHasOffsets) { + startOffset = lastStartOffset + (int) offsetStartDeltaBuffer[posBufferUpto]; + endOffset = startOffset + (int) 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; + } + } + + final class BlockImpactsDocsEnum extends ImpactsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE + 1]; + private final long[] freqBuffer = new long[BLOCK_SIZE]; + + private int docBufferUpto; + + final IndexInput startDocIn; + + IndexInput docIn; + final boolean indexHasFreq; + final boolean indexHasPos; + final boolean indexHasOffsetsOrPayloads; + + private int docFreq; // number of docs in this posting list + private int docCountUpto; // number of docs in or before the current block + private int doc; // doc we last read + private long prevDocID; // last doc ID of the previous block + private long freqFP; + + // true if we shallow-advanced to a new block that we have not decoded yet + private boolean needsRefilling; + + // level 0 skip data + private int level0LastDocID; + private long level0DocEndFP; + private final BytesRef level0SerializedImpacts; + private final ByteArrayDataInput level0SerializedImpactsIn = new ByteArrayDataInput(); + private final MutableImpactList level0Impacts; + // level 1 skip data + private int level1LastDocID; + private long level1DocEndFP; + private int level1DocCountUpto; + private final BytesRef level1SerializedImpacts; + private final ByteArrayDataInput level1SerializedImpactsIn = new ByteArrayDataInput(); + private final MutableImpactList level1Impacts; + + public BlockImpactsDocsEnum(FieldInfo fieldInfo, IntBlockTermState termState) + throws IOException { + this.startDocIn = Lucene912PostingsReader.this.docIn; + this.docIn = null; + indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + indexHasPos = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + indexHasOffsetsOrPayloads = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0 + || fieldInfo.hasPayloads(); + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + + docFreq = termState.docFreq; + if (docFreq > 1 || true) { + if (docIn == null) { + // lazy init + docIn = startDocIn.clone(); + } + prefetchPostings(docIn, termState); + } + + doc = -1; + if (indexHasFreq == 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); + } + prevDocID = -1; + docCountUpto = 0; + level0LastDocID = -1; + level1LastDocID = -1; + level1DocEndFP = termState.docStartFP; + level1DocCountUpto = 0; + docBufferUpto = BLOCK_SIZE; + freqFP = -1; + level0SerializedImpacts = new BytesRef(maxImpactNumBytesAtLevel0); + level1SerializedImpacts = new BytesRef(maxImpactNumBytesAtLevel1); + level0Impacts = new MutableImpactList(maxNumImpactsAtLevel0); + level1Impacts = new MutableImpactList(maxNumImpactsAtLevel1); + } + + @Override + public int freq() throws IOException { + if (freqFP != -1) { + docIn.seek(freqFP); + pforUtil.decode(docIn, freqBuffer); + freqFP = -1; + } + return (int) freqBuffer[docBufferUpto - 1]; + } + + @Override + public int nextPosition() throws IOException { + return -1; + } + + @Override + public int startOffset() throws IOException { + return -1; + } + + @Override + public int endOffset() throws IOException { + return -1; + } + + @Override + public BytesRef getPayload() throws IOException { + return null; + } + + @Override + public int docID() { + return doc; + } + + private void refillDocs() throws IOException { + final int left = docFreq - docCountUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); + + if (indexHasFreq) { + freqFP = docIn.getFilePointer(); + pforUtil.skip(docIn); + } + docCountUpto += BLOCK_SIZE; + } else { + // Read vInts: + PostingsUtil.readVIntBlock(docIn, docBuffer, freqBuffer, left, indexHasFreq, true); + prefixSum(docBuffer, left, prevDocID); + docBuffer[left] = NO_MORE_DOCS; + freqFP = -1; + docCountUpto += left; + } + prevDocID = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[BLOCK_SIZE] == 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 = DocIdSetIterator.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 = DocIdSetIterator.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 = DocIdSetIterator.NO_MORE_DOCS; + } + + refillDocs(); + needsRefilling = false; + } + + @Override + public int nextDoc() throws IOException { + if (doc == level0LastDocID) { + moveToNextLevel0Block(); + } else if (needsRefilling) { + refillDocs(); + needsRefilling = false; + } + + return this.doc = (int) docBuffer[docBufferUpto++]; + } + + @Override + public int advance(int target) throws IOException { + if (target > level0LastDocID || needsRefilling) { + advanceShallow(target); + refillDocs(); + needsRefilling = false; + } + + int next = findFirstGreater(docBuffer, target, docBufferUpto); + this.doc = (int) docBuffer[next]; + docBufferUpto = next + 1; + return doc; + } + + @Override + public Impacts getImpacts() throws IOException { + return new Impacts() { + + @Override + public int numLevels() { + int numLevels = 0; + if (level0LastDocID != NO_MORE_DOCS) { + numLevels++; + } + if (level1LastDocID != NO_MORE_DOCS) { + numLevels++; + } + if (numLevels == 0) { + numLevels++; + } + return numLevels; + } + + @Override + public int getDocIdUpTo(int level) { + if (level0LastDocID != NO_MORE_DOCS) { + if (level == 0) { + return level0LastDocID; + } + level--; + } + + if (level1LastDocID != NO_MORE_DOCS) { + if (level == 0) { + return level1LastDocID; + } + level--; + } + + return NO_MORE_DOCS; + } + + @Override + public List getImpacts(int level) { + if (level0LastDocID != NO_MORE_DOCS) { + if (level == 0) { + level0SerializedImpactsIn.reset( + level0SerializedImpacts.bytes, 0, level0SerializedImpacts.length); + readImpacts(level0SerializedImpactsIn, level0Impacts); + return level0Impacts; + } + level--; + } + + if (level1LastDocID != NO_MORE_DOCS) { + if (level == 0) { + level1SerializedImpactsIn.reset( + level1SerializedImpacts.bytes, 0, level1SerializedImpacts.length); + readImpacts(level1SerializedImpactsIn, level1Impacts); + return level1Impacts; + } + level--; + } + + return Collections.singletonList(new Impact(Integer.MAX_VALUE, 1L)); + } + }; + } + + @Override + public long cost() { + return docFreq; + } + } + + final class BlockImpactsPostingsEnum extends ImpactsEnum { + + final ForUtil forUtil = new ForUtil(); + final ForDeltaUtil forDeltaUtil = new ForDeltaUtil(forUtil); + final PForUtil pforUtil = new PForUtil(forUtil); + + private final long[] docBuffer = new long[BLOCK_SIZE + 1]; + private final long[] freqBuffer = new long[BLOCK_SIZE]; + private final long[] posDeltaBuffer = new long[BLOCK_SIZE]; + + private int docBufferUpto; + private int posBufferUpto; + + final IndexInput startDocIn; + + IndexInput docIn; + final IndexInput posIn; + + final boolean indexHasFreq; + final boolean indexHasPos; + final boolean indexHasOffsets; + final boolean indexHasPayloads; + final boolean indexHasOffsetsOrPayloads; + + private int docFreq; // number of docs in this posting list + private long totalTermFreq; // sum of freqBuffer in this posting list (or docFreq when omitted) + private int docCountUpto; // number of docs in or before the current block + private int doc; // doc we last read + private long prevDocID; // last doc ID of the previous block + 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 long posPendingCount; + + // Where this term's postings start in the .pos file: + private long posTermStartFP; + + // 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; + + // true if we shallow-advanced to a new block that we have not decoded yet + private boolean needsRefilling; + + // level 0 skip data + private int level0LastDocID; + private long level0DocEndFP; + private long level0PosEndFP; + private int level0BlockPosUpto; + private final BytesRefBuilder level0SerializedImpacts = new BytesRefBuilder(); + private final ByteArrayDataInput level0SerializedImpactsIn = new ByteArrayDataInput(); + private final MutableImpactList level0Impacts; + // level 1 skip data + private int level1LastDocID; + private long level1DocEndFP; + private int level1DocCountUpto; + private long level1PosEndFP; + private int level1BlockPosUpto; + private final BytesRefBuilder level1SerializedImpacts = new BytesRefBuilder(); + private final ByteArrayDataInput level1SerializedImpactsIn = new ByteArrayDataInput(); + private final MutableImpactList level1Impacts; + + private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1 + + public BlockImpactsPostingsEnum(FieldInfo fieldInfo, IntBlockTermState termState) + throws IOException { + this.startDocIn = Lucene912PostingsReader.this.docIn; + this.docIn = null; + indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0; + indexHasPos = + fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + indexHasOffsets = + fieldInfo + .getIndexOptions() + .compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) + >= 0; + indexHasPayloads = fieldInfo.hasPayloads(); + indexHasOffsetsOrPayloads = indexHasOffsets || indexHasPayloads; + + this.posIn = Lucene912PostingsReader.this.posIn.clone(); + + // We set the last element of docBuffer to NO_MORE_DOCS, it helps save conditionals in + // advance() + docBuffer[BLOCK_SIZE] = NO_MORE_DOCS; + + docFreq = termState.docFreq; + posTermStartFP = termState.posStartFP; + totalTermFreq = termState.totalTermFreq; + singletonDocID = termState.singletonDocID; + if (docFreq > 1 || true) { + if (docIn == null) { + // lazy init + docIn = startDocIn.clone(); + } + prefetchPostings(docIn, termState); + } + 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; + } + + doc = -1; + prevDocID = -1; + docCountUpto = 0; + level0LastDocID = -1; + level1LastDocID = -1; + level1DocEndFP = termState.docStartFP; + level1DocCountUpto = 0; + level1BlockPosUpto = 0; + docBufferUpto = BLOCK_SIZE; + posBufferUpto = BLOCK_SIZE; + level0SerializedImpacts.growNoCopy(maxImpactNumBytesAtLevel0); + level1SerializedImpacts.growNoCopy(maxImpactNumBytesAtLevel1); + level0Impacts = new MutableImpactList(maxNumImpactsAtLevel0); + level1Impacts = new MutableImpactList(maxNumImpactsAtLevel1); + } + + @Override + public int freq() throws IOException { + return freq; + } + + @Override + public int docID() { + return doc; + } + + private void refillDocs() throws IOException { + final int left = docFreq - docCountUpto; + assert left >= 0; + + if (left >= BLOCK_SIZE) { + forDeltaUtil.decodeAndPrefixSum(docIn, prevDocID, docBuffer); + pforUtil.decode(docIn, freqBuffer); + docCountUpto += BLOCK_SIZE; + } else if (docFreq == 1) { + docBuffer[0] = singletonDocID; + freqBuffer[0] = 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; + } + prevDocID = docBuffer[BLOCK_SIZE - 1]; + docBufferUpto = 0; + assert docBuffer[BLOCK_SIZE] == 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 = DocIdSetIterator.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.setLength(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 { + for (int i = docBufferUpto; i < BLOCK_SIZE; ++i) { + posPendingCount += freqBuffer[i]; + } + } + + 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.setLength(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 = DocIdSetIterator.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 Impacts getImpacts() throws IOException { + return new Impacts() { + + @Override + public int numLevels() { + int numLevels = 0; + if (level0LastDocID != NO_MORE_DOCS) { + numLevels++; + } + if (level1LastDocID != NO_MORE_DOCS) { + numLevels++; + } + if (numLevels == 0) { + numLevels++; + } + return numLevels; + } + + @Override + public int getDocIdUpTo(int level) { + if (level0LastDocID != NO_MORE_DOCS) { + if (level == 0) { + return level0LastDocID; + } + level--; + } + + if (level1LastDocID != NO_MORE_DOCS) { + if (level == 0) { + return level1LastDocID; + } + level--; + } + + return NO_MORE_DOCS; + } + + @Override + public List getImpacts(int level) { + if (level0LastDocID != NO_MORE_DOCS) { + if (level == 0) { + level0SerializedImpactsIn.reset( + level0SerializedImpacts.bytes(), 0, level0SerializedImpacts.length()); + readImpacts(level0SerializedImpactsIn, level0Impacts); + return level0Impacts; + } + level--; + } + + if (level1LastDocID != NO_MORE_DOCS) { + if (level == 0) { + level1SerializedImpactsIn.reset( + level1SerializedImpacts.bytes(), 0, level1SerializedImpacts.length()); + readImpacts(level1SerializedImpactsIn, level1Impacts); + return level1Impacts; + } + level--; + } + + return Collections.singletonList(new Impact(Integer.MAX_VALUE, 1L)); + } + }; + } + + @Override + public int nextDoc() throws IOException { + advanceShallow(doc + 1); + if (needsRefilling) { + refillDocs(); + needsRefilling = false; + } + + doc = (int) docBuffer[docBufferUpto]; + freq = (int) 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 = findFirstGreater(docBuffer, target, docBufferUpto); + for (int i = docBufferUpto; i <= next; ++i) { + posPendingCount += freqBuffer[i]; + } + freq = (int) freqBuffer[next]; + docBufferUpto = next + 1; + position = 0; + return this.doc = (int) docBuffer[next]; + } + + private void skipPositions() throws IOException { + // Skip positions now: + long 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 = (int) 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(posIn, 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; + } + + @Override + public int startOffset() { + return -1; + } + + @Override + public int endOffset() { + return -1; + } + + @Override + public BytesRef getPayload() { + return null; + } + + @Override + public long cost() { + return docFreq; + } + } + + /** + * @see Lucene912PostingsWriter#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 Lucene912PostingsWriter#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 void prefetchPostings(IndexInput docIn, IntBlockTermState state) throws IOException { + if (state.docFreq > 1 && 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/lucene912/Lucene912PostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.java new file mode 100644 index 00000000000..b307080b215 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/Lucene912PostingsWriter.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.lucene912; + +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.*; +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.PAY_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.TERMS_CODEC; +import static org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat.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.lucene912.Lucene912PostingsFormat.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 Lucene912PostingsFormat}. */ +public class Lucene912PostingsWriter 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 long[] docDeltaBuffer; + final long[] freqBuffer; + private int docBufferUpto; + + final long[] posDeltaBuffer; + final long[] payloadLengthBuffer; + final long[] offsetStartDeltaBuffer; + final long[] 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 Lucene912PostingsWriter(SegmentWriteState state) throws IOException { + String metaFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.META_EXTENSION); + String docFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.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); + final ForUtil forUtil = new ForUtil(); + forDeltaUtil = new ForDeltaUtil(forUtil); + pforUtil = new PForUtil(forUtil); + if (state.fieldInfos.hasProx()) { + posDeltaBuffer = new long[BLOCK_SIZE]; + String posFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, state.segmentSuffix, Lucene912PostingsFormat.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 long[BLOCK_SIZE]; + } else { + payloadBytes = null; + payloadLengthBuffer = null; + } + + if (state.fieldInfos.hasOffsets()) { + offsetStartDeltaBuffer = new long[BLOCK_SIZE]; + offsetLengthBuffer = new long[BLOCK_SIZE]; + } else { + offsetStartDeltaBuffer = null; + offsetLengthBuffer = null; + } + + if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) { + String payFileName = + IndexFileNames.segmentFileName( + state.segmentInfo.name, + state.segmentSuffix, + Lucene912PostingsFormat.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 long[BLOCK_SIZE]; + freqBuffer = new long[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() throws IOException { + 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); + long numImpactBytes = scratchOutput.size(); + final long level1End; + if (writeFreqs) { + List impacts = level1CompetitiveFreqNormAccumulator.getCompetitiveFreqNormPairs(); + if (impacts.size() > maxNumImpactsAtLevel1) { + maxNumImpactsAtLevel1 = impacts.size(); + } + writeImpacts(impacts, scratchOutput); + 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 = (int) 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 = (int) posDeltaBuffer[i]; + if (writePayloads) { + final int payloadLength = (int) 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 = (int) offsetStartDeltaBuffer[i]; + int length = (int) 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/lucene99/PForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/PForUtil.java rename to lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java index de62fb6b40b..f4405ae66fa 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/PForUtil.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PForUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.codecs.lucene912; import java.io.IOException; import java.util.Arrays; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingsUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingsUtil.java new file mode 100644 index 00000000000..4834dd73e22 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/PostingsUtil.java @@ -0,0 +1,73 @@ +/* + * 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.lucene912; + +import java.io.IOException; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexInput; + +/** 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, + long[] docBuffer, + long[] freqBuffer, + int num, + boolean indexHasFreq, + boolean decodeFreq) + throws IOException { + docIn.readGroupVInts(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, long[] docBuffer, long[] 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 = (int) freqBuffer[i]; + if (freq != 1) { + docOut.writeVInt(freq); + } + } + } + } +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/gen_ForUtil.py b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py similarity index 99% rename from lucene/core/src/java/org/apache/lucene/codecs/lucene99/gen_ForUtil.py rename to lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py index cd81d493570..c6a33ceef53 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/gen_ForUtil.py +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/gen_ForUtil.py @@ -40,7 +40,7 @@ HEADER = """// This file has been automatically generated, DO NOT EDIT * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.codecs.lucene99; +package org.apache.lucene.codecs.lucene912; import java.io.IOException; import org.apache.lucene.store.DataInput; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene912/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/package-info.java new file mode 100644 index 00000000000..853f86a855a --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene912/package-info.java @@ -0,0 +1,431 @@ +/* + * 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. + */ + +/** + * Lucene 9.12 file format. + * + *

    Apache Lucene - Index File Formats

    + * + * + * + *

    Introduction

    + * + *
    + * + *

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

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

    + * + *

    Definitions

    + * + *
    + * + *

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

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

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

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

    Inverted Indexing

    + * + *

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

    Types of Fields

    + * + *

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

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

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

    Segments

    + * + *

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

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

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

    Document Numbers

    + * + *

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

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

      + *
    • + *

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

    • + *

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

    + * + *
    + * + *

    Index Structure Overview

    + * + *
    + * + *

    Each segment index maintains the following: + * + *

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

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

    + * + *

    File Naming

    + * + *
    + * + *

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

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

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

    + * + *

    Summary of File Extensions

    + * + *
    + * + *

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

    + * + *

    Lock File

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

    History

    + * + *

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

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

    Limitations

    + * + *
    + * + *

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

    + */ +package org.apache.lucene.codecs.lucene912; diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/package-info.java index 2df2cfc9a30..98a029853e7 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/package-info.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/package-info.java @@ -15,414 +15,5 @@ * limitations under the License. */ -/** - * Lucene 9.9 file format. - * - *

    Apache Lucene - Index File Formats

    - * - * - * - *

    Introduction

    - * - *
    - * - *

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

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

    - * - *

    Definitions

    - * - *
    - * - *

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

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

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

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

    Inverted Indexing

    - * - *

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

    Types of Fields

    - * - *

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

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

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

    Segments

    - * - *

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

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

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

    Document Numbers

    - * - *

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

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

      - *
    • - *

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

    • - *

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

    - * - *
    - * - *

    Index Structure Overview

    - * - *
    - * - *

    Each segment index maintains the following: - * - *

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

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

    - * - *

    File Naming

    - * - *
    - * - *

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

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

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

    - * - *

    Summary of File Extensions

    - * - *
    - * - *

    The following table summarizes the names and extensions of the files in Lucene: - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - *
    lucene filenames by extension
    NameExtensionBrief Description
    {@link org.apache.lucene.index.SegmentInfos Segments File}segments_NStores information about a commit point
    Lock Filewrite.lockThe Write lock prevents multiple IndexWriters from writing to the same - * file.
    {@link org.apache.lucene.codecs.lucene99.Lucene99SegmentInfoFormat Segment Info}.siStores metadata about a segment
    {@link org.apache.lucene.codecs.lucene90.Lucene90CompoundFormat Compound File}.cfs, .cfeAn optional "virtual" file consisting of all the other index files for - * systems that frequently run out of file handles.
    {@link org.apache.lucene.codecs.lucene94.Lucene94FieldInfosFormat Fields}.fnmStores information about the fields
    {@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Index}.fdxContains pointers to field data
    {@link org.apache.lucene.codecs.lucene90.Lucene90StoredFieldsFormat Field Data}.fdtThe stored fields for documents
    {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Term Dictionary}.timThe term dictionary, stores term info
    {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Term Index}.tipThe index into the Term Dictionary
    {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Frequencies}.docContains the list of docs which contain each term along with frequency
    {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat Positions}.posStores position information about where a term occurs in the index
    {@link org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat 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}.dii, .dimHolds indexed points
    {@link org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat Vector values}.vec, .vem, .veq, vexHolds indexed vectors; .vec files contain the raw vector data, - * .vem the vector metadata, .veq the quantized vector data, and .vex the - * hnsw graph data.
    - * - *

    - * - *

    Lock File

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

    History

    - * - *

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

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

    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.9 file format. */ package org.apache.lucene.codecs.lucene99; 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 55d0e228d1a..48ac987bee3 100644 --- a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java @@ -21,8 +21,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Objects; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; import org.apache.lucene.index.ImpactsEnum; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; @@ -399,10 +399,10 @@ public class PhraseQuery extends Query { /** * A guess of the average number of simple operations for the initial seek and buffer refill per * document for the positions of a term. See also {@link - * Lucene99PostingsReader.BlockImpactsPostingsEnum#nextPosition()}. + * Lucene912PostingsReader.BlockImpactsPostingsEnum#nextPosition()}. * *

    Aside: Instead of being constant this could depend among others on {@link - * Lucene99PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link + * Lucene912PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link * TermsEnum#totalTermFreq()}, {@link DocIdSetIterator#cost()} (expected number of matching docs), * {@link LeafReader#maxDoc()} (total number of docs in the segment), and the seek time and block * size of the device storing the index. @@ -411,7 +411,7 @@ public class PhraseQuery extends Query { /** * Number of simple operations in {@link - * Lucene99PostingsReader.BlockImpactsPostingsEnum#nextPosition()} when no seek or buffer refill + * Lucene912PostingsReader.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/resources/META-INF/services/org.apache.lucene.codecs.Codec b/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec index 816097b3a66..8b672496601 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.lucene99.Lucene99Codec +org.apache.lucene.codecs.lucene912.Lucene912Codec 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 0fd87daff7d..cd63926a287 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.lucene99.Lucene99PostingsFormat +org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat 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 8ca7fb8a41d..c72bcfeea46 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90StoredFieldsFormatHighCompression.java @@ -18,7 +18,7 @@ package org.apache.lucene.codecs.lucene90; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; import org.apache.lucene.document.Document; import org.apache.lucene.document.StoredField; import org.apache.lucene.index.DirectoryReader; @@ -31,7 +31,7 @@ import org.apache.lucene.tests.index.BaseStoredFieldsFormatTestCase; public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase { @Override protected Codec getCodec() { - return new Lucene99Codec(Lucene99Codec.Mode.BEST_COMPRESSION); + return new Lucene912Codec(Lucene912Codec.Mode.BEST_COMPRESSION); } /** @@ -42,7 +42,7 @@ public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFie for (int i = 0; i < 10; i++) { IndexWriterConfig iwc = newIndexWriterConfig(); iwc.setCodec( - new Lucene99Codec(RandomPicks.randomFrom(random(), Lucene99Codec.Mode.values()))); + new Lucene912Codec(RandomPicks.randomFrom(random(), Lucene912Codec.Mode.values()))); IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig()); Document doc = new Document(); doc.add(new StoredField("field1", "value1")); @@ -72,7 +72,7 @@ public class TestLucene90StoredFieldsFormatHighCompression extends BaseStoredFie expectThrows( NullPointerException.class, () -> { - new Lucene99Codec(null); + new Lucene912Codec(null); }); expectThrows( diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestLucene912PostingsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestLucene912PostingsFormat.java new file mode 100644 index 00000000000..1b8d0618c60 --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene912/TestLucene912PostingsFormat.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.lucene912; + +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.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; +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 TestLucene912PostingsFormat extends BasePostingsFormatTestCase { + + @Override + protected Codec getCodec() { + return TestUtil.alwaysPostingsFormat(new Lucene912PostingsFormat()); + } + + 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); + Lucene912PostingsWriter.writeVInt15(out, i); + in.reset(bytes, 0, out.getPosition()); + assertEquals(i, Lucene912PostingsReader.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); + Lucene912PostingsWriter.writeVLong15(out, i); + in.reset(bytes, 0, out.getPosition()); + assertEquals(i, Lucene912PostingsReader.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)) { + Lucene912PostingsWriter.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 = + Lucene912PostingsReader.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/lucene99/TestLucene99HnswQuantizedVectorsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99HnswQuantizedVectorsFormat.java index e26284eb70b..8e69e833b98 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 @@ -29,6 +29,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.KnnFloatVectorField; @@ -79,7 +80,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat @Override protected Codec getCodec() { - return new Lucene99Codec() { + return new Lucene912Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return format; @@ -95,7 +96,7 @@ public class TestLucene99HnswQuantizedVectorsFormat extends BaseKnnVectorsFormat dir, newIndexWriterConfig() .setCodec( - new Lucene99Codec() { + new Lucene912Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return new Lucene99HnswScalarQuantizedVectorsFormat( diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java index 4d5073ef1e9..58e6c27e326 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java +++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorScorer.java @@ -27,6 +27,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; import org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; @@ -51,7 +52,7 @@ import org.apache.lucene.util.quantization.ScalarQuantizer; public class TestLucene99ScalarQuantizedVectorScorer extends LuceneTestCase { private static Codec getCodec(int bits, boolean compress) { - return new Lucene99Codec() { + return new Lucene912Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return new Lucene99HnswScalarQuantizedVectorsFormat( diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene99/TestLucene99ScalarQuantizedVectorsFormat.java index 107c39eed07..b221cb19dde 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,6 +28,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FilterCodec; import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.KnnVectorsReader; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; import org.apache.lucene.document.Document; import org.apache.lucene.document.KnnFloatVectorField; @@ -67,7 +68,7 @@ public class TestLucene99ScalarQuantizedVectorsFormat extends BaseKnnVectorsForm @Override protected Codec getCodec() { - return new Lucene99Codec() { + return new Lucene912Codec() { @Override public KnnVectorsFormat getKnnVectorsFormatForField(String field) { return format; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java b/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java deleted file mode 100644 index 310d8524be9..00000000000 --- a/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java +++ /dev/null @@ -1,195 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.lucene.index; - -import java.io.IOException; -import org.apache.lucene.analysis.Analyzer; -import org.apache.lucene.document.Document; -import org.apache.lucene.document.Field; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.PhraseQuery; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.store.ByteBuffersDirectory; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.FilterIndexInput; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.tests.analysis.MockAnalyzer; -import org.apache.lucene.tests.analysis.MockTokenizer; -import org.apache.lucene.tests.store.MockDirectoryWrapper; -import org.apache.lucene.tests.util.LuceneTestCase; -import org.apache.lucene.tests.util.TestUtil; -import org.apache.lucene.util.BytesRef; - -/** Tests lazy skipping on the proximity file. */ -public class TestLazyProxSkipping extends LuceneTestCase { - private IndexSearcher searcher; - private int seeksCounter = 0; - - private String field = "tokens"; - private String term1 = "xx"; - private String term2 = "yy"; - private String term3 = "zz"; - - private class SeekCountingDirectory extends MockDirectoryWrapper { - public SeekCountingDirectory(Directory delegate) { - super(random(), delegate); - } - - @Override - public IndexInput openInput(String name, IOContext context) throws IOException { - IndexInput ii = super.openInput(name, context); - if (name.endsWith(".prx") || name.endsWith(".pos")) { - // we decorate the proxStream with a wrapper class that allows to count the number of calls - // of seek() - ii = new SeeksCountingStream(ii); - } - return ii; - } - } - - private void createIndex(int numHits) throws IOException { - int numDocs = 500; - - final Analyzer analyzer = - new Analyzer() { - @Override - public TokenStreamComponents createComponents(String fieldName) { - return new TokenStreamComponents(new MockTokenizer(MockTokenizer.WHITESPACE, true)); - } - }; - Directory directory = new SeekCountingDirectory(new ByteBuffersDirectory()); - // note: test explicitly disables payloads - IndexWriter writer = - new IndexWriter( - directory, - newIndexWriterConfig(analyzer) - .setMaxBufferedDocs(10) - .setMergePolicy(newLogMergePolicy(false))); - - for (int i = 0; i < numDocs; i++) { - Document doc = new Document(); - String content; - if (i % (numDocs / numHits) == 0) { - // add a document that matches the query "term1 term2" - content = this.term1 + " " + this.term2; - } else if (i % 15 == 0) { - // add a document that only contains term1 - content = this.term1 + " " + this.term1; - } else { - // add a document that contains term2 but not term 1 - content = this.term3 + " " + this.term2; - } - - doc.add(newTextField(this.field, content, Field.Store.YES)); - writer.addDocument(doc); - } - - // make sure the index has only a single segment - writer.forceMerge(1); - writer.close(); - - LeafReader reader = getOnlyLeafReader(DirectoryReader.open(directory)); - - this.searcher = newSearcher(reader); - } - - private ScoreDoc[] search() throws IOException { - // create PhraseQuery "term1 term2" and search - PhraseQuery pq = new PhraseQuery(field, term1, term2); - return this.searcher.search(pq, 1000).scoreDocs; - } - - private void performTest(int numHits) throws IOException { - createIndex(numHits); - this.seeksCounter = 0; - ScoreDoc[] hits = search(); - // verify that the right number of docs was found - assertEquals(numHits, hits.length); - - // check if the number of calls of seek() does not exceed the number of hits - assertTrue(this.seeksCounter > 0); - assertTrue( - "seeksCounter=" + this.seeksCounter + " numHits=" + numHits, - this.seeksCounter <= numHits + 1); - searcher.getIndexReader().close(); - } - - public void testLazySkipping() throws IOException { - final String fieldFormat = TestUtil.getPostingsFormat(this.field); - assumeFalse("This test cannot run with Direct postings format", fieldFormat.equals("Direct")); - assumeFalse( - "This test cannot run with SimpleText postings format", fieldFormat.equals("SimpleText")); - - // test whether only the minimum amount of seeks() - // are performed - performTest(5); - performTest(10); - } - - public void testSeek() throws IOException { - Directory directory = newDirectory(); - IndexWriter writer = - new IndexWriter(directory, newIndexWriterConfig(new MockAnalyzer(random()))); - for (int i = 0; i < 10; i++) { - Document doc = new Document(); - doc.add(newTextField(this.field, "a b", Field.Store.YES)); - writer.addDocument(doc); - } - - writer.close(); - IndexReader reader = DirectoryReader.open(directory); - - PostingsEnum tp = MultiTerms.getTermPostingsEnum(reader, this.field, new BytesRef("b")); - - for (int i = 0; i < 10; i++) { - tp.nextDoc(); - assertEquals(tp.docID(), i); - assertEquals(tp.nextPosition(), 1); - } - - tp = MultiTerms.getTermPostingsEnum(reader, this.field, new BytesRef("a")); - - for (int i = 0; i < 10; i++) { - tp.nextDoc(); - assertEquals(tp.docID(), i); - assertEquals(tp.nextPosition(), 0); - } - reader.close(); - directory.close(); - } - - // Simply extends IndexInput in a way that we are able to count the number - // of invocations of seek() - class SeeksCountingStream extends FilterIndexInput { - - SeeksCountingStream(IndexInput input) { - super("SeekCountingStream(" + input + ")", input); - } - - @Override - public void seek(long pos) throws IOException { - TestLazyProxSkipping.this.seeksCounter++; - in.seek(pos); - } - - @Override - public SeeksCountingStream clone() { - return new SeeksCountingStream(in.clone()); - } - } -} 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 2a5d8fffd01..e5ca29b9fae 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java @@ -21,8 +21,8 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.Objects; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.PostingsEnum; @@ -261,10 +261,10 @@ class TermIntervalsSource extends IntervalsSource { /** * A guess of the average number of simple operations for the initial seek and buffer refill per * document for the positions of a term. See also {@link - * Lucene99PostingsReader.EverythingEnum#nextPosition()}. + * Lucene912PostingsReader.EverythingEnum#nextPosition()}. * *

    Aside: Instead of being constant this could depend among others on {@link - * Lucene99PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link + * Lucene912PostingsFormat#BLOCK_SIZE}, {@link TermsEnum#docFreq()}, {@link * TermsEnum#totalTermFreq()}, {@link DocIdSetIterator#cost()} (expected number of matching docs), * {@link LeafReader#maxDoc()} (total number of docs in the segment), and the seek time and block * size of the device storing the index. @@ -272,7 +272,7 @@ class TermIntervalsSource extends IntervalsSource { private static final int TERM_POSNS_SEEK_OPS_PER_DOC = 128; /** - * Number of simple operations in {@link Lucene99PostingsReader.EverythingEnum#nextPosition()} + * Number of simple operations in {@link Lucene912PostingsReader.EverythingEnum#nextPosition()} * when no seek or buffer refill is done. */ private static final int TERM_OPS_PER_POS = 7; diff --git a/lucene/suggest/src/java/module-info.java b/lucene/suggest/src/java/module-info.java index 673fa391213..c3f5b7c178f 100644 --- a/lucene/suggest/src/java/module-info.java +++ b/lucene/suggest/src/java/module-info.java @@ -31,7 +31,8 @@ module org.apache.lucene.suggest { org.apache.lucene.search.suggest.document.Completion50PostingsFormat, 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.Completion99PostingsFormat, + org.apache.lucene.search.suggest.document.Completion912PostingsFormat; provides org.apache.lucene.analysis.TokenFilterFactory with org.apache.lucene.search.suggest.analyzing.SuggestStopFilterFactory; } diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion912PostingsFormat.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion912PostingsFormat.java new file mode 100644 index 00000000000..341b034e8e8 --- /dev/null +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion912PostingsFormat.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.lucene912.Lucene912PostingsFormat} + * + * @lucene.experimental + */ +public class Completion912PostingsFormat extends CompletionPostingsFormat { + /** Creates a {@link Completion912PostingsFormat} that will load the completion FST on-heap. */ + public Completion912PostingsFormat() { + this(FSTLoadMode.ON_HEAP); + } + + /** + * Creates a {@link Completion912PostingsFormat} that will use the provided fstLoadMode + * to determine if the completion FST should be loaded on or off heap. + */ + public Completion912PostingsFormat(FSTLoadMode fstLoadMode) { + super("Completion912", fstLoadMode); + } + + @Override + protected PostingsFormat delegatePostingsFormat() { + return PostingsFormat.forName("Lucene912"); + } +} diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion99PostingsFormat.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion99PostingsFormat.java index 4449d754e29..d3ea76d5425 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion99PostingsFormat.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/Completion99PostingsFormat.java @@ -19,8 +19,10 @@ package org.apache.lucene.search.suggest.document; import org.apache.lucene.codecs.PostingsFormat; /** - * {@link CompletionPostingsFormat} for {@link - * org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat} + * {@link org.apache.lucene.search.suggest.document.CompletionPostingsFormat} for {@code + * org.apache.lucene.backward_codecs.lucene99.Lucene99PostingsFormat}. This format is only used for + * backward-compatibility of the index format and cannot be used to write data, use {@link + * Completion912PostingsFormat} 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 81d1ed9465d..8544a97b88f 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 @@ -34,3 +34,4 @@ org.apache.lucene.search.suggest.document.Completion50PostingsFormat 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 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 ca5eff9cc5a..2d6e9471c98 100644 --- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java +++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java @@ -951,7 +951,7 @@ public class TestSuggestField extends LuceneTestCase { new FilterCodec(TestUtil.getDefaultCodec().getName(), TestUtil.getDefaultCodec()) { final CompletionPostingsFormat.FSTLoadMode fstLoadMode = RandomPicks.randomFrom(random(), CompletionPostingsFormat.FSTLoadMode.values()); - final PostingsFormat postingsFormat = new Completion99PostingsFormat(fstLoadMode); + final PostingsFormat postingsFormat = new Completion912PostingsFormat(fstLoadMode); @Override public PostingsFormat postingsFormat() { diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneFixedGap.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneFixedGap.java index 0a0c8f673db..9688c6c0e35 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneFixedGap.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneFixedGap.java @@ -28,9 +28,9 @@ import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader; import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter; import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase; import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; // javadocs -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -38,7 +38,7 @@ import org.apache.lucene.index.SegmentWriteState; // any PostingsFormat and make it ord-able... /** - * Customized version of {@link Lucene99PostingsFormat} that uses {@link FixedGapTermsIndexWriter}. + * Customized version of {@link Lucene912PostingsFormat} that uses {@link FixedGapTermsIndexWriter}. */ public final class LuceneFixedGap extends PostingsFormat { final int termIndexInterval; @@ -54,7 +54,7 @@ public final class LuceneFixedGap extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase docs = new Lucene99PostingsWriter(state); + PostingsWriterBase docs = new Lucene912PostingsWriter(state); // TODO: should we make the terms index more easily // pluggable? Ie so that this codec would record which @@ -91,7 +91,7 @@ public final class LuceneFixedGap extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene99PostingsReader(state); + PostingsReaderBase postings = new Lucene912PostingsReader(state); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapDocFreqInterval.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapDocFreqInterval.java index a864263b7c3..1fafc8f448c 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapDocFreqInterval.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapDocFreqInterval.java @@ -29,9 +29,9 @@ import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase; import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; // javadocs -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -39,7 +39,7 @@ import org.apache.lucene.index.SegmentWriteState; // any PostingsFormat and make it ord-able... /** - * Customized version of {@link Lucene99PostingsFormat} that uses {@link + * Customized version of {@link Lucene912PostingsFormat} that uses {@link * VariableGapTermsIndexWriter} with a fixed interval, but forcing high docfreq terms to be indexed * terms. */ @@ -59,7 +59,7 @@ public final class LuceneVarGapDocFreqInterval extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase docs = new Lucene99PostingsWriter(state); + PostingsWriterBase docs = new Lucene912PostingsWriter(state); // TODO: should we make the terms index more easily // pluggable? Ie so that this codec would record which @@ -100,7 +100,7 @@ public final class LuceneVarGapDocFreqInterval extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene99PostingsReader(state); + PostingsReaderBase postings = new Lucene912PostingsReader(state); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapFixedInterval.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapFixedInterval.java index 667acf0ceb3..bfaf0914651 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapFixedInterval.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/blockterms/LuceneVarGapFixedInterval.java @@ -29,9 +29,9 @@ import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase; import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader; import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; // javadocs -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; @@ -39,7 +39,7 @@ import org.apache.lucene.index.SegmentWriteState; // any PostingsFormat and make it ord-able... /** - * Customized version of {@link Lucene99PostingsFormat} that uses {@link + * Customized version of {@link Lucene912PostingsFormat} that uses {@link * VariableGapTermsIndexWriter} with a fixed interval. */ public final class LuceneVarGapFixedInterval extends PostingsFormat { @@ -56,7 +56,7 @@ public final class LuceneVarGapFixedInterval extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase docs = new Lucene99PostingsWriter(state); + PostingsWriterBase docs = new Lucene912PostingsWriter(state); // TODO: should we make the terms index more easily // pluggable? Ie so that this codec would record which @@ -95,7 +95,7 @@ public final class LuceneVarGapFixedInterval extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { - PostingsReaderBase postings = new Lucene99PostingsReader(state); + PostingsReaderBase postings = new Lucene912PostingsReader(state); TermsIndexReaderBase indexReader; boolean success = false; diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/mockrandom/MockRandomPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/mockrandom/MockRandomPostingsFormat.java index 10e5ec96332..6b10e7b19f4 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/mockrandom/MockRandomPostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/mockrandom/MockRandomPostingsFormat.java @@ -37,8 +37,8 @@ import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsReader; import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsWriter; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsReader; import org.apache.lucene.codecs.lucene90.blocktree.Lucene90BlockTreeTermsWriter; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.codecs.memory.FSTTermsReader; import org.apache.lucene.codecs.memory.FSTTermsWriter; import org.apache.lucene.index.FieldInfo; @@ -121,7 +121,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat { random.nextInt(); // consume a random for buffersize - PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(state); + PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(state); final FieldsConsumer fields; final int t1 = random.nextInt(4); @@ -289,7 +289,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat { System.out.println("MockRandomCodec: readBufferSize=" + readBufferSize); } - PostingsReaderBase postingsReader = new Lucene99PostingsReader(state); + PostingsReaderBase postingsReader = new Lucene912PostingsReader(state); final FieldsProducer fields; final int t1 = random.nextInt(4); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java index 0c469b392a2..4b13531327b 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/uniformsplit/UniformSplitRot13PostingsFormat.java @@ -23,8 +23,8 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsReaderBase; import org.apache.lucene.codecs.PostingsWriterBase; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsReader; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsWriter; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsReader; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsWriter; import org.apache.lucene.codecs.uniformsplit.BlockDecoder; import org.apache.lucene.codecs.uniformsplit.BlockEncoder; import org.apache.lucene.codecs.uniformsplit.IndexDictionary; @@ -67,7 +67,7 @@ public class UniformSplitRot13PostingsFormat extends PostingsFormat { @Override public FieldsConsumer fieldsConsumer(SegmentWriteState segmentWriteState) throws IOException { - PostingsWriterBase postingsWriter = new Lucene99PostingsWriter(segmentWriteState); + PostingsWriterBase postingsWriter = new Lucene912PostingsWriter(segmentWriteState); boolean success = false; try { FieldsConsumer fieldsConsumer = createFieldsConsumer(segmentWriteState, postingsWriter); @@ -145,7 +145,7 @@ public class UniformSplitRot13PostingsFormat extends PostingsFormat { @Override public FieldsProducer fieldsProducer(SegmentReadState segmentReadState) throws IOException { - PostingsReaderBase postingsReader = new Lucene99PostingsReader(segmentReadState); + PostingsReaderBase postingsReader = new Lucene912PostingsReader(segmentReadState); boolean success = false; try { FieldsProducer fieldsProducer = createFieldsProducer(segmentReadState, postingsReader); diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java index c386a7ef85e..378444e394a 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestRuleSetupAndRestoreClassEnv.java @@ -38,7 +38,7 @@ import java.util.TimeZone; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.PostingsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; import org.apache.lucene.codecs.simpletext.SimpleTextCodec; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.tests.codecs.asserting.AssertingCodec; @@ -188,9 +188,9 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule { } else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) { codec = CompressingCodec.randomInstance(random); - } else if ("Lucene99".equals(TEST_CODEC) - || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene99"))) { - codec = new Lucene99Codec(RandomPicks.randomFrom(random, Lucene99Codec.Mode.values())); + } else if ("Lucene912".equals(TEST_CODEC) + || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene912"))) { + codec = new Lucene912Codec(RandomPicks.randomFrom(random, Lucene912Codec.Mode.values())); } else if (!"random".equals(TEST_CODEC)) { codec = Codec.forName(TEST_CODEC); } else if ("random".equals(TEST_POSTINGSFORMAT)) { diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java index 8bed935f7db..195b5563958 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/util/TestUtil.java @@ -56,9 +56,9 @@ import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat; import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.codecs.lucene912.Lucene912Codec; +import org.apache.lucene.codecs.lucene912.Lucene912PostingsFormat; import org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsFormat; -import org.apache.lucene.codecs.lucene99.Lucene99PostingsFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; import org.apache.lucene.document.BinaryDocValuesField; @@ -1231,7 +1231,7 @@ public final class TestUtil { * different from {@link Codec#getDefault()} because that is randomized. */ public static Codec getDefaultCodec() { - return new Lucene99Codec(); + return new Lucene912Codec(); } /** @@ -1239,7 +1239,7 @@ public final class TestUtil { * Lucene. */ public static PostingsFormat getDefaultPostingsFormat() { - return new Lucene99PostingsFormat(); + return new Lucene912PostingsFormat(); } /** @@ -1250,7 +1250,7 @@ public final class TestUtil { */ public static PostingsFormat getDefaultPostingsFormat( int minItemsPerBlock, int maxItemsPerBlock) { - return new Lucene99PostingsFormat(minItemsPerBlock, maxItemsPerBlock); + return new Lucene912PostingsFormat(minItemsPerBlock, maxItemsPerBlock); } /** Returns a random postings format that supports term ordinals */