diff --git a/.gitattributes b/.gitattributes
index e4f4bf8b496..a3135003e80 100644
--- a/.gitattributes
+++ b/.gitattributes
@@ -1,6 +1,6 @@
# Ignore all differences in line endings for the lock file.
-versions.lock text eol=lf
-versions.props text eol=lf
+versions.lock text eol=lf
+versions.toml text eol=lf
# Gradle files are always in LF.
*.gradle text eol=lf
diff --git a/README.md b/README.md
index fe523af81b2..c613a16986e 100644
--- a/README.md
+++ b/README.md
@@ -23,6 +23,7 @@ Apache Lucene is a high-performance, full-featured text search engine library
written in Java.
[![Build Status](https://ci-builds.apache.org/job/Lucene/job/Lucene-Artifacts-main/badge/icon?subject=Lucene)](https://ci-builds.apache.org/job/Lucene/job/Lucene-Artifacts-main/)
+[![Revved up by Develocity](https://img.shields.io/badge/Revved%20up%20by-Develocity-06A0CE?logo=Gradle&labelColor=02303A)](https://ge.apache.org/scans?search.buildToolType=gradle&search.rootProjectNames=lucene-root)
## Online Documentation
diff --git a/build-tools/build-infra/src/main/java/org/apache/lucene/gradle/Checksum.java b/build-tools/build-infra/src/main/java/org/apache/lucene/gradle/Checksum.java
index 49374214c9c..7566102294f 100644
--- a/build-tools/build-infra/src/main/java/org/apache/lucene/gradle/Checksum.java
+++ b/build-tools/build-infra/src/main/java/org/apache/lucene/gradle/Checksum.java
@@ -68,7 +68,6 @@ public class Checksum extends DefaultTask {
public Checksum() {
outputDir = new File(getProject().getBuildDir(), "checksums");
- algorithm = Algorithm.SHA256;
}
@InputFiles
diff --git a/dev-tools/doap/lucene.rdf b/dev-tools/doap/lucene.rdf
index b91eec32904..7c400eb545b 100644
--- a/dev-tools/doap/lucene.rdf
+++ b/dev-tools/doap/lucene.rdf
@@ -67,6 +67,13 @@
+
+
+ lucene-9.11.1
+ 2024-06-27
+ 9.11.1
+
+ .
lucene-9.11.0
diff --git a/dev-tools/scripts/smokeTestRelease.py b/dev-tools/scripts/smokeTestRelease.py
index 87e70adbb1f..82946f4e5b3 100755
--- a/dev-tools/scripts/smokeTestRelease.py
+++ b/dev-tools/scripts/smokeTestRelease.py
@@ -582,8 +582,8 @@ def verifyUnpacked(java, artifact, unpackPath, gitRevision, version, testArgs):
'luke', 'memory', 'misc', 'monitor', 'queries', 'queryparser', 'replicator',
'sandbox', 'spatial-extras', 'spatial-test-fixtures', 'spatial3d', 'suggest', 'test-framework', 'licenses']
if isSrc:
- expected_src_root_files = ['build.gradle', 'buildSrc', 'CONTRIBUTING.md', 'dev-docs', 'dev-tools', 'gradle', 'gradlew',
- 'gradlew.bat', 'help', 'lucene', 'settings.gradle', 'versions.lock', 'versions.props']
+ expected_src_root_files = ['build.gradle', 'build-tools', 'CONTRIBUTING.md', 'dev-docs', 'dev-tools', 'gradle', 'gradlew',
+ 'gradlew.bat', 'help', 'lucene', 'settings.gradle', 'versions.lock', 'versions.toml']
expected_src_lucene_files = ['build.gradle', 'documentation', 'distribution', 'dev-docs']
is_in_list(in_root_folder, expected_src_root_files)
is_in_list(in_lucene_folder, expected_folders)
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index f6ee2edbde8..26a7c06e483 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -258,7 +258,8 @@ New Features
Improvements
---------------------
-(No changes)
+
+* GITHUB#13548: Refactor and javadoc update for KNN vector writer classes. (Patrick Zhai)
Optimizations
---------------------
@@ -277,14 +278,52 @@ Optimizations
* GITHUB#12941: Don't preserve auxiliary buffer contents in LSBRadixSorter if it grows. (Stefan Vodita)
+* GITHUB#13175: Stop double-checking priority queue inserts in some FacetCount classes. (Jakub Slowinski)
+
+* GITHUB#13538: Slightly reduce heap usage for HNSW and scalar quantized vector writers. (Ben Trent)
+
+Changes in runtime behavior
+---------------------
+
+* GITHUB#13472: When an executor is provided to the IndexSearcher constructor, the searcher now executes tasks on the
+ thread that invoked a search as well as its configured executor. Users should reduce the executor's thread-count by 1
+ to retain the previous level of parallelism. Moreover, it is now possible to start searches from the same executor
+ that is configured in the IndexSearcher without risk of deadlocking. A separate executor for starting searches is no
+ longer required. (Armin Braun)
+
Bug Fixes
---------------------
-(No changes)
+
+* GITHUB#13384: Fix highlighter to use longer passages instead of shorter individual terms. (Zack Kendall)
+
+* GITHUB#13463: Address bug in MultiLeafKnnCollector causing #minCompetitiveSimilarity to stay artificially low in
+ some corner cases. (Greg Miller)
+
+* GITHUB#13553: Correct RamUsageEstimate for scalar quantized knn vector formats so that raw vectors are correctly
+ accounted for. (Ben Trent)
Other
--------------------
(No changes)
+======================== Lucene 9.11.1 =======================
+
+Bug Fixes
+---------------------
+
+* GITHUB#13498: Avoid performance regression by constructing lazily the PointTree in NumericComparator. (Ignacio Vera)
+
+* GITHUB#13501, GITHUB#13478: Remove intra-merge parallelism for everything except HNSW graph merges. (Ben Trent)
+
+* GITHUB#13498, GITHUB#13340: Allow adding a parent field to an index with no fields (Michael Sokolov)
+
+* GITHUB#12431: Fix IndexOutOfBoundsException thrown in DefaultPassageFormatter
+ by unordered matches. (Stephane Campinas)
+
+* GITHUB#13493: StringValueFacetCounts stops throwing NPE when faceting over an empty match-set. (Grebennikov Roman,
+ Stefan Vodita)
+
+
======================== Lucene 9.11.0 =======================
API Changes
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/FieldReader.java b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/FieldReader.java
index 3387c2c7b5c..05b350d5ba2 100644
--- a/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/FieldReader.java
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene40/blocktree/FieldReader.java
@@ -88,21 +88,17 @@ public final class FieldReader extends Terms {
(new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length)).readVLong()
>>> Lucene40BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS;
// Initialize FST always off-heap.
- final IndexInput clone = indexIn.clone();
- clone.seek(indexStartFP);
+ final FST.FSTMetadata fstMetadata;
if (metaIn == indexIn) { // Only true before Lucene 8.6
- index =
- new FST<>(
- readMetadata(clone, ByteSequenceOutputs.getSingleton()),
- clone,
- new OffHeapFSTStore());
+ final IndexInput clone = indexIn.clone();
+ clone.seek(indexStartFP);
+ fstMetadata = readMetadata(clone, ByteSequenceOutputs.getSingleton());
+ // FST bytes actually only start after the metadata.
+ indexStartFP = clone.getFilePointer();
} else {
- index =
- new FST<>(
- readMetadata(metaIn, ByteSequenceOutputs.getSingleton()),
- clone,
- new OffHeapFSTStore());
+ fstMetadata = readMetadata(metaIn, ByteSequenceOutputs.getSingleton());
}
+ index = FST.fromFSTReader(fstMetadata, new OffHeapFSTStore(indexIn, indexStartFP, fstMetadata));
/*
if (false) {
final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsWriterV5.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsWriterV5.java
new file mode 100644
index 00000000000..165f296f282
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene40/blocktree/Lucene40BlockTreeTermsWriterV5.java
@@ -0,0 +1,1096 @@
+/*
+ * 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.lucene40.blocktree;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil;
+import org.apache.lucene.codecs.BlockTermState;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteArrayDataOutput;
+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.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.compress.LZ4;
+import org.apache.lucene.util.compress.LowercaseAsciiCompression;
+import org.apache.lucene.util.fst.ByteSequenceOutputs;
+import org.apache.lucene.util.fst.BytesRefFSTEnum;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.FSTCompiler;
+import org.apache.lucene.util.fst.Util;
+
+/**
+ * Writer for {@link Lucene40BlockTreeTermsReader} prior to the refactoring that separated metadata
+ * to its own file.
+ *
+ * @see Lucene40BlockTreeTermsReader#VERSION_META_FILE
+ */
+public final class Lucene40BlockTreeTermsWriterV5 extends FieldsConsumer {
+
+ /**
+ * Suggested default value for the {@code minItemsInBlock} parameter to {@link
+ * #Lucene40BlockTreeTermsWriterV5(SegmentWriteState,PostingsWriterBase,int,int)}.
+ */
+ public static final int DEFAULT_MIN_BLOCK_SIZE = 25;
+
+ /**
+ * Suggested default value for the {@code maxItemsInBlock} parameter to {@link
+ * #Lucene40BlockTreeTermsWriterV5(SegmentWriteState,PostingsWriterBase,int,int)}.
+ */
+ public static final int DEFAULT_MAX_BLOCK_SIZE = 48;
+
+ private static final int VERSION_CURRENT = Lucene40BlockTreeTermsReader.VERSION_META_FILE - 1;
+
+ // public static boolean DEBUG = false;
+ // public static boolean DEBUG2 = false;
+
+ // private final static boolean SAVE_DOT_FILES = false;
+
+ private final IndexOutput termsOut;
+ private final IndexOutput indexOut;
+ final int maxDoc;
+ final int minItemsInBlock;
+ final int maxItemsInBlock;
+
+ final PostingsWriterBase postingsWriter;
+ final FieldInfos fieldInfos;
+
+ private static class FieldMetaData {
+ public final FieldInfo fieldInfo;
+ public final BytesRef rootCode;
+ public final long numTerms;
+ public final long indexStartFP;
+ public final long sumTotalTermFreq;
+ public final long sumDocFreq;
+ public final int docCount;
+ public final BytesRef minTerm;
+ public final BytesRef maxTerm;
+
+ public FieldMetaData(
+ FieldInfo fieldInfo,
+ BytesRef rootCode,
+ long numTerms,
+ long indexStartFP,
+ long sumTotalTermFreq,
+ long sumDocFreq,
+ int docCount,
+ BytesRef minTerm,
+ BytesRef maxTerm) {
+ assert numTerms > 0;
+ this.fieldInfo = fieldInfo;
+ assert rootCode != null : "field=" + fieldInfo.name + " numTerms=" + numTerms;
+ this.rootCode = rootCode;
+ this.indexStartFP = indexStartFP;
+ this.numTerms = numTerms;
+ this.sumTotalTermFreq = sumTotalTermFreq;
+ this.sumDocFreq = sumDocFreq;
+ this.docCount = docCount;
+ this.minTerm = minTerm;
+ this.maxTerm = maxTerm;
+ }
+ }
+
+ private final List fields = new ArrayList<>();
+
+ /**
+ * Create a new writer. The number of items (terms or sub-blocks) per block will aim to be between
+ * minItemsPerBlock and maxItemsPerBlock, though in some cases the blocks may be smaller than the
+ * min.
+ */
+ public Lucene40BlockTreeTermsWriterV5(
+ SegmentWriteState state,
+ PostingsWriterBase postingsWriter,
+ int minItemsInBlock,
+ int maxItemsInBlock)
+ throws IOException {
+ validateSettings(minItemsInBlock, maxItemsInBlock);
+
+ this.minItemsInBlock = minItemsInBlock;
+ this.maxItemsInBlock = maxItemsInBlock;
+
+ this.maxDoc = state.segmentInfo.maxDoc();
+ this.fieldInfos = state.fieldInfos;
+ this.postingsWriter = postingsWriter;
+
+ final String termsName =
+ IndexFileNames.segmentFileName(
+ state.segmentInfo.name,
+ state.segmentSuffix,
+ Lucene40BlockTreeTermsReader.TERMS_EXTENSION);
+ termsOut = EndiannessReverserUtil.createOutput(state.directory, termsName, state.context);
+ boolean success = false;
+ IndexOutput indexOut = null;
+ try {
+ CodecUtil.writeIndexHeader(
+ termsOut,
+ Lucene40BlockTreeTermsReader.TERMS_CODEC_NAME,
+ VERSION_CURRENT,
+ state.segmentInfo.getId(),
+ state.segmentSuffix);
+
+ final String indexName =
+ IndexFileNames.segmentFileName(
+ state.segmentInfo.name,
+ state.segmentSuffix,
+ Lucene40BlockTreeTermsReader.TERMS_INDEX_EXTENSION);
+ indexOut = EndiannessReverserUtil.createOutput(state.directory, indexName, state.context);
+ CodecUtil.writeIndexHeader(
+ indexOut,
+ Lucene40BlockTreeTermsReader.TERMS_INDEX_CODEC_NAME,
+ VERSION_CURRENT,
+ state.segmentInfo.getId(),
+ state.segmentSuffix);
+ // segment = state.segmentInfo.name;
+
+ postingsWriter.init(termsOut, state); // have consumer write its format/header
+
+ this.indexOut = indexOut;
+ success = true;
+ } finally {
+ if (!success) {
+ IOUtils.closeWhileHandlingException(termsOut, indexOut);
+ }
+ }
+ }
+
+ /** Writes the terms file trailer. */
+ private void writeTrailer(IndexOutput out, long dirStart) throws IOException {
+ out.writeLong(dirStart);
+ }
+
+ /** Writes the index file trailer. */
+ private void writeIndexTrailer(IndexOutput indexOut, long dirStart) throws IOException {
+ indexOut.writeLong(dirStart);
+ }
+
+ /** Throws {@code IllegalArgumentException} if any of these settings is invalid. */
+ public static void validateSettings(int minItemsInBlock, int maxItemsInBlock) {
+ if (minItemsInBlock <= 1) {
+ throw new IllegalArgumentException("minItemsInBlock must be >= 2; got " + minItemsInBlock);
+ }
+ if (minItemsInBlock > maxItemsInBlock) {
+ throw new IllegalArgumentException(
+ "maxItemsInBlock must be >= minItemsInBlock; got maxItemsInBlock="
+ + maxItemsInBlock
+ + " minItemsInBlock="
+ + minItemsInBlock);
+ }
+ if (2 * (minItemsInBlock - 1) > maxItemsInBlock) {
+ throw new IllegalArgumentException(
+ "maxItemsInBlock must be at least 2*(minItemsInBlock-1); got maxItemsInBlock="
+ + maxItemsInBlock
+ + " minItemsInBlock="
+ + minItemsInBlock);
+ }
+ }
+
+ @Override
+ public void write(Fields fields, NormsProducer norms) throws IOException {
+ // if (DEBUG) System.out.println("\nBTTW.write seg=" + segment);
+
+ String lastField = null;
+ for (String field : fields) {
+ assert lastField == null || lastField.compareTo(field) < 0;
+ lastField = field;
+
+ // if (DEBUG) System.out.println("\nBTTW.write seg=" + segment + " field=" + field);
+ Terms terms = fields.terms(field);
+ if (terms == null) {
+ continue;
+ }
+
+ TermsEnum termsEnum = terms.iterator();
+ TermsWriter termsWriter = new TermsWriter(fieldInfos.fieldInfo(field));
+ while (true) {
+ BytesRef term = termsEnum.next();
+ // if (DEBUG) System.out.println("BTTW: next term " + term);
+
+ if (term == null) {
+ break;
+ }
+
+ // if (DEBUG) System.out.println("write field=" + fieldInfo.name + " term=" +
+ // brToString(term));
+ termsWriter.write(term, termsEnum, norms);
+ }
+
+ termsWriter.finish();
+
+ // if (DEBUG) System.out.println("\nBTTW.write done seg=" + segment + " field=" + field);
+ }
+ }
+
+ static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) {
+ assert fp < (1L << 62);
+ return (fp << 2)
+ | (hasTerms ? Lucene40BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0)
+ | (isFloor ? Lucene40BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0);
+ }
+
+ private static class PendingEntry {
+ public final boolean isTerm;
+
+ protected PendingEntry(boolean isTerm) {
+ this.isTerm = isTerm;
+ }
+ }
+
+ private static final class PendingTerm extends PendingEntry {
+ public final byte[] termBytes;
+ // stats + metadata
+ public final BlockTermState state;
+
+ public PendingTerm(BytesRef term, BlockTermState state) {
+ super(true);
+ this.termBytes = new byte[term.length];
+ System.arraycopy(term.bytes, term.offset, termBytes, 0, term.length);
+ this.state = state;
+ }
+
+ @Override
+ public String toString() {
+ return "TERM: " + brToString(termBytes);
+ }
+ }
+
+ // for debugging
+ @SuppressWarnings("unused")
+ static String brToString(BytesRef b) {
+ if (b == null) {
+ return "(null)";
+ } else {
+ try {
+ return b.utf8ToString() + " " + b;
+ } catch (Throwable t) {
+ // If BytesRef isn't actually UTF8, or it's eg a
+ // prefix of UTF8 that ends mid-unicode-char, we
+ // fallback to hex:
+ return b.toString();
+ }
+ }
+ }
+
+ // for debugging
+ @SuppressWarnings("unused")
+ static String brToString(byte[] b) {
+ return brToString(new BytesRef(b));
+ }
+
+ private static final class PendingBlock extends PendingEntry {
+ public final BytesRef prefix;
+ public final long fp;
+ public FST index;
+ public List> subIndices;
+ public final boolean hasTerms;
+ public final boolean isFloor;
+ public final int floorLeadByte;
+
+ public PendingBlock(
+ BytesRef prefix,
+ long fp,
+ boolean hasTerms,
+ boolean isFloor,
+ int floorLeadByte,
+ List> subIndices) {
+ super(false);
+ this.prefix = prefix;
+ this.fp = fp;
+ this.hasTerms = hasTerms;
+ this.isFloor = isFloor;
+ this.floorLeadByte = floorLeadByte;
+ this.subIndices = subIndices;
+ }
+
+ @Override
+ public String toString() {
+ return "BLOCK: prefix=" + brToString(prefix);
+ }
+
+ public void compileIndex(
+ List blocks,
+ ByteBuffersDataOutput scratchBytes,
+ IntsRefBuilder scratchIntsRef)
+ throws IOException {
+
+ assert (isFloor && blocks.size() > 1) || (isFloor == false && blocks.size() == 1)
+ : "isFloor=" + isFloor + " blocks=" + blocks;
+ assert this == blocks.get(0);
+
+ assert scratchBytes.size() == 0;
+
+ // TODO: try writing the leading vLong in MSB order
+ // (opposite of what Lucene does today), for better
+ // outputs sharing in the FST
+ scratchBytes.writeVLong(encodeOutput(fp, hasTerms, isFloor));
+ if (isFloor) {
+ scratchBytes.writeVInt(blocks.size() - 1);
+ for (int i = 1; i < blocks.size(); i++) {
+ PendingBlock sub = blocks.get(i);
+ assert sub.floorLeadByte != -1;
+ // if (DEBUG) {
+ // System.out.println(" write floorLeadByte=" +
+ // Integer.toHexString(sub.floorLeadByte&0xff));
+ // }
+ scratchBytes.writeByte((byte) sub.floorLeadByte);
+ assert sub.fp > fp;
+ scratchBytes.writeVLong((sub.fp - fp) << 1 | (sub.hasTerms ? 1 : 0));
+ }
+ }
+
+ final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
+ final FSTCompiler fstCompiler =
+ new FSTCompiler.Builder<>(FST.INPUT_TYPE.BYTE1, outputs).build();
+ // if (DEBUG) {
+ // System.out.println(" compile index for prefix=" + prefix);
+ // }
+ // indexBuilder.DEBUG = false;
+ final byte[] bytes = scratchBytes.toArrayCopy();
+ assert bytes.length > 0;
+ fstCompiler.add(Util.toIntsRef(prefix, scratchIntsRef), new BytesRef(bytes, 0, bytes.length));
+ scratchBytes.reset();
+
+ // Copy over index for all sub-blocks
+ for (PendingBlock block : blocks) {
+ if (block.subIndices != null) {
+ for (FST subIndex : block.subIndices) {
+ append(fstCompiler, subIndex, scratchIntsRef);
+ }
+ block.subIndices = null;
+ }
+ }
+
+ index = FST.fromFSTReader(fstCompiler.compile(), fstCompiler.getFSTReader());
+
+ assert subIndices == null;
+
+ /*
+ Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"));
+ Util.toDot(index, w, false, false);
+ System.out.println("SAVED to out.dot");
+ w.close();
+ */
+ }
+
+ // TODO: maybe we could add bulk-add method to
+ // Builder? Takes FST and unions it w/ current
+ // FST.
+ private void append(
+ FSTCompiler fstCompiler, FST subIndex, IntsRefBuilder scratchIntsRef)
+ throws IOException {
+ final BytesRefFSTEnum subIndexEnum = new BytesRefFSTEnum<>(subIndex);
+ BytesRefFSTEnum.InputOutput indexEnt;
+ while ((indexEnt = subIndexEnum.next()) != null) {
+ // if (DEBUG) {
+ // System.out.println(" add sub=" + indexEnt.input + " " + indexEnt.input + " output="
+ // + indexEnt.output);
+ // }
+ fstCompiler.add(Util.toIntsRef(indexEnt.input, scratchIntsRef), indexEnt.output);
+ }
+ }
+ }
+
+ private final ByteBuffersDataOutput scratchBytes = ByteBuffersDataOutput.newResettableInstance();
+ private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
+
+ static final BytesRef EMPTY_BYTES_REF = new BytesRef();
+
+ private static class StatsWriter {
+
+ private final DataOutput out;
+ private final boolean hasFreqs;
+ private int singletonCount;
+
+ StatsWriter(DataOutput out, boolean hasFreqs) {
+ this.out = out;
+ this.hasFreqs = hasFreqs;
+ }
+
+ void add(int df, long ttf) throws IOException {
+ // Singletons (DF==1, TTF==1) are run-length encoded
+ if (df == 1 && (hasFreqs == false || ttf == 1)) {
+ singletonCount++;
+ } else {
+ finish();
+ out.writeVInt(df << 1);
+ if (hasFreqs) {
+ out.writeVLong(ttf - df);
+ }
+ }
+ }
+
+ void finish() throws IOException {
+ if (singletonCount > 0) {
+ out.writeVInt(((singletonCount - 1) << 1) | 1);
+ singletonCount = 0;
+ }
+ }
+ }
+
+ class TermsWriter {
+ private final FieldInfo fieldInfo;
+ private long numTerms;
+ final FixedBitSet docsSeen;
+ long sumTotalTermFreq;
+ long sumDocFreq;
+ long indexStartFP;
+
+ // Records index into pending where the current prefix at that
+ // length "started"; for example, if current term starts with 't',
+ // startsByPrefix[0] is the index into pending for the first
+ // term/sub-block starting with 't'. We use this to figure out when
+ // to write a new block:
+ private final BytesRefBuilder lastTerm = new BytesRefBuilder();
+ private int[] prefixStarts = new int[8];
+
+ // Pending stack of terms and blocks. As terms arrive (in sorted order)
+ // we append to this stack, and once the top of the stack has enough
+ // terms starting with a common prefix, we write a new block with
+ // those terms and replace those terms in the stack with a new block:
+ private final List pending = new ArrayList<>();
+
+ // Reused in writeBlocks:
+ private final List newBlocks = new ArrayList<>();
+
+ private PendingTerm firstPendingTerm;
+ private PendingTerm lastPendingTerm;
+
+ /** Writes the top count entries in pending, using prevTerm to compute the prefix. */
+ void writeBlocks(int prefixLength, int count) throws IOException {
+
+ assert count > 0;
+
+ // if (DEBUG2) {
+ // BytesRef br = new BytesRef(lastTerm.bytes());
+ // br.length = prefixLength;
+ // System.out.println("writeBlocks: seg=" + segment + " prefix=" + brToString(br) + " count="
+ // + count);
+ // }
+
+ // Root block better write all remaining pending entries:
+ assert prefixLength > 0 || count == pending.size();
+
+ int lastSuffixLeadLabel = -1;
+
+ // True if we saw at least one term in this block (we record if a block
+ // only points to sub-blocks in the terms index so we can avoid seeking
+ // to it when we are looking for a term):
+ boolean hasTerms = false;
+ boolean hasSubBlocks = false;
+
+ int start = pending.size() - count;
+ int end = pending.size();
+ int nextBlockStart = start;
+ int nextFloorLeadLabel = -1;
+
+ for (int i = start; i < end; i++) {
+
+ PendingEntry ent = pending.get(i);
+
+ int suffixLeadLabel;
+
+ if (ent.isTerm) {
+ PendingTerm term = (PendingTerm) ent;
+ if (term.termBytes.length == prefixLength) {
+ // Suffix is 0, i.e. prefix 'foo' and term is
+ // 'foo' so the term has empty string suffix
+ // in this block
+ assert lastSuffixLeadLabel == -1
+ : "i=" + i + " lastSuffixLeadLabel=" + lastSuffixLeadLabel;
+ suffixLeadLabel = -1;
+ } else {
+ suffixLeadLabel = term.termBytes[prefixLength] & 0xff;
+ }
+ } else {
+ PendingBlock block = (PendingBlock) ent;
+ assert block.prefix.length > prefixLength;
+ suffixLeadLabel = block.prefix.bytes[block.prefix.offset + prefixLength] & 0xff;
+ }
+ // if (DEBUG) System.out.println(" i=" + i + " ent=" + ent + " suffixLeadLabel=" +
+ // suffixLeadLabel);
+
+ if (suffixLeadLabel != lastSuffixLeadLabel) {
+ int itemsInBlock = i - nextBlockStart;
+ if (itemsInBlock >= minItemsInBlock && end - nextBlockStart > maxItemsInBlock) {
+ // The count is too large for one block, so we must break it into "floor" blocks, where
+ // we record
+ // the leading label of the suffix of the first term in each floor block, so at search
+ // time we can
+ // jump to the right floor block. We just use a naive greedy segmenter here: make a new
+ // floor
+ // block as soon as we have at least minItemsInBlock. This is not always best: it often
+ // produces
+ // a too-small block as the final block:
+ boolean isFloor = itemsInBlock < count;
+ newBlocks.add(
+ writeBlock(
+ prefixLength,
+ isFloor,
+ nextFloorLeadLabel,
+ nextBlockStart,
+ i,
+ hasTerms,
+ hasSubBlocks));
+
+ hasTerms = false;
+ hasSubBlocks = false;
+ nextFloorLeadLabel = suffixLeadLabel;
+ nextBlockStart = i;
+ }
+
+ lastSuffixLeadLabel = suffixLeadLabel;
+ }
+
+ if (ent.isTerm) {
+ hasTerms = true;
+ } else {
+ hasSubBlocks = true;
+ }
+ }
+
+ // Write last block, if any:
+ if (nextBlockStart < end) {
+ int itemsInBlock = end - nextBlockStart;
+ boolean isFloor = itemsInBlock < count;
+ newBlocks.add(
+ writeBlock(
+ prefixLength,
+ isFloor,
+ nextFloorLeadLabel,
+ nextBlockStart,
+ end,
+ hasTerms,
+ hasSubBlocks));
+ }
+
+ assert newBlocks.isEmpty() == false;
+
+ PendingBlock firstBlock = newBlocks.get(0);
+
+ assert firstBlock.isFloor || newBlocks.size() == 1;
+
+ firstBlock.compileIndex(newBlocks, scratchBytes, scratchIntsRef);
+
+ // Remove slice from the top of the pending stack, that we just wrote:
+ pending.subList(pending.size() - count, pending.size()).clear();
+
+ // Append new block
+ pending.add(firstBlock);
+
+ newBlocks.clear();
+ }
+
+ private boolean allEqual(byte[] b, int startOffset, int endOffset, byte value) {
+ Objects.checkFromToIndex(startOffset, endOffset, b.length);
+ for (int i = startOffset; i < endOffset; ++i) {
+ if (b[i] != value) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ /**
+ * Writes the specified slice (start is inclusive, end is exclusive) from pending stack as a new
+ * block. If isFloor is true, there were too many (more than maxItemsInBlock) entries sharing
+ * the same prefix, and so we broke it into multiple floor blocks where we record the starting
+ * label of the suffix of each floor block.
+ */
+ private PendingBlock writeBlock(
+ int prefixLength,
+ boolean isFloor,
+ int floorLeadLabel,
+ int start,
+ int end,
+ boolean hasTerms,
+ boolean hasSubBlocks)
+ throws IOException {
+
+ assert end > start;
+
+ long startFP = termsOut.getFilePointer();
+
+ boolean hasFloorLeadLabel = isFloor && floorLeadLabel != -1;
+
+ final BytesRef prefix = new BytesRef(prefixLength + (hasFloorLeadLabel ? 1 : 0));
+ System.arraycopy(lastTerm.get().bytes, 0, prefix.bytes, 0, prefixLength);
+ prefix.length = prefixLength;
+
+ // if (DEBUG2) System.out.println(" writeBlock field=" + fieldInfo.name + " prefix=" +
+ // brToString(prefix) + " fp=" + startFP + " isFloor=" + isFloor + " isLastInFloor=" + (end ==
+ // pending.size()) + " floorLeadLabel=" + floorLeadLabel + " start=" + start + " end=" + end +
+ // " hasTerms=" + hasTerms + " hasSubBlocks=" + hasSubBlocks);
+
+ // Write block header:
+ int numEntries = end - start;
+ int code = numEntries << 1;
+ if (end == pending.size()) {
+ // Last block:
+ code |= 1;
+ }
+ termsOut.writeVInt(code);
+
+ /*
+ if (DEBUG) {
+ System.out.println(" writeBlock " + (isFloor ? "(floor) " : "") + "seg=" + segment + " pending.size()=" + pending.size() + " prefixLength=" + prefixLength + " indexPrefix=" + brToString(prefix) + " entCount=" + (end-start+1) + " startFP=" + startFP + (isFloor ? (" floorLeadLabel=" + Integer.toHexString(floorLeadLabel)) : ""));
+ }
+ */
+
+ // 1st pass: pack term suffix bytes into byte[] blob
+ // TODO: cutover to bulk int codec... simple64?
+
+ // We optimize the leaf block case (block has only terms), writing a more
+ // compact format in this case:
+ boolean isLeafBlock = hasSubBlocks == false;
+
+ // System.out.println(" isLeaf=" + isLeafBlock);
+
+ final List> subIndices;
+
+ boolean absolute = true;
+
+ if (isLeafBlock) {
+ // Block contains only ordinary terms:
+ subIndices = null;
+ StatsWriter statsWriter =
+ new StatsWriter(this.statsWriter, fieldInfo.getIndexOptions() != IndexOptions.DOCS);
+ for (int i = start; i < end; i++) {
+ PendingEntry ent = pending.get(i);
+ assert ent.isTerm : "i=" + i;
+
+ PendingTerm term = (PendingTerm) ent;
+
+ assert StringHelper.startsWith(term.termBytes, prefix)
+ : "term.term=" + new BytesRef(term.termBytes) + " prefix=" + prefix;
+ BlockTermState state = term.state;
+ final int suffix = term.termBytes.length - prefixLength;
+ // if (DEBUG2) {
+ // BytesRef suffixBytes = new BytesRef(suffix);
+ // System.arraycopy(term.termBytes, prefixLength, suffixBytes.bytes, 0, suffix);
+ // suffixBytes.length = suffix;
+ // System.out.println(" write term suffix=" + brToString(suffixBytes));
+ // }
+
+ // For leaf block we write suffix straight
+ suffixLengthsWriter.writeVInt(suffix);
+ suffixWriter.append(term.termBytes, prefixLength, suffix);
+ assert floorLeadLabel == -1 || (term.termBytes[prefixLength] & 0xff) >= floorLeadLabel;
+
+ // Write term stats, to separate byte[] blob:
+ statsWriter.add(state.docFreq, state.totalTermFreq);
+
+ // Write term meta data
+ postingsWriter.encodeTerm(metaWriter, fieldInfo, state, absolute);
+ absolute = false;
+ }
+ statsWriter.finish();
+ } else {
+ // Block has at least one prefix term or a sub block:
+ subIndices = new ArrayList<>();
+ StatsWriter statsWriter =
+ new StatsWriter(this.statsWriter, fieldInfo.getIndexOptions() != IndexOptions.DOCS);
+ for (int i = start; i < end; i++) {
+ PendingEntry ent = pending.get(i);
+ if (ent.isTerm) {
+ PendingTerm term = (PendingTerm) ent;
+
+ assert StringHelper.startsWith(term.termBytes, prefix)
+ : "term.term=" + new BytesRef(term.termBytes) + " prefix=" + prefix;
+ BlockTermState state = term.state;
+ final int suffix = term.termBytes.length - prefixLength;
+ // if (DEBUG2) {
+ // BytesRef suffixBytes = new BytesRef(suffix);
+ // System.arraycopy(term.termBytes, prefixLength, suffixBytes.bytes, 0, suffix);
+ // suffixBytes.length = suffix;
+ // System.out.println(" write term suffix=" + brToString(suffixBytes));
+ // }
+
+ // For non-leaf block we borrow 1 bit to record
+ // if entry is term or sub-block, and 1 bit to record if
+ // it's a prefix term. Terms cannot be larger than ~32 KB
+ // so we won't run out of bits:
+
+ suffixLengthsWriter.writeVInt(suffix << 1);
+ suffixWriter.append(term.termBytes, prefixLength, suffix);
+
+ // Write term stats, to separate byte[] blob:
+ statsWriter.add(state.docFreq, state.totalTermFreq);
+
+ // TODO: now that terms dict "sees" these longs,
+ // we can explore better column-stride encodings
+ // to encode all long[0]s for this block at
+ // once, all long[1]s, etc., e.g. using
+ // Simple64. Alternatively, we could interleave
+ // stats + meta ... no reason to have them
+ // separate anymore:
+
+ // Write term meta data
+ postingsWriter.encodeTerm(metaWriter, fieldInfo, state, absolute);
+ absolute = false;
+ } else {
+ PendingBlock block = (PendingBlock) ent;
+ assert StringHelper.startsWith(block.prefix, prefix);
+ final int suffix = block.prefix.length - prefixLength;
+ assert StringHelper.startsWith(block.prefix, prefix);
+
+ assert suffix > 0;
+
+ // For non-leaf block we borrow 1 bit to record
+ // if entry is term or sub-block:f
+ suffixLengthsWriter.writeVInt((suffix << 1) | 1);
+ suffixWriter.append(block.prefix.bytes, prefixLength, suffix);
+
+ // if (DEBUG2) {
+ // BytesRef suffixBytes = new BytesRef(suffix);
+ // System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
+ // suffixBytes.length = suffix;
+ // System.out.println(" write sub-block suffix=" + brToString(suffixBytes) + "
+ // subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor);
+ // }
+
+ assert floorLeadLabel == -1
+ || (block.prefix.bytes[prefixLength] & 0xff) >= floorLeadLabel
+ : "floorLeadLabel="
+ + floorLeadLabel
+ + " suffixLead="
+ + (block.prefix.bytes[prefixLength] & 0xff);
+ assert block.fp < startFP;
+
+ suffixLengthsWriter.writeVLong(startFP - block.fp);
+ subIndices.add(block.index);
+ }
+ }
+ statsWriter.finish();
+
+ assert subIndices.size() != 0;
+ }
+
+ // Write suffixes byte[] blob to terms dict output, either uncompressed, compressed with LZ4
+ // or with LowercaseAsciiCompression.
+ CompressionAlgorithm compressionAlg = CompressionAlgorithm.NO_COMPRESSION;
+ // If there are 2 suffix bytes or less per term, then we don't bother compressing as suffix
+ // are unlikely what
+ // makes the terms dictionary large, and it also tends to be frequently the case for dense IDs
+ // like
+ // auto-increment IDs, so not compressing in that case helps not hurt ID lookups by too much.
+ // We also only start compressing when the prefix length is greater than 2 since blocks whose
+ // prefix length is
+ // 1 or 2 always all get visited when running a fuzzy query whose max number of edits is 2.
+ if (suffixWriter.length() > 2L * numEntries && prefixLength > 2) {
+ // LZ4 inserts references whenever it sees duplicate strings of 4 chars or more, so only try
+ // it out if the
+ // average suffix length is greater than 6.
+ if (suffixWriter.length() > 6L * numEntries) {
+ LZ4.compress(
+ suffixWriter.bytes(), 0, suffixWriter.length(), spareWriter, compressionHashTable);
+ if (spareWriter.size() < suffixWriter.length() - (suffixWriter.length() >>> 2)) {
+ // LZ4 saved more than 25%, go for it
+ compressionAlg = CompressionAlgorithm.LZ4;
+ }
+ }
+ if (compressionAlg == CompressionAlgorithm.NO_COMPRESSION) {
+ spareWriter.reset();
+ if (spareBytes.length < suffixWriter.length()) {
+ spareBytes = new byte[ArrayUtil.oversize(suffixWriter.length(), 1)];
+ }
+ if (LowercaseAsciiCompression.compress(
+ suffixWriter.bytes(), suffixWriter.length(), spareBytes, spareWriter)) {
+ compressionAlg = CompressionAlgorithm.LOWERCASE_ASCII;
+ }
+ }
+ }
+ long token = ((long) suffixWriter.length()) << 3;
+ if (isLeafBlock) {
+ token |= 0x04;
+ }
+ token |= compressionAlg.code;
+ termsOut.writeVLong(token);
+ if (compressionAlg == CompressionAlgorithm.NO_COMPRESSION) {
+ termsOut.writeBytes(suffixWriter.bytes(), suffixWriter.length());
+ } else {
+ spareWriter.copyTo(termsOut);
+ }
+ suffixWriter.setLength(0);
+ spareWriter.reset();
+
+ // Write suffix lengths
+ final int numSuffixBytes = Math.toIntExact(suffixLengthsWriter.size());
+ spareBytes = ArrayUtil.grow(spareBytes, numSuffixBytes);
+ suffixLengthsWriter.copyTo(new ByteArrayDataOutput(spareBytes));
+ suffixLengthsWriter.reset();
+ if (allEqual(spareBytes, 1, numSuffixBytes, spareBytes[0])) {
+ // Structured fields like IDs often have most values of the same length
+ termsOut.writeVInt((numSuffixBytes << 1) | 1);
+ termsOut.writeByte(spareBytes[0]);
+ } else {
+ termsOut.writeVInt(numSuffixBytes << 1);
+ termsOut.writeBytes(spareBytes, numSuffixBytes);
+ }
+
+ // Stats
+ final int numStatsBytes = Math.toIntExact(statsWriter.size());
+ termsOut.writeVInt(numStatsBytes);
+ statsWriter.copyTo(termsOut);
+ statsWriter.reset();
+
+ // Write term meta data byte[] blob
+ termsOut.writeVInt((int) metaWriter.size());
+ metaWriter.copyTo(termsOut);
+ metaWriter.reset();
+
+ // if (DEBUG) {
+ // System.out.println(" fpEnd=" + out.getFilePointer());
+ // }
+
+ if (hasFloorLeadLabel) {
+ // We already allocated to length+1 above:
+ prefix.bytes[prefix.length++] = (byte) floorLeadLabel;
+ }
+
+ return new PendingBlock(prefix, startFP, hasTerms, isFloor, floorLeadLabel, subIndices);
+ }
+
+ TermsWriter(FieldInfo fieldInfo) {
+ this.fieldInfo = fieldInfo;
+ assert fieldInfo.getIndexOptions() != IndexOptions.NONE;
+ docsSeen = new FixedBitSet(maxDoc);
+ postingsWriter.setField(fieldInfo);
+ }
+
+ /** Writes one term's worth of postings. */
+ public void write(BytesRef text, TermsEnum termsEnum, NormsProducer norms) throws IOException {
+ /*
+ if (DEBUG) {
+ int[] tmp = new int[lastTerm.length];
+ System.arraycopy(prefixStarts, 0, tmp, 0, tmp.length);
+ System.out.println("BTTW: write term=" + brToString(text) + " prefixStarts=" + Arrays.toString(tmp) + " pending.size()=" + pending.size());
+ }
+ */
+
+ BlockTermState state = postingsWriter.writeTerm(text, termsEnum, docsSeen, norms);
+ if (state != null) {
+
+ assert state.docFreq != 0;
+ assert fieldInfo.getIndexOptions() == IndexOptions.DOCS
+ || state.totalTermFreq >= state.docFreq
+ : "postingsWriter=" + postingsWriter;
+ pushTerm(text);
+
+ PendingTerm term = new PendingTerm(text, state);
+ pending.add(term);
+ // if (DEBUG) System.out.println(" add pending term = " + text + " pending.size()=" +
+ // pending.size());
+
+ sumDocFreq += state.docFreq;
+ sumTotalTermFreq += state.totalTermFreq;
+ numTerms++;
+ if (firstPendingTerm == null) {
+ firstPendingTerm = term;
+ }
+ lastPendingTerm = term;
+ }
+ }
+
+ /** Pushes the new term to the top of the stack, and writes new blocks. */
+ private void pushTerm(BytesRef text) throws IOException {
+ // Find common prefix between last term and current term:
+ int prefixLength =
+ Arrays.mismatch(
+ lastTerm.bytes(),
+ 0,
+ lastTerm.length(),
+ text.bytes,
+ text.offset,
+ text.offset + text.length);
+ if (prefixLength == -1) { // Only happens for the first term, if it is empty
+ assert lastTerm.length() == 0;
+ prefixLength = 0;
+ }
+
+ // if (DEBUG) System.out.println(" shared=" + pos + " lastTerm.length=" + lastTerm.length);
+
+ // Close the "abandoned" suffix now:
+ for (int i = lastTerm.length() - 1; i >= prefixLength; i--) {
+
+ // How many items on top of the stack share the current suffix
+ // we are closing:
+ int prefixTopSize = pending.size() - prefixStarts[i];
+ if (prefixTopSize >= minItemsInBlock) {
+ // if (DEBUG) System.out.println("pushTerm i=" + i + " prefixTopSize=" + prefixTopSize + "
+ // minItemsInBlock=" + minItemsInBlock);
+ writeBlocks(i + 1, prefixTopSize);
+ prefixStarts[i] -= prefixTopSize - 1;
+ }
+ }
+
+ if (prefixStarts.length < text.length) {
+ prefixStarts = ArrayUtil.grow(prefixStarts, text.length);
+ }
+
+ // Init new tail:
+ for (int i = prefixLength; i < text.length; i++) {
+ prefixStarts[i] = pending.size();
+ }
+
+ lastTerm.copyBytes(text);
+ }
+
+ // Finishes all terms in this field
+ public void finish() throws IOException {
+ if (numTerms > 0) {
+ // if (DEBUG) System.out.println("BTTW: finish prefixStarts=" +
+ // Arrays.toString(prefixStarts));
+
+ // Add empty term to force closing of all final blocks:
+ pushTerm(new BytesRef());
+
+ // TODO: if pending.size() is already 1 with a non-zero prefix length
+ // we can save writing a "degenerate" root block, but we have to
+ // fix all the places that assume the root block's prefix is the empty string:
+ pushTerm(new BytesRef());
+ writeBlocks(0, pending.size());
+
+ // We better have one final "root" block:
+ assert pending.size() == 1 && !pending.get(0).isTerm
+ : "pending.size()=" + pending.size() + " pending=" + pending;
+ final PendingBlock root = (PendingBlock) pending.get(0);
+ assert root.prefix.length == 0;
+ assert root.index.getEmptyOutput() != null;
+
+ // Write FST to index
+ indexStartFP = indexOut.getFilePointer();
+ root.index.save(indexOut, indexOut);
+ // System.out.println(" write FST " + indexStartFP + " field=" + fieldInfo.name);
+
+ /*
+ if (DEBUG) {
+ final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
+ Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
+ Util.toDot(root.index, w, false, false);
+ System.out.println("SAVED to " + dotFileName);
+ w.close();
+ }
+ */
+ assert firstPendingTerm != null;
+ BytesRef minTerm = new BytesRef(firstPendingTerm.termBytes);
+
+ assert lastPendingTerm != null;
+ BytesRef maxTerm = new BytesRef(lastPendingTerm.termBytes);
+
+ fields.add(
+ new FieldMetaData(
+ fieldInfo,
+ ((PendingBlock) pending.get(0)).index.getEmptyOutput(),
+ numTerms,
+ indexStartFP,
+ sumTotalTermFreq,
+ sumDocFreq,
+ docsSeen.cardinality(),
+ minTerm,
+ maxTerm));
+ } else {
+ assert sumTotalTermFreq == 0
+ || fieldInfo.getIndexOptions() == IndexOptions.DOCS && sumTotalTermFreq == -1;
+ assert sumDocFreq == 0;
+ assert docsSeen.cardinality() == 0;
+ }
+ }
+
+ private final ByteBuffersDataOutput suffixLengthsWriter =
+ ByteBuffersDataOutput.newResettableInstance();
+ private final BytesRefBuilder suffixWriter = new BytesRefBuilder();
+ private final ByteBuffersDataOutput statsWriter = ByteBuffersDataOutput.newResettableInstance();
+ private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
+ private final ByteBuffersDataOutput spareWriter = ByteBuffersDataOutput.newResettableInstance();
+ private byte[] spareBytes = BytesRef.EMPTY_BYTES;
+ private final LZ4.HighCompressionHashTable compressionHashTable =
+ new LZ4.HighCompressionHashTable();
+ }
+
+ private boolean closed;
+
+ @Override
+ public void close() throws IOException {
+ if (closed) {
+ return;
+ }
+ closed = true;
+
+ boolean success = false;
+ try {
+
+ final long dirStart = termsOut.getFilePointer();
+ final long indexDirStart = indexOut.getFilePointer();
+
+ termsOut.writeVInt(fields.size());
+
+ for (FieldMetaData field : fields) {
+ // System.out.println(" field " + field.fieldInfo.name + " " + field.numTerms + " terms");
+ termsOut.writeVInt(field.fieldInfo.number);
+ assert field.numTerms > 0;
+ termsOut.writeVLong(field.numTerms);
+ termsOut.writeVInt(field.rootCode.length);
+ termsOut.writeBytes(field.rootCode.bytes, field.rootCode.offset, field.rootCode.length);
+ assert field.fieldInfo.getIndexOptions() != IndexOptions.NONE;
+ if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS) {
+ termsOut.writeVLong(field.sumTotalTermFreq);
+ }
+ termsOut.writeVLong(field.sumDocFreq);
+ termsOut.writeVInt(field.docCount);
+ indexOut.writeVLong(field.indexStartFP);
+ writeBytesRef(termsOut, field.minTerm);
+ writeBytesRef(termsOut, field.maxTerm);
+ }
+ writeTrailer(termsOut, dirStart);
+ CodecUtil.writeFooter(termsOut);
+ writeIndexTrailer(indexOut, indexDirStart);
+ CodecUtil.writeFooter(indexOut);
+ success = true;
+ } finally {
+ if (success) {
+ IOUtils.close(termsOut, indexOut, postingsWriter);
+ } else {
+ IOUtils.closeWhileHandlingException(termsOut, indexOut, postingsWriter);
+ }
+ }
+ }
+
+ private static void writeBytesRef(IndexOutput out, BytesRef bytes) throws IOException {
+ out.writeVInt(bytes.length);
+ out.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+ }
+}
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/Lucene50RWPostingsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/Lucene50RWPostingsFormat.java
index 7b5f0b3ebc2..fb042a1a9ad 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/Lucene50RWPostingsFormat.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/Lucene50RWPostingsFormat.java
@@ -17,7 +17,7 @@
package org.apache.lucene.backward_codecs.lucene50;
import java.io.IOException;
-import org.apache.lucene.backward_codecs.lucene40.blocktree.Lucene40BlockTreeTermsWriter;
+import org.apache.lucene.backward_codecs.lucene40.blocktree.Lucene40BlockTreeTermsWriterV5;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.SegmentWriteState;
@@ -31,11 +31,11 @@ public class Lucene50RWPostingsFormat extends Lucene50PostingsFormat {
boolean success = false;
try {
FieldsConsumer ret =
- new Lucene40BlockTreeTermsWriter(
+ new Lucene40BlockTreeTermsWriterV5(
state,
postingsWriter,
- Lucene40BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE,
- Lucene40BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
+ Lucene40BlockTreeTermsWriterV5.DEFAULT_MIN_BLOCK_SIZE,
+ Lucene40BlockTreeTermsWriterV5.DEFAULT_MAX_BLOCK_SIZE);
success = true;
return ret;
} finally {
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestAncientIndicesCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestAncientIndicesCompatibility.java
index 938d0de6b90..88adfadf1c8 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestAncientIndicesCompatibility.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/TestAncientIndicesCompatibility.java
@@ -196,6 +196,7 @@ public class TestAncientIndicesCompatibility extends LuceneTestCase {
ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
CheckIndex checker = new CheckIndex(dir);
checker.setInfoStream(new PrintStream(bos, false, UTF_8));
+ checker.setLevel(CheckIndex.Level.MIN_LEVEL_FOR_INTEGRITY_CHECKS);
CheckIndex.Status indexStatus = checker.checkIndex();
if (version.startsWith("8.")) {
assertTrue(indexStatus.clean);
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.9.11.1-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.9.11.1-cfs.zip
new file mode 100644
index 00000000000..51ad7e7997c
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.9.11.1-cfs.zip differ
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.9.11.1-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.9.11.1-nocfs.zip
new file mode 100644
index 00000000000..f14d925827b
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/index.9.11.1-nocfs.zip differ
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/int8_hnsw.9.11.1.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/int8_hnsw.9.11.1.zip
new file mode 100644
index 00000000000..c4bb86b5f1b
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/int8_hnsw.9.11.1.zip differ
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.9.11.1.zip b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.9.11.1.zip
new file mode 100644
index 00000000000..4ccd9a76b00
Binary files /dev/null and b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/sorted.9.11.1.zip differ
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/versions.txt b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/versions.txt
index 3b820b6fc0f..4572b6fadfe 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/versions.txt
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_index/versions.txt
@@ -40,3 +40,4 @@
9.9.2
9.10.0
9.11.0
+9.11.1
diff --git a/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/HammingDistanceBenchmark.java b/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/HammingDistanceBenchmark.java
new file mode 100644
index 00000000000..943d5961820
--- /dev/null
+++ b/lucene/benchmark-jmh/src/java/org/apache/lucene/benchmark/jmh/HammingDistanceBenchmark.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.benchmark.jmh;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import org.apache.lucene.util.VectorUtil;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+
+@Fork(1)
+@Warmup(iterations = 3, time = 3)
+@Measurement(iterations = 5, time = 3)
+@BenchmarkMode(Mode.Throughput)
+@OutputTimeUnit(TimeUnit.SECONDS)
+@State(Scope.Benchmark)
+public class HammingDistanceBenchmark {
+ @Param({"1000000"})
+ int nb = 1_000_000;
+
+ @Param({"1024"})
+ int dims = 1024;
+
+ byte[][] xb;
+ byte[] xq;
+
+ @Setup
+ public void setup() throws IOException {
+ Random rand = new Random();
+ this.xb = new byte[nb][dims / 8];
+ for (int i = 0; i < nb; i++) {
+ for (int j = 0; j < dims / 8; j++) {
+ xb[i][j] = (byte) rand.nextInt(0, 255);
+ }
+ }
+ this.xq = new byte[dims / 8];
+ for (int i = 0; i < xq.length; i++) {
+ xq[i] = (byte) rand.nextInt(0, 255);
+ }
+ }
+
+ @Benchmark
+ public int xorBitCount() {
+ int tot = 0;
+ for (int i = 0; i < nb; i++) {
+ tot += VectorUtil.xorBitCount(xb[i], xq);
+ }
+ return tot;
+ }
+}
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java
index 2c908fcabe3..1daa1761fd8 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java
@@ -43,6 +43,7 @@ import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOBooleanSupplier;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.automaton.CompiledAutomaton;
@@ -315,12 +316,21 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
}
@Override
- public boolean seekExact(BytesRef text) throws IOException {
+ public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
// The magical fail-fast speed up that is the entire point of all of
// this code - save a disk seek if there is a match on an in-memory
// structure
// that may occasionally give a false positive but guaranteed no false
// negatives
+ if (filter.contains(text) == ContainsResult.NO) {
+ return null;
+ }
+ return delegate().prepareSeekExact(text);
+ }
+
+ @Override
+ public boolean seekExact(BytesRef text) throws IOException {
+ // See #prepareSeekExact
if (filter.contains(text) == ContainsResult.NO) {
return false;
}
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java
index 6f4e2db0d6e..d500b75eaab 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsReader.java
@@ -195,9 +195,10 @@ public class FSTTermsReader extends FieldsProducer {
this.sumTotalTermFreq = sumTotalTermFreq;
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
- OffHeapFSTStore offHeapFSTStore = new OffHeapFSTStore();
FSTTermOutputs outputs = new FSTTermOutputs(fieldInfo);
- this.dict = new FST<>(FST.readMetadata(in, outputs), in, offHeapFSTStore);
+ final var fstMetadata = FST.readMetadata(in, outputs);
+ OffHeapFSTStore offHeapFSTStore = new OffHeapFSTStore(in, in.getFilePointer(), fstMetadata);
+ this.dict = FST.fromFSTReader(fstMetadata, offHeapFSTStore);
in.skipBytes(offHeapFSTStore.size());
}
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
index f58ff0873ca..435c2f73fdf 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
@@ -829,7 +829,7 @@ class SimpleTextDocValuesReader extends DocValuesProducer {
clone.seek(0);
// checksum is fixed-width encoded with 20 bytes, plus 1 byte for newline (the space is included
// in SimpleTextUtil.CHECKSUM):
- long footerStartPos = data.length() - (SimpleTextUtil.CHECKSUM.length + 21);
+ long footerStartPos = clone.length() - (SimpleTextUtil.CHECKSUM.length + 21);
ChecksumIndexInput input = new BufferedChecksumIndexInput(clone);
while (true) {
SimpleTextUtil.readLine(input, scratch);
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
index be0e98f906a..5d6c41663ca 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPointsReader.java
@@ -227,7 +227,7 @@ class SimpleTextPointsReader extends PointsReader {
// checksum is fixed-width encoded with 20 bytes, plus 1 byte for newline (the space is included
// in SimpleTextUtil.CHECKSUM):
- long footerStartPos = dataIn.length() - (SimpleTextUtil.CHECKSUM.length + 21);
+ long footerStartPos = clone.length() - (SimpleTextUtil.CHECKSUM.length + 21);
ChecksumIndexInput input = new BufferedChecksumIndexInput(clone);
while (true) {
SimpleTextUtil.readLine(input, scratch);
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/FSTDictionary.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/FSTDictionary.java
index a73fef410dd..ced1dadab6f 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/FSTDictionary.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/FSTDictionary.java
@@ -90,10 +90,15 @@ public class FSTDictionary implements IndexDictionary {
}
PositiveIntOutputs fstOutputs = PositiveIntOutputs.getSingleton();
FST.FSTMetadata metadata = FST.readMetadata(fstDataInput, fstOutputs);
- FST fst =
- isFSTOnHeap
- ? new FST<>(metadata, fstDataInput)
- : new FST<>(metadata, fstDataInput, new OffHeapFSTStore());
+ FST fst;
+ if (isFSTOnHeap) {
+ fst = new FST<>(metadata, fstDataInput);
+ } else {
+ final IndexInput indexInput = (IndexInput) fstDataInput;
+ fst =
+ FST.fromFSTReader(
+ metadata, new OffHeapFSTStore(indexInput, indexInput.getFilePointer(), metadata));
+ }
return new FSTDictionary(fst);
}
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STMergingTermsEnum.java b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STMergingTermsEnum.java
index 7a772c38908..95d63b14e3d 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STMergingTermsEnum.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/sharedterms/STMergingTermsEnum.java
@@ -20,11 +20,11 @@ package org.apache.lucene.codecs.uniformsplit.sharedterms;
import java.io.IOException;
import java.util.List;
import java.util.RandomAccess;
+import org.apache.lucene.index.BaseTermsEnum;
import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.TermState;
-import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
@@ -34,7 +34,7 @@ import org.apache.lucene.util.BytesRef;
*
* @lucene.experimental
*/
-class STMergingTermsEnum extends TermsEnum {
+class STMergingTermsEnum extends BaseTermsEnum {
protected final String fieldName;
protected final MultiSegmentsPostingsEnum multiPostingsEnum;
@@ -63,11 +63,6 @@ class STMergingTermsEnum extends TermsEnum {
throw new UnsupportedOperationException();
}
- @Override
- public boolean seekExact(BytesRef text) throws IOException {
- throw new UnsupportedOperationException();
- }
-
@Override
public SeekStatus seekCeil(BytesRef text) {
throw new UnsupportedOperationException();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
index 0d171812fae..cbb906788e5 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
@@ -23,6 +23,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
+import org.apache.lucene.index.BaseTermsEnum;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.DocIDMerger;
import org.apache.lucene.index.DocValues;
@@ -498,7 +499,7 @@ public abstract class DocValuesConsumer implements Closeable {
* {@link SortedDocValues#lookupOrd(int)} or {@link SortedSetDocValues#lookupOrd(long)} on every
* call to {@link TermsEnum#next()}.
*/
- private static class MergedTermsEnum extends TermsEnum {
+ private static class MergedTermsEnum extends BaseTermsEnum {
private final TermsEnum[] subs;
private final OrdinalMap ordinalMap;
@@ -542,11 +543,6 @@ public abstract class DocValuesConsumer implements Closeable {
throw new UnsupportedOperationException();
}
- @Override
- public boolean seekExact(BytesRef text) throws IOException {
- throw new UnsupportedOperationException();
- }
-
@Override
public SeekStatus seekCeil(BytesRef text) throws IOException {
throw new UnsupportedOperationException();
@@ -557,11 +553,6 @@ public abstract class DocValuesConsumer implements Closeable {
throw new UnsupportedOperationException();
}
- @Override
- public void seekExact(BytesRef term, TermState state) throws IOException {
- throw new UnsupportedOperationException();
- }
-
@Override
public int docFreq() throws IOException {
throw new UnsupportedOperationException();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java
index 8ae86d4c807..053ab893df1 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsWriter.java
@@ -20,14 +20,18 @@ package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.List;
+import java.util.Objects;
import org.apache.lucene.index.ByteVectorValues;
import org.apache.lucene.index.DocIDMerger;
+import org.apache.lucene.index.DocsWithFieldSet;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FloatVectorValues;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.Sorter;
import org.apache.lucene.index.VectorEncoding;
+import org.apache.lucene.internal.hppc.IntIntHashMap;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.VectorScorer;
import org.apache.lucene.util.Accountable;
@@ -139,6 +143,60 @@ public abstract class KnnVectorsWriter implements Accountable, Closeable {
}
}
+ /**
+ * Given old doc ids and an id mapping, maps old ordinal to new ordinal. Note: this method return
+ * nothing and output are written to parameters
+ *
+ * @param oldDocIds the old or current document ordinals. Must not be null.
+ * @param sortMap the document sorting map for how to make the new ordinals. Must not be null.
+ * @param old2NewOrd int[] maps from old ord to new ord
+ * @param new2OldOrd int[] maps from new ord to old ord
+ * @param newDocsWithField set of new doc ids which has the value
+ */
+ public static void mapOldOrdToNewOrd(
+ DocsWithFieldSet oldDocIds,
+ Sorter.DocMap sortMap,
+ int[] old2NewOrd,
+ int[] new2OldOrd,
+ DocsWithFieldSet newDocsWithField)
+ throws IOException {
+ // TODO: a similar function exists in IncrementalHnswGraphMerger#getNewOrdMapping
+ // maybe we can do a further refactoring
+ Objects.requireNonNull(oldDocIds);
+ Objects.requireNonNull(sortMap);
+ assert (old2NewOrd != null || new2OldOrd != null || newDocsWithField != null);
+ assert (old2NewOrd == null || old2NewOrd.length == oldDocIds.cardinality());
+ assert (new2OldOrd == null || new2OldOrd.length == oldDocIds.cardinality());
+ IntIntHashMap newIdToOldOrd = new IntIntHashMap();
+ DocIdSetIterator iterator = oldDocIds.iterator();
+ int[] newDocIds = new int[oldDocIds.cardinality()];
+ int oldOrd = 0;
+ for (int oldDocId = iterator.nextDoc();
+ oldDocId != DocIdSetIterator.NO_MORE_DOCS;
+ oldDocId = iterator.nextDoc()) {
+ int newId = sortMap.oldToNew(oldDocId);
+ newIdToOldOrd.put(newId, oldOrd);
+ newDocIds[oldOrd] = newId;
+ oldOrd++;
+ }
+
+ Arrays.sort(newDocIds);
+ int newOrd = 0;
+ for (int newDocId : newDocIds) {
+ int currOldOrd = newIdToOldOrd.get(newDocId);
+ if (old2NewOrd != null) {
+ old2NewOrd[currOldOrd] = newOrd;
+ }
+ if (new2OldOrd != null) {
+ new2OldOrd[newOrd] = currOldOrd;
+ }
+ if (newDocsWithField != null) {
+ newDocsWithField.add(newDocId);
+ }
+ newOrd++;
+ }
+ }
+
/** View over multiple vector values supporting iterator-style access via DocIdMerger. */
public static final class MergedVectorValues {
private MergedVectorValues() {}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/hnsw/FlatFieldVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/hnsw/FlatFieldVectorsWriter.java
index 313ccccd4eb..fc71bb729db 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/hnsw/FlatFieldVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/hnsw/FlatFieldVectorsWriter.java
@@ -17,7 +17,10 @@
package org.apache.lucene.codecs.hnsw;
+import java.io.IOException;
+import java.util.List;
import org.apache.lucene.codecs.KnnFieldVectorsWriter;
+import org.apache.lucene.index.DocsWithFieldSet;
/**
* Vectors' writer for a field
@@ -26,20 +29,25 @@ import org.apache.lucene.codecs.KnnFieldVectorsWriter;
* @lucene.experimental
*/
public abstract class FlatFieldVectorsWriter extends KnnFieldVectorsWriter {
-
/**
- * The delegate to write to, can be null When non-null, all vectors seen should be written to the
- * delegate along with being written to the flat vectors.
+ * @return a list of vectors to be written
*/
- protected final KnnFieldVectorsWriter indexingDelegate;
+ public abstract List getVectors();
/**
- * Sole constructor that expects some indexingDelegate. All vectors seen should be written to the
- * delegate along with being written to the flat vectors.
+ * @return the docsWithFieldSet for the field writer
+ */
+ public abstract DocsWithFieldSet getDocsWithFieldSet();
+
+ /**
+ * indicates that this writer is done and no new vectors are allowed to be added
*
- * @param indexingDelegate the delegate to write to, can be null
+ * @throws IOException if an I/O error occurs
*/
- protected FlatFieldVectorsWriter(KnnFieldVectorsWriter indexingDelegate) {
- this.indexingDelegate = indexingDelegate;
- }
+ public abstract void finish() throws IOException;
+
+ /**
+ * @return true if the writer is done and no new vectors are allowed to be added
+ */
+ public abstract boolean isFinished();
}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/hnsw/FlatVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/hnsw/FlatVectorsWriter.java
index 3a7803011aa..37c4f546bab 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/hnsw/FlatVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/hnsw/FlatVectorsWriter.java
@@ -18,7 +18,6 @@
package org.apache.lucene.codecs.hnsw;
import java.io.IOException;
-import org.apache.lucene.codecs.KnnFieldVectorsWriter;
import org.apache.lucene.codecs.KnnVectorsWriter;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.MergeState;
@@ -46,21 +45,14 @@ public abstract class FlatVectorsWriter extends KnnVectorsWriter {
}
/**
- * Add a new field for indexing, allowing the user to provide a writer that the flat vectors
- * writer can delegate to if additional indexing logic is required.
+ * Add a new field for indexing
*
* @param fieldInfo fieldInfo of the field to add
- * @param indexWriter the writer to delegate to, can be null
* @return a writer for the field
* @throws IOException if an I/O error occurs when adding the field
*/
- public abstract FlatFieldVectorsWriter> addField(
- FieldInfo fieldInfo, KnnFieldVectorsWriter> indexWriter) throws IOException;
-
@Override
- public FlatFieldVectorsWriter> addField(FieldInfo fieldInfo) throws IOException {
- return addField(fieldInfo, null);
- }
+ public abstract FlatFieldVectorsWriter> addField(FieldInfo fieldInfo) throws IOException;
/**
* Write the field for merging, providing a scorer over the newly merged flat vectors. This way
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java
index 63e4891960c..021eacacd3b 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.lucene90;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SHIFT;
import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.NUMERIC_BLOCK_SIZE;
-import static org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat.SKIP_INDEX_INTERVAL_SIZE;
import java.io.IOException;
import java.util.Arrays;
@@ -63,10 +62,12 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
IndexOutput data, meta;
final int maxDoc;
private byte[] termsDictBuffer;
+ private final int skipIndexIntervalSize;
/** expert: Creates a new writer */
public Lucene90DocValuesConsumer(
SegmentWriteState state,
+ int skipIndexIntervalSize,
String dataCodec,
String dataExtension,
String metaCodec,
@@ -96,6 +97,7 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
state.segmentInfo.getId(),
state.segmentSuffix);
maxDoc = state.segmentInfo.maxDoc();
+ this.skipIndexIntervalSize = skipIndexIntervalSize;
success = true;
} finally {
if (!success) {
@@ -239,7 +241,7 @@ final class Lucene90DocValuesConsumer extends DocValuesConsumer {
for (int i = 0, end = values.docValueCount(); i < end; ++i) {
accumulator.accumulate(values.nextValue());
}
- if (++counter == SKIP_INDEX_INTERVAL_SIZE) {
+ if (++counter == skipIndexIntervalSize) {
globalMaxValue = Math.max(globalMaxValue, accumulator.maxValue);
globalMinValue = Math.min(globalMinValue, accumulator.minValue);
globalDocCount += accumulator.docCount;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java
index 847a5341584..0ae0a7ac2aa 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesFormat.java
@@ -138,15 +138,27 @@ import org.apache.lucene.util.packed.DirectWriter;
*/
public final class Lucene90DocValuesFormat extends DocValuesFormat {
+ private final int skipIndexIntervalSize;
+
/** Default constructor. */
public Lucene90DocValuesFormat() {
+ this(DEFAULT_SKIP_INDEX_INTERVAL_SIZE);
+ }
+
+ /** Doc values fields format with specified skipIndexIntervalSize. */
+ public Lucene90DocValuesFormat(int skipIndexIntervalSize) {
super("Lucene90");
+ if (skipIndexIntervalSize < 2) {
+ throw new IllegalArgumentException(
+ "skipIndexIntervalSize must be > 1, got [" + skipIndexIntervalSize + "]");
+ }
+ this.skipIndexIntervalSize = skipIndexIntervalSize;
}
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new Lucene90DocValuesConsumer(
- state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+ state, skipIndexIntervalSize, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
@Override
@@ -182,6 +194,5 @@ public final class Lucene90DocValuesFormat extends DocValuesFormat {
static final int TERMS_DICT_REVERSE_INDEX_SIZE = 1 << TERMS_DICT_REVERSE_INDEX_SHIFT;
static final int TERMS_DICT_REVERSE_INDEX_MASK = TERMS_DICT_REVERSE_INDEX_SIZE - 1;
- static final int SKIP_INDEX_INTERVAL_SHIFT = 12;
- static final int SKIP_INDEX_INTERVAL_SIZE = 1 << SKIP_INDEX_INTERVAL_SHIFT;
+ private static final int DEFAULT_SKIP_INDEX_INTERVAL_SIZE = 4096;
}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/FieldReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/FieldReader.java
index 6cef964a6a5..ed3e827c37c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/FieldReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/FieldReader.java
@@ -89,13 +89,8 @@ public final class FieldReader extends Terms {
readVLongOutput(new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length))
>>> Lucene90BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS;
// Initialize FST always off-heap.
- final IndexInput clone = indexIn.clone();
- clone.seek(indexStartFP);
- index =
- new FST<>(
- FST.readMetadata(metaIn, ByteSequenceOutputs.getSingleton()),
- clone,
- new OffHeapFSTStore());
+ var metadata = FST.readMetadata(metaIn, ByteSequenceOutputs.getSingleton());
+ index = FST.fromFSTReader(metadata, new OffHeapFSTStore(indexIn, indexStartFP, metadata));
/*
if (false) {
final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java
index e3389931be7..91776585407 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java
@@ -30,6 +30,7 @@ import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOBooleanSupplier;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Util;
@@ -307,15 +308,13 @@ final class SegmentTermsEnum extends BaseTermsEnum {
return true;
}
- @Override
- public boolean seekExact(BytesRef target) throws IOException {
-
+ private IOBooleanSupplier prepareSeekExact(BytesRef target, boolean prefetch) throws IOException {
if (fr.index == null) {
throw new IllegalStateException("terms index was not loaded");
}
if (fr.size() > 0 && (target.compareTo(fr.getMin()) < 0 || target.compareTo(fr.getMax()) > 0)) {
- return false;
+ return null;
}
term.grow(1 + target.length);
@@ -431,7 +430,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
// if (DEBUG) {
// System.out.println(" target is same as current; return true");
// }
- return true;
+ return () -> true;
} else {
// if (DEBUG) {
// System.out.println(" target is same as current but term doesn't exist");
@@ -501,24 +500,30 @@ final class SegmentTermsEnum extends BaseTermsEnum {
// if (DEBUG) {
// System.out.println(" FAST NOT_FOUND term=" + ToStringUtils.bytesRefToString(term));
// }
- return false;
+ return null;
}
- currentFrame.loadBlock();
-
- final SeekStatus result = currentFrame.scanToTerm(target, true);
- if (result == SeekStatus.FOUND) {
- // if (DEBUG) {
- // System.out.println(" return FOUND term=" + term.utf8ToString() + " " + term);
- // }
- return true;
- } else {
- // if (DEBUG) {
- // System.out.println(" got " + result + "; return NOT_FOUND term=" +
- // ToStringUtils.bytesRefToString(term));
- // }
- return false;
+ if (prefetch) {
+ currentFrame.prefetchBlock();
}
+
+ return () -> {
+ currentFrame.loadBlock();
+
+ final SeekStatus result = currentFrame.scanToTerm(target, true);
+ if (result == SeekStatus.FOUND) {
+ // if (DEBUG) {
+ // System.out.println(" return FOUND term=" + term.utf8ToString() + " " + term);
+ // }
+ return true;
+ } else {
+ // if (DEBUG) {
+ // System.out.println(" got " + result + "; return NOT_FOUND term=" +
+ // ToStringUtils.bytesRefToString(term));
+ // }
+ return false;
+ }
+ };
} else {
// Follow this arc
arc = nextArc;
@@ -556,25 +561,42 @@ final class SegmentTermsEnum extends BaseTermsEnum {
// if (DEBUG) {
// System.out.println(" FAST NOT_FOUND term=" + ToStringUtils.bytesRefToString(term));
// }
- return false;
+ return null;
}
- currentFrame.loadBlock();
-
- final SeekStatus result = currentFrame.scanToTerm(target, true);
- if (result == SeekStatus.FOUND) {
- // if (DEBUG) {
- // System.out.println(" return FOUND term=" + term.utf8ToString() + " " + term);
- // }
- return true;
- } else {
- // if (DEBUG) {
- // System.out.println(" got result " + result + "; return NOT_FOUND term=" +
- // term.utf8ToString());
- // }
-
- return false;
+ if (prefetch) {
+ currentFrame.prefetchBlock();
}
+
+ return () -> {
+ currentFrame.loadBlock();
+
+ final SeekStatus result = currentFrame.scanToTerm(target, true);
+ if (result == SeekStatus.FOUND) {
+ // if (DEBUG) {
+ // System.out.println(" return FOUND term=" + term.utf8ToString() + " " + term);
+ // }
+ return true;
+ } else {
+ // if (DEBUG) {
+ // System.out.println(" got result " + result + "; return NOT_FOUND term=" +
+ // term.utf8ToString());
+ // }
+
+ return false;
+ }
+ };
+ }
+
+ @Override
+ public IOBooleanSupplier prepareSeekExact(BytesRef target) throws IOException {
+ return prepareSeekExact(target, true);
+ }
+
+ @Override
+ public boolean seekExact(BytesRef target) throws IOException {
+ IOBooleanSupplier termExistsSupplier = prepareSeekExact(target, false);
+ return termExistsSupplier != null && termExistsSupplier.get();
}
@Override
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java
index c1552c8ce60..5ecbc3c173e 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java
@@ -133,6 +133,21 @@ final class SegmentTermsEnumFrame {
loadBlock();
}
+ void prefetchBlock() throws IOException {
+ if (nextEnt != -1) {
+ // Already loaded
+ return;
+ }
+
+ // Clone the IndexInput lazily, so that consumers
+ // that just pull a TermsEnum to
+ // seekExact(TermState) don't pay this cost:
+ ste.initIndexInput();
+
+ // TODO: Could we know the number of bytes to prefetch?
+ ste.in.prefetch(fp, 1);
+ }
+
/* Does initial decode of next block of terms; this
doesn't actually decode the docFreq, totalTermFreq,
postings details (frq/prx offset, etc.) metadata;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene95/OrdToDocDISIReaderConfiguration.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene95/OrdToDocDISIReaderConfiguration.java
index a13485eed5d..e4c921ddee2 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene95/OrdToDocDISIReaderConfiguration.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene95/OrdToDocDISIReaderConfiguration.java
@@ -40,7 +40,7 @@ public class OrdToDocDISIReaderConfiguration {
*
Within outputMeta the format is as follows:
*
*
- *
[int8] if equals to -2, empty - no vectory values. If equals to -1, dense – all
+ *
[int8] if equals to -2, empty - no vector values. If equals to -1, dense – all
* documents have values for a field. If equals to 0, sparse – some documents missing
* values.
*
DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput,
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99FlatVectorsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99FlatVectorsFormat.java
index 78e0cf000fa..c8ef2709db6 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99FlatVectorsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99FlatVectorsFormat.java
@@ -56,8 +56,8 @@ import org.apache.lucene.store.IndexOutput;
*
[vlong] length of this field's vectors, in bytes
*
[vint] dimension of this field's vectors
*
[int] the number of documents having values for this field
- *
[int8] if equals to -1, dense – all documents have values for a field. If equals to
- * 0, sparse – some documents missing values.
+ *
[int8] if equals to -2, empty - no vector values. If equals to -1, dense – all
+ * documents have values for a field. If equals to 0, sparse – some documents missing values.
*
DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)}
*
OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
* that only in sparse case
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99FlatVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99FlatVectorsWriter.java
index 6232489c08d..5643752796c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99FlatVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99FlatVectorsWriter.java
@@ -27,7 +27,6 @@ import java.nio.ByteOrder;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.KnnFieldVectorsWriter;
import org.apache.lucene.codecs.KnnVectorsWriter;
import org.apache.lucene.codecs.hnsw.FlatFieldVectorsWriter;
import org.apache.lucene.codecs.hnsw.FlatVectorsScorer;
@@ -44,7 +43,6 @@ import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Sorter;
import org.apache.lucene.index.VectorEncoding;
-import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
@@ -112,18 +110,12 @@ public final class Lucene99FlatVectorsWriter extends FlatVectorsWriter {
}
@Override
- public FlatFieldVectorsWriter> addField(
- FieldInfo fieldInfo, KnnFieldVectorsWriter> indexWriter) throws IOException {
- FieldWriter> newField = FieldWriter.create(fieldInfo, indexWriter);
+ public FlatFieldVectorsWriter> addField(FieldInfo fieldInfo) throws IOException {
+ FieldWriter> newField = FieldWriter.create(fieldInfo);
fields.add(newField);
return newField;
}
- @Override
- public FlatFieldVectorsWriter> addField(FieldInfo fieldInfo) throws IOException {
- return addField(fieldInfo, null);
- }
-
@Override
public void flush(int maxDoc, Sorter.DocMap sortMap) throws IOException {
for (FieldWriter> field : fields) {
@@ -132,6 +124,7 @@ public final class Lucene99FlatVectorsWriter extends FlatVectorsWriter {
} else {
writeSortingField(field, maxDoc, sortMap);
}
+ field.finish();
}
}
@@ -191,27 +184,10 @@ public final class Lucene99FlatVectorsWriter extends FlatVectorsWriter {
private void writeSortingField(FieldWriter> fieldData, int maxDoc, Sorter.DocMap sortMap)
throws IOException {
- final int[] docIdOffsets = new int[sortMap.size()];
- int offset = 1; // 0 means no vector for this (field, document)
- DocIdSetIterator iterator = fieldData.docsWithField.iterator();
- for (int docID = iterator.nextDoc();
- docID != DocIdSetIterator.NO_MORE_DOCS;
- docID = iterator.nextDoc()) {
- int newDocID = sortMap.oldToNew(docID);
- docIdOffsets[newDocID] = offset++;
- }
+ final int[] ordMap = new int[fieldData.docsWithField.cardinality()]; // new ord to old ord
+
DocsWithFieldSet newDocsWithField = new DocsWithFieldSet();
- final int[] ordMap = new int[offset - 1]; // new ord to old ord
- int ord = 0;
- int doc = 0;
- for (int docIdOffset : docIdOffsets) {
- if (docIdOffset != 0) {
- ordMap[ord] = docIdOffset - 1;
- newDocsWithField.add(doc);
- ord++;
- }
- doc++;
- }
+ mapOldOrdToNewOrd(fieldData.docsWithField, sortMap, null, ordMap, newDocsWithField);
// write vector values
long vectorDataOffset =
@@ -421,22 +397,20 @@ public final class Lucene99FlatVectorsWriter extends FlatVectorsWriter {
private final int dim;
private final DocsWithFieldSet docsWithField;
private final List vectors;
+ private boolean finished;
private int lastDocID = -1;
- @SuppressWarnings("unchecked")
- static FieldWriter> create(FieldInfo fieldInfo, KnnFieldVectorsWriter> indexWriter) {
+ static FieldWriter> create(FieldInfo fieldInfo) {
int dim = fieldInfo.getVectorDimension();
return switch (fieldInfo.getVectorEncoding()) {
- case BYTE -> new Lucene99FlatVectorsWriter.FieldWriter<>(
- fieldInfo, (KnnFieldVectorsWriter) indexWriter) {
+ case BYTE -> new Lucene99FlatVectorsWriter.FieldWriter(fieldInfo) {
@Override
public byte[] copyValue(byte[] value) {
return ArrayUtil.copyOfSubArray(value, 0, dim);
}
};
- case FLOAT32 -> new Lucene99FlatVectorsWriter.FieldWriter<>(
- fieldInfo, (KnnFieldVectorsWriter) indexWriter) {
+ case FLOAT32 -> new Lucene99FlatVectorsWriter.FieldWriter(fieldInfo) {
@Override
public float[] copyValue(float[] value) {
return ArrayUtil.copyOfSubArray(value, 0, dim);
@@ -445,8 +419,8 @@ public final class Lucene99FlatVectorsWriter extends FlatVectorsWriter {
};
}
- FieldWriter(FieldInfo fieldInfo, KnnFieldVectorsWriter indexWriter) {
- super(indexWriter);
+ FieldWriter(FieldInfo fieldInfo) {
+ super();
this.fieldInfo = fieldInfo;
this.dim = fieldInfo.getVectorDimension();
this.docsWithField = new DocsWithFieldSet();
@@ -455,6 +429,9 @@ public final class Lucene99FlatVectorsWriter extends FlatVectorsWriter {
@Override
public void addValue(int docID, T vectorValue) throws IOException {
+ if (finished) {
+ throw new IllegalStateException("already finished, cannot add more values");
+ }
if (docID == lastDocID) {
throw new IllegalArgumentException(
"VectorValuesField \""
@@ -466,17 +443,11 @@ public final class Lucene99FlatVectorsWriter extends FlatVectorsWriter {
docsWithField.add(docID);
vectors.add(copy);
lastDocID = docID;
- if (indexingDelegate != null) {
- indexingDelegate.addValue(docID, copy);
- }
}
@Override
public long ramBytesUsed() {
long size = SHALLOW_RAM_BYTES_USED;
- if (indexingDelegate != null) {
- size += indexingDelegate.ramBytesUsed();
- }
if (vectors.size() == 0) return size;
return size
+ docsWithField.ramBytesUsed()
@@ -486,6 +457,29 @@ public final class Lucene99FlatVectorsWriter extends FlatVectorsWriter {
* fieldInfo.getVectorDimension()
* fieldInfo.getVectorEncoding().byteSize;
}
+
+ @Override
+ public List getVectors() {
+ return vectors;
+ }
+
+ @Override
+ public DocsWithFieldSet getDocsWithFieldSet() {
+ return docsWithField;
+ }
+
+ @Override
+ public void finish() throws IOException {
+ if (finished) {
+ return;
+ }
+ this.finished = true;
+ }
+
+ @Override
+ public boolean isFinished() {
+ return finished;
+ }
}
static final class FlatCloseableRandomVectorScorerSupplier
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsFormat.java
index 3238fd1f4ae..117393706db 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsFormat.java
@@ -24,14 +24,11 @@ import org.apache.lucene.codecs.KnnVectorsReader;
import org.apache.lucene.codecs.KnnVectorsWriter;
import org.apache.lucene.codecs.hnsw.FlatVectorScorerUtil;
import org.apache.lucene.codecs.hnsw.FlatVectorsFormat;
-import org.apache.lucene.codecs.lucene90.IndexedDISI;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.MergeScheduler;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.TaskExecutor;
-import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.hnsw.HnswGraph;
/**
@@ -69,11 +66,6 @@ import org.apache.lucene.util.hnsw.HnswGraph;
*
[vlong] length of this field's index data, in bytes
*
[vint] dimension of this field's vectors
*
[int] the number of documents having values for this field
- *
[int8] if equals to -1, dense – all documents have values for a field. If equals to
- * 0, sparse – some documents missing values.
- *
DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)}
- *
OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
- * that only in sparse case
*
[vint] the maximum number of connections (neighbours) that each node can have
*
[vint] number of levels in the graph
*
Graph nodes by level. For each level
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java
index 38908efbe68..899140af93a 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsReader.java
@@ -45,6 +45,7 @@ import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RandomAccessInput;
import org.apache.lucene.store.ReadAdvice;
import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.IOSupplier;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.hnsw.HnswGraph;
@@ -248,45 +249,39 @@ public final class Lucene99HnswVectorsReader extends KnnVectorsReader
@Override
public void search(String field, float[] target, KnnCollector knnCollector, Bits acceptDocs)
throws IOException {
- FieldEntry fieldEntry = fields.get(field);
-
- if (fieldEntry.size() == 0
- || knnCollector.k() == 0
- || fieldEntry.vectorEncoding != VectorEncoding.FLOAT32) {
- return;
- }
- final RandomVectorScorer scorer = flatVectorsReader.getRandomVectorScorer(field, target);
- final KnnCollector collector =
- new OrdinalTranslatedKnnCollector(knnCollector, scorer::ordToDoc);
- final Bits acceptedOrds = scorer.getAcceptOrds(acceptDocs);
- if (knnCollector.k() < scorer.maxOrd()) {
- HnswGraphSearcher.search(scorer, collector, getGraph(fieldEntry), acceptedOrds);
- } else {
- // if k is larger than the number of vectors, we can just iterate over all vectors
- // and collect them
- for (int i = 0; i < scorer.maxOrd(); i++) {
- if (acceptedOrds == null || acceptedOrds.get(i)) {
- if (knnCollector.earlyTerminated()) {
- break;
- }
- knnCollector.incVisitedCount(1);
- knnCollector.collect(scorer.ordToDoc(i), scorer.score(i));
- }
- }
- }
+ search(
+ fields.get(field),
+ knnCollector,
+ acceptDocs,
+ VectorEncoding.FLOAT32,
+ () -> flatVectorsReader.getRandomVectorScorer(field, target));
}
@Override
public void search(String field, byte[] target, KnnCollector knnCollector, Bits acceptDocs)
throws IOException {
- FieldEntry fieldEntry = fields.get(field);
+ search(
+ fields.get(field),
+ knnCollector,
+ acceptDocs,
+ VectorEncoding.BYTE,
+ () -> flatVectorsReader.getRandomVectorScorer(field, target));
+ }
+
+ private void search(
+ FieldEntry fieldEntry,
+ KnnCollector knnCollector,
+ Bits acceptDocs,
+ VectorEncoding vectorEncoding,
+ IOSupplier scorerSupplier)
+ throws IOException {
if (fieldEntry.size() == 0
|| knnCollector.k() == 0
- || fieldEntry.vectorEncoding != VectorEncoding.BYTE) {
+ || fieldEntry.vectorEncoding != vectorEncoding) {
return;
}
- final RandomVectorScorer scorer = flatVectorsReader.getRandomVectorScorer(field, target);
+ final RandomVectorScorer scorer = scorerSupplier.get();
final KnnCollector collector =
new OrdinalTranslatedKnnCollector(knnCollector, scorer::ordToDoc);
final Bits acceptedOrds = scorer.getAcceptOrds(acceptDocs);
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java
index 8f715993a2b..bf97426738b 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter.java
@@ -24,9 +24,11 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
+import java.util.Objects;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.KnnFieldVectorsWriter;
import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.hnsw.FlatFieldVectorsWriter;
import org.apache.lucene.codecs.hnsw.FlatVectorsScorer;
import org.apache.lucene.codecs.hnsw.FlatVectorsWriter;
import org.apache.lucene.index.DocsWithFieldSet;
@@ -130,12 +132,13 @@ public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter {
FieldWriter> newField =
FieldWriter.create(
flatVectorWriter.getFlatVectorScorer(),
+ flatVectorWriter.addField(fieldInfo),
fieldInfo,
M,
beamWidth,
segmentWriteState.infoStream);
fields.add(newField);
- return flatVectorWriter.addField(fieldInfo, newField);
+ return newField;
}
@Override
@@ -171,8 +174,8 @@ public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter {
@Override
public long ramBytesUsed() {
long total = SHALLOW_RAM_BYTES_USED;
- total += flatVectorWriter.ramBytesUsed();
for (FieldWriter> field : fields) {
+ // the field tracks the delegate field usage
total += field.ramBytesUsed();
}
return total;
@@ -189,36 +192,19 @@ public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter {
fieldData.fieldInfo,
vectorIndexOffset,
vectorIndexLength,
- fieldData.docsWithField.cardinality(),
+ fieldData.getDocsWithFieldSet().cardinality(),
graph,
graphLevelNodeOffsets);
}
private void writeSortingField(FieldWriter> fieldData, Sorter.DocMap sortMap)
throws IOException {
- final int[] docIdOffsets = new int[sortMap.size()];
- int offset = 1; // 0 means no vector for this (field, document)
- DocIdSetIterator iterator = fieldData.docsWithField.iterator();
- for (int docID = iterator.nextDoc();
- docID != DocIdSetIterator.NO_MORE_DOCS;
- docID = iterator.nextDoc()) {
- int newDocID = sortMap.oldToNew(docID);
- docIdOffsets[newDocID] = offset++;
- }
- DocsWithFieldSet newDocsWithField = new DocsWithFieldSet();
- final int[] ordMap = new int[offset - 1]; // new ord to old ord
- final int[] oldOrdMap = new int[offset - 1]; // old ord to new ord
- int ord = 0;
- int doc = 0;
- for (int docIdOffset : docIdOffsets) {
- if (docIdOffset != 0) {
- ordMap[ord] = docIdOffset - 1;
- oldOrdMap[docIdOffset - 1] = ord;
- newDocsWithField.add(doc);
- ord++;
- }
- doc++;
- }
+ final int[] ordMap =
+ new int[fieldData.getDocsWithFieldSet().cardinality()]; // new ord to old ord
+ final int[] oldOrdMap =
+ new int[fieldData.getDocsWithFieldSet().cardinality()]; // old ord to new ord
+
+ mapOldOrdToNewOrd(fieldData.getDocsWithFieldSet(), sortMap, oldOrdMap, ordMap, null);
// write graph
long vectorIndexOffset = vectorIndex.getFilePointer();
OnHeapHnswGraph graph = fieldData.getGraph();
@@ -230,7 +216,7 @@ public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter {
fieldData.fieldInfo,
vectorIndexOffset,
vectorIndexLength,
- fieldData.docsWithField.cardinality(),
+ fieldData.getDocsWithFieldSet().cardinality(),
mockGraph,
graphLevelNodeOffsets);
}
@@ -542,42 +528,65 @@ public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter {
RamUsageEstimator.shallowSizeOfInstance(FieldWriter.class);
private final FieldInfo fieldInfo;
- private final DocsWithFieldSet docsWithField;
- private final List vectors;
private final HnswGraphBuilder hnswGraphBuilder;
private int lastDocID = -1;
private int node = 0;
+ private final FlatFieldVectorsWriter flatFieldVectorsWriter;
+ @SuppressWarnings("unchecked")
static FieldWriter> create(
- FlatVectorsScorer scorer, FieldInfo fieldInfo, int M, int beamWidth, InfoStream infoStream)
+ FlatVectorsScorer scorer,
+ FlatFieldVectorsWriter> flatFieldVectorsWriter,
+ FieldInfo fieldInfo,
+ int M,
+ int beamWidth,
+ InfoStream infoStream)
throws IOException {
return switch (fieldInfo.getVectorEncoding()) {
- case BYTE -> new FieldWriter(scorer, fieldInfo, M, beamWidth, infoStream);
- case FLOAT32 -> new FieldWriter(scorer, fieldInfo, M, beamWidth, infoStream);
+ case BYTE -> new FieldWriter<>(
+ scorer,
+ (FlatFieldVectorsWriter) flatFieldVectorsWriter,
+ fieldInfo,
+ M,
+ beamWidth,
+ infoStream);
+ case FLOAT32 -> new FieldWriter<>(
+ scorer,
+ (FlatFieldVectorsWriter) flatFieldVectorsWriter,
+ fieldInfo,
+ M,
+ beamWidth,
+ infoStream);
};
}
@SuppressWarnings("unchecked")
FieldWriter(
- FlatVectorsScorer scorer, FieldInfo fieldInfo, int M, int beamWidth, InfoStream infoStream)
+ FlatVectorsScorer scorer,
+ FlatFieldVectorsWriter flatFieldVectorsWriter,
+ FieldInfo fieldInfo,
+ int M,
+ int beamWidth,
+ InfoStream infoStream)
throws IOException {
this.fieldInfo = fieldInfo;
- this.docsWithField = new DocsWithFieldSet();
- vectors = new ArrayList<>();
RandomVectorScorerSupplier scorerSupplier =
switch (fieldInfo.getVectorEncoding()) {
case BYTE -> scorer.getRandomVectorScorerSupplier(
fieldInfo.getVectorSimilarityFunction(),
RandomAccessVectorValues.fromBytes(
- (List) vectors, fieldInfo.getVectorDimension()));
+ (List) flatFieldVectorsWriter.getVectors(),
+ fieldInfo.getVectorDimension()));
case FLOAT32 -> scorer.getRandomVectorScorerSupplier(
fieldInfo.getVectorSimilarityFunction(),
RandomAccessVectorValues.fromFloats(
- (List) vectors, fieldInfo.getVectorDimension()));
+ (List) flatFieldVectorsWriter.getVectors(),
+ fieldInfo.getVectorDimension()));
};
hnswGraphBuilder =
HnswGraphBuilder.create(scorerSupplier, M, beamWidth, HnswGraphBuilder.randSeed);
hnswGraphBuilder.setInfoStream(infoStream);
+ this.flatFieldVectorsWriter = Objects.requireNonNull(flatFieldVectorsWriter);
}
@Override
@@ -588,20 +597,23 @@ public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter {
+ fieldInfo.name
+ "\" appears more than once in this document (only one value is allowed per field)");
}
- assert docID > lastDocID;
- vectors.add(vectorValue);
- docsWithField.add(docID);
+ flatFieldVectorsWriter.addValue(docID, vectorValue);
hnswGraphBuilder.addGraphNode(node);
node++;
lastDocID = docID;
}
+ public DocsWithFieldSet getDocsWithFieldSet() {
+ return flatFieldVectorsWriter.getDocsWithFieldSet();
+ }
+
@Override
public T copyValue(T vectorValue) {
throw new UnsupportedOperationException();
}
OnHeapHnswGraph getGraph() {
+ assert flatFieldVectorsWriter.isFinished();
if (node > 0) {
return hnswGraphBuilder.getGraph();
} else {
@@ -612,9 +624,7 @@ public final class Lucene99HnswVectorsWriter extends KnnVectorsWriter {
@Override
public long ramBytesUsed() {
return SHALLOW_SIZE
- + docsWithField.ramBytesUsed()
- + (long) vectors.size()
- * (RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER)
+ + flatFieldVectorsWriter.ramBytesUsed()
+ hnswGraphBuilder.getGraph().ramBytesUsed();
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsWriter.java
index c052ce20646..311f2df435e 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorsWriter.java
@@ -30,8 +30,8 @@ import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.ArrayList;
import java.util.List;
+import java.util.Objects;
import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.KnnFieldVectorsWriter;
import org.apache.lucene.codecs.KnnVectorsReader;
import org.apache.lucene.codecs.KnnVectorsWriter;
import org.apache.lucene.codecs.hnsw.FlatFieldVectorsWriter;
@@ -56,7 +56,6 @@ import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream;
-import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.VectorUtil;
import org.apache.lucene.util.hnsw.CloseableRandomVectorScorerSupplier;
import org.apache.lucene.util.hnsw.RandomVectorScorer;
@@ -195,8 +194,8 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
}
@Override
- public FlatFieldVectorsWriter> addField(
- FieldInfo fieldInfo, KnnFieldVectorsWriter> indexWriter) throws IOException {
+ public FlatFieldVectorsWriter> addField(FieldInfo fieldInfo) throws IOException {
+ FlatFieldVectorsWriter> rawVectorDelegate = this.rawVectorDelegate.addField(fieldInfo);
if (fieldInfo.getVectorEncoding().equals(VectorEncoding.FLOAT32)) {
if (bits <= 4 && fieldInfo.getVectorDimension() % 2 != 0) {
throw new IllegalArgumentException(
@@ -205,6 +204,7 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
+ " is not supported for odd vector dimensions; vector dimension="
+ fieldInfo.getVectorDimension());
}
+ @SuppressWarnings("unchecked")
FieldWriter quantizedWriter =
new FieldWriter(
confidenceInterval,
@@ -212,11 +212,11 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
compress,
fieldInfo,
segmentWriteState.infoStream,
- indexWriter);
+ (FlatFieldVectorsWriter) rawVectorDelegate);
fields.add(quantizedWriter);
- indexWriter = quantizedWriter;
+ return quantizedWriter;
}
- return rawVectorDelegate.addField(fieldInfo, indexWriter);
+ return rawVectorDelegate;
}
@Override
@@ -270,12 +270,13 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
public void flush(int maxDoc, Sorter.DocMap sortMap) throws IOException {
rawVectorDelegate.flush(maxDoc, sortMap);
for (FieldWriter field : fields) {
- field.finish();
+ ScalarQuantizer quantizer = field.createQuantizer();
if (sortMap == null) {
- writeField(field, maxDoc);
+ writeField(field, maxDoc, quantizer);
} else {
- writeSortingField(field, maxDoc, sortMap);
+ writeSortingField(field, maxDoc, sortMap, quantizer);
}
+ field.finish();
}
}
@@ -300,15 +301,17 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
public long ramBytesUsed() {
long total = SHALLOW_RAM_BYTES_USED;
for (FieldWriter field : fields) {
+ // the field tracks the delegate field usage
total += field.ramBytesUsed();
}
return total;
}
- private void writeField(FieldWriter fieldData, int maxDoc) throws IOException {
+ private void writeField(FieldWriter fieldData, int maxDoc, ScalarQuantizer scalarQuantizer)
+ throws IOException {
// write vector values
long vectorDataOffset = quantizedVectorData.alignFilePointer(Float.BYTES);
- writeQuantizedVectors(fieldData);
+ writeQuantizedVectors(fieldData, scalarQuantizer);
long vectorDataLength = quantizedVectorData.getFilePointer() - vectorDataOffset;
writeMeta(
@@ -319,9 +322,9 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
confidenceInterval,
bits,
compress,
- fieldData.minQuantile,
- fieldData.maxQuantile,
- fieldData.docsWithField);
+ scalarQuantizer.getLowerQuantile(),
+ scalarQuantizer.getUpperQuantile(),
+ fieldData.getDocsWithFieldSet());
}
private void writeMeta(
@@ -366,8 +369,8 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
DIRECT_MONOTONIC_BLOCK_SHIFT, meta, quantizedVectorData, count, maxDoc, docsWithField);
}
- private void writeQuantizedVectors(FieldWriter fieldData) throws IOException {
- ScalarQuantizer scalarQuantizer = fieldData.createQuantizer();
+ private void writeQuantizedVectors(FieldWriter fieldData, ScalarQuantizer scalarQuantizer)
+ throws IOException {
byte[] vector = new byte[fieldData.fieldInfo.getVectorDimension()];
byte[] compressedVector =
fieldData.compress
@@ -376,7 +379,8 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
: null;
final ByteBuffer offsetBuffer = ByteBuffer.allocate(Float.BYTES).order(ByteOrder.LITTLE_ENDIAN);
float[] copy = fieldData.normalize ? new float[fieldData.fieldInfo.getVectorDimension()] : null;
- for (float[] v : fieldData.floatVectors) {
+ assert fieldData.getVectors().isEmpty() || scalarQuantizer != null;
+ for (float[] v : fieldData.getVectors()) {
if (fieldData.normalize) {
System.arraycopy(v, 0, copy, 0, copy.length);
VectorUtil.l2normalize(copy);
@@ -397,33 +401,18 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
}
}
- private void writeSortingField(FieldWriter fieldData, int maxDoc, Sorter.DocMap sortMap)
+ private void writeSortingField(
+ FieldWriter fieldData, int maxDoc, Sorter.DocMap sortMap, ScalarQuantizer scalarQuantizer)
throws IOException {
- final int[] docIdOffsets = new int[sortMap.size()];
- int offset = 1; // 0 means no vector for this (field, document)
- DocIdSetIterator iterator = fieldData.docsWithField.iterator();
- for (int docID = iterator.nextDoc();
- docID != DocIdSetIterator.NO_MORE_DOCS;
- docID = iterator.nextDoc()) {
- int newDocID = sortMap.oldToNew(docID);
- docIdOffsets[newDocID] = offset++;
- }
+ final int[] ordMap =
+ new int[fieldData.getDocsWithFieldSet().cardinality()]; // new ord to old ord
+
DocsWithFieldSet newDocsWithField = new DocsWithFieldSet();
- final int[] ordMap = new int[offset - 1]; // new ord to old ord
- int ord = 0;
- int doc = 0;
- for (int docIdOffset : docIdOffsets) {
- if (docIdOffset != 0) {
- ordMap[ord] = docIdOffset - 1;
- newDocsWithField.add(doc);
- ord++;
- }
- doc++;
- }
+ mapOldOrdToNewOrd(fieldData.getDocsWithFieldSet(), sortMap, null, ordMap, newDocsWithField);
// write vector values
long vectorDataOffset = quantizedVectorData.alignFilePointer(Float.BYTES);
- writeSortedQuantizedVectors(fieldData, ordMap);
+ writeSortedQuantizedVectors(fieldData, ordMap, scalarQuantizer);
long quantizedVectorLength = quantizedVectorData.getFilePointer() - vectorDataOffset;
writeMeta(
fieldData.fieldInfo,
@@ -433,13 +422,13 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
confidenceInterval,
bits,
compress,
- fieldData.minQuantile,
- fieldData.maxQuantile,
+ scalarQuantizer.getLowerQuantile(),
+ scalarQuantizer.getUpperQuantile(),
newDocsWithField);
}
- private void writeSortedQuantizedVectors(FieldWriter fieldData, int[] ordMap) throws IOException {
- ScalarQuantizer scalarQuantizer = fieldData.createQuantizer();
+ private void writeSortedQuantizedVectors(
+ FieldWriter fieldData, int[] ordMap, ScalarQuantizer scalarQuantizer) throws IOException {
byte[] vector = new byte[fieldData.fieldInfo.getVectorDimension()];
byte[] compressedVector =
fieldData.compress
@@ -449,7 +438,7 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
final ByteBuffer offsetBuffer = ByteBuffer.allocate(Float.BYTES).order(ByteOrder.LITTLE_ENDIAN);
float[] copy = fieldData.normalize ? new float[fieldData.fieldInfo.getVectorDimension()] : null;
for (int ordinal : ordMap) {
- float[] v = fieldData.floatVectors.get(ordinal);
+ float[] v = fieldData.getVectors().get(ordinal);
if (fieldData.normalize) {
System.arraycopy(v, 0, copy, 0, copy.length);
VectorUtil.l2normalize(copy);
@@ -762,44 +751,51 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
static class FieldWriter extends FlatFieldVectorsWriter {
private static final long SHALLOW_SIZE = shallowSizeOfInstance(FieldWriter.class);
- private final List floatVectors;
private final FieldInfo fieldInfo;
private final Float confidenceInterval;
private final byte bits;
private final boolean compress;
private final InfoStream infoStream;
private final boolean normalize;
- private float minQuantile = Float.POSITIVE_INFINITY;
- private float maxQuantile = Float.NEGATIVE_INFINITY;
private boolean finished;
- private final DocsWithFieldSet docsWithField;
+ private final FlatFieldVectorsWriter flatFieldVectorsWriter;
- @SuppressWarnings("unchecked")
FieldWriter(
Float confidenceInterval,
byte bits,
boolean compress,
FieldInfo fieldInfo,
InfoStream infoStream,
- KnnFieldVectorsWriter> indexWriter) {
- super((KnnFieldVectorsWriter) indexWriter);
+ FlatFieldVectorsWriter indexWriter) {
+ super();
this.confidenceInterval = confidenceInterval;
this.bits = bits;
this.fieldInfo = fieldInfo;
this.normalize = fieldInfo.getVectorSimilarityFunction() == VectorSimilarityFunction.COSINE;
- this.floatVectors = new ArrayList<>();
this.infoStream = infoStream;
- this.docsWithField = new DocsWithFieldSet();
this.compress = compress;
+ this.flatFieldVectorsWriter = Objects.requireNonNull(indexWriter);
}
- void finish() throws IOException {
+ @Override
+ public boolean isFinished() {
+ return finished && flatFieldVectorsWriter.isFinished();
+ }
+
+ @Override
+ public void finish() throws IOException {
if (finished) {
return;
}
+ assert flatFieldVectorsWriter.isFinished();
+ finished = true;
+ }
+
+ ScalarQuantizer createQuantizer() throws IOException {
+ assert flatFieldVectorsWriter.isFinished();
+ List floatVectors = flatFieldVectorsWriter.getVectors();
if (floatVectors.size() == 0) {
- finished = true;
- return;
+ return new ScalarQuantizer(0, 0, bits);
}
FloatVectorValues floatVectorValues = new FloatVectorWrapper(floatVectors, normalize);
ScalarQuantizer quantizer =
@@ -809,8 +805,6 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
fieldInfo.getVectorSimilarityFunction(),
confidenceInterval,
bits);
- minQuantile = quantizer.getLowerQuantile();
- maxQuantile = quantizer.getUpperQuantile();
if (infoStream.isEnabled(QUANTIZED_VECTOR_COMPONENT)) {
infoStream.message(
QUANTIZED_VECTOR_COMPONENT,
@@ -820,41 +814,39 @@ public final class Lucene99ScalarQuantizedVectorsWriter extends FlatVectorsWrite
+ " bits="
+ bits
+ " minQuantile="
- + minQuantile
+ + quantizer.getLowerQuantile()
+ " maxQuantile="
- + maxQuantile);
+ + quantizer.getUpperQuantile());
}
- finished = true;
- }
-
- ScalarQuantizer createQuantizer() {
- assert finished;
- return new ScalarQuantizer(minQuantile, maxQuantile, bits);
+ return quantizer;
}
@Override
public long ramBytesUsed() {
long size = SHALLOW_SIZE;
- if (indexingDelegate != null) {
- size += indexingDelegate.ramBytesUsed();
- }
- if (floatVectors.size() == 0) return size;
- return size + (long) floatVectors.size() * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
+ size += flatFieldVectorsWriter.ramBytesUsed();
+ return size;
}
@Override
public void addValue(int docID, float[] vectorValue) throws IOException {
- docsWithField.add(docID);
- floatVectors.add(vectorValue);
- if (indexingDelegate != null) {
- indexingDelegate.addValue(docID, vectorValue);
- }
+ flatFieldVectorsWriter.addValue(docID, vectorValue);
}
@Override
public float[] copyValue(float[] vectorValue) {
throw new UnsupportedOperationException();
}
+
+ @Override
+ public List getVectors() {
+ return flatFieldVectorsWriter.getVectors();
+ }
+
+ @Override
+ public DocsWithFieldSet getDocsWithFieldSet() {
+ return flatFieldVectorsWriter.getDocsWithFieldSet();
+ }
}
static class FloatVectorWrapper extends FloatVectorValues {
diff --git a/lucene/core/src/java/org/apache/lucene/index/BaseTermsEnum.java b/lucene/core/src/java/org/apache/lucene/index/BaseTermsEnum.java
index 37b8395aebb..6809a7a8682 100644
--- a/lucene/core/src/java/org/apache/lucene/index/BaseTermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/index/BaseTermsEnum.java
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOBooleanSupplier;
/**
* A base TermsEnum that adds default implementations for
@@ -58,6 +59,11 @@ public abstract class BaseTermsEnum extends TermsEnum {
return seekCeil(text) == SeekStatus.FOUND;
}
+ @Override
+ public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
+ return () -> seekExact(text);
+ }
+
@Override
public void seekExact(BytesRef term, TermState state) throws IOException {
if (!seekExact(term)) {
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 0f6020f7873..aaa76f418a9 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -79,6 +79,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.CommandLineUtil;
import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOBooleanSupplier;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LongBitSet;
import org.apache.lucene.util.NamedThreadFactory;
@@ -3869,6 +3870,7 @@ public final class CheckIndex implements Closeable {
TermsEnum postingsTermsEnum = postingsTerms.iterator();
final boolean hasProx = terms.hasOffsets() || terms.hasPositions();
+ int seekExactCounter = 0;
BytesRef term;
while ((term = termsEnum.next()) != null) {
@@ -3876,7 +3878,14 @@ public final class CheckIndex implements Closeable {
postings = termsEnum.postings(postings, PostingsEnum.ALL);
assert postings != null;
- if (postingsTermsEnum.seekExact(term) == false) {
+ boolean termExists;
+ if ((seekExactCounter++ & 0x01) == 0) {
+ termExists = postingsTermsEnum.seekExact(term);
+ } else {
+ IOBooleanSupplier termExistsSupplier = postingsTermsEnum.prepareSeekExact(term);
+ termExists = termExistsSupplier != null && termExistsSupplier.get();
+ }
+ if (termExists == false) {
throw new CheckIndexException(
"vector term="
+ term
diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldUpdatesBuffer.java b/lucene/core/src/java/org/apache/lucene/index/FieldUpdatesBuffer.java
index 9739df9cd83..a4d38dcd203 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FieldUpdatesBuffer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FieldUpdatesBuffer.java
@@ -356,7 +356,7 @@ final class FieldUpdatesBuffer {
}
}
- BytesRef nextTerm() throws IOException {
+ private BytesRef nextTerm() throws IOException {
if (lookAheadTermIterator != null) {
if (bufferedUpdate.termValue == null) {
lookAheadTermIterator.next();
diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
index 4935237178a..87d62f22d04 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
@@ -22,6 +22,7 @@ import org.apache.lucene.search.KnnCollector;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOBooleanSupplier;
import org.apache.lucene.util.Unwrappable;
/**
@@ -161,6 +162,7 @@ public abstract class FilterLeafReader extends LeafReader {
/** Base class for filtering {@link TermsEnum} implementations. */
public abstract static class FilterTermsEnum extends TermsEnum {
+
/** The underlying TermsEnum instance. */
protected final TermsEnum in;
@@ -236,6 +238,11 @@ public abstract class FilterLeafReader extends LeafReader {
in.seekExact(term, state);
}
+ @Override
+ public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
+ return in.prepareSeekExact(text);
+ }
+
@Override
public TermState termState() throws IOException {
return in.termState();
diff --git a/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java b/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java
index 5ee99878567..c8354cd881f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOBooleanSupplier;
/**
* Abstract class for enumerating a subset of all terms.
@@ -155,6 +156,16 @@ public abstract class FilteredTermsEnum extends TermsEnum {
throw new UnsupportedOperationException(getClass().getName() + " does not support seeking");
}
+ /**
+ * This enum does not support seeking!
+ *
+ * @throws UnsupportedOperationException In general, subclasses do not support seeking.
+ */
+ @Override
+ public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
+ throw new UnsupportedOperationException(getClass().getName() + " does not support seeking");
+ }
+
/**
* This enum does not support seeking!
*
diff --git a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
index f04096c78cf..5f6abd69c29 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
@@ -70,9 +70,8 @@ final class FreqProxTermsWriter extends TermsHash {
state.liveDocs = new FixedBitSet(state.segmentInfo.maxDoc());
state.liveDocs.set(0, state.segmentInfo.maxDoc());
}
- if (state.liveDocs.get(doc)) {
+ if (state.liveDocs.getAndClear(doc)) {
state.delCountOnFlush++;
- state.liveDocs.clear(doc);
}
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 44d8bee8460..83c2cbdaf1f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -4270,13 +4270,7 @@ public class IndexWriter
synchronized (fullFlushLock) {
boolean flushSuccess = false;
try {
- long seqNo = docWriter.flushAllThreads();
- if (seqNo < 0) {
- seqNo = -seqNo;
- anyChanges = true;
- } else {
- anyChanges = false;
- }
+ anyChanges = (docWriter.flushAllThreads() < 0);
if (!anyChanges) {
// flushCount is incremented in flushAllThreads
flushCount.incrementAndGet();
diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
index 513f897d725..4be3e41ad19 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
@@ -97,9 +97,8 @@ class PendingDeletes {
+ info.info.name
+ " maxDoc="
+ info.info.maxDoc();
- final boolean didDelete = mutableBits.get(docID);
+ final boolean didDelete = mutableBits.getAndClear(docID);
if (didDelete) {
- mutableBits.clear(docID);
pendingDeleteCount++;
}
return didDelete;
diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
index accdb57d1ba..557d31ad441 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
@@ -53,8 +53,7 @@ final class PendingSoftDeletes extends PendingDeletes {
FixedBitSet mutableBits = getMutableBits();
// hardDeletes
if (hardDeletes.delete(docID)) {
- if (mutableBits.get(docID)) { // delete it here too!
- mutableBits.clear(docID);
+ if (mutableBits.getAndClear(docID)) { // delete it here too!
assert hardDeletes.delete(docID) == false;
} else {
// if it was deleted subtract the delCount
@@ -135,16 +134,14 @@ final class PendingSoftDeletes extends PendingDeletes {
: null;
while ((docID = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
if (hasValue == null || hasValue.hasValue()) {
- if (bits.get(docID)) { // doc is live - clear it
- bits.clear(docID);
+ if (bits.getAndClear(docID)) { // doc is live - clear it
newDeletes++;
// now that we know we deleted it and we fully control the hard deletes we can do correct
// accounting
// below.
}
} else {
- if (bits.get(docID) == false) {
- bits.set(docID);
+ if (bits.getAndSet(docID) == false) {
newDeletes--;
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/index/Term.java b/lucene/core/src/java/org/apache/lucene/index/Term.java
index 173a78241b5..6baa34cdf8c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/Term.java
+++ b/lucene/core/src/java/org/apache/lucene/index/Term.java
@@ -89,7 +89,7 @@ public final class Term implements Comparable, Accountable {
* Returns the field of this term. The field indicates the part of a document which this term came
* from.
*/
- public final String field() {
+ public String field() {
return field;
}
@@ -97,7 +97,7 @@ public final class Term implements Comparable, Accountable {
* Returns the text of this term. In the case of words, this is simply the text of the word. In
* the case of dates and other types, this is an encoding of the object as a string.
*/
- public final String text() {
+ public String text() {
return toString(bytes);
}
@@ -105,7 +105,7 @@ public final class Term implements Comparable, Accountable {
* Returns human-readable form of the term text. If the term is not unicode, the raw bytes will be
* printed instead.
*/
- public static final String toString(BytesRef termText) {
+ public static String toString(BytesRef termText) {
// the term might not be text, but usually is. so we make a best effort
CharsetDecoder decoder =
StandardCharsets.UTF_8
@@ -124,7 +124,7 @@ public final class Term implements Comparable, Accountable {
}
/** Returns the bytes of this term, these should not be modified. */
- public final BytesRef bytes() {
+ public BytesRef bytes() {
return bytes;
}
@@ -160,7 +160,7 @@ public final class Term implements Comparable, Accountable {
*
The ordering of terms is first by field, then by text.
*/
@Override
- public final int compareTo(Term other) {
+ public int compareTo(Term other) {
if (field.equals(other.field)) {
return bytes.compareTo(other.bytes);
} else {
@@ -181,7 +181,7 @@ public final class Term implements Comparable, Accountable {
}
@Override
- public final String toString() {
+ public String toString() {
return field + ":" + text();
}
diff --git a/lucene/core/src/java/org/apache/lucene/index/TermStates.java b/lucene/core/src/java/org/apache/lucene/index/TermStates.java
index 75c64a907b1..2321104d30f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/TermStates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/TermStates.java
@@ -17,12 +17,12 @@
package org.apache.lucene.index;
import java.io.IOException;
-import java.util.ArrayList;
import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.Callable;
+import java.util.function.Supplier;
import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.TaskExecutor;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.IOBooleanSupplier;
+import org.apache.lucene.util.IOSupplier;
/**
* Maintains a {@link IndexReader} {@link TermState} view over {@link IndexReader} instances
@@ -80,6 +80,8 @@ public final class TermStates {
register(state, ord, docFreq, totalTermFreq);
}
+ private record PendingTermLookup(TermsEnum termsEnum, IOBooleanSupplier supplier) {}
+
/**
* Creates a {@link TermStates} from a top-level {@link IndexReaderContext} and the given {@link
* Term}. This method will lookup the given term in all context's leaf readers and register each
@@ -97,42 +99,29 @@ public final class TermStates {
assert context != null;
final TermStates perReaderTermState = new TermStates(needsStats ? null : term, context);
if (needsStats) {
- TaskExecutor taskExecutor = indexSearcher.getTaskExecutor();
- // build the term states concurrently
- List> tasks = new ArrayList<>(context.leaves().size());
+ PendingTermLookup[] pendingTermLookups = new PendingTermLookup[0];
for (LeafReaderContext ctx : context.leaves()) {
- tasks.add(
- () -> {
- TermsEnum termsEnum = loadTermsEnum(ctx, term);
- return termsEnum == null
- ? null
- : new TermStateInfo(
- termsEnum.termState(),
- ctx.ord,
- termsEnum.docFreq(),
- termsEnum.totalTermFreq());
- });
+ Terms terms = Terms.getTerms(ctx.reader(), term.field());
+ TermsEnum termsEnum = terms.iterator();
+ // Schedule the I/O in the terms dictionary in the background.
+ IOBooleanSupplier termExistsSupplier = termsEnum.prepareSeekExact(term.bytes());
+ if (termExistsSupplier != null) {
+ pendingTermLookups = ArrayUtil.grow(pendingTermLookups, ctx.ord + 1);
+ pendingTermLookups[ctx.ord] = new PendingTermLookup(termsEnum, termExistsSupplier);
+ }
}
- List resultInfos = taskExecutor.invokeAll(tasks);
- for (TermStateInfo info : resultInfos) {
- if (info != null) {
+ for (int ord = 0; ord < pendingTermLookups.length; ++ord) {
+ PendingTermLookup pendingTermLookup = pendingTermLookups[ord];
+ if (pendingTermLookup != null && pendingTermLookup.supplier.get()) {
+ TermsEnum termsEnum = pendingTermLookup.termsEnum();
perReaderTermState.register(
- info.getState(), info.getOrdinal(), info.getDocFreq(), info.getTotalTermFreq());
+ termsEnum.termState(), ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
}
}
}
return perReaderTermState;
}
- private static TermsEnum loadTermsEnum(LeafReaderContext ctx, Term term) throws IOException {
- final Terms terms = Terms.getTerms(ctx.reader(), term.field());
- final TermsEnum termsEnum = terms.iterator();
- if (termsEnum.seekExact(term.bytes())) {
- return termsEnum;
- }
- return null;
- }
-
/** Clears the {@link TermStates} internal state and removes all registered {@link TermState}s */
public void clear() {
docFreq = 0;
@@ -172,22 +161,60 @@ public final class TermStates {
}
/**
- * Returns the {@link TermState} for a leaf reader context or null if no {@link
- * TermState} for the context was registered.
+ * Returns a {@link Supplier} for a {@link TermState} for the given {@link LeafReaderContext}.
+ * This may return {@code null} if some cheap checks help figure out that this term doesn't exist
+ * in this leaf. The {@link Supplier} may then also return {@code null} if the term doesn't exist.
+ *
+ *
Calling this method typically schedules some I/O in the background, so it is recommended to
+ * retrieve {@link Supplier}s across all required terms first before calling {@link Supplier#get}
+ * on all {@link Supplier}s so that the I/O for these terms can be performed in parallel.
*
* @param ctx the {@link LeafReaderContext} to get the {@link TermState} for.
- * @return the {@link TermState} for the given readers ord or null if no {@link
- * TermState} for the reader was registered
+ * @return a Supplier for a TermState.
*/
- public TermState get(LeafReaderContext ctx) throws IOException {
+ public IOSupplier get(LeafReaderContext ctx) throws IOException {
assert ctx.ord >= 0 && ctx.ord < states.length;
- if (term == null) return states[ctx.ord];
- if (this.states[ctx.ord] == null) {
- TermsEnum te = loadTermsEnum(ctx, term);
- this.states[ctx.ord] = te == null ? EMPTY_TERMSTATE : te.termState();
+ if (term == null) {
+ if (states[ctx.ord] == null) {
+ return null;
+ } else {
+ return () -> states[ctx.ord];
+ }
}
- if (this.states[ctx.ord] == EMPTY_TERMSTATE) return null;
- return this.states[ctx.ord];
+ if (this.states[ctx.ord] == null) {
+ final Terms terms = ctx.reader().terms(term.field());
+ if (terms == null) {
+ this.states[ctx.ord] = EMPTY_TERMSTATE;
+ return null;
+ }
+ final TermsEnum termsEnum = terms.iterator();
+ IOBooleanSupplier termExistsSupplier = termsEnum.prepareSeekExact(term.bytes());
+ if (termExistsSupplier == null) {
+ this.states[ctx.ord] = EMPTY_TERMSTATE;
+ return null;
+ }
+ return () -> {
+ if (this.states[ctx.ord] == null) {
+ TermState state = null;
+ if (termExistsSupplier.get()) {
+ state = termsEnum.termState();
+ this.states[ctx.ord] = state;
+ } else {
+ this.states[ctx.ord] = EMPTY_TERMSTATE;
+ }
+ }
+ TermState state = this.states[ctx.ord];
+ if (state == EMPTY_TERMSTATE) {
+ return null;
+ }
+ return state;
+ };
+ }
+ TermState state = this.states[ctx.ord];
+ if (state == EMPTY_TERMSTATE) {
+ return null;
+ }
+ return () -> state;
}
/**
@@ -230,40 +257,4 @@ public final class TermStates {
return sb.toString();
}
-
- /** Wrapper over TermState, ordinal value, term doc frequency and total term frequency */
- private static final class TermStateInfo {
- private final TermState state;
- private final int ordinal;
- private final int docFreq;
- private final long totalTermFreq;
-
- /** Initialize TermStateInfo */
- public TermStateInfo(TermState state, int ordinal, int docFreq, long totalTermFreq) {
- this.state = state;
- this.ordinal = ordinal;
- this.docFreq = docFreq;
- this.totalTermFreq = totalTermFreq;
- }
-
- /** Get term state */
- public TermState getState() {
- return state;
- }
-
- /** Get ordinal value */
- public int getOrdinal() {
- return ordinal;
- }
-
- /** Get term doc frequency */
- public int getDocFreq() {
- return docFreq;
- }
-
- /** Get total term frequency */
- public long getTotalTermFreq() {
- return totalTermFreq;
- }
- }
}
diff --git a/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java b/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java
index 79e985c9204..2ff6a2719b6 100644
--- a/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java
@@ -17,9 +17,11 @@
package org.apache.lucene.index;
import java.io.IOException;
+import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
+import org.apache.lucene.util.IOBooleanSupplier;
/**
* Iterator to seek ({@link #seekCeil(BytesRef)}, {@link #seekExact(BytesRef)}) or step through
@@ -61,6 +63,23 @@ public abstract class TermsEnum implements BytesRefIterator {
*/
public abstract boolean seekExact(BytesRef text) throws IOException;
+ /**
+ * Two-phase {@link #seekExact}. The first phase typically calls {@link IndexInput#prefetch} on
+ * the right range of bytes under the hood, while the second phase {@link IOBooleanSupplier#get()}
+ * actually seeks the term within these bytes. This can be used to parallelize I/O across multiple
+ * terms by calling {@link #prepareSeekExact} on multiple terms enums before calling {@link
+ * IOBooleanSupplier#get()}.
+ *
+ *
NOTE: It is illegal to call other methods on this {@link TermsEnum} after calling
+ * this method until {@link IOBooleanSupplier#get()} is called.
+ *
+ *
NOTE: This may return {@code null} if this {@link TermsEnum} can identify that the
+ * term may not exist without performing any I/O.
+ *
+ *
NOTE: The returned {@link IOBooleanSupplier} must be consumed in the same thread.
+ */
+ public abstract IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException;
+
/**
* Seeks to the specified term, if it exists, or to the next (ceiling) term. Returns SeekStatus to
* indicate whether exact term was found, a different term was found, or EOF was hit. The target
@@ -178,9 +197,7 @@ public abstract class TermsEnum implements BytesRefIterator {
* of unused Attributes does not matter.
*/
public static final TermsEnum EMPTY =
- new TermsEnum() {
-
- private AttributeSource atts = null;
+ new BaseTermsEnum() {
@Override
public SeekStatus seekCeil(BytesRef term) {
@@ -225,19 +242,6 @@ public abstract class TermsEnum implements BytesRefIterator {
return null;
}
- @Override // make it synchronized here, to prevent double lazy init
- public synchronized AttributeSource attributes() {
- if (atts == null) {
- atts = new AttributeSource();
- }
- return atts;
- }
-
- @Override
- public boolean seekExact(BytesRef text) throws IOException {
- return seekCeil(text) == SeekStatus.FOUND;
- }
-
@Override
public TermState termState() {
throw new IllegalStateException("this method should never be called");
diff --git a/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java b/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java
index 8fceb0cfec5..c0ce4eea3c6 100644
--- a/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/AbstractKnnVectorQuery.java
@@ -28,6 +28,7 @@ import java.util.concurrent.Callable;
import org.apache.lucene.codecs.KnnVectorsReader;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.QueryTimeout;
import org.apache.lucene.search.knn.KnnCollectorManager;
@@ -120,8 +121,8 @@ abstract class AbstractKnnVectorQuery extends Query {
Weight filterWeight,
TimeLimitingKnnCollectorManager timeLimitingKnnCollectorManager)
throws IOException {
- Bits liveDocs = ctx.reader().getLiveDocs();
- int maxDoc = ctx.reader().maxDoc();
+ final LeafReader reader = ctx.reader();
+ final Bits liveDocs = reader.getLiveDocs();
if (filterWeight == null) {
return approximateSearch(ctx, liveDocs, Integer.MAX_VALUE, timeLimitingKnnCollectorManager);
@@ -132,7 +133,7 @@ abstract class AbstractKnnVectorQuery extends Query {
return NO_RESULTS;
}
- BitSet acceptDocs = createBitSet(scorer.iterator(), liveDocs, maxDoc);
+ BitSet acceptDocs = createBitSet(scorer.iterator(), liveDocs, reader.maxDoc());
final int cost = acceptDocs.cardinality();
QueryTimeout queryTimeout = timeLimitingKnnCollectorManager.getQueryTimeout();
@@ -267,19 +268,19 @@ abstract class AbstractKnnVectorQuery extends Query {
docs[i] = topK.scoreDocs[i].doc;
scores[i] = topK.scoreDocs[i].score;
}
- int[] segmentStarts = findSegmentStarts(reader, docs);
+ int[] segmentStarts = findSegmentStarts(reader.leaves(), docs);
return new DocAndScoreQuery(docs, scores, maxScore, segmentStarts, reader.getContext().id());
}
- static int[] findSegmentStarts(IndexReader reader, int[] docs) {
- int[] starts = new int[reader.leaves().size() + 1];
+ static int[] findSegmentStarts(List leaves, int[] docs) {
+ int[] starts = new int[leaves.size() + 1];
starts[starts.length - 1] = docs.length;
if (starts.length == 2) {
return starts;
}
int resultIndex = 0;
for (int i = 1; i < starts.length - 1; i++) {
- int upper = reader.leaves().get(i).docBase;
+ int upper = leaves.get(i).docBase;
resultIndex = Arrays.binarySearch(docs, resultIndex, docs.length, upper);
if (resultIndex < 0) {
resultIndex = -1 - resultIndex;
diff --git a/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java b/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java
index 8b7d3e80fcd..8dc66036969 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BlendedTermQuery.java
@@ -26,6 +26,7 @@ import org.apache.lucene.index.TermState;
import org.apache.lucene.index.TermStates;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.IOSupplier;
import org.apache.lucene.util.InPlaceMergeSorter;
/**
@@ -316,7 +317,11 @@ public final class BlendedTermQuery extends Query {
List leaves = readerContext.leaves();
TermStates newCtx = new TermStates(readerContext);
for (int i = 0; i < leaves.size(); ++i) {
- TermState termState = ctx.get(leaves.get(i));
+ IOSupplier supplier = ctx.get(leaves.get(i));
+ if (supplier == null) {
+ continue;
+ }
+ TermState termState = supplier.get();
if (termState == null) {
continue;
}
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java
index fa86505088d..13852b1b9ee 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMatchesIterator.java
@@ -194,6 +194,15 @@ final class DisjunctionMatchesIterator implements MatchesIterator {
new PriorityQueue(matches.size()) {
@Override
protected boolean lessThan(MatchesIterator a, MatchesIterator b) {
+ if (a.startPosition() == -1 && b.startPosition() == -1) {
+ try {
+ return a.startOffset() < b.startOffset()
+ || (a.startOffset() == b.startOffset() && a.endOffset() < b.endOffset())
+ || (a.startOffset() == b.startOffset() && a.endOffset() == b.endOffset());
+ } catch (IOException e) {
+ throw new IllegalArgumentException("Failed to retrieve term offset", e);
+ }
+ }
return a.startPosition() < b.startPosition()
|| (a.startPosition() == b.startPosition() && a.endPosition() < b.endPosition())
|| (a.startPosition() == b.startPosition() && a.endPosition() == b.endPosition());
diff --git a/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java b/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
index cfd6ed232de..bf9aa0f8f2c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
+++ b/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
@@ -18,6 +18,7 @@ package org.apache.lucene.search;
import java.io.IOException;
import java.util.function.Supplier;
+import org.apache.lucene.index.BaseTermsEnum;
import org.apache.lucene.index.ImpactsEnum;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Term;
@@ -30,6 +31,7 @@ import org.apache.lucene.util.AttributeReflector;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IOBooleanSupplier;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.automaton.CompiledAutomaton;
@@ -39,7 +41,7 @@ import org.apache.lucene.util.automaton.CompiledAutomaton;
*
Term enumerations are always ordered by {@link BytesRef#compareTo}. Each term in the
* enumeration is greater than all that precede it.
*/
-public final class FuzzyTermsEnum extends TermsEnum {
+public final class FuzzyTermsEnum extends BaseTermsEnum {
// NOTE: we can't subclass FilteredTermsEnum here because we need to sometimes change actualEnum:
private TermsEnum actualEnum;
@@ -324,6 +326,11 @@ public final class FuzzyTermsEnum extends TermsEnum {
return actualEnum.seekExact(text);
}
+ @Override
+ public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
+ return actualEnum.prepareSeekExact(text);
+ }
+
@Override
public SeekStatus seekCeil(BytesRef text) throws IOException {
return actualEnum.seekCeil(text);
diff --git a/lucene/core/src/java/org/apache/lucene/search/HitsThresholdChecker.java b/lucene/core/src/java/org/apache/lucene/search/HitsThresholdChecker.java
index fb6f6bf4343..4232af53fe6 100644
--- a/lucene/core/src/java/org/apache/lucene/search/HitsThresholdChecker.java
+++ b/lucene/core/src/java/org/apache/lucene/search/HitsThresholdChecker.java
@@ -17,13 +17,13 @@
package org.apache.lucene.search;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
/** Used for defining custom algorithms to allow searches to early terminate */
abstract class HitsThresholdChecker {
/** Implementation of HitsThresholdChecker which allows global hit counting */
private static class GlobalHitsThresholdChecker extends HitsThresholdChecker {
- private final AtomicLong globalHitCount = new AtomicLong();
+ private final LongAdder globalHitCount = new LongAdder();
GlobalHitsThresholdChecker(int totalHitsThreshold) {
super(totalHitsThreshold);
@@ -32,12 +32,12 @@ abstract class HitsThresholdChecker {
@Override
void incrementHitCount() {
- globalHitCount.incrementAndGet();
+ globalHitCount.increment();
}
@Override
boolean isThresholdReached() {
- return globalHitCount.getAcquire() > getHitsThreshold();
+ return globalHitCount.longValue() > getHitsThreshold();
}
@Override
diff --git a/lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java b/lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java
index 5b60e680e10..db5ae4a0d9d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java
@@ -23,6 +23,7 @@ import org.apache.lucene.codecs.KnnVectorsReader;
import org.apache.lucene.document.KnnFloatVectorField;
import org.apache.lucene.index.ByteVectorValues;
import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.knn.KnnCollectorManager;
import org.apache.lucene.util.ArrayUtil;
@@ -83,24 +84,26 @@ public class KnnByteVectorQuery extends AbstractKnnVectorQuery {
KnnCollectorManager knnCollectorManager)
throws IOException {
KnnCollector knnCollector = knnCollectorManager.newCollector(visitedLimit, context);
- ByteVectorValues byteVectorValues = context.reader().getByteVectorValues(field);
+ LeafReader reader = context.reader();
+ ByteVectorValues byteVectorValues = reader.getByteVectorValues(field);
if (byteVectorValues == null) {
- ByteVectorValues.checkField(context.reader(), field);
+ ByteVectorValues.checkField(reader, field);
return NO_RESULTS;
}
if (Math.min(knnCollector.k(), byteVectorValues.size()) == 0) {
return NO_RESULTS;
}
- context.reader().searchNearestVectors(field, target, knnCollector, acceptDocs);
+ reader.searchNearestVectors(field, target, knnCollector, acceptDocs);
TopDocs results = knnCollector.topDocs();
return results != null ? results : NO_RESULTS;
}
@Override
VectorScorer createVectorScorer(LeafReaderContext context, FieldInfo fi) throws IOException {
- ByteVectorValues vectorValues = context.reader().getByteVectorValues(field);
+ LeafReader reader = context.reader();
+ ByteVectorValues vectorValues = reader.getByteVectorValues(field);
if (vectorValues == null) {
- ByteVectorValues.checkField(context.reader(), field);
+ ByteVectorValues.checkField(reader, field);
return null;
}
return vectorValues.scorer(target);
diff --git a/lucene/core/src/java/org/apache/lucene/search/KnnFloatVectorQuery.java b/lucene/core/src/java/org/apache/lucene/search/KnnFloatVectorQuery.java
index b06e81eb5d8..585893fa3c2 100644
--- a/lucene/core/src/java/org/apache/lucene/search/KnnFloatVectorQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/KnnFloatVectorQuery.java
@@ -23,6 +23,7 @@ import org.apache.lucene.codecs.KnnVectorsReader;
import org.apache.lucene.document.KnnFloatVectorField;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FloatVectorValues;
+import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.knn.KnnCollectorManager;
import org.apache.lucene.util.ArrayUtil;
@@ -84,24 +85,26 @@ public class KnnFloatVectorQuery extends AbstractKnnVectorQuery {
KnnCollectorManager knnCollectorManager)
throws IOException {
KnnCollector knnCollector = knnCollectorManager.newCollector(visitedLimit, context);
- FloatVectorValues floatVectorValues = context.reader().getFloatVectorValues(field);
+ LeafReader reader = context.reader();
+ FloatVectorValues floatVectorValues = reader.getFloatVectorValues(field);
if (floatVectorValues == null) {
- FloatVectorValues.checkField(context.reader(), field);
+ FloatVectorValues.checkField(reader, field);
return NO_RESULTS;
}
if (Math.min(knnCollector.k(), floatVectorValues.size()) == 0) {
return NO_RESULTS;
}
- context.reader().searchNearestVectors(field, target, knnCollector, acceptDocs);
+ reader.searchNearestVectors(field, target, knnCollector, acceptDocs);
TopDocs results = knnCollector.topDocs();
return results != null ? results : NO_RESULTS;
}
@Override
VectorScorer createVectorScorer(LeafReaderContext context, FieldInfo fi) throws IOException {
- FloatVectorValues vectorValues = context.reader().getFloatVectorValues(field);
+ LeafReader reader = context.reader();
+ FloatVectorValues vectorValues = reader.getFloatVectorValues(field);
if (vectorValues == null) {
- FloatVectorValues.checkField(context.reader(), field);
+ FloatVectorValues.checkField(reader, field);
return null;
}
return vectorValues.scorer(target);
diff --git a/lucene/core/src/java/org/apache/lucene/search/MatchesIterator.java b/lucene/core/src/java/org/apache/lucene/search/MatchesIterator.java
index 371ba0ee695..24da0246c6d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MatchesIterator.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MatchesIterator.java
@@ -45,14 +45,14 @@ public interface MatchesIterator {
boolean next() throws IOException;
/**
- * The start position of the current match
+ * The start position of the current match, or {@code -1} if positions are not available
*
*
Should only be called after {@link #next()} has returned {@code true}
*/
int startPosition();
/**
- * The end position of the current match
+ * The end position of the current match, or {@code -1} if positions are not available
*
*
Should only be called after {@link #next()} has returned {@code true}
*/
diff --git a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java
index 026bf1f7d53..f250abfbcb9 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MaxScoreBulkScorer.java
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
import java.io.IOException;
import java.util.Arrays;
-import java.util.Comparator;
import java.util.List;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FixedBitSet;
@@ -43,7 +42,7 @@ final class MaxScoreBulkScorer extends BulkScorer {
int firstRequiredScorer;
private final long cost;
float minCompetitiveScore;
- private Score scorable = new Score();
+ private final Score scorable = new Score();
final double[] maxScoreSums;
private final long[] windowMatches = new long[FixedBitSet.bits2words(INNER_WINDOW_SIZE)];
@@ -333,10 +332,14 @@ final class MaxScoreBulkScorer extends BulkScorer {
// make a difference when using custom scores (like FuzzyQuery), high query-time boosts, or
// scoring based on wacky weights.
System.arraycopy(allScorers, 0, scratch, 0, allScorers.length);
+ // Do not use Comparator#comparingDouble below, it might cause unnecessary allocations
Arrays.sort(
scratch,
- Comparator.comparingDouble(
- scorer -> (double) scorer.maxWindowScore / Math.max(1L, scorer.cost)));
+ (scorer1, scorer2) -> {
+ return Double.compare(
+ (double) scorer1.maxWindowScore / Math.max(1L, scorer1.cost),
+ (double) scorer2.maxWindowScore / Math.max(1L, scorer2.cost));
+ });
double maxScoreSum = 0;
firstEssentialScorer = 0;
for (int i = 0; i < allScorers.length; ++i) {
diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java b/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
index 315b94a2acd..83510c02991 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
@@ -38,6 +38,7 @@ import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.search.similarities.Similarity.SimScorer;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOSupplier;
import org.apache.lucene.util.PriorityQueue;
/**
@@ -271,7 +272,8 @@ public class MultiPhraseQuery extends Query {
List postings = new ArrayList<>();
for (Term term : terms) {
- TermState termState = termStates.get(term).get(context);
+ IOSupplier supplier = termStates.get(term).get(context);
+ TermState termState = supplier == null ? null : supplier.get();
if (termState != null) {
termsEnum.seekExact(term.bytes(), termState);
postings.add(
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 c5a5ee36fd4..55d0e228d1a 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
@@ -38,6 +38,7 @@ import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.search.similarities.Similarity.SimScorer;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOSupplier;
/**
* A Query that matches documents containing a particular sequence of terms. A PhraseQuery is built
@@ -498,7 +499,8 @@ public class PhraseQuery extends Query {
for (int i = 0; i < terms.length; i++) {
final Term t = terms[i];
- final TermState state = states[i].get(context);
+ final IOSupplier supplier = states[i].get(context);
+ final TermState state = supplier == null ? null : supplier.get();
if (state == null) {
/* term doesnt exist in this segment */
assert termNotInReader(reader, t) : "no termstate found but term exists in reader";
diff --git a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
index 0500f4630dc..82a3b6e0148 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
@@ -17,6 +17,7 @@
package org.apache.lucene.search;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@@ -38,6 +39,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOSupplier;
import org.apache.lucene.util.PriorityQueue;
/**
@@ -277,80 +279,120 @@ public final class SynonymQuery extends Query {
@Override
public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
- final Scorer synonymScorer;
- List iterators = new ArrayList<>();
- List impacts = new ArrayList<>();
- List termBoosts = new ArrayList<>();
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ IOSupplier[] termStateSuppliers = new IOSupplier[terms.length];
for (int i = 0; i < terms.length; i++) {
- TermState state = termStates[i].get(context);
- if (state != null) {
- TermsEnum termsEnum = context.reader().terms(field).iterator();
- termsEnum.seekExact(terms[i].term, state);
- if (scoreMode == ScoreMode.TOP_SCORES) {
- ImpactsEnum impactsEnum = termsEnum.impacts(PostingsEnum.FREQS);
- iterators.add(impactsEnum);
- impacts.add(impactsEnum);
- } else {
- PostingsEnum postingsEnum = termsEnum.postings(null, PostingsEnum.FREQS);
- iterators.add(postingsEnum);
- impacts.add(new SlowImpactsEnum(postingsEnum));
+ // schedule the I/O for terms dictionary lookups in the background
+ termStateSuppliers[i] = termStates[i].get(context);
+ }
+
+ return new ScorerSupplier() {
+
+ List iterators;
+ List impacts;
+ List termBoosts;
+ long cost;
+
+ private void init() throws IOException {
+ if (iterators != null) {
+ return;
+ }
+ iterators = new ArrayList<>();
+ impacts = new ArrayList<>();
+ termBoosts = new ArrayList<>();
+ cost = 0L;
+
+ for (int i = 0; i < terms.length; i++) {
+ IOSupplier supplier = termStateSuppliers[i];
+ TermState state = supplier == null ? null : supplier.get();
+ if (state != null) {
+ TermsEnum termsEnum = context.reader().terms(field).iterator();
+ termsEnum.seekExact(terms[i].term, state);
+ if (scoreMode == ScoreMode.TOP_SCORES) {
+ ImpactsEnum impactsEnum = termsEnum.impacts(PostingsEnum.FREQS);
+ iterators.add(impactsEnum);
+ impacts.add(impactsEnum);
+ } else {
+ PostingsEnum postingsEnum = termsEnum.postings(null, PostingsEnum.FREQS);
+ iterators.add(postingsEnum);
+ impacts.add(new SlowImpactsEnum(postingsEnum));
+ }
+ termBoosts.add(terms[i].boost);
+ }
+ }
+
+ for (DocIdSetIterator iterator : iterators) {
+ cost += iterator.cost();
}
- termBoosts.add(terms[i].boost);
}
- }
- if (iterators.isEmpty()) {
- return null;
- }
+ @Override
+ public Scorer get(long leadCost) throws IOException {
+ init();
- LeafSimScorer simScorer = new LeafSimScorer(simWeight, context.reader(), field, true);
+ if (iterators.isEmpty()) {
+ return new ConstantScoreScorer(0f, scoreMode, DocIdSetIterator.empty());
+ }
- // we must optimize this case (term not in segment), disjunctions require >= 2 subs
- if (iterators.size() == 1) {
- final TermScorer scorer;
- if (scoreMode == ScoreMode.TOP_SCORES) {
- scorer = new TermScorer(impacts.get(0), simScorer);
- } else {
- scorer = new TermScorer(iterators.get(0), simScorer);
- }
- float boost = termBoosts.get(0);
- synonymScorer =
- scoreMode == ScoreMode.COMPLETE_NO_SCORES || boost == 1f
+ LeafSimScorer simScorer = new LeafSimScorer(simWeight, context.reader(), field, true);
+
+ // we must optimize this case (term not in segment), disjunctions require >= 2 subs
+ if (iterators.size() == 1) {
+ final TermScorer scorer;
+ if (scoreMode == ScoreMode.TOP_SCORES) {
+ scorer = new TermScorer(impacts.get(0), simScorer);
+ } else {
+ scorer = new TermScorer(iterators.get(0), simScorer);
+ }
+ float boost = termBoosts.get(0);
+ return scoreMode == ScoreMode.COMPLETE_NO_SCORES || boost == 1f
? scorer
: new FreqBoostTermScorer(boost, scorer, simScorer);
- } else {
+ } else {
- // we use termscorers + disjunction as an impl detail
- DisiPriorityQueue queue = new DisiPriorityQueue(iterators.size());
- for (int i = 0; i < iterators.size(); i++) {
- PostingsEnum postings = iterators.get(i);
- final TermScorer termScorer = new TermScorer(postings, simScorer);
- float boost = termBoosts.get(i);
- final DisiWrapperFreq wrapper = new DisiWrapperFreq(termScorer, boost);
- queue.add(wrapper);
- }
- // Even though it is called approximation, it is accurate since none of
- // the sub iterators are two-phase iterators.
- DocIdSetIterator iterator = new DisjunctionDISIApproximation(queue);
+ // we use termscorers + disjunction as an impl detail
+ DisiPriorityQueue queue = new DisiPriorityQueue(iterators.size());
+ for (int i = 0; i < iterators.size(); i++) {
+ PostingsEnum postings = iterators.get(i);
+ final TermScorer termScorer = new TermScorer(postings, simScorer);
+ float boost = termBoosts.get(i);
+ final DisiWrapperFreq wrapper = new DisiWrapperFreq(termScorer, boost);
+ queue.add(wrapper);
+ }
+ // Even though it is called approximation, it is accurate since none of
+ // the sub iterators are two-phase iterators.
+ DocIdSetIterator iterator = new DisjunctionDISIApproximation(queue);
- float[] boosts = new float[impacts.size()];
- for (int i = 0; i < boosts.length; i++) {
- boosts[i] = termBoosts.get(i);
- }
- ImpactsSource impactsSource = mergeImpacts(impacts.toArray(new ImpactsEnum[0]), boosts);
- MaxScoreCache maxScoreCache = new MaxScoreCache(impactsSource, simScorer.getSimScorer());
- ImpactsDISI impactsDisi = new ImpactsDISI(iterator, maxScoreCache);
+ float[] boosts = new float[impacts.size()];
+ for (int i = 0; i < boosts.length; i++) {
+ boosts[i] = termBoosts.get(i);
+ }
+ ImpactsSource impactsSource = mergeImpacts(impacts.toArray(new ImpactsEnum[0]), boosts);
+ MaxScoreCache maxScoreCache =
+ new MaxScoreCache(impactsSource, simScorer.getSimScorer());
+ ImpactsDISI impactsDisi = new ImpactsDISI(iterator, maxScoreCache);
- if (scoreMode == ScoreMode.TOP_SCORES) {
- // TODO: only do this when this is the top-level scoring clause
- // (ScorerSupplier#setTopLevelScoringClause) to save the overhead of wrapping with
- // ImpactsDISI when it would not help
- iterator = impactsDisi;
+ if (scoreMode == ScoreMode.TOP_SCORES) {
+ // TODO: only do this when this is the top-level scoring clause
+ // (ScorerSupplier#setTopLevelScoringClause) to save the overhead of wrapping with
+ // ImpactsDISI when it would not help
+ iterator = impactsDisi;
+ }
+
+ return new SynonymScorer(queue, iterator, impactsDisi, simScorer);
+ }
}
- synonymScorer = new SynonymScorer(queue, iterator, impactsDisi, simScorer);
- }
- return new DefaultScorerSupplier(synonymScorer);
+ @Override
+ public long cost() {
+ try {
+ init();
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ return cost;
+ }
+ };
}
@Override
diff --git a/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java b/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java
index 2763a9800e8..8832cf478e9 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TaskExecutor.java
@@ -30,6 +30,7 @@ import java.util.concurrent.Future;
import java.util.concurrent.FutureTask;
import java.util.concurrent.RunnableFuture;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.ThreadInterruptedException;
@@ -37,20 +38,13 @@ import org.apache.lucene.util.ThreadInterruptedException;
* Executor wrapper responsible for the execution of concurrent tasks. Used to parallelize search
* across segments as well as query rewrite in some cases. Exposes a single {@link
* #invokeAll(Collection)} method that takes a collection of {@link Callable}s and executes them
- * concurrently/ Once all tasks are submitted to the executor, it blocks and wait for all tasks to
- * be completed, and then returns a list with the obtained results. Ensures that the underlying
- * executor is only used for top-level {@link #invokeAll(Collection)} calls, and not for potential
- * {@link #invokeAll(Collection)} calls made from one of the tasks. This is to prevent deadlock with
- * certain types of pool based executors (e.g. {@link java.util.concurrent.ThreadPoolExecutor}).
+ * concurrently. Once all but one task have been submitted to the executor, it tries to run as many
+ * tasks as possible on the calling thread, then waits for all tasks that have been executed in
+ * parallel on the executor to be completed and then returns a list with the obtained results.
*
* @lucene.experimental
*/
public final class TaskExecutor {
- // a static thread local is ok as long as we use a counter, which accounts for multiple
- // searchers holding a different TaskExecutor all backed by the same executor
- private static final ThreadLocal numberOfRunningTasksInCurrentThread =
- ThreadLocal.withInitial(() -> 0);
-
private final Executor executor;
/**
@@ -84,26 +78,21 @@ public final class TaskExecutor {
/**
* Holds all the sub-tasks that a certain operation gets split into as it gets parallelized and
* exposes the ability to invoke such tasks and wait for them all to complete their execution and
- * provide their results. Ensures that each task does not get parallelized further: this is
- * important to avoid a deadlock in situations where one executor thread waits on other executor
- * threads to complete before it can progress. This happens in situations where for instance
- * {@link Query#createWeight(IndexSearcher, ScoreMode, float)} is called as part of searching each
- * slice, like {@link TopFieldCollector#populateScores(ScoreDoc[], IndexSearcher, Query)} does.
- * Additionally, if one task throws an exception, all other tasks from the same group are
- * cancelled, to avoid needless computation as their results would not be exposed anyways. Creates
- * one {@link FutureTask} for each {@link Callable} provided
+ * provide their results. Additionally, if one task throws an exception, all other tasks from the
+ * same group are cancelled, to avoid needless computation as their results would not be exposed
+ * anyways. Creates one {@link FutureTask} for each {@link Callable} provided
*
* @param the return type of all the callables
*/
private static final class TaskGroup {
- private final Collection> futures;
+ private final List> futures;
TaskGroup(Collection> callables) {
List> tasks = new ArrayList<>(callables.size());
for (Callable callable : callables) {
tasks.add(createTask(callable));
}
- this.futures = Collections.unmodifiableCollection(tasks);
+ this.futures = Collections.unmodifiableList(tasks);
}
RunnableFuture createTask(Callable callable) {
@@ -112,15 +101,10 @@ public final class TaskExecutor {
() -> {
if (startedOrCancelled.compareAndSet(false, true)) {
try {
- Integer counter = numberOfRunningTasksInCurrentThread.get();
- numberOfRunningTasksInCurrentThread.set(counter + 1);
return callable.call();
} catch (Throwable t) {
cancelAll();
throw t;
- } finally {
- Integer counter = numberOfRunningTasksInCurrentThread.get();
- numberOfRunningTasksInCurrentThread.set(counter - 1);
}
}
// task is cancelled hence it has no results to return. That's fine: they would be
@@ -144,32 +128,45 @@ public final class TaskExecutor {
}
List invokeAll(Executor executor) throws IOException {
- boolean runOnCallerThread = numberOfRunningTasksInCurrentThread.get() > 0;
- for (Runnable runnable : futures) {
- if (runOnCallerThread) {
- runnable.run();
- } else {
- executor.execute(runnable);
+ final int count = futures.size();
+ // taskId provides the first index of an un-executed task in #futures
+ final AtomicInteger taskId = new AtomicInteger(0);
+ // we fork execution count - 1 tasks to execute at least one task on the current thread to
+ // minimize needless forking and blocking of the current thread
+ if (count > 1) {
+ final Runnable work =
+ () -> {
+ int id = taskId.getAndIncrement();
+ if (id < count) {
+ futures.get(id).run();
+ }
+ };
+ for (int j = 0; j < count - 1; j++) {
+ executor.execute(work);
+ }
+ }
+ // try to execute as many tasks as possible on the current thread to minimize context
+ // switching in case of long running concurrent
+ // tasks as well as dead-locking if the current thread is part of #executor for executors that
+ // have limited or no parallelism
+ int id;
+ while ((id = taskId.getAndIncrement()) < count) {
+ futures.get(id).run();
+ if (id >= count - 1) {
+ // save redundant CAS in case this was the last task
+ break;
}
}
Throwable exc = null;
- List results = new ArrayList<>(futures.size());
- for (Future future : futures) {
+ List results = new ArrayList<>(count);
+ for (int i = 0; i < count; i++) {
+ Future future = futures.get(i);
try {
results.add(future.get());
} catch (InterruptedException e) {
- var newException = new ThreadInterruptedException(e);
- if (exc == null) {
- exc = newException;
- } else {
- exc.addSuppressed(newException);
- }
+ exc = IOUtils.useOrSuppress(exc, new ThreadInterruptedException(e));
} catch (ExecutionException e) {
- if (exc == null) {
- exc = e.getCause();
- } else {
- exc.addSuppressed(e.getCause());
- }
+ exc = IOUtils.useOrSuppress(exc, e.getCause());
}
}
assert assertAllFuturesCompleted() : "Some tasks are still running?";
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
index 84037acd0d4..3a843addcc3 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
@@ -17,6 +17,7 @@
package org.apache.lucene.search;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.util.Objects;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.LeafReader;
@@ -28,6 +29,7 @@ import org.apache.lucene.index.TermState;
import org.apache.lucene.index.TermStates;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.util.IOSupplier;
/**
* A Query that matches documents containing a term. This may be combined with other terms with a
@@ -119,18 +121,35 @@ public class TermQuery extends Query {
: "The top-reader used to create Weight is not the same as the current reader's top-reader ("
+ ReaderUtil.getTopLevelContext(context);
- final TermsEnum termsEnum = getTermsEnum(context);
- if (termsEnum == null) {
+ final IOSupplier stateSupplier = termStates.get(context);
+ if (stateSupplier == null) {
return null;
}
- final int docFreq = termsEnum.docFreq();
return new ScorerSupplier() {
+ private TermsEnum termsEnum;
private boolean topLevelScoringClause = false;
+ private TermsEnum getTermsEnum() throws IOException {
+ if (termsEnum == null) {
+ TermState state = stateSupplier.get();
+ if (state == null) {
+ return null;
+ }
+ termsEnum = context.reader().terms(term.field()).iterator();
+ termsEnum.seekExact(term.bytes(), state);
+ }
+ return termsEnum;
+ }
+
@Override
public Scorer get(long leadCost) throws IOException {
+ TermsEnum termsEnum = getTermsEnum();
+ if (termsEnum == null) {
+ return new ConstantScoreScorer(0f, scoreMode, DocIdSetIterator.empty());
+ }
+
LeafSimScorer scorer =
new LeafSimScorer(simScorer, context.reader(), term.field(), scoreMode.needsScores());
if (scoreMode == ScoreMode.TOP_SCORES) {
@@ -149,7 +168,12 @@ public class TermQuery extends Query {
@Override
public long cost() {
- return docFreq;
+ try {
+ TermsEnum te = getTermsEnum();
+ return te == null ? 0 : te.docFreq();
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
}
@Override
@@ -173,7 +197,8 @@ public class TermQuery extends Query {
assert termStates.wasBuiltFor(ReaderUtil.getTopLevelContext(context))
: "The top-reader used to create Weight is not the same as the current reader's top-reader ("
+ ReaderUtil.getTopLevelContext(context);
- final TermState state = termStates.get(context);
+ final IOSupplier supplier = termStates.get(context);
+ final TermState state = supplier == null ? null : supplier.get();
if (state == null) { // term is not present in that reader
assert termNotInReader(context.reader(), term)
: "no termstate found but term exists in reader term=" + term;
@@ -193,11 +218,11 @@ public class TermQuery extends Query {
@Override
public Explanation explain(LeafReaderContext context, int doc) throws IOException {
- TermScorer scorer = (TermScorer) scorer(context);
+ Scorer scorer = scorer(context);
if (scorer != null) {
int newDoc = scorer.iterator().advance(doc);
if (newDoc == doc) {
- float freq = scorer.freq();
+ float freq = ((TermScorer) scorer).freq();
LeafSimScorer docScorer =
new LeafSimScorer(simScorer, context.reader(), term.field(), true);
Explanation freqExplanation =
diff --git a/lucene/core/src/java/org/apache/lucene/search/knn/MultiLeafKnnCollector.java b/lucene/core/src/java/org/apache/lucene/search/knn/MultiLeafKnnCollector.java
index 5f7b26f95d4..1ca979d6794 100644
--- a/lucene/core/src/java/org/apache/lucene/search/knn/MultiLeafKnnCollector.java
+++ b/lucene/core/src/java/org/apache/lucene/search/knn/MultiLeafKnnCollector.java
@@ -41,6 +41,7 @@ public final class MultiLeafKnnCollector implements KnnCollector {
private final float greediness;
// the queue of the local similarities to periodically update with the global queue
private final FloatHeap updatesQueue;
+ private final float[] updatesScratch;
// interval to synchronize the local and global queues, as a number of visited vectors
private final int interval = 0xff; // 255
private boolean kResultsCollected = false;
@@ -62,6 +63,7 @@ public final class MultiLeafKnnCollector implements KnnCollector {
this.globalSimilarityQueue = globalSimilarityQueue;
this.nonCompetitiveQueue = new FloatHeap(Math.max(1, Math.round((1 - greediness) * k)));
this.updatesQueue = new FloatHeap(k);
+ this.updatesScratch = new float[k];
}
@Override
@@ -103,9 +105,18 @@ public final class MultiLeafKnnCollector implements KnnCollector {
if (kResultsCollected) {
// as we've collected k results, we can start do periodic updates with the global queue
if (firstKResultsCollected || (subCollector.visitedCount() & interval) == 0) {
- cachedGlobalMinSim = globalSimilarityQueue.offer(updatesQueue.getHeap());
- updatesQueue.clear();
- globalSimUpdated = true;
+ // BlockingFloatHeap#offer requires input to be sorted in ascending order, so we can't
+ // pass in the underlying updatesQueue array as-is since it is only partially ordered
+ // (see GH#13462):
+ int len = updatesQueue.size();
+ if (len > 0) {
+ for (int i = 0; i < len; i++) {
+ updatesScratch[i] = updatesQueue.poll();
+ }
+ assert updatesQueue.size() == 0;
+ cachedGlobalMinSim = globalSimilarityQueue.offer(updatesScratch, len);
+ globalSimUpdated = true;
+ }
}
}
return localSimUpdated || globalSimUpdated;
diff --git a/lucene/core/src/java/org/apache/lucene/store/IOContext.java b/lucene/core/src/java/org/apache/lucene/store/IOContext.java
index b2d82af20f8..f318b3a9015 100644
--- a/lucene/core/src/java/org/apache/lucene/store/IOContext.java
+++ b/lucene/core/src/java/org/apache/lucene/store/IOContext.java
@@ -55,7 +55,12 @@ public record IOContext(
*/
public static final IOContext DEFAULT = new IOContext(Constants.DEFAULT_READADVICE);
- /** A default context for reads with {@link ReadAdvice#SEQUENTIAL}. */
+ /**
+ * A default context for reads with {@link ReadAdvice#SEQUENTIAL}.
+ *
+ *
This context should only be used when the read operations will be performed in the same
+ * thread as the thread that opens the underlying storage.
+ */
public static final IOContext READONCE = new IOContext(ReadAdvice.SEQUENTIAL);
@SuppressWarnings("incomplete-switch")
diff --git a/lucene/core/src/java/org/apache/lucene/store/IndexInput.java b/lucene/core/src/java/org/apache/lucene/store/IndexInput.java
index 881f707d07a..ee84d908838 100644
--- a/lucene/core/src/java/org/apache/lucene/store/IndexInput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/IndexInput.java
@@ -18,6 +18,7 @@ package org.apache.lucene.store;
import java.io.Closeable;
import java.io.IOException;
+import org.apache.lucene.codecs.CompoundFormat;
/**
* Abstract base class for input from a file in a {@link Directory}. A random-access input stream.
@@ -121,6 +122,19 @@ public abstract class IndexInput extends DataInput implements Closeable {
public abstract IndexInput slice(String sliceDescription, long offset, long length)
throws IOException;
+ /**
+ * Create a slice with a specific {@link ReadAdvice}. This is typically used by {@link
+ * CompoundFormat} implementations to honor the {@link ReadAdvice} of each file within the
+ * compound file.
+ *
+ *
The default implementation delegates to {@link #slice(String, long, long)} and ignores the
+ * {@link ReadAdvice}.
+ */
+ public IndexInput slice(String sliceDescription, long offset, long length, ReadAdvice readAdvice)
+ throws IOException {
+ return slice(sliceDescription, offset, length);
+ }
+
/**
* Subclasses call this to get the String for resourceDescription of a slice of this {@code
* IndexInput}.
diff --git a/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java b/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java
index 5c3b4dbd88a..14b56a67e8a 100644
--- a/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/OutputStreamIndexOutput.java
@@ -135,5 +135,19 @@ public class OutputStreamIndexOutput extends IndexOutput {
BitUtil.VH_LE_LONG.set(buf, count, i);
count += Long.BYTES;
}
+
+ @Override
+ public void write(int b) throws IOException {
+ // override single byte write to avoid synchronization overhead now that JEP374 removed biased
+ // locking
+ byte[] buffer = buf;
+ int count = this.count;
+ if (count >= buffer.length) {
+ super.write(b);
+ } else {
+ buffer[count] = (byte) b;
+ this.count = count + 1;
+ }
+ }
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FSTStore.java b/lucene/core/src/java/org/apache/lucene/util/IOBooleanSupplier.java
similarity index 63%
rename from lucene/core/src/java/org/apache/lucene/util/fst/FSTStore.java
rename to lucene/core/src/java/org/apache/lucene/util/IOBooleanSupplier.java
index 682503b2c44..4100c6c53c5 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/FSTStore.java
+++ b/lucene/core/src/java/org/apache/lucene/util/IOBooleanSupplier.java
@@ -14,21 +14,24 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.lucene.util.fst;
+
+package org.apache.lucene.util;
import java.io.IOException;
-import org.apache.lucene.store.DataInput;
-/** A type of {@link FSTReader} which needs data to be initialized before use */
-public interface FSTStore extends FSTReader {
+/**
+ * Boolean supplier that is allowed to throw an IOException.
+ *
+ * @see java.util.function.BooleanSupplier
+ */
+@FunctionalInterface
+public interface IOBooleanSupplier {
/**
- * Initialize the FSTStore
+ * Gets the boolean result.
*
- * @param in the DataInput to read from
- * @param numBytes the number of bytes to read
- * @return this FSTStore
- * @throws IOException if exception occurred during reading the DataInput
+ * @return the result
+ * @throws IOException if supplying the result throws an {@link IOException}
*/
- FSTStore init(DataInput in, long numBytes) throws IOException;
+ boolean get() throws IOException;
}
diff --git a/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java
index 43e5077695c..0ae563c8701 100644
--- a/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/util/VectorUtil.java
@@ -212,6 +212,14 @@ public final class VectorUtil {
return IMPL.int4DotProduct(unpacked, false, packed, true);
}
+ /**
+ * For xorBitCount we stride over the values as either 64-bits (long) or 32-bits (int) at a time.
+ * On ARM Long::bitCount is not vectorized, and therefore produces less than optimal code, when
+ * compared to Integer::bitCount. While Long::bitCount is optimal on x64. See
+ * https://bugs.openjdk.org/browse/JDK-8336000
+ */
+ static final boolean XOR_BIT_COUNT_STRIDE_AS_INT = Constants.OS_ARCH.equals("aarch64");
+
/**
* XOR bit count computed over signed bytes.
*
@@ -223,8 +231,32 @@ public final class VectorUtil {
if (a.length != b.length) {
throw new IllegalArgumentException("vector dimensions differ: " + a.length + "!=" + b.length);
}
+ if (XOR_BIT_COUNT_STRIDE_AS_INT) {
+ return xorBitCountInt(a, b);
+ } else {
+ return xorBitCountLong(a, b);
+ }
+ }
+
+ /** XOR bit count striding over 4 bytes at a time. */
+ static int xorBitCountInt(byte[] a, byte[] b) {
int distance = 0, i = 0;
- for (final int upperBound = a.length & ~(Long.BYTES - 1); i < upperBound; i += Long.BYTES) {
+ for (final int upperBound = a.length & -Integer.BYTES; i < upperBound; i += Integer.BYTES) {
+ distance +=
+ Integer.bitCount(
+ (int) BitUtil.VH_NATIVE_INT.get(a, i) ^ (int) BitUtil.VH_NATIVE_INT.get(b, i));
+ }
+ // tail:
+ for (; i < a.length; i++) {
+ distance += Integer.bitCount((a[i] ^ b[i]) & 0xFF);
+ }
+ return distance;
+ }
+
+ /** XOR bit count striding over 8 bytes at a time. */
+ static int xorBitCountLong(byte[] a, byte[] b) {
+ int distance = 0, i = 0;
+ for (final int upperBound = a.length & -Long.BYTES; i < upperBound; i += Long.BYTES) {
distance +=
Long.bitCount(
(long) BitUtil.VH_NATIVE_LONG.get(a, i) ^ (long) BitUtil.VH_NATIVE_LONG.get(b, i));
diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java
index c83222cebf2..91eb4649efc 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Version.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Version.java
@@ -142,9 +142,17 @@ public final class Version {
*
* @deprecated Use latest
* @deprecated (9.12.0) Use latest
+ * @deprecated (9.11.1) Use latest
*/
@Deprecated public static final Version LUCENE_9_11_0 = new Version(9, 11, 0);
+ /**
+ * Match settings and bugs in Lucene's 9.11.1 release.
+ *
+ * @deprecated Use latest
+ */
+ @Deprecated public static final Version LUCENE_9_11_1 = new Version(9, 11, 1);
+
/**
* Match settings and bugs in Lucene's 9.12.0 release.
*
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
index 6bb5718d5c7..17201194da4 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
@@ -417,16 +417,7 @@ public final class FST implements Accountable {
* maxBlockBits set to {@link #DEFAULT_MAX_BLOCK_BITS}
*/
public FST(FSTMetadata metadata, DataInput in) throws IOException {
- this(metadata, in, new OnHeapFSTStore(DEFAULT_MAX_BLOCK_BITS));
- }
-
- /**
- * Load a previously saved FST with a metdata object and a FSTStore. If using {@link
- * OnHeapFSTStore}, setting maxBlockBits allows you to control the size of the byte[] pages used
- * to hold the FST bytes.
- */
- public FST(FSTMetadata metadata, DataInput in, FSTStore fstStore) throws IOException {
- this(metadata, fstStore.init(in, metadata.numBytes));
+ this(metadata, new OnHeapFSTStore(DEFAULT_MAX_BLOCK_BITS, in, metadata.numBytes));
}
/** Create the FST with a metadata object and a FSTReader. */
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/OffHeapFSTStore.java b/lucene/core/src/java/org/apache/lucene/util/fst/OffHeapFSTStore.java
index f88715b191c..e8b00037d15 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/OffHeapFSTStore.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/OffHeapFSTStore.java
@@ -17,7 +17,6 @@
package org.apache.lucene.util.fst;
import java.io.IOException;
-import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.RamUsageEstimator;
@@ -28,27 +27,19 @@ import org.apache.lucene.util.RamUsageEstimator;
*
* @lucene.experimental
*/
-public final class OffHeapFSTStore implements FSTStore {
+public final class OffHeapFSTStore implements FSTReader {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(OffHeapFSTStore.class);
- private IndexInput in;
- private long offset;
- private long numBytes;
+ private final IndexInput in;
+ private final long offset;
+ private final long numBytes;
- @Override
- public FSTStore init(DataInput in, long numBytes) throws IOException {
- if (in instanceof IndexInput) {
- this.in = (IndexInput) in;
- this.numBytes = numBytes;
- this.offset = this.in.getFilePointer();
- } else {
- throw new IllegalArgumentException(
- "parameter:in should be an instance of IndexInput for using OffHeapFSTStore, not a "
- + in.getClass().getName());
- }
- return this;
+ public OffHeapFSTStore(IndexInput in, long offset, FST.FSTMetadata> metadata) {
+ this.in = in;
+ this.offset = offset;
+ this.numBytes = metadata.numBytes;
}
@Override
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java b/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java
index 949babdaa88..15bb0bb5f8a 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/OnHeapFSTStore.java
@@ -28,7 +28,7 @@ import org.apache.lucene.util.RamUsageEstimator;
*
* @lucene.experimental
*/
-public final class OnHeapFSTStore implements FSTStore {
+public final class OnHeapFSTStore implements FSTReader {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(OnHeapFSTStore.class);
@@ -40,31 +40,24 @@ public final class OnHeapFSTStore implements FSTStore {
private ReadWriteDataOutput dataOutput;
/** Used at read time when the FST fits into a single byte[]. */
- private byte[] bytesArray;
+ private final byte[] bytesArray;
- private final int maxBlockBits;
-
- public OnHeapFSTStore(int maxBlockBits) {
+ public OnHeapFSTStore(int maxBlockBits, DataInput in, long numBytes) throws IOException {
if (maxBlockBits < 1 || maxBlockBits > 30) {
throw new IllegalArgumentException("maxBlockBits should be 1 .. 30; got " + maxBlockBits);
}
- this.maxBlockBits = maxBlockBits;
- }
-
- @Override
- public FSTStore init(DataInput in, long numBytes) throws IOException {
- if (numBytes > 1 << this.maxBlockBits) {
+ if (numBytes > 1 << maxBlockBits) {
// FST is big: we need multiple pages
dataOutput = (ReadWriteDataOutput) getOnHeapReaderWriter(maxBlockBits);
dataOutput.copyBytes(in, numBytes);
dataOutput.freeze();
+ bytesArray = null;
} else {
// FST fits into a single block: use ByteArrayBytesStoreReader for less overhead
bytesArray = new byte[(int) numBytes];
in.readBytes(bytesArray, 0, bytesArray.length);
}
- return this;
}
@Override
diff --git a/lucene/core/src/java/org/apache/lucene/util/hnsw/BlockingFloatHeap.java b/lucene/core/src/java/org/apache/lucene/util/hnsw/BlockingFloatHeap.java
index a81eaf2fee0..6bbf6fdb741 100644
--- a/lucene/core/src/java/org/apache/lucene/util/hnsw/BlockingFloatHeap.java
+++ b/lucene/core/src/java/org/apache/lucene/util/hnsw/BlockingFloatHeap.java
@@ -72,12 +72,13 @@ public final class BlockingFloatHeap {
*
Values must be sorted in ascending order.
*
* @param values a set of values to insert, must be sorted in ascending order
+ * @param len number of values from the {@code values} array to insert
* @return the new 'top' element in the queue.
*/
- public float offer(float[] values) {
+ public float offer(float[] values, int len) {
lock.lock();
try {
- for (int i = values.length - 1; i >= 0; i--) {
+ for (int i = len - 1; i >= 0; i--) {
if (size < maxSize) {
push(values[i]);
} else {
diff --git a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java
index b689088c304..e9805f0f7a6 100644
--- a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java
+++ b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java
@@ -28,6 +28,7 @@ import java.util.Optional;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BitUtil;
import org.apache.lucene.util.GroupVIntUtil;
+import org.apache.lucene.util.IOConsumer;
/**
* Base IndexInput implementation that uses an array of MemorySegments to represent a file.
@@ -52,6 +53,7 @@ abstract class MemorySegmentIndexInput extends IndexInput
final long length;
final long chunkSizeMask;
final int chunkSizePower;
+ final boolean confined;
final Arena arena;
final MemorySegment[] segments;
@@ -66,12 +68,15 @@ abstract class MemorySegmentIndexInput extends IndexInput
Arena arena,
MemorySegment[] segments,
long length,
- int chunkSizePower) {
+ int chunkSizePower,
+ boolean confined) {
assert Arrays.stream(segments).map(MemorySegment::scope).allMatch(arena.scope()::equals);
if (segments.length == 1) {
- return new SingleSegmentImpl(resourceDescription, arena, segments[0], length, chunkSizePower);
+ return new SingleSegmentImpl(
+ resourceDescription, arena, segments[0], length, chunkSizePower, confined);
} else {
- return new MultiSegmentImpl(resourceDescription, arena, segments, 0, length, chunkSizePower);
+ return new MultiSegmentImpl(
+ resourceDescription, arena, segments, 0, length, chunkSizePower, confined);
}
}
@@ -80,12 +85,14 @@ abstract class MemorySegmentIndexInput extends IndexInput
Arena arena,
MemorySegment[] segments,
long length,
- int chunkSizePower) {
+ int chunkSizePower,
+ boolean confined) {
super(resourceDescription);
this.arena = arena;
this.segments = segments;
this.length = length;
this.chunkSizePower = chunkSizePower;
+ this.confined = confined;
this.chunkSizeMask = (1L << chunkSizePower) - 1L;
this.curSegment = segments[0];
}
@@ -96,6 +103,12 @@ abstract class MemorySegmentIndexInput extends IndexInput
}
}
+ void ensureAccessible() {
+ if (confined && curSegment.isAccessibleBy(Thread.currentThread()) == false) {
+ throw new IllegalStateException("confined");
+ }
+ }
+
// the unused parameter is just to silence javac about unused variables
RuntimeException handlePositionalIOOBE(RuntimeException unused, String action, long pos)
throws IOException {
@@ -333,14 +346,36 @@ abstract class MemorySegmentIndexInput extends IndexInput
return;
}
+ final NativeAccess nativeAccess = NATIVE_ACCESS.get();
+ advise(
+ offset,
+ length,
+ segment -> {
+ if (segment.isLoaded() == false) {
+ // We have a cache miss on at least one page, let's reset the counter.
+ consecutivePrefetchHitCount = 0;
+ nativeAccess.madviseWillNeed(segment);
+ }
+ });
+ }
+
+ void advise(long offset, long length, IOConsumer advice) throws IOException {
+ if (NATIVE_ACCESS.isEmpty()) {
+ return;
+ }
+
+ ensureOpen();
+
+ Objects.checkFromIndexSize(offset, length, length());
+
final NativeAccess nativeAccess = NATIVE_ACCESS.get();
try {
final MemorySegment segment = segments[(int) (offset >> chunkSizePower)];
offset &= chunkSizeMask;
- // Compute the intersection of the current segment and the region that should be prefetched.
+ // Compute the intersection of the current segment and the region that should be advised.
if (offset + length > segment.byteSize()) {
- // Only prefetch bytes that are stored in the current segment. There may be bytes on the
+ // Only advise bytes that are stored in the current segment. There may be bytes on the
// next segment but this case is rare enough that we don't try to optimize it and keep
// things simple instead.
length = segment.byteSize() - offset;
@@ -351,16 +386,17 @@ abstract class MemorySegmentIndexInput extends IndexInput
offset -= offsetInPage;
length += offsetInPage;
if (offset < 0) {
- // The start of the page is outside of this segment, ignore.
- return;
+ // The start of the page is before the start of this segment, ignore the first page.
+ offset += nativeAccess.getPageSize();
+ length -= nativeAccess.getPageSize();
+ if (length <= 0) {
+ // This segment has no data beyond the first page.
+ return;
+ }
}
- final MemorySegment prefetchSlice = segment.asSlice(offset, length);
- if (prefetchSlice.isLoaded() == false) {
- // We have a cache miss on at least one page, let's reset the counter.
- consecutivePrefetchHitCount = 0;
- nativeAccess.madviseWillNeed(prefetchSlice);
- }
+ final MemorySegment advisedSlice = segment.asSlice(offset, length);
+ advice.accept(advisedSlice);
} catch (
@SuppressWarnings("unused")
IndexOutOfBoundsException e) {
@@ -527,9 +563,26 @@ abstract class MemorySegmentIndexInput extends IndexInput
return buildSlice(sliceDescription, offset, length);
}
+ @Override
+ public final MemorySegmentIndexInput slice(
+ String sliceDescription, long offset, long length, ReadAdvice advice) throws IOException {
+ MemorySegmentIndexInput slice = slice(sliceDescription, offset, length);
+ if (NATIVE_ACCESS.isPresent()) {
+ final NativeAccess nativeAccess = NATIVE_ACCESS.get();
+ slice.advise(
+ 0,
+ slice.length,
+ segment -> {
+ nativeAccess.madvise(segment, advice);
+ });
+ }
+ return slice;
+ }
+
/** Builds the actual sliced IndexInput (may apply extra offset in subclasses). * */
MemorySegmentIndexInput buildSlice(String sliceDescription, long offset, long length) {
ensureOpen();
+ ensureAccessible();
final long sliceEnd = offset + length;
final int startIndex = (int) (offset >>> chunkSizePower);
@@ -551,7 +604,8 @@ abstract class MemorySegmentIndexInput extends IndexInput
null, // clones don't have an Arena, as they can't close)
slices[0].asSlice(offset, length),
length,
- chunkSizePower);
+ chunkSizePower,
+ confined);
} else {
return new MultiSegmentImpl(
newResourceDescription,
@@ -559,7 +613,8 @@ abstract class MemorySegmentIndexInput extends IndexInput
slices,
offset,
length,
- chunkSizePower);
+ chunkSizePower,
+ confined);
}
}
@@ -603,8 +658,15 @@ abstract class MemorySegmentIndexInput extends IndexInput
Arena arena,
MemorySegment segment,
long length,
- int chunkSizePower) {
- super(resourceDescription, arena, new MemorySegment[] {segment}, length, chunkSizePower);
+ int chunkSizePower,
+ boolean confined) {
+ super(
+ resourceDescription,
+ arena,
+ new MemorySegment[] {segment},
+ length,
+ chunkSizePower,
+ confined);
this.curSegmentIndex = 0;
}
@@ -700,8 +762,9 @@ abstract class MemorySegmentIndexInput extends IndexInput
MemorySegment[] segments,
long offset,
long length,
- int chunkSizePower) {
- super(resourceDescription, arena, segments, length, chunkSizePower);
+ int chunkSizePower,
+ boolean confined) {
+ super(resourceDescription, arena, segments, length, chunkSizePower, confined);
this.offset = offset;
try {
seek(0L);
diff --git a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java
index e1655101d75..08f6149746b 100644
--- a/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java
+++ b/lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java
@@ -45,7 +45,8 @@ final class MemorySegmentIndexInputProvider implements MMapDirectory.MMapIndexIn
path = Unwrappable.unwrapAll(path);
boolean success = false;
- final Arena arena = Arena.ofShared();
+ final boolean confined = context == IOContext.READONCE;
+ final Arena arena = confined ? Arena.ofConfined() : Arena.ofShared();
try (var fc = FileChannel.open(path, StandardOpenOption.READ)) {
final long fileSize = fc.size();
final IndexInput in =
@@ -61,7 +62,8 @@ final class MemorySegmentIndexInputProvider implements MMapDirectory.MMapIndexIn
preload,
fileSize),
fileSize,
- chunkSizePower);
+ chunkSizePower,
+ confined);
success = true;
return in;
} finally {
diff --git a/lucene/core/src/java21/org/apache/lucene/store/PosixNativeAccess.java b/lucene/core/src/java21/org/apache/lucene/store/PosixNativeAccess.java
index 93caca788b1..80c1665cdd1 100644
--- a/lucene/core/src/java21/org/apache/lucene/store/PosixNativeAccess.java
+++ b/lucene/core/src/java21/org/apache/lucene/store/PosixNativeAccess.java
@@ -158,10 +158,10 @@ final class PosixNativeAccess extends NativeAccess {
private Integer mapReadAdvice(ReadAdvice readAdvice) {
return switch (readAdvice) {
- case NORMAL -> null;
+ case NORMAL -> POSIX_MADV_NORMAL;
case RANDOM -> POSIX_MADV_RANDOM;
case SEQUENTIAL -> POSIX_MADV_SEQUENTIAL;
- case RANDOM_PRELOAD -> null;
+ case RANDOM_PRELOAD -> POSIX_MADV_NORMAL;
};
}
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormatVariableSkipInterval.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormatVariableSkipInterval.java
new file mode 100644
index 00000000000..94204fb2a8d
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/TestLucene90DocValuesFormatVariableSkipInterval.java
@@ -0,0 +1,38 @@
+/*
+ * 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.lucene90;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.tests.index.BaseDocValuesFormatTestCase;
+import org.apache.lucene.tests.util.TestUtil;
+
+/** Tests Lucene90DocValuesFormat */
+public class TestLucene90DocValuesFormatVariableSkipInterval extends BaseDocValuesFormatTestCase {
+
+ @Override
+ protected Codec getCodec() {
+ return TestUtil.alwaysDocValuesFormat(new Lucene90DocValuesFormat(random().nextInt(2, 1024)));
+ }
+
+ public void testSkipIndexIntervalSize() {
+ IllegalArgumentException ex =
+ expectThrows(
+ IllegalArgumentException.class,
+ () -> new Lucene90DocValuesFormat(random().nextInt(Integer.MIN_VALUE, 2)));
+ assertTrue(ex.getMessage().contains("skipIndexIntervalSize must be > 1"));
+ }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/search/BaseVectorSimilarityQueryTestCase.java b/lucene/core/src/test/org/apache/lucene/search/BaseVectorSimilarityQueryTestCase.java
index f0bce64acd0..3347b9478dd 100644
--- a/lucene/core/src/test/org/apache/lucene/search/BaseVectorSimilarityQueryTestCase.java
+++ b/lucene/core/src/test/org/apache/lucene/search/BaseVectorSimilarityQueryTestCase.java
@@ -131,6 +131,7 @@ abstract class BaseVectorSimilarityQueryTestCase<
try (Directory indexStore = getIndexStore(getRandomVectors(numDocs, dim));
IndexReader reader = DirectoryReader.open(indexStore)) {
IndexSearcher searcher = newSearcher(reader);
+ assumeTrue("graph is disconnected", HnswTestUtil.graphIsConnected(reader, vectorField));
// All vectors are above -Infinity
Query query1 =
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java
index 87b8068d2f1..15f0c0d0c94 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanRewrites.java
@@ -92,7 +92,7 @@ public class TestBooleanRewrites extends LuceneTestCase {
// make sure to set score=0
BooleanQuery.Builder query2 = new BooleanQuery.Builder();
query2.add(new TermQuery(new Term("field", "a")), Occur.FILTER);
- query2.add(new TermQuery(new Term("field", "b")), Occur.SHOULD);
+ query2.add(new TermQuery(new Term("missing_field", "b")), Occur.SHOULD);
final Weight weight =
searcher.createWeight(searcher.rewrite(query2.build()), ScoreMode.COMPLETE, 1);
final Scorer scorer = weight.scorer(reader.leaves().get(0));
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java
index fec70cb3dcc..1258dd8b10e 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java
@@ -180,7 +180,7 @@ public class TestBooleanScorer extends LuceneTestCase {
Query query =
new BooleanQuery.Builder()
.add(new TermQuery(new Term("foo", "bar")), Occur.SHOULD) // existing term
- .add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD) // missing term
+ .add(new TermQuery(new Term("missing_field", "baz")), Occur.SHOULD) // missing term
.build();
// no scores -> term scorer
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java b/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
index 3cd2ecbdd19..724013abac7 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestIndexSearcher.java
@@ -266,7 +266,7 @@ public class TestIndexSearcher extends LuceneTestCase {
IOUtils.close(r, dir);
}
- public void testSlicesAllOffloadedToTheExecutor() throws IOException {
+ public void testSlicesOffloadedToTheExecutor() throws IOException {
List leaves = reader.leaves();
AtomicInteger numExecutions = new AtomicInteger(0);
IndexSearcher searcher =
@@ -286,7 +286,7 @@ public class TestIndexSearcher extends LuceneTestCase {
}
};
searcher.search(new MatchAllDocsQuery(), 10);
- assertEquals(leaves.size(), numExecutions.get());
+ assertEquals(leaves.size() - 1, numExecutions.get());
}
public void testNullExecutorNonNullTaskExecutor() {
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestKnnFloatVectorQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestKnnFloatVectorQuery.java
index 6ecc758c0e4..f2e5a3e274a 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestKnnFloatVectorQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestKnnFloatVectorQuery.java
@@ -216,7 +216,7 @@ public class TestKnnFloatVectorQuery extends BaseKnnVectorQueryTestCase {
maxScore = Math.max(maxScore, scores[i]);
}
IndexReader indexReader = searcher.getIndexReader();
- int[] segments = AbstractKnnVectorQuery.findSegmentStarts(indexReader, docs);
+ int[] segments = AbstractKnnVectorQuery.findSegmentStarts(indexReader.leaves(), docs);
AbstractKnnVectorQuery.DocAndScoreQuery query =
new AbstractKnnVectorQuery.DocAndScoreQuery(
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java b/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java
index ba5fa90f227..1949afbd51c 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestTaskExecutor.java
@@ -21,10 +21,7 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
+import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.DirectoryReader;
@@ -111,7 +108,20 @@ public class TestTaskExecutor extends LuceneTestCase {
assertEquals("exc", runtimeException.getCause().getMessage());
}
- public void testInvokeAllFromTaskDoesNotDeadlockSameSearcher() throws IOException {
+ public void testInvokeAllFromTaskDoesNotDeadlockSameSearcher() throws Exception {
+ doTestInvokeAllFromTaskDoesNotDeadlockSameSearcher(executorService);
+ doTestInvokeAllFromTaskDoesNotDeadlockSameSearcher(Runnable::run);
+ executorService
+ .submit(
+ () -> {
+ doTestInvokeAllFromTaskDoesNotDeadlockSameSearcher(executorService);
+ return null;
+ })
+ .get();
+ }
+
+ private static void doTestInvokeAllFromTaskDoesNotDeadlockSameSearcher(Executor executor)
+ throws IOException {
try (Directory dir = newDirectory();
RandomIndexWriter iw = new RandomIndexWriter(random(), dir)) {
for (int i = 0; i < 500; i++) {
@@ -119,7 +129,7 @@ public class TestTaskExecutor extends LuceneTestCase {
}
try (DirectoryReader reader = iw.getReader()) {
IndexSearcher searcher =
- new IndexSearcher(reader, executorService) {
+ new IndexSearcher(reader, executor) {
@Override
protected LeafSlice[] slices(List leaves) {
return slices(leaves, 1, 1);
@@ -172,7 +182,20 @@ public class TestTaskExecutor extends LuceneTestCase {
}
}
- public void testInvokeAllFromTaskDoesNotDeadlockMultipleSearchers() throws IOException {
+ public void testInvokeAllFromTaskDoesNotDeadlockMultipleSearchers() throws Exception {
+ doTestInvokeAllFromTaskDoesNotDeadlockMultipleSearchers(executorService);
+ doTestInvokeAllFromTaskDoesNotDeadlockMultipleSearchers(Runnable::run);
+ executorService
+ .submit(
+ () -> {
+ doTestInvokeAllFromTaskDoesNotDeadlockMultipleSearchers(executorService);
+ return null;
+ })
+ .get();
+ }
+
+ private static void doTestInvokeAllFromTaskDoesNotDeadlockMultipleSearchers(Executor executor)
+ throws IOException {
try (Directory dir = newDirectory();
RandomIndexWriter iw = new RandomIndexWriter(random(), dir)) {
for (int i = 0; i < 500; i++) {
@@ -180,7 +203,7 @@ public class TestTaskExecutor extends LuceneTestCase {
}
try (DirectoryReader reader = iw.getReader()) {
IndexSearcher searcher =
- new IndexSearcher(reader, executorService) {
+ new IndexSearcher(reader, executor) {
@Override
protected LeafSlice[] slices(List leaves) {
return slices(leaves, 1, 1);
@@ -202,7 +225,7 @@ public class TestTaskExecutor extends LuceneTestCase {
// searcher has its own
// TaskExecutor, the safeguard is shared among all the searchers that get
// the same executor
- IndexSearcher indexSearcher = new IndexSearcher(reader, executorService);
+ IndexSearcher indexSearcher = new IndexSearcher(reader, executor);
indexSearcher
.getTaskExecutor()
.invokeAll(Collections.singletonList(() -> null));
@@ -234,11 +257,8 @@ public class TestTaskExecutor extends LuceneTestCase {
TaskExecutor taskExecutor =
new TaskExecutor(
command -> {
- executorService.execute(
- () -> {
- tasksStarted.incrementAndGet();
- command.run();
- });
+ tasksStarted.incrementAndGet();
+ command.run();
});
AtomicInteger tasksExecuted = new AtomicInteger(0);
List> callables = new ArrayList<>();
@@ -251,14 +271,14 @@ public class TestTaskExecutor extends LuceneTestCase {
for (int i = 0; i < tasksWithNormalExit; i++) {
callables.add(
() -> {
- tasksExecuted.incrementAndGet();
- return null;
+ throw new AssertionError(
+ "must not be called since the first task failing cancels all subsequent tasks");
});
}
expectThrows(RuntimeException.class, () -> taskExecutor.invokeAll(callables));
assertEquals(1, tasksExecuted.get());
// the callables are technically all run, but the cancelled ones will be no-op
- assertEquals(100, tasksStarted.get());
+ assertEquals(tasksWithNormalExit, tasksStarted.get());
}
/**
@@ -308,7 +328,7 @@ public class TestTaskExecutor extends LuceneTestCase {
}
public void testCancelTasksOnException() {
- TaskExecutor taskExecutor = new TaskExecutor(executorService);
+ TaskExecutor taskExecutor = new TaskExecutor(Runnable::run);
final int numTasks = random().nextInt(10, 50);
final int throwingTask = random().nextInt(numTasks);
boolean error = random().nextBoolean();
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTermQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestTermQuery.java
index 3b35f67cad1..8911500546b 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestTermQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestTermQuery.java
@@ -43,6 +43,7 @@ import org.apache.lucene.tests.search.QueryUtils;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOBooleanSupplier;
import org.apache.lucene.util.IOUtils;
public class TestTermQuery extends LuceneTestCase {
@@ -259,6 +260,11 @@ public class TestTermQuery extends LuceneTestCase {
throw new AssertionError("no seek");
}
+ @Override
+ public IOBooleanSupplier prepareSeekExact(BytesRef text) throws IOException {
+ throw new AssertionError("no seek");
+ }
+
@Override
public void seekExact(BytesRef term, TermState state) throws IOException {
throw new AssertionError("no seek");
diff --git a/lucene/core/src/test/org/apache/lucene/search/knn/TestMultiLeafKnnCollector.java b/lucene/core/src/test/org/apache/lucene/search/knn/TestMultiLeafKnnCollector.java
new file mode 100644
index 00000000000..8466293f61a
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/knn/TestMultiLeafKnnCollector.java
@@ -0,0 +1,62 @@
+/*
+ * 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.knn;
+
+import org.apache.lucene.search.TopKnnCollector;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.util.hnsw.BlockingFloatHeap;
+
+public class TestMultiLeafKnnCollector extends LuceneTestCase {
+
+ /** Validates a fix for GH#13462 */
+ public void testGlobalScoreCoordination() {
+ int k = 7;
+ BlockingFloatHeap globalHeap = new BlockingFloatHeap(k);
+ MultiLeafKnnCollector collector1 =
+ new MultiLeafKnnCollector(k, globalHeap, new TopKnnCollector(k, Integer.MAX_VALUE));
+ MultiLeafKnnCollector collector2 =
+ new MultiLeafKnnCollector(k, globalHeap, new TopKnnCollector(k, Integer.MAX_VALUE));
+
+ // Collect k (7) hits in collector1 with scores [100, 106]:
+ for (int i = 0; i < k; i++) {
+ collector1.collect(0, 100f + i);
+ }
+
+ // The global heap should be updated since k hits were collected, and have a min score of
+ // 100:
+ assertEquals(100f, globalHeap.peek(), 0f);
+ assertEquals(100f, collector1.minCompetitiveSimilarity(), 0f);
+
+ // Collect k (7) hits in collector2 with only two that are competitive (200 and 300),
+ // which also forces an update of the global heap with collector2's hits. This is a tricky
+ // case where the heap will not be fully ordered, so it ensures global queue updates don't
+ // incorrectly short-circuit (see GH#13462):
+ collector2.collect(0, 10f);
+ collector2.collect(0, 11f);
+ collector2.collect(0, 12f);
+ collector2.collect(0, 13f);
+ collector2.collect(0, 200f);
+ collector2.collect(0, 14f);
+ collector2.collect(0, 300f);
+
+ // At this point, our global heap should contain [102, 103, 104, 105, 106, 200, 300] since
+ // values 200 and 300 from collector2 should have pushed out 100 and 101 from collector1.
+ // The min value on the global heap should be 102:
+ assertEquals(102f, globalHeap.peek(), 0f);
+ assertEquals(102f, collector2.minCompetitiveSimilarity(), 0f);
+ }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestMMapDirectory.java b/lucene/core/src/test/org/apache/lucene/store/TestMMapDirectory.java
index 39d3dbda9ac..f7c49c9b661 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestMMapDirectory.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestMMapDirectory.java
@@ -19,9 +19,14 @@ package org.apache.lucene.store;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Random;
+import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
import org.apache.lucene.tests.store.BaseDirectoryTestCase;
import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.NamedThreadFactory;
/** Tests MMapDirectory */
// See: https://issues.apache.org/jira/browse/SOLR-12028 Tests cannot remove files on Windows
@@ -117,4 +122,54 @@ public class TestMMapDirectory extends BaseDirectoryTestCase {
}
}
}
+
+ // Opens the input with ReadAdvice.READONCE to ensure slice and clone are appropriately confined
+ public void testConfined() throws Exception {
+ final int size = 16;
+ byte[] bytes = new byte[size];
+ random().nextBytes(bytes);
+
+ try (Directory dir = new MMapDirectory(createTempDir("testConfined"))) {
+ try (IndexOutput out = dir.createOutput("test", IOContext.DEFAULT)) {
+ out.writeBytes(bytes, 0, bytes.length);
+ }
+
+ try (var in = dir.openInput("test", IOContext.READONCE);
+ var executor = Executors.newFixedThreadPool(1, new NamedThreadFactory("testConfined"))) {
+ // ensure accessible
+ assertEquals(16L, in.slice("test", 0, in.length()).length());
+ assertEquals(15L, in.slice("test", 1, in.length() - 1).length());
+
+ // ensure not accessible
+ Callable