mirror of https://github.com/apache/lucene.git
Merge remote-tracking branch 'upstream/main' into feature/scalar-quantized-off-heap-scoring
This commit is contained in:
commit
2640493d49
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -68,7 +68,6 @@ public class Checksum extends DefaultTask {
|
|||
|
||||
public Checksum() {
|
||||
outputDir = new File(getProject().getBuildDir(), "checksums");
|
||||
algorithm = Algorithm.SHA256;
|
||||
}
|
||||
|
||||
@InputFiles
|
||||
|
|
|
@ -67,6 +67,13 @@
|
|||
</maintainer>
|
||||
|
||||
<!-- NOTE: please insert releases in numeric order, NOT chronologically. -->
|
||||
<release>
|
||||
<Version>
|
||||
<name>lucene-9.11.1</name>
|
||||
<created>2024-06-27</created>
|
||||
<revision>9.11.1</revision>
|
||||
</Version>
|
||||
</release>.
|
||||
<release>
|
||||
<Version>
|
||||
<name>lucene-9.11.0</name>
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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<BytesRef> 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";
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
|
@ -40,3 +40,4 @@
|
|||
9.9.2
|
||||
9.10.0
|
||||
9.11.0
|
||||
9.11.1
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -90,10 +90,15 @@ public class FSTDictionary implements IndexDictionary {
|
|||
}
|
||||
PositiveIntOutputs fstOutputs = PositiveIntOutputs.getSingleton();
|
||||
FST.FSTMetadata<Long> metadata = FST.readMetadata(fstDataInput, fstOutputs);
|
||||
FST<Long> fst =
|
||||
isFSTOnHeap
|
||||
? new FST<>(metadata, fstDataInput)
|
||||
: new FST<>(metadata, fstDataInput, new OffHeapFSTStore());
|
||||
FST<Long> 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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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() {}
|
||||
|
|
|
@ -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<T> extends KnnFieldVectorsWriter<T> {
|
||||
|
||||
/**
|
||||
* 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<T> indexingDelegate;
|
||||
public abstract List<T> 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<T> 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();
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -40,7 +40,7 @@ public class OrdToDocDISIReaderConfiguration {
|
|||
* <p>Within outputMeta the format is as follows:
|
||||
*
|
||||
* <ul>
|
||||
* <li><b>[int8]</b> if equals to -2, empty - no vectory values. If equals to -1, dense – all
|
||||
* <li><b>[int8]</b> 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.
|
||||
* <li>DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput,
|
||||
|
|
|
@ -56,8 +56,8 @@ import org.apache.lucene.store.IndexOutput;
|
|||
* <li><b>[vlong]</b> length of this field's vectors, in bytes
|
||||
* <li><b>[vint]</b> dimension of this field's vectors
|
||||
* <li><b>[int]</b> the number of documents having values for this field
|
||||
* <li><b>[int8]</b> if equals to -1, dense – all documents have values for a field. If equals to
|
||||
* 0, sparse – some documents missing values.
|
||||
* <li><b>[int8]</b> 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.
|
||||
* <li>DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)}
|
||||
* <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
|
||||
* that only in sparse case
|
||||
|
|
|
@ -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<T> 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<byte[]>) indexWriter) {
|
||||
case BYTE -> new Lucene99FlatVectorsWriter.FieldWriter<byte[]>(fieldInfo) {
|
||||
@Override
|
||||
public byte[] copyValue(byte[] value) {
|
||||
return ArrayUtil.copyOfSubArray(value, 0, dim);
|
||||
}
|
||||
};
|
||||
case FLOAT32 -> new Lucene99FlatVectorsWriter.FieldWriter<>(
|
||||
fieldInfo, (KnnFieldVectorsWriter<float[]>) indexWriter) {
|
||||
case FLOAT32 -> new Lucene99FlatVectorsWriter.FieldWriter<float[]>(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<T> 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<T> 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
|
||||
|
|
|
@ -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;
|
|||
* <li><b>[vlong]</b> length of this field's index data, in bytes
|
||||
* <li><b>[vint]</b> dimension of this field's vectors
|
||||
* <li><b>[int]</b> the number of documents having values for this field
|
||||
* <li><b>[int8]</b> if equals to -1, dense – all documents have values for a field. If equals to
|
||||
* 0, sparse – some documents missing values.
|
||||
* <li>DocIds were encoded by {@link IndexedDISI#writeBitSet(DocIdSetIterator, IndexOutput, byte)}
|
||||
* <li>OrdToDoc was encoded by {@link org.apache.lucene.util.packed.DirectMonotonicWriter}, note
|
||||
* that only in sparse case
|
||||
* <li><b>[vint]</b> the maximum number of connections (neighbours) that each node can have
|
||||
* <li><b>[vint]</b> number of levels in the graph
|
||||
* <li>Graph nodes by level. For each level
|
||||
|
|
|
@ -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<RandomVectorScorer> 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);
|
||||
|
|
|
@ -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<T> vectors;
|
||||
private final HnswGraphBuilder hnswGraphBuilder;
|
||||
private int lastDocID = -1;
|
||||
private int node = 0;
|
||||
private final FlatFieldVectorsWriter<T> 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<byte[]>(scorer, fieldInfo, M, beamWidth, infoStream);
|
||||
case FLOAT32 -> new FieldWriter<float[]>(scorer, fieldInfo, M, beamWidth, infoStream);
|
||||
case BYTE -> new FieldWriter<>(
|
||||
scorer,
|
||||
(FlatFieldVectorsWriter<byte[]>) flatFieldVectorsWriter,
|
||||
fieldInfo,
|
||||
M,
|
||||
beamWidth,
|
||||
infoStream);
|
||||
case FLOAT32 -> new FieldWriter<>(
|
||||
scorer,
|
||||
(FlatFieldVectorsWriter<float[]>) flatFieldVectorsWriter,
|
||||
fieldInfo,
|
||||
M,
|
||||
beamWidth,
|
||||
infoStream);
|
||||
};
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
FieldWriter(
|
||||
FlatVectorsScorer scorer, FieldInfo fieldInfo, int M, int beamWidth, InfoStream infoStream)
|
||||
FlatVectorsScorer scorer,
|
||||
FlatFieldVectorsWriter<T> 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<byte[]>) vectors, fieldInfo.getVectorDimension()));
|
||||
(List<byte[]>) flatFieldVectorsWriter.getVectors(),
|
||||
fieldInfo.getVectorDimension()));
|
||||
case FLOAT32 -> scorer.getRandomVectorScorerSupplier(
|
||||
fieldInfo.getVectorSimilarityFunction(),
|
||||
RandomAccessVectorValues.fromFloats(
|
||||
(List<float[]>) vectors, fieldInfo.getVectorDimension()));
|
||||
(List<float[]>) 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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<float[]>) 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<float[]> {
|
||||
private static final long SHALLOW_SIZE = shallowSizeOfInstance(FieldWriter.class);
|
||||
private final List<float[]> 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<float[]> flatFieldVectorsWriter;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
FieldWriter(
|
||||
Float confidenceInterval,
|
||||
byte bits,
|
||||
boolean compress,
|
||||
FieldInfo fieldInfo,
|
||||
InfoStream infoStream,
|
||||
KnnFieldVectorsWriter<?> indexWriter) {
|
||||
super((KnnFieldVectorsWriter<float[]>) indexWriter);
|
||||
FlatFieldVectorsWriter<float[]> 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<float[]> 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<float[]> getVectors() {
|
||||
return flatFieldVectorsWriter.getVectors();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsWithFieldSet getDocsWithFieldSet() {
|
||||
return flatFieldVectorsWriter.getDocsWithFieldSet();
|
||||
}
|
||||
}
|
||||
|
||||
static class FloatVectorWrapper extends FloatVectorValues {
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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!
|
||||
*
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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--;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -89,7 +89,7 @@ public final class Term implements Comparable<Term>, 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<Term>, 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<Term>, 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<Term>, 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<Term>, Accountable {
|
|||
* <p>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<Term>, Accountable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final String toString() {
|
||||
public String toString() {
|
||||
return field + ":" + text();
|
||||
}
|
||||
|
||||
|
|
|
@ -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<Callable<TermStateInfo>> 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<TermStateInfo> 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 <code>null</code> 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.
|
||||
*
|
||||
* <p>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 <code>null</code> if no {@link
|
||||
* TermState} for the reader was registered
|
||||
* @return a Supplier for a TermState.
|
||||
*/
|
||||
public TermState get(LeafReaderContext ctx) throws IOException {
|
||||
public IOSupplier<TermState> 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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()}.
|
||||
*
|
||||
* <p><b>NOTE</b>: It is illegal to call other methods on this {@link TermsEnum} after calling
|
||||
* this method until {@link IOBooleanSupplier#get()} is called.
|
||||
*
|
||||
* <p><b>NOTE</b>: This may return {@code null} if this {@link TermsEnum} can identify that the
|
||||
* term may not exist without performing any I/O.
|
||||
*
|
||||
* <p><b>NOTE</b>: 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");
|
||||
|
|
|
@ -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<LeafReaderContext> 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;
|
||||
|
|
|
@ -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<LeafReaderContext> leaves = readerContext.leaves();
|
||||
TermStates newCtx = new TermStates(readerContext);
|
||||
for (int i = 0; i < leaves.size(); ++i) {
|
||||
TermState termState = ctx.get(leaves.get(i));
|
||||
IOSupplier<TermState> supplier = ctx.get(leaves.get(i));
|
||||
if (supplier == null) {
|
||||
continue;
|
||||
}
|
||||
TermState termState = supplier.get();
|
||||
if (termState == null) {
|
||||
continue;
|
||||
}
|
||||
|
|
|
@ -194,6 +194,15 @@ final class DisjunctionMatchesIterator implements MatchesIterator {
|
|||
new PriorityQueue<MatchesIterator>(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());
|
||||
|
|
|
@ -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;
|
|||
* <p>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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
*
|
||||
* <p>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
|
||||
*
|
||||
* <p>Should only be called after {@link #next()} has returned {@code true}
|
||||
*/
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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<PostingsEnum> postings = new ArrayList<>();
|
||||
|
||||
for (Term term : terms) {
|
||||
TermState termState = termStates.get(term).get(context);
|
||||
IOSupplier<TermState> supplier = termStates.get(term).get(context);
|
||||
TermState termState = supplier == null ? null : supplier.get();
|
||||
if (termState != null) {
|
||||
termsEnum.seekExact(term.bytes(), termState);
|
||||
postings.add(
|
||||
|
|
|
@ -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<TermState> 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";
|
||||
|
|
|
@ -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<PostingsEnum> iterators = new ArrayList<>();
|
||||
List<ImpactsEnum> impacts = new ArrayList<>();
|
||||
List<Float> termBoosts = new ArrayList<>();
|
||||
@SuppressWarnings({"rawtypes", "unchecked"})
|
||||
IOSupplier<TermState>[] 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<PostingsEnum> iterators;
|
||||
List<ImpactsEnum> impacts;
|
||||
List<Float> 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<TermState> 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
|
||||
|
|
|
@ -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<Integer> 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 <T> the return type of all the callables
|
||||
*/
|
||||
private static final class TaskGroup<T> {
|
||||
private final Collection<RunnableFuture<T>> futures;
|
||||
private final List<RunnableFuture<T>> futures;
|
||||
|
||||
TaskGroup(Collection<Callable<T>> callables) {
|
||||
List<RunnableFuture<T>> tasks = new ArrayList<>(callables.size());
|
||||
for (Callable<T> callable : callables) {
|
||||
tasks.add(createTask(callable));
|
||||
}
|
||||
this.futures = Collections.unmodifiableCollection(tasks);
|
||||
this.futures = Collections.unmodifiableList(tasks);
|
||||
}
|
||||
|
||||
RunnableFuture<T> createTask(Callable<T> 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<T> 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<T> results = new ArrayList<>(futures.size());
|
||||
for (Future<T> future : futures) {
|
||||
List<T> results = new ArrayList<>(count);
|
||||
for (int i = 0; i < count; i++) {
|
||||
Future<T> 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?";
|
||||
|
|
|
@ -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<TermState> 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<TermState> 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 =
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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}.
|
||||
*
|
||||
* <p>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")
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
* <p>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}.
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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));
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
|
|
|
@ -417,16 +417,7 @@ public final class FST<T> implements Accountable {
|
|||
* maxBlockBits set to {@link #DEFAULT_MAX_BLOCK_BITS}
|
||||
*/
|
||||
public FST(FSTMetadata<T> 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<T> 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. */
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -72,12 +72,13 @@ public final class BlockingFloatHeap {
|
|||
* <p>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 {
|
||||
|
|
|
@ -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<MemorySegment> 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);
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -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"));
|
||||
}
|
||||
}
|
|
@ -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 =
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -266,7 +266,7 @@ public class TestIndexSearcher extends LuceneTestCase {
|
|||
IOUtils.close(r, dir);
|
||||
}
|
||||
|
||||
public void testSlicesAllOffloadedToTheExecutor() throws IOException {
|
||||
public void testSlicesOffloadedToTheExecutor() throws IOException {
|
||||
List<LeafReaderContext> 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() {
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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<LeafReaderContext> 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<LeafReaderContext> 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<Callable<Void>> 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();
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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<Object> task1 = () -> in.slice("test", 0, in.length());
|
||||
var x = expectThrows(ISE, () -> getAndUnwrap(executor.submit(task1)));
|
||||
assertTrue(x.getMessage().contains("confined"));
|
||||
|
||||
int offset = random().nextInt((int) in.length());
|
||||
int length = (int) in.length() - offset;
|
||||
Callable<Object> task2 = () -> in.slice("test", offset, length);
|
||||
x = expectThrows(ISE, () -> getAndUnwrap(executor.submit(task2)));
|
||||
assertTrue(x.getMessage().contains("confined"));
|
||||
|
||||
// slice.slice
|
||||
var slice = in.slice("test", 0, in.length());
|
||||
Callable<Object> task3 = () -> slice.slice("test", 0, in.length());
|
||||
x = expectThrows(ISE, () -> getAndUnwrap(executor.submit(task3)));
|
||||
assertTrue(x.getMessage().contains("confined"));
|
||||
// slice.clone
|
||||
x = expectThrows(ISE, () -> getAndUnwrap(executor.submit(slice::clone)));
|
||||
assertTrue(x.getMessage().contains("confined"));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static final Class<IllegalStateException> ISE = IllegalStateException.class;
|
||||
|
||||
static Object getAndUnwrap(Future<Object> future) throws Throwable {
|
||||
try {
|
||||
return future.get();
|
||||
} catch (ExecutionException ee) {
|
||||
throw ee.getCause();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -276,4 +276,81 @@ public class TestVectorUtil extends LuceneTestCase {
|
|||
u[1] = -v[0];
|
||||
assertEquals(0, VectorUtil.cosine(u, v), DELTA);
|
||||
}
|
||||
|
||||
interface ToIntBiFunction {
|
||||
int apply(byte[] a, byte[] b);
|
||||
}
|
||||
|
||||
public void testBasicXorBitCount() {
|
||||
testBasicXorBitCountImpl(VectorUtil::xorBitCount);
|
||||
testBasicXorBitCountImpl(VectorUtil::xorBitCountInt);
|
||||
testBasicXorBitCountImpl(VectorUtil::xorBitCountLong);
|
||||
// test sanity
|
||||
testBasicXorBitCountImpl(TestVectorUtil::xorBitCount);
|
||||
}
|
||||
|
||||
void testBasicXorBitCountImpl(ToIntBiFunction xorBitCount) {
|
||||
assertEquals(0, xorBitCount.apply(new byte[] {1}, new byte[] {1}));
|
||||
assertEquals(0, xorBitCount.apply(new byte[] {1, 2, 3}, new byte[] {1, 2, 3}));
|
||||
assertEquals(1, xorBitCount.apply(new byte[] {1, 2, 3}, new byte[] {0, 2, 3}));
|
||||
assertEquals(2, xorBitCount.apply(new byte[] {1, 2, 3}, new byte[] {0, 6, 3}));
|
||||
assertEquals(3, xorBitCount.apply(new byte[] {1, 2, 3}, new byte[] {0, 6, 7}));
|
||||
assertEquals(4, xorBitCount.apply(new byte[] {1, 2, 3}, new byte[] {2, 6, 7}));
|
||||
|
||||
// 32-bit / int boundary
|
||||
assertEquals(0, xorBitCount.apply(new byte[] {1, 2, 3, 4}, new byte[] {1, 2, 3, 4}));
|
||||
assertEquals(1, xorBitCount.apply(new byte[] {1, 2, 3, 4}, new byte[] {0, 2, 3, 4}));
|
||||
assertEquals(0, xorBitCount.apply(new byte[] {1, 2, 3, 4, 5}, new byte[] {1, 2, 3, 4, 5}));
|
||||
assertEquals(1, xorBitCount.apply(new byte[] {1, 2, 3, 4, 5}, new byte[] {0, 2, 3, 4, 5}));
|
||||
|
||||
// 64-bit / long boundary
|
||||
assertEquals(
|
||||
0,
|
||||
xorBitCount.apply(
|
||||
new byte[] {1, 2, 3, 4, 5, 6, 7, 8}, new byte[] {1, 2, 3, 4, 5, 6, 7, 8}));
|
||||
assertEquals(
|
||||
1,
|
||||
xorBitCount.apply(
|
||||
new byte[] {1, 2, 3, 4, 5, 6, 7, 8}, new byte[] {0, 2, 3, 4, 5, 6, 7, 8}));
|
||||
|
||||
assertEquals(
|
||||
0,
|
||||
xorBitCount.apply(
|
||||
new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9}, new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
assertEquals(
|
||||
1,
|
||||
xorBitCount.apply(
|
||||
new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9}, new byte[] {0, 2, 3, 4, 5, 6, 7, 8, 9}));
|
||||
}
|
||||
|
||||
public void testXorBitCount() {
|
||||
int iterations = atLeast(100);
|
||||
for (int i = 0; i < iterations; i++) {
|
||||
int size = random().nextInt(1024);
|
||||
byte[] a = new byte[size];
|
||||
byte[] b = new byte[size];
|
||||
random().nextBytes(a);
|
||||
random().nextBytes(b);
|
||||
|
||||
int expected = xorBitCount(a, b);
|
||||
assertEquals(expected, VectorUtil.xorBitCount(a, b));
|
||||
assertEquals(expected, VectorUtil.xorBitCountInt(a, b));
|
||||
assertEquals(expected, VectorUtil.xorBitCountLong(a, b));
|
||||
}
|
||||
}
|
||||
|
||||
private static int xorBitCount(byte[] a, byte[] b) {
|
||||
int res = 0;
|
||||
for (int i = 0; i < a.length; i++) {
|
||||
byte x = a[i];
|
||||
byte y = b[i];
|
||||
for (int j = 0; j < Byte.SIZE; j++) {
|
||||
if (x == y) break;
|
||||
if ((x & 0x01) != (y & 0x01)) res++;
|
||||
x = (byte) ((x & 0xFF) >> 1);
|
||||
y = (byte) ((y & 0xFF) >> 1);
|
||||
}
|
||||
}
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,10 +29,11 @@
|
|||
|
||||
package org.apache.lucene.util.automaton;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
import org.apache.lucene.internal.hppc.IntArrayList;
|
||||
import org.apache.lucene.internal.hppc.IntCursor;
|
||||
import org.apache.lucene.internal.hppc.IntHashSet;
|
||||
|
||||
/**
|
||||
* Operations for minimizing automata.
|
||||
|
@ -75,13 +76,9 @@ public final class MinimizationOperations {
|
|||
final int[] sigma = a.getStartPoints();
|
||||
final int sigmaLen = sigma.length, statesLen = a.getNumStates();
|
||||
|
||||
@SuppressWarnings({"rawtypes", "unchecked"})
|
||||
final ArrayList<Integer>[][] reverse =
|
||||
(ArrayList<Integer>[][]) new ArrayList[statesLen][sigmaLen];
|
||||
@SuppressWarnings({"rawtypes", "unchecked"})
|
||||
final HashSet<Integer>[] partition = (HashSet<Integer>[]) new HashSet[statesLen];
|
||||
@SuppressWarnings({"rawtypes", "unchecked"})
|
||||
final ArrayList<Integer>[] splitblock = (ArrayList<Integer>[]) new ArrayList[statesLen];
|
||||
final IntArrayList[][] reverse = new IntArrayList[statesLen][sigmaLen];
|
||||
final IntHashSet[] partition = new IntHashSet[statesLen];
|
||||
final IntArrayList[] splitblock = new IntArrayList[statesLen];
|
||||
final int[] block = new int[statesLen];
|
||||
final StateList[][] active = new StateList[statesLen][sigmaLen];
|
||||
final StateListNode[][] active2 = new StateListNode[statesLen][sigmaLen];
|
||||
|
@ -91,10 +88,10 @@ public final class MinimizationOperations {
|
|||
refine = new BitSet(statesLen),
|
||||
refine2 = new BitSet(statesLen);
|
||||
for (int q = 0; q < statesLen; q++) {
|
||||
splitblock[q] = new ArrayList<>();
|
||||
partition[q] = new HashSet<>();
|
||||
splitblock[q] = new IntArrayList();
|
||||
partition[q] = new IntHashSet();
|
||||
for (int x = 0; x < sigmaLen; x++) {
|
||||
active[q][x] = new StateList();
|
||||
active[q][x] = StateList.EMPTY;
|
||||
}
|
||||
}
|
||||
// find initial partition and reverse edges
|
||||
|
@ -106,9 +103,9 @@ public final class MinimizationOperations {
|
|||
transition.source = q;
|
||||
transition.transitionUpto = -1;
|
||||
for (int x = 0; x < sigmaLen; x++) {
|
||||
final ArrayList<Integer>[] r = reverse[a.next(transition, sigma[x])];
|
||||
final IntArrayList[] r = reverse[a.next(transition, sigma[x])];
|
||||
if (r[x] == null) {
|
||||
r[x] = new ArrayList<>();
|
||||
r[x] = new IntArrayList();
|
||||
}
|
||||
r[x].add(q);
|
||||
}
|
||||
|
@ -116,9 +113,15 @@ public final class MinimizationOperations {
|
|||
// initialize active sets
|
||||
for (int j = 0; j <= 1; j++) {
|
||||
for (int x = 0; x < sigmaLen; x++) {
|
||||
for (int q : partition[j]) {
|
||||
for (IntCursor qCursor : partition[j]) {
|
||||
int q = qCursor.value;
|
||||
if (reverse[q][x] != null) {
|
||||
active2[q][x] = active[j][x].add(q);
|
||||
StateList stateList = active[j][x];
|
||||
if (stateList == StateList.EMPTY) {
|
||||
stateList = new StateList();
|
||||
active[j][x] = stateList;
|
||||
}
|
||||
active2[q][x] = stateList.add(q);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -143,9 +146,10 @@ public final class MinimizationOperations {
|
|||
pending2.clear(x * statesLen + p);
|
||||
// find states that need to be split off their blocks
|
||||
for (StateListNode m = active[p][x].first; m != null; m = m.next) {
|
||||
final ArrayList<Integer> r = reverse[m.q][x];
|
||||
final IntArrayList r = reverse[m.q][x];
|
||||
if (r != null) {
|
||||
for (int i : r) {
|
||||
for (IntCursor iCursor : r) {
|
||||
final int i = iCursor.value;
|
||||
if (!split.get(i)) {
|
||||
split.set(i);
|
||||
final int j = block[i];
|
||||
|
@ -161,11 +165,12 @@ public final class MinimizationOperations {
|
|||
|
||||
// refine blocks
|
||||
for (int j = refine.nextSetBit(0); j >= 0; j = refine.nextSetBit(j + 1)) {
|
||||
final ArrayList<Integer> sb = splitblock[j];
|
||||
final IntArrayList sb = splitblock[j];
|
||||
if (sb.size() < partition[j].size()) {
|
||||
final HashSet<Integer> b1 = partition[j];
|
||||
final HashSet<Integer> b2 = partition[k];
|
||||
for (int s : sb) {
|
||||
final IntHashSet b1 = partition[j];
|
||||
final IntHashSet b2 = partition[k];
|
||||
for (IntCursor iCursor : sb) {
|
||||
final int s = iCursor.value;
|
||||
b1.remove(s);
|
||||
b2.add(s);
|
||||
block[s] = k;
|
||||
|
@ -173,7 +178,12 @@ public final class MinimizationOperations {
|
|||
final StateListNode sn = active2[s][c];
|
||||
if (sn != null && sn.sl == active[j][c]) {
|
||||
sn.remove();
|
||||
active2[s][c] = active[k][c].add(s);
|
||||
StateList stateList = active[k][c];
|
||||
if (stateList == StateList.EMPTY) {
|
||||
stateList = new StateList();
|
||||
active[k][c] = stateList;
|
||||
}
|
||||
active2[s][c] = stateList.add(s);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -191,7 +201,8 @@ public final class MinimizationOperations {
|
|||
k++;
|
||||
}
|
||||
refine2.clear(j);
|
||||
for (int s : sb) {
|
||||
for (IntCursor iCursor : sb) {
|
||||
final int s = iCursor.value;
|
||||
split.clear(s);
|
||||
}
|
||||
sb.clear();
|
||||
|
@ -215,17 +226,11 @@ public final class MinimizationOperations {
|
|||
for (int n = 0; n < k; n++) {
|
||||
// System.out.println(" n=" + n);
|
||||
|
||||
boolean isInitial = false;
|
||||
for (int q : partition[n]) {
|
||||
if (q == 0) {
|
||||
isInitial = true;
|
||||
// System.out.println(" isInitial!");
|
||||
break;
|
||||
}
|
||||
}
|
||||
boolean isInitial = partition[n].contains(0);
|
||||
|
||||
int newState;
|
||||
if (isInitial) {
|
||||
// System.out.println(" isInitial!");
|
||||
newState = 0;
|
||||
} else {
|
||||
newState = result.createState();
|
||||
|
@ -233,7 +238,8 @@ public final class MinimizationOperations {
|
|||
|
||||
// System.out.println(" newState=" + newState);
|
||||
|
||||
for (int q : partition[n]) {
|
||||
for (IntCursor qCursor : partition[n]) {
|
||||
int q = qCursor.value;
|
||||
stateMap[q] = newState;
|
||||
// System.out.println(" q=" + q + " isAccept?=" + a.isAccept(q));
|
||||
result.setAccept(newState, a.isAccept(q));
|
||||
|
@ -268,11 +274,16 @@ public final class MinimizationOperations {
|
|||
|
||||
static final class StateList {
|
||||
|
||||
// Empty list that should never be mutated, used as a memory saving optimization instead of null
|
||||
// so we don't need to branch the read path in #minimize
|
||||
static final StateList EMPTY = new StateList();
|
||||
|
||||
int size;
|
||||
|
||||
StateListNode first, last;
|
||||
|
||||
StateListNode add(int q) {
|
||||
assert this != EMPTY;
|
||||
return new StateListNode(q, this);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -93,7 +93,10 @@ public class Test2BFSTOffHeap extends LuceneTestCase {
|
|||
FST.FSTMetadata<Object> fstMetadata = fstCompiler.compile();
|
||||
indexOutput.close();
|
||||
try (IndexInput indexInput = dir.openInput("fst", IOContext.DEFAULT)) {
|
||||
FST<Object> fst = new FST<>(fstMetadata, indexInput, new OffHeapFSTStore());
|
||||
FST<Object> fst =
|
||||
FST.fromFSTReader(
|
||||
fstMetadata,
|
||||
new OffHeapFSTStore(indexInput, indexInput.getFilePointer(), fstMetadata));
|
||||
|
||||
for (int verify = 0; verify < 2; verify++) {
|
||||
System.out.println(
|
||||
|
@ -181,7 +184,10 @@ public class Test2BFSTOffHeap extends LuceneTestCase {
|
|||
FST.FSTMetadata<BytesRef> fstMetadata = fstCompiler.compile();
|
||||
indexOutput.close();
|
||||
try (IndexInput indexInput = dir.openInput("fst", IOContext.DEFAULT)) {
|
||||
FST<BytesRef> fst = new FST<>(fstMetadata, indexInput, new OffHeapFSTStore());
|
||||
FST<BytesRef> fst =
|
||||
FST.fromFSTReader(
|
||||
fstMetadata,
|
||||
new OffHeapFSTStore(indexInput, indexInput.getFilePointer(), fstMetadata));
|
||||
for (int verify = 0; verify < 2; verify++) {
|
||||
|
||||
System.out.println(
|
||||
|
@ -266,7 +272,10 @@ public class Test2BFSTOffHeap extends LuceneTestCase {
|
|||
FST.FSTMetadata<Long> fstMetadata = fstCompiler.compile();
|
||||
indexOutput.close();
|
||||
try (IndexInput indexInput = dir.openInput("fst", IOContext.DEFAULT)) {
|
||||
FST<Long> fst = new FST<>(fstMetadata, indexInput, new OffHeapFSTStore());
|
||||
FST<Long> fst =
|
||||
FST.fromFSTReader(
|
||||
fstMetadata,
|
||||
new OffHeapFSTStore(indexInput, indexInput.getFilePointer(), fstMetadata));
|
||||
|
||||
for (int verify = 0; verify < 2; verify++) {
|
||||
|
||||
|
|
|
@ -154,7 +154,7 @@ public class StringValueFacetCounts extends Facets {
|
|||
final BytesRef term = docValues.lookupOrd(sparseCount.key);
|
||||
labelValues.add(new LabelAndValue(term.utf8ToString(), count));
|
||||
}
|
||||
} else {
|
||||
} else if (denseCounts != null) {
|
||||
for (int i = 0; i < denseCounts.length; i++) {
|
||||
int count = denseCounts[i];
|
||||
if (count != 0) {
|
||||
|
@ -180,8 +180,6 @@ public class StringValueFacetCounts extends Facets {
|
|||
topN = Math.min(topN, cardinality);
|
||||
TopOrdAndIntQueue q = null;
|
||||
TopOrdAndIntQueue.OrdAndInt reuse = null;
|
||||
int bottomCount = 0;
|
||||
int bottomOrd = Integer.MAX_VALUE;
|
||||
int childCount = 0; // total number of labels with non-zero count
|
||||
|
||||
if (sparseCounts != null) {
|
||||
|
@ -189,7 +187,22 @@ public class StringValueFacetCounts extends Facets {
|
|||
childCount++; // every count in sparseValues should be non-zero
|
||||
int ord = sparseCount.key;
|
||||
int count = sparseCount.value;
|
||||
if (count > bottomCount || (count == bottomCount && ord < bottomOrd)) {
|
||||
if (q == null) {
|
||||
// Lazy init for sparse case:
|
||||
q = new TopOrdAndIntQueue(topN);
|
||||
}
|
||||
if (reuse == null) {
|
||||
reuse = (TopOrdAndIntQueue.OrdAndInt) q.newOrdAndValue();
|
||||
}
|
||||
reuse.ord = ord;
|
||||
reuse.value = count;
|
||||
reuse = (TopOrdAndIntQueue.OrdAndInt) q.insertWithOverflow(reuse);
|
||||
}
|
||||
} else if (denseCounts != null) {
|
||||
for (int i = 0; i < denseCounts.length; i++) {
|
||||
int count = denseCounts[i];
|
||||
if (count != 0) {
|
||||
childCount++;
|
||||
if (q == null) {
|
||||
// Lazy init for sparse case:
|
||||
q = new TopOrdAndIntQueue(topN);
|
||||
|
@ -197,36 +210,9 @@ public class StringValueFacetCounts extends Facets {
|
|||
if (reuse == null) {
|
||||
reuse = (TopOrdAndIntQueue.OrdAndInt) q.newOrdAndValue();
|
||||
}
|
||||
reuse.ord = ord;
|
||||
reuse.ord = i;
|
||||
reuse.value = count;
|
||||
reuse = (TopOrdAndIntQueue.OrdAndInt) q.insertWithOverflow(reuse);
|
||||
if (q.size() == topN) {
|
||||
bottomCount = ((TopOrdAndIntQueue.OrdAndInt) q.top()).value;
|
||||
bottomOrd = q.top().ord;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (int i = 0; i < denseCounts.length; i++) {
|
||||
int count = denseCounts[i];
|
||||
if (count != 0) {
|
||||
childCount++;
|
||||
if (count > bottomCount || (count == bottomCount && i < bottomOrd)) {
|
||||
if (q == null) {
|
||||
// Lazy init for sparse case:
|
||||
q = new TopOrdAndIntQueue(topN);
|
||||
}
|
||||
if (reuse == null) {
|
||||
reuse = (TopOrdAndIntQueue.OrdAndInt) q.newOrdAndValue();
|
||||
}
|
||||
reuse.ord = i;
|
||||
reuse.value = count;
|
||||
reuse = (TopOrdAndIntQueue.OrdAndInt) q.insertWithOverflow(reuse);
|
||||
if (q.size() == topN) {
|
||||
bottomCount = ((TopOrdAndIntQueue.OrdAndInt) q.top()).value;
|
||||
bottomOrd = q.top().ord;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -256,7 +242,13 @@ public class StringValueFacetCounts extends Facets {
|
|||
return -1;
|
||||
}
|
||||
|
||||
return sparseCounts != null ? sparseCounts.get(ord) : denseCounts[ord];
|
||||
if (sparseCounts != null) {
|
||||
return sparseCounts.get(ord);
|
||||
}
|
||||
if (denseCounts != null) {
|
||||
return denseCounts[ord];
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -322,8 +322,6 @@ abstract class AbstractSortedSetDocValueFacetCounts extends Facets {
|
|||
private TopChildrenForPath computeTopChildren(
|
||||
PrimitiveIterator.OfInt childOrds, int topN, DimConfig dimConfig, int pathOrd) {
|
||||
TopOrdAndIntQueue q = null;
|
||||
int bottomCount = 0;
|
||||
int bottomOrd = Integer.MAX_VALUE;
|
||||
int pathCount = 0;
|
||||
int childCount = 0;
|
||||
|
||||
|
@ -334,23 +332,17 @@ abstract class AbstractSortedSetDocValueFacetCounts extends Facets {
|
|||
if (count > 0) {
|
||||
pathCount += count;
|
||||
childCount++;
|
||||
if (count > bottomCount || (count == bottomCount && ord < bottomOrd)) {
|
||||
if (q == null) {
|
||||
// Lazy init, so we don't create this for the
|
||||
// sparse case unnecessarily
|
||||
q = new TopOrdAndIntQueue(topN);
|
||||
}
|
||||
if (reuse == null) {
|
||||
reuse = (TopOrdAndIntQueue.OrdAndInt) q.newOrdAndValue();
|
||||
}
|
||||
reuse.ord = ord;
|
||||
reuse.value = count;
|
||||
reuse = (TopOrdAndIntQueue.OrdAndInt) q.insertWithOverflow(reuse);
|
||||
if (q.size() == topN) {
|
||||
bottomCount = ((TopOrdAndIntQueue.OrdAndInt) q.top()).value;
|
||||
bottomOrd = q.top().ord;
|
||||
}
|
||||
if (q == null) {
|
||||
// Lazy init, so we don't create this for the
|
||||
// sparse case unnecessarily
|
||||
q = new TopOrdAndIntQueue(topN);
|
||||
}
|
||||
if (reuse == null) {
|
||||
reuse = (TopOrdAndIntQueue.OrdAndInt) q.newOrdAndValue();
|
||||
}
|
||||
reuse.ord = ord;
|
||||
reuse.value = count;
|
||||
reuse = (TopOrdAndIntQueue.OrdAndInt) q.insertWithOverflow(reuse);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -20,7 +20,6 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -85,7 +84,8 @@ public abstract class FacetTestCase extends LuceneTestCase {
|
|||
* @param docId docId for which facet labels are needed.
|
||||
* @param dimension Retain facet labels for supplied dimension only. A null value fetches all
|
||||
* facet labels.
|
||||
* @param facetLabelReader {@FacetLabelReader} instance use to get facet labels for input docId.
|
||||
* @param facetLabelReader {@link FacetLabelReader} instance use to get facet labels for input
|
||||
* docId.
|
||||
* @return {@code List<FacetLabel>} containing matching facet labels.
|
||||
* @throws IOException when a low-level IO issue occurs while reading facet labels.
|
||||
*/
|
||||
|
@ -178,12 +178,9 @@ public abstract class FacetTestCase extends LuceneTestCase {
|
|||
labelValues,
|
||||
i - numInRow,
|
||||
i,
|
||||
new Comparator<LabelAndValue>() {
|
||||
@Override
|
||||
public int compare(LabelAndValue a, LabelAndValue b) {
|
||||
assert a.value.doubleValue() == b.value.doubleValue();
|
||||
return new BytesRef(a.label).compareTo(new BytesRef(b.label));
|
||||
}
|
||||
(a, b) -> {
|
||||
assert a.value.doubleValue() == b.value.doubleValue();
|
||||
return new BytesRef(a.label).compareTo(new BytesRef(b.label));
|
||||
});
|
||||
}
|
||||
numInRow = 1;
|
||||
|
@ -198,16 +195,13 @@ public abstract class FacetTestCase extends LuceneTestCase {
|
|||
protected void sortLabelValues(List<LabelAndValue> labelValues) {
|
||||
Collections.sort(
|
||||
labelValues,
|
||||
new Comparator<LabelAndValue>() {
|
||||
@Override
|
||||
public int compare(LabelAndValue a, LabelAndValue b) {
|
||||
if (a.value.doubleValue() > b.value.doubleValue()) {
|
||||
return -1;
|
||||
} else if (a.value.doubleValue() < b.value.doubleValue()) {
|
||||
return 1;
|
||||
} else {
|
||||
return new BytesRef(a.label).compareTo(new BytesRef(b.label));
|
||||
}
|
||||
(a, b) -> {
|
||||
if (a.value.doubleValue() > b.value.doubleValue()) {
|
||||
return -1;
|
||||
} else if (a.value.doubleValue() < b.value.doubleValue()) {
|
||||
return 1;
|
||||
} else {
|
||||
return new BytesRef(a.label).compareTo(new BytesRef(b.label));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -215,16 +209,13 @@ public abstract class FacetTestCase extends LuceneTestCase {
|
|||
protected void sortFacetResults(List<FacetResult> results) {
|
||||
Collections.sort(
|
||||
results,
|
||||
new Comparator<FacetResult>() {
|
||||
@Override
|
||||
public int compare(FacetResult a, FacetResult b) {
|
||||
if (a.value.doubleValue() > b.value.doubleValue()) {
|
||||
return -1;
|
||||
} else if (b.value.doubleValue() > a.value.doubleValue()) {
|
||||
return 1;
|
||||
} else {
|
||||
return a.dim.compareTo(b.dim);
|
||||
}
|
||||
(a, b) -> {
|
||||
if (a.value.doubleValue() > b.value.doubleValue()) {
|
||||
return -1;
|
||||
} else if (b.value.doubleValue() > a.value.doubleValue()) {
|
||||
return 1;
|
||||
} else {
|
||||
return a.dim.compareTo(b.dim);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -80,14 +80,12 @@ public class TestLongValueFacetCounts extends FacetTestCase {
|
|||
new String[0],
|
||||
6,
|
||||
101,
|
||||
new LabelAndValue[] {
|
||||
new LabelAndValue("0", 20),
|
||||
new LabelAndValue("1", 20),
|
||||
new LabelAndValue("2", 20),
|
||||
new LabelAndValue("3", 20),
|
||||
new LabelAndValue("4", 20),
|
||||
new LabelAndValue("9223372036854775807", 1)
|
||||
});
|
||||
new LabelAndValue("0", 20),
|
||||
new LabelAndValue("1", 20),
|
||||
new LabelAndValue("2", 20),
|
||||
new LabelAndValue("3", 20),
|
||||
new LabelAndValue("4", 20),
|
||||
new LabelAndValue("9223372036854775807", 1));
|
||||
|
||||
r.close();
|
||||
d.close();
|
||||
|
@ -123,9 +121,8 @@ public class TestLongValueFacetCounts extends FacetTestCase {
|
|||
new String[0],
|
||||
2,
|
||||
9,
|
||||
new LabelAndValue[] {
|
||||
new LabelAndValue("0", 4), new LabelAndValue("1", 5),
|
||||
});
|
||||
new LabelAndValue("0", 4),
|
||||
new LabelAndValue("1", 5));
|
||||
|
||||
r.close();
|
||||
d.close();
|
||||
|
@ -156,11 +153,9 @@ public class TestLongValueFacetCounts extends FacetTestCase {
|
|||
new String[0],
|
||||
3,
|
||||
3,
|
||||
new LabelAndValue[] {
|
||||
new LabelAndValue("9223372036854775805", 1),
|
||||
new LabelAndValue("9223372036854775806", 1),
|
||||
new LabelAndValue("9223372036854775807", 1)
|
||||
});
|
||||
new LabelAndValue("9223372036854775805", 1),
|
||||
new LabelAndValue("9223372036854775806", 1),
|
||||
new LabelAndValue("9223372036854775807", 1));
|
||||
|
||||
// since we have no insight into the value order in the hashMap, we sort labels by value and
|
||||
// count in
|
||||
|
@ -221,11 +216,7 @@ public class TestLongValueFacetCounts extends FacetTestCase {
|
|||
List<FacetResult> topDimsResults2 = facets.getTopDims(0, 1);
|
||||
assertEquals(0, topDimsResults2.size());
|
||||
// test getAllDims(0)
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
facets.getAllDims(0);
|
||||
});
|
||||
expectThrows(IllegalArgumentException.class, () -> facets.getAllDims(0));
|
||||
|
||||
r.close();
|
||||
d.close();
|
||||
|
@ -364,8 +355,7 @@ public class TestLongValueFacetCounts extends FacetTestCase {
|
|||
|
||||
// test getAllChildren
|
||||
expectedCounts.sort(
|
||||
Comparator.comparing((Map.Entry<Long, Integer> a) -> a.getKey())
|
||||
.thenComparingLong(Map.Entry::getValue));
|
||||
Map.Entry.<Long, Integer>comparingByKey().thenComparingLong(Map.Entry::getValue));
|
||||
FacetResult allChildren = facetCounts.getAllChildren("field");
|
||||
// sort labels by value, count in ascending order
|
||||
Arrays.sort(
|
||||
|
@ -627,8 +617,7 @@ public class TestLongValueFacetCounts extends FacetTestCase {
|
|||
|
||||
// test getAllChildren
|
||||
expectedCounts.sort(
|
||||
Comparator.comparing((Map.Entry<Long, Integer> a) -> a.getKey())
|
||||
.thenComparingLong(Map.Entry::getValue));
|
||||
Map.Entry.<Long, Integer>comparingByKey().thenComparingLong(Map.Entry::getValue));
|
||||
FacetResult allChildren = facetCounts.getAllChildren("field");
|
||||
// sort labels by value, count in ascending order
|
||||
Arrays.sort(
|
||||
|
@ -833,9 +822,8 @@ public class TestLongValueFacetCounts extends FacetTestCase {
|
|||
new String[0],
|
||||
2,
|
||||
2,
|
||||
new LabelAndValue[] {
|
||||
new LabelAndValue("42", 1), new LabelAndValue("43", 1),
|
||||
});
|
||||
new LabelAndValue("42", 1),
|
||||
new LabelAndValue("43", 1));
|
||||
|
||||
r.close();
|
||||
dir.close();
|
||||
|
|
|
@ -86,7 +86,7 @@ public class TestMultipleIndexFields extends FacetTestCase {
|
|||
// prepare searcher to search against
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
|
||||
FacetsCollector sfc = performSearch(tr, ir, searcher);
|
||||
FacetsCollector sfc = performSearch(searcher);
|
||||
|
||||
// Obtain facets results and hand-test them
|
||||
assertCorrectResults(getTaxonomyFacetCounts(tr, config, sfc));
|
||||
|
@ -124,7 +124,7 @@ public class TestMultipleIndexFields extends FacetTestCase {
|
|||
// prepare searcher to search against
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
|
||||
FacetsCollector sfc = performSearch(tr, ir, searcher);
|
||||
FacetsCollector sfc = performSearch(searcher);
|
||||
|
||||
Map<String, Facets> facetsMap = new HashMap<>();
|
||||
facetsMap.put("Author", getTaxonomyFacetCounts(tr, config, sfc, "$author"));
|
||||
|
@ -168,7 +168,7 @@ public class TestMultipleIndexFields extends FacetTestCase {
|
|||
// prepare searcher to search against
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
|
||||
FacetsCollector sfc = performSearch(tr, ir, searcher);
|
||||
FacetsCollector sfc = performSearch(searcher);
|
||||
|
||||
Map<String, Facets> facetsMap = new HashMap<>();
|
||||
Facets facets2 = getTaxonomyFacetCounts(tr, config, sfc, "$music");
|
||||
|
@ -225,7 +225,7 @@ public class TestMultipleIndexFields extends FacetTestCase {
|
|||
// prepare searcher to search against
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
|
||||
FacetsCollector sfc = performSearch(tr, ir, searcher);
|
||||
FacetsCollector sfc = performSearch(searcher);
|
||||
|
||||
Map<String, Facets> facetsMap = new HashMap<>();
|
||||
facetsMap.put("Band", getTaxonomyFacetCounts(tr, config, sfc, "$bands"));
|
||||
|
@ -271,7 +271,7 @@ public class TestMultipleIndexFields extends FacetTestCase {
|
|||
// prepare searcher to search against
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
|
||||
FacetsCollector sfc = performSearch(tr, ir, searcher);
|
||||
FacetsCollector sfc = performSearch(searcher);
|
||||
|
||||
Map<String, Facets> facetsMap = new HashMap<>();
|
||||
Facets facets2 = getTaxonomyFacetCounts(tr, config, sfc, "$music");
|
||||
|
@ -300,9 +300,8 @@ public class TestMultipleIndexFields extends FacetTestCase {
|
|||
new String[0],
|
||||
2,
|
||||
5,
|
||||
new LabelAndValue[] {
|
||||
new LabelAndValue("Punk", 1), new LabelAndValue("Rock & Pop", 4),
|
||||
});
|
||||
new LabelAndValue("Punk", 1),
|
||||
new LabelAndValue("Rock & Pop", 4));
|
||||
assertEquals(
|
||||
"dim=Band path=[Rock & Pop] value=4 childCount=4\n The Beatles (1)\n U2 (1)\n REM (1)\n Dave Matthews Band (1)\n",
|
||||
facets.getTopChildren(10, "Band", "Rock & Pop").toString());
|
||||
|
@ -312,12 +311,10 @@ public class TestMultipleIndexFields extends FacetTestCase {
|
|||
new String[] {"Rock & Pop"},
|
||||
4,
|
||||
4,
|
||||
new LabelAndValue[] {
|
||||
new LabelAndValue("Dave Matthews Band", 1),
|
||||
new LabelAndValue("REM", 1),
|
||||
new LabelAndValue("The Beatles", 1),
|
||||
new LabelAndValue("U2", 1),
|
||||
});
|
||||
new LabelAndValue("Dave Matthews Band", 1),
|
||||
new LabelAndValue("REM", 1),
|
||||
new LabelAndValue("The Beatles", 1),
|
||||
new LabelAndValue("U2", 1));
|
||||
|
||||
assertEquals(
|
||||
"dim=Author path=[] value=3 childCount=3\n Mark Twain (1)\n Stephen King (1)\n Kurt Vonnegut (1)\n",
|
||||
|
@ -328,15 +325,12 @@ public class TestMultipleIndexFields extends FacetTestCase {
|
|||
new String[0],
|
||||
3,
|
||||
3,
|
||||
new LabelAndValue[] {
|
||||
new LabelAndValue("Kurt Vonnegut", 1),
|
||||
new LabelAndValue("Mark Twain", 1),
|
||||
new LabelAndValue("Stephen King", 1),
|
||||
});
|
||||
new LabelAndValue("Kurt Vonnegut", 1),
|
||||
new LabelAndValue("Mark Twain", 1),
|
||||
new LabelAndValue("Stephen King", 1));
|
||||
}
|
||||
|
||||
private FacetsCollector performSearch(TaxonomyReader tr, IndexReader ir, IndexSearcher searcher)
|
||||
throws IOException {
|
||||
private FacetsCollector performSearch(IndexSearcher searcher) throws IOException {
|
||||
FacetsCollector fc = new FacetsCollector();
|
||||
FacetsCollector.search(searcher, new MatchAllDocsQuery(), 10, fc);
|
||||
return fc;
|
||||
|
|
|
@ -119,7 +119,7 @@ public class TestRandomSamplingFacetsCollector extends FacetTestCase {
|
|||
float ei = (float) md.totalHits / totalHits;
|
||||
if (ei > 0.0f) {
|
||||
float oi = (float) numSampledDocs[i] / totalSampledDocs;
|
||||
chi_square += (Math.pow(ei - oi, 2) / ei);
|
||||
chi_square += (float) (Math.pow(ei - oi, 2) / ei);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.lucene.index.IndexReader;
|
|||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.MatchNoDocsQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.tests.index.RandomIndexWriter;
|
||||
import org.apache.lucene.tests.util.LuceneTestCase;
|
||||
|
@ -80,6 +81,42 @@ public class TestStringValueFacetCounts extends FacetTestCase {
|
|||
IOUtils.close(searcher.getIndexReader(), dir);
|
||||
}
|
||||
|
||||
private void assertEmptyFacetResult(FacetResult result) {
|
||||
assertEquals(0, result.path.length);
|
||||
assertEquals(0, result.value);
|
||||
assertEquals(0, result.childCount);
|
||||
assertEquals(0, result.labelValues.length);
|
||||
}
|
||||
|
||||
public void testEmptyMatchset() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
|
||||
|
||||
Document doc = new Document();
|
||||
doc.add(new SortedSetDocValuesField("field", new BytesRef("foo")));
|
||||
writer.addDocument(doc);
|
||||
|
||||
IndexSearcher searcher = newSearcher(writer.getReader());
|
||||
writer.close();
|
||||
|
||||
FacetsCollector facetsCollector =
|
||||
searcher.search(new MatchNoDocsQuery(), new FacetsCollectorManager());
|
||||
StringDocValuesReaderState state =
|
||||
new StringDocValuesReaderState(searcher.getIndexReader(), "field");
|
||||
|
||||
StringValueFacetCounts counts = new StringValueFacetCounts(state, facetsCollector);
|
||||
|
||||
FacetResult top = counts.getTopChildren(10, "field");
|
||||
assertEmptyFacetResult(top);
|
||||
|
||||
FacetResult all = counts.getAllChildren("field");
|
||||
assertEmptyFacetResult(all);
|
||||
|
||||
assertEquals(0, counts.getSpecificValue("field", "foo"));
|
||||
|
||||
IOUtils.close(searcher.getIndexReader(), dir);
|
||||
}
|
||||
|
||||
// See: LUCENE-10070
|
||||
public void testCountAll() throws Exception {
|
||||
|
||||
|
@ -431,11 +468,7 @@ public class TestStringValueFacetCounts extends FacetTestCase {
|
|||
assertEquals(facetResult, topNDimsResult.get(0));
|
||||
|
||||
// test getAllDims(0)
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
facets.getAllDims(0);
|
||||
});
|
||||
expectThrows(IllegalArgumentException.class, () -> facets.getAllDims(0));
|
||||
|
||||
// This is a little strange, but we request all labels at this point so that when we
|
||||
// secondarily sort by label value in order to compare to the expected results, we have
|
||||
|
@ -501,8 +534,7 @@ public class TestStringValueFacetCounts extends FacetTestCase {
|
|||
|
||||
// sort expected counts by value, count
|
||||
expectedCountsSortedByValue.sort(
|
||||
Comparator.comparing((Map.Entry<String, Integer> a) -> a.getKey())
|
||||
.thenComparingInt(Map.Entry::getValue));
|
||||
Map.Entry.<String, Integer>comparingByKey().thenComparingInt(Map.Entry::getValue));
|
||||
|
||||
FacetResult facetResult = facets.getAllChildren("field");
|
||||
assertEquals(expectedTotalDocsWithValue, facetResult.value);
|
||||
|
|
|
@ -112,11 +112,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
result.toString());
|
||||
|
||||
// test getTopChildren(0, dim)
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
facets.getTopChildren(0, "field");
|
||||
});
|
||||
expectThrows(IllegalArgumentException.class, () -> facets.getTopChildren(0, "field"));
|
||||
|
||||
r.close();
|
||||
d.close();
|
||||
|
@ -169,11 +165,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
result.toString());
|
||||
|
||||
// test getTopChildren(0, dim)
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
facets.getTopChildren(0, "field");
|
||||
});
|
||||
expectThrows(IllegalArgumentException.class, () -> facets.getTopChildren(0, "field"));
|
||||
|
||||
r.close();
|
||||
d.close();
|
||||
|
@ -287,37 +279,19 @@ public class TestRangeFacetCounts extends FacetTestCase {
|
|||
assertEquals(0, topNDimsResult.size());
|
||||
|
||||
// test getAllDims(0)
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
facets.getAllDims(0);
|
||||
});
|
||||
expectThrows(IllegalArgumentException.class, () -> facets.getAllDims(0));
|
||||
|
||||
r.close();
|
||||
d.close();
|
||||
}
|
||||
|
||||
public void testUselessRange() {
|
||||
expectThrows(IllegalArgumentException.class, () -> new LongRange("useless", 7, true, 6, true));
|
||||
expectThrows(IllegalArgumentException.class, () -> new LongRange("useless", 7, true, 7, false));
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
new LongRange("useless", 7, true, 6, true);
|
||||
});
|
||||
IllegalArgumentException.class, () -> new DoubleRange("useless", 7.0, true, 6.0, true));
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
new LongRange("useless", 7, true, 7, false);
|
||||
});
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
new DoubleRange("useless", 7.0, true, 6.0, true);
|
||||
});
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
new DoubleRange("useless", 7.0, true, 7.0, false);
|
||||
});
|
||||
IllegalArgumentException.class, () -> new DoubleRange("useless", 7.0, true, 7.0, false));
|
||||
}
|
||||
|
||||
public void testLongMinMax() throws Exception {
|
||||
|
|
|
@ -100,11 +100,7 @@ public class TestRangeOnRangeFacetCounts extends FacetTestCase {
|
|||
result.toString());
|
||||
|
||||
// test getTopChildren(0, dim)
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
facets.getTopChildren(0, "field");
|
||||
});
|
||||
expectThrows(IllegalArgumentException.class, () -> facets.getTopChildren(0, "field"));
|
||||
|
||||
r.close();
|
||||
d.close();
|
||||
|
@ -160,11 +156,7 @@ public class TestRangeOnRangeFacetCounts extends FacetTestCase {
|
|||
result.toString());
|
||||
|
||||
// test getTopChildren(0, dim)
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
facets.getTopChildren(0, "field");
|
||||
});
|
||||
expectThrows(IllegalArgumentException.class, () -> facets.getTopChildren(0, "field"));
|
||||
|
||||
r.close();
|
||||
d.close();
|
||||
|
@ -224,11 +216,7 @@ public class TestRangeOnRangeFacetCounts extends FacetTestCase {
|
|||
assertEquals(0, topNDimsResult.size());
|
||||
|
||||
// test getAllDims(0)
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
facets.getAllDims(0);
|
||||
});
|
||||
expectThrows(IllegalArgumentException.class, () -> facets.getAllDims(0));
|
||||
|
||||
r.close();
|
||||
d.close();
|
||||
|
@ -289,60 +277,34 @@ public class TestRangeOnRangeFacetCounts extends FacetTestCase {
|
|||
result.get(0).toString());
|
||||
|
||||
// test getTopChildren(0, dim)
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
facets.getTopChildren(0, "field");
|
||||
});
|
||||
expectThrows(IllegalArgumentException.class, () -> facets.getTopChildren(0, "field"));
|
||||
|
||||
r.close();
|
||||
d.close();
|
||||
}
|
||||
|
||||
public void testUselessRangeSingleDim() {
|
||||
expectThrows(IllegalArgumentException.class, () -> new LongRange("useless", 7, true, 6, true));
|
||||
expectThrows(IllegalArgumentException.class, () -> new LongRange("useless", 7, true, 7, false));
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
new LongRange("useless", 7, true, 6, true);
|
||||
});
|
||||
IllegalArgumentException.class, () -> new DoubleRange("useless", 7.0, true, 6.0, true));
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
new LongRange("useless", 7, true, 7, false);
|
||||
});
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
new DoubleRange("useless", 7.0, true, 6.0, true);
|
||||
});
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
new DoubleRange("useless", 7.0, true, 7.0, false);
|
||||
});
|
||||
IllegalArgumentException.class, () -> new DoubleRange("useless", 7.0, true, 7.0, false));
|
||||
}
|
||||
|
||||
public void testUselessMultiDimRange() {
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
new LongRange("useless", longArray(7L, 7L), longArray(6L, 6L));
|
||||
});
|
||||
() -> new LongRange("useless", longArray(7L, 7L), longArray(6L, 6L)));
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
new LongRange("useless", longArray(7L, 7L), longArray(7L, 6L));
|
||||
});
|
||||
() -> new LongRange("useless", longArray(7L, 7L), longArray(7L, 6L)));
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
new DoubleRange("useless", doubleArray(7.0, 7.0), doubleArray(6.0, 6.0));
|
||||
});
|
||||
() -> new DoubleRange("useless", doubleArray(7.0, 7.0), doubleArray(6.0, 6.0)));
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
new DoubleRange("useless", doubleArray(7.0, 7.0), doubleArray(7.0, 6.0));
|
||||
});
|
||||
() -> new DoubleRange("useless", doubleArray(7.0, 7.0), doubleArray(7.0, 6.0)));
|
||||
}
|
||||
|
||||
public void testSingleDimLongMinMax() throws Exception {
|
||||
|
@ -769,11 +731,7 @@ public class TestRangeOnRangeFacetCounts extends FacetTestCase {
|
|||
result.toString());
|
||||
|
||||
// test getTopChildren(0, dim)
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
facets.getTopChildren(0, "field");
|
||||
});
|
||||
expectThrows(IllegalArgumentException.class, () -> facets.getTopChildren(0, "field"));
|
||||
|
||||
IOUtils.close(r, d);
|
||||
}
|
||||
|
@ -830,11 +788,7 @@ public class TestRangeOnRangeFacetCounts extends FacetTestCase {
|
|||
result.toString());
|
||||
|
||||
// test getTopChildren(0, dim)
|
||||
expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> {
|
||||
facets.getTopChildren(0, "field");
|
||||
});
|
||||
expectThrows(IllegalArgumentException.class, () -> facets.getTopChildren(0, "field"));
|
||||
|
||||
IOUtils.close(r, d);
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue